From af4dd1f2cb50101c9df2aa004c45b22767020b89 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 13 Nov 2024 09:46:09 +0000 Subject: [PATCH 001/397] s3::client: Make "readable_file" abortable Adds optional abortable source to "readable_file" interface. Note: the abortable aspect is not preserved across a "dup()" call however, since these objects are generally not used in a cross-shard fashion, it should be ok. --- utils/s3/client.cc | 17 ++++++++++------- utils/s3/client.hh | 2 +- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/utils/s3/client.cc b/utils/s3/client.cc index deb887aa53b4..ef070f793342 100644 --- a/utils/s3/client.cc +++ b/utils/s3/client.cc @@ -1198,6 +1198,7 @@ class client::readable_file : public file_impl { shared_ptr _client; sstring _object_name; std::optional _stats; + seastar::abort_source* _as; [[noreturn]] void unsupported() { throw_with_backtrace("unsupported operation on s3 readable file"); @@ -1215,9 +1216,10 @@ class client::readable_file : public file_impl { } public: - readable_file(shared_ptr cln, sstring object_name) + readable_file(shared_ptr cln, sstring object_name, seastar::abort_source* as = nullptr) : _client(std::move(cln)) , _object_name(std::move(object_name)) + , _as(as) { } @@ -1245,7 +1247,8 @@ class client::readable_file : public file_impl { } virtual shared_ptr to_file() && override { - return make_shared(std::move(_h).to_client(), std::move(_object_name)); + // TODO: cannot traverse abort source across shards. + return make_shared(std::move(_h).to_client(), std::move(_object_name), nullptr); } }; @@ -1277,7 +1280,7 @@ class client::readable_file : public file_impl { co_return 0; } - auto buf = co_await _client->get_object_contiguous(_object_name, range{ pos, len }); + auto buf = co_await _client->get_object_contiguous(_object_name, range{ pos, len }, _as); std::copy_n(buf.get(), buf.size(), reinterpret_cast(buffer)); co_return buf.size(); } @@ -1288,7 +1291,7 @@ class client::readable_file : public file_impl { co_return 0; } - auto buf = co_await _client->get_object_contiguous(_object_name, range{ pos, utils::iovec_len(iov) }); + auto buf = co_await _client->get_object_contiguous(_object_name, range{ pos, utils::iovec_len(iov) }, _as); uint64_t off = 0; for (auto& v : iov) { auto sz = std::min(v.iov_len, buf.size() - off); @@ -1307,7 +1310,7 @@ class client::readable_file : public file_impl { co_return temporary_buffer(); } - auto buf = co_await _client->get_object_contiguous(_object_name, range{ offset, range_size }); + auto buf = co_await _client->get_object_contiguous(_object_name, range{ offset, range_size }, _as); co_return temporary_buffer(reinterpret_cast(buf.get_write()), buf.size(), buf.release()); } @@ -1316,8 +1319,8 @@ class client::readable_file : public file_impl { } }; -file client::make_readable_file(sstring object_name) { - return file(make_shared(shared_from_this(), std::move(object_name))); +file client::make_readable_file(sstring object_name, seastar::abort_source* as) { + return file(make_shared(shared_from_this(), std::move(object_name), as)); } future<> client::close() { diff --git a/utils/s3/client.hh b/utils/s3/client.hh index 329aff9fe2ef..8f3c6d829cfc 100644 --- a/utils/s3/client.hh +++ b/utils/s3/client.hh @@ -106,7 +106,7 @@ public: future<> put_object(sstring object_name, ::memory_data_sink_buffers bufs, seastar::abort_source* = nullptr); future<> delete_object(sstring object_name, seastar::abort_source* = nullptr); - file make_readable_file(sstring object_name); + file make_readable_file(sstring object_name, seastar::abort_source* = nullptr); data_sink make_upload_sink(sstring object_name, seastar::abort_source* = nullptr); data_sink make_upload_jumbo_sink(sstring object_name, std::optional max_parts_per_piece = {}, seastar::abort_source* = nullptr); /// upload a file with specified path to s3 From f30864b571c76307f6ac519df48fc81ab3e2766a Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 13 Nov 2024 09:48:24 +0000 Subject: [PATCH 002/397] s3_storage: Add optional abort_source to params/object Adds an abort_source to s3 storage params and resulting storage interface. Propagates said source to s3 objects created. --- data_dictionary/storage_options.hh | 5 +++++ sstables/storage.cc | 12 +++++++----- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/data_dictionary/storage_options.hh b/data_dictionary/storage_options.hh index a8b9c5cb1b4e..794095e846f2 100644 --- a/data_dictionary/storage_options.hh +++ b/data_dictionary/storage_options.hh @@ -15,6 +15,10 @@ #include "schema/schema_fwd.hh" #include "seastarx.hh" +namespace seastar { + class abort_source; +} + namespace data_dictionary { struct storage_options { @@ -30,6 +34,7 @@ struct storage_options { sstring bucket; sstring endpoint; std::variant location; + seastar::abort_source* abort_source = nullptr; static constexpr std::string_view name = "S3"; static s3 from_map(const std::map&); diff --git a/sstables/storage.cc b/sstables/storage.cc index 45f016affd0d..dc719b9f9657 100644 --- a/sstables/storage.cc +++ b/sstables/storage.cc @@ -507,6 +507,7 @@ class s3_storage : public sstables::storage { shared_ptr _client; sstring _bucket; std::variant _location; + seastar::abort_source* _as; static constexpr auto status_creating = "creating"; static constexpr auto status_sealed = "sealed"; @@ -522,10 +523,11 @@ class s3_storage : public sstables::storage { } public: - s3_storage(shared_ptr client, sstring bucket, std::variant loc) + s3_storage(shared_ptr client, sstring bucket, std::variant loc, seastar::abort_source* as) : _client(std::move(client)) , _bucket(std::move(bucket)) , _location(std::move(loc)) + , _as(as) { } @@ -585,17 +587,17 @@ void s3_storage::open(sstable& sst) { } future s3_storage::open_component(const sstable& sst, component_type type, open_flags flags, file_open_options options, bool check_integrity) { - co_return _client->make_readable_file(make_s3_object_name(sst, type)); + co_return _client->make_readable_file(make_s3_object_name(sst, type), _as); } future s3_storage::make_data_or_index_sink(sstable& sst, component_type type) { SCYLLA_ASSERT(type == component_type::Data || type == component_type::Index); // FIXME: if we have file size upper bound upfront, it's better to use make_upload_sink() instead - co_return _client->make_upload_jumbo_sink(make_s3_object_name(sst, type)); + co_return _client->make_upload_jumbo_sink(make_s3_object_name(sst, type), std::nullopt, _as); } future s3_storage::make_component_sink(sstable& sst, component_type type, open_flags oflags, file_output_stream_options options) { - co_return _client->make_upload_sink(make_s3_object_name(sst, type)); + co_return _client->make_upload_sink(make_s3_object_name(sst, type), _as); } future<> s3_storage::seal(const sstable& sst) { @@ -676,7 +678,7 @@ std::unique_ptr make_storage(sstables_manager& manager, const }, os.location)) { on_internal_error(sstlog, "S3 storage options is missing 'location'"); } - return std::make_unique(manager.get_endpoint_client(os.endpoint), os.bucket, os.location); + return std::make_unique(manager.get_endpoint_client(os.endpoint), os.bucket, os.location, os.abort_source); } }, s_opts.value); } From 6a2a18a2fc934deaf543fcf89e5513fb990422b6 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 13 Nov 2024 09:50:05 +0000 Subject: [PATCH 003/397] distributed_loader: Add optional abort_source to get_sstables_from_object_store --- replica/distributed_loader.cc | 9 +++++---- replica/distributed_loader.hh | 6 +++++- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/replica/distributed_loader.cc b/replica/distributed_loader.cc index ad61093e9f30..a09e5faf912b 100644 --- a/replica/distributed_loader.cc +++ b/replica/distributed_loader.cc @@ -234,12 +234,13 @@ distributed_loader::get_sstables_from_upload_dir(distributed& } future>>> -distributed_loader::get_sstables_from_object_store(distributed& db, sstring ks, sstring cf, std::vector sstables, sstring endpoint, sstring bucket, sstring prefix, sstables::sstable_open_config cfg) { - return get_sstables_from(db, ks, cf, cfg, [bucket, endpoint, prefix, sstables=std::move(sstables)] (auto& global_table, auto& directory) { +distributed_loader::get_sstables_from_object_store(distributed& db, sstring ks, sstring cf, std::vector sstables, sstring endpoint, sstring bucket, sstring prefix, sstables::sstable_open_config cfg, std::function get_abort_src) { + return get_sstables_from(db, ks, cf, cfg, [bucket, endpoint, prefix, sstables=std::move(sstables), &get_abort_src] (auto& global_table, auto& directory) { return directory.start(global_table.as_sharded_parameter(), - sharded_parameter([bucket, endpoint, prefix] { + sharded_parameter([bucket, endpoint, prefix, &get_abort_src] { data_dictionary::storage_options opts; - opts.value = data_dictionary::storage_options::s3{bucket, endpoint, prefix}; + seastar::abort_source* as = get_abort_src ? get_abort_src() : nullptr; + opts.value = data_dictionary::storage_options::s3{bucket, endpoint, prefix, as}; return make_lw_shared(std::move(opts)); }), sstables, diff --git a/replica/distributed_loader.hh b/replica/distributed_loader.hh index 3a4559ac4df2..b0d10d32a1a7 100644 --- a/replica/distributed_loader.hh +++ b/replica/distributed_loader.hh @@ -22,6 +22,10 @@ #include "db/system_keyspace.hh" #include "sstables/sstable_directory.hh" +namespace seastar { + class abort_source; +} + namespace replica { class database; class table; @@ -89,7 +93,7 @@ public: static future>>> get_sstables_from_upload_dir(distributed& db, sstring ks, sstring cf, sstables::sstable_open_config cfg); static future>>> - get_sstables_from_object_store(distributed& db, sstring ks, sstring cf, std::vector sstables, sstring endpoint, sstring bucket, sstring prefix, sstables::sstable_open_config cfg); + get_sstables_from_object_store(distributed& db, sstring ks, sstring cf, std::vector sstables, sstring endpoint, sstring bucket, sstring prefix, sstables::sstable_open_config cfg, std::function = {}); static future<> process_upload_dir(distributed& db, sharded& vb, sstring ks_name, sstring cf_name); }; From cbe255e736718949df4346c32d381d167e629cbe Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 13 Nov 2024 09:56:38 +0000 Subject: [PATCH 004/397] sstables_loader: Make restore task abortable Fixes #20717 Enables abortable interface and propagates abort_source to all s3 objects used for reading the restore data. Note: because restore is done on each shard, we have to maintain a per-shard abort source proxy for each, and do a background per-shard abort on abort call. This is synced at the end of "run()" v2: * Simplify abortability by using a function-local gate instead. --- sstables_loader.cc | 54 ++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 48 insertions(+), 6 deletions(-) diff --git a/sstables_loader.cc b/sstables_loader.cc index 382ef1f270a1..53d86426061c 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -495,6 +495,11 @@ class sstables_loader::download_task_impl : public tasks::task_manager::task::im virtual tasks::is_user_task is_user_task() const noexcept override { return tasks::is_user_task::yes; } + + tasks::is_abortable is_abortable() const noexcept override { + return tasks::is_abortable::yes; + } + virtual future get_progress() const override { llog.debug("get_progress: {}", _num_sstables_processed); unsigned processed = co_await _loader.map_reduce(adder(), [this] (auto&) { @@ -514,14 +519,50 @@ future<> sstables_loader::download_task_impl::run() { .load_bloom_filter = false, }; llog.debug("Loading sstables from {}({}/{})", _endpoint, _bucket, _prefix); - auto [ table_id, sstables_on_shards ] = co_await replica::distributed_loader::get_sstables_from_object_store(_loader.local()._db, _ks, _cf, _sstables, _endpoint, _bucket, _prefix, cfg); + + std::vector shard_aborts(smp::count); + auto [ table_id, sstables_on_shards ] = co_await replica::distributed_loader::get_sstables_from_object_store(_loader.local()._db, _ks, _cf, _sstables, _endpoint, _bucket, _prefix, cfg, [&] { + return &shard_aborts[this_shard_id()]; + }); llog.debug("Streaming sstables from {}({}/{})", _endpoint, _bucket, _prefix); - co_await _loader.invoke_on_all([this, &sstables_on_shards, table_id] (sstables_loader& loader) mutable -> future<> { - co_await loader.load_and_stream(_ks, _cf, table_id, std::move(sstables_on_shards[this_shard_id()]), false, false, - [this] (unsigned num_streamed) { - _num_sstables_processed[this_shard_id()] += num_streamed; + std::exception_ptr ex; + gate g; + try { + _as.check(); + + auto s = _as.subscribe([&]() noexcept { + try { + auto h = g.hold(); + (void)smp::invoke_on_all([&shard_aborts, ex = _as.abort_requested_exception_ptr()] { + shard_aborts[this_shard_id()].request_abort_ex(ex); + }).finally([h = std::move(h)] {}); + } catch (...) { + } }); - }); + + co_await _loader.invoke_on_all([this, &sstables_on_shards, table_id] (sstables_loader& loader) mutable -> future<> { + co_await loader.load_and_stream(_ks, _cf, table_id, std::move(sstables_on_shards[this_shard_id()]), false, false, [this] (unsigned num_streamed) { + _num_sstables_processed[this_shard_id()] += num_streamed; + }); + }); + } catch (...) { + ex = std::current_exception(); + } + + co_await g.close(); + + if (_as.abort_requested()) { + if (!ex) { + ex = _as.abort_requested_exception_ptr(); + } + } + + if (ex) { + co_await _loader.invoke_on_all([&sstables_on_shards] (sstables_loader&) { + sstables_on_shards[this_shard_id()] = {}; // clear on correct shard + }); + co_await coroutine::return_exception_ptr(std::move(ex)); + } } sstables_loader::sstables_loader(sharded& db, @@ -551,6 +592,7 @@ future sstables_loader::download_new_sstables(sstring ks_name, s throw std::invalid_argument(format("endpoint {} not found", endpoint)); } llog.info("Restore sstables from {}({}) to {}", endpoint, prefix, ks_name); + auto task = co_await _task_manager_module->make_and_start_task({}, container(), std::move(endpoint), std::move(bucket), std::move(ks_name), std::move(cf_name), std::move(prefix), std::move(sstables)); co_return task->id(); } From 91d77987bee0f1cd1817852c4f114b9c7ce07fc3 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 13 Nov 2024 10:05:52 +0000 Subject: [PATCH 005/397] test_backup: Add restore abort test case Not a very good test, since the end result cannot be very well verified, but at least does some checking. --- test/object_store/test_backup.py | 55 ++++++++++++++++++++++++-------- 1 file changed, 42 insertions(+), 13 deletions(-) diff --git a/test/object_store/test_backup.py b/test/object_store/test_backup.py index 685ab86c6455..5e222a3cb929 100644 --- a/test/object_store/test_backup.py +++ b/test/object_store/test_backup.py @@ -203,8 +203,7 @@ async def test_backup_is_abortable_in_s3_client(manager: ManagerClient, s3_serve await do_test_backup_abort(manager, s3_server, breakpoint_name="backup_task_pre_upload", min_files=0, max_files=1) -@pytest.mark.asyncio -async def test_simple_backup_and_restore(manager: ManagerClient, s3_server): +async def do_test_simple_backup_and_restore(manager: ManagerClient, s3_server, do_abort = False): '''check that restoring from backed up snapshot for a keyspace:table works''' cfg = {'enable_user_defined_functions': False, @@ -252,7 +251,8 @@ def list_sstables(): # - {suffix}/2-TOC.txt # - ... suffix = 'suffix' - toc_names = [f'{suffix}/{entry.name}' for entry in list_sstables() if entry.name.endswith('TOC.txt')] + old_files = list_sstables(); + toc_names = [f'{suffix}/{entry.name}' for entry in old_files if entry.name.endswith('TOC.txt')] prefix = f'{cf}/{snap_name}' tid = await manager.api.backup(server.ip_addr, ks, cf, snap_name, s3_server.address, s3_server.bucket_name, f'{prefix}/{suffix}') @@ -270,21 +270,50 @@ def list_sstables(): print('Try to restore') tid = await manager.api.restore(server.ip_addr, ks, cf, s3_server.address, s3_server.bucket_name, prefix, toc_names) + + if do_abort: + await manager.api.abort_task(server.ip_addr, tid) + status = await manager.api.wait_task(server.ip_addr, tid) - assert status is not None - assert status['state'] == 'done' - assert status['progress_units'] == "sstables" - assert status['progress_completed'] == len(toc_names) - assert status['progress_total'] == len(toc_names) + if not do_abort: + assert status is not None + assert status['state'] == 'done' + assert status['progress_units'] == "sstables" + assert status['progress_completed'] == len(toc_names) + assert status['progress_total'] == len(toc_names) print('Check that sstables came back') files = list_sstables() - assert len(files) > 0 - print('Check that data came back too') - res = cql.execute(f"SELECT * FROM {ks}.{cf};") - rows = {x.name: x.value for x in res} - assert rows == orig_rows, "Unexpected table contents after restore" + + sstable_names = [f'{entry.name}' for entry in files if entry.name.endswith('.db')] + db_objects = [object for object in objects if object.endswith('.db')] + + if do_abort: + assert len(files) >= 0 + # These checks can be viewed as dubious. We restore (atm) on a mutation basis mostly. + # There is no guarantee we'll generate the same amount of sstables as was in the original + # backup (?). But, since we are not stressing the server here (not provoking memtable flushes), + # we should in principle never generate _more_ sstables than originated the backup. + assert len(old_files) >= len(files) + assert len(sstable_names) <= len(db_objects) + else: + assert len(files) > 0 + assert (status is not None) and (status['state'] == 'done') + print(f'Check that data came back too') + res = cql.execute(f"SELECT * FROM {ks}.{cf};") + rows = { x.name: x.value for x in res } + assert rows == orig_rows, "Unexpected table contents after restore" print('Check that backup files are still there') # regression test for #20938 post_objects = set(o.key for o in get_s3_resource(s3_server).Bucket(s3_server.bucket_name).objects.filter(Prefix=prefix)) assert objects == post_objects + +@pytest.mark.asyncio +async def test_simple_backup_and_restore(manager: ManagerClient, s3_server): + '''check that restoring from backed up snapshot for a keyspace:table works''' + await do_test_simple_backup_and_restore(manager, s3_server, False) + +@pytest.mark.asyncio +async def test_abort_simple_backup_and_restore(manager: ManagerClient, s3_server): + '''check that restoring from backed up snapshot for a keyspace:table works''' + await do_test_simple_backup_and_restore(manager, s3_server, True) From 7d6f8d728b80707a28186b7dbb87ae5962d0dad3 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 12:56:02 +0300 Subject: [PATCH 006/397] util: Include fmt/ranges.h in config_file.hh The operator() of named_value() prints the allowed values on error which can be a vector, so the ranges formatting should be there. Signed-off-by: Pavel Emelyanov --- utils/config_file.hh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/config_file.hh b/utils/config_file.hh index 7216a380a2bf..713368e46c81 100644 --- a/utils/config_file.hh +++ b/utils/config_file.hh @@ -11,6 +11,8 @@ #include #include +#include + #include #include #include From d2c9c2abe8398eb454bc014a268751fc4c1f3479 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 12:22:22 +0300 Subject: [PATCH 007/397] api: Move set_compaction_throughput_mb_per_sec to config block In order to update compaction throughput API would need to update the db::config value, so the endpoint in question should sit in the block that has db::config at hand. Signed-off-by: Pavel Emelyanov --- api/compaction_manager.cc | 9 --------- api/config.cc | 7 +++++++ 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/api/compaction_manager.cc b/api/compaction_manager.cc index c36cb3bf6b5e..2b2567284199 100644 --- a/api/compaction_manager.cc +++ b/api/compaction_manager.cc @@ -204,14 +204,6 @@ void set_compaction_manager(http_context& ctx, routes& r, sharded(value); }); - - ss::set_compaction_throughput_mb_per_sec.set(r, [](std::unique_ptr req) { - //TBD - unimplemented(); - auto value = req->get_query_param("value"); - return make_ready_future(json_void()); - }); - } void unset_compaction_manager(http_context& ctx, routes& r) { @@ -227,7 +219,6 @@ void unset_compaction_manager(http_context& ctx, routes& r) { cm::get_compaction_history.unset(r); cm::get_compaction_info.unset(r); ss::get_compaction_throughput_mb_per_sec.unset(r); - ss::set_compaction_throughput_mb_per_sec.unset(r); } } diff --git a/api/config.cc b/api/config.cc index c4e5cc25c3d6..0c1867a95a8d 100644 --- a/api/config.cc +++ b/api/config.cc @@ -193,6 +193,12 @@ void set_config(std::shared_ptr < api_registry_builder20 > rb, http_context& ctx return cfg.saved_caches_directory(); }); + ss::set_compaction_throughput_mb_per_sec.set(r, [](std::unique_ptr req) { + //TBD + unimplemented(); + auto value = req->get_query_param("value"); + return make_ready_future(json::json_void()); + }); } void unset_config(http_context& ctx, routes& r) { @@ -213,6 +219,7 @@ void unset_config(http_context& ctx, routes& r) { sp::set_truncate_rpc_timeout.unset(r); ss::get_all_data_file_locations.unset(r); ss::get_saved_caches_location.unset(r); + ss::set_compaction_throughput_mb_per_sec.unset(r); } } From b8bd17021206ef829c7a640e9bfb6095812551f9 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 11:59:27 +0300 Subject: [PATCH 008/397] api: Move stream throughput set/get endpoints from storage service block In order to get stream throughput, the API will need stream_manager. In order to set stream throughput, the API will need db::config to update the corresponding named value on it. Said that, move the endpoints to relevant blocks. Signed-off-by: Pavel Emelyanov --- api/config.cc | 8 ++++++++ api/storage_service.cc | 15 --------------- api/stream_manager.cc | 9 +++++++++ 3 files changed, 17 insertions(+), 15 deletions(-) diff --git a/api/config.cc b/api/config.cc index 0c1867a95a8d..f4260a3d6497 100644 --- a/api/config.cc +++ b/api/config.cc @@ -199,6 +199,13 @@ void set_config(std::shared_ptr < api_registry_builder20 > rb, http_context& ctx auto value = req->get_query_param("value"); return make_ready_future(json::json_void()); }); + + ss::set_stream_throughput_mb_per_sec.set(r, [](std::unique_ptr req) { + //TBD + unimplemented(); + auto value = req->get_query_param("value"); + return make_ready_future(json::json_void()); + }); } void unset_config(http_context& ctx, routes& r) { @@ -220,6 +227,7 @@ void unset_config(http_context& ctx, routes& r) { ss::get_all_data_file_locations.unset(r); ss::get_saved_caches_location.unset(r); ss::set_compaction_throughput_mb_per_sec.unset(r); + ss::set_stream_throughput_mb_per_sec.unset(r); } } diff --git a/api/storage_service.cc b/api/storage_service.cc index 8f4a055893ab..ad9c47a12cbb 100644 --- a/api/storage_service.cc +++ b/api/storage_service.cc @@ -1048,19 +1048,6 @@ void set_storage_service(http_context& ctx, routes& r, sharded req) { - //TBD - unimplemented(); - auto value = req->get_query_param("value"); - return make_ready_future(json_void()); - }); - - ss::get_stream_throughput_mb_per_sec.set(r, [](std::unique_ptr req) { - //TBD - unimplemented(); - return make_ready_future(0); - }); - ss::is_incremental_backups_enabled.set(r, [&ctx](std::unique_ptr req) { // If this is issued in parallel with an ongoing change, we may see values not agreeing. // Reissuing is asking for trouble, so we will just return true upon seeing any true value. @@ -1625,8 +1612,6 @@ void unset_storage_service(http_context& ctx, routes& r) { ss::is_initialized.unset(r); ss::join_ring.unset(r); ss::is_joined.unset(r); - ss::set_stream_throughput_mb_per_sec.unset(r); - ss::get_stream_throughput_mb_per_sec.unset(r); ss::is_incremental_backups_enabled.unset(r); ss::set_incremental_backups_enabled.unset(r); ss::rebuild.unset(r); diff --git a/api/stream_manager.cc b/api/stream_manager.cc index 027a01ad6036..ad884962ec13 100644 --- a/api/stream_manager.cc +++ b/api/stream_manager.cc @@ -11,6 +11,7 @@ #include "streaming/stream_result_future.hh" #include "api/api.hh" #include "api/api-doc/stream_manager.json.hh" +#include "api/api-doc/storage_service.json.hh" #include #include #include "gms/gossiper.hh" @@ -18,6 +19,7 @@ namespace api { using namespace seastar::httpd; +namespace ss = httpd::storage_service_json; namespace hs = httpd::stream_manager_json; static void set_summaries(const std::vector& from, @@ -148,6 +150,12 @@ void set_stream_manager(http_context& ctx, routes& r, sharded(res); }); }); + + ss::get_stream_throughput_mb_per_sec.set(r, [](std::unique_ptr req) { + //TBD + unimplemented(); + return make_ready_future(0); + }); } void unset_stream_manager(http_context& ctx, routes& r) { @@ -157,6 +165,7 @@ void unset_stream_manager(http_context& ctx, routes& r) { hs::get_all_total_incoming_bytes.unset(r); hs::get_total_outgoing_bytes.unset(r); hs::get_all_total_outgoing_bytes.unset(r); + ss::get_stream_throughput_mb_per_sec.unset(r); } } From f3775ba9579bc3646c1ebac0c4c318816258636d Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 12:03:34 +0300 Subject: [PATCH 009/397] api: Implement /storage_service/stream_throughput endpoint The value can be obtained from the stream_manager Signed-off-by: Pavel Emelyanov --- api/stream_manager.cc | 7 +++---- streaming/stream_manager.hh | 4 ++++ 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/api/stream_manager.cc b/api/stream_manager.cc index ad884962ec13..44bdfff27453 100644 --- a/api/stream_manager.cc +++ b/api/stream_manager.cc @@ -151,10 +151,9 @@ void set_stream_manager(http_context& ctx, routes& r, sharded req) { - //TBD - unimplemented(); - return make_ready_future(0); + ss::get_stream_throughput_mb_per_sec.set(r, [&sm](std::unique_ptr req) { + auto value = sm.local().throughput_mbs(); + return make_ready_future(value); }); } diff --git a/streaming/stream_manager.hh b/streaming/stream_manager.hh index bdaa24c9f701..97bf6ce35bfd 100644 --- a/streaming/stream_manager.hh +++ b/streaming/stream_manager.hh @@ -189,6 +189,10 @@ private: public: void update_finished_percentage(streaming::stream_reason reason, float percentage); + + uint32_t throughput_mbs() const noexcept { + return _io_throughput_mbs.get(); + } }; } // namespace streaming From 6659ceca4f0a2132cdc14ba39ba704b4eff81ff6 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 12:43:49 +0300 Subject: [PATCH 010/397] api: Disqualify const config reference Some endpoints in config block will need to actually _update_ values on config (see next patches why), and const reference stands on the way. Signed-off-by: Pavel Emelyanov --- api/api.cc | 2 +- api/api_init.hh | 2 +- api/config.cc | 2 +- api/config.hh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/api/api.cc b/api/api.cc index 8069acc4aa9a..fdf3c1ea1f18 100644 --- a/api/api.cc +++ b/api/api.cc @@ -80,7 +80,7 @@ future<> set_server_init(http_context& ctx) { }); } -future<> set_server_config(http_context& ctx, const db::config& cfg) { +future<> set_server_config(http_context& ctx, db::config& cfg) { auto rb02 = std::make_shared < api_registry_builder20 > (ctx.api_doc, "/v2"); return ctx.http_server.set_routes([&ctx, &cfg, rb02](routes& r) { set_config(rb02, ctx, r, cfg, false); diff --git a/api/api_init.hh b/api/api_init.hh index dee938e388e6..140d171055ac 100644 --- a/api/api_init.hh +++ b/api/api_init.hh @@ -88,7 +88,7 @@ struct http_context { }; future<> set_server_init(http_context& ctx); -future<> set_server_config(http_context& ctx, const db::config& cfg); +future<> set_server_config(http_context& ctx, db::config& cfg); future<> unset_server_config(http_context& ctx); future<> set_server_snitch(http_context& ctx, sharded& snitch); future<> unset_server_snitch(http_context& ctx); diff --git a/api/config.cc b/api/config.cc index f4260a3d6497..2b6b3d3a27ca 100644 --- a/api/config.cc +++ b/api/config.cc @@ -83,7 +83,7 @@ future<> get_config_swagger_entry(std::string_view name, const std::string& desc namespace cs = httpd::config_json; -void set_config(std::shared_ptr < api_registry_builder20 > rb, http_context& ctx, routes& r, const db::config& cfg, bool first) { +void set_config(std::shared_ptr < api_registry_builder20 > rb, http_context& ctx, routes& r, db::config& cfg, bool first) { rb->register_function(r, [&cfg, first] (output_stream& os) { return do_with(first, [&os, &cfg] (bool& first) { auto f = make_ready_future(); diff --git a/api/config.hh b/api/config.hh index 06bd1e92d04c..5bf42b368b1e 100644 --- a/api/config.hh +++ b/api/config.hh @@ -13,6 +13,6 @@ namespace api { -void set_config(std::shared_ptr rb, http_context& ctx, httpd::routes& r, const db::config& cfg, bool first = false); +void set_config(std::shared_ptr rb, http_context& ctx, httpd::routes& r, db::config& cfg, bool first = false); void unset_config(http_context& ctx, httpd::routes& r); } From fa1ad5ecfdbcf57ff4c5f94eb513ee5c94465a2e Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 12:56:22 +0300 Subject: [PATCH 011/397] api: Implement /storage_service/(stream|compaction)_throughput endpoints Both values are in fact db::config named values. They are observed by, respectively, compaction manager and stream manager: when changed, the observer kicks corresponding sched group's update_io_bandwidth() method. Despite being referenced by managers, there's no way to update those values anyhow other than updating config's named values themselves. Signed-off-by: Pavel Emelyanov --- api/config.cc | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/api/config.cc b/api/config.cc index 2b6b3d3a27ca..ede33c560afc 100644 --- a/api/config.cc +++ b/api/config.cc @@ -193,17 +193,15 @@ void set_config(std::shared_ptr < api_registry_builder20 > rb, http_context& ctx return cfg.saved_caches_directory(); }); - ss::set_compaction_throughput_mb_per_sec.set(r, [](std::unique_ptr req) { - //TBD - unimplemented(); - auto value = req->get_query_param("value"); + ss::set_compaction_throughput_mb_per_sec.set(r, [&cfg](std::unique_ptr req) mutable { + api::req_param value(*req, "value", 0); + cfg.compaction_throughput_mb_per_sec(value.value, utils::config_file::config_source::API); return make_ready_future(json::json_void()); }); - ss::set_stream_throughput_mb_per_sec.set(r, [](std::unique_ptr req) { - //TBD - unimplemented(); - auto value = req->get_query_param("value"); + ss::set_stream_throughput_mb_per_sec.set(r, [&cfg](std::unique_ptr req) mutable { + api::req_param value(*req, "value", 0); + cfg.stream_io_throughput_mb_per_sec(value.value, utils::config_file::config_source::API); return make_ready_future(json::json_void()); }); } From eb29d6f4b0d96ad6035ec259517f231e5c1dc34e Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 18:12:43 +0300 Subject: [PATCH 012/397] test: Add validation of how IO-updating endpoints work There are now four of those and these are all the same in the way they interpret the value parameter (though it's named differently) Signed-off-by: Pavel Emelyanov --- test/rest_api/test_system.py | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/test/rest_api/test_system.py b/test/rest_api/test_system.py index 322dca8c3db9..74ef64687787 100644 --- a/test/rest_api/test_system.py +++ b/test/rest_api/test_system.py @@ -2,6 +2,9 @@ # # SPDX-License-Identifier: AGPL-3.0-or-later +import pytest +import requests + def test_system_uptime_ms(rest_api): resp = rest_api.send('GET', "system/uptime_ms") resp.raise_for_status() @@ -11,3 +14,21 @@ def test_system_highest_sstable_format(rest_api): resp = rest_api.send('GET', "system/highest_supported_sstable_version") resp.raise_for_status() assert resp.json() == "me" + +@pytest.mark.parametrize("params", [ + ("storage_service/compaction_throughput", "value"), + ("storage_service/stream_throughput", "value") +]) +def test_io_throughput(rest_api, params): + resp = rest_api.send("POST", params[0], params={ params[1]: 100 }) + resp.raise_for_status() + resp = rest_api.send("GET", params[0]) + resp.raise_for_status() + assert resp.json() == 100 + resp = rest_api.send("POST", params[0], params={ params[1]: 0 }) + resp.raise_for_status() + resp = rest_api.send("GET", params[0]) + resp.raise_for_status() + assert resp.json() == 0 + resp = rest_api.send("POST", params[0]) + assert resp.status_code == requests.codes.bad_request From 67089fd5a184e746d68fba935f4c19b19fea2bb5 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 13:49:50 +0300 Subject: [PATCH 013/397] nodetool: Implement [gs]etcompationthroughput commands They exist in the original documentation, but are not yet implemented. Now it's possible to do it. Signed-off-by: Pavel Emelyanov --- .../getcompactionthroughput.rst | 19 ++++++++ .../setcompactionthroughput.rst | 19 ++++++++ docs/operating-scylla/nodetool.rst | 4 ++ test/nodetool/test_compaction_throughput.py | 19 ++++++++ tools/scylla-nodetool.cc | 44 +++++++++++++++++++ 5 files changed, 105 insertions(+) create mode 100644 docs/operating-scylla/nodetool-commands/getcompactionthroughput.rst create mode 100644 docs/operating-scylla/nodetool-commands/setcompactionthroughput.rst create mode 100644 test/nodetool/test_compaction_throughput.py diff --git a/docs/operating-scylla/nodetool-commands/getcompactionthroughput.rst b/docs/operating-scylla/nodetool-commands/getcompactionthroughput.rst new file mode 100644 index 000000000000..c885c203cdc8 --- /dev/null +++ b/docs/operating-scylla/nodetool-commands/getcompactionthroughput.rst @@ -0,0 +1,19 @@ +================================ +Nodetool getcompactionthroughput +================================ +**getcompactionthroughput** - Print the throughput cap for compaction in the system + +If zero is printed, it means throughput is uncapped + +Syntax +------- +.. code-block:: console + + nodetool [options] getcompactionthroughput + +See also + +* :doc:`setcompactionthroughput ` + +.. include:: nodetool-index.rst + diff --git a/docs/operating-scylla/nodetool-commands/setcompactionthroughput.rst b/docs/operating-scylla/nodetool-commands/setcompactionthroughput.rst new file mode 100644 index 000000000000..887bda94083d --- /dev/null +++ b/docs/operating-scylla/nodetool-commands/setcompactionthroughput.rst @@ -0,0 +1,19 @@ +================================ +Nodetool setcompactionthroughput +================================ +**setcompactionthroughput** - Print the throughput cap for compaction in the system + +Setting zero throughput disables capping + +Syntax +------- +.. code-block:: console + + nodetool [options] setcompactionthroughput + +See also + +* :doc:`getcompactionthroughput ` + +.. include:: nodetool-index.rst + diff --git a/docs/operating-scylla/nodetool.rst b/docs/operating-scylla/nodetool.rst index c0cc73ea782f..675bfb4a5695 100644 --- a/docs/operating-scylla/nodetool.rst +++ b/docs/operating-scylla/nodetool.rst @@ -60,6 +60,8 @@ Nodetool nodetool-commands/upgradesstables nodetool-commands/viewbuildstatus nodetool-commands/version + nodetool-commands/getcompactionthroughput + nodetool-commands/setcompactionthroughput The ``nodetool`` utility provides a simple command-line interface to the following exposed operations and attributes. @@ -132,5 +134,7 @@ Operations that are not listed below are currently not available. * :doc:`upgradesstables ` - Upgrades each table that is not running the latest ScyllaDB version, by rewriting SSTables. * :doc:`viewbuildstatus ` - Shows the progress of a materialized view build. * :doc:`version ` - Print the DB version. +* :doc:`getcompactionthroughput ` - Print the throughput cap for compaction in the system +* :doc:`setcompactionthroughput ` - Set the throughput cap for compaction in the system diff --git a/test/nodetool/test_compaction_throughput.py b/test/nodetool/test_compaction_throughput.py new file mode 100644 index 000000000000..f53f46006bec --- /dev/null +++ b/test/nodetool/test_compaction_throughput.py @@ -0,0 +1,19 @@ +# +# Copyright 2024-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# + +from test.nodetool.rest_api_mock import expected_request +import pytest + +def test_get_compaction_throughput(nodetool, scylla_only): + res = nodetool("getcompactionthroughput", expected_requests = [ + expected_request("GET", "/storage_service/compaction_throughput", response=0) + ]) + assert res.stdout == '0\n' + +def test_set_compaction_throughput(nodetool, scylla_only): + nodetool("setcompactionthroughput", "100", expected_requests = [ + expected_request("POST", "/storage_service/compaction_throughput", params={"value": "100"}) + ]) diff --git a/tools/scylla-nodetool.cc b/tools/scylla-nodetool.cc index b85210f043b1..2d8859820852 100644 --- a/tools/scylla-nodetool.cc +++ b/tools/scylla-nodetool.cc @@ -3249,6 +3249,22 @@ void version_operation(scylla_rest_client& client, const bpo::variables_map& vm) fmt::print(std::cout, "ReleaseVersion: {}\n", rjson::to_string_view(version_json)); } +void getcompactionthroughput_operation(scylla_rest_client& client, const bpo::variables_map& vm) { + auto res = client.get("/storage_service/compaction_throughput"); + uint32_t compaction_throughput_mb_per_sec = res.GetInt(); + fmt::print("{}\n", compaction_throughput_mb_per_sec); +} + +void setcompactionthroughput_operation(scylla_rest_client& client, const bpo::variables_map& vm) { + std::unordered_map params; + if (vm.contains("mbs")) { + params["value"] = fmt::to_string(vm["mbs"].as()); + } else { + throw std::invalid_argument(fmt::format("The throughput value must be specified")); + } + client.post("/storage_service/compaction_throughput", std::move(params)); +} + const std::vector global_options{ typed_option("host,h", "localhost", "the hostname or ip address of the ScyllaDB node"), typed_option("port,p", 10000, "the port of the REST API of the ScyllaDB node"), @@ -4451,6 +4467,34 @@ For more information, see: {}" version_operation } }, + { + { + "getcompactionthroughput", + "Get compaction IO throughput", +R"( +Print the MiB/s throughput cap for compaction in the system +)", + }, + { + getcompactionthroughput_operation + } + }, + { + { + "setcompactionthroughput", + "Set compaction IO throughput", +R"( +Set the MiB/s throughput for compaction, or 0 to disable throttling +)", + {}, + { + typed_option("mbs", "Value in MiB, 0 to disable throttling ", 1), + }, + }, + { + setcompactionthroughput_operation + } + }, }; return operations_with_func; From 3081ce24cd5c27a37e094d291be96413b9c2ae73 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 10 Dec 2024 15:59:44 +0300 Subject: [PATCH 014/397] nodetool: Implement [gs]etstreamthroughput commands They exist in the original documentation, but are not yet implemented. Now it's possible to do it. It slightly more complex that its compaction counterpart in a sense than get method reports megabits/s by default and has an option to convert to MiBs. Signed-off-by: Pavel Emelyanov --- .../nodetool-commands/getstreamthroughput.rst | 24 +++++++++ .../nodetool-commands/setstreamthroughput.rst | 19 +++++++ docs/operating-scylla/nodetool.rst | 5 +- test/nodetool/test_stream_throughput.py | 25 +++++++++ tools/scylla-nodetool.cc | 52 +++++++++++++++++++ 5 files changed, 124 insertions(+), 1 deletion(-) create mode 100644 docs/operating-scylla/nodetool-commands/getstreamthroughput.rst create mode 100644 docs/operating-scylla/nodetool-commands/setstreamthroughput.rst create mode 100644 test/nodetool/test_stream_throughput.py diff --git a/docs/operating-scylla/nodetool-commands/getstreamthroughput.rst b/docs/operating-scylla/nodetool-commands/getstreamthroughput.rst new file mode 100644 index 000000000000..ba2579c5a7c7 --- /dev/null +++ b/docs/operating-scylla/nodetool-commands/getstreamthroughput.rst @@ -0,0 +1,24 @@ +============================ +Nodetool getstreamthroughput +============================ +**getstreamthroughput** - Print the throughput cap for SSTables streaming in the system + +If zero is printed, it means throughput is uncapped + +Syntax +------- +.. code-block:: console + + nodetool [options] getstreamthroughput [--mib] + +Options +-------- + +* ``--mib`` - Print the value in MiB rather than megabits per second + +See also + +* :doc:`setstreamthroughput ` + +.. include:: nodetool-index.rst + diff --git a/docs/operating-scylla/nodetool-commands/setstreamthroughput.rst b/docs/operating-scylla/nodetool-commands/setstreamthroughput.rst new file mode 100644 index 000000000000..8e0d8ce52b9b --- /dev/null +++ b/docs/operating-scylla/nodetool-commands/setstreamthroughput.rst @@ -0,0 +1,19 @@ +============================ +Nodetool setstreamthroughput +============================ +**setstreamthroughput** - Print the throughput cap for SSTables streaming in the system + +Setting zero throughput disables capping + +Syntax +------- +.. code-block:: console + + nodetool [options] setstreamthroughput + +See also + +* :doc:`getstreamthroughput ` + +.. include:: nodetool-index.rst + diff --git a/docs/operating-scylla/nodetool.rst b/docs/operating-scylla/nodetool.rst index 675bfb4a5695..c940379e1d9d 100644 --- a/docs/operating-scylla/nodetool.rst +++ b/docs/operating-scylla/nodetool.rst @@ -62,6 +62,8 @@ Nodetool nodetool-commands/version nodetool-commands/getcompactionthroughput nodetool-commands/setcompactionthroughput + nodetool-commands/getstreamthroughput + nodetool-commands/setstreamthroughput The ``nodetool`` utility provides a simple command-line interface to the following exposed operations and attributes. @@ -136,5 +138,6 @@ Operations that are not listed below are currently not available. * :doc:`version ` - Print the DB version. * :doc:`getcompactionthroughput ` - Print the throughput cap for compaction in the system * :doc:`setcompactionthroughput ` - Set the throughput cap for compaction in the system - +* :doc:`getstreamthroughput ` - Print the throughput cap for SSTables streaming in the system +* :doc:`setstreamthroughput ` - Set the throughput cap for SSTables streaming in the system diff --git a/test/nodetool/test_stream_throughput.py b/test/nodetool/test_stream_throughput.py new file mode 100644 index 000000000000..b3ea1d85db48 --- /dev/null +++ b/test/nodetool/test_stream_throughput.py @@ -0,0 +1,25 @@ +# +# Copyright 2024-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# + +from test.nodetool.rest_api_mock import expected_request +import pytest + +def test_get_stream_throughput(nodetool, scylla_only): + res = nodetool("getstreamthroughput", "--mib", expected_requests = [ + expected_request("GET", "/storage_service/stream_throughput", response=100) + ]) + assert res.stdout == "100\n" + +def test_get_stream_throughput_mbits(nodetool, scylla_only): + res = nodetool("getstreamthroughput", expected_requests = [ + expected_request("GET", "/storage_service/stream_throughput", response=100) + ]) + assert res.stdout == f"{int(100*1024*1024*8/1000000)}\n" + +def test_set_stream_throughput(nodetool, scylla_only): + nodetool("setstreamthroughput", "100", expected_requests = [ + expected_request("POST", "/storage_service/stream_throughput", params={"value": "100"}) + ]) diff --git a/tools/scylla-nodetool.cc b/tools/scylla-nodetool.cc index 2d8859820852..0ee03dfcab21 100644 --- a/tools/scylla-nodetool.cc +++ b/tools/scylla-nodetool.cc @@ -3265,6 +3265,26 @@ void setcompactionthroughput_operation(scylla_rest_client& client, const bpo::va client.post("/storage_service/compaction_throughput", std::move(params)); } +void getstreamthroughput_operation(scylla_rest_client& client, const bpo::variables_map& vm) { + auto res = client.get("/storage_service/stream_throughput"); + uint32_t throughput_mb_per_sec = res.GetInt(); + if (vm.contains("mib")) { + fmt::print("{}\n", throughput_mb_per_sec); + } else { + fmt::print("{}\n", (((uint64_t)throughput_mb_per_sec) << 23) / 1'000'000); + } +} + +void setstreamthroughput_operation(scylla_rest_client& client, const bpo::variables_map& vm) { + std::unordered_map params; + if (vm.contains("mbs")) { + params["value"] = fmt::to_string(vm["mbs"].as()); + } else { + throw std::invalid_argument(fmt::format("The throughput value must be specified")); + } + client.post("/storage_service/stream_throughput", std::move(params)); +} + const std::vector global_options{ typed_option("host,h", "localhost", "the hostname or ip address of the ScyllaDB node"), typed_option("port,p", 10000, "the port of the REST API of the ScyllaDB node"), @@ -4495,6 +4515,38 @@ Set the MiB/s throughput for compaction, or 0 to disable throttling setcompactionthroughput_operation } }, + { + { + "getstreamthroughput", + "Get streaming throughput", +R"( +Print throughput cap for streaming in the system in megabits +)", + { + typed_option<>("mib", "Print the throughput cap for streaming in MiB/s"), + }, + {}, + }, + { + getstreamthroughput_operation + }, + }, + { + { + "setstreamthroughput", + "Set compaction IO throughput", +R"( +Set the MiB/s throughput for streaming, or 0 to disable throttling +)", + {}, + { + typed_option("mbs", "Value in MiB, 0 to disable throttling ", 1), + }, + }, + { + setstreamthroughput_operation + } + }, }; return operations_with_func; From 6eda41f305d9e67d69af229dc5248e79c4a7a3af Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 3 Dec 2024 11:50:05 +0800 Subject: [PATCH 015/397] install-dependencies.sh: define local variable using "local -A" "declare -A local GO_ARCH" does not define a single variable, instead it defines two variables named "local" and "GO_ARCH". shellcheck warns when analyzing this script: ``` In ./install-dependencies.sh line 188: declare -A local GO_ARCH=( ^---^ SC2316 (error): This applies declare to the variable named local, which is probably not what you want. Use a separate command or the appropriate `declare` optionsinstead. ^---^ SC2034 (warning): local appears unused. Verify use (or export if used externally). ``` and per the output of "help declare": ``` declare: declare [-aAfFgiIlnrtux] [name[=value] ...] or declare -p [-aAfFilnrtux] [name ...] ``` we defined two associative arrays instead of one. In this change, we use the correct Bash syntax `local -A GO_ARCH` to: - Create a single, locally-scoped associative array - Eliminate static analysis warnings - Improve code readability and maintainability This is a cleanup change with no production impact. Signed-off-by: Kefu Chai --- install-dependencies.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/install-dependencies.sh b/install-dependencies.sh index 77b34361048d..65f76e74d0eb 100755 --- a/install-dependencies.sh +++ b/install-dependencies.sh @@ -185,7 +185,7 @@ arch_packages=( ) go_arch() { - declare -A local GO_ARCH=( + local -A GO_ARCH=( ["x86_64"]=amd64 ["aarch64"]=arm64 ["s390x"]=s390x From 56971602388a5c57e42db7db672692ff1a14e829 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 3 Dec 2024 12:03:12 +0800 Subject: [PATCH 016/397] install-dependencies.sh: quote array to avoid re-splitting Before this change, we didn't quote the array of the keys of an associative array. and shellcheck warns like: ``` In install-dependencies.sh line 330: for package in ${!pip_packages[@]} ^-----------------^ SC2068 (error): Double quote array expansions to avoid re-splitting elements. ``` While the current keys in the associative array do not contain spaces, quoting array expansions is a recommended defensive programming practice. This change: - Prevents potential future issues with unexpected whitespace - Silences Shellcheck warning without changing functionality - Improves code quality and maintainability Specifically modified the array iteration from: `for package in ${!pip_packages[@]}` to: `for package in "${!pip_packages[@]}"` This change has no functional impact and serves as a proactive code improvement. Signed-off-by: Kefu Chai --- install-dependencies.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/install-dependencies.sh b/install-dependencies.sh index 65f76e74d0eb..360fb41cc053 100755 --- a/install-dependencies.sh +++ b/install-dependencies.sh @@ -327,7 +327,7 @@ elif [ "$ID" = "fedora" ]; then dnf install -y "${fedora_packages[@]}" "${fedora_python3_packages[@]}" PIP_DEFAULT_ARGS="--only-binary=:all: -v" pip_constrained_packages="" - for package in ${!pip_packages[@]} + for package in "${!pip_packages[@]}" do pip_constrained_packages="${pip_constrained_packages} ${package}${pip_packages[$package]}" done From 10c4cf930c122663eb453654a136331b4871d8bf Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Sat, 7 Dec 2024 11:58:34 +0200 Subject: [PATCH 017/397] table: make update_effective_replication_map sync again Commit f2ff701489995fdc84ea7ca0addbec2918c35be4 introduced a yield in update_effective_replication_map that might cause the storage_group manager to be inconsistent with the new effective_replication_map (e.g. if yielding right before calling `handle_tablet_split_completion`. Also, yielding inside storage_service::replicate_to_all_cores update loop means that base tables and their views aren't updated atomically, that caused scylladb/scylladb#17786 This change essentially reverts f2ff701489995fdc84ea7ca0addbec2918c35be4 and makes handle_tablet_split_completion synchronous too. The stopped compaction groups future is kept as a memebr and storage_group_manager::stop() consumes this future during table::stop(). Signed-off-by: Benny Halevy --- replica/compaction_group.hh | 2 +- replica/database.hh | 2 +- replica/table.cc | 35 +++++++++++++++-------------------- service/storage_service.cc | 2 +- 4 files changed, 18 insertions(+), 23 deletions(-) diff --git a/replica/compaction_group.hh b/replica/compaction_group.hh index 7d71ba70a051..76197522e19b 100644 --- a/replica/compaction_group.hh +++ b/replica/compaction_group.hh @@ -352,7 +352,7 @@ public: // refresh_mutation_source must be called when there are changes to data source // structures but logical state of data is not changed (e.g. when state for a // new tablet replica is allocated). - virtual future<> update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) = 0; + virtual void update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) = 0; virtual compaction_group& compaction_group_for_token(dht::token token) const noexcept = 0; virtual utils::chunked_vector compaction_groups_for_token_range(dht::token_range tr) const = 0; diff --git a/replica/database.hh b/replica/database.hh index ae05963d02da..35351c7c8621 100644 --- a/replica/database.hh +++ b/replica/database.hh @@ -863,7 +863,7 @@ public: void set_schema(schema_ptr); db::commitlog* commitlog() const; const locator::effective_replication_map_ptr& get_effective_replication_map() const { return _erm; } - future<> update_effective_replication_map(locator::effective_replication_map_ptr); + void update_effective_replication_map(locator::effective_replication_map_ptr); [[gnu::always_inline]] bool uses_tablets() const; private: future<> clear_inactive_reads_for_tablet(database& db, storage_group& sg); diff --git a/replica/table.cc b/replica/table.cc index 80ec0c1a02f8..18164cf0bce1 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -693,7 +693,7 @@ class single_storage_group_manager final : public storage_group_manager { return make_ready_future<>(); } - future<> update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) override { return make_ready_future(); } + void update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) override {} compaction_group& compaction_group_for_token(dht::token token) const noexcept override { return get_compaction_group(); @@ -739,6 +739,7 @@ class tablet_storage_group_manager final : public storage_group_manager { replica::table& _t; locator::host_id _my_host_id; const locator::tablet_map* _tablet_map; + future<> _stop_fut = make_ready_future(); // Every table replica that completes split work will load the seq number from tablet metadata into its local // state. So when coordinator pull the local state of a table, it will know whether the table is ready for the // current split, and not a previously revoked (stale) decision. @@ -764,12 +765,12 @@ class tablet_storage_group_manager final : public storage_group_manager { // Called when coordinator executes tablet splitting, i.e. commit the new tablet map with // each tablet split into two, so this replica will remap all of its compaction groups // that were previously split. - future<> handle_tablet_split_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap); + void handle_tablet_split_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap); // Called when coordinator executes tablet merge. Tablet ids X and X+1 are merged into // the new tablet id (X >> 1). In practice, that means storage groups for X and X+1 // are merged into a new storage group with id (X >> 1). - future<> handle_tablet_merge_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap); + void handle_tablet_merge_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap); // When merge completes, compaction groups of sibling tablets are added to same storage // group, but they're not merged yet into one, since the merge completion handler happens @@ -835,10 +836,11 @@ class tablet_storage_group_manager final : public storage_group_manager { future<> stop() override { _merge_completion_event.signal(); - return std::exchange(_merge_completion_fiber, make_ready_future<>()); + return when_all(std::exchange(_merge_completion_fiber, make_ready_future<>()), + std::exchange(_stop_fut, make_ready_future())).discard_result(); } - future<> update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) override; + void update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) override; compaction_group& compaction_group_for_token(dht::token token) const noexcept override; utils::chunked_vector compaction_groups_for_token_range(dht::token_range tr) const override; @@ -2469,7 +2471,7 @@ locator::table_load_stats table::table_load_stats(std::functiontable_load_stats(std::move(tablet_filter)); } -future<> tablet_storage_group_manager::handle_tablet_split_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap) { +void tablet_storage_group_manager::handle_tablet_split_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap) { auto table_id = schema()->id(); size_t old_tablet_count = old_tmap.tablet_count(); size_t new_tablet_count = new_tmap.tablet_count(); @@ -2492,7 +2494,6 @@ future<> tablet_storage_group_manager::handle_tablet_split_completion(const loca } // Stop the released main compaction groups asynchronously - future<> stop_fut = make_ready_future<>(); for (auto& [id, sg] : _storage_groups) { if (!sg->split_unready_groups_are_empty()) { on_internal_error(tlogger, format("Found that storage of group {} for table {} wasn't split correctly, " \ @@ -2503,7 +2504,7 @@ future<> tablet_storage_group_manager::handle_tablet_split_completion(const loca auto cg_ptr = sg->main_compaction_group(); auto f = cg_ptr->stop("tablet split"); if (!f.available() || f.failed()) [[unlikely]] { - stop_fut = stop_fut.then([f = std::move(f), cg_ptr = std::move(cg_ptr)] () mutable { + _stop_fut = _stop_fut.then([f = std::move(f), cg_ptr = std::move(cg_ptr)] () mutable { return std::move(f).handle_exception([cg_ptr = std::move(cg_ptr)] (std::exception_ptr ex) { tlogger.warn("Failed to stop compaction group: {}. Ignored", std::move(ex)); }); @@ -2525,8 +2526,6 @@ future<> tablet_storage_group_manager::handle_tablet_split_completion(const loca } _storage_groups = std::move(new_storage_groups); - - return stop_fut; } future<> tablet_storage_group_manager::merge_completion_fiber() { @@ -2557,7 +2556,7 @@ future<> tablet_storage_group_manager::merge_completion_fiber() { } } -future<> tablet_storage_group_manager::handle_tablet_merge_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap) { +void tablet_storage_group_manager::handle_tablet_merge_completion(const locator::tablet_map& old_tmap, const locator::tablet_map& new_tmap) { auto table_id = schema()->id(); size_t old_tablet_count = old_tmap.tablet_count(); size_t new_tablet_count = new_tmap.tablet_count(); @@ -2599,10 +2598,9 @@ future<> tablet_storage_group_manager::handle_tablet_merge_completion(const loca } _storage_groups = std::move(new_storage_groups); _merge_completion_event.signal(); - return make_ready_future<>(); } -future<> tablet_storage_group_manager::update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) { +void tablet_storage_group_manager::update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) { auto* new_tablet_map = &erm.get_token_metadata().tablets().get_tablet_map(schema()->id()); auto* old_tablet_map = std::exchange(_tablet_map, new_tablet_map); @@ -2611,13 +2609,11 @@ future<> tablet_storage_group_manager::update_effective_replication_map(const lo if (new_tablet_count > old_tablet_count) { tlogger.info0("Detected tablet split for table {}.{}, increasing from {} to {} tablets", schema()->ks_name(), schema()->cf_name(), old_tablet_count, new_tablet_count); - co_await handle_tablet_split_completion(*old_tablet_map, *new_tablet_map); - co_return; + handle_tablet_split_completion(*old_tablet_map, *new_tablet_map); } else if (new_tablet_count < old_tablet_count) { tlogger.info0("Detected tablet merge for table {}.{}, decreasing from {} to {} tablets", schema()->ks_name(), schema()->cf_name(), old_tablet_count, new_tablet_count); - co_await handle_tablet_merge_completion(*old_tablet_map, *new_tablet_map); - co_return; + handle_tablet_merge_completion(*old_tablet_map, *new_tablet_map); } // Allocate storage group if tablet is migrating in. @@ -2658,10 +2654,9 @@ future<> tablet_storage_group_manager::update_effective_replication_map(const lo if (tablet_migrating_in) { refresh_mutation_source(); } - co_return; } -future<> table::update_effective_replication_map(locator::effective_replication_map_ptr erm) { +void table::update_effective_replication_map(locator::effective_replication_map_ptr erm) { auto old_erm = std::exchange(_erm, std::move(erm)); auto refresh_mutation_source = [this] { @@ -2670,7 +2665,7 @@ future<> table::update_effective_replication_map(locator::effective_replication_ }; if (uses_tablets()) { - co_await _sg_manager->update_effective_replication_map(*_erm, refresh_mutation_source); + _sg_manager->update_effective_replication_map(*_erm, refresh_mutation_source); } if (old_erm) { old_erm->invalidate(); diff --git a/service/storage_service.cc b/service/storage_service.cc index 357e7f9bf842..da8f2d16deff 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -3160,7 +3160,7 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt auto& table_erms = pending_table_erms[this_shard_id()]; for (auto it = table_erms.begin(); it != table_erms.end(); ) { auto& cf = db.find_column_family(it->first); - co_await cf.update_effective_replication_map(std::move(it->second)); + cf.update_effective_replication_map(std::move(it->second)); co_await utils::get_local_injector().inject("delay_after_erm_update", [&cf, &ss] (auto& handler) -> future<> { auto& ss_ = ss; const auto ks_name = handler.get("ks_name"); From 4bfa3060d0a26763fda05af4e77e7d8d267bc926 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Wed, 4 Dec 2024 16:47:07 +0200 Subject: [PATCH 018/397] storage_service: replicate_to_all_cores: update base and view tables atomically Currently, the loop updating all tables (including views) with the new effective_replication_map may yield, and therefore expose a state where the base and view tables effective_replication_map and topology are out of sync (as seen in scylladb/scylladb#17786) To prevent that, loop over all base tables and for each table update the base table and all views atomically, without yielding, and so allow yielding only between base tables. Signed-off-by: Benny Halevy --- service/storage_service.cc | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index da8f2d16deff..ddb2b38b30ff 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -3050,7 +3050,9 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt std::vector> pending_effective_replication_maps; pending_effective_replication_maps.resize(smp::count); std::vector> pending_table_erms; + std::vector> pending_view_erms; pending_table_erms.resize(smp::count); + pending_view_erms.resize(smp::count); std::unordered_set open_sessions; @@ -3119,7 +3121,11 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt } else { erm = pending_effective_replication_maps[this_shard_id()][table->schema()->ks_name()]; } - pending_table_erms[this_shard_id()].emplace(id, std::move(erm)); + if (table->schema()->is_view()) { + pending_view_erms[this_shard_id()].emplace(id, std::move(erm)); + } else { + pending_table_erms[this_shard_id()].emplace(id, std::move(erm)); + } }); }); } catch (...) { @@ -3133,6 +3139,7 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt auto tmptr = std::move(pending_token_metadata_ptr[this_shard_id()]); auto erms = std::move(pending_effective_replication_maps[this_shard_id()]); auto table_erms = std::move(pending_table_erms[this_shard_id()]); + auto view_erms = std::move(pending_view_erms[this_shard_id()]); co_await utils::clear_gently(erms); co_await utils::clear_gently(tmptr); @@ -3158,9 +3165,21 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt } auto& table_erms = pending_table_erms[this_shard_id()]; + auto& view_erms = pending_view_erms[this_shard_id()]; for (auto it = table_erms.begin(); it != table_erms.end(); ) { + // Update base/views effective_replication_maps atomically. auto& cf = db.find_column_family(it->first); cf.update_effective_replication_map(std::move(it->second)); + for (const auto& view_ptr : cf.views()) { + const auto& view_id = view_ptr->id(); + auto view_it = view_erms.find(view_id); + if (view_it == view_erms.end()) { + throw std::runtime_error(format("Could not find pending effective_replication_map for view {}.{} id={}", view_ptr->ks_name(), view_ptr->cf_name(), view_id)); + } + auto& view = db.find_column_family(view_id); + view.update_effective_replication_map(std::move(view_it->second)); + view_erms.erase(view_it); + } co_await utils::get_local_injector().inject("delay_after_erm_update", [&cf, &ss] (auto& handler) -> future<> { auto& ss_ = ss; const auto ks_name = handler.get("ks_name"); @@ -3179,6 +3198,10 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt it = table_erms.erase(it); } + if (!view_erms.empty()) { + throw std::runtime_error(fmt::format("Found orphaned pending effective_replication_maps for the following views: {}", std::views::keys(view_erms))); + } + auto& session_mgr = get_topology_session_manager(); session_mgr.initiate_close_of_sessions_except(open_sessions); for (auto id : open_sessions) { From 500ca1737051226e97fcfb03ab2c71e4d9f9cdbf Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 12 Dec 2024 09:00:36 +0200 Subject: [PATCH 019/397] test_mv_topology_change: drop delay_after_erm_update injection case After last patch, we deliberately don't yield between update of base table erm and updating its view, which was the scenario tested with the `delay_after_erm_update` error injection point. Instead, call maybe_yield in between base/views updates to prevent reactor stalls with many tables. Signed-off-by: Benny Halevy --- service/storage_service.cc | 13 +------------ test/topology_custom/test_mv_topology_change.py | 1 - 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index ddb2b38b30ff..b6161a5ce175 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -3167,6 +3167,7 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt auto& table_erms = pending_table_erms[this_shard_id()]; auto& view_erms = pending_view_erms[this_shard_id()]; for (auto it = table_erms.begin(); it != table_erms.end(); ) { + co_await coroutine::maybe_yield(); // Update base/views effective_replication_maps atomically. auto& cf = db.find_column_family(it->first); cf.update_effective_replication_map(std::move(it->second)); @@ -3180,18 +3181,6 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt view.update_effective_replication_map(std::move(view_it->second)); view_erms.erase(view_it); } - co_await utils::get_local_injector().inject("delay_after_erm_update", [&cf, &ss] (auto& handler) -> future<> { - auto& ss_ = ss; - const auto ks_name = handler.get("ks_name"); - const auto cf_name = handler.get("cf_name"); - SCYLLA_ASSERT(ks_name); - SCYLLA_ASSERT(cf_name); - if (cf.schema()->ks_name() != *ks_name || cf.schema()->cf_name() != *cf_name) { - co_return; - } - - co_await sleep_abortable(std::chrono::seconds{5}, ss_._abort_source); - }); if (cf.uses_tablets()) { register_tablet_split_candidate(it->first); } diff --git a/test/topology_custom/test_mv_topology_change.py b/test/topology_custom/test_mv_topology_change.py index 40700d756a06..a0e3ffe8e5e2 100644 --- a/test/topology_custom/test_mv_topology_change.py +++ b/test/topology_custom/test_mv_topology_change.py @@ -71,7 +71,6 @@ async def do_writes(start_it, repeat) -> int: await asyncio.gather(*tasks) [await manager.api.disable_injection(s.ip_addr, "delay_before_get_view_natural_endpoint") for s in servers] - [await manager.api.enable_injection(s.ip_addr, "delay_after_erm_update", False, parameters={'ks_name': 'ks', 'cf_name': 't'}) for s in servers] # to hit the issue #17786 we need to run multiple batches of writes, so that some write is processed while the # effective replication maps for base and view are different From 8832301fe0b08c1978a3a0f4fd5b20e894e25a76 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Sat, 7 Dec 2024 12:34:42 +0200 Subject: [PATCH 020/397] storage_service: replicate_to_all_cores: clear_gently pending erms In case the update is rolled back on error, call clear_gently for table_erms and view_erms to prevent potential stalls with a large number of tables. Signed-off-by: Benny Halevy --- service/storage_service.cc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/service/storage_service.cc b/service/storage_service.cc index b6161a5ce175..4c2b2d0383a6 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -3143,6 +3143,8 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt co_await utils::clear_gently(erms); co_await utils::clear_gently(tmptr); + co_await utils::clear_gently(table_erms); + co_await utils::clear_gently(view_erms); }); } catch (...) { slogger.warn("Failure to reset pending token_metadata in cleanup path: {}. Ignored.", std::current_exception()); From 2de3c079b28555f068fc792fe052485aad58c15f Mon Sep 17 00:00:00 2001 From: Tomasz Grabiec Date: Mon, 16 Dec 2024 01:11:04 +0100 Subject: [PATCH 021/397] tablets: topology_coordinator: Keep tablet_draining transition if nodes are not drained Empty plan with nodes to drain meant that we can exit tablet_draining transition and move to the next stage of decommission/removenode. In case tablet scheduler creates an empty plan for some reason but there are still underained tablets, that could put topology in an invalid state. For example, this can currently happen if there are no non-draining nodes in a DC. This patch adds a safety net in the topology coordinator which prevents moving forward with undrained tablets. --- service/topology_coordinator.cc | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index d56b428ddb34..cef1df092017 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -1615,8 +1615,11 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { on_internal_error(rtlogger, "should_preempt_balancing() retook the guard"); } } + + bool has_nodes_to_drain = false; if (!preempt) { auto plan = co_await _tablet_allocator.balance_tablets(get_token_metadata_ptr(), _tablet_load_stats, get_dead_nodes()); + has_nodes_to_drain = plan.has_nodes_to_drain(); if (!drain || plan.has_nodes_to_drain()) { co_await generate_migration_updates(updates, guard, plan); } @@ -1664,6 +1667,14 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { } if (drain) { + if (has_nodes_to_drain) { + // Prevent jumping to write_both_read_old with un-drained tablets. + // This can happen when all candidate nodes are down. + rtlogger.warn("Tablet draining stalled: No tablets migrating but there are nodes to drain"); + release_guard(std::move(guard)); + co_await sleep(3s); // Throttle retries + co_return; + } updates.emplace_back( topology_mutation_builder(guard.write_timestamp()) .set_transition_state(topology::transition_state::write_both_read_old) From 8718450172e86e2f9505feba9c5df06bbbb512cc Mon Sep 17 00:00:00 2001 From: Tomasz Grabiec Date: Mon, 16 Dec 2024 01:14:19 +0100 Subject: [PATCH 022/397] tablets: load_balancer: Ignore skip_list when draining When doing normal load balancing, we can ignore DOWN nodes in the node set and just balance the UP nodes among themselves because it's ok to equalize load just in that set, it improves the situation. It's dangerous to do that when draining because that can lead to overloading of the UP nodes. In the worst case, we can have only one non-drained node in the UP set, which would receive all the tablets of the drained node, doubling its load. It's safer to let the drain fail or stall. This is decided by topology coordinator, currently we will fail (on barrier) and rollback. --- service/tablet_allocator.cc | 15 ++++++-- test/boost/tablets_test.cc | 71 +++++++++++++++++++++++++++++++++++++ 2 files changed, 84 insertions(+), 2 deletions(-) diff --git a/service/tablet_allocator.cc b/service/tablet_allocator.cc index 74d26d60a9bb..3aef0642824c 100644 --- a/service/tablet_allocator.cc +++ b/service/tablet_allocator.cc @@ -2443,15 +2443,26 @@ class load_balancer { lblogger.info("Will drain node {} ({}) from DC {}", node.host_id(), node.get_state(), dc); nodes_to_drain.emplace(node.host_id()); nodes[node.host_id()].drained = true; - } else if (node.is_excluded() || _skiplist.contains(node.host_id())) { + } else if (node.is_excluded()) { // Excluded nodes should not be chosen as targets for migration. - lblogger.debug("Ignoring excluded or dead node {}: state={}", node.host_id(), node.get_state()); + lblogger.debug("Ignoring excluded node {}: state={}", node.host_id(), node.get_state()); } else { ensure_node(node.host_id()); } } }); + // Apply skiplist only when not draining. + // It's unsafe to move tablets to non-skip nodes as this can lead to node overload. + if (nodes_to_drain.empty()) { + for (auto host_to_skip : _skiplist) { + if (auto handle = nodes.extract(host_to_skip)) { + auto& node = handle.mapped(); + lblogger.debug("Ignoring dead node {}: state={}", node.id, node.node->get_state()); + } + } + } + // Compute tablet load on nodes. for (auto&& [table, tmap_] : _tm->tablets().all_tables()) { diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 430c6e78ddf1..b59b6b649718 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -2512,6 +2512,77 @@ SEASTAR_THREAD_TEST_CASE(test_drained_node_is_not_balanced_internally) { }).get(); } +SEASTAR_THREAD_TEST_CASE(test_skiplist_is_ignored_when_draining) { + // When doing normal load balancing, we can ignore DOWN nodes in the node set + // and just balance the UP nodes among themselves because it's ok to equalize + // load in that set. + // It's dangerous to do that when draining because that can lead to overloading of the UP nodes. + // In the worst case, we can have only one non-drained node in the UP set, which would receive + // all the tablets of the drained node, doubling its load. + // It's safer to let the drain fail/stall. + do_with_cql_env_thread([] (auto& e) { + inet_address ip1("192.168.0.1"); + inet_address ip2("192.168.0.2"); + inet_address ip3("192.168.0.3"); + + auto host1 = host_id(next_uuid()); + auto host2 = host_id(next_uuid()); + auto host3 = host_id(next_uuid()); + + auto table1 = table_id(next_uuid()); + + unsigned shard_count = 1; + + semaphore sem(1); + shared_token_metadata stm([&sem] () noexcept { return get_units(sem, 1); }, locator::token_metadata::config{ + locator::topology::config{ + .this_endpoint = ip1, + .this_host_id = host1, + .local_dc_rack = locator::endpoint_dc_rack::default_location + } + }); + + stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> { + tm.update_host_id(host1, ip1); + tm.update_host_id(host2, ip2); + tm.update_host_id(host3, ip3); + tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count); + tm.update_topology(host2, locator::endpoint_dc_rack::default_location, locator::node::state::normal, shard_count); + tm.update_topology(host3, locator::endpoint_dc_rack::default_location, locator::node::state::normal, shard_count); + co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 3))}, host1); + co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(2. / 3))}, host2); + co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(3. / 3))}, host3); + + tablet_map tmap(2); + auto tid = tmap.first_tablet(); + tmap.set_tablet(tid, tablet_info { + tablet_replica_set{tablet_replica{host1, 0}} + }); + tid = *tmap.next_tablet(tid); + tmap.set_tablet(tid, tablet_info { + tablet_replica_set{tablet_replica{host1, 0}} + }); + tablet_metadata tmeta; + tmeta.set_tablet_map(table1, std::move(tmap)); + tm.set_tablets(std::move(tmeta)); + co_return; + }).get(); + + std::unordered_set skiplist = {host2}; + rebalance_tablets(e.get_tablet_allocator().local(), stm, {}, skiplist); + + { + load_sketch load(stm.get()); + load.populate().get(); + + for (auto h : {host2, host3}) { + testlog.debug("Checking host {}", h); + BOOST_REQUIRE_EQUAL(load.get_avg_shard_load(h), 1); + } + } + }).get(); +} + static void check_tablet_invariants(const tablet_metadata& tmeta) { for (auto&& [table, tmap] : tmeta.all_tables()) { From e732ff7cd89d8a78b38d889c86dc18a0d9742fa9 Mon Sep 17 00:00:00 2001 From: Tomasz Grabiec Date: Mon, 16 Dec 2024 01:18:11 +0100 Subject: [PATCH 023/397] tablets: load_balancer: Fail when draining with no candidate nodes If we're draining the last node in a DC, we won't have a chance to evaluate candidates and notice that constraints cannot be satisfied (N < RF). Draining will succeed and node will be removed with replicas still present on that node. This will cause later draining in the same DC to fail when we will have 2 replicas which need relocaiton for a given tablet. The expected behvior is for draining to fail, because we cannot keep the RF in the DC. This is consistent, for example, with what happens when removing a node in a 2-node cluster with RF=2. Fixes #21826 --- service/tablet_allocator.cc | 4 +++ test/boost/tablets_test.cc | 42 ++++++++++++++++++++++++++++ test/topology_custom/test_tablets.py | 3 +- 3 files changed, 47 insertions(+), 2 deletions(-) diff --git a/service/tablet_allocator.cc b/service/tablet_allocator.cc index 3aef0642824c..ec9b269cbf84 100644 --- a/service/tablet_allocator.cc +++ b/service/tablet_allocator.cc @@ -2559,6 +2559,10 @@ class load_balancer { } if (!min_load_node) { + if (!nodes_to_drain.empty()) { + throw std::runtime_error(format("There are nodes with tablets to drain but no candidate nodes in DC {}." + " Consider adding new nodes or reducing replication factor.", dc)); + } lblogger.debug("No candidate nodes"); _stats.for_dc(dc).stop_no_candidates++; co_return plan; diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index b59b6b649718..5fb39cb92087 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -2512,6 +2512,48 @@ SEASTAR_THREAD_TEST_CASE(test_drained_node_is_not_balanced_internally) { }).get(); } +SEASTAR_THREAD_TEST_CASE(test_plan_fails_when_removing_last_replica) { + do_with_cql_env_thread([] (auto& e) { + inet_address ip1("192.168.0.1"); + inet_address ip2("192.168.0.2"); + + auto host1 = host_id(next_uuid()); + + auto table1 = table_id(next_uuid()); + + unsigned shard_count = 1; + + semaphore sem(1); + shared_token_metadata stm([&sem] () noexcept { return get_units(sem, 1); }, locator::token_metadata::config{ + locator::topology::config{ + .this_endpoint = ip1, + .this_host_id = host1, + .local_dc_rack = locator::endpoint_dc_rack::default_location + } + }); + + stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> { + tm.update_host_id(host1, ip1); + tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count); + co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1); + + tablet_map tmap(1); + for (auto tid : tmap.tablet_ids()) { + tmap.set_tablet(tid, tablet_info { + tablet_replica_set{tablet_replica{host1, 0}} + }); + } + tablet_metadata tmeta; + tmeta.set_tablet_map(table1, std::move(tmap)); + tm.set_tablets(std::move(tmeta)); + co_return; + }).get(); + + std::unordered_set skiplist = {host1}; + BOOST_REQUIRE_THROW(rebalance_tablets(e.get_tablet_allocator().local(), stm, {}, skiplist), std::runtime_error); + }).get(); +} + SEASTAR_THREAD_TEST_CASE(test_skiplist_is_ignored_when_draining) { // When doing normal load balancing, we can ignore DOWN nodes in the node set // and just balance the UP nodes among themselves because it's ok to equalize diff --git a/test/topology_custom/test_tablets.py b/test/topology_custom/test_tablets.py index 79172caca1df..30a4c8dabc59 100644 --- a/test/topology_custom/test_tablets.py +++ b/test/topology_custom/test_tablets.py @@ -631,7 +631,6 @@ async def test_orphaned_sstables_on_startup(manager: ManagerClient): @pytest.mark.asyncio @pytest.mark.parametrize("with_zero_token_node", [False, True]) -@pytest.mark.xfail(reason="https://github.com/scylladb/scylladb/issues/21826") async def test_remove_failure_with_no_normal_token_owners_in_dc(manager: ManagerClient, with_zero_token_node: bool): """ Reproducer for #21826 @@ -664,7 +663,7 @@ async def test_remove_failure_with_no_normal_token_owners_in_dc(manager: Manager logger.info("Attempting removenode - expected to fail") await manager.remove_node(initiator_node.server_id, server_id=node_to_remove.server_id, ignore_dead=[replaced_host_id], - expected_error="Removenode failed. See earlier errors (Rolled back: Failed to drain tablets: std::runtime_error (Unable to find new replica for tablet") + expected_error="Removenode failed. See earlier errors (Rolled back: Failed to drain tablets: std::runtime_error (There are nodes with tablets to drain") logger.info(f"Replacing {node_to_replace} with a new node") replace_cfg = ReplaceConfig(replaced_id=node_to_remove.server_id, reuse_ip_addr = False, use_host_id=True, wait_replaced_dead=True) From e3656535602e945a127469a790a3aa7690beefcc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dawid=20M=C4=99drek?= Date: Tue, 17 Dec 2024 13:42:15 +0100 Subject: [PATCH 024/397] test/boost: Add test for creating roles with hashed passwords We add a new test verifying that after creating a role with a hashed password using one of the supported encryption algorithms: bcrypt, sha256, sha512, or md5, the user can successfully log in. --- test/boost/auth_test.cc | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/test/boost/auth_test.cc b/test/boost/auth_test.cc index e28aad135dde..49d80aeb31dd 100644 --- a/test/boost/auth_test.cc +++ b/test/boost/auth_test.cc @@ -383,6 +383,40 @@ SEASTAR_TEST_CASE(test_try_create_role_with_hashed_password_as_anonymous_user) { }, auth_on(true)); } +SEASTAR_TEST_CASE(test_create_roles_with_hashed_password_and_log_in) { + // This test ensures that Scylla allows for creating roles with hashed passwords + // following the format of one of the supported algorithms, as well as logging in + // as that role is performed successfully. + return do_with_cql_env_thread([] (cql_test_env& env) { + // Pairs of form (password, hashed password). + constexpr std::pair passwords[] = { + // bcrypt's. + {"myPassword", "$2a$05$ae4qyC7lYe47n8K2f/fgKuW/TCRCCpEvcYrA4Dl14VYJAjAEz3tli"}, + {"myPassword", "$2b$05$ae4qyC7lYe47n8K2f/fgKuW/TCRCCpEvcYrA4Dl14VYJAjAEz3tli"}, + {"myPassword", "$2x$05$ae4qyC7lYe47n8K2f/fgKuW/TCRCCpEvcYrA4Dl14VYJAjAEz3tli"}, + {"myPassword", "$2y$05$ae4qyC7lYe47n8K2f/fgKuW/TCRCCpEvcYrA4Dl14VYJAjAEz3tli"}, + // sha512. + {"myPassword", "$6$pffOF1SkGYpLPe7h$tsYwSqUvbzh2O79dtMNadUsYawCrHMfK06XWFh3vJIMwqaVsaiFsubB2a7uZshDVpJWhTCnGWGKsy3fAteFw9/"}, + // sha256. + {"myPassword", "$5$AKS.nD1e18H.7gu9$IWy7QB0K.qoYkrWmFn6rZ4BO6Y.FWdCchrFg3beXfx8"}, + // md5. + {"myPassword", "$1$rVcnG0Et$qAhrrNev1JVV9Zu5qhnry1"} + }; + for (auto [pwd, hash] : passwords) { + env.execute_cql(seastar::format("CREATE ROLE r WITH HASHED PASSWORD = '{}' AND LOGIN = true", hash)).get(); + // First, try to log in using an incorrect password. + BOOST_REQUIRE_EXCEPTION(authenticate(env, "r", "notThePassword").get(), exceptions::authentication_exception, + exception_predicate::message_equals("Username and/or password are incorrect")); + // Now use the correct one. + authenticate(env, "r", pwd).get(); + + // We need to log in as a superuser to be able to drop the role. + authenticate(env, "cassandra", "cassandra").get(); + env.execute_cql("DROP ROLE r").get(); + } + }, auth_on(true)); +} + SEASTAR_TEST_CASE(test_try_login_after_creating_roles_with_hashed_password) { return do_with_cql_env_thread([] (cql_test_env& env) { // Note: crypt(5) specifies: From 461a6b129c41b2b92a2dda5c15c5c87a3343a701 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dawid=20M=C4=99drek?= Date: Mon, 2 Dec 2024 18:38:14 +0100 Subject: [PATCH 025/397] docs: Update documentation on CREATE ROLE WITH HASHED PASSWORD As part of #18750, we added a CQL statement CREATE ROLE WITH SALTED HASH that prevented hashing a password when creating a role, effectively leading to inserting a hash given by the user directly into the database. In #21350, we noticed that Cassandra had implemented a CQL statement of similar semantics but different syntax. We decided to rename Scylla's statement to be compatible with Cassandra. Unfortunately, we didn't notice one more difference between what we had in Scylla and what was part of Cassandra. Scylla's statement was originally supposed to only be used when restoring the schema and the user needn't have to be aware of its existence at all: the database produced a sequence of CQL statements that the user saved to a file and when a need to restore the schema arose, they would execute the contents of the file. That's why that although we documented the feature, it was only done in the necessary places. Those that weren't related to the backup & restore procedure were deliberately skipped. Cassandra, on the other hand, added the statement for a different purpose (for details, see the relevant issue) and it was supposed to be used by the user by design. The statement is also documented as such. Since we want to preserve compatibility with Cassandra, we document the statement and its semantics in the user documentation, explicitly implying that it can be used by the user. Fixes scylladb/scylladb#21691 --- docs/cql/describe-schema.rst | 4 +- .../security/authorization.rst | 44 +++++++++++++++++++ 2 files changed, 46 insertions(+), 2 deletions(-) diff --git a/docs/cql/describe-schema.rst b/docs/cql/describe-schema.rst index 42cca27a2b7b..89fd6d148518 100644 --- a/docs/cql/describe-schema.rst +++ b/docs/cql/describe-schema.rst @@ -107,5 +107,5 @@ following form: CREATE ROLE [IF NOT EXISTS] WITH HASHED PASSWORD = '' AND LOGIN = AND SUPERUSER = The semantics of this statement is analogous to the regular ``CREATE ROLE`` statement except that it circumvents -the encryption phase of the execution and inserts the hashed password directly into ``system.roles``. You should not use -this statement unless it was returned by ``DESCRIBE SCHEMA``. +the encryption phase of the execution and inserts the hashed password directly into the database. +See :doc:`the article on authorization in ScyllaDB ` to learn more about it. diff --git a/docs/operating-scylla/security/authorization.rst b/docs/operating-scylla/security/authorization.rst index e6dc6688510c..346763df8ee7 100644 --- a/docs/operating-scylla/security/authorization.rst +++ b/docs/operating-scylla/security/authorization.rst @@ -91,6 +91,50 @@ If the option is used and the role exists, the statement is a no-op:: CREATE ROLE other_role; CREATE ROLE IF NOT EXISTS other_role; +Creating a role with a hashed password +`````````````````````````````````````` + +When you create a role with a password, the password is hashed by ScyllaDB and only then inserted into the database. +However, there also exists a version of the statement allowing you to create a role with a pre-hashed password; the syntax: + +.. code-block:: + + create_role_statement: CREATE ROLE [ IF NOT EXISTS ] `role_name` + : WITH HASHED PASSWORD `hashed_password` + : ( AND `role_option` )* + hashed_password: `string` + role_option: LOGIN '=' `boolean` + :| SUPERUSER '=' `boolean` + :| OPTIONS '=' `map_literal` + +When that statement is executed, ScyllaDB will not hash the provided hashed password; it will be inserted into +the database directly, with no additional processing of it. The statement only works with an authenticator that +uses passwords, e.g. PasswordAuthenticator. It can only be executed by a role with the superuser privilege. + +The provided hashed password must be an encrypted string using the library function `crypt(3)` specified by POSIX, +and it must use one of the encryption algorithms that ScyllaDB supports: bcrypt, sha512crypt, sha256crypt, md5crypt. +If the hashed password doesn't satisfy those requirements, logging in as the role may be impossible. + +Example how to generate a hashed password using `crypt(3)`: + +.. code-block:: c + + #include + #include + + int main(void) { + // Generate a hashed password using sha256. + const char* pwd = crypt("myPassword", "$5$"); + printf("The hashed password is %s\n", pwd); + return 0; + } + +Example how to create a role with a hashed password: + +.. code-block:: cql + + cqlsh> CREATE ROLE bob WITH HASHED PASSWORD = '$5$$QdpObVOY40UOyo.BuWdpRq0Cr/DPkw7ckBEs2NqWVn5'; + .. _alter-role-statement: From 6ad962cb38aa5ac44d401c8082916999bd6beded Mon Sep 17 00:00:00 2001 From: Lakshmi Narayanan Sreethar Date: Tue, 17 Dec 2024 15:23:30 +0530 Subject: [PATCH 026/397] sstables_manager: log sstable name when reclaiming components Signed-off-by: Lakshmi Narayanan Sreethar --- sstables/sstables_manager.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sstables/sstables_manager.cc b/sstables/sstables_manager.cc index 371757089257..06fd44d639e6 100644 --- a/sstables/sstables_manager.cc +++ b/sstables/sstables_manager.cc @@ -171,7 +171,9 @@ void sstables_manager::increment_total_reclaimable_memory_and_maybe_reclaim(ssta _total_memory_reclaimed += memory_reclaimed; _total_reclaimable_memory -= memory_reclaimed; _reclaimed.insert(*sst_with_max_memory); - smlogger.info("Reclaimed {} bytes of memory from SSTable components. Total memory reclaimed so far is {} bytes", memory_reclaimed, _total_memory_reclaimed); + // TODO: As of now only bloom filter is reclaimed. Print actual component names when adding support for more components. + smlogger.info("Reclaimed {} bytes of memory from components of {}. Total memory reclaimed so far is {} bytes", + memory_reclaimed, sst_with_max_memory->get_filename(), _total_memory_reclaimed); } size_t sstables_manager::get_memory_available_for_reclaimable_components() { From b7b4c5c661e10013a7e1022857a5a1028d06e8a3 Mon Sep 17 00:00:00 2001 From: Lakshmi Narayanan Sreethar Date: Tue, 17 Dec 2024 14:05:11 +0530 Subject: [PATCH 027/397] sstables: introduce disable_component_memory_reload() Added a new method to disable reload of previously reclaimed components from the sstable. This will be used to disable reload of bloom filters after an sstable has been unlinked or deactivated. Signed-off-by: Lakshmi Narayanan Sreethar --- sstables/sstables.cc | 9 +++++++++ sstables/sstables.hh | 2 ++ 2 files changed, 11 insertions(+) diff --git a/sstables/sstables.cc b/sstables/sstables.cc index 283022d27896..648182124376 100644 --- a/sstables/sstables.cc +++ b/sstables/sstables.cc @@ -1553,6 +1553,15 @@ future<> sstable::reload_reclaimed_components() { sstlog.info("Reloaded bloom filter of {}", get_filename()); } +void sstable::disable_component_memory_reload() { + if (total_reclaimable_memory_size() > 0) { + // should be called only when the components have been dropped already + on_internal_error(sstlog, "disable_component_memory_reload() called with reclaimable memory"); + } + + _total_memory_reclaimed = 0; +} + future<> sstable::load_metadata(sstable_open_config cfg, bool validate) noexcept { co_await read_toc(); // read scylla-meta after toc. Might need it to parse diff --git a/sstables/sstables.hh b/sstables/sstables.hh index 79d1383bcaf1..a7acd4b5873d 100644 --- a/sstables/sstables.hh +++ b/sstables/sstables.hh @@ -710,6 +710,8 @@ private: size_t total_memory_reclaimed() const; // Reload components from which memory was previously reclaimed future<> reload_reclaimed_components(); + // Disable reload of components for this sstable + void disable_component_memory_reload(); public: // Finds first position_in_partition in a given partition. From 5dffc19f2d20308226d11c5b4bc876ca41f7d72f Mon Sep 17 00:00:00 2001 From: Lakshmi Narayanan Sreethar Date: Tue, 17 Dec 2024 14:52:26 +0530 Subject: [PATCH 028/397] sstables_manager: introduce reclaim_memory_and_stop_tracking_sstable() When an sstable is unlinked or deactivated, it should be removed from the component memory tracking metrics and any further reload/reclaim should be disabled. This patch adds a new method that implements the above mentioned functionality. This patch also updates the deactivate() to use the new method. Next patch will use it to disable tracking when an sstable is unlinked. Signed-off-by: Lakshmi Narayanan Sreethar --- sstables/sstables_manager.cc | 18 ++++++++++++++---- sstables/sstables_manager.hh | 4 ++++ 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/sstables/sstables_manager.cc b/sstables/sstables_manager.cc index 06fd44d639e6..cd6d4d5a6361 100644 --- a/sstables/sstables_manager.cc +++ b/sstables/sstables_manager.cc @@ -227,15 +227,25 @@ future<> sstables_manager::components_reloader_fiber() { } } +void sstables_manager::reclaim_memory_and_stop_tracking_sstable(sstable* sst) { + // remove the sstable from the memory tracking metrics + _total_reclaimable_memory -= sst->total_reclaimable_memory_size(); + _total_memory_reclaimed -= sst->total_memory_reclaimed(); + // reclaim any remaining memory from the sstable + sst->reclaim_memory_from_components(); + // disable further reload of components + _reclaimed.erase(*sst); + sst->disable_component_memory_reload(); +} + void sstables_manager::add(sstable* sst) { _active.push_back(*sst); } void sstables_manager::deactivate(sstable* sst) { - // Remove SSTable from the reclaimable memory tracking - _total_reclaimable_memory -= sst->total_reclaimable_memory_size(); - _total_memory_reclaimed -= sst->total_memory_reclaimed(); - _reclaimed.erase(*sst); + // Drop reclaimable components if they are still in memory + // and remove SSTable from the reclaimable memory tracking + reclaim_memory_and_stop_tracking_sstable(sst); // At this point, sst has a reference count of zero, since we got here from // lw_shared_ptr_deleter::dispose(). diff --git a/sstables/sstables_manager.hh b/sstables/sstables_manager.hh index e792f7e2a919..d861b9240dbb 100644 --- a/sstables/sstables_manager.hh +++ b/sstables/sstables_manager.hh @@ -219,6 +219,10 @@ private: // Fiber to reload reclaimed components back into memory when memory becomes available. future<> components_reloader_fiber(); size_t get_memory_available_for_reclaimable_components(); + // Reclaim memory from the SSTable and remove it from the memory tracking metrics. + // The method is idempotent and for an sstable that is deleted, it is called both + // during unlink and during deactivation. + void reclaim_memory_and_stop_tracking_sstable(sstable* sst); private: db::large_data_handler& get_large_data_handler() const { return _large_data_handler; From 4fe43672424f531b7cb7b51d9b853f1521fbdcf1 Mon Sep 17 00:00:00 2001 From: Lakshmi Narayanan Sreethar Date: Wed, 11 Dec 2024 21:09:42 +0530 Subject: [PATCH 029/397] sstables_manager: reclaim memory from sstables on unlink When an sstable is unlinked, it remains in the _active list of the sstable manager. Its memory might be reclaimed and later reloaded, causing issues since the sstable is already unlinked. This patch updates the on_unlink method to reclaim memory from the sstable upon unlinking, remove it from memory tracking, and thereby prevent the issues described above. Added a testcase to verify the fix. Fixes #21887 Signed-off-by: Lakshmi Narayanan Sreethar --- sstables/sstables_manager.cc | 3 +- test/boost/bloom_filter_test.cc | 75 ++++++++++++++++++++++++++++++--- test/lib/sstable_test_env.hh | 4 ++ 3 files changed, 74 insertions(+), 8 deletions(-) diff --git a/sstables/sstables_manager.cc b/sstables/sstables_manager.cc index cd6d4d5a6361..a25bb324723b 100644 --- a/sstables/sstables_manager.cc +++ b/sstables/sstables_manager.cc @@ -342,8 +342,7 @@ std::vector sstables_manager::get_local_directories(const } void sstables_manager::on_unlink(sstable* sst) { - // Remove the sst from manager's reclaimed list to prevent any attempts to reload its components. - _reclaimed.erase(*sst); + reclaim_memory_and_stop_tracking_sstable(sst); } sstables_registry::~sstables_registry() = default; diff --git a/test/boost/bloom_filter_test.cc b/test/boost/bloom_filter_test.cc index a11d745e1a23..90976c74e063 100644 --- a/test/boost/bloom_filter_test.cc +++ b/test/boost/bloom_filter_test.cc @@ -246,7 +246,6 @@ SEASTAR_TEST_CASE(test_bloom_filter_reload_after_unlink) { auto sst = make_sstable_containing(env.make_sstable(schema), {mut}); auto& sst_mgr = env.manager(); BOOST_REQUIRE_EQUAL(sst->filter_memory_size(), 0); - auto memory_reclaimed = sst_mgr.get_total_memory_reclaimed(); // manager's reclaimed set has the sst now auto& reclaimed_set = sst_mgr.get_reclaimed_set(); @@ -267,19 +266,83 @@ SEASTAR_TEST_CASE(test_bloom_filter_reload_after_unlink) { sst->unlink().get(); sst.release(); - // reclaimed set should be now empty but the total memory reclaimed should - // be still the same as the sst object is not deactivated yet due to a copy - // being alive in the async thread. + // verify that the sstable manager's reclaimed set and the reclaim metrics + // do not track unlinked sst1 at this point. BOOST_REQUIRE_EQUAL(sst_mgr.get_reclaimed_set().size(), 0); - BOOST_REQUIRE_EQUAL(sst_mgr.get_total_memory_reclaimed(), memory_reclaimed); + BOOST_REQUIRE_EQUAL(sst_mgr.get_total_memory_reclaimed(), 0); // message async thread to complete waiting and thus release its copy of sst, triggering deactivation utils::get_local_injector().receive_message("test_bloom_filter_reload_after_unlink"); async_sst_holder.get(); - REQUIRE_EVENTUALLY_EQUAL(sst_mgr.get_total_memory_reclaimed(), 0); + REQUIRE_EVENTUALLY_EQUAL(sst_mgr.get_active_list().size(), 0); }, { // set available memory = 0 to force reclaim the bloom filter .available_memory = 0 }); }; + +SEASTAR_TEST_CASE(test_bloom_filter_reclaim_after_unlink) { + return test_env::do_with_async([] (test_env& env) { +#ifndef SCYLLA_ENABLE_ERROR_INJECTION + fmt::print("Skipping test as it depends on error injection. Please run in mode where it's enabled (debug,dev).\n"); + return; +#endif + simple_schema ss; + auto schema = ss.schema(); + + std::vector mutations; + for (int i = 0; i < 10; i++) { + auto mut = mutation(schema, ss.make_pkey(i)); + mut.partition().apply_insert(*schema, ss.make_ckey(1), ss.new_timestamp()); + mutations.push_back(std::move(mut)); + } + + // create one sst; there is sufficient memory for the bloom filter, so it won't be reclaimed + auto sst1 = make_sstable_containing(env.make_sstable(schema), mutations); + auto& sst_mgr = env.manager(); + auto sst1_filename = sst1->get_filename(); + BOOST_REQUIRE(sst1->filter_memory_size() != 0); + BOOST_REQUIRE_EQUAL(sst_mgr.get_total_memory_reclaimed(), 0); + + // hold a copy of shared sst object in async thread to test reclaim after unlink + utils::get_local_injector().enable("test_bloom_filter_reload_after_unlink"); + auto async_sst_holder = seastar::async([sst1] { + // do nothing just hold a copy of sst and wait for message signalling test completion + utils::get_local_injector().inject("test_bloom_filter_reload_after_unlink", [] (auto& handler) { + auto ret = handler.wait_for_message(std::chrono::steady_clock::now() + std::chrono::seconds{5}); + return ret; + }).get(); + }); + + // unlink sst1 - this will also reclaim the memory from sst1's bloom filter. + sst1->unlink().get(); + BOOST_REQUIRE_EQUAL(sst1->filter_memory_size(), 0); + // verify that the sstable manager's reclaimed set and the reclaim metrics + // do not track unlinked sst1 at this point. + BOOST_REQUIRE_EQUAL(sst_mgr.get_reclaimed_set().size(), 0); + BOOST_REQUIRE_EQUAL(sst_mgr.get_total_memory_reclaimed(), 0); + // release the sst1's reference from this thread + sst1.release(); + // despite being unlinked and released, the sstable manager's _active list will still + // hold sst1 as the async thread still has a reference. + auto& active_list = sst_mgr.get_active_list(); + BOOST_REQUIRE_EQUAL(active_list.size(), 1); + BOOST_REQUIRE_EQUAL(active_list.front().get_filename(), sst1_filename); + + // create another sst and unlink it to trigger reload of components. + // the reload should not attempt to load sst'1 bloom filter into memory depsite its presence in the _active list. + auto sst2 = make_sstable_containing(env.make_sstable(schema), {mutations[0]}); + sst2->unlink().get(); + sst2.release(); + + // message async thread to complete waiting and thus release its copy of sst, triggering deactivation + utils::get_local_injector().receive_message("test_bloom_filter_reload_after_unlink"); + async_sst_holder.get(); + + REQUIRE_EVENTUALLY_EQUAL(active_list.size(), 0); + }, { + // set available memory = 0 to force reclaim the bloom filter + .available_memory = 100 + }); +}; diff --git a/test/lib/sstable_test_env.hh b/test/lib/sstable_test_env.hh index 9e687539a1f3..3ef09b142481 100644 --- a/test/lib/sstable_test_env.hh +++ b/test/lib/sstable_test_env.hh @@ -68,6 +68,10 @@ public: _reclaimed.erase(*sst); } + auto& get_active_list() { + return _active; + } + auto& get_reclaimed_set() { return _reclaimed; } From e80355d3a1178f97c5b6cac91daa7f7bb4a4d6fd Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Tue, 17 Dec 2024 16:54:25 +0200 Subject: [PATCH 030/397] gossiper: do not call apply for the node's old state If a nodes changed its address an old state may be still in a gossiper, so ignore it. --- gms/gossiper.cc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/gms/gossiper.cc b/gms/gossiper.cc index e20b995baaf3..51a6d777f494 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -657,6 +657,10 @@ future<> gossiper::apply_state_locally(std::map ma // If there is no host id in the new state there should be one locally hid = get_host_id(ep); } + if (hid == my_host_id()) { + logger.trace("Ignoring gossip for {} because it maps to local id, but is not local address", ep); + return make_ready_future<>(); + } if (_topo_sm->_topology.left_nodes.contains(raft::server_id(hid.uuid()))) { logger.trace("Ignoring gossip for {} because it left", ep); return make_ready_future<>(); From 336801998241b924b91b52a347fba3d0ddebad8f Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Tue, 17 Dec 2024 16:43:08 +0200 Subject: [PATCH 031/397] gossiper: do not send echo message to yourself When sending by ID we should check that we do not translate our old address to our ID and sending locally. mark_alive should not be called with node's old ip anyway. --- gms/gossiper.cc | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/gms/gossiper.cc b/gms/gossiper.cc index 51a6d777f494..6d83438613fb 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -1709,6 +1709,11 @@ void gossiper::mark_alive(inet_address addr) { }); auto id = get_host_id(addr); + if (id == my_host_id()) { + // We are here because this node changed address and now tries to + // ping an old gossip entry. + return; + } auto generation = my_endpoint_state().get_heart_beat_state().get_generation(); // Enter the _background_msg gate so stop() would wait on it auto gh = _background_msg.hold(); From e318dfb83a03db5dd7691db743034eb84c31e83b Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Tue, 17 Dec 2024 16:55:44 +0200 Subject: [PATCH 032/397] gossiper: do not reset _just_removed_endpoints in non raft mode By the time the function is called during start it may already be populated. Fixes: scylladb/scylladb#21930 --- gms/gossiper.hh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/gms/gossiper.hh b/gms/gossiper.hh index 415edf65ee02..f109d9889ca0 100644 --- a/gms/gossiper.hh +++ b/gms/gossiper.hh @@ -327,8 +327,10 @@ public: void set_topology_state_machine(service::topology_state_machine* m) { _topo_sm = m; - // In raft topology mode the coodinator maintains banned nodes list - _just_removed_endpoints.clear(); + if (m) { + // In raft topology mode the coodinator maintains banned nodes list + _just_removed_endpoints.clear(); + } } private: From f3eade2f6249ef4a964f34da76ec75aaacec26ff Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Wed, 18 Dec 2024 13:00:28 +0200 Subject: [PATCH 033/397] treewide: relicense to ScyllaDB-Source-Available-1.0 Drop the AGPL license in favor of a source-available license. See the blog post [1] for details. [1] https://www.scylladb.com/2024/12/18/why-were-moving-to-a-source-available-license/ --- LICENSE-ScyllaDB-Source-Available.md | 62 ++ LICENSE.AGPL | 661 ---------------- absl-flat_hash_map.cc | 2 +- absl-flat_hash_map.hh | 2 +- alternator/auth.cc | 2 +- alternator/auth.hh | 2 +- alternator/conditions.cc | 2 +- alternator/conditions.hh | 2 +- alternator/consumed_capacity.cc | 2 +- alternator/consumed_capacity.hh | 2 +- alternator/controller.cc | 2 +- alternator/controller.hh | 2 +- alternator/error.hh | 2 +- alternator/executor.cc | 2 +- alternator/executor.hh | 2 +- alternator/expressions.cc | 2 +- alternator/expressions.g | 2 +- alternator/expressions.hh | 2 +- alternator/expressions_types.hh | 2 +- alternator/rmw_operation.hh | 2 +- alternator/serialization.cc | 2 +- alternator/serialization.hh | 2 +- alternator/server.cc | 2 +- alternator/server.hh | 2 +- alternator/stats.cc | 2 +- alternator/stats.hh | 2 +- alternator/streams.cc | 2 +- alternator/ttl.cc | 2 +- alternator/ttl.hh | 2 +- api/api-doc/swagger20_header.json | 4 +- api/api.cc | 2 +- api/api.hh | 2 +- api/api_init.hh | 2 +- api/authorization_cache.cc | 2 +- api/authorization_cache.hh | 2 +- api/cache_service.cc | 2 +- api/cache_service.hh | 2 +- api/collectd.cc | 2 +- api/collectd.hh | 2 +- api/column_family.cc | 2 +- api/column_family.hh | 2 +- api/commitlog.cc | 2 +- api/commitlog.hh | 2 +- api/compaction_manager.cc | 2 +- api/compaction_manager.hh | 2 +- api/config.cc | 2 +- api/config.hh | 2 +- api/cql_server_test.cc | 2 +- api/cql_server_test.hh | 2 +- api/endpoint_snitch.cc | 2 +- api/endpoint_snitch.hh | 2 +- api/error_injection.cc | 2 +- api/error_injection.hh | 2 +- api/failure_detector.cc | 2 +- api/failure_detector.hh | 2 +- api/gossiper.cc | 2 +- api/gossiper.hh | 2 +- api/hinted_handoff.cc | 2 +- api/hinted_handoff.hh | 2 +- api/lsa.cc | 2 +- api/lsa.hh | 2 +- api/messaging_service.cc | 2 +- api/messaging_service.hh | 2 +- api/raft.cc | 2 +- api/raft.hh | 2 +- api/scrub_status.hh | 2 +- api/storage_proxy.cc | 2 +- api/storage_proxy.hh | 2 +- api/storage_service.cc | 2 +- api/storage_service.hh | 2 +- api/stream_manager.cc | 2 +- api/stream_manager.hh | 2 +- api/system.cc | 2 +- api/system.hh | 2 +- api/task_manager.cc | 2 +- api/task_manager.hh | 2 +- api/task_manager_test.cc | 2 +- api/task_manager_test.hh | 2 +- api/tasks.cc | 2 +- api/tasks.hh | 2 +- api/token_metadata.cc | 2 +- api/token_metadata.hh | 2 +- auth/allow_all_authenticator.cc | 2 +- auth/allow_all_authenticator.hh | 2 +- auth/allow_all_authorizer.cc | 2 +- auth/allow_all_authorizer.hh | 2 +- auth/authenticated_user.cc | 2 +- auth/authenticated_user.hh | 2 +- auth/authentication_options.hh | 2 +- auth/authenticator.cc | 2 +- auth/authenticator.hh | 2 +- auth/authorizer.hh | 2 +- auth/certificate_authenticator.cc | 2 +- auth/certificate_authenticator.hh | 2 +- auth/common.cc | 2 +- auth/common.hh | 2 +- auth/default_authorizer.cc | 2 +- auth/default_authorizer.hh | 2 +- auth/maintenance_socket_role_manager.cc | 2 +- auth/maintenance_socket_role_manager.hh | 2 +- auth/password_authenticator.cc | 2 +- auth/password_authenticator.hh | 2 +- auth/passwords.cc | 2 +- auth/passwords.hh | 2 +- auth/permission.cc | 2 +- auth/permission.hh | 2 +- auth/permissions_cache.cc | 2 +- auth/permissions_cache.hh | 2 +- auth/resource.cc | 2 +- auth/resource.hh | 2 +- auth/role_manager.hh | 2 +- auth/role_or_anonymous.cc | 2 +- auth/role_or_anonymous.hh | 2 +- auth/roles-metadata.cc | 2 +- auth/roles-metadata.hh | 2 +- auth/sasl_challenge.cc | 2 +- auth/sasl_challenge.hh | 2 +- auth/service.cc | 2 +- auth/service.hh | 2 +- auth/standard_role_manager.cc | 2 +- auth/standard_role_manager.hh | 2 +- auth/transitional.cc | 2 +- backlog_controller.hh | 2 +- bin/cqlsh | 2 +- bin/nodetool | 2 +- build_mode.hh | 2 +- bytes.cc | 2 +- bytes.hh | 2 +- bytes_fwd.hh | 2 +- bytes_ostream.hh | 2 +- cache_mutation_reader.hh | 2 +- cache_temperature.hh | 2 +- cartesian_product.hh | 2 +- cdc/cdc_extension.hh | 2 +- cdc/cdc_options.hh | 2 +- cdc/cdc_partitioner.cc | 2 +- cdc/cdc_partitioner.hh | 2 +- cdc/change_visitor.hh | 2 +- cdc/generation.cc | 2 +- cdc/generation.hh | 2 +- cdc/generation_id.hh | 2 +- cdc/log.cc | 2 +- cdc/log.hh | 2 +- cdc/metadata.cc | 2 +- cdc/metadata.hh | 2 +- cdc/split.cc | 2 +- cdc/split.hh | 2 +- cdc/stats.hh | 2 +- cell_locking.hh | 2 +- checked-file-impl.hh | 2 +- client_data.cc | 2 +- client_data.hh | 2 +- clocks-impl.cc | 2 +- clocks-impl.hh | 2 +- clustering_bounds_comparator.hh | 2 +- clustering_interval_set.hh | 2 +- clustering_key_filter.hh | 2 +- clustering_ranges_walker.hh | 2 +- cmake/FindANTLR3.cmake | 2 +- cmake/FindSeastar.cmake | 2 +- cmake/Findcryptopp.cmake | 2 +- cmake/Findlibdeflate.cmake | 2 +- cmake/Findlibxcrypt.cmake | 2 +- cmake/Findrapidxml.cmake | 2 +- cmake/FindxxHash.cmake | 2 +- cmake/Findzstd.cmake | 2 +- collection_mutation.cc | 2 +- collection_mutation.hh | 2 +- column_computation.hh | 2 +- combine.hh | 2 +- compaction/compaction.cc | 2 +- compaction/compaction.hh | 2 +- compaction/compaction_backlog_manager.hh | 2 +- compaction/compaction_descriptor.hh | 2 +- compaction/compaction_fwd.hh | 2 +- compaction/compaction_garbage_collector.hh | 2 +- compaction/compaction_manager.cc | 2 +- compaction/compaction_manager.hh | 2 +- compaction/compaction_state.hh | 2 +- compaction/compaction_strategy.cc | 2 +- compaction/compaction_strategy.hh | 2 +- compaction/compaction_strategy_impl.hh | 2 +- compaction/compaction_strategy_state.hh | 2 +- compaction/compaction_strategy_type.hh | 2 +- compaction/compaction_weight_registration.hh | 2 +- compaction/leveled_compaction_strategy.cc | 2 +- compaction/leveled_compaction_strategy.hh | 2 +- compaction/leveled_manifest.hh | 2 +- compaction/size_tiered_backlog_tracker.hh | 2 +- compaction/size_tiered_compaction_strategy.cc | 2 +- compaction/size_tiered_compaction_strategy.hh | 2 +- compaction/strategy_control.hh | 2 +- compaction/table_state.hh | 2 +- compaction/task_manager_module.cc | 2 +- compaction/task_manager_module.hh | 2 +- compaction/time_window_compaction_strategy.cc | 2 +- compaction/time_window_compaction_strategy.hh | 2 +- compound.hh | 2 +- compound_compat.hh | 2 +- compress.cc | 2 +- compress.hh | 2 +- concrete_types.hh | 2 +- configure.py | 2 +- converting_mutation_partition_applier.cc | 2 +- converting_mutation_partition_applier.hh | 2 +- counters.cc | 2 +- counters.hh | 2 +- cql3/assignment_testable.hh | 2 +- cql3/attributes.cc | 2 +- cql3/attributes.hh | 2 +- cql3/authorized_prepared_statements_cache.hh | 2 +- cql3/cf_name.cc | 2 +- cql3/cf_name.hh | 2 +- cql3/column_identifier.cc | 2 +- cql3/column_identifier.hh | 2 +- cql3/column_specification.cc | 2 +- cql3/column_specification.hh | 2 +- cql3/constants.cc | 2 +- cql3/constants.hh | 2 +- cql3/cql3_type.cc | 2 +- cql3/cql3_type.hh | 2 +- cql3/cql_config.hh | 2 +- cql3/cql_statement.hh | 2 +- cql3/description.cc | 2 +- cql3/description.hh | 2 +- cql3/dialect.hh | 2 +- cql3/error_collector.hh | 2 +- cql3/error_listener.hh | 2 +- cql3/expr/evaluate.hh | 2 +- cql3/expr/expr-utils.hh | 2 +- cql3/expr/expression.cc | 2 +- cql3/expr/expression.hh | 2 +- cql3/expr/prepare_expr.cc | 2 +- cql3/expr/restrictions.cc | 2 +- cql3/expr/restrictions.hh | 2 +- cql3/expr/unset.hh | 2 +- cql3/functions/abstract_function.hh | 2 +- cql3/functions/aggregate_fcts.cc | 2 +- cql3/functions/aggregate_fcts.hh | 2 +- cql3/functions/aggregate_function.hh | 2 +- cql3/functions/as_json_function.hh | 2 +- cql3/functions/bytes_conversion_fcts.hh | 2 +- cql3/functions/castas_fcts.cc | 2 +- cql3/functions/castas_fcts.hh | 2 +- cql3/functions/error_injection_fcts.cc | 2 +- cql3/functions/error_injection_fcts.hh | 2 +- cql3/functions/first_function.hh | 2 +- cql3/functions/function.hh | 2 +- cql3/functions/function_name.hh | 2 +- cql3/functions/functions.cc | 2 +- cql3/functions/functions.hh | 2 +- cql3/functions/native_function.hh | 2 +- cql3/functions/native_scalar_function.hh | 2 +- cql3/functions/scalar_function.hh | 2 +- cql3/functions/time_uuid_fcts.hh | 2 +- cql3/functions/token_fct.hh | 2 +- cql3/functions/user_aggregate.hh | 2 +- cql3/functions/user_function.cc | 2 +- cql3/functions/user_function.hh | 2 +- cql3/functions/uuid_fcts.hh | 2 +- cql3/index_name.cc | 2 +- cql3/index_name.hh | 2 +- cql3/keyspace_element_name.cc | 2 +- cql3/keyspace_element_name.hh | 2 +- cql3/lists.cc | 2 +- cql3/lists.hh | 2 +- cql3/maps.cc | 2 +- cql3/maps.hh | 2 +- cql3/operation.cc | 2 +- cql3/operation.hh | 2 +- cql3/operation_impl.hh | 2 +- cql3/prepare_context.cc | 2 +- cql3/prepare_context.hh | 2 +- cql3/prepared_statements_cache.hh | 2 +- cql3/query_options.cc | 2 +- cql3/query_options.hh | 2 +- cql3/query_options_fwd.hh | 2 +- cql3/query_processor.cc | 2 +- cql3/query_processor.hh | 2 +- cql3/restrictions/bounds_slice.hh | 2 +- cql3/restrictions/restrictions_config.hh | 2 +- cql3/restrictions/statement_restrictions.cc | 2 +- cql3/restrictions/statement_restrictions.hh | 2 +- cql3/result_generator.hh | 2 +- cql3/result_set.cc | 2 +- cql3/result_set.hh | 2 +- cql3/role_name.cc | 2 +- cql3/role_name.hh | 2 +- cql3/selection/raw_selector.hh | 2 +- cql3/selection/selectable-expr.hh | 2 +- cql3/selection/selectable.cc | 2 +- cql3/selection/selection.cc | 2 +- cql3/selection/selection.hh | 2 +- cql3/selection/selector.cc | 2 +- cql3/selection/selector.hh | 2 +- cql3/sets.cc | 2 +- cql3/sets.hh | 2 +- cql3/statements/alter_keyspace_statement.cc | 2 +- cql3/statements/alter_keyspace_statement.hh | 2 +- cql3/statements/alter_role_statement.hh | 2 +- .../alter_service_level_statement.cc | 2 +- .../alter_service_level_statement.hh | 2 +- cql3/statements/alter_table_statement.cc | 2 +- cql3/statements/alter_table_statement.hh | 2 +- cql3/statements/alter_type_statement.cc | 2 +- cql3/statements/alter_type_statement.hh | 2 +- cql3/statements/alter_view_statement.cc | 2 +- cql3/statements/alter_view_statement.hh | 2 +- .../attach_service_level_statement.cc | 2 +- .../attach_service_level_statement.hh | 2 +- cql3/statements/authentication_statement.cc | 2 +- cql3/statements/authentication_statement.hh | 2 +- cql3/statements/authorization_statement.cc | 2 +- cql3/statements/authorization_statement.hh | 2 +- cql3/statements/batch_statement.cc | 2 +- cql3/statements/batch_statement.hh | 2 +- cql3/statements/bound.hh | 2 +- cql3/statements/cas_request.cc | 2 +- cql3/statements/cas_request.hh | 2 +- cql3/statements/cf_prop_defs.cc | 2 +- cql3/statements/cf_prop_defs.hh | 2 +- cql3/statements/cf_properties.hh | 2 +- cql3/statements/cf_statement.cc | 2 +- cql3/statements/create_aggregate_statement.cc | 2 +- cql3/statements/create_aggregate_statement.hh | 2 +- cql3/statements/create_function_statement.cc | 2 +- cql3/statements/create_function_statement.hh | 2 +- cql3/statements/create_index_statement.cc | 2 +- cql3/statements/create_index_statement.hh | 2 +- cql3/statements/create_keyspace_statement.cc | 2 +- cql3/statements/create_keyspace_statement.hh | 2 +- cql3/statements/create_role_statement.hh | 2 +- .../create_service_level_statement.cc | 2 +- .../create_service_level_statement.hh | 2 +- cql3/statements/create_table_statement.cc | 2 +- cql3/statements/create_table_statement.hh | 2 +- cql3/statements/create_type_statement.cc | 2 +- cql3/statements/create_type_statement.hh | 2 +- cql3/statements/create_view_statement.cc | 2 +- cql3/statements/create_view_statement.hh | 2 +- cql3/statements/delete_statement.cc | 2 +- cql3/statements/delete_statement.hh | 2 +- cql3/statements/describe_statement.cc | 2 +- cql3/statements/describe_statement.hh | 2 +- .../detach_service_level_statement.cc | 2 +- .../detach_service_level_statement.hh | 2 +- cql3/statements/drop_aggregate_statement.cc | 2 +- cql3/statements/drop_aggregate_statement.hh | 2 +- cql3/statements/drop_function_statement.cc | 2 +- cql3/statements/drop_function_statement.hh | 2 +- cql3/statements/drop_index_statement.cc | 2 +- cql3/statements/drop_index_statement.hh | 2 +- cql3/statements/drop_keyspace_statement.cc | 2 +- cql3/statements/drop_keyspace_statement.hh | 2 +- cql3/statements/drop_role_statement.hh | 2 +- .../drop_service_level_statement.cc | 2 +- .../drop_service_level_statement.hh | 2 +- cql3/statements/drop_table_statement.cc | 2 +- cql3/statements/drop_table_statement.hh | 2 +- cql3/statements/drop_type_statement.cc | 2 +- cql3/statements/drop_type_statement.hh | 2 +- cql3/statements/drop_view_statement.cc | 2 +- cql3/statements/drop_view_statement.hh | 2 +- cql3/statements/function_statement.cc | 2 +- cql3/statements/function_statement.hh | 2 +- cql3/statements/grant_role_statement.hh | 2 +- cql3/statements/grant_statement.cc | 2 +- cql3/statements/grant_statement.hh | 2 +- cql3/statements/index_prop_defs.cc | 2 +- cql3/statements/index_prop_defs.hh | 2 +- cql3/statements/index_target.cc | 2 +- cql3/statements/index_target.hh | 2 +- cql3/statements/ks_prop_defs.cc | 2 +- cql3/statements/ks_prop_defs.hh | 2 +- .../list_effective_service_level_statement.cc | 2 +- .../list_effective_service_level_statement.hh | 2 +- cql3/statements/list_permissions_statement.cc | 2 +- cql3/statements/list_permissions_statement.hh | 2 +- cql3/statements/list_roles_statement.hh | 2 +- ...ist_service_level_attachments_statement.cc | 2 +- ...ist_service_level_attachments_statement.hh | 2 +- .../list_service_level_statement.cc | 2 +- .../list_service_level_statement.hh | 2 +- cql3/statements/list_users_statement.cc | 2 +- cql3/statements/list_users_statement.hh | 2 +- cql3/statements/modification_statement.cc | 2 +- cql3/statements/modification_statement.hh | 2 +- .../permission_altering_statement.cc | 2 +- .../permission_altering_statement.hh | 2 +- cql3/statements/prepared_statement.hh | 2 +- cql3/statements/property_definitions.cc | 2 +- cql3/statements/property_definitions.hh | 2 +- .../prune_materialized_view_statement.cc | 2 +- .../prune_materialized_view_statement.hh | 2 +- cql3/statements/raw/batch_statement.hh | 2 +- cql3/statements/raw/cf_statement.hh | 2 +- cql3/statements/raw/delete_statement.hh | 2 +- cql3/statements/raw/describe_statement.hh | 2 +- cql3/statements/raw/insert_statement.hh | 2 +- cql3/statements/raw/modification_statement.hh | 2 +- cql3/statements/raw/parsed_statement.cc | 2 +- cql3/statements/raw/parsed_statement.hh | 2 +- cql3/statements/raw/select_statement.hh | 2 +- cql3/statements/raw/truncate_statement.hh | 2 +- cql3/statements/raw/update_statement.hh | 2 +- cql3/statements/raw/use_statement.hh | 2 +- cql3/statements/request_validations.hh | 2 +- cql3/statements/revoke_role_statement.hh | 2 +- cql3/statements/revoke_statement.cc | 2 +- cql3/statements/revoke_statement.hh | 2 +- cql3/statements/role-management-statements.cc | 2 +- cql3/statements/schema_altering_statement.cc | 2 +- cql3/statements/schema_altering_statement.hh | 2 +- cql3/statements/select_statement.cc | 2 +- cql3/statements/select_statement.hh | 2 +- cql3/statements/service_level_statement.cc | 2 +- cql3/statements/service_level_statement.hh | 2 +- cql3/statements/sl_prop_defs.cc | 2 +- cql3/statements/sl_prop_defs.hh | 2 +- cql3/statements/statement_type.hh | 2 +- ...ongly_consistent_modification_statement.cc | 2 +- ...ongly_consistent_modification_statement.hh | 2 +- .../strongly_consistent_select_statement.cc | 2 +- .../strongly_consistent_select_statement.hh | 2 +- cql3/statements/truncate_statement.cc | 2 +- cql3/statements/truncate_statement.hh | 2 +- cql3/statements/update_statement.cc | 2 +- cql3/statements/update_statement.hh | 2 +- cql3/statements/use_statement.cc | 2 +- cql3/statements/use_statement.hh | 2 +- cql3/stats.hh | 2 +- cql3/type_json.cc | 2 +- cql3/type_json.hh | 2 +- cql3/untyped_result_set.cc | 2 +- cql3/untyped_result_set.hh | 2 +- cql3/update_parameters.cc | 2 +- cql3/update_parameters.hh | 2 +- cql3/user_types.cc | 2 +- cql3/user_types.hh | 2 +- cql3/ut_name.cc | 2 +- cql3/ut_name.hh | 2 +- cql3/util.cc | 2 +- cql3/util.hh | 2 +- cql3/values.cc | 2 +- cql3/values.hh | 2 +- cql_serialization_format.hh | 2 +- data_dictionary/data_dictionary.cc | 2 +- data_dictionary/data_dictionary.hh | 2 +- data_dictionary/impl.hh | 2 +- data_dictionary/keyspace_metadata.hh | 2 +- data_dictionary/storage_options.hh | 2 +- data_dictionary/user_types_metadata.hh | 2 +- db/auth_version.hh | 2 +- db/batchlog_manager.cc | 2 +- db/batchlog_manager.hh | 2 +- db/cache_tracker.hh | 2 +- db/chained_delegating_reader.hh | 2 +- db/commitlog/commitlog.cc | 2 +- db/commitlog/commitlog.hh | 2 +- db/commitlog/commitlog_entry.cc | 2 +- db/commitlog/commitlog_entry.hh | 2 +- db/commitlog/commitlog_extensions.hh | 2 +- db/commitlog/commitlog_replayer.cc | 2 +- db/commitlog/commitlog_replayer.hh | 2 +- db/commitlog/commitlog_types.hh | 2 +- db/commitlog/replay_position.hh | 2 +- db/commitlog/rp_set.hh | 2 +- db/config.cc | 2 +- db/config.hh | 2 +- db/consistency_level.cc | 2 +- db/consistency_level.hh | 2 +- db/consistency_level_type.hh | 2 +- db/consistency_level_validations.hh | 2 +- db/cql_type_parser.cc | 2 +- db/cql_type_parser.hh | 2 +- db/data_listeners.cc | 2 +- db/data_listeners.hh | 2 +- db/extensions.cc | 2 +- db/extensions.hh | 2 +- db/functions/aggregate_function.hh | 2 +- db/functions/function.cc | 2 +- db/functions/function.hh | 2 +- db/functions/function_name.hh | 2 +- db/functions/scalar_function.hh | 2 +- db/functions/stateless_aggregate_function.hh | 2 +- db/heat_load_balance.cc | 2 +- db/heat_load_balance.hh | 2 +- db/hints/host_filter.cc | 2 +- db/hints/host_filter.hh | 2 +- db/hints/internal/common.hh | 2 +- db/hints/internal/hint_endpoint_manager.cc | 2 +- db/hints/internal/hint_endpoint_manager.hh | 2 +- db/hints/internal/hint_logger.hh | 2 +- db/hints/internal/hint_sender.cc | 2 +- db/hints/internal/hint_sender.hh | 2 +- db/hints/internal/hint_storage.cc | 2 +- db/hints/internal/hint_storage.hh | 2 +- db/hints/manager.cc | 2 +- db/hints/manager.hh | 2 +- db/hints/resource_manager.cc | 2 +- db/hints/resource_manager.hh | 2 +- db/hints/sync_point.cc | 2 +- db/hints/sync_point.hh | 2 +- db/large_data_handler.cc | 2 +- db/large_data_handler.hh | 2 +- db/legacy_schema_migrator.cc | 2 +- db/legacy_schema_migrator.hh | 2 +- db/marshal/type_parser.cc | 2 +- db/marshal/type_parser.hh | 2 +- db/operation_type.hh | 2 +- db/paxos_grace_seconds_extension.hh | 2 +- db/per_partition_rate_limit_extension.hh | 2 +- db/per_partition_rate_limit_info.hh | 2 +- db/per_partition_rate_limit_options.cc | 2 +- db/per_partition_rate_limit_options.hh | 2 +- db/rate_limiter.cc | 2 +- db/rate_limiter.hh | 2 +- db/read_repair_decision.hh | 2 +- db/schema_applier.cc | 2 +- db/schema_applier.hh | 2 +- db/schema_features.hh | 2 +- db/schema_tables.cc | 2 +- db/schema_tables.hh | 2 +- db/size_estimates_virtual_reader.cc | 2 +- db/size_estimates_virtual_reader.hh | 2 +- db/snapshot-ctl.cc | 2 +- db/snapshot-ctl.hh | 2 +- db/snapshot/backup_task.cc | 2 +- db/snapshot/backup_task.hh | 2 +- db/sstables-format-selector.cc | 2 +- db/sstables-format-selector.hh | 2 +- db/system_distributed_keyspace.cc | 2 +- db/system_distributed_keyspace.hh | 2 +- db/system_keyspace.cc | 2 +- db/system_keyspace.hh | 2 +- db/system_keyspace_sstables_registry.hh | 2 +- db/system_keyspace_view_types.hh | 2 +- db/tags/extension.hh | 2 +- db/tags/utils.cc | 2 +- db/tags/utils.hh | 2 +- db/timeout_clock.hh | 2 +- db/view/build_progress_virtual_reader.hh | 2 +- db/view/delete_ghost_rows_visitor.hh | 2 +- db/view/node_view_update_backlog.hh | 2 +- db/view/row_locking.cc | 2 +- db/view/row_locking.hh | 2 +- db/view/view.cc | 2 +- db/view/view.hh | 2 +- db/view/view_builder.hh | 2 +- db/view/view_stats.hh | 2 +- db/view/view_update_backlog.hh | 2 +- db/view/view_update_checks.hh | 2 +- db/view/view_update_generator.cc | 2 +- db/view/view_update_generator.hh | 2 +- db/view/view_updating_consumer.hh | 2 +- db/virtual_table.cc | 2 +- db/virtual_table.hh | 2 +- db/virtual_tables.cc | 2 +- db/virtual_tables.hh | 2 +- db/write_type.hh | 2 +- db_clock.hh | 2 +- debug.cc | 2 +- debug.hh | 2 +- default.nix | 2 +- dht/auto_refreshing_sharder.hh | 2 +- dht/boot_strapper.cc | 2 +- dht/boot_strapper.hh | 2 +- dht/decorated_key.hh | 2 +- dht/i_partitioner.cc | 2 +- dht/i_partitioner.hh | 2 +- dht/i_partitioner_fwd.hh | 2 +- dht/murmur3_partitioner.cc | 2 +- dht/murmur3_partitioner.hh | 2 +- dht/partition_filter.hh | 2 +- dht/range_streamer.cc | 2 +- dht/range_streamer.hh | 2 +- dht/ring_position.hh | 2 +- dht/sharder.hh | 2 +- dht/token-sharding.hh | 2 +- dht/token.cc | 2 +- dht/token.hh | 2 +- dht/token_range_endpoints.hh | 2 +- direct_failure_detector/failure_detector.cc | 2 +- direct_failure_detector/failure_detector.hh | 2 +- .../dep/scylla-env-1.0/debian/copyright | 668 +--------------- dist/common/dep/scylla-env.spec | 2 +- dist/common/kernel_conf/post_install.sh | 2 +- dist/common/kernel_conf/scylla_tune_sched | 2 +- dist/common/nodetool-completion | 2 +- dist/common/scripts/node_health_check | 2 +- dist/common/scripts/scylla-blocktune | 2 +- dist/common/scripts/scylla-housekeeping | 2 +- dist/common/scripts/scylla_blocktune.py | 2 +- dist/common/scripts/scylla_config_get.py | 2 +- dist/common/scripts/scylla_coredump_setup | 2 +- dist/common/scripts/scylla_cpuscaling_setup | 2 +- dist/common/scripts/scylla_cpuset_setup | 2 +- dist/common/scripts/scylla_dev_mode_setup | 2 +- dist/common/scripts/scylla_fstrim | 2 +- dist/common/scripts/scylla_fstrim_setup | 2 +- dist/common/scripts/scylla_io_setup | 2 +- dist/common/scripts/scylla_kernel_check | 2 +- dist/common/scripts/scylla_logrotate | 2 +- dist/common/scripts/scylla_memory_setup | 2 +- dist/common/scripts/scylla_nofile_setup | 2 +- dist/common/scripts/scylla_ntp_setup | 2 +- dist/common/scripts/scylla_prepare | 2 +- dist/common/scripts/scylla_raid_setup | 2 +- dist/common/scripts/scylla_rsyslog_setup | 2 +- dist/common/scripts/scylla_selinux_setup | 2 +- dist/common/scripts/scylla_setup | 2 +- dist/common/scripts/scylla_stop | 2 +- dist/common/scripts/scylla_swap_setup | 2 +- dist/common/scripts/scylla_sysconfig_setup | 2 +- dist/common/scripts/scylla_util.py | 2 +- dist/debian/debian/copyright | 714 +----------------- dist/debian/debian_files_gen.py | 2 +- dist/debuginfo/install.sh | 2 +- .../scripts/create-relocatable-package.py | 2 +- dist/docker/debian/build_docker.sh | 2 +- .../debian/build_offline_installer.sh | 2 +- dist/offline_installer/debian/header | 2 +- .../redhat/build_offline_installer.sh | 2 +- dist/offline_installer/redhat/lib/header | 2 +- dist/redhat/scylla.spec | 2 +- duration.cc | 2 +- duration.hh | 2 +- encoding_stats.hh | 2 +- enum_set.hh | 2 +- exceptions/coordinator_result.hh | 2 +- exceptions/exceptions.cc | 2 +- exceptions/exceptions.hh | 2 +- exceptions/unrecognized_entity_exception.hh | 2 +- fix_system_distributed_tables.py | 2 +- frozen_schema.cc | 2 +- frozen_schema.hh | 2 +- full_position.hh | 2 +- gc_clock.hh | 2 +- gen_segmented_compress_params.py | 4 +- generic_server.cc | 2 +- generic_server.hh | 2 +- gms/application_state.cc | 2 +- gms/application_state.hh | 2 +- gms/endpoint_state.cc | 2 +- gms/endpoint_state.hh | 2 +- gms/feature.hh | 2 +- gms/feature_service.cc | 2 +- gms/feature_service.hh | 2 +- gms/generation-number.cc | 2 +- gms/generation-number.hh | 2 +- gms/gossip_address_map.hh | 2 +- gms/gossip_digest.hh | 2 +- gms/gossip_digest_ack.cc | 2 +- gms/gossip_digest_ack.hh | 2 +- gms/gossip_digest_ack2.cc | 2 +- gms/gossip_digest_ack2.hh | 2 +- gms/gossip_digest_syn.cc | 2 +- gms/gossip_digest_syn.hh | 2 +- gms/gossiper.cc | 2 +- gms/gossiper.hh | 2 +- gms/heart_beat_state.hh | 2 +- gms/i_endpoint_state_change_subscriber.hh | 2 +- gms/inet_address.cc | 2 +- gms/inet_address.hh | 2 +- gms/inet_address_serializer.hh | 2 +- gms/version_generator.cc | 2 +- gms/version_generator.hh | 2 +- gms/versioned_value.cc | 2 +- gms/versioned_value.hh | 2 +- hashing_partition_visitor.hh | 2 +- idl-compiler.py | 4 +- idl/cache_temperature.idl.hh | 2 +- idl/commitlog.idl.hh | 2 +- idl/consistency_level.idl.hh | 2 +- idl/experimental/broadcast_tables_lang.idl.hh | 2 +- idl/frozen_mutation.idl.hh | 2 +- idl/frozen_schema.idl.hh | 2 +- idl/gossip.idl.hh | 2 +- idl/gossip_digest.idl.hh | 2 +- idl/group0.idl.hh | 2 +- idl/group0_state_machine.idl.hh | 2 +- idl/hinted_handoff.idl.hh | 2 +- idl/idl_test.idl.hh | 2 +- idl/join_node.idl.hh | 2 +- idl/keys.idl.hh | 2 +- idl/mapreduce_request.idl.hh | 2 +- idl/messaging_service.idl.hh | 2 +- idl/migration_manager.idl.hh | 2 +- idl/mutation.idl.hh | 2 +- idl/node_ops.idl.hh | 2 +- idl/paging_state.idl.hh | 2 +- idl/paxos.idl.hh | 2 +- idl/per_partition_rate_limit_info.idl.hh | 2 +- idl/position_in_partition.idl.hh | 2 +- idl/query.idl.hh | 2 +- idl/raft.idl.hh | 2 +- idl/raft_storage.idl.hh | 2 +- idl/range.idl.hh | 2 +- idl/read_command.idl.hh | 2 +- idl/reconcilable_result.idl.hh | 2 +- idl/repair.idl.hh | 2 +- idl/replay_position.idl.hh | 2 +- idl/replica_exception.idl.hh | 2 +- idl/result.idl.hh | 2 +- idl/ring_position.idl.hh | 2 +- idl/storage_proxy.idl.hh | 2 +- idl/storage_service.idl.hh | 2 +- idl/streaming.idl.hh | 2 +- idl/token.idl.hh | 2 +- idl/tracing.idl.hh | 2 +- idl/utils.idl.hh | 2 +- idl/uuid.idl.hh | 2 +- idl/view.idl.hh | 2 +- index/built_indexes_virtual_reader.hh | 2 +- index/secondary_index.cc | 2 +- index/secondary_index.hh | 2 +- index/secondary_index_manager.cc | 2 +- index/secondary_index_manager.hh | 2 +- index/target_parser.hh | 2 +- inet_address_vectors.hh | 2 +- init.cc | 2 +- init.hh | 2 +- install.sh | 2 +- interval.hh | 2 +- keys.cc | 2 +- keys.hh | 2 +- lang/lua.cc | 2 +- lang/lua.hh | 2 +- lang/lua_scylla_types.hh | 2 +- lang/manager.cc | 2 +- lang/manager.hh | 2 +- lang/wasm.cc | 2 +- lang/wasm.hh | 2 +- lang/wasm_alien_thread_runner.cc | 2 +- lang/wasm_alien_thread_runner.hh | 2 +- lang/wasm_instance_cache.cc | 2 +- lang/wasm_instance_cache.hh | 2 +- licenses/README.md | 12 +- locator/abstract_replication_strategy.cc | 2 +- locator/abstract_replication_strategy.hh | 2 +- locator/azure_snitch.cc | 2 +- locator/azure_snitch.hh | 2 +- locator/ec2_multi_region_snitch.cc | 2 +- locator/ec2_multi_region_snitch.hh | 2 +- locator/everywhere_replication_strategy.cc | 2 +- locator/everywhere_replication_strategy.hh | 2 +- locator/gce_snitch.cc | 2 +- locator/gce_snitch.hh | 2 +- locator/gossiping_property_file_snitch.cc | 2 +- locator/gossiping_property_file_snitch.hh | 2 +- locator/host_id.hh | 2 +- locator/load_sketch.hh | 2 +- locator/local_strategy.cc | 2 +- locator/local_strategy.hh | 2 +- locator/network_topology_strategy.cc | 2 +- locator/network_topology_strategy.hh | 2 +- locator/production_snitch_base.cc | 2 +- locator/production_snitch_base.hh | 2 +- locator/rack_inferring_snitch.cc | 2 +- locator/rack_inferring_snitch.hh | 2 +- locator/simple_snitch.cc | 2 +- locator/simple_snitch.hh | 2 +- locator/simple_strategy.cc | 2 +- locator/simple_strategy.hh | 2 +- locator/snitch_base.cc | 2 +- locator/snitch_base.hh | 2 +- locator/tablet_metadata_guard.hh | 2 +- locator/tablet_replication_strategy.hh | 2 +- locator/tablet_sharder.hh | 2 +- locator/tablets.cc | 2 +- locator/tablets.hh | 2 +- locator/token_metadata.cc | 2 +- locator/token_metadata.hh | 2 +- locator/token_metadata_fwd.hh | 2 +- locator/token_range_splitter.hh | 2 +- locator/topology.cc | 2 +- locator/topology.hh | 2 +- locator/types.hh | 2 +- locator/util.cc | 2 +- locator/util.hh | 2 +- main.cc | 2 +- map_difference.hh | 2 +- marshal_exception.hh | 2 +- message/messaging_service.cc | 2 +- message/messaging_service.hh | 2 +- message/messaging_service_fwd.hh | 2 +- message/msg_addr.hh | 2 +- message/rpc_protocol_impl.hh | 2 +- multishard_mutation_query.cc | 2 +- multishard_mutation_query.hh | 2 +- mutation/async_utils.cc | 2 +- mutation/async_utils.hh | 2 +- mutation/atomic_cell.cc | 2 +- mutation/atomic_cell.hh | 2 +- mutation/atomic_cell_hash.hh | 2 +- mutation/atomic_cell_or_collection.hh | 2 +- mutation/canonical_mutation.cc | 2 +- mutation/canonical_mutation.hh | 2 +- mutation/compact_and_expire_result.hh | 2 +- mutation/frozen_mutation.cc | 2 +- mutation/frozen_mutation.hh | 2 +- mutation/json.hh | 2 +- mutation/mutation.cc | 2 +- mutation/mutation.hh | 2 +- mutation/mutation_cleaner.hh | 2 +- mutation/mutation_compactor.hh | 2 +- mutation/mutation_consumer.hh | 2 +- mutation/mutation_consumer_concepts.hh | 2 +- mutation/mutation_fragment.cc | 2 +- mutation/mutation_fragment.hh | 2 +- mutation/mutation_fragment_fwd.hh | 2 +- .../mutation_fragment_stream_validator.cc | 2 +- .../mutation_fragment_stream_validator.hh | 2 +- mutation/mutation_fragment_v2.hh | 2 +- mutation/mutation_partition.cc | 2 +- mutation/mutation_partition.hh | 2 +- mutation/mutation_partition_serializer.cc | 2 +- mutation/mutation_partition_serializer.hh | 2 +- mutation/mutation_partition_v2.cc | 2 +- mutation/mutation_partition_v2.hh | 2 +- mutation/mutation_partition_view.cc | 2 +- mutation/mutation_partition_view.hh | 2 +- mutation/mutation_partition_visitor.hh | 2 +- mutation/mutation_rebuilder.hh | 2 +- mutation/mutation_source_metadata.hh | 2 +- mutation/partition_version.cc | 2 +- mutation/partition_version.hh | 2 +- mutation/partition_version_list.hh | 2 +- mutation/position_in_partition.hh | 2 +- mutation/range_tombstone.cc | 2 +- mutation/range_tombstone.hh | 2 +- mutation/range_tombstone_assembler.hh | 2 +- mutation/range_tombstone_change_generator.hh | 2 +- mutation/range_tombstone_list.cc | 2 +- mutation/range_tombstone_list.hh | 2 +- mutation/range_tombstone_splitter.hh | 2 +- mutation/tombstone.hh | 2 +- mutation_query.cc | 2 +- mutation_query.hh | 2 +- mutation_writer/feed_writers.cc | 2 +- mutation_writer/feed_writers.hh | 2 +- mutation_writer/multishard_writer.cc | 2 +- mutation_writer/multishard_writer.hh | 2 +- .../partition_based_splitting_writer.cc | 2 +- .../partition_based_splitting_writer.hh | 2 +- .../shard_based_splitting_writer.cc | 2 +- .../shard_based_splitting_writer.hh | 2 +- .../timestamp_based_splitting_writer.cc | 2 +- .../timestamp_based_splitting_writer.hh | 2 +- .../token_group_based_splitting_writer.cc | 2 +- .../token_group_based_splitting_writer.hh | 2 +- node_ops/id.hh | 2 +- node_ops/node_ops_ctl.cc | 2 +- node_ops/node_ops_ctl.hh | 2 +- node_ops/task_manager_module.cc | 2 +- node_ops/task_manager_module.hh | 2 +- noexcept_traits.hh | 2 +- partition_builder.hh | 2 +- partition_range_compat.hh | 2 +- partition_slice_builder.cc | 2 +- partition_slice_builder.hh | 2 +- partition_snapshot_reader.hh | 2 +- partition_snapshot_row_cursor.hh | 2 +- protocol_server.hh | 2 +- querier.cc | 2 +- querier.hh | 2 +- query-request.hh | 2 +- query-result-reader.hh | 2 +- query-result-set.cc | 2 +- query-result-set.hh | 2 +- query-result-writer.hh | 2 +- query-result.hh | 2 +- query.cc | 2 +- query_id.hh | 2 +- query_ranges_to_vnodes.cc | 2 +- query_ranges_to_vnodes.hh | 2 +- query_result_merger.hh | 2 +- raft/fsm.cc | 2 +- raft/fsm.hh | 2 +- raft/internal.hh | 2 +- raft/log.cc | 2 +- raft/log.hh | 2 +- raft/logical_clock.hh | 2 +- raft/raft.cc | 2 +- raft/raft.hh | 2 +- raft/server.cc | 2 +- raft/server.hh | 2 +- raft/tracker.cc | 2 +- raft/tracker.hh | 2 +- read_context.hh | 2 +- reader_concurrency_semaphore.cc | 2 +- reader_concurrency_semaphore.hh | 2 +- reader_permit.hh | 2 +- readers/clustering_combined.hh | 2 +- readers/combined.cc | 2 +- readers/combined.hh | 2 +- readers/combined_reader_stats.hh | 2 +- readers/compacting.hh | 2 +- readers/delegating_v2.hh | 2 +- readers/empty_v2.hh | 2 +- readers/evictable.hh | 2 +- readers/filtering.hh | 2 +- readers/foreign.hh | 2 +- readers/forwardable_v2.hh | 2 +- readers/from_fragments_v2.hh | 2 +- readers/from_mutations_v2.hh | 2 +- readers/generating_v2.hh | 2 +- readers/multi_range.hh | 2 +- readers/multishard.cc | 2 +- readers/multishard.hh | 2 +- readers/mutation_reader.cc | 2 +- readers/mutation_reader.hh | 2 +- readers/mutation_reader_fwd.hh | 2 +- readers/mutation_readers.cc | 2 +- readers/mutation_source.hh | 2 +- readers/next_partition_adaptor.hh | 2 +- readers/nonforwardable.hh | 2 +- readers/queue.hh | 2 +- readers/range_tombstone_change_merger.hh | 2 +- readers/reversing_v2.hh | 2 +- readers/slicing_filtering.hh | 2 +- readers/upgrading_consumer.hh | 2 +- real_dirty_memory_accounter.hh | 2 +- redis/abstract_command.cc | 2 +- redis/abstract_command.hh | 2 +- redis/command_factory.cc | 2 +- redis/command_factory.hh | 2 +- redis/commands.cc | 2 +- redis/commands.hh | 2 +- redis/controller.cc | 2 +- redis/controller.hh | 2 +- redis/exceptions.hh | 2 +- redis/keyspace_utils.cc | 2 +- redis/keyspace_utils.hh | 2 +- redis/lolwut.cc | 2 +- redis/lolwut.hh | 2 +- redis/mutation_utils.cc | 2 +- redis/mutation_utils.hh | 2 +- redis/options.cc | 2 +- redis/options.hh | 2 +- redis/protocol_parser.rl | 2 +- redis/query_processor.cc | 2 +- redis/query_processor.hh | 2 +- redis/query_utils.cc | 2 +- redis/query_utils.hh | 2 +- redis/reply.hh | 2 +- redis/request.hh | 2 +- redis/server.cc | 2 +- redis/server.hh | 2 +- redis/stats.cc | 2 +- redis/stats.hh | 2 +- redis/version.hh | 2 +- release.cc | 2 +- release.hh | 2 +- repair/decorated_key_with_hash.hh | 2 +- repair/repair.cc | 2 +- repair/repair.hh | 2 +- repair/row.hh | 2 +- repair/row_level.cc | 2 +- repair/row_level.hh | 2 +- repair/sync_boundary.hh | 2 +- repair/table_check.cc | 2 +- repair/table_check.hh | 2 +- repair/task_manager_module.hh | 2 +- replica/compaction_group.hh | 2 +- replica/data_dictionary_impl.hh | 2 +- replica/database.cc | 2 +- replica/database.hh | 2 +- replica/database_fwd.hh | 2 +- replica/dirty_memory_manager.cc | 2 +- replica/dirty_memory_manager.hh | 2 +- replica/distributed_loader.cc | 2 +- replica/distributed_loader.hh | 2 +- replica/exceptions.cc | 2 +- replica/exceptions.hh | 2 +- replica/global_table_ptr.hh | 2 +- replica/memtable-sstable.hh | 2 +- replica/memtable.cc | 2 +- replica/memtable.hh | 2 +- replica/mutation_dump.cc | 2 +- replica/mutation_dump.hh | 2 +- replica/query.hh | 2 +- replica/query_state.hh | 2 +- replica/schema_describe_helper.hh | 2 +- replica/table.cc | 2 +- replica/tablet_mutation_builder.hh | 2 +- replica/tablets.cc | 2 +- replica/tablets.hh | 2 +- reversibly_mergeable.hh | 2 +- row_cache.cc | 2 +- row_cache.hh | 2 +- rust/inc/src/lib.rs | 2 +- rust/src/lib.rs | 2 +- rust/wasmtime_bindings/src/lib.rs | 2 +- rust/wasmtime_bindings/src/memory_creator.rs | 2 +- .../src/test_memory_creator.rs | 2 +- schema/caching_options.cc | 2 +- schema/caching_options.hh | 2 +- schema/schema.cc | 2 +- schema/schema.hh | 2 +- schema/schema_builder.hh | 2 +- schema/schema_fwd.hh | 2 +- schema/schema_registry.cc | 2 +- schema/schema_registry.hh | 2 +- schema_mutations.cc | 2 +- schema_mutations.hh | 2 +- schema_upgrader.hh | 2 +- scripts/base36-uuid.py | 2 +- scripts/coverage.py | 2 +- scripts/cpp-name-format.py | 2 +- scripts/create-relocatable-package.py | 2 +- scripts/jobs | 2 +- scripts/merge-compdb.py | 2 +- scripts/nodetool-toppartitions | 2 +- scripts/open-coredump.sh | 2 +- scripts/refresh-submodules.sh | 2 +- scripts/tablet-mon.py | 2 +- scylla_post_install.sh | 2 +- seastarx.hh | 2 +- serialization_visitors.hh | 2 +- serializer.cc | 2 +- serializer.hh | 2 +- serializer_impl.hh | 2 +- service/address_map.hh | 2 +- service/broadcast_tables/experimental/lang.cc | 2 +- service/broadcast_tables/experimental/lang.hh | 2 +- .../experimental/query_result.hh | 2 +- service/cache_hitrate_calculator.hh | 2 +- service/client_state.cc | 2 +- service/client_state.hh | 2 +- service/endpoint_lifecycle_subscriber.hh | 2 +- service/load_broadcaster.hh | 2 +- service/load_meter.hh | 2 +- service/maintenance_mode.hh | 2 +- service/mapreduce_service.cc | 2 +- service/mapreduce_service.hh | 2 +- service/memory_limiter.hh | 2 +- service/migration_listener.hh | 2 +- service/migration_manager.cc | 2 +- service/migration_manager.hh | 2 +- service/misc_services.cc | 2 +- service/pager/paging_state.cc | 2 +- service/pager/paging_state.hh | 2 +- service/pager/query_pager.hh | 2 +- service/pager/query_pagers.cc | 2 +- service/pager/query_pagers.hh | 2 +- service/paxos/cas_request.hh | 2 +- service/paxos/paxos_state.cc | 2 +- service/paxos/paxos_state.hh | 2 +- service/paxos/prepare_response.cc | 2 +- service/paxos/prepare_response.hh | 2 +- service/paxos/prepare_summary.cc | 2 +- service/paxos/prepare_summary.hh | 2 +- service/paxos/proposal.cc | 2 +- service/paxos/proposal.hh | 2 +- service/qos/qos_common.cc | 2 +- service/qos/qos_common.hh | 2 +- .../qos_configuration_change_subscriber.hh | 2 +- ...service_level_distributed_data_accessor.cc | 2 +- ...service_level_distributed_data_accessor.hh | 2 +- service/qos/service_level_controller.cc | 2 +- service/qos/service_level_controller.hh | 2 +- ...service_level_distributed_data_accessor.cc | 2 +- ...service_level_distributed_data_accessor.hh | 2 +- service/query_state.hh | 2 +- service/raft/discovery.cc | 2 +- service/raft/discovery.hh | 2 +- service/raft/group0_fwd.hh | 2 +- service/raft/group0_state_id_handler.cc | 2 +- service/raft/group0_state_id_handler.hh | 2 +- service/raft/group0_state_machine.cc | 2 +- service/raft/group0_state_machine.hh | 2 +- service/raft/group0_state_machine_merger.cc | 2 +- service/raft/group0_state_machine_merger.hh | 2 +- service/raft/join_node.hh | 2 +- service/raft/raft_group0.cc | 2 +- service/raft/raft_group0.hh | 2 +- service/raft/raft_group0_client.cc | 2 +- service/raft/raft_group0_client.hh | 2 +- service/raft/raft_group_registry.cc | 2 +- service/raft/raft_group_registry.hh | 2 +- service/raft/raft_rpc.cc | 2 +- service/raft/raft_rpc.hh | 2 +- service/raft/raft_state_machine.hh | 2 +- service/raft/raft_sys_table_storage.cc | 2 +- service/raft/raft_sys_table_storage.hh | 2 +- service/raft/raft_timeout.hh | 2 +- service/session.cc | 2 +- service/session.hh | 2 +- service/state_id.hh | 2 +- service/storage_proxy.cc | 2 +- service/storage_proxy.hh | 2 +- service/storage_proxy_stats.hh | 2 +- service/storage_service.cc | 2 +- service/storage_service.hh | 2 +- service/tablet_allocator.cc | 2 +- service/tablet_allocator.hh | 2 +- service/tablet_allocator_fwd.hh | 2 +- service/task_manager_module.cc | 2 +- service/task_manager_module.hh | 2 +- service/topology_coordinator.cc | 2 +- service/topology_coordinator.hh | 2 +- service/topology_guard.hh | 2 +- service/topology_mutation.cc | 2 +- service/topology_mutation.hh | 2 +- service/topology_state_machine.cc | 2 +- service/topology_state_machine.hh | 2 +- service/view_update_backlog_broker.hh | 2 +- service_permit.hh | 2 +- setup.py | 2 +- shell.nix | 2 +- sstables/binary_search.hh | 2 +- sstables/checksum_utils.hh | 2 +- sstables/checksummed_data_source.cc | 2 +- sstables/checksummed_data_source.hh | 2 +- sstables/column_translation.hh | 2 +- sstables/component_type.hh | 2 +- sstables/compress.cc | 2 +- sstables/compress.hh | 2 +- sstables/consumer.hh | 2 +- sstables/data_source_types.hh | 2 +- sstables/disk_types.hh | 2 +- sstables/downsampling.hh | 2 +- sstables/exceptions.hh | 2 +- sstables/file_writer.hh | 2 +- sstables/filter.hh | 2 +- sstables/generation_type.hh | 2 +- sstables/hyperloglog.hh | 2 +- sstables/index_entry.hh | 2 +- sstables/index_reader.hh | 2 +- sstables/integrity_checked_file_impl.cc | 2 +- sstables/integrity_checked_file_impl.hh | 2 +- sstables/key.hh | 2 +- sstables/kl/reader.cc | 2 +- sstables/kl/reader.hh | 2 +- sstables/liveness_info.hh | 2 +- sstables/m_format_read_helpers.cc | 2 +- sstables/m_format_read_helpers.hh | 2 +- sstables/metadata_collector.cc | 2 +- sstables/metadata_collector.hh | 2 +- sstables/mutation_fragment_filter.hh | 2 +- sstables/mx/bsearch_clustered_cursor.hh | 2 +- sstables/mx/parsers.hh | 2 +- .../mx/partition_reversing_data_source.cc | 2 +- .../mx/partition_reversing_data_source.hh | 2 +- sstables/mx/reader.cc | 2 +- sstables/mx/reader.hh | 2 +- sstables/mx/types.hh | 2 +- sstables/mx/writer.cc | 2 +- sstables/mx/writer.hh | 2 +- sstables/open_info.hh | 2 +- sstables/partition_index_cache.hh | 2 +- sstables/partition_index_cache_stats.hh | 2 +- sstables/prepended_input_stream.cc | 2 +- sstables/prepended_input_stream.hh | 2 +- sstables/processing_result_generator.hh | 2 +- sstables/progress_monitor.hh | 2 +- sstables/promoted_index_blocks_reader.hh | 2 +- sstables/random_access_reader.cc | 2 +- sstables/random_access_reader.hh | 2 +- sstables/scanning_clustered_index_cursor.hh | 2 +- sstables/segmented_compress_params.hh | 2 +- sstables/shareable_components.hh | 2 +- sstables/shared_sstable.hh | 2 +- sstables/sstable_directory.cc | 2 +- sstables/sstable_directory.hh | 2 +- sstables/sstable_mutation_reader.cc | 2 +- sstables/sstable_mutation_reader.hh | 2 +- sstables/sstable_set.cc | 2 +- sstables/sstable_set.hh | 2 +- sstables/sstable_set_impl.hh | 2 +- sstables/sstable_version.cc | 2 +- sstables/sstable_version.hh | 2 +- sstables/sstable_version_k_l.hh | 2 +- sstables/sstable_version_m.hh | 2 +- sstables/sstable_writer.hh | 2 +- sstables/sstables.cc | 2 +- sstables/sstables.hh | 2 +- sstables/sstables_manager.cc | 2 +- sstables/sstables_manager.hh | 2 +- sstables/sstables_registry.hh | 2 +- sstables/stats.hh | 2 +- sstables/storage.cc | 2 +- sstables/storage.hh | 2 +- sstables/types.hh | 2 +- sstables/types_fwd.hh | 2 +- sstables/version.hh | 2 +- sstables/writer.cc | 2 +- sstables/writer.hh | 2 +- sstables/writer_impl.hh | 2 +- sstables_loader.cc | 2 +- sstables_loader.hh | 2 +- streaming/consumer.cc | 2 +- streaming/consumer.hh | 2 +- streaming/prepare_message.hh | 2 +- streaming/progress_info.cc | 2 +- streaming/progress_info.hh | 2 +- streaming/session_info.cc | 2 +- streaming/session_info.hh | 2 +- streaming/stream_coordinator.cc | 2 +- streaming/stream_coordinator.hh | 2 +- streaming/stream_detail.hh | 2 +- streaming/stream_event.hh | 2 +- streaming/stream_event_handler.hh | 2 +- streaming/stream_exception.hh | 2 +- streaming/stream_fwd.hh | 2 +- streaming/stream_manager.cc | 2 +- streaming/stream_manager.hh | 2 +- streaming/stream_mutation_fragments_cmd.hh | 2 +- streaming/stream_plan.cc | 2 +- streaming/stream_plan.hh | 2 +- streaming/stream_reason.hh | 2 +- streaming/stream_receive_task.cc | 2 +- streaming/stream_receive_task.hh | 2 +- streaming/stream_request.cc | 2 +- streaming/stream_request.hh | 2 +- streaming/stream_result_future.cc | 2 +- streaming/stream_result_future.hh | 2 +- streaming/stream_session.cc | 2 +- streaming/stream_session.hh | 2 +- streaming/stream_session_state.cc | 2 +- streaming/stream_session_state.hh | 2 +- streaming/stream_state.hh | 2 +- streaming/stream_summary.cc | 2 +- streaming/stream_summary.hh | 2 +- streaming/stream_task.cc | 2 +- streaming/stream_task.hh | 2 +- streaming/stream_transfer_task.cc | 2 +- streaming/stream_transfer_task.hh | 2 +- supervisor.hh | 2 +- table_helper.cc | 2 +- table_helper.hh | 2 +- tasks/task_handler.cc | 2 +- tasks/task_handler.hh | 2 +- tasks/task_manager.cc | 2 +- tasks/task_manager.hh | 2 +- tasks/test_module.hh | 2 +- tasks/types.hh | 2 +- tasks/virtual_task_hint.hh | 2 +- test.py | 2 +- test/alternator/conftest.py | 2 +- test/alternator/test_authorization.py | 2 +- test/alternator/test_backup.py | 2 +- test/alternator/test_batch.py | 2 +- test/alternator/test_condition_expression.py | 2 +- test/alternator/test_cors.py | 2 +- test/alternator/test_cql_rbac.py | 2 +- test/alternator/test_describe_endpoints.py | 2 +- test/alternator/test_describe_table.py | 2 +- test/alternator/test_expected.py | 2 +- test/alternator/test_filter_expression.py | 2 +- test/alternator/test_gsi.py | 2 +- test/alternator/test_gsi_updatetable.py | 2 +- test/alternator/test_health.py | 2 +- test/alternator/test_item.py | 2 +- .../test_key_condition_expression.py | 2 +- test/alternator/test_key_conditions.py | 2 +- test/alternator/test_limits.py | 2 +- test/alternator/test_lsi.py | 2 +- test/alternator/test_manual_requests.py | 2 +- test/alternator/test_metrics.py | 2 +- test/alternator/test_nested.py | 2 +- test/alternator/test_number.py | 2 +- test/alternator/test_projection_expression.py | 2 +- .../alternator/test_provisioned_throughput.py | 2 +- test/alternator/test_query.py | 2 +- test/alternator/test_query_filter.py | 2 +- .../alternator/test_returnconsumedcapacity.py | 2 +- test/alternator/test_returnvalues.py | 2 +- test/alternator/test_scan.py | 2 +- test/alternator/test_scylla.py | 2 +- test/alternator/test_streams.py | 2 +- test/alternator/test_system_tables.py | 2 +- test/alternator/test_table.py | 2 +- test/alternator/test_tag.py | 2 +- test/alternator/test_tracing.py | 2 +- test/alternator/test_ttl.py | 2 +- test/alternator/test_update_expression.py | 2 +- test/alternator/util.py | 2 +- test/auth_cluster/conftest.py | 2 +- test/auth_cluster/test_auth_no_quorum.py | 2 +- .../test_auth_raft_command_split.py | 2 +- test/auth_cluster/test_auth_v2_migration.py | 2 +- test/auth_cluster/test_maintenance_socket.py | 2 +- test/auth_cluster/test_raft_service_levels.py | 2 +- test/boost/UUID_test.cc | 2 +- test/boost/address_map_test.cc | 2 +- test/boost/aggregate_fcts_test.cc | 2 +- test/boost/allocation_strategy_test.cc | 2 +- test/boost/alternator_unit_test.cc | 2 +- test/boost/anchorless_list_test.cc | 2 +- test/boost/auth_passwords_test.cc | 2 +- test/boost/auth_resource_test.cc | 2 +- test/boost/auth_test.cc | 2 +- test/boost/aws_error_injection_test.cc | 2 +- test/boost/aws_errors_test.cc | 2 +- test/boost/batchlog_manager_test.cc | 2 +- test/boost/big_decimal_test.cc | 2 +- test/boost/bloom_filter_test.cc | 2 +- test/boost/bptree_test.cc | 2 +- test/boost/bptree_validation.hh | 2 +- test/boost/broken_sstable_test.cc | 2 +- test/boost/btree_test.cc | 2 +- test/boost/btree_validation.hh | 2 +- test/boost/bytes_ostream_test.cc | 2 +- test/boost/cache_algorithm_test.cc | 2 +- test/boost/cache_mutation_reader_test.cc | 2 +- test/boost/cached_file_test.cc | 2 +- test/boost/caching_options_test.cc | 2 +- test/boost/canonical_mutation_test.cc | 2 +- test/boost/cartesian_product_test.cc | 2 +- test/boost/castas_fcts_test.cc | 2 +- test/boost/cdc_generation_test.cc | 2 +- test/boost/cdc_test.cc | 2 +- test/boost/cell_locker_test.cc | 2 +- test/boost/checksum_utils_test.cc | 2 +- test/boost/chunked_managed_vector_test.cc | 2 +- test/boost/chunked_vector_test.cc | 2 +- test/boost/clustering_ranges_walker_test.cc | 2 +- test/boost/collection_stress.hh | 2 +- test/boost/column_mapping_test.cc | 2 +- test/boost/commitlog_cleanup_test.cc | 2 +- test/boost/commitlog_test.cc | 2 +- test/boost/compaction_group_test.cc | 2 +- test/boost/compound_test.cc | 2 +- test/boost/compress_test.cc | 2 +- test/boost/config_test.cc | 2 +- test/boost/continuous_data_consumer_test.cc | 2 +- test/boost/counter_test.cc | 2 +- test/boost/cql_auth_query_test.cc | 2 +- test/boost/cql_auth_syntax_test.cc | 2 +- test/boost/cql_functions_test.cc | 2 +- test/boost/cql_query_group_test.cc | 2 +- test/boost/cql_query_large_test.cc | 2 +- test/boost/cql_query_like_test.cc | 2 +- test/boost/cql_query_test.cc | 2 +- test/boost/crc_test.cc | 2 +- test/boost/data_listeners_test.cc | 2 +- test/boost/database_test.cc | 2 +- test/boost/dirty_memory_manager_test.cc | 2 +- test/boost/double_decker_test.cc | 2 +- test/boost/duration_test.cc | 2 +- test/boost/dynamic_bitset_test.cc | 2 +- test/boost/enum_option_test.cc | 2 +- test/boost/enum_set_test.cc | 2 +- test/boost/error_injection_test.cc | 2 +- test/boost/estimated_histogram_test.cc | 2 +- test/boost/exception_container_test.cc | 2 +- test/boost/exceptions_fallback_test.cc | 2 +- test/boost/exceptions_optimized_test.cc | 2 +- test/boost/exceptions_test.inc.cc | 2 +- test/boost/expr_test.cc | 2 +- test/boost/extensions_test.cc | 2 +- test/boost/filtering_test.cc | 2 +- test/boost/flush_queue_test.cc | 2 +- .../boost/fragmented_temporary_buffer_test.cc | 2 +- test/boost/frozen_mutation_test.cc | 2 +- test/boost/generic_server_test.cc | 2 +- .../gossiping_property_file_snitch_test.cc | 2 +- test/boost/group0_cmd_merge_test.cc | 2 +- test/boost/group0_test.cc | 2 +- test/boost/hash_test.cc | 2 +- test/boost/hashers_test.cc | 2 +- test/boost/hint_test.cc | 2 +- test/boost/idl_test.cc | 2 +- test/boost/index_reader_test.cc | 2 +- test/boost/index_with_paging_test.cc | 2 +- test/boost/input_stream_test.cc | 2 +- test/boost/intrusive_array_test.cc | 2 +- test/boost/json_cql_query_test.cc | 2 +- test/boost/json_test.cc | 2 +- test/boost/keys_test.cc | 2 +- test/boost/large_paging_state_test.cc | 2 +- test/boost/like_matcher_test.cc | 2 +- test/boost/limiting_data_source_test.cc | 2 +- test/boost/linearizing_input_stream_test.cc | 2 +- test/boost/lister_test.cc | 2 +- test/boost/loading_cache_test.cc | 2 +- test/boost/locator_topology_test.cc | 2 +- test/boost/log_heap_test.cc | 2 +- test/boost/logalloc_test.cc | 2 +- test/boost/managed_bytes_test.cc | 2 +- test/boost/managed_vector_test.cc | 2 +- test/boost/map_difference_test.cc | 2 +- test/boost/memtable_test.cc | 2 +- ...ombining_reader_as_mutation_source_test.cc | 2 +- test/boost/multishard_mutation_query_test.cc | 2 +- test/boost/murmur_hash_test.cc | 2 +- test/boost/mutation_fragment_test.cc | 2 +- test/boost/mutation_query_test.cc | 2 +- test/boost/mutation_reader_another_test.cc | 2 +- test/boost/mutation_reader_test.cc | 2 +- test/boost/mutation_test.cc | 2 +- test/boost/mutation_writer_test.cc | 2 +- test/boost/mvcc_test.cc | 2 +- test/boost/network_topology_strategy_test.cc | 2 +- test/boost/nonwrapping_interval_test.cc | 2 +- test/boost/observable_test.cc | 2 +- test/boost/partitioner_test.cc | 2 +- test/boost/pretty_printers_test.cc | 2 +- test/boost/querier_cache_test.cc | 2 +- test/boost/query_processor_test.cc | 2 +- test/boost/radix_tree_printer.hh | 2 +- test/boost/radix_tree_test.cc | 2 +- test/boost/range_assert.hh | 2 +- test/boost/range_tombstone_list_assertions.hh | 2 +- test/boost/range_tombstone_list_test.cc | 2 +- test/boost/rate_limiter_test.cc | 2 +- .../reader_concurrency_semaphore_test.cc | 2 +- test/boost/recent_entries_map_test.cc | 2 +- test/boost/repair_test.cc | 2 +- test/boost/restrictions_test.cc | 2 +- test/boost/result_utils_test.cc | 2 +- test/boost/reusable_buffer_test.cc | 2 +- test/boost/role_manager_test.cc | 2 +- test/boost/row_cache_test.cc | 2 +- test/boost/rust_test.cc | 2 +- test/boost/s3_test.cc | 2 +- test/boost/schema_change_test.cc | 2 +- test/boost/schema_changes_test.cc | 2 +- test/boost/schema_loader_test.cc | 2 +- test/boost/schema_registry_test.cc | 2 +- test/boost/secondary_index_test.cc | 2 +- test/boost/serialization_test.cc | 2 +- test/boost/serialized_action_test.cc | 2 +- test/boost/service_level_controller_test.cc | 2 +- test/boost/sessions_test.cc | 2 +- test/boost/small_vector_test.cc | 2 +- test/boost/snitch_reset_test.cc | 2 +- test/boost/sorting_test.cc | 2 +- test/boost/sstable_3_x_test.cc | 2 +- test/boost/sstable_compaction_test.cc | 2 +- ...stable_conforms_to_mutation_source_test.cc | 2 +- test/boost/sstable_datafile_test.cc | 2 +- test/boost/sstable_directory_test.cc | 2 +- test/boost/sstable_generation_test.cc | 2 +- test/boost/sstable_move_test.cc | 2 +- test/boost/sstable_mutation_test.cc | 2 +- .../sstable_partition_index_cache_test.cc | 2 +- test/boost/sstable_set_test.cc | 2 +- test/boost/sstable_test.cc | 2 +- test/boost/sstable_test.hh | 2 +- test/boost/stall_free_test.cc | 2 +- test/boost/statement_restrictions_test.cc | 2 +- test/boost/storage_proxy_test.cc | 2 +- test/boost/string_format_test.cc | 2 +- test/boost/summary_test.cc | 2 +- test/boost/tablets_test.cc | 2 +- test/boost/tagged_integer_test.cc | 2 +- test/boost/token_metadata_test.cc | 2 +- test/boost/top_k_test.cc | 2 +- test/boost/total_order_check.hh | 2 +- test/boost/tracing_test.cc | 2 +- test/boost/transport_test.cc | 2 +- test/boost/tree_test_key.hh | 2 +- test/boost/types_test.cc | 2 +- test/boost/user_function_test.cc | 2 +- test/boost/user_types_test.cc | 2 +- test/boost/utf8_test.cc | 2 +- test/boost/view_build_test.cc | 2 +- test/boost/view_complex_test.cc | 2 +- test/boost/view_schema_ckey_test.cc | 2 +- test/boost/view_schema_pkey_test.cc | 2 +- test/boost/view_schema_test.cc | 2 +- test/boost/vint_serialization_test.cc | 2 +- test/boost/virtual_reader_test.cc | 2 +- .../virtual_table_mutation_source_test.cc | 2 +- test/boost/virtual_table_test.cc | 2 +- test/boost/wasm_alloc_test.cc | 2 +- test/boost/wasm_test.cc | 2 +- test/boost/wrapping_interval_test.cc | 2 +- test/broadcast_tables/conftest.py | 2 +- .../broadcast_tables/test_broadcast_tables.py | 2 +- test/cql/cassandra_batch_test.cql | 2 +- test/cql/cassandra_batch_test.result | 2 +- test/cql/cassandra_cql_test.cql | 2 +- test/cql/cassandra_cql_test.result | 2 +- test/cqlpy/cassandra_tests/porting.py | 2 +- test/cqlpy/conftest.py | 2 +- test/cqlpy/fetch_scylla.py | 2 +- test/cqlpy/nodetool.py | 2 +- test/cqlpy/rest_api.py | 2 +- test/cqlpy/test_aggregate.py | 2 +- test/cqlpy/test_allow_filtering.py | 2 +- test/cqlpy/test_alter_table.py | 2 +- test/cqlpy/test_bad_grammar.py | 2 +- test/cqlpy/test_batch.py | 2 +- test/cqlpy/test_bloom_filter.py | 2 +- test/cqlpy/test_cast.py | 2 +- test/cqlpy/test_cast_data.py | 2 +- test/cqlpy/test_cdc.py | 2 +- test/cqlpy/test_clustering_order.py | 2 +- test/cqlpy/test_compaction.py | 2 +- .../test_compaction_strategy_validation.py | 2 +- test/cqlpy/test_counter.py | 2 +- test/cqlpy/test_describe.py | 2 +- test/cqlpy/test_distinct.py | 2 +- test/cqlpy/test_empty.py | 2 +- test/cqlpy/test_filtering.py | 2 +- test/cqlpy/test_frozen_collection.py | 2 +- test/cqlpy/test_group_by.py | 2 +- test/cqlpy/test_json.py | 2 +- test/cqlpy/test_key_length.py | 2 +- test/cqlpy/test_keyspace.py | 2 +- test/cqlpy/test_large_cells_rows.py | 2 +- test/cqlpy/test_limit.py | 2 +- test/cqlpy/test_logs.py | 2 +- test/cqlpy/test_lwt.py | 2 +- test/cqlpy/test_materialized_view.py | 2 +- test/cqlpy/test_native_functions.py | 2 +- test/cqlpy/test_native_transport.py | 2 +- .../cqlpy/test_non_deterministic_functions.py | 2 +- test/cqlpy/test_null.py | 2 +- test/cqlpy/test_paging.py | 2 +- test/cqlpy/test_permissions.py | 2 +- test/cqlpy/test_prepare.py | 2 +- test/cqlpy/test_range_and_slice.py | 2 +- test/cqlpy/test_restrictions.py | 2 +- test/cqlpy/test_scan.py | 2 +- test/cqlpy/test_secondary_index.py | 2 +- .../test_select_from_mutation_fragments.py | 2 +- test/cqlpy/test_service_levels.py | 2 +- test/cqlpy/test_shedding.py | 2 +- test/cqlpy/test_ssl.py | 2 +- test/cqlpy/test_sstable.py | 2 +- test/cqlpy/test_sstable_compression.py | 2 +- test/cqlpy/test_sstable_validation.py | 2 +- test/cqlpy/test_static.py | 2 +- test/cqlpy/test_system_tables.py | 2 +- test/cqlpy/test_tablets.py | 2 +- test/cqlpy/test_tombstone_limit.py | 2 +- test/cqlpy/test_tools.py | 2 +- test/cqlpy/test_ttl.py | 2 +- test/cqlpy/test_type_date.py | 2 +- test/cqlpy/test_type_decimal.py | 2 +- test/cqlpy/test_type_duration.py | 2 +- test/cqlpy/test_type_string.py | 2 +- test/cqlpy/test_type_time.py | 2 +- test/cqlpy/test_type_timestamp.py | 2 +- test/cqlpy/test_type_uuid.py | 2 +- test/cqlpy/test_uda.py | 2 +- test/cqlpy/test_udf.py | 2 +- test/cqlpy/test_unset.py | 2 +- test/cqlpy/test_use.py | 2 +- test/cqlpy/test_using_service_level.py | 2 +- test/cqlpy/test_using_timeout.py | 2 +- test/cqlpy/test_using_timestamp.py | 2 +- test/cqlpy/test_utf8.py | 2 +- test/cqlpy/test_validation.py | 2 +- test/cqlpy/test_virtual_tables.py | 2 +- test/cqlpy/test_wasm.py | 2 +- test/cqlpy/util.py | 2 +- test/lib/alternator_test_env.cc | 2 +- test/lib/alternator_test_env.hh | 2 +- test/lib/cql_assertions.cc | 2 +- test/lib/cql_assertions.hh | 2 +- test/lib/cql_test_env.cc | 2 +- test/lib/cql_test_env.hh | 2 +- test/lib/data_model.cc | 2 +- test/lib/data_model.hh | 2 +- test/lib/dummy_sharder.cc | 2 +- test/lib/dummy_sharder.hh | 2 +- test/lib/eventually.hh | 2 +- test/lib/exception_utils.cc | 2 +- test/lib/exception_utils.hh | 2 +- test/lib/expr_test_utils.cc | 2 +- test/lib/expr_test_utils.hh | 2 +- .../failure_injecting_allocation_strategy.hh | 2 +- test/lib/fragment_scatterer.hh | 2 +- test/lib/index_reader_assertions.hh | 2 +- test/lib/key_utils.cc | 2 +- test/lib/key_utils.hh | 2 +- test/lib/log.cc | 2 +- test/lib/log.hh | 2 +- test/lib/make_random_string.hh | 2 +- test/lib/memtable_snapshot_source.hh | 2 +- test/lib/mutation_assertions.hh | 2 +- test/lib/mutation_reader_assertions.hh | 2 +- test/lib/mutation_source_test.cc | 2 +- test/lib/mutation_source_test.hh | 2 +- test/lib/random_schema.cc | 2 +- test/lib/random_schema.hh | 2 +- test/lib/random_utils.hh | 2 +- test/lib/reader_concurrency_semaphore.hh | 2 +- test/lib/reader_lifecycle_policy.hh | 2 +- test/lib/result_set_assertions.cc | 2 +- test/lib/result_set_assertions.hh | 2 +- test/lib/scylla_test_case.hh | 2 +- test/lib/scylla_tests_cmdline_options.hh | 2 +- test/lib/select_statement_utils.hh | 2 +- test/lib/simple_position_reader_queue.hh | 2 +- test/lib/simple_schema.hh | 2 +- ...run_based_compaction_strategy_for_tests.cc | 2 +- ...run_based_compaction_strategy_for_tests.hh | 2 +- test/lib/sstable_test_env.hh | 2 +- test/lib/sstable_utils.cc | 2 +- test/lib/sstable_utils.hh | 2 +- test/lib/test_services.cc | 2 +- test/lib/test_utils.cc | 2 +- test/lib/test_utils.hh | 2 +- test/lib/tmpdir.cc | 2 +- test/lib/tmpdir.hh | 2 +- test/manual/ec2_snitch_test.cc | 2 +- test/manual/enormous_table_scan_test.cc | 2 +- test/manual/gce_snitch_test.cc | 2 +- test/manual/gossip.cc | 2 +- test/manual/hint_test.cc | 2 +- test/manual/message.cc | 2 +- test/manual/partition_data_test.cc | 2 +- test/manual/row_locker_test.cc | 2 +- test/manual/sstable_scan_footprint_test.cc | 2 +- test/manual/streaming_histogram_test.cc | 2 +- test/nodetool/conftest.py | 2 +- test/nodetool/rest_api_mock.py | 2 +- test/nodetool/test_autocompaction.py | 2 +- test/nodetool/test_backup.py | 2 +- test/nodetool/test_binary.py | 2 +- .../test_check_and_repair_cdc_streams.py | 2 +- test/nodetool/test_cleanup.py | 2 +- test/nodetool/test_compact.py | 2 +- test/nodetool/test_compactionhistory.py | 2 +- test/nodetool/test_compactionstats.py | 2 +- test/nodetool/test_describecluster.py | 2 +- test/nodetool/test_describering.py | 2 +- test/nodetool/test_drain.py | 2 +- test/nodetool/test_flush.py | 2 +- test/nodetool/test_getendpoints.py | 2 +- test/nodetool/test_gossip.py | 2 +- test/nodetool/test_gossipinfo.py | 2 +- test/nodetool/test_help.py | 2 +- test/nodetool/test_info.py | 2 +- test/nodetool/test_logging.py | 2 +- test/nodetool/test_netstats.py | 2 +- test/nodetool/test_nodeops.py | 2 +- test/nodetool/test_nodetool.py | 2 +- test/nodetool/test_proxyhistograms.py | 2 +- test/nodetool/test_rebuild.py | 2 +- test/nodetool/test_refresh.py | 2 +- test/nodetool/test_repair.py | 2 +- test/nodetool/test_resetlocalschema.py | 2 +- test/nodetool/test_restore.py | 2 +- test/nodetool/test_ring.py | 2 +- test/nodetool/test_scrub.py | 2 +- test/nodetool/test_snapshot.py | 2 +- test/nodetool/test_sstable.py | 2 +- test/nodetool/test_status.py | 2 +- test/nodetool/test_stop.py | 2 +- test/nodetool/test_tablehistograms.py | 2 +- test/nodetool/test_tablestats.py | 2 +- test/nodetool/test_tasks.py | 2 +- test/nodetool/test_toppartitions.py | 2 +- test/nodetool/test_traceprobability.py | 2 +- test/nodetool/test_upgradesstables.py | 2 +- test/nodetool/test_version.py | 2 +- test/nodetool/test_viewbuildstatus.py | 2 +- test/nodetool/utils.py | 2 +- test/perf/entry_point.hh | 2 +- test/perf/logalloc.cc | 2 +- test/perf/memory_footprint_test.cc | 2 +- test/perf/perf.cc | 2 +- test/perf/perf.hh | 2 +- test/perf/perf_alternator.cc | 2 +- test/perf/perf_big_decimal.cc | 2 +- test/perf/perf_cache_eviction.cc | 2 +- test/perf/perf_checksum.cc | 2 +- test/perf/perf_collection.cc | 2 +- test/perf/perf_commitlog.cc | 2 +- test/perf/perf_cql_parser.cc | 2 +- test/perf/perf_fast_forward.cc | 2 +- test/perf/perf_fast_forward_report.py | 2 +- test/perf/perf_hash.cc | 2 +- test/perf/perf_idl.cc | 2 +- test/perf/perf_mutation.cc | 2 +- test/perf/perf_mutation_fragment.cc | 2 +- test/perf/perf_mutation_readers.cc | 2 +- test/perf/perf_row_cache_reads.cc | 2 +- test/perf/perf_row_cache_update.cc | 2 +- test/perf/perf_s3_client.cc | 2 +- test/perf/perf_simple_query.cc | 2 +- test/perf/perf_sstable.cc | 2 +- test/perf/perf_sstable.hh | 2 +- test/perf/perf_tablets.cc | 2 +- test/perf/perf_vint.cc | 2 +- test/perf/tablet_load_balancing.cc | 2 +- test/pylib/artifact_registry.py | 2 +- test/pylib/async_cql.py | 2 +- test/pylib/coverage_utils.py | 2 +- test/pylib/cql_repl/conftest.py | 2 +- test/pylib/cql_repl/cql_repl.py | 2 +- test/pylib/db/model.py | 2 +- test/pylib/db/writer.py | 2 +- test/pylib/host_registry.py | 2 +- test/pylib/internal_types.py | 2 +- test/pylib/lcov_utils.py | 2 +- test/pylib/log_browsing.py | 2 +- test/pylib/manager_client.py | 2 +- test/pylib/minio_server.py | 2 +- test/pylib/random_tables.py | 2 +- test/pylib/repair.py | 2 +- test/pylib/report_plugin.py | 2 +- test/pylib/resource_gather.py | 2 +- test/pylib/rest_client.py | 2 +- test/pylib/s3_proxy.py | 2 +- test/pylib/s3_server_mock.py | 2 +- test/pylib/scylla_cluster.py | 2 +- test/pylib/tablets.py | 2 +- test/pylib/util.py | 2 +- test/raft/discovery_test.cc | 2 +- test/raft/etcd_test.cc | 2 +- test/raft/failure_detector_test.cc | 2 +- test/raft/fsm_test.cc | 2 +- test/raft/future_set.hh | 2 +- test/raft/generator.hh | 2 +- test/raft/helpers.cc | 2 +- test/raft/helpers.hh | 2 +- test/raft/logical_timer.hh | 2 +- test/raft/many_test.cc | 2 +- test/raft/raft_sys_table_storage_test.cc | 2 +- test/raft/randomized_nemesis_test.cc | 2 +- test/raft/replication.cc | 2 +- test/raft/replication.hh | 2 +- test/raft/replication_test.cc | 2 +- test/raft/ticker.hh | 2 +- test/redis/conftest.py | 2 +- test/redis/test_hashes.py | 2 +- test/redis/test_raw_cmd.py | 2 +- test/redis/test_strings.py | 2 +- test/redis/util.py | 2 +- test/rest_api/conftest.py | 2 +- test/rest_api/test_column_family.py | 2 +- test/rest_api/test_compaction_manager.py | 2 +- test/rest_api/test_compactionhistory.py | 2 +- test/rest_api/test_failure_detector.py | 2 +- test/rest_api/test_gossiper.py | 2 +- test/rest_api/test_storage_service.py | 2 +- test/rest_api/test_system.py | 2 +- test/scylla_gdb/conftest.py | 2 +- test/topology/conftest.py | 2 +- test/topology/test_automatic_cleanup.py | 2 +- test/topology/test_change_ip.py | 2 +- test/topology/test_cluster_features.py | 2 +- test/topology/test_concurrent_schema.py | 2 +- .../test_coordinator_queue_management.py | 2 +- test/topology/test_global_ignore_nodes.py | 2 +- test/topology/test_gossiper.py | 2 +- test/topology/test_mutation_schema_change.py | 2 +- test/topology/test_mv.py | 2 +- test/topology/test_random_tables.py | 2 +- test/topology/test_replace_alive_node.py | 2 +- test/topology/test_snapshot.py | 2 +- ...st_start_bootstrapped_with_invalid_seed.py | 2 +- test/topology/test_tls.py | 2 +- .../test_topology_failure_recovery.py | 2 +- test/topology/test_topology_rejoin.py | 2 +- test/topology/test_topology_remove_decom.py | 2 +- test/topology/test_topology_schema.py | 2 +- test/topology/util.py | 2 +- test/topology_custom/conftest.py | 2 +- .../test_boot_after_ip_change.py | 2 +- .../test_change_replication_factor_1_to_0.py | 2 +- .../test_change_rpc_address.py | 2 +- test/topology_custom/test_commitlog.py | 2 +- ...est_commitlog_segment_data_resurrection.py | 2 +- .../test_compacting_reader_tombstone_gc.py | 2 +- .../test_data_resurrection_after_cleanup.py | 2 +- test/topology_custom/test_decommission.py | 2 +- .../test_deprecating_cluster_features.py | 2 +- .../test_different_group0_ids.py | 2 +- .../test_gossiper_orphan_remover.py | 2 +- .../test_group0_schema_versioning.py | 2 +- test/topology_custom/test_hints.py | 2 +- test/topology_custom/test_initial_token.py | 2 +- test/topology_custom/test_ip_mappings.py | 2 +- test/topology_custom/test_long_join.py | 2 +- test/topology_custom/test_lwt_semaphore.py | 2 +- test/topology_custom/test_maintenance_mode.py | 2 +- test/topology_custom/test_major_compaction.py | 2 +- test/topology_custom/test_multidc.py | 2 +- .../test_mv_admission_control.py | 2 +- test/topology_custom/test_mv_backlog.py | 2 +- test/topology_custom/test_mv_building.py | 2 +- .../test_mv_delete_partitions.py | 2 +- test/topology_custom/test_mv_fail_building.py | 2 +- .../test_mv_read_concurrency.py | 2 +- .../test_mv_tablets_empty_ip.py | 2 +- .../test_mv_topology_change.py | 2 +- ...test_no_removed_node_event_on_ip_change.py | 2 +- test/topology_custom/test_node_ops_metrics.py | 2 +- .../test_old_ip_notification_repro.py | 2 +- test/topology_custom/test_query_rebounce.py | 2 +- .../test_raft_fix_broken_snapshot.py | 2 +- test/topology_custom/test_raft_no_quorum.py | 2 +- .../test_raft_recovery_basic.py | 2 +- .../test_raft_recovery_majority_loss.py | 2 +- .../test_raft_recovery_stuck.py | 2 +- .../test_raft_snapshot_request.py | 2 +- .../test_raft_snapshot_truncation.py | 2 +- test/topology_custom/test_read_repair.py | 2 +- .../topology_custom/test_remove_alive_node.py | 2 +- ...remove_rpc_client_with_pending_requests.py | 2 +- test/topology_custom/test_repair.py | 2 +- test/topology_custom/test_replace.py | 2 +- .../test_replace_ignore_nodes.py | 2 +- .../test_replace_with_encryption.py | 2 +- .../test_replace_with_same_ip_twice.py | 2 +- ...ueries_during_simulated_upgrade_process.py | 2 +- .../test_select_from_mutation_fragments.py | 2 +- test/topology_custom/test_shutdown_hang.py | 2 +- test/topology_custom/test_snitch_change.py | 2 +- .../test_table_desc_read_barrier.py | 2 +- test/topology_custom/test_tablets.py | 2 +- test/topology_custom/test_tablets_cql.py | 2 +- .../topology_custom/test_tablets_migration.py | 2 +- .../test_topology_failure_recovery.py | 2 +- .../test_topology_remove_garbage_group0.py | 2 +- test/topology_custom/test_topology_smp.py | 2 +- .../test_truncate_with_tablets.py | 2 +- .../topology_custom/test_view_build_status.py | 2 +- ...test_writes_to_previous_cdc_generations.py | 2 +- .../test_zero_token_nodes_topology_ops.py | 2 +- test/topology_experimental_raft/conftest.py | 2 +- .../test_alternator.py | 2 +- .../test_blocked_bootstrap.py | 2 +- .../test_cdc_generation_clearing.py | 2 +- .../test_cdc_generation_publishing.py | 2 +- ...test_crash_coordinator_before_streaming.py | 2 +- .../test_fencing.py | 2 +- .../test_mv_tablets.py | 2 +- .../test_mv_tablets_replace.py | 2 +- .../test_node_isolation.py | 2 +- .../test_not_enough_token_owners.py | 2 +- .../test_raft_cluster_features.py | 2 +- .../test_raft_ignore_nodes.py | 2 +- .../test_restart_cluster.py | 2 +- .../test_tablet_repair_scheduler.py | 2 +- .../test_tablets.py | 2 +- .../test_tablets_intranode.py | 2 +- .../test_tablets_merge.py | 2 +- .../test_tablets_removenode.py | 2 +- .../test_tombstone_gc.py | 2 +- .../test_topology_ops.py | 2 +- .../test_topology_recovery_basic.py | 2 +- .../test_topology_recovery_majority_loss.py | 2 +- .../test_topology_upgrade.py | 2 +- ..._upgrade_not_stuck_after_recent_removal.py | 2 +- .../test_zero_token_nodes_multidc.py | 2 +- .../test_zero_token_nodes_no_replication.py | 2 +- .../cluster_events.py | 2 +- test/topology_random_failures/conftest.py | 2 +- .../error_injections.py | 2 +- .../test_random_failures.py | 2 +- test/topology_tasks/conftest.py | 2 +- test/topology_tasks/task_manager_client.py | 2 +- test/topology_tasks/task_manager_types.py | 2 +- test/topology_tasks/test_node_ops_tasks.py | 2 +- test/topology_tasks/test_tablet_tasks.py | 2 +- test/unit/cross_shard_barrier_test.cc | 2 +- test/unit/lsa_async_eviction_test.cc | 2 +- test/unit/lsa_sync_eviction_test.cc | 2 +- test/unit/row_cache_alloc_stress_test.cc | 2 +- test/unit/row_cache_stress_test.cc | 2 +- timeout_config.cc | 2 +- timeout_config.hh | 2 +- timestamp.hh | 2 +- tombstone_gc-internals.hh | 2 +- tombstone_gc.cc | 2 +- tombstone_gc.hh | 2 +- tombstone_gc_extension.hh | 2 +- tombstone_gc_options.cc | 2 +- tombstone_gc_options.hh | 2 +- tools/build_info.hh | 2 +- tools/entry_point.hh | 2 +- tools/format_printers.hh | 2 +- tools/json_writer.hh | 2 +- tools/load_system_tablets.cc | 2 +- tools/load_system_tablets.hh | 2 +- tools/lua_sstable_consumer.cc | 2 +- tools/lua_sstable_consumer.hh | 2 +- tools/read_mutation.cc | 2 +- tools/read_mutation.hh | 2 +- tools/schema_loader.cc | 2 +- tools/schema_loader.hh | 2 +- tools/scylla-nodetool.cc | 2 +- tools/scylla-sstable-scripts/dump.lua | 2 +- .../find-incomplete-clustering-row-keys.lua | 2 +- .../scylla-sstable-scripts/fragment-stats.lua | 2 +- tools/scylla-sstable-scripts/keys.lua | 2 +- tools/scylla-sstable-scripts/largest-key.lua | 2 +- tools/scylla-sstable-scripts/slice.lua | 2 +- tools/scylla-sstable.cc | 2 +- tools/scylla-types.cc | 2 +- tools/sstable_consumer.hh | 2 +- tools/testing/dist-check/dist-check.sh | 2 +- .../dist-check/docker.io/rockylinux-9.sh | 2 +- tools/testing/dist-check/docker.io/util.sh | 2 +- tools/utils.cc | 2 +- tools/utils.hh | 2 +- tracing/trace_keyspace_helper.cc | 2 +- tracing/trace_keyspace_helper.hh | 2 +- tracing/trace_state.cc | 2 +- tracing/trace_state.hh | 2 +- tracing/traced_file.cc | 2 +- tracing/traced_file.hh | 2 +- tracing/tracing.cc | 2 +- tracing/tracing.hh | 2 +- transport/controller.cc | 2 +- transport/controller.hh | 2 +- transport/cql_protocol_extension.cc | 2 +- transport/cql_protocol_extension.hh | 2 +- transport/event.cc | 2 +- transport/event.hh | 2 +- transport/event_notifier.cc | 2 +- transport/messages/result_message.cc | 2 +- transport/messages/result_message.hh | 2 +- transport/messages/result_message_base.hh | 2 +- transport/messages_fwd.hh | 2 +- transport/request.hh | 2 +- transport/response.hh | 2 +- transport/server.cc | 2 +- transport/server.hh | 2 +- types/collection.hh | 2 +- types/list.hh | 2 +- ...listlike_partial_deserializing_iterator.hh | 2 +- types/map.hh | 2 +- types/set.hh | 2 +- types/tuple.hh | 2 +- types/types.cc | 2 +- types/types.hh | 2 +- types/user.hh | 2 +- unified/build_unified.sh | 2 +- unified/install.sh | 2 +- unified/uninstall.sh | 2 +- unimplemented.cc | 2 +- unimplemented.hh | 2 +- utils/UUID.hh | 2 +- utils/UUID_gen.cc | 2 +- utils/UUID_gen.hh | 2 +- utils/abi/eh_ia64.hh | 2 +- utils/allocation_strategy.hh | 2 +- utils/amortized_reserve.hh | 2 +- utils/anchorless_list.hh | 2 +- utils/array-search.cc | 2 +- utils/array-search.hh | 2 +- utils/ascii.cc | 2 +- utils/ascii.hh | 2 +- utils/assert.hh | 2 +- utils/atomic_vector.hh | 2 +- utils/aws_sigv4.cc | 2 +- utils/aws_sigv4.hh | 2 +- utils/base64.cc | 2 +- utils/base64.hh | 2 +- utils/big_decimal.cc | 2 +- utils/big_decimal.hh | 2 +- utils/bit_cast.hh | 2 +- utils/bloom_calculations.cc | 2 +- utils/bloom_calculations.hh | 2 +- utils/bloom_filter.cc | 2 +- utils/bloom_filter.hh | 2 +- utils/bounded_stats_deque.hh | 2 +- utils/bptree.hh | 2 +- utils/buffer_input_stream.cc | 2 +- utils/buffer_input_stream.hh | 2 +- utils/buffer_view-to-managed_bytes_view.hh | 2 +- utils/cached_file.hh | 2 +- utils/cached_file_stats.hh | 2 +- utils/chunked_vector.hh | 2 +- utils/class_registrator.hh | 2 +- utils/clmul.hh | 2 +- utils/coarse_steady_clock.hh | 2 +- utils/collection-concepts.hh | 2 +- utils/compact-radix-tree.hh | 2 +- utils/config_file.cc | 2 +- utils/config_file.hh | 2 +- utils/config_file_impl.hh | 2 +- utils/contiguous_shared_buffer.hh | 2 +- utils/coroutine.hh | 2 +- utils/crc.hh | 2 +- utils/cross-shard-barrier.hh | 2 +- utils/data_input.hh | 2 +- utils/digest_algorithm.hh | 2 +- utils/digester.hh | 2 +- utils/directories.cc | 2 +- utils/directories.hh | 2 +- utils/disk-error-handler.cc | 2 +- utils/disk-error-handler.hh | 2 +- utils/div_ceil.hh | 2 +- utils/double-decker.hh | 2 +- utils/dynamic_bitset.cc | 2 +- utils/dynamic_bitset.hh | 2 +- utils/entangled.hh | 2 +- utils/enum_option.hh | 2 +- utils/error_injection.cc | 2 +- utils/error_injection.hh | 2 +- utils/estimated_histogram.hh | 2 +- utils/exception_container.hh | 2 +- utils/exceptions.cc | 2 +- utils/exceptions.hh | 2 +- utils/exponential_backoff_retry.hh | 2 +- utils/extremum_tracking.hh | 2 +- utils/file_lock.cc | 2 +- utils/file_lock.hh | 2 +- utils/flush_queue.hh | 2 +- utils/fragment_range.hh | 2 +- utils/fragmented_temporary_buffer.hh | 2 +- utils/gz/crc_combine.cc | 2 +- utils/gz/crc_combine.hh | 2 +- utils/gz/crc_combine_table.cc | 2 +- utils/gz/crc_combine_table.hh | 2 +- utils/hash.hh | 2 +- utils/hashers.cc | 2 +- utils/hashers.hh | 2 +- utils/hashing.hh | 2 +- utils/histogram.hh | 2 +- utils/histogram_metrics_helper.cc | 2 +- utils/histogram_metrics_helper.hh | 2 +- utils/http.hh | 2 +- utils/human_readable.cc | 2 +- utils/human_readable.hh | 2 +- utils/i_filter.cc | 2 +- utils/i_filter.hh | 2 +- utils/immutable-collection.hh | 2 +- utils/input_stream.hh | 2 +- utils/int_range.hh | 2 +- utils/intrusive-array.hh | 2 +- utils/intrusive_btree.hh | 2 +- utils/large_bitset.cc | 2 +- utils/large_bitset.hh | 2 +- utils/latency.hh | 2 +- utils/lexicographical_compare.hh | 2 +- utils/like_matcher.cc | 2 +- utils/like_matcher.hh | 2 +- utils/limiting_data_source.cc | 2 +- utils/limiting_data_source.hh | 2 +- utils/linearizing_input_stream.hh | 2 +- utils/lister.hh | 2 +- utils/loading_cache.hh | 2 +- utils/loading_shared_values.hh | 2 +- utils/log.hh | 2 +- utils/log_heap.hh | 2 +- utils/logalloc.cc | 2 +- utils/logalloc.hh | 2 +- utils/lru.hh | 2 +- utils/lsa/chunked_managed_vector.hh | 2 +- utils/lsa/weak_ptr.hh | 2 +- utils/managed_bytes.cc | 2 +- utils/managed_bytes.hh | 2 +- utils/managed_ref.hh | 2 +- utils/managed_vector.hh | 2 +- utils/maybe_yield.hh | 2 +- utils/memory_data_sink.hh | 2 +- utils/memory_limit_reached.hh | 2 +- utils/multiprecision_int.cc | 2 +- utils/multiprecision_int.hh | 2 +- utils/murmur_hash.cc | 2 +- utils/murmur_hash.hh | 2 +- utils/mutable_view.hh | 2 +- utils/neat-object-id.hh | 2 +- utils/observable.hh | 2 +- utils/on_internal_error.cc | 2 +- utils/on_internal_error.hh | 2 +- utils/overloaded_functor.hh | 2 +- utils/phased_barrier.hh | 2 +- utils/preempt.hh | 2 +- utils/pretty_printers.cc | 2 +- utils/pretty_printers.hh | 2 +- utils/rate_limiter.cc | 2 +- utils/rate_limiter.hh | 2 +- utils/recent_entries_map.hh | 2 +- utils/result.hh | 2 +- utils/result_combinators.hh | 2 +- utils/result_loop.hh | 2 +- utils/result_try.hh | 2 +- utils/reusable_buffer.hh | 2 +- utils/rjson.cc | 2 +- utils/rjson.hh | 2 +- utils/rpc_utils.hh | 2 +- utils/runtime.cc | 2 +- utils/runtime.hh | 2 +- utils/s3/aws_error.cc | 2 +- utils/s3/aws_error.hh | 2 +- utils/s3/client.cc | 2 +- utils/s3/client.hh | 2 +- utils/s3/client_fwd.hh | 2 +- utils/s3/creds.hh | 2 +- utils/s3/retry_strategy.cc | 2 +- utils/s3/retry_strategy.hh | 2 +- utils/sequenced_set.hh | 2 +- utils/serialization.hh | 2 +- utils/serialized_action.hh | 2 +- utils/simple_hashers.hh | 2 +- utils/small_vector.hh | 2 +- utils/sorting.hh | 2 +- utils/stall_free.hh | 2 +- utils/streaming_histogram.hh | 2 +- utils/tagged_integer.hh | 2 +- utils/to_string.cc | 2 +- utils/to_string.hh | 2 +- utils/top_k.hh | 2 +- utils/tuple_utils.hh | 2 +- utils/unconst.hh | 2 +- utils/updateable_value.cc | 2 +- utils/updateable_value.hh | 2 +- utils/user_provided_param.hh | 2 +- utils/utf8.cc | 2 +- utils/utf8.hh | 2 +- utils/uuid.cc | 2 +- utils/value_or_reference.hh | 2 +- utils/variant_element.hh | 2 +- utils/vle.hh | 2 +- utils/xx_hasher.hh | 2 +- validation.cc | 2 +- validation.hh | 2 +- version.hh | 2 +- view_info.hh | 2 +- vint-serialization.cc | 2 +- vint-serialization.hh | 2 +- zstd.cc | 2 +- 2117 files changed, 2187 insertions(+), 4160 deletions(-) create mode 100644 LICENSE-ScyllaDB-Source-Available.md delete mode 100644 LICENSE.AGPL diff --git a/LICENSE-ScyllaDB-Source-Available.md b/LICENSE-ScyllaDB-Source-Available.md new file mode 100644 index 000000000000..ff7f058e35a4 --- /dev/null +++ b/LICENSE-ScyllaDB-Source-Available.md @@ -0,0 +1,62 @@ +## **SCYLLADB SOFTWARE LICENSE AGREEMENT** + +| Version: | 1.0 | +| :---- | :---- | +| Last updated: | December 18, 2024 | + +**Your Acceptance** + +By utilizing or accessing the Software in any manner, You hereby confirm and agree to be bound by this ScyllaDB Software License Agreement (the "**Agreement**"), which sets forth the terms and conditions on which ScyllaDB Ltd. ("**Licensor**") makes the Software available to You, as the Licensee. If Licensee does not agree to the terms of this Agreement or cannot otherwise comply with the Agreement, Licensee shall not utilize or access the Software. + +The terms "**You**" or "**Licensee**" refer to any individual accessing or using the Software under this Agreement ("**Use**"). In case that such individual is Using the Software on behalf of a legal entity, You hereby irrevocably represents and warrants that You have full legal capacity and authority to enter into this Agreement on behalf of such entity as well as bind such entity to this Agreement, and in such case, the term "You" or "Licensee" in this Agreement will refer to such entity. + +**Grant of License** + +* **Software Definitions:** Software means the ScyllaDB software provided by Licensor, including the source code, object code, and any accompanying documentation or tools, or any part thereof, as made available under this Agreement. +* **Grant of License:** Subject to the terms and conditions of this Agreement, Licensor grants You a limited, non-exclusive, revocable, non-sublicensable, non-transferable, royalty free license to Use the Software, in each case solely for the purposes of: + 1) Copying, distributing, evaluating (including performing benchmarking or comparative tests or evaluations , subject to the limitations below) and improving the Software and ScyllaDB; and + 2) create a modified version of the Software (each, a "**Licensed Work**"); provided however, that each such Licensed Work keeps all or substantially all of the functions and features of the Software, and/or using all or substantially all of the source code of the Software. You hereby agree that all the Licensed Work are, upon creation, considered Licensed Work of the Licensor, shall be the sole property of the Licensor and its assignees, and the Licensor and its assignees shall be the sole owner of all rights of any kind or nature, in connection with such Licensed Work. You hereby irrevocably and unconditionally assign to the Licensor all the Licensed Work and any part thereof. This License applies separately for each version of the Licensed Work, which shall be considered "Software" for the purpose of this Agreement. + + +**License Limitations, Restrictions and Obligations:** The license grant above is subject to the following limitations, restrictions, and obligations. If Licensee’s Use of the Software does not comply with the above license grant or the terms of this section (including exceeding the Usage Limit set forth below), Licensee must: (i) refrain from any Use of the Software; and (ii) purchase a [commercial paid licens](https://www.scylladb.com/scylladb-proprietary-software-license-agreement/)e from the Licensor. + +* **Updates:** You shall be solely responsible for providing all equipment, systems, assets, access, and ancillary goods and services needed to access and Use the Software. Licensor may modify or update the Software at any time, without notification, in its sole and absolute discretion. After the effective date of each such update, Licensor shall bear no obligation to run, provide or support legacy versions of the Software. +* **"Usage Limit":** Licensee's total overall available storage across all deployments and clusters of the Software and the Licensed Work under this License shall not exceed 10TB and/or an upper limit of 50 VCPUs (hyper threads). +* **IP Markings:** Licensee must retain all copyright, trademark, and other proprietary notices contained in the Software. You will not modify, delete, alter, remove, or obscure any intellectual property, including without limitations licensing, copyright, trademark, or any other notices of Licensor in the Software. +* **License Reproduction:** You must conspicuously display this Agreement on each copy of the Software. If You receive the Software from a third party, this Agreement still applies to Your Use of the Software. You will be responsible for any breach of this Agreement by any such third-party. +* Distribution of any Licensed Works is permitted, provided that: (i) You must include in any Licensed Work prominent notices stating that You have modified the Software, (ii) You include a copy of this Agreement with the Licensed Work, and (iii) You clearly identify all modifications made in the Licensed Work and provides attribution to the Licensor as the original author(s) of the Software. +* **Commercial Use Restrictions:** Licensee may not offer the Software as a software-as-a-service (SaaS) or commercial database-as-as-service (dBaaS) offering. Licensee may not use the Software to compete with Licensor's existing or future products or services. If your Use of the Software does not comply with the requirements currently in effect as described in this License, you must purchase a commercial license from the Licensor, its affiliated entities, or you must refrain from using the Software and all Licensed Work. Furthermore, if You make any written claim of patent infringement relating to the Software, Your patent license for the Software granted under this Agreement terminates immediately. +* Notwithstanding anything to the contrary, under the License granted hereunder, You shall not and shall not permit others to: (i) transfer the Software or any portions thereof to any other party except as expressly permitted herein; (ii) attempt to circumvent or overcome any technological protection measures incorporated into the Software; (iii) incorporate the Software into the structure, machinery or controls of any aircraft, other aerial device, military vehicle, hovercraft, waterborne craft or any medical equipment of any kind; or (iv) use the Software or any part thereof in any unlawful, harmful or illegal manner, or in a manner which infringes third parties’ rights in any way, including intellectual property rights. + +**Monitoring; Audit** + +* **License Key:** Licensor may implement a method of authentication, e.g., a unique license token ("License Key") as a condition of accessing or using the Software. Upon the implementation of such License Key, Licensee agrees to comply with Licensor terms and requirements with regards to such License Key +* **Monitoring & Data Sharing:** Licensor do not collect customer data from its database. Notwithstanding, Licensee acknowledges and agrees that the License Key and Software may share telemetry metrics and information regarding the execution volume and statistics with Licensor regarding Licensee’s use of the same. Any disclosure or use of such information shall be subject to, and in accordance with, Licensor’s Privacy Policy and Data Processing Agreement, which can be found at [https://www.scylladb.com/policies-agreements](https://www.scylladb.com/policies-agreements). +* **Information Requests; Audits:** Licensee shall keep accurate records of its access to and use of any Software, and shall promptly respond to any Licensor requests for information regarding the same. To ensure compliance with the terms of this Agreement, during the term of this Agreement and for a period of one (1) year thereafter, Licensor (or an agent bound by customary confidentiality undertakings on its behalf) may audit Licensee’s records which are related to its access to or use of the Software. The cost of such audit shall be borne by Licensor unless it is determined that Licensee has materially breached this Agreement. + +**Termination** + +* **Termination:** Licensor may immediately terminate this Agreement will automatically terminate if You for any reason, including without limitation for (i) Licensee’s breach of any term, condition, or restriction of this Agreement, unless such breach was cured to Licensor’s satisfaction within no more than 15 days from the date of the breach. Notwithstanding the foregoing, intentional; or (ii) if Licensee brings any claim, demand or repeated breaches lawsuit against Licensor. +* **Obligations on Termination:** Upon termination of this Agreement by You will cause Your licenses to terminate automatically and permanently, at Licensor’s sole discretion, Licensee must (i) immediately stop using any Software, (ii) return all copies of any tools or documentation provided by Licensor; and (iii) pay amount due to Licensor hereunder (e.g., audit costs). All obligations which by their nature must survive the termination of this Agreement shall so survive. + +**Indemnity; Disclaimer; Limitation of Liability** + +* **Indemnity:** Licensee hereby agrees to indemnify, defend and hold harmless Licensor and its affiliates from any losses or damages incurred due to a third party claim arising out of: (i) Licensee’s breach of this Agreement; (ii) Licensee’s negligence, willful misconduct or violation of law, or (iii) Licensee’s products or services. +* DISCLAIMER OF WARRANTIES: LICENSEE AGREES THAT LICENSOR HAS MADE NO EXPRESS WARRANTIES REGARDING THE SOFTWARE AND THAT THE SOFTWARE IS BEING PROVIDED "AS IS" WITHOUT WARRANTY OF ANY KIND. LICENSOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THE SOFTWARE, EXPRESS OR IMPLIED, INCLUDING WITHOUT LIMITATION, ANY IMPLIED WARRANTIES OF FITNESS FOR A PARTICULAR PURPOSE; TITLE; MERCHANTABILITY; OR NON-INFRINGEMENT OF THIRD PARTY RIGHTS. LICENSOR DOES NOT WARRANT THAT THE SOFTWARE WILL OPERATE UNINTERRUPTED OR ERROR FREE, OR THAT ALL ERRORS WILL BE CORRECTED. LICENSOR DOES NOT GUARANTEE ANY PARTICULAR RESULTS FROM THE USE OF THE SOFTWARE, AND DOES NOT WARRANT THAT THE SOFTWARE IS FIT FOR ANY PARTICULAR PURPOSE. +* LIMITATION OF LIABILITY: TO THE FULLEST EXTENT PERMISSIBLE UNDER APPLICABLE LAW, IN NO EVENT WILL LICENSOR AND/OR ITS AFFILIATES, EMPLOYEES, OFFICERS AND DIRECTORS BE LIABLE TO LICENSEE FOR (I) ANY LOSS OF USE OR DATA; INTERRUPTION OF BUSINESS; OR ANY INDIRECT; SPECIAL; INCIDENTAL; OR CONSEQUENTIAL DAMAGES OF ANY KIND (INCLUDING LOST PROFITS); AND (II) ANY DIRECT DAMAGES EXCEEDING THE TOTAL AMOUNT OF ONE THOUSAND US DOLLARS ($1,000). THE FOREGOING PROVISIONS LIMITING THE LIABILITY OF LICENSOR SHALL APPLY REGARDLESS OF THE FORM OR CAUSE OF ACTION, WHETHER IN STRICT LIABILITY, CONTRACT OR TORT. + +**Proprietary Rights; No Other Rights** + +* **Ownership:** Licensor retains sole and exclusive ownership of all rights, interests and title in the Software and any scripts, processes, techniques, methodologies, inventions, know-how, concepts, formatting, arrangements, visual attributes, ideas, database rights, copyrights, patents, trade secrets, and other intellectual property related thereto, and all derivatives, enhancements, modifications and improvements thereof. Except for the limited license rights granted herein, Licensee has no rights in or to the Software and/ or Licensor’s trademarks, logo, or branding and You acknowledge that such Software, trademarks, logo, or branding is the sole property of Licensor. +* **Feedback:** Licensee is not required to provide any suggestions, enhancement requests, recommendations or other feedback regarding the Software ("Feedback"). If, notwithstanding this policy, Licensee submits Feedback, Licensee understands and acknowledges that such Feedback is not submitted in confidence and Licensor assumes no obligation, expressed or implied, by considering it. All right in any trademark or logo of Licensor or its affiliates and You shall make no claim of right to the Software or any part thereof to be supplied by Licensor hereunder and acknowledges that as between Licensor and You, such Software is the sole proprietary, title and interest in and to Licensor.such Feedback shall be assigned to, and shall become the sole and exclusive property of, Licensor upon its creation. +* Except for the rights expressly granted to You under this Agreement, You are not granted any other licenses or rights in the Software or otherwise. This Agreement constitutes the entire agreement between the You and the Licensor with respect to the subject matter hereof and supersedes all prior or contemporaneous communications, representations, or agreements, whether oral or written. +* **Third-Party Software:** Customer acknowledges that the Software may contain open and closed source components (“OSS Components”) that are governed separately by certain licenses, in each case as further provided by Company upon request. Any applicable OSS Component license is solely between Licensee and the applicable licensor of the OSS Component and Licensee shall comply with the applicable OSS Component license. +* If any provision of this Agreement is held to be invalid or unenforceable, such provision shall be struck and the remaining provisions shall remain in full force and effect. + +**Miscellaneous** + +* **Miscellaneous:** This Agreement may be modified at any time by Licensor, and constitutes the entire agreement between the parties with respect to the subject matter hereof. Licensee may not assign or subcontract its rights or obligations under this Agreement. This Agreement does not, and shall not be construed to create any relationship, partnership, joint venture, employer-employee, agency, or franchisor-franchisee relationship between the parties. +* **Governing Law & Jurisdiction:** This Agreement shall be governed and construed in accordance with the laws of Israel, without giving effect to their respective conflicts of laws provisions, and the competent courts situated in Tel Aviv, Israel, shall have sole and exclusive jurisdiction over the parties and any conflict and/or dispute arising out of, or in connection to, this Agreement + +\[*End of ScyllaDB Software License Agreement*\] + diff --git a/LICENSE.AGPL b/LICENSE.AGPL deleted file mode 100644 index dba13ed2ddf7..000000000000 --- a/LICENSE.AGPL +++ /dev/null @@ -1,661 +0,0 @@ - GNU AFFERO GENERAL PUBLIC LICENSE - Version 3, 19 November 2007 - - Copyright (C) 2007 Free Software Foundation, Inc. - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The GNU Affero General Public License is a free, copyleft license for -software and other kinds of works, specifically designed to ensure -cooperation with the community in the case of network server software. - - The licenses for most software and other practical works are designed -to take away your freedom to share and change the works. By contrast, -our General Public Licenses are intended to guarantee your freedom to -share and change all versions of a program--to make sure it remains free -software for all its users. - - When we speak of free software, we are referring to freedom, not -price. Our General Public Licenses are designed to make sure that you -have the freedom to distribute copies of free software (and charge for -them if you wish), that you receive source code or can get it if you -want it, that you can change the software or use pieces of it in new -free programs, and that you know you can do these things. - - Developers that use our General Public Licenses protect your rights -with two steps: (1) assert copyright on the software, and (2) offer -you this License which gives you legal permission to copy, distribute -and/or modify the software. - - A secondary benefit of defending all users' freedom is that -improvements made in alternate versions of the program, if they -receive widespread use, become available for other developers to -incorporate. Many developers of free software are heartened and -encouraged by the resulting cooperation. However, in the case of -software used on network servers, this result may fail to come about. -The GNU General Public License permits making a modified version and -letting the public access it on a server without ever releasing its -source code to the public. - - The GNU Affero General Public License is designed specifically to -ensure that, in such cases, the modified source code becomes available -to the community. It requires the operator of a network server to -provide the source code of the modified version running there to the -users of that server. Therefore, public use of a modified version, on -a publicly accessible server, gives the public access to the source -code of the modified version. - - An older license, called the Affero General Public License and -published by Affero, was designed to accomplish similar goals. This is -a different license, not a version of the Affero GPL, but Affero has -released a new version of the Affero GPL which permits relicensing under -this license. - - The precise terms and conditions for copying, distribution and -modification follow. - - TERMS AND CONDITIONS - - 0. Definitions. - - "This License" refers to version 3 of the GNU Affero General Public License. - - "Copyright" also means copyright-like laws that apply to other kinds of -works, such as semiconductor masks. - - "The Program" refers to any copyrightable work licensed under this -License. Each licensee is addressed as "you". "Licensees" and -"recipients" may be individuals or organizations. - - To "modify" a work means to copy from or adapt all or part of the work -in a fashion requiring copyright permission, other than the making of an -exact copy. The resulting work is called a "modified version" of the -earlier work or a work "based on" the earlier work. - - A "covered work" means either the unmodified Program or a work based -on the Program. - - To "propagate" a work means to do anything with it that, without -permission, would make you directly or secondarily liable for -infringement under applicable copyright law, except executing it on a -computer or modifying a private copy. Propagation includes copying, -distribution (with or without modification), making available to the -public, and in some countries other activities as well. - - To "convey" a work means any kind of propagation that enables other -parties to make or receive copies. Mere interaction with a user through -a computer network, with no transfer of a copy, is not conveying. - - An interactive user interface displays "Appropriate Legal Notices" -to the extent that it includes a convenient and prominently visible -feature that (1) displays an appropriate copyright notice, and (2) -tells the user that there is no warranty for the work (except to the -extent that warranties are provided), that licensees may convey the -work under this License, and how to view a copy of this License. If -the interface presents a list of user commands or options, such as a -menu, a prominent item in the list meets this criterion. - - 1. Source Code. - - The "source code" for a work means the preferred form of the work -for making modifications to it. "Object code" means any non-source -form of a work. - - A "Standard Interface" means an interface that either is an official -standard defined by a recognized standards body, or, in the case of -interfaces specified for a particular programming language, one that -is widely used among developers working in that language. - - The "System Libraries" of an executable work include anything, other -than the work as a whole, that (a) is included in the normal form of -packaging a Major Component, but which is not part of that Major -Component, and (b) serves only to enable use of the work with that -Major Component, or to implement a Standard Interface for which an -implementation is available to the public in source code form. A -"Major Component", in this context, means a major essential component -(kernel, window system, and so on) of the specific operating system -(if any) on which the executable work runs, or a compiler used to -produce the work, or an object code interpreter used to run it. - - The "Corresponding Source" for a work in object code form means all -the source code needed to generate, install, and (for an executable -work) run the object code and to modify the work, including scripts to -control those activities. However, it does not include the work's -System Libraries, or general-purpose tools or generally available free -programs which are used unmodified in performing those activities but -which are not part of the work. For example, Corresponding Source -includes interface definition files associated with source files for -the work, and the source code for shared libraries and dynamically -linked subprograms that the work is specifically designed to require, -such as by intimate data communication or control flow between those -subprograms and other parts of the work. - - The Corresponding Source need not include anything that users -can regenerate automatically from other parts of the Corresponding -Source. - - The Corresponding Source for a work in source code form is that -same work. - - 2. Basic Permissions. - - All rights granted under this License are granted for the term of -copyright on the Program, and are irrevocable provided the stated -conditions are met. This License explicitly affirms your unlimited -permission to run the unmodified Program. The output from running a -covered work is covered by this License only if the output, given its -content, constitutes a covered work. This License acknowledges your -rights of fair use or other equivalent, as provided by copyright law. - - You may make, run and propagate covered works that you do not -convey, without conditions so long as your license otherwise remains -in force. You may convey covered works to others for the sole purpose -of having them make modifications exclusively for you, or provide you -with facilities for running those works, provided that you comply with -the terms of this License in conveying all material for which you do -not control copyright. Those thus making or running the covered works -for you must do so exclusively on your behalf, under your direction -and control, on terms that prohibit them from making any copies of -your copyrighted material outside their relationship with you. - - Conveying under any other circumstances is permitted solely under -the conditions stated below. Sublicensing is not allowed; section 10 -makes it unnecessary. - - 3. Protecting Users' Legal Rights From Anti-Circumvention Law. - - No covered work shall be deemed part of an effective technological -measure under any applicable law fulfilling obligations under article -11 of the WIPO copyright treaty adopted on 20 December 1996, or -similar laws prohibiting or restricting circumvention of such -measures. - - When you convey a covered work, you waive any legal power to forbid -circumvention of technological measures to the extent such circumvention -is effected by exercising rights under this License with respect to -the covered work, and you disclaim any intention to limit operation or -modification of the work as a means of enforcing, against the work's -users, your or third parties' legal rights to forbid circumvention of -technological measures. - - 4. Conveying Verbatim Copies. - - You may convey verbatim copies of the Program's source code as you -receive it, in any medium, provided that you conspicuously and -appropriately publish on each copy an appropriate copyright notice; -keep intact all notices stating that this License and any -non-permissive terms added in accord with section 7 apply to the code; -keep intact all notices of the absence of any warranty; and give all -recipients a copy of this License along with the Program. - - You may charge any price or no price for each copy that you convey, -and you may offer support or warranty protection for a fee. - - 5. Conveying Modified Source Versions. - - You may convey a work based on the Program, or the modifications to -produce it from the Program, in the form of source code under the -terms of section 4, provided that you also meet all of these conditions: - - a) The work must carry prominent notices stating that you modified - it, and giving a relevant date. - - b) The work must carry prominent notices stating that it is - released under this License and any conditions added under section - 7. This requirement modifies the requirement in section 4 to - "keep intact all notices". - - c) You must license the entire work, as a whole, under this - License to anyone who comes into possession of a copy. This - License will therefore apply, along with any applicable section 7 - additional terms, to the whole of the work, and all its parts, - regardless of how they are packaged. This License gives no - permission to license the work in any other way, but it does not - invalidate such permission if you have separately received it. - - d) If the work has interactive user interfaces, each must display - Appropriate Legal Notices; however, if the Program has interactive - interfaces that do not display Appropriate Legal Notices, your - work need not make them do so. - - A compilation of a covered work with other separate and independent -works, which are not by their nature extensions of the covered work, -and which are not combined with it such as to form a larger program, -in or on a volume of a storage or distribution medium, is called an -"aggregate" if the compilation and its resulting copyright are not -used to limit the access or legal rights of the compilation's users -beyond what the individual works permit. Inclusion of a covered work -in an aggregate does not cause this License to apply to the other -parts of the aggregate. - - 6. Conveying Non-Source Forms. - - You may convey a covered work in object code form under the terms -of sections 4 and 5, provided that you also convey the -machine-readable Corresponding Source under the terms of this License, -in one of these ways: - - a) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by the - Corresponding Source fixed on a durable physical medium - customarily used for software interchange. - - b) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by a - written offer, valid for at least three years and valid for as - long as you offer spare parts or customer support for that product - model, to give anyone who possesses the object code either (1) a - copy of the Corresponding Source for all the software in the - product that is covered by this License, on a durable physical - medium customarily used for software interchange, for a price no - more than your reasonable cost of physically performing this - conveying of source, or (2) access to copy the - Corresponding Source from a network server at no charge. - - c) Convey individual copies of the object code with a copy of the - written offer to provide the Corresponding Source. This - alternative is allowed only occasionally and noncommercially, and - only if you received the object code with such an offer, in accord - with subsection 6b. - - d) Convey the object code by offering access from a designated - place (gratis or for a charge), and offer equivalent access to the - Corresponding Source in the same way through the same place at no - further charge. You need not require recipients to copy the - Corresponding Source along with the object code. If the place to - copy the object code is a network server, the Corresponding Source - may be on a different server (operated by you or a third party) - that supports equivalent copying facilities, provided you maintain - clear directions next to the object code saying where to find the - Corresponding Source. Regardless of what server hosts the - Corresponding Source, you remain obligated to ensure that it is - available for as long as needed to satisfy these requirements. - - e) Convey the object code using peer-to-peer transmission, provided - you inform other peers where the object code and Corresponding - Source of the work are being offered to the general public at no - charge under subsection 6d. - - A separable portion of the object code, whose source code is excluded -from the Corresponding Source as a System Library, need not be -included in conveying the object code work. - - A "User Product" is either (1) a "consumer product", which means any -tangible personal property which is normally used for personal, family, -or household purposes, or (2) anything designed or sold for incorporation -into a dwelling. In determining whether a product is a consumer product, -doubtful cases shall be resolved in favor of coverage. For a particular -product received by a particular user, "normally used" refers to a -typical or common use of that class of product, regardless of the status -of the particular user or of the way in which the particular user -actually uses, or expects or is expected to use, the product. A product -is a consumer product regardless of whether the product has substantial -commercial, industrial or non-consumer uses, unless such uses represent -the only significant mode of use of the product. - - "Installation Information" for a User Product means any methods, -procedures, authorization keys, or other information required to install -and execute modified versions of a covered work in that User Product from -a modified version of its Corresponding Source. The information must -suffice to ensure that the continued functioning of the modified object -code is in no case prevented or interfered with solely because -modification has been made. - - If you convey an object code work under this section in, or with, or -specifically for use in, a User Product, and the conveying occurs as -part of a transaction in which the right of possession and use of the -User Product is transferred to the recipient in perpetuity or for a -fixed term (regardless of how the transaction is characterized), the -Corresponding Source conveyed under this section must be accompanied -by the Installation Information. But this requirement does not apply -if neither you nor any third party retains the ability to install -modified object code on the User Product (for example, the work has -been installed in ROM). - - The requirement to provide Installation Information does not include a -requirement to continue to provide support service, warranty, or updates -for a work that has been modified or installed by the recipient, or for -the User Product in which it has been modified or installed. Access to a -network may be denied when the modification itself materially and -adversely affects the operation of the network or violates the rules and -protocols for communication across the network. - - Corresponding Source conveyed, and Installation Information provided, -in accord with this section must be in a format that is publicly -documented (and with an implementation available to the public in -source code form), and must require no special password or key for -unpacking, reading or copying. - - 7. Additional Terms. - - "Additional permissions" are terms that supplement the terms of this -License by making exceptions from one or more of its conditions. -Additional permissions that are applicable to the entire Program shall -be treated as though they were included in this License, to the extent -that they are valid under applicable law. If additional permissions -apply only to part of the Program, that part may be used separately -under those permissions, but the entire Program remains governed by -this License without regard to the additional permissions. - - When you convey a copy of a covered work, you may at your option -remove any additional permissions from that copy, or from any part of -it. (Additional permissions may be written to require their own -removal in certain cases when you modify the work.) You may place -additional permissions on material, added by you to a covered work, -for which you have or can give appropriate copyright permission. - - Notwithstanding any other provision of this License, for material you -add to a covered work, you may (if authorized by the copyright holders of -that material) supplement the terms of this License with terms: - - a) Disclaiming warranty or limiting liability differently from the - terms of sections 15 and 16 of this License; or - - b) Requiring preservation of specified reasonable legal notices or - author attributions in that material or in the Appropriate Legal - Notices displayed by works containing it; or - - c) Prohibiting misrepresentation of the origin of that material, or - requiring that modified versions of such material be marked in - reasonable ways as different from the original version; or - - d) Limiting the use for publicity purposes of names of licensors or - authors of the material; or - - e) Declining to grant rights under trademark law for use of some - trade names, trademarks, or service marks; or - - f) Requiring indemnification of licensors and authors of that - material by anyone who conveys the material (or modified versions of - it) with contractual assumptions of liability to the recipient, for - any liability that these contractual assumptions directly impose on - those licensors and authors. - - All other non-permissive additional terms are considered "further -restrictions" within the meaning of section 10. If the Program as you -received it, or any part of it, contains a notice stating that it is -governed by this License along with a term that is a further -restriction, you may remove that term. If a license document contains -a further restriction but permits relicensing or conveying under this -License, you may add to a covered work material governed by the terms -of that license document, provided that the further restriction does -not survive such relicensing or conveying. - - If you add terms to a covered work in accord with this section, you -must place, in the relevant source files, a statement of the -additional terms that apply to those files, or a notice indicating -where to find the applicable terms. - - Additional terms, permissive or non-permissive, may be stated in the -form of a separately written license, or stated as exceptions; -the above requirements apply either way. - - 8. Termination. - - You may not propagate or modify a covered work except as expressly -provided under this License. Any attempt otherwise to propagate or -modify it is void, and will automatically terminate your rights under -this License (including any patent licenses granted under the third -paragraph of section 11). - - However, if you cease all violation of this License, then your -license from a particular copyright holder is reinstated (a) -provisionally, unless and until the copyright holder explicitly and -finally terminates your license, and (b) permanently, if the copyright -holder fails to notify you of the violation by some reasonable means -prior to 60 days after the cessation. - - Moreover, your license from a particular copyright holder is -reinstated permanently if the copyright holder notifies you of the -violation by some reasonable means, this is the first time you have -received notice of violation of this License (for any work) from that -copyright holder, and you cure the violation prior to 30 days after -your receipt of the notice. - - Termination of your rights under this section does not terminate the -licenses of parties who have received copies or rights from you under -this License. If your rights have been terminated and not permanently -reinstated, you do not qualify to receive new licenses for the same -material under section 10. - - 9. Acceptance Not Required for Having Copies. - - You are not required to accept this License in order to receive or -run a copy of the Program. Ancillary propagation of a covered work -occurring solely as a consequence of using peer-to-peer transmission -to receive a copy likewise does not require acceptance. However, -nothing other than this License grants you permission to propagate or -modify any covered work. These actions infringe copyright if you do -not accept this License. Therefore, by modifying or propagating a -covered work, you indicate your acceptance of this License to do so. - - 10. Automatic Licensing of Downstream Recipients. - - Each time you convey a covered work, the recipient automatically -receives a license from the original licensors, to run, modify and -propagate that work, subject to this License. You are not responsible -for enforcing compliance by third parties with this License. - - An "entity transaction" is a transaction transferring control of an -organization, or substantially all assets of one, or subdividing an -organization, or merging organizations. If propagation of a covered -work results from an entity transaction, each party to that -transaction who receives a copy of the work also receives whatever -licenses to the work the party's predecessor in interest had or could -give under the previous paragraph, plus a right to possession of the -Corresponding Source of the work from the predecessor in interest, if -the predecessor has it or can get it with reasonable efforts. - - You may not impose any further restrictions on the exercise of the -rights granted or affirmed under this License. For example, you may -not impose a license fee, royalty, or other charge for exercise of -rights granted under this License, and you may not initiate litigation -(including a cross-claim or counterclaim in a lawsuit) alleging that -any patent claim is infringed by making, using, selling, offering for -sale, or importing the Program or any portion of it. - - 11. Patents. - - A "contributor" is a copyright holder who authorizes use under this -License of the Program or a work on which the Program is based. The -work thus licensed is called the contributor's "contributor version". - - A contributor's "essential patent claims" are all patent claims -owned or controlled by the contributor, whether already acquired or -hereafter acquired, that would be infringed by some manner, permitted -by this License, of making, using, or selling its contributor version, -but do not include claims that would be infringed only as a -consequence of further modification of the contributor version. For -purposes of this definition, "control" includes the right to grant -patent sublicenses in a manner consistent with the requirements of -this License. - - Each contributor grants you a non-exclusive, worldwide, royalty-free -patent license under the contributor's essential patent claims, to -make, use, sell, offer for sale, import and otherwise run, modify and -propagate the contents of its contributor version. - - In the following three paragraphs, a "patent license" is any express -agreement or commitment, however denominated, not to enforce a patent -(such as an express permission to practice a patent or covenant not to -sue for patent infringement). To "grant" such a patent license to a -party means to make such an agreement or commitment not to enforce a -patent against the party. - - If you convey a covered work, knowingly relying on a patent license, -and the Corresponding Source of the work is not available for anyone -to copy, free of charge and under the terms of this License, through a -publicly available network server or other readily accessible means, -then you must either (1) cause the Corresponding Source to be so -available, or (2) arrange to deprive yourself of the benefit of the -patent license for this particular work, or (3) arrange, in a manner -consistent with the requirements of this License, to extend the patent -license to downstream recipients. "Knowingly relying" means you have -actual knowledge that, but for the patent license, your conveying the -covered work in a country, or your recipient's use of the covered work -in a country, would infringe one or more identifiable patents in that -country that you have reason to believe are valid. - - If, pursuant to or in connection with a single transaction or -arrangement, you convey, or propagate by procuring conveyance of, a -covered work, and grant a patent license to some of the parties -receiving the covered work authorizing them to use, propagate, modify -or convey a specific copy of the covered work, then the patent license -you grant is automatically extended to all recipients of the covered -work and works based on it. - - A patent license is "discriminatory" if it does not include within -the scope of its coverage, prohibits the exercise of, or is -conditioned on the non-exercise of one or more of the rights that are -specifically granted under this License. You may not convey a covered -work if you are a party to an arrangement with a third party that is -in the business of distributing software, under which you make payment -to the third party based on the extent of your activity of conveying -the work, and under which the third party grants, to any of the -parties who would receive the covered work from you, a discriminatory -patent license (a) in connection with copies of the covered work -conveyed by you (or copies made from those copies), or (b) primarily -for and in connection with specific products or compilations that -contain the covered work, unless you entered into that arrangement, -or that patent license was granted, prior to 28 March 2007. - - Nothing in this License shall be construed as excluding or limiting -any implied license or other defenses to infringement that may -otherwise be available to you under applicable patent law. - - 12. No Surrender of Others' Freedom. - - If conditions are imposed on you (whether by court order, agreement or -otherwise) that contradict the conditions of this License, they do not -excuse you from the conditions of this License. If you cannot convey a -covered work so as to satisfy simultaneously your obligations under this -License and any other pertinent obligations, then as a consequence you may -not convey it at all. For example, if you agree to terms that obligate you -to collect a royalty for further conveying from those to whom you convey -the Program, the only way you could satisfy both those terms and this -License would be to refrain entirely from conveying the Program. - - 13. Remote Network Interaction; Use with the GNU General Public License. - - Notwithstanding any other provision of this License, if you modify the -Program, your modified version must prominently offer all users -interacting with it remotely through a computer network (if your version -supports such interaction) an opportunity to receive the Corresponding -Source of your version by providing access to the Corresponding Source -from a network server at no charge, through some standard or customary -means of facilitating copying of software. This Corresponding Source -shall include the Corresponding Source for any work covered by version 3 -of the GNU General Public License that is incorporated pursuant to the -following paragraph. - - Notwithstanding any other provision of this License, you have -permission to link or combine any covered work with a work licensed -under version 3 of the GNU General Public License into a single -combined work, and to convey the resulting work. The terms of this -License will continue to apply to the part which is the covered work, -but the work with which it is combined will remain governed by version -3 of the GNU General Public License. - - 14. Revised Versions of this License. - - The Free Software Foundation may publish revised and/or new versions of -the GNU Affero General Public License from time to time. Such new versions -will be similar in spirit to the present version, but may differ in detail to -address new problems or concerns. - - Each version is given a distinguishing version number. If the -Program specifies that a certain numbered version of the GNU Affero General -Public License "or any later version" applies to it, you have the -option of following the terms and conditions either of that numbered -version or of any later version published by the Free Software -Foundation. If the Program does not specify a version number of the -GNU Affero General Public License, you may choose any version ever published -by the Free Software Foundation. - - If the Program specifies that a proxy can decide which future -versions of the GNU Affero General Public License can be used, that proxy's -public statement of acceptance of a version permanently authorizes you -to choose that version for the Program. - - Later license versions may give you additional or different -permissions. However, no additional obligations are imposed on any -author or copyright holder as a result of your choosing to follow a -later version. - - 15. Disclaimer of Warranty. - - THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY -APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT -HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY -OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, -THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR -PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM -IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF -ALL NECESSARY SERVICING, REPAIR OR CORRECTION. - - 16. Limitation of Liability. - - IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING -WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MODIFIES AND/OR CONVEYS -THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY -GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE -USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF -DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD -PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), -EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF -SUCH DAMAGES. - - 17. Interpretation of Sections 15 and 16. - - If the disclaimer of warranty and limitation of liability provided -above cannot be given local legal effect according to their terms, -reviewing courts shall apply local law that most closely approximates -an absolute waiver of all civil liability in connection with the -Program, unless a warranty or assumption of liability accompanies a -copy of the Program in return for a fee. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest -possible use to the public, the best way to achieve this is to make it -free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest -to attach them to the start of each source file to most effectively -state the exclusion of warranty; and each file should have at least -the "copyright" line and a pointer to where the full notice is found. - - - Copyright (C) - - This program is free software: you can redistribute it and/or modify - it under the terms of the GNU Affero General Public License as published by - the Free Software Foundation, either version 3 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU Affero General Public License for more details. - - You should have received a copy of the GNU Affero General Public License - along with this program. If not, see . - -Also add information on how to contact you by electronic and paper mail. - - If your software can interact with users remotely through a computer -network, you should also make sure that it provides a way for users to -get its source. For example, if your program is a web application, its -interface could display a "Source" link that leads users to an archive -of the code. There are many ways you could offer source, and different -solutions will be better for different programs; see section 13 for the -specific requirements. - - You should also get your employer (if you work as a programmer) or school, -if any, to sign a "copyright disclaimer" for the program, if necessary. -For more information on this, and how to apply and follow the GNU AGPL, see -. diff --git a/absl-flat_hash_map.cc b/absl-flat_hash_map.cc index 73c4f7a438be..47c07ddbb8ae 100644 --- a/absl-flat_hash_map.cc +++ b/absl-flat_hash_map.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "absl-flat_hash_map.hh" diff --git a/absl-flat_hash_map.hh b/absl-flat_hash_map.hh index 8626c25458bd..c4da4d03f33e 100644 --- a/absl-flat_hash_map.hh +++ b/absl-flat_hash_map.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/auth.cc b/alternator/auth.cc index 61e20ad5dd6b..881f10513d64 100644 --- a/alternator/auth.cc +++ b/alternator/auth.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "alternator/error.hh" diff --git a/alternator/auth.hh b/alternator/auth.hh index e13f0aa2c822..11ed0aa534e6 100644 --- a/alternator/auth.hh +++ b/alternator/auth.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/conditions.cc b/alternator/conditions.cc index b02522f3e9c2..dc5a3ae38687 100644 --- a/alternator/conditions.cc +++ b/alternator/conditions.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/alternator/conditions.hh b/alternator/conditions.hh index f0b34791d614..e3c16b6bfb49 100644 --- a/alternator/conditions.hh +++ b/alternator/conditions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/alternator/consumed_capacity.cc b/alternator/consumed_capacity.cc index e635b63fa49e..a306b1441a6a 100644 --- a/alternator/consumed_capacity.cc +++ b/alternator/consumed_capacity.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "consumed_capacity.hh" diff --git a/alternator/consumed_capacity.hh b/alternator/consumed_capacity.hh index de8980b9eaa8..3dcbb65355b2 100644 --- a/alternator/consumed_capacity.hh +++ b/alternator/consumed_capacity.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/controller.cc b/alternator/controller.cc index b33a7ba4eef7..b791ea8de2d1 100644 --- a/alternator/controller.cc +++ b/alternator/controller.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/alternator/controller.hh b/alternator/controller.hh index c0c75235bf83..f4e926a80177 100644 --- a/alternator/controller.hh +++ b/alternator/controller.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/error.hh b/alternator/error.hh index bb01d4807d9e..8b67be908a02 100644 --- a/alternator/error.hh +++ b/alternator/error.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/executor.cc b/alternator/executor.cc index e96d8063e91e..30bc2c071cf4 100644 --- a/alternator/executor.cc +++ b/alternator/executor.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/alternator/executor.hh b/alternator/executor.hh index 56542d533ad6..1e0d82a94568 100644 --- a/alternator/executor.hh +++ b/alternator/executor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/expressions.cc b/alternator/expressions.cc index 67c789e29637..db3435d5975e 100644 --- a/alternator/expressions.cc +++ b/alternator/expressions.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "expressions.hh" diff --git a/alternator/expressions.g b/alternator/expressions.g index 1c2125cef7d7..c3ac1f48371b 100644 --- a/alternator/expressions.g +++ b/alternator/expressions.g @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/alternator/expressions.hh b/alternator/expressions.hh index f9bf771d0ec0..27ca11947c4e 100644 --- a/alternator/expressions.hh +++ b/alternator/expressions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/expressions_types.hh b/alternator/expressions_types.hh index de62ad24c123..0265d9efd9aa 100644 --- a/alternator/expressions_types.hh +++ b/alternator/expressions_types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/rmw_operation.hh b/alternator/rmw_operation.hh index 87efe8288b06..0133d0737868 100644 --- a/alternator/rmw_operation.hh +++ b/alternator/rmw_operation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/serialization.cc b/alternator/serialization.cc index 0abd7ce0650e..1ab60feed7c6 100644 --- a/alternator/serialization.cc +++ b/alternator/serialization.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/base64.hh" diff --git a/alternator/serialization.hh b/alternator/serialization.hh index b87dbbcfd75e..90d842cd2213 100644 --- a/alternator/serialization.hh +++ b/alternator/serialization.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/server.cc b/alternator/server.cc index fbfeafbadd6d..1f3fd4b7ea36 100644 --- a/alternator/server.cc +++ b/alternator/server.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "alternator/server.hh" diff --git a/alternator/server.hh b/alternator/server.hh index 3c3fdfcf37e4..939df847ec7e 100644 --- a/alternator/server.hh +++ b/alternator/server.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/stats.cc b/alternator/stats.cc index 3129855beb39..0ef91d2ea6e2 100644 --- a/alternator/stats.cc +++ b/alternator/stats.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "stats.hh" diff --git a/alternator/stats.hh b/alternator/stats.hh index 426c5dabcd36..58bf891315cf 100644 --- a/alternator/stats.hh +++ b/alternator/stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/alternator/streams.cc b/alternator/streams.cc index a0e866466d8d..8bdb4f30d763 100644 --- a/alternator/streams.cc +++ b/alternator/streams.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/alternator/ttl.cc b/alternator/ttl.cc index 7eced460a4d6..8da8bb490161 100644 --- a/alternator/ttl.cc +++ b/alternator/ttl.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/alternator/ttl.hh b/alternator/ttl.hh index 72bddac571f5..60a231ab2649 100644 --- a/alternator/ttl.hh +++ b/alternator/ttl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/api-doc/swagger20_header.json b/api/api-doc/swagger20_header.json index 484d260a436e..a43e31d8f91f 100644 --- a/api/api-doc/swagger20_header.json +++ b/api/api-doc/swagger20_header.json @@ -11,8 +11,8 @@ "url": "http://scylladb.com" }, "license": { - "name": "AGPL", - "url": "https://github.com/scylladb/scylla/blob/master/LICENSE.AGPL" + "name": "ScyllaDB-Source-Available-1.0", + "url": "https://github.com/scylladb/scylla/blob/master/LICENSE-ScyllaDB-Source-Available.md" } }, "host": "{{Host}}", diff --git a/api/api.cc b/api/api.cc index 8069acc4aa9a..f3ba941f0722 100644 --- a/api/api.cc +++ b/api/api.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "api.hh" diff --git a/api/api.hh b/api/api.hh index 10ad2aa4fee7..76f9b44e4aa7 100644 --- a/api/api.hh +++ b/api/api.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/api_init.hh b/api/api_init.hh index dee938e388e6..236600c79729 100644 --- a/api/api_init.hh +++ b/api/api_init.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/authorization_cache.cc b/api/authorization_cache.cc index e8b872185a63..eabf908b1212 100644 --- a/api/authorization_cache.cc +++ b/api/authorization_cache.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "api/api-doc/authorization_cache.json.hh" diff --git a/api/authorization_cache.hh b/api/authorization_cache.hh index 4fc340a56bb8..71ae7fa4bab3 100644 --- a/api/authorization_cache.hh +++ b/api/authorization_cache.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/cache_service.cc b/api/cache_service.cc index fa16e5e95ea1..55af4b3b8341 100644 --- a/api/cache_service.cc +++ b/api/cache_service.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cache_service.hh" diff --git a/api/cache_service.hh b/api/cache_service.hh index ac438af3bce9..602b06841d9b 100644 --- a/api/cache_service.hh +++ b/api/cache_service.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/collectd.cc b/api/collectd.cc index f83417d7b733..695ed4f42695 100644 --- a/api/collectd.cc +++ b/api/collectd.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "collectd.hh" diff --git a/api/collectd.hh b/api/collectd.hh index c2efec5e206c..6239e2fd2cff 100644 --- a/api/collectd.hh +++ b/api/collectd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/column_family.cc b/api/column_family.cc index b00db050ba30..ba47c02e065f 100644 --- a/api/column_family.cc +++ b/api/column_family.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/api/column_family.hh b/api/column_family.hh index cf6d3e3b04c2..eb018dd48083 100644 --- a/api/column_family.hh +++ b/api/column_family.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/commitlog.cc b/api/commitlog.cc index 3a2ec151244d..85f404f7ce52 100644 --- a/api/commitlog.cc +++ b/api/commitlog.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "commitlog.hh" diff --git a/api/commitlog.hh b/api/commitlog.hh index 2fefef0e2104..0e86b0650138 100644 --- a/api/commitlog.hh +++ b/api/commitlog.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/compaction_manager.cc b/api/compaction_manager.cc index c36cb3bf6b5e..b0f1787f82d0 100644 --- a/api/compaction_manager.cc +++ b/api/compaction_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/api/compaction_manager.hh b/api/compaction_manager.hh index ab781fe3605b..adb0c0d17455 100644 --- a/api/compaction_manager.hh +++ b/api/compaction_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/config.cc b/api/config.cc index c4e5cc25c3d6..c0fe2f2c2be5 100644 --- a/api/config.cc +++ b/api/config.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "api/api.hh" diff --git a/api/config.hh b/api/config.hh index 06bd1e92d04c..360ad4b8cc96 100644 --- a/api/config.hh +++ b/api/config.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/cql_server_test.cc b/api/cql_server_test.cc index 9e965b797b89..77e0301d1b7d 100644 --- a/api/cql_server_test.cc +++ b/api/cql_server_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef SCYLLA_BUILD_MODE_RELEASE diff --git a/api/cql_server_test.hh b/api/cql_server_test.hh index 72d47fcb3130..2127e5393b74 100644 --- a/api/cql_server_test.hh +++ b/api/cql_server_test.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef SCYLLA_BUILD_MODE_RELEASE diff --git a/api/endpoint_snitch.cc b/api/endpoint_snitch.cc index 3490288c8bbc..4af4852b54e6 100644 --- a/api/endpoint_snitch.cc +++ b/api/endpoint_snitch.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/snitch_base.hh" diff --git a/api/endpoint_snitch.hh b/api/endpoint_snitch.hh index e5e497c5ab9f..ba884ec37ebd 100644 --- a/api/endpoint_snitch.hh +++ b/api/endpoint_snitch.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/error_injection.cc b/api/error_injection.cc index 265a81946a5b..489eb27eed8d 100644 --- a/api/error_injection.cc +++ b/api/error_injection.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "api/api-doc/error_injection.json.hh" diff --git a/api/error_injection.hh b/api/error_injection.hh index 6c8c5cad7acd..d7b63c752576 100644 --- a/api/error_injection.hh +++ b/api/error_injection.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/failure_detector.cc b/api/failure_detector.cc index 0cb72daeaa43..cb712aebc231 100644 --- a/api/failure_detector.cc +++ b/api/failure_detector.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "failure_detector.hh" diff --git a/api/failure_detector.hh b/api/failure_detector.hh index 6413994274f7..ea456ff360fe 100644 --- a/api/failure_detector.hh +++ b/api/failure_detector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/gossiper.cc b/api/gossiper.cc index b8cf5db8c659..e58076076263 100644 --- a/api/gossiper.cc +++ b/api/gossiper.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/api/gossiper.hh b/api/gossiper.hh index f98a5369467f..bb2878bd3ad3 100644 --- a/api/gossiper.hh +++ b/api/gossiper.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/hinted_handoff.cc b/api/hinted_handoff.cc index f28adebdfd06..73cd41bafff1 100644 --- a/api/hinted_handoff.cc +++ b/api/hinted_handoff.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/api/hinted_handoff.hh b/api/hinted_handoff.hh index 2f6e07d0e67d..7a83daa30d53 100644 --- a/api/hinted_handoff.hh +++ b/api/hinted_handoff.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/lsa.cc b/api/lsa.cc index 28ea8ad5fcc3..d61ed7997536 100644 --- a/api/lsa.cc +++ b/api/lsa.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "api/api-doc/lsa.json.hh" diff --git a/api/lsa.hh b/api/lsa.hh index df6321518418..228b29407f9f 100644 --- a/api/lsa.hh +++ b/api/lsa.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/messaging_service.cc b/api/messaging_service.cc index 4b2d11bfa8d9..b1665c8c5c20 100644 --- a/api/messaging_service.cc +++ b/api/messaging_service.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "messaging_service.hh" diff --git a/api/messaging_service.hh b/api/messaging_service.hh index ba305c4fe7e5..2c1cc37ffe13 100644 --- a/api/messaging_service.hh +++ b/api/messaging_service.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/raft.cc b/api/raft.cc index ab5e4c7c131d..2ac4b8345292 100644 --- a/api/raft.cc +++ b/api/raft.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/api/raft.hh b/api/raft.hh index e514a1bc22f9..d22df873cfc4 100644 --- a/api/raft.hh +++ b/api/raft.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/scrub_status.hh b/api/scrub_status.hh index 983affb58a23..ab224c41b120 100644 --- a/api/scrub_status.hh +++ b/api/scrub_status.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/storage_proxy.cc b/api/storage_proxy.cc index 27510d199f3c..88823d368680 100644 --- a/api/storage_proxy.cc +++ b/api/storage_proxy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "storage_proxy.hh" diff --git a/api/storage_proxy.hh b/api/storage_proxy.hh index 9dfed8c7b994..2d2a9018b464 100644 --- a/api/storage_proxy.hh +++ b/api/storage_proxy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/storage_service.cc b/api/storage_service.cc index 8f4a055893ab..2496b2d8c491 100644 --- a/api/storage_service.cc +++ b/api/storage_service.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "storage_service.hh" diff --git a/api/storage_service.hh b/api/storage_service.hh index 7e078288be1e..339bf8f37a6f 100644 --- a/api/storage_service.hh +++ b/api/storage_service.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/stream_manager.cc b/api/stream_manager.cc index 027a01ad6036..3008dafcb47a 100644 --- a/api/stream_manager.cc +++ b/api/stream_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "stream_manager.hh" diff --git a/api/stream_manager.hh b/api/stream_manager.hh index 25831f8d70be..91813f258a21 100644 --- a/api/stream_manager.hh +++ b/api/stream_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/system.cc b/api/system.cc index 66d004f3aad3..735216c917f0 100644 --- a/api/system.cc +++ b/api/system.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "api/api_init.hh" diff --git a/api/system.hh b/api/system.hh index d7dd8ea0e4e8..d4ee7830f696 100644 --- a/api/system.hh +++ b/api/system.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/task_manager.cc b/api/task_manager.cc index 4d9143586004..0ff7b2e7f183 100644 --- a/api/task_manager.cc +++ b/api/task_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/api/task_manager.hh b/api/task_manager.hh index 87afe855b695..7081d59ec916 100644 --- a/api/task_manager.hh +++ b/api/task_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/task_manager_test.cc b/api/task_manager_test.cc index 7905cbe31681..5353c95c5c36 100644 --- a/api/task_manager_test.cc +++ b/api/task_manager_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef SCYLLA_BUILD_MODE_RELEASE diff --git a/api/task_manager_test.hh b/api/task_manager_test.hh index 80b327d0f0f9..38b7f7f5c29f 100644 --- a/api/task_manager_test.hh +++ b/api/task_manager_test.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef SCYLLA_BUILD_MODE_RELEASE diff --git a/api/tasks.cc b/api/tasks.cc index dc1e1209ca84..1341dba19932 100644 --- a/api/tasks.cc +++ b/api/tasks.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/api/tasks.hh b/api/tasks.hh index 00206facf044..f5b2ca52479a 100644 --- a/api/tasks.hh +++ b/api/tasks.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/api/token_metadata.cc b/api/token_metadata.cc index bce2a5183c3a..a8c3234befe7 100644 --- a/api/token_metadata.cc +++ b/api/token_metadata.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "api/api.hh" diff --git a/api/token_metadata.hh b/api/token_metadata.hh index 8c8f662f5904..0bab6d999fdc 100644 --- a/api/token_metadata.hh +++ b/api/token_metadata.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/allow_all_authenticator.cc b/auth/allow_all_authenticator.cc index d6cd2c591edb..fb78d3de9ab1 100644 --- a/auth/allow_all_authenticator.cc +++ b/auth/allow_all_authenticator.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/allow_all_authenticator.hh" diff --git a/auth/allow_all_authenticator.hh b/auth/allow_all_authenticator.hh index 884338e4c900..f705887403af 100644 --- a/auth/allow_all_authenticator.hh +++ b/auth/allow_all_authenticator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/allow_all_authorizer.cc b/auth/allow_all_authorizer.cc index 0dc6f0767684..28003af11348 100644 --- a/auth/allow_all_authorizer.cc +++ b/auth/allow_all_authorizer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/allow_all_authorizer.hh" diff --git a/auth/allow_all_authorizer.hh b/auth/allow_all_authorizer.hh index d297c883d367..ad790cb76037 100644 --- a/auth/allow_all_authorizer.hh +++ b/auth/allow_all_authorizer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/authenticated_user.cc b/auth/authenticated_user.cc index c2194e0a6cf9..e11f7eb5edd2 100644 --- a/auth/authenticated_user.cc +++ b/auth/authenticated_user.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/authenticated_user.hh" diff --git a/auth/authenticated_user.hh b/auth/authenticated_user.hh index b00d2da78293..0d295547bc22 100644 --- a/auth/authenticated_user.hh +++ b/auth/authenticated_user.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/authentication_options.hh b/auth/authentication_options.hh index cd5daf2dc618..d1961dccd39d 100644 --- a/auth/authentication_options.hh +++ b/auth/authentication_options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/authenticator.cc b/auth/authenticator.cc index 38ab93a37a75..500556fa040c 100644 --- a/auth/authenticator.cc +++ b/auth/authenticator.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/authenticator.hh" diff --git a/auth/authenticator.hh b/auth/authenticator.hh index 45de9a3d4020..c0e6a51e4331 100644 --- a/auth/authenticator.hh +++ b/auth/authenticator.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/authorizer.hh b/auth/authorizer.hh index 0be945aa2e2c..80cfec72e642 100644 --- a/auth/authorizer.hh +++ b/auth/authorizer.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/certificate_authenticator.cc b/auth/certificate_authenticator.cc index cd16a45ecfc0..0bcebec9c58b 100644 --- a/auth/certificate_authenticator.cc +++ b/auth/certificate_authenticator.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/certificate_authenticator.hh" diff --git a/auth/certificate_authenticator.hh b/auth/certificate_authenticator.hh index 6aa27e700b18..ee08918db66c 100644 --- a/auth/certificate_authenticator.hh +++ b/auth/certificate_authenticator.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/common.cc b/auth/common.cc index 5dcefb05abb2..c5ce58f8c1c5 100644 --- a/auth/common.cc +++ b/auth/common.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/common.hh" diff --git a/auth/common.hh b/auth/common.hh index 09f640fe4275..48e5b5214669 100644 --- a/auth/common.hh +++ b/auth/common.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/default_authorizer.cc b/auth/default_authorizer.cc index 2dafd4e39c2a..260896e989d4 100644 --- a/auth/default_authorizer.cc +++ b/auth/default_authorizer.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/default_authorizer.hh" diff --git a/auth/default_authorizer.hh b/auth/default_authorizer.hh index 8ed500d03284..6b9cdccdbb43 100644 --- a/auth/default_authorizer.hh +++ b/auth/default_authorizer.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/maintenance_socket_role_manager.cc b/auth/maintenance_socket_role_manager.cc index ca4985b697ca..c939ed6382c3 100644 --- a/auth/maintenance_socket_role_manager.cc +++ b/auth/maintenance_socket_role_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/maintenance_socket_role_manager.hh" diff --git a/auth/maintenance_socket_role_manager.hh b/auth/maintenance_socket_role_manager.hh index b0107453680e..d17c510abe4f 100644 --- a/auth/maintenance_socket_role_manager.hh +++ b/auth/maintenance_socket_role_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/password_authenticator.cc b/auth/password_authenticator.cc index d546cfa0ea4e..825550d86f6d 100644 --- a/auth/password_authenticator.cc +++ b/auth/password_authenticator.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/password_authenticator.hh" diff --git a/auth/password_authenticator.hh b/auth/password_authenticator.hh index 99238ab99273..9effb11ed565 100644 --- a/auth/password_authenticator.hh +++ b/auth/password_authenticator.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/passwords.cc b/auth/passwords.cc index 87e8aff293dc..6bd49a31730c 100644 --- a/auth/passwords.cc +++ b/auth/passwords.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/passwords.hh" diff --git a/auth/passwords.hh b/auth/passwords.hh index 603fd742d323..88ce9af75a05 100644 --- a/auth/passwords.hh +++ b/auth/passwords.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/permission.cc b/auth/permission.cc index 1594ea3256f7..1404896b8c5d 100644 --- a/auth/permission.cc +++ b/auth/permission.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/permission.hh" diff --git a/auth/permission.hh b/auth/permission.hh index 013b5e90ab97..de7c5f092701 100644 --- a/auth/permission.hh +++ b/auth/permission.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/permissions_cache.cc b/auth/permissions_cache.cc index 0f38cb4d036b..bc3132607e7f 100644 --- a/auth/permissions_cache.cc +++ b/auth/permissions_cache.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/permissions_cache.hh" diff --git a/auth/permissions_cache.hh b/auth/permissions_cache.hh index 09099ff596d5..6b3096cc6c26 100644 --- a/auth/permissions_cache.hh +++ b/auth/permissions_cache.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/resource.cc b/auth/resource.cc index e0c7c1cd5ac0..0e486bd6613d 100644 --- a/auth/resource.cc +++ b/auth/resource.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/resource.hh" diff --git a/auth/resource.hh b/auth/resource.hh index 959ed8599172..866023188014 100644 --- a/auth/resource.hh +++ b/auth/resource.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/role_manager.hh b/auth/role_manager.hh index f52188321d36..983e398ad085 100644 --- a/auth/role_manager.hh +++ b/auth/role_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/role_or_anonymous.cc b/auth/role_or_anonymous.cc index 0efdb7e81312..9d8a82edce22 100644 --- a/auth/role_or_anonymous.cc +++ b/auth/role_or_anonymous.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/role_or_anonymous.hh" diff --git a/auth/role_or_anonymous.hh b/auth/role_or_anonymous.hh index 7015ff25891d..c96b9e7523a0 100644 --- a/auth/role_or_anonymous.hh +++ b/auth/role_or_anonymous.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/roles-metadata.cc b/auth/roles-metadata.cc index b0f007bf9832..1f13151c5851 100644 --- a/auth/roles-metadata.cc +++ b/auth/roles-metadata.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/roles-metadata.hh" diff --git a/auth/roles-metadata.hh b/auth/roles-metadata.hh index aec5d95c27e1..4d6490fb013e 100644 --- a/auth/roles-metadata.hh +++ b/auth/roles-metadata.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/sasl_challenge.cc b/auth/sasl_challenge.cc index ad07d5313702..27796d35e6c0 100644 --- a/auth/sasl_challenge.cc +++ b/auth/sasl_challenge.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/sasl_challenge.hh" diff --git a/auth/sasl_challenge.hh b/auth/sasl_challenge.hh index 9e95bd58b41a..ee2d943f0b75 100644 --- a/auth/sasl_challenge.hh +++ b/auth/sasl_challenge.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/auth/service.cc b/auth/service.cc index 2ec560e5f0b7..94174b0f9ec1 100644 --- a/auth/service.cc +++ b/auth/service.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/auth/service.hh b/auth/service.hh index ad34e814fd8f..73765992b085 100644 --- a/auth/service.hh +++ b/auth/service.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/standard_role_manager.cc b/auth/standard_role_manager.cc index aba9e11ae176..989f06e8a0e1 100644 --- a/auth/standard_role_manager.cc +++ b/auth/standard_role_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/standard_role_manager.hh" diff --git a/auth/standard_role_manager.hh b/auth/standard_role_manager.hh index 3d0024dbf778..fb79d940b480 100644 --- a/auth/standard_role_manager.hh +++ b/auth/standard_role_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/auth/transitional.cc b/auth/transitional.cc index ab3ec7f0e3a6..3f7b895bff46 100644 --- a/auth/transitional.cc +++ b/auth/transitional.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "auth/authenticated_user.hh" diff --git a/backlog_controller.hh b/backlog_controller.hh index 6320c1ce89ea..2546fd56cf5f 100644 --- a/backlog_controller.hh +++ b/backlog_controller.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/bin/cqlsh b/bin/cqlsh index ceef5f66d912..c69966b97fb4 100755 --- a/bin/cqlsh +++ b/bin/cqlsh @@ -1,7 +1,7 @@ #!/bin/bash # Copyright (C) 2023-present ScyllaDB -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 here=$(dirname "$0") exec "$here/../tools/cqlsh/bin/cqlsh.py" "$@" diff --git a/bin/nodetool b/bin/nodetool index 3aba70c9135f..bee1d07dc5d3 100755 --- a/bin/nodetool +++ b/bin/nodetool @@ -3,7 +3,7 @@ # Copyright (C) 2024-present ScyllaDB # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # SCRIPT_PATH=$(dirname $(realpath "$0")) diff --git a/build_mode.hh b/build_mode.hh index c8f287aa9940..3b1abdcaa670 100644 --- a/build_mode.hh +++ b/build_mode.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/bytes.cc b/bytes.cc index 8700f6b70eff..78cdc9addf18 100644 --- a/bytes.cc +++ b/bytes.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "bytes.hh" diff --git a/bytes.hh b/bytes.hh index 19364785047c..1891cc6ae2c8 100644 --- a/bytes.hh +++ b/bytes.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/bytes_fwd.hh b/bytes_fwd.hh index d5517796572c..f20bbb26ae64 100644 --- a/bytes_fwd.hh +++ b/bytes_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/bytes_ostream.hh b/bytes_ostream.hh index 962ca5ecb225..9f62e84ac805 100644 --- a/bytes_ostream.hh +++ b/bytes_ostream.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cache_mutation_reader.hh b/cache_mutation_reader.hh index d70eac076ab6..79bd5443d7d0 100644 --- a/cache_mutation_reader.hh +++ b/cache_mutation_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cache_temperature.hh b/cache_temperature.hh index 206732ba25e7..2f34c30dd7ae 100644 --- a/cache_temperature.hh +++ b/cache_temperature.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cartesian_product.hh b/cartesian_product.hh index 4a31fd4eb13b..f924f409ae8c 100644 --- a/cartesian_product.hh +++ b/cartesian_product.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/cdc_extension.hh b/cdc/cdc_extension.hh index f7f922103d89..d85511a64734 100644 --- a/cdc/cdc_extension.hh +++ b/cdc/cdc_extension.hh @@ -2,7 +2,7 @@ * Copyright 2020-present ScyllaDB */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/cdc_options.hh b/cdc/cdc_options.hh index c7408cc7a107..2752944a7d76 100644 --- a/cdc/cdc_options.hh +++ b/cdc/cdc_options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/cdc_partitioner.cc b/cdc/cdc_partitioner.cc index a91948004a33..9edcb564db0c 100644 --- a/cdc/cdc_partitioner.cc +++ b/cdc/cdc_partitioner.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cdc_partitioner.hh" diff --git a/cdc/cdc_partitioner.hh b/cdc/cdc_partitioner.hh index 5d600d06cc9f..20a1a08d31b4 100644 --- a/cdc/cdc_partitioner.hh +++ b/cdc/cdc_partitioner.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/change_visitor.hh b/cdc/change_visitor.hh index 97c18ffad161..633f66fa6581 100644 --- a/cdc/change_visitor.hh +++ b/cdc/change_visitor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/generation.cc b/cdc/generation.cc index 0773b773d2a1..8ace3cea7e28 100644 --- a/cdc/generation.cc +++ b/cdc/generation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/cdc/generation.hh b/cdc/generation.hh index e2b0bee63088..417c617c09ae 100644 --- a/cdc/generation.hh +++ b/cdc/generation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/cdc/generation_id.hh b/cdc/generation_id.hh index 0fb2ea03c620..ce6df3243206 100644 --- a/cdc/generation_id.hh +++ b/cdc/generation_id.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/log.cc b/cdc/log.cc index 21c390a8a8ed..3aab6ee57479 100644 --- a/cdc/log.cc +++ b/cdc/log.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/cdc/log.hh b/cdc/log.hh index 2b01e70e33d8..13b84bdbad4e 100644 --- a/cdc/log.hh +++ b/cdc/log.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/cdc/metadata.cc b/cdc/metadata.cc index e765ee678130..e1990a165eeb 100644 --- a/cdc/metadata.cc +++ b/cdc/metadata.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "dht/token-sharding.hh" diff --git a/cdc/metadata.hh b/cdc/metadata.hh index 333a84bc740a..a7b893ee4a7e 100644 --- a/cdc/metadata.hh +++ b/cdc/metadata.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/split.cc b/cdc/split.cc index f1f0348b2787..88023f748811 100644 --- a/cdc/split.cc +++ b/cdc/split.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation/mutation.hh" diff --git a/cdc/split.hh b/cdc/split.hh index 5daebb5ad136..6971c699e0a6 100644 --- a/cdc/split.hh +++ b/cdc/split.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cdc/stats.hh b/cdc/stats.hh index c11e00cbe6de..34300db4a3d9 100644 --- a/cdc/stats.hh +++ b/cdc/stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cell_locking.hh b/cell_locking.hh index 05be31fd6dbe..7a41be7e2501 100644 --- a/cell_locking.hh +++ b/cell_locking.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/checked-file-impl.hh b/checked-file-impl.hh index 4580ebbdb994..5259996d541a 100644 --- a/checked-file-impl.hh +++ b/checked-file-impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/client_data.cc b/client_data.cc index 91270c2df340..870f5c930a82 100644 --- a/client_data.cc +++ b/client_data.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "client_data.hh" diff --git a/client_data.hh b/client_data.hh index 36515ef3da95..91a77c13b746 100644 --- a/client_data.hh +++ b/client_data.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/clocks-impl.cc b/clocks-impl.cc index 2848a22c44db..53ab60709033 100644 --- a/clocks-impl.cc +++ b/clocks-impl.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/clocks-impl.hh b/clocks-impl.hh index 87861a4119d2..3a01cd209b6d 100644 --- a/clocks-impl.hh +++ b/clocks-impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/clustering_bounds_comparator.hh b/clustering_bounds_comparator.hh index f9746d5fa4dc..cb768487f1f7 100644 --- a/clustering_bounds_comparator.hh +++ b/clustering_bounds_comparator.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/clustering_interval_set.hh b/clustering_interval_set.hh index 977478db33e0..1fda6a9869ad 100644 --- a/clustering_interval_set.hh +++ b/clustering_interval_set.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/clustering_key_filter.hh b/clustering_key_filter.hh index ef1cf1aa9715..f26c2ba1f32c 100644 --- a/clustering_key_filter.hh +++ b/clustering_key_filter.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/clustering_ranges_walker.hh b/clustering_ranges_walker.hh index 82120d7cfed9..24d90bc26bff 100644 --- a/clustering_ranges_walker.hh +++ b/clustering_ranges_walker.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cmake/FindANTLR3.cmake b/cmake/FindANTLR3.cmake index b484ef99a189..6f7e779722f3 100644 --- a/cmake/FindANTLR3.cmake +++ b/cmake/FindANTLR3.cmake @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # find_path(ANTLR3_INCLUDE_DIR NAMES antlr3.hpp diff --git a/cmake/FindSeastar.cmake b/cmake/FindSeastar.cmake index b47fefb232d0..b9c8ba294ae5 100644 --- a/cmake/FindSeastar.cmake +++ b/cmake/FindSeastar.cmake @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # find_package(PkgConfig QUIET REQUIRED) diff --git a/cmake/Findcryptopp.cmake b/cmake/Findcryptopp.cmake index 713a703fdeb5..d5f7b6930458 100644 --- a/cmake/Findcryptopp.cmake +++ b/cmake/Findcryptopp.cmake @@ -1,7 +1,7 @@ # # Copyright (C) 2018-present ScyllaDB # -# SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +# SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) # find_library (cryptopp_LIBRARY diff --git a/cmake/Findlibdeflate.cmake b/cmake/Findlibdeflate.cmake index 53844e96133a..58826719701a 100644 --- a/cmake/Findlibdeflate.cmake +++ b/cmake/Findlibdeflate.cmake @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # find_package (PkgConfig REQUIRED) diff --git a/cmake/Findlibxcrypt.cmake b/cmake/Findlibxcrypt.cmake index 76d08883be09..efb1c8513a30 100644 --- a/cmake/Findlibxcrypt.cmake +++ b/cmake/Findlibxcrypt.cmake @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # find_package(PkgConfig REQUIRED) diff --git a/cmake/Findrapidxml.cmake b/cmake/Findrapidxml.cmake index f4addfcd64f8..76bdabcca85b 100644 --- a/cmake/Findrapidxml.cmake +++ b/cmake/Findrapidxml.cmake @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # find_path(rapidxml_INCLUDE_DIR NAMES rapidxml.h rapidxml/rapidxml.hpp) diff --git a/cmake/FindxxHash.cmake b/cmake/FindxxHash.cmake index 2af8e4c7322c..901ed9958f23 100644 --- a/cmake/FindxxHash.cmake +++ b/cmake/FindxxHash.cmake @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # find_package(PkgConfig REQUIRED) diff --git a/cmake/Findzstd.cmake b/cmake/Findzstd.cmake index aaac2474d8bc..22be8e04b3f0 100644 --- a/cmake/Findzstd.cmake +++ b/cmake/Findzstd.cmake @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # find_package (PkgConfig REQUIRED) diff --git a/collection_mutation.cc b/collection_mutation.cc index d6f6b6674204..75bc8bad18ba 100644 --- a/collection_mutation.cc +++ b/collection_mutation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/collection_mutation.hh b/collection_mutation.hh index 213a2bf85369..914a7d93e4d8 100644 --- a/collection_mutation.hh +++ b/collection_mutation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/column_computation.hh b/column_computation.hh index edcbc07f575d..256bb654fcca 100644 --- a/column_computation.hh +++ b/column_computation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/combine.hh b/combine.hh index 9ccf495e571b..4d37d740e6b6 100644 --- a/combine.hh +++ b/combine.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction.cc b/compaction/compaction.cc index 9b161e8e89eb..de04659b4ad7 100644 --- a/compaction/compaction.cc +++ b/compaction/compaction.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ /* diff --git a/compaction/compaction.hh b/compaction/compaction.hh index 7b17e92dc14d..cd8624ffeb2a 100644 --- a/compaction/compaction.hh +++ b/compaction/compaction.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_backlog_manager.hh b/compaction/compaction_backlog_manager.hh index b5ad3a5648bd..51b458ff6cc2 100644 --- a/compaction/compaction_backlog_manager.hh +++ b/compaction/compaction_backlog_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_descriptor.hh b/compaction/compaction_descriptor.hh index a13143c27af0..07e250180481 100644 --- a/compaction/compaction_descriptor.hh +++ b/compaction/compaction_descriptor.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_fwd.hh b/compaction/compaction_fwd.hh index f4a5777699b7..b24fcfa32a6f 100644 --- a/compaction/compaction_fwd.hh +++ b/compaction/compaction_fwd.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_garbage_collector.hh b/compaction/compaction_garbage_collector.hh index 39f6cd60f6ce..78550d8df3bf 100644 --- a/compaction/compaction_garbage_collector.hh +++ b/compaction/compaction_garbage_collector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_manager.cc b/compaction/compaction_manager.cc index 9f6900cc1c3b..a0c05a0a219a 100644 --- a/compaction/compaction_manager.cc +++ b/compaction/compaction_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "compaction_manager.hh" diff --git a/compaction/compaction_manager.hh b/compaction/compaction_manager.hh index 31e7a4560420..2f9984a29c36 100644 --- a/compaction/compaction_manager.hh +++ b/compaction/compaction_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_state.hh b/compaction/compaction_state.hh index 47bf58f844ff..c2a49371fa65 100644 --- a/compaction/compaction_state.hh +++ b/compaction/compaction_state.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_strategy.cc b/compaction/compaction_strategy.cc index c6cdc8d0be46..315ac35f676a 100644 --- a/compaction/compaction_strategy.cc +++ b/compaction/compaction_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ /* diff --git a/compaction/compaction_strategy.hh b/compaction/compaction_strategy.hh index b1612dfd98f5..0ea40b167206 100644 --- a/compaction/compaction_strategy.hh +++ b/compaction/compaction_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_strategy_impl.hh b/compaction/compaction_strategy_impl.hh index 3a8d44636bfb..0dddfe3a2f6f 100644 --- a/compaction/compaction_strategy_impl.hh +++ b/compaction/compaction_strategy_impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_strategy_state.hh b/compaction/compaction_strategy_state.hh index 33af485d27b2..969a0f061a2d 100644 --- a/compaction/compaction_strategy_state.hh +++ b/compaction/compaction_strategy_state.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_strategy_type.hh b/compaction/compaction_strategy_type.hh index 462d586f9f11..87d4e8d073e8 100644 --- a/compaction/compaction_strategy_type.hh +++ b/compaction/compaction_strategy_type.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/compaction_weight_registration.hh b/compaction/compaction_weight_registration.hh index 0acb7479d8ae..39338edb6f0c 100644 --- a/compaction/compaction_weight_registration.hh +++ b/compaction/compaction_weight_registration.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/leveled_compaction_strategy.cc b/compaction/leveled_compaction_strategy.cc index 6c25210dc230..a2d67ac4eba3 100644 --- a/compaction/leveled_compaction_strategy.cc +++ b/compaction/leveled_compaction_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "leveled_compaction_strategy.hh" diff --git a/compaction/leveled_compaction_strategy.hh b/compaction/leveled_compaction_strategy.hh index 3d38286b54ac..e0ce467252db 100644 --- a/compaction/leveled_compaction_strategy.hh +++ b/compaction/leveled_compaction_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/leveled_manifest.hh b/compaction/leveled_manifest.hh index ab20ffc902ff..5678f957cd4a 100644 --- a/compaction/leveled_manifest.hh +++ b/compaction/leveled_manifest.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/compaction/size_tiered_backlog_tracker.hh b/compaction/size_tiered_backlog_tracker.hh index 901c6e82e9ba..bf8466e0131c 100644 --- a/compaction/size_tiered_backlog_tracker.hh +++ b/compaction/size_tiered_backlog_tracker.hh @@ -2,7 +2,7 @@ * Copyright (C) 2018-present ScyllaDB */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/size_tiered_compaction_strategy.cc b/compaction/size_tiered_compaction_strategy.cc index bf700ff2c44e..933c5fcbd8f8 100644 --- a/compaction/size_tiered_compaction_strategy.cc +++ b/compaction/size_tiered_compaction_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/compaction/size_tiered_compaction_strategy.hh b/compaction/size_tiered_compaction_strategy.hh index 09efad8ca3d0..317e2cb5d2b2 100644 --- a/compaction/size_tiered_compaction_strategy.hh +++ b/compaction/size_tiered_compaction_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/strategy_control.hh b/compaction/strategy_control.hh index cb9dd7e225ad..70df6ba9c039 100644 --- a/compaction/strategy_control.hh +++ b/compaction/strategy_control.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/table_state.hh b/compaction/table_state.hh index e65af78af281..ac440651481b 100644 --- a/compaction/table_state.hh +++ b/compaction/table_state.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/task_manager_module.cc b/compaction/task_manager_module.cc index 851d4798ce3c..58349f13b5e6 100644 --- a/compaction/task_manager_module.cc +++ b/compaction/task_manager_module.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/compaction/task_manager_module.hh b/compaction/task_manager_module.hh index 9c1d9b1a0919..38334c7f2560 100644 --- a/compaction/task_manager_module.hh +++ b/compaction/task_manager_module.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compaction/time_window_compaction_strategy.cc b/compaction/time_window_compaction_strategy.cc index 157d82e1897f..3c6bd1713cd5 100644 --- a/compaction/time_window_compaction_strategy.cc +++ b/compaction/time_window_compaction_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "time_window_compaction_strategy.hh" diff --git a/compaction/time_window_compaction_strategy.hh b/compaction/time_window_compaction_strategy.hh index 73e722576f5a..ab3b8af5b359 100644 --- a/compaction/time_window_compaction_strategy.hh +++ b/compaction/time_window_compaction_strategy.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/compound.hh b/compound.hh index 83c0ab912094..f395fc343b1a 100644 --- a/compound.hh +++ b/compound.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compound_compat.hh b/compound_compat.hh index 50e29f41e59d..8ef1d4fb6c29 100644 --- a/compound_compat.hh +++ b/compound_compat.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/compress.cc b/compress.cc index 306b4d003d23..7fcf540b0bc9 100644 --- a/compress.cc +++ b/compress.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/compress.hh b/compress.hh index 42396b06f22c..e468b5a13ade 100644 --- a/compress.hh +++ b/compress.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/concrete_types.hh b/concrete_types.hh index 56be973392e5..b3a1c5dbded6 100644 --- a/concrete_types.hh +++ b/concrete_types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/configure.py b/configure.py index 8df146d7e56b..6d05746b3bee 100755 --- a/configure.py +++ b/configure.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/converting_mutation_partition_applier.cc b/converting_mutation_partition_applier.cc index 0ba50235a68c..f7b72200a3bc 100644 --- a/converting_mutation_partition_applier.cc +++ b/converting_mutation_partition_applier.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/converting_mutation_partition_applier.hh b/converting_mutation_partition_applier.hh index 0aaaad64fde7..22949d866873 100644 --- a/converting_mutation_partition_applier.hh +++ b/converting_mutation_partition_applier.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/counters.cc b/counters.cc index d46cccae884b..e289f7a1e513 100644 --- a/counters.cc +++ b/counters.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/counters.hh b/counters.hh index 426347a32fc9..6d20462c10c4 100644 --- a/counters.hh +++ b/counters.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/assignment_testable.hh b/cql3/assignment_testable.hh index c0521ee4df01..5455cd61b5a2 100644 --- a/cql3/assignment_testable.hh +++ b/cql3/assignment_testable.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/attributes.cc b/cql3/attributes.cc index 698f24c1a6ac..99398500c83d 100644 --- a/cql3/attributes.cc +++ b/cql3/attributes.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/attributes.hh" diff --git a/cql3/attributes.hh b/cql3/attributes.hh index 473be768478e..27e6e73149aa 100644 --- a/cql3/attributes.hh +++ b/cql3/attributes.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/authorized_prepared_statements_cache.hh b/cql3/authorized_prepared_statements_cache.hh index ef111abb33a9..18320862db40 100644 --- a/cql3/authorized_prepared_statements_cache.hh +++ b/cql3/authorized_prepared_statements_cache.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/cf_name.cc b/cql3/cf_name.cc index cd1c6baf5079..f5ec077b81a4 100644 --- a/cql3/cf_name.cc +++ b/cql3/cf_name.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/cf_name.hh" diff --git a/cql3/cf_name.hh b/cql3/cf_name.hh index a2a6a7abc678..da7391bc744f 100644 --- a/cql3/cf_name.hh +++ b/cql3/cf_name.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/column_identifier.cc b/cql3/column_identifier.cc index e25cdb77a8b1..43d1340d51f5 100644 --- a/cql3/column_identifier.cc +++ b/cql3/column_identifier.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/column_identifier.hh" diff --git a/cql3/column_identifier.hh b/cql3/column_identifier.hh index 66393dd7e4c7..1048673c5aa8 100644 --- a/cql3/column_identifier.hh +++ b/cql3/column_identifier.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/column_specification.cc b/cql3/column_specification.cc index 12d04184857e..ccc8928b36da 100644 --- a/cql3/column_specification.cc +++ b/cql3/column_specification.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/column_specification.hh b/cql3/column_specification.hh index bda754d30b91..7090c5ed27ba 100644 --- a/cql3/column_specification.hh +++ b/cql3/column_specification.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/constants.cc b/cql3/constants.cc index a58a6358b17e..f47a6bf45829 100644 --- a/cql3/constants.cc +++ b/cql3/constants.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/constants.hh" diff --git a/cql3/constants.hh b/cql3/constants.hh index d98f9b8f965e..090f0907290c 100644 --- a/cql3/constants.hh +++ b/cql3/constants.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/cql3_type.cc b/cql3/cql3_type.cc index 1e660b2c7066..dfd10b5f5f8a 100644 --- a/cql3/cql3_type.cc +++ b/cql3/cql3_type.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/cql3/cql3_type.hh b/cql3/cql3_type.hh index aad834032a4e..23546522a894 100644 --- a/cql3/cql3_type.hh +++ b/cql3/cql3_type.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/cql_config.hh b/cql3/cql_config.hh index 0164bf334d55..06a3b726071d 100644 --- a/cql3/cql_config.hh +++ b/cql3/cql_config.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/cql3/cql_statement.hh b/cql3/cql_statement.hh index affd0a2b933b..951eaa2dfd4b 100644 --- a/cql3/cql_statement.hh +++ b/cql3/cql_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/description.cc b/cql3/description.cc index c06ac5dcd45f..94d798a579b4 100644 --- a/cql3/description.cc +++ b/cql3/description.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/description.hh" diff --git a/cql3/description.hh b/cql3/description.hh index 5bed6266809c..d679ea315ba9 100644 --- a/cql3/description.hh +++ b/cql3/description.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/dialect.hh b/cql3/dialect.hh index b107dfd0dcca..82bf0a7ea235 100644 --- a/cql3/dialect.hh +++ b/cql3/dialect.hh @@ -1,5 +1,5 @@ // Copyright (C) 2024-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/cql3/error_collector.hh b/cql3/error_collector.hh index b5d291a6ff27..1f1f7046c0a9 100644 --- a/cql3/error_collector.hh +++ b/cql3/error_collector.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/error_listener.hh b/cql3/error_listener.hh index 3c9fb5a60ef7..b222a57ead45 100644 --- a/cql3/error_listener.hh +++ b/cql3/error_listener.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/expr/evaluate.hh b/cql3/expr/evaluate.hh index 44571baa1b34..f39b42d7af54 100644 --- a/cql3/expr/evaluate.hh +++ b/cql3/expr/evaluate.hh @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/cql3/expr/expr-utils.hh b/cql3/expr/expr-utils.hh index 3cd070daf930..d31618513db6 100644 --- a/cql3/expr/expr-utils.hh +++ b/cql3/expr/expr-utils.hh @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/cql3/expr/expression.cc b/cql3/expr/expression.cc index dc18e7e19f6a..3e9335a995c8 100644 --- a/cql3/expr/expression.cc +++ b/cql3/expr/expression.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "expression.hh" diff --git a/cql3/expr/expression.hh b/cql3/expr/expression.hh index 2926b3fb9a56..a9273a682ea3 100644 --- a/cql3/expr/expression.hh +++ b/cql3/expr/expression.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/expr/prepare_expr.cc b/cql3/expr/prepare_expr.cc index 0229d017efc5..7a5f9adae747 100644 --- a/cql3/expr/prepare_expr.cc +++ b/cql3/expr/prepare_expr.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "expression.hh" diff --git a/cql3/expr/restrictions.cc b/cql3/expr/restrictions.cc index 33d8770d40d7..7c6fee8da793 100644 --- a/cql3/expr/restrictions.cc +++ b/cql3/expr/restrictions.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/statements/request_validations.hh" diff --git a/cql3/expr/restrictions.hh b/cql3/expr/restrictions.hh index 9e5a36c1610f..093ce6ffa081 100644 --- a/cql3/expr/restrictions.hh +++ b/cql3/expr/restrictions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/expr/unset.hh b/cql3/expr/unset.hh index 12bbeb0314fa..2f9bdb48ccdc 100644 --- a/cql3/expr/unset.hh +++ b/cql3/expr/unset.hh @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +// SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) #pragma once diff --git a/cql3/functions/abstract_function.hh b/cql3/functions/abstract_function.hh index 4a08b190b97a..37b204f52aa8 100644 --- a/cql3/functions/abstract_function.hh +++ b/cql3/functions/abstract_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/aggregate_fcts.cc b/cql3/functions/aggregate_fcts.cc index ac9c06eecd9e..4db447ffbcb2 100644 --- a/cql3/functions/aggregate_fcts.cc +++ b/cql3/functions/aggregate_fcts.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "bytes.hh" diff --git a/cql3/functions/aggregate_fcts.hh b/cql3/functions/aggregate_fcts.hh index 3c77f05a9287..bece5f067c54 100644 --- a/cql3/functions/aggregate_fcts.hh +++ b/cql3/functions/aggregate_fcts.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/aggregate_function.hh b/cql3/functions/aggregate_function.hh index fef9c3449a38..772162b65327 100644 --- a/cql3/functions/aggregate_function.hh +++ b/cql3/functions/aggregate_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/as_json_function.hh b/cql3/functions/as_json_function.hh index 76e62f67a3c2..1cbf1ec376b7 100644 --- a/cql3/functions/as_json_function.hh +++ b/cql3/functions/as_json_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/bytes_conversion_fcts.hh b/cql3/functions/bytes_conversion_fcts.hh index 50442fc02f3f..aa5723a4efeb 100644 --- a/cql3/functions/bytes_conversion_fcts.hh +++ b/cql3/functions/bytes_conversion_fcts.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/castas_fcts.cc b/cql3/functions/castas_fcts.cc index eddfd44ddabe..8afa5d02541b 100644 --- a/cql3/functions/castas_fcts.cc +++ b/cql3/functions/castas_fcts.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "castas_fcts.hh" diff --git a/cql3/functions/castas_fcts.hh b/cql3/functions/castas_fcts.hh index efaa7a5aa721..53b4f3ed4faf 100644 --- a/cql3/functions/castas_fcts.hh +++ b/cql3/functions/castas_fcts.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/error_injection_fcts.cc b/cql3/functions/error_injection_fcts.cc index 8df90a21a71f..17ea60a20de0 100644 --- a/cql3/functions/error_injection_fcts.cc +++ b/cql3/functions/error_injection_fcts.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "error_injection_fcts.hh" diff --git a/cql3/functions/error_injection_fcts.hh b/cql3/functions/error_injection_fcts.hh index 061bd043649c..d33a37ec0962 100644 --- a/cql3/functions/error_injection_fcts.hh +++ b/cql3/functions/error_injection_fcts.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/functions/first_function.hh b/cql3/functions/first_function.hh index 97cb1d0a69f3..74c523529310 100644 --- a/cql3/functions/first_function.hh +++ b/cql3/functions/first_function.hh @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/cql3/functions/function.hh b/cql3/functions/function.hh index 186cfe15e307..75f2563a1aa2 100644 --- a/cql3/functions/function.hh +++ b/cql3/functions/function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/function_name.hh b/cql3/functions/function_name.hh index 530e9ed30b1d..5124ef0ba97b 100644 --- a/cql3/functions/function_name.hh +++ b/cql3/functions/function_name.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/functions.cc b/cql3/functions/functions.cc index a5593cddc90b..6218b5bc8833 100644 --- a/cql3/functions/functions.cc +++ b/cql3/functions/functions.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/cql3/functions/functions.hh b/cql3/functions/functions.hh index 93818c4aad78..f8ce98bf8e0a 100644 --- a/cql3/functions/functions.hh +++ b/cql3/functions/functions.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/native_function.hh b/cql3/functions/native_function.hh index 2e4799300943..3672ad23b279 100644 --- a/cql3/functions/native_function.hh +++ b/cql3/functions/native_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/native_scalar_function.hh b/cql3/functions/native_scalar_function.hh index 7bac62fb2355..3f4e1deb1407 100644 --- a/cql3/functions/native_scalar_function.hh +++ b/cql3/functions/native_scalar_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/scalar_function.hh b/cql3/functions/scalar_function.hh index 67a95e5c3e8f..ce4a82c3cb68 100644 --- a/cql3/functions/scalar_function.hh +++ b/cql3/functions/scalar_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/time_uuid_fcts.hh b/cql3/functions/time_uuid_fcts.hh index e2234e6aca31..0f1de0de7051 100644 --- a/cql3/functions/time_uuid_fcts.hh +++ b/cql3/functions/time_uuid_fcts.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/token_fct.hh b/cql3/functions/token_fct.hh index a552526d714f..cf9a3db7a488 100644 --- a/cql3/functions/token_fct.hh +++ b/cql3/functions/token_fct.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/functions/user_aggregate.hh b/cql3/functions/user_aggregate.hh index ba930ab7bcb0..1c04017ed30b 100644 --- a/cql3/functions/user_aggregate.hh +++ b/cql3/functions/user_aggregate.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/functions/user_function.cc b/cql3/functions/user_function.cc index d36d9ba690cf..8b6e9f1e4e15 100644 --- a/cql3/functions/user_function.cc +++ b/cql3/functions/user_function.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "user_function.hh" diff --git a/cql3/functions/user_function.hh b/cql3/functions/user_function.hh index 1ce36848a32d..934fd0e4eb16 100644 --- a/cql3/functions/user_function.hh +++ b/cql3/functions/user_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/functions/uuid_fcts.hh b/cql3/functions/uuid_fcts.hh index c4bb3610700c..a7e12d6fd5eb 100644 --- a/cql3/functions/uuid_fcts.hh +++ b/cql3/functions/uuid_fcts.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/index_name.cc b/cql3/index_name.cc index e9cac5c28fb3..cf0e969074e5 100644 --- a/cql3/index_name.cc +++ b/cql3/index_name.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/index_name.hh" diff --git a/cql3/index_name.hh b/cql3/index_name.hh index 9a04806f0936..c2d6896bc470 100644 --- a/cql3/index_name.hh +++ b/cql3/index_name.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/keyspace_element_name.cc b/cql3/keyspace_element_name.cc index ba47e2e8b639..9f27140baa8e 100644 --- a/cql3/keyspace_element_name.cc +++ b/cql3/keyspace_element_name.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/keyspace_element_name.hh b/cql3/keyspace_element_name.hh index a69891975f5b..aa8df2f5e16e 100644 --- a/cql3/keyspace_element_name.hh +++ b/cql3/keyspace_element_name.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/lists.cc b/cql3/lists.cc index f838e673cf2a..0228d9177dfd 100644 --- a/cql3/lists.cc +++ b/cql3/lists.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "lists.hh" diff --git a/cql3/lists.hh b/cql3/lists.hh index 0a81cf79642b..c13ee80b3a39 100644 --- a/cql3/lists.hh +++ b/cql3/lists.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/maps.cc b/cql3/maps.cc index 7f8c36723e57..fd222f48b84d 100644 --- a/cql3/maps.cc +++ b/cql3/maps.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/maps.hh b/cql3/maps.hh index 246955744e0c..fbf277e02594 100644 --- a/cql3/maps.hh +++ b/cql3/maps.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/operation.cc b/cql3/operation.cc index 048221095e3b..4c849641a4cb 100644 --- a/cql3/operation.cc +++ b/cql3/operation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/operation.hh b/cql3/operation.hh index e94804b3d6d0..cde5123096cf 100644 --- a/cql3/operation.hh +++ b/cql3/operation.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/operation_impl.hh b/cql3/operation_impl.hh index 31e194717736..cbe030c3b0e7 100644 --- a/cql3/operation_impl.hh +++ b/cql3/operation_impl.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/prepare_context.cc b/cql3/prepare_context.cc index 2b1243263760..2e3fd2688dba 100644 --- a/cql3/prepare_context.cc +++ b/cql3/prepare_context.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/prepare_context.hh" diff --git a/cql3/prepare_context.hh b/cql3/prepare_context.hh index 341829ef49af..7367c9ca82fe 100644 --- a/cql3/prepare_context.hh +++ b/cql3/prepare_context.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/prepared_statements_cache.hh b/cql3/prepared_statements_cache.hh index 7b041265b39d..08b48e8f436d 100644 --- a/cql3/prepared_statements_cache.hh +++ b/cql3/prepared_statements_cache.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/query_options.cc b/cql3/query_options.cc index 57b9fe320bcf..1cc7fcea78ae 100644 --- a/cql3/query_options.cc +++ b/cql3/query_options.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/cql_config.hh" diff --git a/cql3/query_options.hh b/cql3/query_options.hh index 9aabbc627276..c34b9b00c7b2 100644 --- a/cql3/query_options.hh +++ b/cql3/query_options.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/query_options_fwd.hh b/cql3/query_options_fwd.hh index 1a7929b1e68a..0ad79c317897 100644 --- a/cql3/query_options_fwd.hh +++ b/cql3/query_options_fwd.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/query_processor.cc b/cql3/query_processor.cc index 975b2799a428..d5c2301fdfa4 100644 --- a/cql3/query_processor.cc +++ b/cql3/query_processor.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/query_processor.hh" diff --git a/cql3/query_processor.hh b/cql3/query_processor.hh index 0c6ff85f35ab..0da6b1b405ad 100644 --- a/cql3/query_processor.hh +++ b/cql3/query_processor.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/restrictions/bounds_slice.hh b/cql3/restrictions/bounds_slice.hh index 233287f3db2f..ac0b1545d902 100644 --- a/cql3/restrictions/bounds_slice.hh +++ b/cql3/restrictions/bounds_slice.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/restrictions/restrictions_config.hh b/cql3/restrictions/restrictions_config.hh index 698d28bcf5b8..b790b6acd156 100644 --- a/cql3/restrictions/restrictions_config.hh +++ b/cql3/restrictions/restrictions_config.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/cql3/restrictions/statement_restrictions.cc b/cql3/restrictions/statement_restrictions.cc index edc13cfa9b2e..19aabbcba584 100644 --- a/cql3/restrictions/statement_restrictions.cc +++ b/cql3/restrictions/statement_restrictions.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/cql3/restrictions/statement_restrictions.hh b/cql3/restrictions/statement_restrictions.hh index 8ef13c4c2db8..11328bb343bf 100644 --- a/cql3/restrictions/statement_restrictions.hh +++ b/cql3/restrictions/statement_restrictions.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/result_generator.hh b/cql3/result_generator.hh index a1b92bfb1735..feac73fc8669 100644 --- a/cql3/result_generator.hh +++ b/cql3/result_generator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/result_set.cc b/cql3/result_set.cc index 093e37fea571..9cbd060d4f6c 100644 --- a/cql3/result_set.cc +++ b/cql3/result_set.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/result_set.hh b/cql3/result_set.hh index a47e0d227523..f5a85486ff09 100644 --- a/cql3/result_set.hh +++ b/cql3/result_set.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/role_name.cc b/cql3/role_name.cc index 71ca176ba83b..7862d41e7717 100644 --- a/cql3/role_name.cc +++ b/cql3/role_name.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/role_name.hh" diff --git a/cql3/role_name.hh b/cql3/role_name.hh index 62ba96356e51..e9a67fd75587 100644 --- a/cql3/role_name.hh +++ b/cql3/role_name.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/selection/raw_selector.hh b/cql3/selection/raw_selector.hh index a6b771966c04..128e13b8ab0e 100644 --- a/cql3/selection/raw_selector.hh +++ b/cql3/selection/raw_selector.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/selection/selectable-expr.hh b/cql3/selection/selectable-expr.hh index 1c7f8592ab4e..1e9aead1e868 100644 --- a/cql3/selection/selectable-expr.hh +++ b/cql3/selection/selectable-expr.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/cql3/selection/selectable.cc b/cql3/selection/selectable.cc index cfe14cbbf20e..5c9fb8544f2d 100644 --- a/cql3/selection/selectable.cc +++ b/cql3/selection/selectable.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/query_options.hh" diff --git a/cql3/selection/selection.cc b/cql3/selection/selection.cc index 8e50db3b7132..777deeab359b 100644 --- a/cql3/selection/selection.cc +++ b/cql3/selection/selection.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/selection/selection.hh b/cql3/selection/selection.hh index c2103e661b18..f75dc270a6ed 100644 --- a/cql3/selection/selection.hh +++ b/cql3/selection/selection.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/selection/selector.cc b/cql3/selection/selector.cc index 964b8636cae5..a33ed57db72a 100644 --- a/cql3/selection/selector.cc +++ b/cql3/selection/selector.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "selector.hh" diff --git a/cql3/selection/selector.hh b/cql3/selection/selector.hh index ed01603cddc5..14d1bb91614d 100644 --- a/cql3/selection/selector.hh +++ b/cql3/selection/selector.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/sets.cc b/cql3/sets.cc index a98cbcceca01..6359666a6e79 100644 --- a/cql3/sets.cc +++ b/cql3/sets.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/cql3/sets.hh b/cql3/sets.hh index 631ad0b87185..d7b531facdcd 100644 --- a/cql3/sets.hh +++ b/cql3/sets.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/alter_keyspace_statement.cc b/cql3/statements/alter_keyspace_statement.cc index cab68e498aa6..42cbeca93223 100644 --- a/cql3/statements/alter_keyspace_statement.cc +++ b/cql3/statements/alter_keyspace_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/alter_keyspace_statement.hh b/cql3/statements/alter_keyspace_statement.hh index 1fb0d49bd3e0..c5878538ebb5 100644 --- a/cql3/statements/alter_keyspace_statement.hh +++ b/cql3/statements/alter_keyspace_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/alter_role_statement.hh b/cql3/statements/alter_role_statement.hh index bcf4aea7c25c..5648c08e44ef 100644 --- a/cql3/statements/alter_role_statement.hh +++ b/cql3/statements/alter_role_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/alter_service_level_statement.cc b/cql3/statements/alter_service_level_statement.cc index 532c7ecd647a..c1a9a15b614e 100644 --- a/cql3/statements/alter_service_level_statement.cc +++ b/cql3/statements/alter_service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "seastarx.hh" diff --git a/cql3/statements/alter_service_level_statement.hh b/cql3/statements/alter_service_level_statement.hh index 9b483e469d64..58f865b79a59 100644 --- a/cql3/statements/alter_service_level_statement.hh +++ b/cql3/statements/alter_service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/alter_table_statement.cc b/cql3/statements/alter_table_statement.cc index b57696bd7b9b..3b73282e4642 100644 --- a/cql3/statements/alter_table_statement.cc +++ b/cql3/statements/alter_table_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/statements/alter_table_statement.hh b/cql3/statements/alter_table_statement.hh index 2c4277d75427..28df9c8ef287 100644 --- a/cql3/statements/alter_table_statement.hh +++ b/cql3/statements/alter_table_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/alter_type_statement.cc b/cql3/statements/alter_type_statement.cc index 86431a640a7c..eddb963050a8 100644 --- a/cql3/statements/alter_type_statement.cc +++ b/cql3/statements/alter_type_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/alter_type_statement.hh b/cql3/statements/alter_type_statement.hh index 0ad8a00a0776..38e690cd09a0 100644 --- a/cql3/statements/alter_type_statement.hh +++ b/cql3/statements/alter_type_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/alter_view_statement.cc b/cql3/statements/alter_view_statement.cc index 5b8b84f23415..127da0980278 100644 --- a/cql3/statements/alter_view_statement.cc +++ b/cql3/statements/alter_view_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/alter_view_statement.hh b/cql3/statements/alter_view_statement.hh index d618c76f0a09..d36a066e8976 100644 --- a/cql3/statements/alter_view_statement.hh +++ b/cql3/statements/alter_view_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/attach_service_level_statement.cc b/cql3/statements/attach_service_level_statement.cc index 1b48f85c9ecf..bbb0cd5d4dfc 100644 --- a/cql3/statements/attach_service_level_statement.cc +++ b/cql3/statements/attach_service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "seastarx.hh" diff --git a/cql3/statements/attach_service_level_statement.hh b/cql3/statements/attach_service_level_statement.hh index 85580ba7069d..3f1be5547744 100644 --- a/cql3/statements/attach_service_level_statement.hh +++ b/cql3/statements/attach_service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/authentication_statement.cc b/cql3/statements/authentication_statement.cc index 43bcf75558b9..4e3caa2a046e 100644 --- a/cql3/statements/authentication_statement.cc +++ b/cql3/statements/authentication_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "authentication_statement.hh" diff --git a/cql3/statements/authentication_statement.hh b/cql3/statements/authentication_statement.hh index 4968f0899ae4..3e4f51008d3f 100644 --- a/cql3/statements/authentication_statement.hh +++ b/cql3/statements/authentication_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/authorization_statement.cc b/cql3/statements/authorization_statement.cc index f6eba3d5cf53..a8fccdc8a903 100644 --- a/cql3/statements/authorization_statement.cc +++ b/cql3/statements/authorization_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "authorization_statement.hh" diff --git a/cql3/statements/authorization_statement.hh b/cql3/statements/authorization_statement.hh index d9dec701292d..79343cf14f79 100644 --- a/cql3/statements/authorization_statement.hh +++ b/cql3/statements/authorization_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/batch_statement.cc b/cql3/statements/batch_statement.cc index aee5f30b6e04..b6977d2dfc4b 100644 --- a/cql3/statements/batch_statement.cc +++ b/cql3/statements/batch_statement.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "batch_statement.hh" diff --git a/cql3/statements/batch_statement.hh b/cql3/statements/batch_statement.hh index 9a51bc93c7e5..6b7aa6bb9f16 100644 --- a/cql3/statements/batch_statement.hh +++ b/cql3/statements/batch_statement.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/bound.hh b/cql3/statements/bound.hh index 7ee42219652a..975bc03581e1 100644 --- a/cql3/statements/bound.hh +++ b/cql3/statements/bound.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/cas_request.cc b/cql3/statements/cas_request.cc index 94d1915dda63..a7cbe6c933cf 100644 --- a/cql3/statements/cas_request.cc +++ b/cql3/statements/cas_request.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "mutation/mutation.hh" diff --git a/cql3/statements/cas_request.hh b/cql3/statements/cas_request.hh index d511a6880a3d..4df657cb5e0b 100644 --- a/cql3/statements/cas_request.hh +++ b/cql3/statements/cas_request.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once #include "utils/assert.hh" diff --git a/cql3/statements/cf_prop_defs.cc b/cql3/statements/cf_prop_defs.cc index 3965465f43c9..46c044022267 100644 --- a/cql3/statements/cf_prop_defs.cc +++ b/cql3/statements/cf_prop_defs.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/statements/cf_prop_defs.hh" diff --git a/cql3/statements/cf_prop_defs.hh b/cql3/statements/cf_prop_defs.hh index 71f79049e3f7..b07ce04f7796 100644 --- a/cql3/statements/cf_prop_defs.hh +++ b/cql3/statements/cf_prop_defs.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/cf_properties.hh b/cql3/statements/cf_properties.hh index 5d1b1c804335..b12668f5a9e2 100644 --- a/cql3/statements/cf_properties.hh +++ b/cql3/statements/cf_properties.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/cf_statement.cc b/cql3/statements/cf_statement.cc index 33409e3df487..ca7315824711 100644 --- a/cql3/statements/cf_statement.cc +++ b/cql3/statements/cf_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/statements/create_aggregate_statement.cc b/cql3/statements/create_aggregate_statement.cc index 2d41c2e96c4e..8c133f3f8b8d 100644 --- a/cql3/statements/create_aggregate_statement.cc +++ b/cql3/statements/create_aggregate_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/cql3/statements/create_aggregate_statement.hh b/cql3/statements/create_aggregate_statement.hh index 946f010e6ae6..b595d94cb0f8 100644 --- a/cql3/statements/create_aggregate_statement.hh +++ b/cql3/statements/create_aggregate_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/create_function_statement.cc b/cql3/statements/create_function_statement.cc index 9e9f35d9203b..a723aaf992e9 100644 --- a/cql3/statements/create_function_statement.cc +++ b/cql3/statements/create_function_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/cql3/statements/create_function_statement.hh b/cql3/statements/create_function_statement.hh index 256da7a4b504..3735ba1b8441 100644 --- a/cql3/statements/create_function_statement.hh +++ b/cql3/statements/create_function_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/create_index_statement.cc b/cql3/statements/create_index_statement.cc index 20f57c02a5ba..7718b2252d92 100644 --- a/cql3/statements/create_index_statement.cc +++ b/cql3/statements/create_index_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/create_index_statement.hh b/cql3/statements/create_index_statement.hh index bebb48a9a774..239c1968ad4a 100644 --- a/cql3/statements/create_index_statement.hh +++ b/cql3/statements/create_index_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/create_keyspace_statement.cc b/cql3/statements/create_keyspace_statement.cc index 85ccb54e3049..2960ba827438 100644 --- a/cql3/statements/create_keyspace_statement.cc +++ b/cql3/statements/create_keyspace_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/create_keyspace_statement.hh b/cql3/statements/create_keyspace_statement.hh index d4ef605c2aa7..726e4c3c6df5 100644 --- a/cql3/statements/create_keyspace_statement.hh +++ b/cql3/statements/create_keyspace_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/create_role_statement.hh b/cql3/statements/create_role_statement.hh index 4876c074a18a..e3713bd8f30a 100644 --- a/cql3/statements/create_role_statement.hh +++ b/cql3/statements/create_role_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/create_service_level_statement.cc b/cql3/statements/create_service_level_statement.cc index 00d20b72c707..394741685e91 100644 --- a/cql3/statements/create_service_level_statement.cc +++ b/cql3/statements/create_service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "auth/service.hh" diff --git a/cql3/statements/create_service_level_statement.hh b/cql3/statements/create_service_level_statement.hh index 635b7f2355c9..7976200e5689 100644 --- a/cql3/statements/create_service_level_statement.hh +++ b/cql3/statements/create_service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/create_table_statement.cc b/cql3/statements/create_table_statement.cc index d29498e14109..42e64494ff59 100644 --- a/cql3/statements/create_table_statement.cc +++ b/cql3/statements/create_table_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ diff --git a/cql3/statements/create_table_statement.hh b/cql3/statements/create_table_statement.hh index 3f8c8cbec9fb..d80acf484afc 100644 --- a/cql3/statements/create_table_statement.hh +++ b/cql3/statements/create_table_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/create_type_statement.cc b/cql3/statements/create_type_statement.cc index 8b995a1639dd..f6586d52f969 100644 --- a/cql3/statements/create_type_statement.cc +++ b/cql3/statements/create_type_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/create_type_statement.hh b/cql3/statements/create_type_statement.hh index 5179df93490f..07bf4c23be60 100644 --- a/cql3/statements/create_type_statement.hh +++ b/cql3/statements/create_type_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/create_view_statement.cc b/cql3/statements/create_view_statement.cc index 157768c6ddb9..da7f1ac40dbe 100644 --- a/cql3/statements/create_view_statement.cc +++ b/cql3/statements/create_view_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "exceptions/exceptions.hh" diff --git a/cql3/statements/create_view_statement.hh b/cql3/statements/create_view_statement.hh index 2868cbb743fe..056c4db762df 100644 --- a/cql3/statements/create_view_statement.hh +++ b/cql3/statements/create_view_statement.hh @@ -1,7 +1,7 @@ /* * Copyright (C) 2016-present ScyllaDB * - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/delete_statement.cc b/cql3/statements/delete_statement.cc index 7a6048aa8eca..1735ebec7001 100644 --- a/cql3/statements/delete_statement.cc +++ b/cql3/statements/delete_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/statements/delete_statement.hh b/cql3/statements/delete_statement.hh index 909465245ae8..22ec9975c04b 100644 --- a/cql3/statements/delete_statement.hh +++ b/cql3/statements/delete_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/describe_statement.cc b/cql3/statements/describe_statement.cc index a6eabbab643d..595c414d3548 100644 --- a/cql3/statements/describe_statement.cc +++ b/cql3/statements/describe_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include #include diff --git a/cql3/statements/describe_statement.hh b/cql3/statements/describe_statement.hh index 66e824a0c9fc..e3ad9420ee9a 100644 --- a/cql3/statements/describe_statement.hh +++ b/cql3/statements/describe_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/detach_service_level_statement.cc b/cql3/statements/detach_service_level_statement.cc index 2ccd056763e3..e1b8481abc90 100644 --- a/cql3/statements/detach_service_level_statement.cc +++ b/cql3/statements/detach_service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "seastarx.hh" diff --git a/cql3/statements/detach_service_level_statement.hh b/cql3/statements/detach_service_level_statement.hh index 00e1be325eff..4d4868737fcb 100644 --- a/cql3/statements/detach_service_level_statement.hh +++ b/cql3/statements/detach_service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/drop_aggregate_statement.cc b/cql3/statements/drop_aggregate_statement.cc index 6ebc5373dc55..e41eed42d575 100644 --- a/cql3/statements/drop_aggregate_statement.cc +++ b/cql3/statements/drop_aggregate_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/cql3/statements/drop_aggregate_statement.hh b/cql3/statements/drop_aggregate_statement.hh index 5e3cc27bfcf9..10a219a1a7cb 100644 --- a/cql3/statements/drop_aggregate_statement.hh +++ b/cql3/statements/drop_aggregate_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/drop_function_statement.cc b/cql3/statements/drop_function_statement.cc index 776335ca01f7..f541b618125b 100644 --- a/cql3/statements/drop_function_statement.cc +++ b/cql3/statements/drop_function_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/cql3/statements/drop_function_statement.hh b/cql3/statements/drop_function_statement.hh index b20949d0ca1a..0853e7718ba1 100644 --- a/cql3/statements/drop_function_statement.hh +++ b/cql3/statements/drop_function_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/drop_index_statement.cc b/cql3/statements/drop_index_statement.cc index e5b9312992b8..a6cd3ec2515a 100644 --- a/cql3/statements/drop_index_statement.cc +++ b/cql3/statements/drop_index_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/drop_index_statement.hh b/cql3/statements/drop_index_statement.hh index 079e07dbf636..d69f39108e71 100644 --- a/cql3/statements/drop_index_statement.hh +++ b/cql3/statements/drop_index_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/drop_keyspace_statement.cc b/cql3/statements/drop_keyspace_statement.cc index 3ad6cfa3f7c5..8e336efe8c8e 100644 --- a/cql3/statements/drop_keyspace_statement.cc +++ b/cql3/statements/drop_keyspace_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/drop_keyspace_statement.hh b/cql3/statements/drop_keyspace_statement.hh index abd7d313fb2a..34acc096dc29 100644 --- a/cql3/statements/drop_keyspace_statement.hh +++ b/cql3/statements/drop_keyspace_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/drop_role_statement.hh b/cql3/statements/drop_role_statement.hh index 94f67c72c65d..5b1afd048eb5 100644 --- a/cql3/statements/drop_role_statement.hh +++ b/cql3/statements/drop_role_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/drop_service_level_statement.cc b/cql3/statements/drop_service_level_statement.cc index 92aac6c0c06a..2f5085662d00 100644 --- a/cql3/statements/drop_service_level_statement.cc +++ b/cql3/statements/drop_service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "seastarx.hh" diff --git a/cql3/statements/drop_service_level_statement.hh b/cql3/statements/drop_service_level_statement.hh index 1d749e0f01e6..969c1fbba755 100644 --- a/cql3/statements/drop_service_level_statement.hh +++ b/cql3/statements/drop_service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/drop_table_statement.cc b/cql3/statements/drop_table_statement.cc index 97d08bbda240..5333629507e0 100644 --- a/cql3/statements/drop_table_statement.cc +++ b/cql3/statements/drop_table_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/drop_table_statement.hh b/cql3/statements/drop_table_statement.hh index 526f05f12fa2..fa29d386ce39 100644 --- a/cql3/statements/drop_table_statement.hh +++ b/cql3/statements/drop_table_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/drop_type_statement.cc b/cql3/statements/drop_type_statement.cc index 6189b6be7af6..fb2a94d99d58 100644 --- a/cql3/statements/drop_type_statement.cc +++ b/cql3/statements/drop_type_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/drop_type_statement.hh b/cql3/statements/drop_type_statement.hh index e912db98ab51..5409ab23d6e1 100644 --- a/cql3/statements/drop_type_statement.hh +++ b/cql3/statements/drop_type_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/drop_view_statement.cc b/cql3/statements/drop_view_statement.cc index 15fb58345b9e..a1ca2669dc80 100644 --- a/cql3/statements/drop_view_statement.cc +++ b/cql3/statements/drop_view_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/drop_view_statement.hh b/cql3/statements/drop_view_statement.hh index e332b0eab119..44cd07786f41 100644 --- a/cql3/statements/drop_view_statement.hh +++ b/cql3/statements/drop_view_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/function_statement.cc b/cql3/statements/function_statement.cc index 4f987fc24fac..0e4a15126d6e 100644 --- a/cql3/statements/function_statement.cc +++ b/cql3/statements/function_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/statements/function_statement.hh" diff --git a/cql3/statements/function_statement.hh b/cql3/statements/function_statement.hh index 2a0756a79b3b..e8efd830498f 100644 --- a/cql3/statements/function_statement.hh +++ b/cql3/statements/function_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/grant_role_statement.hh b/cql3/statements/grant_role_statement.hh index 877748a74cc6..0f57e773976d 100644 --- a/cql3/statements/grant_role_statement.hh +++ b/cql3/statements/grant_role_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/grant_statement.cc b/cql3/statements/grant_statement.cc index 6d65c8382395..7e681fd5469b 100644 --- a/cql3/statements/grant_statement.cc +++ b/cql3/statements/grant_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "grant_statement.hh" diff --git a/cql3/statements/grant_statement.hh b/cql3/statements/grant_statement.hh index a434119662f9..5c10494865a1 100644 --- a/cql3/statements/grant_statement.hh +++ b/cql3/statements/grant_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/index_prop_defs.cc b/cql3/statements/index_prop_defs.cc index 81fa4c7689d0..1eb3355ef821 100644 --- a/cql3/statements/index_prop_defs.cc +++ b/cql3/statements/index_prop_defs.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/index_prop_defs.hh b/cql3/statements/index_prop_defs.hh index d025e17a8090..9ff014aee4e9 100644 --- a/cql3/statements/index_prop_defs.hh +++ b/cql3/statements/index_prop_defs.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/index_target.cc b/cql3/statements/index_target.cc index 092c66b8078b..d07194c9a8ff 100644 --- a/cql3/statements/index_target.cc +++ b/cql3/statements/index_target.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/index_target.hh b/cql3/statements/index_target.hh index 529521496bfb..0a2bd0022172 100644 --- a/cql3/statements/index_target.hh +++ b/cql3/statements/index_target.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/ks_prop_defs.cc b/cql3/statements/ks_prop_defs.cc index 9c0b9d6877cd..dbadbf1a3ec7 100644 --- a/cql3/statements/ks_prop_defs.cc +++ b/cql3/statements/ks_prop_defs.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/statements/ks_prop_defs.hh b/cql3/statements/ks_prop_defs.hh index 82a732cda129..276aefd4df99 100644 --- a/cql3/statements/ks_prop_defs.hh +++ b/cql3/statements/ks_prop_defs.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/list_effective_service_level_statement.cc b/cql3/statements/list_effective_service_level_statement.cc index 8734a3472f54..918c6dbf1080 100644 --- a/cql3/statements/list_effective_service_level_statement.cc +++ b/cql3/statements/list_effective_service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "list_effective_service_level_statement.hh" diff --git a/cql3/statements/list_effective_service_level_statement.hh b/cql3/statements/list_effective_service_level_statement.hh index aaae45ced50c..c3ff8df80367 100644 --- a/cql3/statements/list_effective_service_level_statement.hh +++ b/cql3/statements/list_effective_service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/list_permissions_statement.cc b/cql3/statements/list_permissions_statement.cc index 6a9d6cc338f0..f34bcb808d04 100644 --- a/cql3/statements/list_permissions_statement.cc +++ b/cql3/statements/list_permissions_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/list_permissions_statement.hh b/cql3/statements/list_permissions_statement.hh index 00add9631f85..74d1aecf50bc 100644 --- a/cql3/statements/list_permissions_statement.hh +++ b/cql3/statements/list_permissions_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/list_roles_statement.hh b/cql3/statements/list_roles_statement.hh index 6f102d03de91..516f9d1a409c 100644 --- a/cql3/statements/list_roles_statement.hh +++ b/cql3/statements/list_roles_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/list_service_level_attachments_statement.cc b/cql3/statements/list_service_level_attachments_statement.cc index 381dbd73413d..2cd16a5b2659 100644 --- a/cql3/statements/list_service_level_attachments_statement.cc +++ b/cql3/statements/list_service_level_attachments_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "seastarx.hh" diff --git a/cql3/statements/list_service_level_attachments_statement.hh b/cql3/statements/list_service_level_attachments_statement.hh index 997084463f5e..6e0dd917b662 100644 --- a/cql3/statements/list_service_level_attachments_statement.hh +++ b/cql3/statements/list_service_level_attachments_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/list_service_level_statement.cc b/cql3/statements/list_service_level_statement.cc index 6afbe8bec7a6..a9b861678df0 100644 --- a/cql3/statements/list_service_level_statement.cc +++ b/cql3/statements/list_service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "seastarx.hh" diff --git a/cql3/statements/list_service_level_statement.hh b/cql3/statements/list_service_level_statement.hh index 82236b3f0f92..841f210237cc 100644 --- a/cql3/statements/list_service_level_statement.hh +++ b/cql3/statements/list_service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/list_users_statement.cc b/cql3/statements/list_users_statement.cc index f8740de1d014..819542442076 100644 --- a/cql3/statements/list_users_statement.cc +++ b/cql3/statements/list_users_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "list_users_statement.hh" diff --git a/cql3/statements/list_users_statement.hh b/cql3/statements/list_users_statement.hh index 0957b2e86ba2..750e7b7e060d 100644 --- a/cql3/statements/list_users_statement.hh +++ b/cql3/statements/list_users_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/modification_statement.cc b/cql3/statements/modification_statement.cc index 0bbb548f3c21..434598e68842 100644 --- a/cql3/statements/modification_statement.cc +++ b/cql3/statements/modification_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/statements/modification_statement.hh b/cql3/statements/modification_statement.hh index ee91c24bd794..40d961c922c8 100644 --- a/cql3/statements/modification_statement.hh +++ b/cql3/statements/modification_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/permission_altering_statement.cc b/cql3/statements/permission_altering_statement.cc index fe011c055176..b8bdc843a2f1 100644 --- a/cql3/statements/permission_altering_statement.cc +++ b/cql3/statements/permission_altering_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/permission_altering_statement.hh b/cql3/statements/permission_altering_statement.hh index eb06dfa1d0ee..55d61e3c51f3 100644 --- a/cql3/statements/permission_altering_statement.hh +++ b/cql3/statements/permission_altering_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/prepared_statement.hh b/cql3/statements/prepared_statement.hh index a61836cf4e99..741d19135bfc 100644 --- a/cql3/statements/prepared_statement.hh +++ b/cql3/statements/prepared_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/property_definitions.cc b/cql3/statements/property_definitions.cc index a3c11ab4cfb5..63a25faa5fe7 100644 --- a/cql3/statements/property_definitions.cc +++ b/cql3/statements/property_definitions.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/property_definitions.hh b/cql3/statements/property_definitions.hh index 8e05682e3214..0733a9269b0d 100644 --- a/cql3/statements/property_definitions.hh +++ b/cql3/statements/property_definitions.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/prune_materialized_view_statement.cc b/cql3/statements/prune_materialized_view_statement.cc index ac59399ac348..d9e58e92f4b2 100644 --- a/cql3/statements/prune_materialized_view_statement.cc +++ b/cql3/statements/prune_materialized_view_statement.cc @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* Copyright 2022-present ScyllaDB */ diff --git a/cql3/statements/prune_materialized_view_statement.hh b/cql3/statements/prune_materialized_view_statement.hh index 6c7cfd95ef44..9b0818b02ad5 100644 --- a/cql3/statements/prune_materialized_view_statement.hh +++ b/cql3/statements/prune_materialized_view_statement.hh @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* Copyright 2022-present ScyllaDB */ diff --git a/cql3/statements/raw/batch_statement.hh b/cql3/statements/raw/batch_statement.hh index 7981fab5ba8d..4371b6dd37a0 100644 --- a/cql3/statements/raw/batch_statement.hh +++ b/cql3/statements/raw/batch_statement.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/cf_statement.hh b/cql3/statements/raw/cf_statement.hh index 295fa02eff38..e3a248c0862d 100644 --- a/cql3/statements/raw/cf_statement.hh +++ b/cql3/statements/raw/cf_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/delete_statement.hh b/cql3/statements/raw/delete_statement.hh index 2e241b747114..5d17247f3f44 100644 --- a/cql3/statements/raw/delete_statement.hh +++ b/cql3/statements/raw/delete_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/describe_statement.hh b/cql3/statements/raw/describe_statement.hh index cdbfc75ef5fd..62c83dc89567 100644 --- a/cql3/statements/raw/describe_statement.hh +++ b/cql3/statements/raw/describe_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/insert_statement.hh b/cql3/statements/raw/insert_statement.hh index f660d4a8be50..35580f70bd57 100644 --- a/cql3/statements/raw/insert_statement.hh +++ b/cql3/statements/raw/insert_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/modification_statement.hh b/cql3/statements/raw/modification_statement.hh index 32fb0cac5ff4..f96b1db51b59 100644 --- a/cql3/statements/raw/modification_statement.hh +++ b/cql3/statements/raw/modification_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/parsed_statement.cc b/cql3/statements/raw/parsed_statement.cc index 809a47c49ff8..1e0151ef4ade 100644 --- a/cql3/statements/raw/parsed_statement.cc +++ b/cql3/statements/raw/parsed_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "parsed_statement.hh" diff --git a/cql3/statements/raw/parsed_statement.hh b/cql3/statements/raw/parsed_statement.hh index 5922f7e27d3b..d4055553b580 100644 --- a/cql3/statements/raw/parsed_statement.hh +++ b/cql3/statements/raw/parsed_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/select_statement.hh b/cql3/statements/raw/select_statement.hh index ab52e1ad0239..43686a01ad50 100644 --- a/cql3/statements/raw/select_statement.hh +++ b/cql3/statements/raw/select_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/truncate_statement.hh b/cql3/statements/raw/truncate_statement.hh index 4fdd732c6f49..99474c8e28d4 100644 --- a/cql3/statements/raw/truncate_statement.hh +++ b/cql3/statements/raw/truncate_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/update_statement.hh b/cql3/statements/raw/update_statement.hh index 7a63783718c1..57fdad566110 100644 --- a/cql3/statements/raw/update_statement.hh +++ b/cql3/statements/raw/update_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/raw/use_statement.hh b/cql3/statements/raw/use_statement.hh index ddf27cb5f4d2..db7d765fa210 100644 --- a/cql3/statements/raw/use_statement.hh +++ b/cql3/statements/raw/use_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/request_validations.hh b/cql3/statements/request_validations.hh index 9667bd96d8bf..d48f6eba8d69 100644 --- a/cql3/statements/request_validations.hh +++ b/cql3/statements/request_validations.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/revoke_role_statement.hh b/cql3/statements/revoke_role_statement.hh index 960d8e1de786..5e297b22bec4 100644 --- a/cql3/statements/revoke_role_statement.hh +++ b/cql3/statements/revoke_role_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/revoke_statement.cc b/cql3/statements/revoke_statement.cc index 89e793792f11..974b870c2dbd 100644 --- a/cql3/statements/revoke_statement.cc +++ b/cql3/statements/revoke_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "revoke_statement.hh" diff --git a/cql3/statements/revoke_statement.hh b/cql3/statements/revoke_statement.hh index 4df1cc489020..5fd47e3c95ff 100644 --- a/cql3/statements/revoke_statement.hh +++ b/cql3/statements/revoke_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/role-management-statements.cc b/cql3/statements/role-management-statements.cc index 0e7bd1541e6e..864e8de42602 100644 --- a/cql3/statements/role-management-statements.cc +++ b/cql3/statements/role-management-statements.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/schema_altering_statement.cc b/cql3/statements/schema_altering_statement.cc index 7a2b15122f71..0e4c3ad6b2d8 100644 --- a/cql3/statements/schema_altering_statement.cc +++ b/cql3/statements/schema_altering_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/cql3/statements/schema_altering_statement.hh b/cql3/statements/schema_altering_statement.hh index ab03cbefcc76..1af3531c3db1 100644 --- a/cql3/statements/schema_altering_statement.hh +++ b/cql3/statements/schema_altering_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/select_statement.cc b/cql3/statements/select_statement.cc index 40c6d5b6ef36..fd47f218a04f 100644 --- a/cql3/statements/select_statement.cc +++ b/cql3/statements/select_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/statements/select_statement.hh" diff --git a/cql3/statements/select_statement.hh b/cql3/statements/select_statement.hh index b8d1ce24b756..d2664433242f 100644 --- a/cql3/statements/select_statement.hh +++ b/cql3/statements/select_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/service_level_statement.cc b/cql3/statements/service_level_statement.cc index ef7ec820335b..d0e48b64542a 100644 --- a/cql3/statements/service_level_statement.cc +++ b/cql3/statements/service_level_statement.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service_level_statement.hh" diff --git a/cql3/statements/service_level_statement.hh b/cql3/statements/service_level_statement.hh index 60990480daca..354fb2c4ee60 100644 --- a/cql3/statements/service_level_statement.hh +++ b/cql3/statements/service_level_statement.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/sl_prop_defs.cc b/cql3/statements/sl_prop_defs.cc index 6ea7a7a6f7a2..5e9230feae88 100644 --- a/cql3/statements/sl_prop_defs.cc +++ b/cql3/statements/sl_prop_defs.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/statements/sl_prop_defs.hh" diff --git a/cql3/statements/sl_prop_defs.hh b/cql3/statements/sl_prop_defs.hh index 14edf27c7bdd..072eb29e86cb 100644 --- a/cql3/statements/sl_prop_defs.hh +++ b/cql3/statements/sl_prop_defs.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/statements/statement_type.hh b/cql3/statements/statement_type.hh index 02fbed50ba7f..f65127959274 100644 --- a/cql3/statements/statement_type.hh +++ b/cql3/statements/statement_type.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/strongly_consistent_modification_statement.cc b/cql3/statements/strongly_consistent_modification_statement.cc index c6f56fa31b02..bba301851083 100644 --- a/cql3/statements/strongly_consistent_modification_statement.cc +++ b/cql3/statements/strongly_consistent_modification_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ diff --git a/cql3/statements/strongly_consistent_modification_statement.hh b/cql3/statements/strongly_consistent_modification_statement.hh index 0ffee12d1dcd..840999892878 100644 --- a/cql3/statements/strongly_consistent_modification_statement.hh +++ b/cql3/statements/strongly_consistent_modification_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/strongly_consistent_select_statement.cc b/cql3/statements/strongly_consistent_select_statement.cc index 89e504f4a08a..b7286a96634b 100644 --- a/cql3/statements/strongly_consistent_select_statement.cc +++ b/cql3/statements/strongly_consistent_select_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ diff --git a/cql3/statements/strongly_consistent_select_statement.hh b/cql3/statements/strongly_consistent_select_statement.hh index a6971f48b4d3..894ba9aa01c8 100644 --- a/cql3/statements/strongly_consistent_select_statement.hh +++ b/cql3/statements/strongly_consistent_select_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/truncate_statement.cc b/cql3/statements/truncate_statement.cc index 98728212ab1a..9c24f2339afd 100644 --- a/cql3/statements/truncate_statement.cc +++ b/cql3/statements/truncate_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/statements/truncate_statement.hh b/cql3/statements/truncate_statement.hh index 6aa4ecbfbaff..54739a90c660 100644 --- a/cql3/statements/truncate_statement.hh +++ b/cql3/statements/truncate_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/update_statement.cc b/cql3/statements/update_statement.cc index 17a65800e30c..b0556f45cf1f 100644 --- a/cql3/statements/update_statement.cc +++ b/cql3/statements/update_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/statements/update_statement.hh b/cql3/statements/update_statement.hh index 7bad5cbea6d4..101ce1526ba9 100644 --- a/cql3/statements/update_statement.hh +++ b/cql3/statements/update_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/statements/use_statement.cc b/cql3/statements/use_statement.cc index 57fef1144e03..1e58380f9839 100644 --- a/cql3/statements/use_statement.cc +++ b/cql3/statements/use_statement.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/statements/use_statement.hh" diff --git a/cql3/statements/use_statement.hh b/cql3/statements/use_statement.hh index 14f47e247dec..8e102d85b575 100644 --- a/cql3/statements/use_statement.hh +++ b/cql3/statements/use_statement.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/stats.hh b/cql3/stats.hh index 2460eb18ebf3..ef87254eb8ca 100644 --- a/cql3/stats.hh +++ b/cql3/stats.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/type_json.cc b/cql3/type_json.cc index f01eb95dac83..578a56fb068a 100644 --- a/cql3/type_json.cc +++ b/cql3/type_json.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/type_json.hh" diff --git a/cql3/type_json.hh b/cql3/type_json.hh index 15a56edacecf..0bfb657d3fda 100644 --- a/cql3/type_json.hh +++ b/cql3/type_json.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/untyped_result_set.cc b/cql3/untyped_result_set.cc index 2fd843c79ebc..be599e5e8c44 100644 --- a/cql3/untyped_result_set.cc +++ b/cql3/untyped_result_set.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include #include diff --git a/cql3/untyped_result_set.hh b/cql3/untyped_result_set.hh index e4245243601e..4efcdc380378 100644 --- a/cql3/untyped_result_set.hh +++ b/cql3/untyped_result_set.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include #include diff --git a/cql3/update_parameters.cc b/cql3/update_parameters.cc index e85ad261922d..34b657a5357f 100644 --- a/cql3/update_parameters.cc +++ b/cql3/update_parameters.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/update_parameters.hh b/cql3/update_parameters.hh index ea3e1e526fe4..750da0294eb4 100644 --- a/cql3/update_parameters.hh +++ b/cql3/update_parameters.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/user_types.cc b/cql3/user_types.cc index 02f96a864fb0..e4f67015807c 100644 --- a/cql3/user_types.cc +++ b/cql3/user_types.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/cql3/user_types.hh b/cql3/user_types.hh index 867cef63ff90..5bc0e1fa3c6b 100644 --- a/cql3/user_types.hh +++ b/cql3/user_types.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/ut_name.cc b/cql3/ut_name.cc index abc51706efcf..bdd2e12b5757 100644 --- a/cql3/ut_name.cc +++ b/cql3/ut_name.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "cql3/ut_name.hh" diff --git a/cql3/ut_name.hh b/cql3/ut_name.hh index 37860fabd312..444e3e4dc721 100644 --- a/cql3/ut_name.hh +++ b/cql3/ut_name.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/cql3/util.cc b/cql3/util.cc index 566cff198919..c1105c172020 100644 --- a/cql3/util.cc +++ b/cql3/util.cc @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* Copyright 2020-present ScyllaDB */ diff --git a/cql3/util.hh b/cql3/util.hh index 107a36857df8..c08ff1f043b8 100644 --- a/cql3/util.hh +++ b/cql3/util.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql3/values.cc b/cql3/values.cc index 2bd7f866dd55..e7e2b1c1a63e 100644 --- a/cql3/values.cc +++ b/cql3/values.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/values.hh" diff --git a/cql3/values.hh b/cql3/values.hh index 2e55a5cf52e0..956d43d0b31f 100644 --- a/cql3/values.hh +++ b/cql3/values.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/cql_serialization_format.hh b/cql_serialization_format.hh index 4ca87c35921b..1707060082c8 100644 --- a/cql_serialization_format.hh +++ b/cql_serialization_format.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/data_dictionary/data_dictionary.cc b/data_dictionary/data_dictionary.cc index 99fe3280d644..00cf400ab264 100644 --- a/data_dictionary/data_dictionary.cc +++ b/data_dictionary/data_dictionary.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/data_dictionary/data_dictionary.hh b/data_dictionary/data_dictionary.hh index 0e3dc3831e9d..e4539dc6e3bb 100644 --- a/data_dictionary/data_dictionary.hh +++ b/data_dictionary/data_dictionary.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/data_dictionary/impl.hh b/data_dictionary/impl.hh index dee4c1dcd066..2e7e42077663 100644 --- a/data_dictionary/impl.hh +++ b/data_dictionary/impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/data_dictionary/keyspace_metadata.hh b/data_dictionary/keyspace_metadata.hh index 1eed754bcd48..d1a66193fdf5 100644 --- a/data_dictionary/keyspace_metadata.hh +++ b/data_dictionary/keyspace_metadata.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/data_dictionary/storage_options.hh b/data_dictionary/storage_options.hh index a8b9c5cb1b4e..f25d6d13b17f 100644 --- a/data_dictionary/storage_options.hh +++ b/data_dictionary/storage_options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/data_dictionary/user_types_metadata.hh b/data_dictionary/user_types_metadata.hh index 738b865ec7d0..3b005460d0b5 100644 --- a/data_dictionary/user_types_metadata.hh +++ b/data_dictionary/user_types_metadata.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/auth_version.hh b/db/auth_version.hh index dd0be81b5440..15bccc627532 100644 --- a/db/auth_version.hh +++ b/db/auth_version.hh @@ -1,5 +1,5 @@ // Copyright (C) 2024-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/db/batchlog_manager.cc b/db/batchlog_manager.cc index e739a0b17c11..eb8f76913a71 100644 --- a/db/batchlog_manager.cc +++ b/db/batchlog_manager.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/db/batchlog_manager.hh b/db/batchlog_manager.hh index cb4410b365cf..f71fd62068ee 100644 --- a/db/batchlog_manager.hh +++ b/db/batchlog_manager.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/cache_tracker.hh b/db/cache_tracker.hh index eb86b2df1df6..1bd90fcb4617 100644 --- a/db/cache_tracker.hh +++ b/db/cache_tracker.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/chained_delegating_reader.hh b/db/chained_delegating_reader.hh index 23503557434d..53e51ccf80c6 100644 --- a/db/chained_delegating_reader.hh +++ b/db/chained_delegating_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/commitlog/commitlog.cc b/db/commitlog/commitlog.cc index 79b208eba50b..a81405acd31a 100644 --- a/db/commitlog/commitlog.cc +++ b/db/commitlog/commitlog.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/db/commitlog/commitlog.hh b/db/commitlog/commitlog.hh index b96a3e97b7ef..50c6cbadf588 100644 --- a/db/commitlog/commitlog.hh +++ b/db/commitlog/commitlog.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/commitlog/commitlog_entry.cc b/db/commitlog/commitlog_entry.cc index 666d269d9cb1..373c8b1b13da 100644 --- a/db/commitlog/commitlog_entry.cc +++ b/db/commitlog/commitlog_entry.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "counters.hh" diff --git a/db/commitlog/commitlog_entry.hh b/db/commitlog/commitlog_entry.hh index 58e61ef760b7..9c53af639b99 100644 --- a/db/commitlog/commitlog_entry.hh +++ b/db/commitlog/commitlog_entry.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/commitlog/commitlog_extensions.hh b/db/commitlog/commitlog_extensions.hh index 54057b38217e..316d70f4fe34 100644 --- a/db/commitlog/commitlog_extensions.hh +++ b/db/commitlog/commitlog_extensions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/commitlog/commitlog_replayer.cc b/db/commitlog/commitlog_replayer.cc index 6e729ebf621c..aae9117a516a 100644 --- a/db/commitlog/commitlog_replayer.cc +++ b/db/commitlog/commitlog_replayer.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/db/commitlog/commitlog_replayer.hh b/db/commitlog/commitlog_replayer.hh index 023f98edaeb8..71faba2d7ee3 100644 --- a/db/commitlog/commitlog_replayer.hh +++ b/db/commitlog/commitlog_replayer.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/commitlog/commitlog_types.hh b/db/commitlog/commitlog_types.hh index 74d18fba4d9c..5f428a8de3e4 100644 --- a/db/commitlog/commitlog_types.hh +++ b/db/commitlog/commitlog_types.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/commitlog/replay_position.hh b/db/commitlog/replay_position.hh index 8a4f64b9f378..c9d557dfa8dd 100644 --- a/db/commitlog/replay_position.hh +++ b/db/commitlog/replay_position.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/commitlog/rp_set.hh b/db/commitlog/rp_set.hh index ef4fbb644efc..52ad790c0ea8 100644 --- a/db/commitlog/rp_set.hh +++ b/db/commitlog/rp_set.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/config.cc b/db/config.cc index e278141d2b17..7c21860e306b 100644 --- a/db/config.cc +++ b/db/config.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/config.hh b/db/config.hh index af7ff12e056e..a1641cebfabf 100644 --- a/db/config.hh +++ b/db/config.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/consistency_level.cc b/db/consistency_level.cc index 3e437b10a119..84bc7d46ebc7 100644 --- a/db/consistency_level.cc +++ b/db/consistency_level.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "db/consistency_level.hh" diff --git a/db/consistency_level.hh b/db/consistency_level.hh index 048dd8a41cf2..787702b1273a 100644 --- a/db/consistency_level.hh +++ b/db/consistency_level.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/consistency_level_type.hh b/db/consistency_level_type.hh index b652b9ebe4f0..091dacdc22ff 100644 --- a/db/consistency_level_type.hh +++ b/db/consistency_level_type.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/consistency_level_validations.hh b/db/consistency_level_validations.hh index 3835044e006d..c8444c2c402f 100644 --- a/db/consistency_level_validations.hh +++ b/db/consistency_level_validations.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/cql_type_parser.cc b/db/cql_type_parser.cc index e8661dde5b61..c85a593cb9c3 100644 --- a/db/cql_type_parser.cc +++ b/db/cql_type_parser.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include #include diff --git a/db/cql_type_parser.hh b/db/cql_type_parser.hh index 04c9c1f7196d..df131a8b7840 100644 --- a/db/cql_type_parser.hh +++ b/db/cql_type_parser.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/data_listeners.cc b/db/data_listeners.cc index 50fdb666c178..e46735896829 100755 --- a/db/data_listeners.cc +++ b/db/data_listeners.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/data_listeners.hh" diff --git a/db/data_listeners.hh b/db/data_listeners.hh index e03001dd6c8c..29eb12fa8d43 100755 --- a/db/data_listeners.hh +++ b/db/data_listeners.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/extensions.cc b/db/extensions.cc index 64aee28e2d7a..ea928bb88900 100644 --- a/db/extensions.cc +++ b/db/extensions.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "extensions.hh" diff --git a/db/extensions.hh b/db/extensions.hh index 44a6cf0fb4d0..19a68107d00a 100644 --- a/db/extensions.hh +++ b/db/extensions.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/functions/aggregate_function.hh b/db/functions/aggregate_function.hh index 51d258ec4826..d299552cccbf 100644 --- a/db/functions/aggregate_function.hh +++ b/db/functions/aggregate_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/functions/function.cc b/db/functions/function.cc index 0601a0903937..7f813f00919c 100644 --- a/db/functions/function.cc +++ b/db/functions/function.cc @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #include "aggregate_function.hh" diff --git a/db/functions/function.hh b/db/functions/function.hh index a5024fd664cc..9e9ab0fa6b5f 100644 --- a/db/functions/function.hh +++ b/db/functions/function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/functions/function_name.hh b/db/functions/function_name.hh index be1a5b0da859..c0ac65cd59d7 100644 --- a/db/functions/function_name.hh +++ b/db/functions/function_name.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/functions/scalar_function.hh b/db/functions/scalar_function.hh index 8005c23b05ee..393128b627bd 100644 --- a/db/functions/scalar_function.hh +++ b/db/functions/scalar_function.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/functions/stateless_aggregate_function.hh b/db/functions/stateless_aggregate_function.hh index 8133029e33da..afe1ab26cf22 100644 --- a/db/functions/stateless_aggregate_function.hh +++ b/db/functions/stateless_aggregate_function.hh @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +// SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) #pragma once diff --git a/db/heat_load_balance.cc b/db/heat_load_balance.cc index f980fc78a608..0c9180def154 100644 --- a/db/heat_load_balance.cc +++ b/db/heat_load_balance.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include #include diff --git a/db/heat_load_balance.hh b/db/heat_load_balance.hh index a377fecab01d..c73b644ed38b 100644 --- a/db/heat_load_balance.hh +++ b/db/heat_load_balance.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/host_filter.cc b/db/hints/host_filter.cc index 5b915229e99d..8701df7d5c18 100644 --- a/db/hints/host_filter.cc +++ b/db/hints/host_filter.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/hints/host_filter.hh b/db/hints/host_filter.hh index 6c37b07c07c0..4433713303da 100644 --- a/db/hints/host_filter.hh +++ b/db/hints/host_filter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/internal/common.hh b/db/hints/internal/common.hh index 6132c12d00c3..7f500438610a 100644 --- a/db/hints/internal/common.hh +++ b/db/hints/internal/common.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/internal/hint_endpoint_manager.cc b/db/hints/internal/hint_endpoint_manager.cc index 4e931ad3d86c..8b7134480cc6 100644 --- a/db/hints/internal/hint_endpoint_manager.cc +++ b/db/hints/internal/hint_endpoint_manager.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/hints/internal/hint_endpoint_manager.hh" diff --git a/db/hints/internal/hint_endpoint_manager.hh b/db/hints/internal/hint_endpoint_manager.hh index 25b699b24f40..0a94aece8dc5 100644 --- a/db/hints/internal/hint_endpoint_manager.hh +++ b/db/hints/internal/hint_endpoint_manager.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/internal/hint_logger.hh b/db/hints/internal/hint_logger.hh index d9c1319f87d5..ced3cfaeab08 100644 --- a/db/hints/internal/hint_logger.hh +++ b/db/hints/internal/hint_logger.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/internal/hint_sender.cc b/db/hints/internal/hint_sender.cc index 6d11a0067e3c..b6dc0cf0334f 100644 --- a/db/hints/internal/hint_sender.cc +++ b/db/hints/internal/hint_sender.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/hints/internal/hint_sender.hh" diff --git a/db/hints/internal/hint_sender.hh b/db/hints/internal/hint_sender.hh index b9d98d4fccb4..40904eb4b78d 100644 --- a/db/hints/internal/hint_sender.hh +++ b/db/hints/internal/hint_sender.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/internal/hint_storage.cc b/db/hints/internal/hint_storage.cc index 1096e75e1daf..5904db1ab4bf 100644 --- a/db/hints/internal/hint_storage.cc +++ b/db/hints/internal/hint_storage.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/hints/internal/hint_storage.hh" diff --git a/db/hints/internal/hint_storage.hh b/db/hints/internal/hint_storage.hh index 6e52b48fd1cb..509b4dde12eb 100644 --- a/db/hints/internal/hint_storage.hh +++ b/db/hints/internal/hint_storage.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/manager.cc b/db/hints/manager.cc index 158888687105..95d5ea0f2a03 100644 --- a/db/hints/manager.cc +++ b/db/hints/manager.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/hints/manager.hh" diff --git a/db/hints/manager.hh b/db/hints/manager.hh index a5bfb1b07a79..69d18199ed57 100644 --- a/db/hints/manager.hh +++ b/db/hints/manager.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/resource_manager.cc b/db/hints/resource_manager.cc index 66b15d05a4d5..6e0045945f7f 100644 --- a/db/hints/resource_manager.cc +++ b/db/hints/resource_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "resource_manager.hh" diff --git a/db/hints/resource_manager.hh b/db/hints/resource_manager.hh index 27323c9fd23d..45f8a5e3e5c2 100644 --- a/db/hints/resource_manager.hh +++ b/db/hints/resource_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/hints/sync_point.cc b/db/hints/sync_point.cc index 2180e120de06..61bd0d57842c 100644 --- a/db/hints/sync_point.cc +++ b/db/hints/sync_point.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/hints/sync_point.hh b/db/hints/sync_point.hh index af24d784205c..976f765c544f 100644 --- a/db/hints/sync_point.hh +++ b/db/hints/sync_point.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/large_data_handler.cc b/db/large_data_handler.cc index 1da23f799716..c02d15cb65de 100644 --- a/db/large_data_handler.cc +++ b/db/large_data_handler.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/db/large_data_handler.hh b/db/large_data_handler.hh index 645d9330165d..718ca0eb1a28 100644 --- a/db/large_data_handler.hh +++ b/db/large_data_handler.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/legacy_schema_migrator.cc b/db/legacy_schema_migrator.cc index 7e1d94ea25e7..d3cbc8ac6576 100644 --- a/db/legacy_schema_migrator.cc +++ b/db/legacy_schema_migrator.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ // Since Scylla 2.0, we use system tables whose schemas were introduced in diff --git a/db/legacy_schema_migrator.hh b/db/legacy_schema_migrator.hh index 50d2339938c0..eb3ebd3c4657 100644 --- a/db/legacy_schema_migrator.hh +++ b/db/legacy_schema_migrator.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/marshal/type_parser.cc b/db/marshal/type_parser.cc index 3b60295013af..1aeefe889c41 100644 --- a/db/marshal/type_parser.cc +++ b/db/marshal/type_parser.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "db/marshal/type_parser.hh" diff --git a/db/marshal/type_parser.hh b/db/marshal/type_parser.hh index fae040574beb..13b6644d58d3 100644 --- a/db/marshal/type_parser.hh +++ b/db/marshal/type_parser.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/operation_type.hh b/db/operation_type.hh index b3910ae0b778..4bd422da7f64 100644 --- a/db/operation_type.hh +++ b/db/operation_type.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/paxos_grace_seconds_extension.hh b/db/paxos_grace_seconds_extension.hh index 5c1debf2e50a..d1a3bba595c3 100644 --- a/db/paxos_grace_seconds_extension.hh +++ b/db/paxos_grace_seconds_extension.hh @@ -2,7 +2,7 @@ * Copyright 2020-present ScyllaDB */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/per_partition_rate_limit_extension.hh b/db/per_partition_rate_limit_extension.hh index 300d4e2ff33d..76263e8a28d6 100644 --- a/db/per_partition_rate_limit_extension.hh +++ b/db/per_partition_rate_limit_extension.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/per_partition_rate_limit_info.hh b/db/per_partition_rate_limit_info.hh index e9cf0dfd4545..75dfab7731f3 100644 --- a/db/per_partition_rate_limit_info.hh +++ b/db/per_partition_rate_limit_info.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/per_partition_rate_limit_options.cc b/db/per_partition_rate_limit_options.cc index 5d0da743118d..fa4d9f32cf6d 100644 --- a/db/per_partition_rate_limit_options.cc +++ b/db/per_partition_rate_limit_options.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/per_partition_rate_limit_options.hh b/db/per_partition_rate_limit_options.hh index 29bd15c93705..b099089713c1 100644 --- a/db/per_partition_rate_limit_options.hh +++ b/db/per_partition_rate_limit_options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/rate_limiter.cc b/db/rate_limiter.cc index 195387289a9f..79c3dc61f35d 100644 --- a/db/rate_limiter.cc +++ b/db/rate_limiter.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/rate_limiter.hh b/db/rate_limiter.hh index 62b29e1052a9..b3862dd324d6 100644 --- a/db/rate_limiter.hh +++ b/db/rate_limiter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/read_repair_decision.hh b/db/read_repair_decision.hh index 0c2ccbf74cc9..2abb4a0882e1 100644 --- a/db/read_repair_decision.hh +++ b/db/read_repair_decision.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/schema_applier.cc b/db/schema_applier.cc index 9030a8ebc54a..7cb11c24191f 100644 --- a/db/schema_applier.cc +++ b/db/schema_applier.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "schema_applier.hh" diff --git a/db/schema_applier.hh b/db/schema_applier.hh index 753df827ebc0..fef6af4d906f 100644 --- a/db/schema_applier.hh +++ b/db/schema_applier.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/schema_features.hh b/db/schema_features.hh index ad0fdeb4598a..792a0db2dd8e 100644 --- a/db/schema_features.hh +++ b/db/schema_features.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/schema_tables.cc b/db/schema_tables.cc index 7e621bf4bea6..b7ab716466f4 100644 --- a/db/schema_tables.cc +++ b/db/schema_tables.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "db/schema_tables.hh" diff --git a/db/schema_tables.hh b/db/schema_tables.hh index e6d5039d7441..c7846c9ab841 100644 --- a/db/schema_tables.hh +++ b/db/schema_tables.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/size_estimates_virtual_reader.cc b/db/size_estimates_virtual_reader.cc index 17843455782b..06f02c2d3c77 100644 --- a/db/size_estimates_virtual_reader.cc +++ b/db/size_estimates_virtual_reader.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/size_estimates_virtual_reader.hh b/db/size_estimates_virtual_reader.hh index d3215f659f73..e3f8a4170301 100644 --- a/db/size_estimates_virtual_reader.hh +++ b/db/size_estimates_virtual_reader.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/snapshot-ctl.cc b/db/snapshot-ctl.cc index 8ac0fc3dfddb..1c4e7b6449e6 100644 --- a/db/snapshot-ctl.cc +++ b/db/snapshot-ctl.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) * * Copyright (C) 2020-present ScyllaDB */ diff --git a/db/snapshot-ctl.hh b/db/snapshot-ctl.hh index 81dc5b0c555c..a77a02a37fa6 100644 --- a/db/snapshot-ctl.hh +++ b/db/snapshot-ctl.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) * * Copyright (C) 2020-present ScyllaDB */ diff --git a/db/snapshot/backup_task.cc b/db/snapshot/backup_task.cc index edb717c5f464..c069c0d01aec 100644 --- a/db/snapshot/backup_task.cc +++ b/db/snapshot/backup_task.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/snapshot/backup_task.hh b/db/snapshot/backup_task.hh index 1e5e036b3919..d430fa058918 100644 --- a/db/snapshot/backup_task.hh +++ b/db/snapshot/backup_task.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/sstables-format-selector.cc b/db/sstables-format-selector.cc index 22f422f34efc..baad59fbeada 100644 --- a/db/sstables-format-selector.cc +++ b/db/sstables-format-selector.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/db/sstables-format-selector.hh b/db/sstables-format-selector.hh index 24914506d362..3668b919d66f 100644 --- a/db/sstables-format-selector.hh +++ b/db/sstables-format-selector.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/system_distributed_keyspace.cc b/db/system_distributed_keyspace.cc index e6a2a476cdea..adae10f3b5e9 100644 --- a/db/system_distributed_keyspace.cc +++ b/db/system_distributed_keyspace.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/db/system_distributed_keyspace.hh b/db/system_distributed_keyspace.hh index 69721797347b..6a3a9ab76674 100644 --- a/db/system_distributed_keyspace.hh +++ b/db/system_distributed_keyspace.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 4fd17eef9a6d..075c7a8f462a 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/db/system_keyspace.hh b/db/system_keyspace.hh index a6f051a4325a..cb7e0a4bc656 100644 --- a/db/system_keyspace.hh +++ b/db/system_keyspace.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/system_keyspace_sstables_registry.hh b/db/system_keyspace_sstables_registry.hh index 5ae1ce269872..4b02c6410b60 100644 --- a/db/system_keyspace_sstables_registry.hh +++ b/db/system_keyspace_sstables_registry.hh @@ -1,5 +1,5 @@ // Copyright (C) 2024-present ScyllaDB -// SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +// SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) #pragma once diff --git a/db/system_keyspace_view_types.hh b/db/system_keyspace_view_types.hh index 32754fc268ef..bd842a822573 100644 --- a/db/system_keyspace_view_types.hh +++ b/db/system_keyspace_view_types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/tags/extension.hh b/db/tags/extension.hh index 42a3d2ade26f..7a97ed07ba81 100644 --- a/db/tags/extension.hh +++ b/db/tags/extension.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/tags/utils.cc b/db/tags/utils.cc index 36a2efa1a070..f13464f11111 100644 --- a/db/tags/utils.cc +++ b/db/tags/utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/tags/utils.hh" diff --git a/db/tags/utils.hh b/db/tags/utils.hh index 64eb5609e060..18d95c9f8b4b 100644 --- a/db/tags/utils.hh +++ b/db/tags/utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/timeout_clock.hh b/db/timeout_clock.hh index 1ec2b92da18c..30ffe4df6ada 100644 --- a/db/timeout_clock.hh +++ b/db/timeout_clock.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/build_progress_virtual_reader.hh b/db/view/build_progress_virtual_reader.hh index 6a9b6fa4146c..38e595df64b6 100644 --- a/db/view/build_progress_virtual_reader.hh +++ b/db/view/build_progress_virtual_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/delete_ghost_rows_visitor.hh b/db/view/delete_ghost_rows_visitor.hh index 46af73226d19..166639637911 100644 --- a/db/view/delete_ghost_rows_visitor.hh +++ b/db/view/delete_ghost_rows_visitor.hh @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* Copyright 2022-present ScyllaDB */ diff --git a/db/view/node_view_update_backlog.hh b/db/view/node_view_update_backlog.hh index d146f667936c..8ff7827eddb4 100644 --- a/db/view/node_view_update_backlog.hh +++ b/db/view/node_view_update_backlog.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/row_locking.cc b/db/view/row_locking.cc index 3d0ff751d5a0..729b68285c85 100644 --- a/db/view/row_locking.cc +++ b/db/view/row_locking.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/db/view/row_locking.hh b/db/view/row_locking.hh index 991dfdc40b2b..6c7a13b4fcba 100644 --- a/db/view/row_locking.hh +++ b/db/view/row_locking.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/view.cc b/db/view/view.cc index 6f54dcf1f956..a199a6a28cd6 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/db/view/view.hh b/db/view/view.hh index 6e7ac88e440d..f35a07a63abd 100644 --- a/db/view/view.hh +++ b/db/view/view.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/view_builder.hh b/db/view/view_builder.hh index 970347c95f51..89dfddc070f7 100644 --- a/db/view/view_builder.hh +++ b/db/view/view_builder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/view_stats.hh b/db/view/view_stats.hh index 8da4d808a572..33362a40998c 100644 --- a/db/view/view_stats.hh +++ b/db/view/view_stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/view_update_backlog.hh b/db/view/view_update_backlog.hh index 8fafb93a8b61..a2c36b5f7c47 100644 --- a/db/view/view_update_backlog.hh +++ b/db/view/view_update_backlog.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/view_update_checks.hh b/db/view/view_update_checks.hh index 8b58f57988de..8b3031e3b610 100644 --- a/db/view/view_update_checks.hh +++ b/db/view/view_update_checks.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/view_update_generator.cc b/db/view/view_update_generator.cc index cdf578b0cd1d..93716685ced2 100644 --- a/db/view/view_update_generator.cc +++ b/db/view/view_update_generator.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/view/view_update_backlog.hh" diff --git a/db/view/view_update_generator.hh b/db/view/view_update_generator.hh index 0bc78237789b..918c8594947a 100644 --- a/db/view/view_update_generator.hh +++ b/db/view/view_update_generator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/view/view_updating_consumer.hh b/db/view/view_updating_consumer.hh index 609a4c34fb12..52a258013839 100644 --- a/db/view/view_updating_consumer.hh +++ b/db/view/view_updating_consumer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/virtual_table.cc b/db/virtual_table.cc index 820b5f56cdd1..f4f8997b49fd 100644 --- a/db/virtual_table.cc +++ b/db/virtual_table.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "db/virtual_table.hh" diff --git a/db/virtual_table.hh b/db/virtual_table.hh index f5a7c91ca6b7..18fb8ece7a96 100644 --- a/db/virtual_table.hh +++ b/db/virtual_table.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc index 11aa6078b38a..03f9bbc1e5c6 100644 --- a/db/virtual_tables.cc +++ b/db/virtual_tables.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/db/virtual_tables.hh b/db/virtual_tables.hh index f03421680f3e..63f4d9534d7f 100644 --- a/db/virtual_tables.hh +++ b/db/virtual_tables.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db/write_type.hh b/db/write_type.hh index 106685371edc..af6ddbe2252d 100644 --- a/db/write_type.hh +++ b/db/write_type.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/db_clock.hh b/db_clock.hh index da73ecaf557e..4857dc8f7d84 100644 --- a/db_clock.hh +++ b/db_clock.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/debug.cc b/debug.cc index 7861013d27f1..fac4aec30fc9 100644 --- a/debug.cc +++ b/debug.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "debug.hh" diff --git a/debug.hh b/debug.hh index 34047e935d63..b0874e19597a 100644 --- a/debug.hh +++ b/debug.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/default.nix b/default.nix index 9e217e487ded..9ddcd6215f2d 100644 --- a/default.nix +++ b/default.nix @@ -2,7 +2,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # diff --git a/dht/auto_refreshing_sharder.hh b/dht/auto_refreshing_sharder.hh index c00661f86ce1..7ad6e6ed2205 100644 --- a/dht/auto_refreshing_sharder.hh +++ b/dht/auto_refreshing_sharder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/dht/boot_strapper.cc b/dht/boot_strapper.cc index 16ceb250f4db..5656634a86f6 100644 --- a/dht/boot_strapper.cc +++ b/dht/boot_strapper.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/dht/boot_strapper.hh b/dht/boot_strapper.hh index f10ab426d5a6..41f34504c317 100644 --- a/dht/boot_strapper.hh +++ b/dht/boot_strapper.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once #include "gms/inet_address.hh" diff --git a/dht/decorated_key.hh b/dht/decorated_key.hh index 941e8fecd2d5..657a3bbe722d 100644 --- a/dht/decorated_key.hh +++ b/dht/decorated_key.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/dht/i_partitioner.cc b/dht/i_partitioner.cc index affcd652415b..7029e5f736a1 100644 --- a/dht/i_partitioner.cc +++ b/dht/i_partitioner.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "i_partitioner.hh" diff --git a/dht/i_partitioner.hh b/dht/i_partitioner.hh index 266eda5e6db8..aaddd072fb2b 100644 --- a/dht/i_partitioner.hh +++ b/dht/i_partitioner.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/dht/i_partitioner_fwd.hh b/dht/i_partitioner_fwd.hh index 782e00c8dd7c..b6e8f39b1ca9 100644 --- a/dht/i_partitioner_fwd.hh +++ b/dht/i_partitioner_fwd.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/dht/murmur3_partitioner.cc b/dht/murmur3_partitioner.cc index bdd3ce645bb8..62cccdf9b1c5 100644 --- a/dht/murmur3_partitioner.cc +++ b/dht/murmur3_partitioner.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "murmur3_partitioner.hh" diff --git a/dht/murmur3_partitioner.hh b/dht/murmur3_partitioner.hh index 6681ad9bd8ab..5f60104cebca 100644 --- a/dht/murmur3_partitioner.hh +++ b/dht/murmur3_partitioner.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/dht/partition_filter.hh b/dht/partition_filter.hh index 8b082072659b..2abba6ea28aa 100644 --- a/dht/partition_filter.hh +++ b/dht/partition_filter.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/dht/range_streamer.cc b/dht/range_streamer.cc index cdb439bfff14..9bafa1ccd355 100644 --- a/dht/range_streamer.cc +++ b/dht/range_streamer.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "dht/range_streamer.hh" diff --git a/dht/range_streamer.hh b/dht/range_streamer.hh index 356a20310397..0f13bee2998a 100644 --- a/dht/range_streamer.hh +++ b/dht/range_streamer.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/dht/ring_position.hh b/dht/ring_position.hh index 5bc3c114033b..dd9ededf45f5 100644 --- a/dht/ring_position.hh +++ b/dht/ring_position.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/dht/sharder.hh b/dht/sharder.hh index b03a6bfd9c5a..67dbf57ac8fb 100644 --- a/dht/sharder.hh +++ b/dht/sharder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/dht/token-sharding.hh b/dht/token-sharding.hh index 5cc3a8d0e68b..3cb4a0864891 100644 --- a/dht/token-sharding.hh +++ b/dht/token-sharding.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/dht/token.cc b/dht/token.cc index be6ae2bfd70c..c6b4420aee8a 100644 --- a/dht/token.cc +++ b/dht/token.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/dht/token.hh b/dht/token.hh index e25b5809b18f..98e753ca5642 100644 --- a/dht/token.hh +++ b/dht/token.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/dht/token_range_endpoints.hh b/dht/token_range_endpoints.hh index d4d1c18923e3..de3bcbc8e92e 100644 --- a/dht/token_range_endpoints.hh +++ b/dht/token_range_endpoints.hh @@ -3,7 +3,7 @@ */ -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once #include diff --git a/direct_failure_detector/failure_detector.cc b/direct_failure_detector/failure_detector.cc index bf3ffe784860..a4520a8098bb 100644 --- a/direct_failure_detector/failure_detector.cc +++ b/direct_failure_detector/failure_detector.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/direct_failure_detector/failure_detector.hh b/direct_failure_detector/failure_detector.hh index f122dd85af41..a7ea07334d61 100644 --- a/direct_failure_detector/failure_detector.hh +++ b/direct_failure_detector/failure_detector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/dist/common/dep/scylla-env-1.0/debian/copyright b/dist/common/dep/scylla-env-1.0/debian/copyright index 3a80cb9902e7..09153d8ca59f 100644 --- a/dist/common/dep/scylla-env-1.0/debian/copyright +++ b/dist/common/dep/scylla-env-1.0/debian/copyright @@ -5,672 +5,8 @@ Source: https://github.com/scylladb/scylla Files: * Copyright: Copyright (c) 2016-present ScyllaDB -License: AGPL-3.0 +License: ScyllaDB-Source-Available-1.0 Files: debian/* Copyright: Copyright (c) 2016-present ScyllaDB -License: AGPL-3.0 - -License: AGPL-3.0 - GNU AFFERO GENERAL PUBLIC LICENSE - Version 3, 19 November 2007 - . - Copyright (C) 2007 Free Software Foundation, Inc. - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - . - Preamble - . - The GNU Affero General Public License is a free, copyleft license for - software and other kinds of works, specifically designed to ensure - cooperation with the community in the case of network server software. - . - The licenses for most software and other practical works are designed - to take away your freedom to share and change the works. By contrast, - our General Public Licenses are intended to guarantee your freedom to - share and change all versions of a program--to make sure it remains free - software for all its users. - . - When we speak of free software, we are referring to freedom, not - price. Our General Public Licenses are designed to make sure that you - have the freedom to distribute copies of free software (and charge for - them if you wish), that you receive source code or can get it if you - want it, that you can change the software or use pieces of it in new - free programs, and that you know you can do these things. - . - Developers that use our General Public Licenses protect your rights - with two steps: (1) assert copyright on the software, and (2) offer - you this License which gives you legal permission to copy, distribute - and/or modify the software. - . - A secondary benefit of defending all users' freedom is that - improvements made in alternate versions of the program, if they - receive widespread use, become available for other developers to - incorporate. Many developers of free software are heartened and - encouraged by the resulting cooperation. However, in the case of - software used on network servers, this result may fail to come about. - The GNU General Public License permits making a modified version and - letting the public access it on a server without ever releasing its - source code to the public. - . - The GNU Affero General Public License is designed specifically to - ensure that, in such cases, the modified source code becomes available - to the community. It requires the operator of a network server to - provide the source code of the modified version running there to the - users of that server. Therefore, public use of a modified version, on - a publicly accessible server, gives the public access to the source - code of the modified version. - . - An older license, called the Affero General Public License and - published by Affero, was designed to accomplish similar goals. This is - a different license, not a version of the Affero GPL, but Affero has - released a new version of the Affero GPL which permits relicensing under - this license. - . - The precise terms and conditions for copying, distribution and - modification follow. - . - TERMS AND CONDITIONS - . - 0. Definitions. - . - "This License" refers to version 3 of the GNU Affero General Public License. - . - "Copyright" also means copyright-like laws that apply to other kinds of - works, such as semiconductor masks. - . - "The Program" refers to any copyrightable work licensed under this - License. Each licensee is addressed as "you". "Licensees" and - "recipients" may be individuals or organizations. - . - To "modify" a work means to copy from or adapt all or part of the work - in a fashion requiring copyright permission, other than the making of an - exact copy. The resulting work is called a "modified version" of the - earlier work or a work "based on" the earlier work. - . - A "covered work" means either the unmodified Program or a work based - on the Program. - . - To "propagate" a work means to do anything with it that, without - permission, would make you directly or secondarily liable for - infringement under applicable copyright law, except executing it on a - computer or modifying a private copy. Propagation includes copying, - distribution (with or without modification), making available to the - public, and in some countries other activities as well. - . - To "convey" a work means any kind of propagation that enables other - parties to make or receive copies. Mere interaction with a user through - a computer network, with no transfer of a copy, is not conveying. - . - An interactive user interface displays "Appropriate Legal Notices" - to the extent that it includes a convenient and prominently visible - feature that (1) displays an appropriate copyright notice, and (2) - tells the user that there is no warranty for the work (except to the - extent that warranties are provided), that licensees may convey the - work under this License, and how to view a copy of this License. If - the interface presents a list of user commands or options, such as a - menu, a prominent item in the list meets this criterion. - . - 1. Source Code. - . - The "source code" for a work means the preferred form of the work - for making modifications to it. "Object code" means any non-source - form of a work. - . - A "Standard Interface" means an interface that either is an official - standard defined by a recognized standards body, or, in the case of - interfaces specified for a particular programming language, one that - is widely used among developers working in that language. - . - The "System Libraries" of an executable work include anything, other - than the work as a whole, that (a) is included in the normal form of - packaging a Major Component, but which is not part of that Major - Component, and (b) serves only to enable use of the work with that - Major Component, or to implement a Standard Interface for which an - implementation is available to the public in source code form. A - "Major Component", in this context, means a major essential component - (kernel, window system, and so on) of the specific operating system - (if any) on which the executable work runs, or a compiler used to - produce the work, or an object code interpreter used to run it. - . - The "Corresponding Source" for a work in object code form means all - the source code needed to generate, install, and (for an executable - work) run the object code and to modify the work, including scripts to - control those activities. However, it does not include the work's - System Libraries, or general-purpose tools or generally available free - programs which are used unmodified in performing those activities but - which are not part of the work. For example, Corresponding Source - includes interface definition files associated with source files for - the work, and the source code for shared libraries and dynamically - linked subprograms that the work is specifically designed to require, - such as by intimate data communication or control flow between those - subprograms and other parts of the work. - . - The Corresponding Source need not include anything that users - can regenerate automatically from other parts of the Corresponding - Source. - . - The Corresponding Source for a work in source code form is that - same work. - . - 2. Basic Permissions. - . - All rights granted under this License are granted for the term of - copyright on the Program, and are irrevocable provided the stated - conditions are met. This License explicitly affirms your unlimited - permission to run the unmodified Program. The output from running a - covered work is covered by this License only if the output, given its - content, constitutes a covered work. This License acknowledges your - rights of fair use or other equivalent, as provided by copyright law. - . - You may make, run and propagate covered works that you do not - convey, without conditions so long as your license otherwise remains - in force. You may convey covered works to others for the sole purpose - of having them make modifications exclusively for you, or provide you - with facilities for running those works, provided that you comply with - the terms of this License in conveying all material for which you do - not control copyright. Those thus making or running the covered works - for you must do so exclusively on your behalf, under your direction - and control, on terms that prohibit them from making any copies of - your copyrighted material outside their relationship with you. - . - Conveying under any other circumstances is permitted solely under - the conditions stated below. Sublicensing is not allowed; section 10 - makes it unnecessary. - . - 3. Protecting Users' Legal Rights From Anti-Circumvention Law. - . - No covered work shall be deemed part of an effective technological - measure under any applicable law fulfilling obligations under article - 11 of the WIPO copyright treaty adopted on 20 December 1996, or - similar laws prohibiting or restricting circumvention of such - measures. - . - When you convey a covered work, you waive any legal power to forbid - circumvention of technological measures to the extent such circumvention - is effected by exercising rights under this License with respect to - the covered work, and you disclaim any intention to limit operation or - modification of the work as a means of enforcing, against the work's - users, your or third parties' legal rights to forbid circumvention of - technological measures. - . - 4. Conveying Verbatim Copies. - . - You may convey verbatim copies of the Program's source code as you - receive it, in any medium, provided that you conspicuously and - appropriately publish on each copy an appropriate copyright notice; - keep intact all notices stating that this License and any - non-permissive terms added in accord with section 7 apply to the code; - keep intact all notices of the absence of any warranty; and give all - recipients a copy of this License along with the Program. - . - You may charge any price or no price for each copy that you convey, - and you may offer support or warranty protection for a fee. - . - 5. Conveying Modified Source Versions. - . - You may convey a work based on the Program, or the modifications to - produce it from the Program, in the form of source code under the - terms of section 4, provided that you also meet all of these conditions: - . - a) The work must carry prominent notices stating that you modified - it, and giving a relevant date. - . - b) The work must carry prominent notices stating that it is - released under this License and any conditions added under section - 7. This requirement modifies the requirement in section 4 to - "keep intact all notices". - . - c) You must license the entire work, as a whole, under this - License to anyone who comes into possession of a copy. This - License will therefore apply, along with any applicable section 7 - additional terms, to the whole of the work, and all its parts, - regardless of how they are packaged. This License gives no - permission to license the work in any other way, but it does not - invalidate such permission if you have separately received it. - . - d) If the work has interactive user interfaces, each must display - Appropriate Legal Notices; however, if the Program has interactive - interfaces that do not display Appropriate Legal Notices, your - work need not make them do so. - . - A compilation of a covered work with other separate and independent - works, which are not by their nature extensions of the covered work, - and which are not combined with it such as to form a larger program, - in or on a volume of a storage or distribution medium, is called an - "aggregate" if the compilation and its resulting copyright are not - used to limit the access or legal rights of the compilation's users - beyond what the individual works permit. Inclusion of a covered work - in an aggregate does not cause this License to apply to the other - parts of the aggregate. - . - 6. Conveying Non-Source Forms. - . - You may convey a covered work in object code form under the terms - of sections 4 and 5, provided that you also convey the - machine-readable Corresponding Source under the terms of this License, - in one of these ways: - . - a) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by the - Corresponding Source fixed on a durable physical medium - customarily used for software interchange. - . - b) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by a - written offer, valid for at least three years and valid for as - long as you offer spare parts or customer support for that product - model, to give anyone who possesses the object code either (1) a - copy of the Corresponding Source for all the software in the - product that is covered by this License, on a durable physical - medium customarily used for software interchange, for a price no - more than your reasonable cost of physically performing this - conveying of source, or (2) access to copy the - Corresponding Source from a network server at no charge. - . - c) Convey individual copies of the object code with a copy of the - written offer to provide the Corresponding Source. This - alternative is allowed only occasionally and noncommercially, and - only if you received the object code with such an offer, in accord - with subsection 6b. - . - d) Convey the object code by offering access from a designated - place (gratis or for a charge), and offer equivalent access to the - Corresponding Source in the same way through the same place at no - further charge. You need not require recipients to copy the - Corresponding Source along with the object code. If the place to - copy the object code is a network server, the Corresponding Source - may be on a different server (operated by you or a third party) - that supports equivalent copying facilities, provided you maintain - clear directions next to the object code saying where to find the - Corresponding Source. Regardless of what server hosts the - Corresponding Source, you remain obligated to ensure that it is - available for as long as needed to satisfy these requirements. - . - e) Convey the object code using peer-to-peer transmission, provided - you inform other peers where the object code and Corresponding - Source of the work are being offered to the general public at no - charge under subsection 6d. - . - A separable portion of the object code, whose source code is excluded - from the Corresponding Source as a System Library, need not be - included in conveying the object code work. - . - A "User Product" is either (1) a "consumer product", which means any - tangible personal property which is normally used for personal, family, - or household purposes, or (2) anything designed or sold for incorporation - into a dwelling. In determining whether a product is a consumer product, - doubtful cases shall be resolved in favor of coverage. For a particular - product received by a particular user, "normally used" refers to a - typical or common use of that class of product, regardless of the status - of the particular user or of the way in which the particular user - actually uses, or expects or is expected to use, the product. A product - is a consumer product regardless of whether the product has substantial - commercial, industrial or non-consumer uses, unless such uses represent - the only significant mode of use of the product. - . - "Installation Information" for a User Product means any methods, - procedures, authorization keys, or other information required to install - and execute modified versions of a covered work in that User Product from - a modified version of its Corresponding Source. The information must - suffice to ensure that the continued functioning of the modified object - code is in no case prevented or interfered with solely because - modification has been made. - . - If you convey an object code work under this section in, or with, or - specifically for use in, a User Product, and the conveying occurs as - part of a transaction in which the right of possession and use of the - User Product is transferred to the recipient in perpetuity or for a - fixed term (regardless of how the transaction is characterized), the - Corresponding Source conveyed under this section must be accompanied - by the Installation Information. But this requirement does not apply - if neither you nor any third party retains the ability to install - modified object code on the User Product (for example, the work has - been installed in ROM). - . - The requirement to provide Installation Information does not include a - requirement to continue to provide support service, warranty, or updates - for a work that has been modified or installed by the recipient, or for - the User Product in which it has been modified or installed. Access to a - network may be denied when the modification itself materially and - adversely affects the operation of the network or violates the rules and - protocols for communication across the network. - . - Corresponding Source conveyed, and Installation Information provided, - in accord with this section must be in a format that is publicly - documented (and with an implementation available to the public in - source code form), and must require no special password or key for - unpacking, reading or copying. - . - 7. Additional Terms. - . - "Additional permissions" are terms that supplement the terms of this - License by making exceptions from one or more of its conditions. - Additional permissions that are applicable to the entire Program shall - be treated as though they were included in this License, to the extent - that they are valid under applicable law. If additional permissions - apply only to part of the Program, that part may be used separately - under those permissions, but the entire Program remains governed by - this License without regard to the additional permissions. - . - When you convey a copy of a covered work, you may at your option - remove any additional permissions from that copy, or from any part of - it. (Additional permissions may be written to require their own - removal in certain cases when you modify the work.) You may place - additional permissions on material, added by you to a covered work, - for which you have or can give appropriate copyright permission. - . - Notwithstanding any other provision of this License, for material you - add to a covered work, you may (if authorized by the copyright holders of - that material) supplement the terms of this License with terms: - . - a) Disclaiming warranty or limiting liability differently from the - terms of sections 15 and 16 of this License; or - . - b) Requiring preservation of specified reasonable legal notices or - author attributions in that material or in the Appropriate Legal - Notices displayed by works containing it; or - . - c) Prohibiting misrepresentation of the origin of that material, or - requiring that modified versions of such material be marked in - reasonable ways as different from the original version; or - . - d) Limiting the use for publicity purposes of names of licensors or - authors of the material; or - . - e) Declining to grant rights under trademark law for use of some - trade names, trademarks, or service marks; or - . - f) Requiring indemnification of licensors and authors of that - material by anyone who conveys the material (or modified versions of - it) with contractual assumptions of liability to the recipient, for - any liability that these contractual assumptions directly impose on - those licensors and authors. - . - All other non-permissive additional terms are considered "further - restrictions" within the meaning of section 10. If the Program as you - received it, or any part of it, contains a notice stating that it is - governed by this License along with a term that is a further - restriction, you may remove that term. If a license document contains - a further restriction but permits relicensing or conveying under this - License, you may add to a covered work material governed by the terms - of that license document, provided that the further restriction does - not survive such relicensing or conveying. - . - If you add terms to a covered work in accord with this section, you - must place, in the relevant source files, a statement of the - additional terms that apply to those files, or a notice indicating - where to find the applicable terms. - . - Additional terms, permissive or non-permissive, may be stated in the - form of a separately written license, or stated as exceptions; - the above requirements apply either way. - . - 8. Termination. - . - You may not propagate or modify a covered work except as expressly - provided under this License. Any attempt otherwise to propagate or - modify it is void, and will automatically terminate your rights under - this License (including any patent licenses granted under the third - paragraph of section 11). - . - However, if you cease all violation of this License, then your - license from a particular copyright holder is reinstated (a) - provisionally, unless and until the copyright holder explicitly and - finally terminates your license, and (b) permanently, if the copyright - holder fails to notify you of the violation by some reasonable means - prior to 60 days after the cessation. - . - Moreover, your license from a particular copyright holder is - reinstated permanently if the copyright holder notifies you of the - violation by some reasonable means, this is the first time you have - received notice of violation of this License (for any work) from that - copyright holder, and you cure the violation prior to 30 days after - your receipt of the notice. - . - Termination of your rights under this section does not terminate the - licenses of parties who have received copies or rights from you under - this License. If your rights have been terminated and not permanently - reinstated, you do not qualify to receive new licenses for the same - material under section 10. - . - 9. Acceptance Not Required for Having Copies. - . - You are not required to accept this License in order to receive or - run a copy of the Program. Ancillary propagation of a covered work - occurring solely as a consequence of using peer-to-peer transmission - to receive a copy likewise does not require acceptance. However, - nothing other than this License grants you permission to propagate or - modify any covered work. These actions infringe copyright if you do - not accept this License. Therefore, by modifying or propagating a - covered work, you indicate your acceptance of this License to do so. - . - 10. Automatic Licensing of Downstream Recipients. - . - Each time you convey a covered work, the recipient automatically - receives a license from the original licensors, to run, modify and - propagate that work, subject to this License. You are not responsible - for enforcing compliance by third parties with this License. - . - An "entity transaction" is a transaction transferring control of an - organization, or substantially all assets of one, or subdividing an - organization, or merging organizations. If propagation of a covered - work results from an entity transaction, each party to that - transaction who receives a copy of the work also receives whatever - licenses to the work the party's predecessor in interest had or could - give under the previous paragraph, plus a right to possession of the - Corresponding Source of the work from the predecessor in interest, if - the predecessor has it or can get it with reasonable efforts. - . - You may not impose any further restrictions on the exercise of the - rights granted or affirmed under this License. For example, you may - not impose a license fee, royalty, or other charge for exercise of - rights granted under this License, and you may not initiate litigation - (including a cross-claim or counterclaim in a lawsuit) alleging that - any patent claim is infringed by making, using, selling, offering for - sale, or importing the Program or any portion of it. - . - 11. Patents. - . - A "contributor" is a copyright holder who authorizes use under this - License of the Program or a work on which the Program is based. The - work thus licensed is called the contributor's "contributor version". - . - A contributor's "essential patent claims" are all patent claims - owned or controlled by the contributor, whether already acquired or - hereafter acquired, that would be infringed by some manner, permitted - by this License, of making, using, or selling its contributor version, - but do not include claims that would be infringed only as a - consequence of further modification of the contributor version. For - purposes of this definition, "control" includes the right to grant - patent sublicenses in a manner consistent with the requirements of - this License. - . - Each contributor grants you a non-exclusive, worldwide, royalty-free - patent license under the contributor's essential patent claims, to - make, use, sell, offer for sale, import and otherwise run, modify and - propagate the contents of its contributor version. - . - In the following three paragraphs, a "patent license" is any express - agreement or commitment, however denominated, not to enforce a patent - (such as an express permission to practice a patent or covenant not to - sue for patent infringement). To "grant" such a patent license to a - party means to make such an agreement or commitment not to enforce a - patent against the party. - . - If you convey a covered work, knowingly relying on a patent license, - and the Corresponding Source of the work is not available for anyone - to copy, free of charge and under the terms of this License, through a - publicly available network server or other readily accessible means, - then you must either (1) cause the Corresponding Source to be so - available, or (2) arrange to deprive yourself of the benefit of the - patent license for this particular work, or (3) arrange, in a manner - consistent with the requirements of this License, to extend the patent - license to downstream recipients. "Knowingly relying" means you have - actual knowledge that, but for the patent license, your conveying the - covered work in a country, or your recipient's use of the covered work - in a country, would infringe one or more identifiable patents in that - country that you have reason to believe are valid. - . - If, pursuant to or in connection with a single transaction or - arrangement, you convey, or propagate by procuring conveyance of, a - covered work, and grant a patent license to some of the parties - receiving the covered work authorizing them to use, propagate, modify - or convey a specific copy of the covered work, then the patent license - you grant is automatically extended to all recipients of the covered - work and works based on it. - . - A patent license is "discriminatory" if it does not include within - the scope of its coverage, prohibits the exercise of, or is - conditioned on the non-exercise of one or more of the rights that are - specifically granted under this License. You may not convey a covered - work if you are a party to an arrangement with a third party that is - in the business of distributing software, under which you make payment - to the third party based on the extent of your activity of conveying - the work, and under which the third party grants, to any of the - parties who would receive the covered work from you, a discriminatory - patent license (a) in connection with copies of the covered work - conveyed by you (or copies made from those copies), or (b) primarily - for and in connection with specific products or compilations that - contain the covered work, unless you entered into that arrangement, - or that patent license was granted, prior to 28 March 2007. - . - Nothing in this License shall be construed as excluding or limiting - any implied license or other defenses to infringement that may - otherwise be available to you under applicable patent law. - . - 12. No Surrender of Others' Freedom. - . - If conditions are imposed on you (whether by court order, agreement or - otherwise) that contradict the conditions of this License, they do not - excuse you from the conditions of this License. If you cannot convey a - covered work so as to satisfy simultaneously your obligations under this - License and any other pertinent obligations, then as a consequence you may - not convey it at all. For example, if you agree to terms that obligate you - to collect a royalty for further conveying from those to whom you convey - the Program, the only way you could satisfy both those terms and this - License would be to refrain entirely from conveying the Program. - . - 13. Remote Network Interaction; Use with the GNU General Public License. - . - Notwithstanding any other provision of this License, if you modify the - Program, your modified version must prominently offer all users - interacting with it remotely through a computer network (if your version - supports such interaction) an opportunity to receive the Corresponding - Source of your version by providing access to the Corresponding Source - from a network server at no charge, through some standard or customary - means of facilitating copying of software. This Corresponding Source - shall include the Corresponding Source for any work covered by version 3 - of the GNU General Public License that is incorporated pursuant to the - following paragraph. - . - Notwithstanding any other provision of this License, you have - permission to link or combine any covered work with a work licensed - under version 3 of the GNU General Public License into a single - combined work, and to convey the resulting work. The terms of this - License will continue to apply to the part which is the covered work, - but the work with which it is combined will remain governed by version - 3 of the GNU General Public License. - . - 14. Revised Versions of this License. - . - The Free Software Foundation may publish revised and/or new versions of - the GNU Affero General Public License from time to time. Such new versions - will be similar in spirit to the present version, but may differ in detail to - address new problems or concerns. - . - Each version is given a distinguishing version number. If the - Program specifies that a certain numbered version of the GNU Affero General - Public License "or any later version" applies to it, you have the - option of following the terms and conditions either of that numbered - version or of any later version published by the Free Software - Foundation. If the Program does not specify a version number of the - GNU Affero General Public License, you may choose any version ever published - by the Free Software Foundation. - . - If the Program specifies that a proxy can decide which future - versions of the GNU Affero General Public License can be used, that proxy's - public statement of acceptance of a version permanently authorizes you - to choose that version for the Program. - . - Later license versions may give you additional or different - permissions. However, no additional obligations are imposed on any - author or copyright holder as a result of your choosing to follow a - later version. - . - 15. Disclaimer of Warranty. - . - THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY - APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT - HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY - OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, - THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR - PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM - IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF - ALL NECESSARY SERVICING, REPAIR OR CORRECTION. - . - 16. Limitation of Liability. - . - IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING - WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MODIFIES AND/OR CONVEYS - THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY - GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE - USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF - DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD - PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), - EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF - SUCH DAMAGES. - . - 17. Interpretation of Sections 15 and 16. - . - If the disclaimer of warranty and limitation of liability provided - above cannot be given local legal effect according to their terms, - reviewing courts shall apply local law that most closely approximates - an absolute waiver of all civil liability in connection with the - Program, unless a warranty or assumption of liability accompanies a - copy of the Program in return for a fee. - . - END OF TERMS AND CONDITIONS - . - How to Apply These Terms to Your New Programs - . - If you develop a new program, and you want it to be of the greatest - possible use to the public, the best way to achieve this is to make it - free software which everyone can redistribute and change under these terms. - . - To do so, attach the following notices to the program. It is safest - to attach them to the start of each source file to most effectively - state the exclusion of warranty; and each file should have at least - the "copyright" line and a pointer to where the full notice is found. - . - - Copyright (C) - . - This program is free software: you can redistribute it and/or modify - it under the terms of the GNU Affero General Public License as published by - the Free Software Foundation, either version 3 of the License, or - (at your option) any later version. - . - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU Affero General Public License for more details. - . - You should have received a copy of the GNU Affero General Public License - along with this program. If not, see . - . - Also add information on how to contact you by electronic and paper mail. - . - If your software can interact with users remotely through a computer - network, you should also make sure that it provides a way for users to - get its source. For example, if your program is a web application, its - interface could display a "Source" link that leads users to an archive - of the code. There are many ways you could offer source, and different - solutions will be better for different programs; see section 13 for the - specific requirements. - . - You should also get your employer (if you work as a programmer) or school, - if any, to sign a "copyright disclaimer" for the program, if necessary. - For more information on this, and how to apply and follow the GNU AGPL, see - . - +License: ScyllaDB-Source-Available-1.0 diff --git a/dist/common/dep/scylla-env.spec b/dist/common/dep/scylla-env.spec index 3c11e4163ba1..a0c475424c29 100644 --- a/dist/common/dep/scylla-env.spec +++ b/dist/common/dep/scylla-env.spec @@ -4,7 +4,7 @@ Release: 1%{?dist} Summary: Scylla is a highly scalable, eventually consistent, distributed, partitioned row DB. Group: Applications/Databases -License: AGPLv3 +License: ScyllaDB-Source-Available-1.0 URL: http://www.scylladb.com/ Source0: scylla-env-1.0.tar BuildArch: noarch diff --git a/dist/common/kernel_conf/post_install.sh b/dist/common/kernel_conf/post_install.sh index 6b28acd675f2..375c55c5445f 100644 --- a/dist/common/kernel_conf/post_install.sh +++ b/dist/common/kernel_conf/post_install.sh @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # version_ge() { diff --git a/dist/common/kernel_conf/scylla_tune_sched b/dist/common/kernel_conf/scylla_tune_sched index 87f5e6813cda..5fd579d080c5 100755 --- a/dist/common/kernel_conf/scylla_tune_sched +++ b/dist/common/kernel_conf/scylla_tune_sched @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/nodetool-completion b/dist/common/nodetool-completion index 9e6957a1a988..d6f6e3d3b2cb 100644 --- a/dist/common/nodetool-completion +++ b/dist/common/nodetool-completion @@ -7,7 +7,7 @@ # Modified by ScyllaDB # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # have nodetool && have cqlsh && diff --git a/dist/common/scripts/node_health_check b/dist/common/scripts/node_health_check index b9c3451d8ab0..f121c73d57f9 100755 --- a/dist/common/scripts/node_health_check +++ b/dist/common/scripts/node_health_check @@ -2,7 +2,7 @@ # # Copyright (C) 2017-present ScyllaDB -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # When a CLI tool is not installed, use relocatable CLI tool provided by Scylla export PATH=$PATH:/opt/scylladb/bin diff --git a/dist/common/scripts/scylla-blocktune b/dist/common/scripts/scylla-blocktune index 7404465b86b6..9cb69763da21 100755 --- a/dist/common/scripts/scylla-blocktune +++ b/dist/common/scripts/scylla-blocktune @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import argparse from scylla_blocktune import * diff --git a/dist/common/scripts/scylla-housekeeping b/dist/common/scripts/scylla-housekeeping index ef6c3e2aaddc..27f087b657da 100755 --- a/dist/common/scripts/scylla-housekeeping +++ b/dist/common/scripts/scylla-housekeeping @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/dist/common/scripts/scylla_blocktune.py b/dist/common/scripts/scylla_blocktune.py index 32235674136e..02039180ebd8 100644 --- a/dist/common/scripts/scylla_blocktune.py +++ b/dist/common/scripts/scylla_blocktune.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import os diff --git a/dist/common/scripts/scylla_config_get.py b/dist/common/scripts/scylla_config_get.py index c19e2c12570d..45d43e26c615 100755 --- a/dist/common/scripts/scylla_config_get.py +++ b/dist/common/scripts/scylla_config_get.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import argparse import sys diff --git a/dist/common/scripts/scylla_coredump_setup b/dist/common/scripts/scylla_coredump_setup index cb27ecc44c53..b05c87c193f4 100755 --- a/dist/common/scripts/scylla_coredump_setup +++ b/dist/common/scripts/scylla_coredump_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_cpuscaling_setup b/dist/common/scripts/scylla_cpuscaling_setup index bd1f196076ff..7379f4b79bb7 100755 --- a/dist/common/scripts/scylla_cpuscaling_setup +++ b/dist/common/scripts/scylla_cpuscaling_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_cpuset_setup b/dist/common/scripts/scylla_cpuset_setup index 6f503c0936d0..36babfc1f923 100755 --- a/dist/common/scripts/scylla_cpuset_setup +++ b/dist/common/scripts/scylla_cpuset_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_dev_mode_setup b/dist/common/scripts/scylla_dev_mode_setup index 911d47206d6a..41002c6b8479 100755 --- a/dist/common/scripts/scylla_dev_mode_setup +++ b/dist/common/scripts/scylla_dev_mode_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_fstrim b/dist/common/scripts/scylla_fstrim index 3e9cb250809f..961c70d19ccf 100755 --- a/dist/common/scripts/scylla_fstrim +++ b/dist/common/scripts/scylla_fstrim @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_fstrim_setup b/dist/common/scripts/scylla_fstrim_setup index e9f466673b4d..56de6f218791 100755 --- a/dist/common/scripts/scylla_fstrim_setup +++ b/dist/common/scripts/scylla_fstrim_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import subprocess diff --git a/dist/common/scripts/scylla_io_setup b/dist/common/scripts/scylla_io_setup index 8c35eda016e7..c185cb667e6a 100755 --- a/dist/common/scripts/scylla_io_setup +++ b/dist/common/scripts/scylla_io_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import re diff --git a/dist/common/scripts/scylla_kernel_check b/dist/common/scripts/scylla_kernel_check index b71ec17a0579..19418f0e3010 100755 --- a/dist/common/scripts/scylla_kernel_check +++ b/dist/common/scripts/scylla_kernel_check @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_logrotate b/dist/common/scripts/scylla_logrotate index d65504c447e1..df50489a0e23 100755 --- a/dist/common/scripts/scylla_logrotate +++ b/dist/common/scripts/scylla_logrotate @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 from pathlib import Path from datetime import datetime diff --git a/dist/common/scripts/scylla_memory_setup b/dist/common/scripts/scylla_memory_setup index 0d726429c5ef..12a8da228715 100755 --- a/dist/common/scripts/scylla_memory_setup +++ b/dist/common/scripts/scylla_memory_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_nofile_setup b/dist/common/scripts/scylla_nofile_setup index 2b5c575e877b..cb12e76d4717 100755 --- a/dist/common/scripts/scylla_nofile_setup +++ b/dist/common/scripts/scylla_nofile_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_ntp_setup b/dist/common/scripts/scylla_ntp_setup index 4511265ff31f..e43935486de0 100755 --- a/dist/common/scripts/scylla_ntp_setup +++ b/dist/common/scripts/scylla_ntp_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_prepare b/dist/common/scripts/scylla_prepare index 2b3c777e8ff4..8f3e4b2fc2dd 100755 --- a/dist/common/scripts/scylla_prepare +++ b/dist/common/scripts/scylla_prepare @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_raid_setup b/dist/common/scripts/scylla_raid_setup index c10b0df0184c..479f8aa851a8 100755 --- a/dist/common/scripts/scylla_raid_setup +++ b/dist/common/scripts/scylla_raid_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import argparse diff --git a/dist/common/scripts/scylla_rsyslog_setup b/dist/common/scripts/scylla_rsyslog_setup index f611a99a59ef..cb66d48da97c 100755 --- a/dist/common/scripts/scylla_rsyslog_setup +++ b/dist/common/scripts/scylla_rsyslog_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import argparse diff --git a/dist/common/scripts/scylla_selinux_setup b/dist/common/scripts/scylla_selinux_setup index ac1e3a3817a3..5fa8e3ea6a58 100755 --- a/dist/common/scripts/scylla_selinux_setup +++ b/dist/common/scripts/scylla_selinux_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_setup b/dist/common/scripts/scylla_setup index ad7efe64b978..91519dcc2445 100755 --- a/dist/common/scripts/scylla_setup +++ b/dist/common/scripts/scylla_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_stop b/dist/common/scripts/scylla_stop index a742e3a849fe..f414827243bb 100755 --- a/dist/common/scripts/scylla_stop +++ b/dist/common/scripts/scylla_stop @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_swap_setup b/dist/common/scripts/scylla_swap_setup index 31b5011d7306..b45f83085340 100755 --- a/dist/common/scripts/scylla_swap_setup +++ b/dist/common/scripts/scylla_swap_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_sysconfig_setup b/dist/common/scripts/scylla_sysconfig_setup index 71af79129e9d..c775cb2f46ce 100755 --- a/dist/common/scripts/scylla_sysconfig_setup +++ b/dist/common/scripts/scylla_sysconfig_setup @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import os import sys diff --git a/dist/common/scripts/scylla_util.py b/dist/common/scripts/scylla_util.py index 990bdc31ffc1..a5068bbfc48b 100644 --- a/dist/common/scripts/scylla_util.py +++ b/dist/common/scripts/scylla_util.py @@ -1,6 +1,6 @@ # Copyright (C) 2017-present ScyllaDB -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import configparser import glob diff --git a/dist/debian/debian/copyright b/dist/debian/debian/copyright index 5436969a9cb1..010a80e9dafd 100644 --- a/dist/debian/debian/copyright +++ b/dist/debian/debian/copyright @@ -5,7 +5,7 @@ Source: https://github.com/scylladb/scylla Files: * Copyright: Copyright (C) 2015-present ScyllaDB -License: AGPL-3.0 +License: ScyllaDB-Source-Available-1.0 Files: seastar/* Copyright: Copyright (C) 2015-present ScyllaDB @@ -15,714 +15,4 @@ Files: seastar/dpdk/* Copyright: Copyright(c) 2015 Intel Corporation. All rights reserved. License: BSD-3-clause -License: AGPL-3.0 - GNU AFFERO GENERAL PUBLIC LICENSE - Version 3, 19 November 2007 - . - Copyright (C) 2007 Free Software Foundation, Inc. - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - . - Preamble - . - The GNU Affero General Public License is a free, copyleft license for - software and other kinds of works, specifically designed to ensure - cooperation with the community in the case of network server software. - . - The licenses for most software and other practical works are designed - to take away your freedom to share and change the works. By contrast, - our General Public Licenses are intended to guarantee your freedom to - share and change all versions of a program--to make sure it remains free - software for all its users. - . - When we speak of free software, we are referring to freedom, not - price. Our General Public Licenses are designed to make sure that you - have the freedom to distribute copies of free software (and charge for - them if you wish), that you receive source code or can get it if you - want it, that you can change the software or use pieces of it in new - free programs, and that you know you can do these things. - . - Developers that use our General Public Licenses protect your rights - with two steps: (1) assert copyright on the software, and (2) offer - you this License which gives you legal permission to copy, distribute - and/or modify the software. - . - A secondary benefit of defending all users' freedom is that - improvements made in alternate versions of the program, if they - receive widespread use, become available for other developers to - incorporate. Many developers of free software are heartened and - encouraged by the resulting cooperation. However, in the case of - software used on network servers, this result may fail to come about. - The GNU General Public License permits making a modified version and - letting the public access it on a server without ever releasing its - source code to the public. - . - The GNU Affero General Public License is designed specifically to - ensure that, in such cases, the modified source code becomes available - to the community. It requires the operator of a network server to - provide the source code of the modified version running there to the - users of that server. Therefore, public use of a modified version, on - a publicly accessible server, gives the public access to the source - code of the modified version. - . - An older license, called the Affero General Public License and - published by Affero, was designed to accomplish similar goals. This is - a different license, not a version of the Affero GPL, but Affero has - released a new version of the Affero GPL which permits relicensing under - this license. - . - The precise terms and conditions for copying, distribution and - modification follow. - . - TERMS AND CONDITIONS - . - 0. Definitions. - . - "This License" refers to version 3 of the GNU Affero General Public License. - . - "Copyright" also means copyright-like laws that apply to other kinds of - works, such as semiconductor masks. - . - "The Program" refers to any copyrightable work licensed under this - License. Each licensee is addressed as "you". "Licensees" and - "recipients" may be individuals or organizations. - . - To "modify" a work means to copy from or adapt all or part of the work - in a fashion requiring copyright permission, other than the making of an - exact copy. The resulting work is called a "modified version" of the - earlier work or a work "based on" the earlier work. - . - A "covered work" means either the unmodified Program or a work based - on the Program. - . - To "propagate" a work means to do anything with it that, without - permission, would make you directly or secondarily liable for - infringement under applicable copyright law, except executing it on a - computer or modifying a private copy. Propagation includes copying, - distribution (with or without modification), making available to the - public, and in some countries other activities as well. - . - To "convey" a work means any kind of propagation that enables other - parties to make or receive copies. Mere interaction with a user through - a computer network, with no transfer of a copy, is not conveying. - . - An interactive user interface displays "Appropriate Legal Notices" - to the extent that it includes a convenient and prominently visible - feature that (1) displays an appropriate copyright notice, and (2) - tells the user that there is no warranty for the work (except to the - extent that warranties are provided), that licensees may convey the - work under this License, and how to view a copy of this License. If - the interface presents a list of user commands or options, such as a - menu, a prominent item in the list meets this criterion. - . - 1. Source Code. - . - The "source code" for a work means the preferred form of the work - for making modifications to it. "Object code" means any non-source - form of a work. - . - A "Standard Interface" means an interface that either is an official - standard defined by a recognized standards body, or, in the case of - interfaces specified for a particular programming language, one that - is widely used among developers working in that language. - . - The "System Libraries" of an executable work include anything, other - than the work as a whole, that (a) is included in the normal form of - packaging a Major Component, but which is not part of that Major - Component, and (b) serves only to enable use of the work with that - Major Component, or to implement a Standard Interface for which an - implementation is available to the public in source code form. A - "Major Component", in this context, means a major essential component - (kernel, window system, and so on) of the specific operating system - (if any) on which the executable work runs, or a compiler used to - produce the work, or an object code interpreter used to run it. - . - The "Corresponding Source" for a work in object code form means all - the source code needed to generate, install, and (for an executable - work) run the object code and to modify the work, including scripts to - control those activities. However, it does not include the work's - System Libraries, or general-purpose tools or generally available free - programs which are used unmodified in performing those activities but - which are not part of the work. For example, Corresponding Source - includes interface definition files associated with source files for - the work, and the source code for shared libraries and dynamically - linked subprograms that the work is specifically designed to require, - such as by intimate data communication or control flow between those - subprograms and other parts of the work. - . - The Corresponding Source need not include anything that users - can regenerate automatically from other parts of the Corresponding - Source. - . - The Corresponding Source for a work in source code form is that - same work. - . - 2. Basic Permissions. - . - All rights granted under this License are granted for the term of - copyright on the Program, and are irrevocable provided the stated - conditions are met. This License explicitly affirms your unlimited - permission to run the unmodified Program. The output from running a - covered work is covered by this License only if the output, given its - content, constitutes a covered work. This License acknowledges your - rights of fair use or other equivalent, as provided by copyright law. - . - You may make, run and propagate covered works that you do not - convey, without conditions so long as your license otherwise remains - in force. You may convey covered works to others for the sole purpose - of having them make modifications exclusively for you, or provide you - with facilities for running those works, provided that you comply with - the terms of this License in conveying all material for which you do - not control copyright. Those thus making or running the covered works - for you must do so exclusively on your behalf, under your direction - and control, on terms that prohibit them from making any copies of - your copyrighted material outside their relationship with you. - . - Conveying under any other circumstances is permitted solely under - the conditions stated below. Sublicensing is not allowed; section 10 - makes it unnecessary. - . - 3. Protecting Users' Legal Rights From Anti-Circumvention Law. - . - No covered work shall be deemed part of an effective technological - measure under any applicable law fulfilling obligations under article - 11 of the WIPO copyright treaty adopted on 20 December 1996, or - similar laws prohibiting or restricting circumvention of such - measures. - . - When you convey a covered work, you waive any legal power to forbid - circumvention of technological measures to the extent such circumvention - is effected by exercising rights under this License with respect to - the covered work, and you disclaim any intention to limit operation or - modification of the work as a means of enforcing, against the work's - users, your or third parties' legal rights to forbid circumvention of - technological measures. - . - 4. Conveying Verbatim Copies. - . - You may convey verbatim copies of the Program's source code as you - receive it, in any medium, provided that you conspicuously and - appropriately publish on each copy an appropriate copyright notice; - keep intact all notices stating that this License and any - non-permissive terms added in accord with section 7 apply to the code; - keep intact all notices of the absence of any warranty; and give all - recipients a copy of this License along with the Program. - . - You may charge any price or no price for each copy that you convey, - and you may offer support or warranty protection for a fee. - . - 5. Conveying Modified Source Versions. - . - You may convey a work based on the Program, or the modifications to - produce it from the Program, in the form of source code under the - terms of section 4, provided that you also meet all of these conditions: - . - a) The work must carry prominent notices stating that you modified - it, and giving a relevant date. - . - b) The work must carry prominent notices stating that it is - released under this License and any conditions added under section - 7. This requirement modifies the requirement in section 4 to - "keep intact all notices". - . - c) You must license the entire work, as a whole, under this - License to anyone who comes into possession of a copy. This - License will therefore apply, along with any applicable section 7 - additional terms, to the whole of the work, and all its parts, - regardless of how they are packaged. This License gives no - permission to license the work in any other way, but it does not - invalidate such permission if you have separately received it. - . - d) If the work has interactive user interfaces, each must display - Appropriate Legal Notices; however, if the Program has interactive - interfaces that do not display Appropriate Legal Notices, your - work need not make them do so. - . - A compilation of a covered work with other separate and independent - works, which are not by their nature extensions of the covered work, - and which are not combined with it such as to form a larger program, - in or on a volume of a storage or distribution medium, is called an - "aggregate" if the compilation and its resulting copyright are not - used to limit the access or legal rights of the compilation's users - beyond what the individual works permit. Inclusion of a covered work - in an aggregate does not cause this License to apply to the other - parts of the aggregate. - . - 6. Conveying Non-Source Forms. - . - You may convey a covered work in object code form under the terms - of sections 4 and 5, provided that you also convey the - machine-readable Corresponding Source under the terms of this License, - in one of these ways: - . - a) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by the - Corresponding Source fixed on a durable physical medium - customarily used for software interchange. - . - b) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by a - written offer, valid for at least three years and valid for as - long as you offer spare parts or customer support for that product - model, to give anyone who possesses the object code either (1) a - copy of the Corresponding Source for all the software in the - product that is covered by this License, on a durable physical - medium customarily used for software interchange, for a price no - more than your reasonable cost of physically performing this - conveying of source, or (2) access to copy the - Corresponding Source from a network server at no charge. - . - c) Convey individual copies of the object code with a copy of the - written offer to provide the Corresponding Source. This - alternative is allowed only occasionally and noncommercially, and - only if you received the object code with such an offer, in accord - with subsection 6b. - . - d) Convey the object code by offering access from a designated - place (gratis or for a charge), and offer equivalent access to the - Corresponding Source in the same way through the same place at no - further charge. You need not require recipients to copy the - Corresponding Source along with the object code. If the place to - copy the object code is a network server, the Corresponding Source - may be on a different server (operated by you or a third party) - that supports equivalent copying facilities, provided you maintain - clear directions next to the object code saying where to find the - Corresponding Source. Regardless of what server hosts the - Corresponding Source, you remain obligated to ensure that it is - available for as long as needed to satisfy these requirements. - . - e) Convey the object code using peer-to-peer transmission, provided - you inform other peers where the object code and Corresponding - Source of the work are being offered to the general public at no - charge under subsection 6d. - . - A separable portion of the object code, whose source code is excluded - from the Corresponding Source as a System Library, need not be - included in conveying the object code work. - . - A "User Product" is either (1) a "consumer product", which means any - tangible personal property which is normally used for personal, family, - or household purposes, or (2) anything designed or sold for incorporation - into a dwelling. In determining whether a product is a consumer product, - doubtful cases shall be resolved in favor of coverage. For a particular - product received by a particular user, "normally used" refers to a - typical or common use of that class of product, regardless of the status - of the particular user or of the way in which the particular user - actually uses, or expects or is expected to use, the product. A product - is a consumer product regardless of whether the product has substantial - commercial, industrial or non-consumer uses, unless such uses represent - the only significant mode of use of the product. - . - "Installation Information" for a User Product means any methods, - procedures, authorization keys, or other information required to install - and execute modified versions of a covered work in that User Product from - a modified version of its Corresponding Source. The information must - suffice to ensure that the continued functioning of the modified object - code is in no case prevented or interfered with solely because - modification has been made. - . - If you convey an object code work under this section in, or with, or - specifically for use in, a User Product, and the conveying occurs as - part of a transaction in which the right of possession and use of the - User Product is transferred to the recipient in perpetuity or for a - fixed term (regardless of how the transaction is characterized), the - Corresponding Source conveyed under this section must be accompanied - by the Installation Information. But this requirement does not apply - if neither you nor any third party retains the ability to install - modified object code on the User Product (for example, the work has - been installed in ROM). - . - The requirement to provide Installation Information does not include a - requirement to continue to provide support service, warranty, or updates - for a work that has been modified or installed by the recipient, or for - the User Product in which it has been modified or installed. Access to a - network may be denied when the modification itself materially and - adversely affects the operation of the network or violates the rules and - protocols for communication across the network. - . - Corresponding Source conveyed, and Installation Information provided, - in accord with this section must be in a format that is publicly - documented (and with an implementation available to the public in - source code form), and must require no special password or key for - unpacking, reading or copying. - . - 7. Additional Terms. - . - "Additional permissions" are terms that supplement the terms of this - License by making exceptions from one or more of its conditions. - Additional permissions that are applicable to the entire Program shall - be treated as though they were included in this License, to the extent - that they are valid under applicable law. If additional permissions - apply only to part of the Program, that part may be used separately - under those permissions, but the entire Program remains governed by - this License without regard to the additional permissions. - . - When you convey a copy of a covered work, you may at your option - remove any additional permissions from that copy, or from any part of - it. (Additional permissions may be written to require their own - removal in certain cases when you modify the work.) You may place - additional permissions on material, added by you to a covered work, - for which you have or can give appropriate copyright permission. - . - Notwithstanding any other provision of this License, for material you - add to a covered work, you may (if authorized by the copyright holders of - that material) supplement the terms of this License with terms: - . - a) Disclaiming warranty or limiting liability differently from the - terms of sections 15 and 16 of this License; or - . - b) Requiring preservation of specified reasonable legal notices or - author attributions in that material or in the Appropriate Legal - Notices displayed by works containing it; or - . - c) Prohibiting misrepresentation of the origin of that material, or - requiring that modified versions of such material be marked in - reasonable ways as different from the original version; or - . - d) Limiting the use for publicity purposes of names of licensors or - authors of the material; or - . - e) Declining to grant rights under trademark law for use of some - trade names, trademarks, or service marks; or - . - f) Requiring indemnification of licensors and authors of that - material by anyone who conveys the material (or modified versions of - it) with contractual assumptions of liability to the recipient, for - any liability that these contractual assumptions directly impose on - those licensors and authors. - . - All other non-permissive additional terms are considered "further - restrictions" within the meaning of section 10. If the Program as you - received it, or any part of it, contains a notice stating that it is - governed by this License along with a term that is a further - restriction, you may remove that term. If a license document contains - a further restriction but permits relicensing or conveying under this - License, you may add to a covered work material governed by the terms - of that license document, provided that the further restriction does - not survive such relicensing or conveying. - . - If you add terms to a covered work in accord with this section, you - must place, in the relevant source files, a statement of the - additional terms that apply to those files, or a notice indicating - where to find the applicable terms. - . - Additional terms, permissive or non-permissive, may be stated in the - form of a separately written license, or stated as exceptions; - the above requirements apply either way. - . - 8. Termination. - . - You may not propagate or modify a covered work except as expressly - provided under this License. Any attempt otherwise to propagate or - modify it is void, and will automatically terminate your rights under - this License (including any patent licenses granted under the third - paragraph of section 11). - . - However, if you cease all violation of this License, then your - license from a particular copyright holder is reinstated (a) - provisionally, unless and until the copyright holder explicitly and - finally terminates your license, and (b) permanently, if the copyright - holder fails to notify you of the violation by some reasonable means - prior to 60 days after the cessation. - . - Moreover, your license from a particular copyright holder is - reinstated permanently if the copyright holder notifies you of the - violation by some reasonable means, this is the first time you have - received notice of violation of this License (for any work) from that - copyright holder, and you cure the violation prior to 30 days after - your receipt of the notice. - . - Termination of your rights under this section does not terminate the - licenses of parties who have received copies or rights from you under - this License. If your rights have been terminated and not permanently - reinstated, you do not qualify to receive new licenses for the same - material under section 10. - . - 9. Acceptance Not Required for Having Copies. - . - You are not required to accept this License in order to receive or - run a copy of the Program. Ancillary propagation of a covered work - occurring solely as a consequence of using peer-to-peer transmission - to receive a copy likewise does not require acceptance. However, - nothing other than this License grants you permission to propagate or - modify any covered work. These actions infringe copyright if you do - not accept this License. Therefore, by modifying or propagating a - covered work, you indicate your acceptance of this License to do so. - . - 10. Automatic Licensing of Downstream Recipients. - . - Each time you convey a covered work, the recipient automatically - receives a license from the original licensors, to run, modify and - propagate that work, subject to this License. You are not responsible - for enforcing compliance by third parties with this License. - . - An "entity transaction" is a transaction transferring control of an - organization, or substantially all assets of one, or subdividing an - organization, or merging organizations. If propagation of a covered - work results from an entity transaction, each party to that - transaction who receives a copy of the work also receives whatever - licenses to the work the party's predecessor in interest had or could - give under the previous paragraph, plus a right to possession of the - Corresponding Source of the work from the predecessor in interest, if - the predecessor has it or can get it with reasonable efforts. - . - You may not impose any further restrictions on the exercise of the - rights granted or affirmed under this License. For example, you may - not impose a license fee, royalty, or other charge for exercise of - rights granted under this License, and you may not initiate litigation - (including a cross-claim or counterclaim in a lawsuit) alleging that - any patent claim is infringed by making, using, selling, offering for - sale, or importing the Program or any portion of it. - . - 11. Patents. - . - A "contributor" is a copyright holder who authorizes use under this - License of the Program or a work on which the Program is based. The - work thus licensed is called the contributor's "contributor version". - . - A contributor's "essential patent claims" are all patent claims - owned or controlled by the contributor, whether already acquired or - hereafter acquired, that would be infringed by some manner, permitted - by this License, of making, using, or selling its contributor version, - but do not include claims that would be infringed only as a - consequence of further modification of the contributor version. For - purposes of this definition, "control" includes the right to grant - patent sublicenses in a manner consistent with the requirements of - this License. - . - Each contributor grants you a non-exclusive, worldwide, royalty-free - patent license under the contributor's essential patent claims, to - make, use, sell, offer for sale, import and otherwise run, modify and - propagate the contents of its contributor version. - . - In the following three paragraphs, a "patent license" is any express - agreement or commitment, however denominated, not to enforce a patent - (such as an express permission to practice a patent or covenant not to - sue for patent infringement). To "grant" such a patent license to a - party means to make such an agreement or commitment not to enforce a - patent against the party. - . - If you convey a covered work, knowingly relying on a patent license, - and the Corresponding Source of the work is not available for anyone - to copy, free of charge and under the terms of this License, through a - publicly available network server or other readily accessible means, - then you must either (1) cause the Corresponding Source to be so - available, or (2) arrange to deprive yourself of the benefit of the - patent license for this particular work, or (3) arrange, in a manner - consistent with the requirements of this License, to extend the patent - license to downstream recipients. "Knowingly relying" means you have - actual knowledge that, but for the patent license, your conveying the - covered work in a country, or your recipient's use of the covered work - in a country, would infringe one or more identifiable patents in that - country that you have reason to believe are valid. - . - If, pursuant to or in connection with a single transaction or - arrangement, you convey, or propagate by procuring conveyance of, a - covered work, and grant a patent license to some of the parties - receiving the covered work authorizing them to use, propagate, modify - or convey a specific copy of the covered work, then the patent license - you grant is automatically extended to all recipients of the covered - work and works based on it. - . - A patent license is "discriminatory" if it does not include within - the scope of its coverage, prohibits the exercise of, or is - conditioned on the non-exercise of one or more of the rights that are - specifically granted under this License. You may not convey a covered - work if you are a party to an arrangement with a third party that is - in the business of distributing software, under which you make payment - to the third party based on the extent of your activity of conveying - the work, and under which the third party grants, to any of the - parties who would receive the covered work from you, a discriminatory - patent license (a) in connection with copies of the covered work - conveyed by you (or copies made from those copies), or (b) primarily - for and in connection with specific products or compilations that - contain the covered work, unless you entered into that arrangement, - or that patent license was granted, prior to 28 March 2007. - . - Nothing in this License shall be construed as excluding or limiting - any implied license or other defenses to infringement that may - otherwise be available to you under applicable patent law. - . - 12. No Surrender of Others' Freedom. - . - If conditions are imposed on you (whether by court order, agreement or - otherwise) that contradict the conditions of this License, they do not - excuse you from the conditions of this License. If you cannot convey a - covered work so as to satisfy simultaneously your obligations under this - License and any other pertinent obligations, then as a consequence you may - not convey it at all. For example, if you agree to terms that obligate you - to collect a royalty for further conveying from those to whom you convey - the Program, the only way you could satisfy both those terms and this - License would be to refrain entirely from conveying the Program. - . - 13. Remote Network Interaction; Use with the GNU General Public License. - . - Notwithstanding any other provision of this License, if you modify the - Program, your modified version must prominently offer all users - interacting with it remotely through a computer network (if your version - supports such interaction) an opportunity to receive the Corresponding - Source of your version by providing access to the Corresponding Source - from a network server at no charge, through some standard or customary - means of facilitating copying of software. This Corresponding Source - shall include the Corresponding Source for any work covered by version 3 - of the GNU General Public License that is incorporated pursuant to the - following paragraph. - . - Notwithstanding any other provision of this License, you have - permission to link or combine any covered work with a work licensed - under version 3 of the GNU General Public License into a single - combined work, and to convey the resulting work. The terms of this - License will continue to apply to the part which is the covered work, - but the work with which it is combined will remain governed by version - 3 of the GNU General Public License. - . - 14. Revised Versions of this License. - . - The Free Software Foundation may publish revised and/or new versions of - the GNU Affero General Public License from time to time. Such new versions - will be similar in spirit to the present version, but may differ in detail to - address new problems or concerns. - . - Each version is given a distinguishing version number. If the - Program specifies that a certain numbered version of the GNU Affero General - Public License "or any later version" applies to it, you have the - option of following the terms and conditions either of that numbered - version or of any later version published by the Free Software - Foundation. If the Program does not specify a version number of the - GNU Affero General Public License, you may choose any version ever published - by the Free Software Foundation. - . - If the Program specifies that a proxy can decide which future - versions of the GNU Affero General Public License can be used, that proxy's - public statement of acceptance of a version permanently authorizes you - to choose that version for the Program. - . - Later license versions may give you additional or different - permissions. However, no additional obligations are imposed on any - author or copyright holder as a result of your choosing to follow a - later version. - . - 15. Disclaimer of Warranty. - . - THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY - APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT - HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY - OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, - THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR - PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM - IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF - ALL NECESSARY SERVICING, REPAIR OR CORRECTION. - . - 16. Limitation of Liability. - . - IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING - WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MODIFIES AND/OR CONVEYS - THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY - GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE - USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF - DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD - PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), - EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF - SUCH DAMAGES. - . - 17. Interpretation of Sections 15 and 16. - . - If the disclaimer of warranty and limitation of liability provided - above cannot be given local legal effect according to their terms, - reviewing courts shall apply local law that most closely approximates - an absolute waiver of all civil liability in connection with the - Program, unless a warranty or assumption of liability accompanies a - copy of the Program in return for a fee. - . - END OF TERMS AND CONDITIONS - . - How to Apply These Terms to Your New Programs - . - If you develop a new program, and you want it to be of the greatest - possible use to the public, the best way to achieve this is to make it - free software which everyone can redistribute and change under these terms. - . - To do so, attach the following notices to the program. It is safest - to attach them to the start of each source file to most effectively - state the exclusion of warranty; and each file should have at least - the "copyright" line and a pointer to where the full notice is found. - . - - Copyright (C) - . - This program is free software: you can redistribute it and/or modify - it under the terms of the GNU Affero General Public License as published by - the Free Software Foundation, either version 3 of the License, or - (at your option) any later version. - . - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU Affero General Public License for more details. - . - You should have received a copy of the GNU Affero General Public License - along with this program. If not, see . - . - Also add information on how to contact you by electronic and paper mail. - . - If your software can interact with users remotely through a computer - network, you should also make sure that it provides a way for users to - get its source. For example, if your program is a web application, its - interface could display a "Source" link that leads users to an archive - of the code. There are many ways you could offer source, and different - solutions will be better for different programs; see section 13 for the - specific requirements. - . - You should also get your employer (if you work as a programmer) or school, - if any, to sign a "copyright disclaimer" for the program, if necessary. - For more information on this, and how to apply and follow the GNU AGPL, see - . - -License: Apache-2.0 - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - . - http://www.apache.org/licenses/LICENSE-2.0 - . - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - . - On Debian systems, the complete text of the Apache version 2.0 license - can be found in "/usr/share/common-licenses/Apache-2.0". - -License: BSD-3-clause - BSD LICENSE - . - Copyright(c) 2010-2014 Intel Corporation. All rights reserved. - Copyright(c) 2012-2014 6WIND S.A. - All rights reserved. - . - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - . - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in - the documentation and/or other materials provided with the - distribution. - * Neither the name of Intel Corporation nor the names of its - contributors may be used to endorse or promote products derived - from this software without specific prior written permission. - . - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +License: ScyllaDB-Source-Available-1.0 \ No newline at end of file diff --git a/dist/debian/debian_files_gen.py b/dist/debian/debian_files_gen.py index 397046739e61..e9782393f8d6 100755 --- a/dist/debian/debian_files_gen.py +++ b/dist/debian/debian_files_gen.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/dist/debuginfo/install.sh b/dist/debuginfo/install.sh index e6e5d8ef6965..72b36d906e79 100755 --- a/dist/debuginfo/install.sh +++ b/dist/debuginfo/install.sh @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # set -e diff --git a/dist/debuginfo/scripts/create-relocatable-package.py b/dist/debuginfo/scripts/create-relocatable-package.py index 969babab8a3d..eccff1835a24 100755 --- a/dist/debuginfo/scripts/create-relocatable-package.py +++ b/dist/debuginfo/scripts/create-relocatable-package.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/dist/docker/debian/build_docker.sh b/dist/docker/debian/build_docker.sh index 8e2ce7065790..2f7a2603ca94 100755 --- a/dist/docker/debian/build_docker.sh +++ b/dist/docker/debian/build_docker.sh @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # product="$( diff --git a/gms/inet_address.hh b/gms/inet_address.hh index 7a0caca7d4e1..c43794876673 100644 --- a/gms/inet_address.hh +++ b/gms/inet_address.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/gms/inet_address_serializer.hh b/gms/inet_address_serializer.hh index ddac6ea611d4..13df8d46b00d 100644 --- a/gms/inet_address_serializer.hh +++ b/gms/inet_address_serializer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/gms/version_generator.cc b/gms/version_generator.cc index 9dc9102daa5e..1fa61747564e 100644 --- a/gms/version_generator.cc +++ b/gms/version_generator.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/gms/version_generator.hh b/gms/version_generator.hh index b0d4312f3b89..512dc430b31a 100644 --- a/gms/version_generator.hh +++ b/gms/version_generator.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/gms/versioned_value.cc b/gms/versioned_value.cc index 4dbc758dcc2c..9b441e331357 100644 --- a/gms/versioned_value.cc +++ b/gms/versioned_value.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "gms/versioned_value.hh" #include "message/messaging_service.hh" diff --git a/gms/versioned_value.hh b/gms/versioned_value.hh index 122d0cd5f5c8..85b9cb4a3091 100644 --- a/gms/versioned_value.hh +++ b/gms/versioned_value.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/hashing_partition_visitor.hh b/hashing_partition_visitor.hh index 5dd4470dfa1d..49f76ff310d1 100644 --- a/hashing_partition_visitor.hh +++ b/hashing_partition_visitor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/idl-compiler.py b/idl-compiler.py index 45650ffbae89..9c70b3963162 100755 --- a/idl-compiler.py +++ b/idl-compiler.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import argparse import pyparsing as pp @@ -47,7 +47,7 @@ def print_cw(f): * Copyright 2016-present ScyllaDB */ -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 /* * This is an auto-generated code, do not modify directly. diff --git a/idl/cache_temperature.idl.hh b/idl/cache_temperature.idl.hh index e0e3bfacfb78..b90171f2c1b6 100644 --- a/idl/cache_temperature.idl.hh +++ b/idl/cache_temperature.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ class cache_temperature final { diff --git a/idl/commitlog.idl.hh b/idl/commitlog.idl.hh index 973818b1c4f2..c57ab8d804f5 100644 --- a/idl/commitlog.idl.hh +++ b/idl/commitlog.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "idl/mutation.idl.hh" diff --git a/idl/consistency_level.idl.hh b/idl/consistency_level.idl.hh index b8936b427a22..5b199a95aff0 100644 --- a/idl/consistency_level.idl.hh +++ b/idl/consistency_level.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace db { diff --git a/idl/experimental/broadcast_tables_lang.idl.hh b/idl/experimental/broadcast_tables_lang.idl.hh index 3c8a7772a73e..e5ecb9d88318 100644 --- a/idl/experimental/broadcast_tables_lang.idl.hh +++ b/idl/experimental/broadcast_tables_lang.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace service { diff --git a/idl/frozen_mutation.idl.hh b/idl/frozen_mutation.idl.hh index ab33fd5d7b28..a9cfc9ac5513 100644 --- a/idl/frozen_mutation.idl.hh +++ b/idl/frozen_mutation.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ class frozen_mutation final { diff --git a/idl/frozen_schema.idl.hh b/idl/frozen_schema.idl.hh index 7f1919edf676..d84babfcd59c 100644 --- a/idl/frozen_schema.idl.hh +++ b/idl/frozen_schema.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation/canonical_mutation.hh" diff --git a/idl/gossip.idl.hh b/idl/gossip.idl.hh index 2a2124cc36d9..89691f998bd7 100644 --- a/idl/gossip.idl.hh +++ b/idl/gossip.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "gms/inet_address.hh" #include "gms/gossip_digest_syn.hh" diff --git a/idl/gossip_digest.idl.hh b/idl/gossip_digest.idl.hh index d8bf42c394f5..5172695229f0 100644 --- a/idl/gossip_digest.idl.hh +++ b/idl/gossip_digest.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "gms/inet_address_serializer.hh" diff --git a/idl/group0.idl.hh b/idl/group0.idl.hh index 0f2d8276b7d1..97cacada52a4 100644 --- a/idl/group0.idl.hh +++ b/idl/group0.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "idl/raft_storage.idl.hh" diff --git a/idl/group0_state_machine.idl.hh b/idl/group0_state_machine.idl.hh index a1f4c889d81f..56829d0411ed 100644 --- a/idl/group0_state_machine.idl.hh +++ b/idl/group0_state_machine.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "raft/raft.hh" diff --git a/idl/hinted_handoff.idl.hh b/idl/hinted_handoff.idl.hh index 77e038a30658..250f15e90fa6 100644 --- a/idl/hinted_handoff.idl.hh +++ b/idl/hinted_handoff.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "gms/inet_address_serializer.hh" diff --git a/idl/idl_test.idl.hh b/idl/idl_test.idl.hh index 4003bd5b5449..c80dfb67324a 100644 --- a/idl/idl_test.idl.hh +++ b/idl/idl_test.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // TODO: test final types more diff --git a/idl/join_node.idl.hh b/idl/join_node.idl.hh index a6673a1a4452..839c8f767b4d 100644 --- a/idl/join_node.idl.hh +++ b/idl/join_node.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace service { diff --git a/idl/keys.idl.hh b/idl/keys.idl.hh index 51e281675cfd..cd3158abc172 100644 --- a/idl/keys.idl.hh +++ b/idl/keys.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ class clustering_key_prefix { diff --git a/idl/mapreduce_request.idl.hh b/idl/mapreduce_request.idl.hh index 167b4fae6513..1bf205d924ef 100644 --- a/idl/mapreduce_request.idl.hh +++ b/idl/mapreduce_request.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "dht/i_partitioner_fwd.hh" diff --git a/idl/messaging_service.idl.hh b/idl/messaging_service.idl.hh index ea8cb6313b38..16af9b727365 100644 --- a/idl/messaging_service.idl.hh +++ b/idl/messaging_service.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace netw { diff --git a/idl/migration_manager.idl.hh b/idl/migration_manager.idl.hh index 7696890a7a64..21b58c11c4e7 100644 --- a/idl/migration_manager.idl.hh +++ b/idl/migration_manager.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "idl/frozen_mutation.idl.hh" diff --git a/idl/mutation.idl.hh b/idl/mutation.idl.hh index 288872f07088..1d8adb7d677b 100644 --- a/idl/mutation.idl.hh +++ b/idl/mutation.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "counters.hh" diff --git a/idl/node_ops.idl.hh b/idl/node_ops.idl.hh index 0279456c3647..e1a6e2b85cef 100644 --- a/idl/node_ops.idl.hh +++ b/idl/node_ops.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "node_ops/id.hh" diff --git a/idl/paging_state.idl.hh b/idl/paging_state.idl.hh index 0d8742ca7580..0566e0605b96 100644 --- a/idl/paging_state.idl.hh +++ b/idl/paging_state.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "dht/i_partitioner_fwd.hh" diff --git a/idl/paxos.idl.hh b/idl/paxos.idl.hh index a557f490630b..9674d72aee35 100644 --- a/idl/paxos.idl.hh +++ b/idl/paxos.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "idl/result.idl.hh" diff --git a/idl/per_partition_rate_limit_info.idl.hh b/idl/per_partition_rate_limit_info.idl.hh index 87f5aebe8997..3eebb19859de 100644 --- a/idl/per_partition_rate_limit_info.idl.hh +++ b/idl/per_partition_rate_limit_info.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/per_partition_rate_limit_info.hh" diff --git a/idl/position_in_partition.idl.hh b/idl/position_in_partition.idl.hh index a4dcb5cd76cd..62e1618a95ba 100644 --- a/idl/position_in_partition.idl.hh +++ b/idl/position_in_partition.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "idl/keys.idl.hh" diff --git a/idl/query.idl.hh b/idl/query.idl.hh index fd41716a7a85..e134dd32b665 100644 --- a/idl/query.idl.hh +++ b/idl/query.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "idl/keys.idl.hh" diff --git a/idl/raft.idl.hh b/idl/raft.idl.hh index 81845257c5de..674523c87b5b 100644 --- a/idl/raft.idl.hh +++ b/idl/raft.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "raft/raft.hh" diff --git a/idl/raft_storage.idl.hh b/idl/raft_storage.idl.hh index ec98d308a31f..cdcf1efed4e2 100644 --- a/idl/raft_storage.idl.hh +++ b/idl/raft_storage.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "raft/raft.hh" diff --git a/idl/range.idl.hh b/idl/range.idl.hh index fb75b8f62205..9c28e7d7e76f 100644 --- a/idl/range.idl.hh +++ b/idl/range.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ template diff --git a/idl/read_command.idl.hh b/idl/read_command.idl.hh index 3ca5d3d4584a..e03ce3f62a28 100644 --- a/idl/read_command.idl.hh +++ b/idl/read_command.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "query-request.hh" diff --git a/idl/reconcilable_result.idl.hh b/idl/reconcilable_result.idl.hh index 840ce55eba66..25625feee84f 100644 --- a/idl/reconcilable_result.idl.hh +++ b/idl/reconcilable_result.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ class partition { diff --git a/idl/repair.idl.hh b/idl/repair.idl.hh index 981fcaf45819..f6ae961b2021 100644 --- a/idl/repair.idl.hh +++ b/idl/repair.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "dht/decorated_key.hh" diff --git a/idl/replay_position.idl.hh b/idl/replay_position.idl.hh index 1290686e4c70..fed592798d81 100644 --- a/idl/replay_position.idl.hh +++ b/idl/replay_position.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace db { diff --git a/idl/replica_exception.idl.hh b/idl/replica_exception.idl.hh index 88bf6d2f986b..b5a434ffc8ab 100644 --- a/idl/replica_exception.idl.hh +++ b/idl/replica_exception.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace replica { diff --git a/idl/result.idl.hh b/idl/result.idl.hh index a6ea2c84eb42..5cd5f2fbfd32 100644 --- a/idl/result.idl.hh +++ b/idl/result.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace query { diff --git a/idl/ring_position.idl.hh b/idl/ring_position.idl.hh index 19945432bbf8..6c940f9b60d1 100644 --- a/idl/ring_position.idl.hh +++ b/idl/ring_position.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace dht { diff --git a/idl/storage_proxy.idl.hh b/idl/storage_proxy.idl.hh index b02870d2736f..c2e6230255eb 100644 --- a/idl/storage_proxy.idl.hh +++ b/idl/storage_proxy.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "inet_address_vectors.hh" diff --git a/idl/storage_service.idl.hh b/idl/storage_service.idl.hh index 2e300ff7a6ad..48fd5a31e702 100644 --- a/idl/storage_service.idl.hh +++ b/idl/storage_service.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace locator { diff --git a/idl/streaming.idl.hh b/idl/streaming.idl.hh index 82c101f93827..61539410d0c4 100644 --- a/idl/streaming.idl.hh +++ b/idl/streaming.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "idl/range.idl.hh" diff --git a/idl/token.idl.hh b/idl/token.idl.hh index 99438da65a0b..5e93ac8e1f88 100644 --- a/idl/token.idl.hh +++ b/idl/token.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "dht/token.hh" diff --git a/idl/tracing.idl.hh b/idl/tracing.idl.hh index 244bf9b182aa..6952153d0a13 100644 --- a/idl/tracing.idl.hh +++ b/idl/tracing.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "tracing/tracing.hh" diff --git a/idl/utils.idl.hh b/idl/utils.idl.hh index bfe7a3c98de6..fea5e8b770f9 100644 --- a/idl/utils.idl.hh +++ b/idl/utils.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/tagged_integer.hh" diff --git a/idl/uuid.idl.hh b/idl/uuid.idl.hh index 2b99b2560ac3..b69946fe04b8 100644 --- a/idl/uuid.idl.hh +++ b/idl/uuid.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/UUID.hh" diff --git a/idl/view.idl.hh b/idl/view.idl.hh index 76e979ba872e..6ff92af2d0f5 100644 --- a/idl/view.idl.hh +++ b/idl/view.idl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ namespace db { diff --git a/index/built_indexes_virtual_reader.hh b/index/built_indexes_virtual_reader.hh index a1861d81ac7a..c5d78aa47231 100644 --- a/index/built_indexes_virtual_reader.hh +++ b/index/built_indexes_virtual_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/index/secondary_index.cc b/index/secondary_index.cc index bdc5c2c8af4b..c96e013413f0 100644 --- a/index/secondary_index.cc +++ b/index/secondary_index.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "secondary_index.hh" diff --git a/index/secondary_index.hh b/index/secondary_index.hh index c44c1cbcc017..1f5e06c8d324 100644 --- a/index/secondary_index.hh +++ b/index/secondary_index.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/index/secondary_index_manager.cc b/index/secondary_index_manager.cc index 170f26e7db93..1ed6dda2b35b 100644 --- a/index/secondary_index_manager.cc +++ b/index/secondary_index_manager.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/index/secondary_index_manager.hh b/index/secondary_index_manager.hh index 7961a8bb8725..38ab2ac6d96f 100644 --- a/index/secondary_index_manager.hh +++ b/index/secondary_index_manager.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/index/target_parser.hh b/index/target_parser.hh index 9f9419a7033a..83b826087d9f 100644 --- a/index/target_parser.hh +++ b/index/target_parser.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/inet_address_vectors.hh b/inet_address_vectors.hh index 5408a51be4fd..6eefe9946102 100644 --- a/inet_address_vectors.hh +++ b/inet_address_vectors.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/init.cc b/init.cc index 9130dfb8a167..3de4fc2875ce 100644 --- a/init.cc +++ b/init.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "init.hh" diff --git a/init.hh b/init.hh index c3cfa39edb5b..867021999c34 100644 --- a/init.hh +++ b/init.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/install.sh b/install.sh index 0226e2acf636..fca2e747d05e 100755 --- a/install.sh +++ b/install.sh @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # set -e diff --git a/interval.hh b/interval.hh index 99905562e2da..3f7e6f565a38 100644 --- a/interval.hh +++ b/interval.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/keys.cc b/keys.cc index 0b67d872e554..f9d1d2308a30 100644 --- a/keys.cc +++ b/keys.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/keys.hh b/keys.hh index cfacbe18c5f5..54505d6ec9ec 100644 --- a/keys.hh +++ b/keys.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/lang/lua.cc b/lang/lua.cc index 9422f540963f..e919fe405f4b 100644 --- a/lang/lua.cc +++ b/lang/lua.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/lang/lua.hh b/lang/lua.hh index 18deb244716c..8cdc2cb7296b 100644 --- a/lang/lua.hh +++ b/lang/lua.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/lang/lua_scylla_types.hh b/lang/lua_scylla_types.hh index d8012eeb7866..01bb68c545fb 100644 --- a/lang/lua_scylla_types.hh +++ b/lang/lua_scylla_types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/lang/manager.cc b/lang/manager.cc index 88765d5c64a7..f21c0377b947 100644 --- a/lang/manager.cc +++ b/lang/manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "lang/wasm.hh" diff --git a/lang/manager.hh b/lang/manager.hh index 5bbbc46ee65d..3d8128724207 100644 --- a/lang/manager.hh +++ b/lang/manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/lang/wasm.cc b/lang/wasm.cc index 4f6f7dfd2ab5..0db3bfac341a 100644 --- a/lang/wasm.cc +++ b/lang/wasm.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "wasm.hh" diff --git a/lang/wasm.hh b/lang/wasm.hh index 63511c5bc77d..f8697b8ad0ea 100644 --- a/lang/wasm.hh +++ b/lang/wasm.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/lang/wasm_alien_thread_runner.cc b/lang/wasm_alien_thread_runner.cc index 684b88f5760e..14257a21a774 100644 --- a/lang/wasm_alien_thread_runner.cc +++ b/lang/wasm_alien_thread_runner.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/lang/wasm_alien_thread_runner.hh b/lang/wasm_alien_thread_runner.hh index 987602cf46e0..09536c76bde7 100644 --- a/lang/wasm_alien_thread_runner.hh +++ b/lang/wasm_alien_thread_runner.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/lang/wasm_instance_cache.cc b/lang/wasm_instance_cache.cc index 76609208aa5f..e3fa6e99c7c0 100644 --- a/lang/wasm_instance_cache.cc +++ b/lang/wasm_instance_cache.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "lang/wasm_instance_cache.hh" diff --git a/lang/wasm_instance_cache.hh b/lang/wasm_instance_cache.hh index 6e95a8074b6f..57895c4a6f47 100644 --- a/lang/wasm_instance_cache.hh +++ b/lang/wasm_instance_cache.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/licenses/README.md b/licenses/README.md index cb5a6b98c74e..f08454b5363c 100644 --- a/licenses/README.md +++ b/licenses/README.md @@ -3,15 +3,15 @@ The files in this directory represent licenses that apply to portions of the work. See each source file for applicable licenses. -The work in whole is licensed under the Affero GPL, version 3 or above. See -the LICENSE.AGPL file in the top-level directory. +The work in whole is licensed under the ScyllaDB-Source-Available-1.0 license. +the LICENSE-ScyllaDB-Source-Available.md file in the top-level directory. Individual files contain the following tag: - SPDX-License-Identifier: AGPL-3.0-or-later + SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 Some files are derived from Apache projects. These are dual-licensed -with the Apache License (version 2) and AGPL 3.0. They contain the -following tag: +with the Apache License (version 2) and ScyllaDB-Source-Available-1.0. +They contain the following tag: - SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) diff --git a/locator/abstract_replication_strategy.cc b/locator/abstract_replication_strategy.cc index 5976de20cb18..468426e8ffa1 100644 --- a/locator/abstract_replication_strategy.cc +++ b/locator/abstract_replication_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/abstract_replication_strategy.hh" diff --git a/locator/abstract_replication_strategy.hh b/locator/abstract_replication_strategy.hh index 2648c58ddf07..46f469f80ea2 100644 --- a/locator/abstract_replication_strategy.hh +++ b/locator/abstract_replication_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/azure_snitch.cc b/locator/azure_snitch.cc index f22ed7d3fb54..041e5c915fe6 100644 --- a/locator/azure_snitch.cc +++ b/locator/azure_snitch.cc @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ /* diff --git a/locator/azure_snitch.hh b/locator/azure_snitch.hh index fd9ab2b8f05d..8f7ac7079fff 100644 --- a/locator/azure_snitch.hh +++ b/locator/azure_snitch.hh @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ /* diff --git a/locator/ec2_multi_region_snitch.cc b/locator/ec2_multi_region_snitch.cc index 85fb82d2c6ae..cc5019500b68 100644 --- a/locator/ec2_multi_region_snitch.cc +++ b/locator/ec2_multi_region_snitch.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "locator/ec2_multi_region_snitch.hh" diff --git a/locator/ec2_multi_region_snitch.hh b/locator/ec2_multi_region_snitch.hh index 69c875fd3e67..dbc867f2c1e9 100644 --- a/locator/ec2_multi_region_snitch.hh +++ b/locator/ec2_multi_region_snitch.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/everywhere_replication_strategy.cc b/locator/everywhere_replication_strategy.cc index 30aa06463cf9..49532ca45f0f 100644 --- a/locator/everywhere_replication_strategy.cc +++ b/locator/everywhere_replication_strategy.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ diff --git a/locator/everywhere_replication_strategy.hh b/locator/everywhere_replication_strategy.hh index f646d31591ae..0247f68480b2 100644 --- a/locator/everywhere_replication_strategy.hh +++ b/locator/everywhere_replication_strategy.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/gce_snitch.cc b/locator/gce_snitch.cc index 81bfc288a97f..a79986a98894 100644 --- a/locator/gce_snitch.cc +++ b/locator/gce_snitch.cc @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ /* diff --git a/locator/gce_snitch.hh b/locator/gce_snitch.hh index 6093311e2348..0a25757835bd 100644 --- a/locator/gce_snitch.hh +++ b/locator/gce_snitch.hh @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ /* diff --git a/locator/gossiping_property_file_snitch.cc b/locator/gossiping_property_file_snitch.cc index 9bb1a288923a..7839c8943a42 100644 --- a/locator/gossiping_property_file_snitch.cc +++ b/locator/gossiping_property_file_snitch.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "locator/gossiping_property_file_snitch.hh" diff --git a/locator/gossiping_property_file_snitch.hh b/locator/gossiping_property_file_snitch.hh index a833c352ae6c..8181c8c0c541 100644 --- a/locator/gossiping_property_file_snitch.hh +++ b/locator/gossiping_property_file_snitch.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/host_id.hh b/locator/host_id.hh index 6ecc64016eca..f7d847448ba8 100644 --- a/locator/host_id.hh +++ b/locator/host_id.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/load_sketch.hh b/locator/load_sketch.hh index 32ac55cf0e93..f5062dc649f8 100644 --- a/locator/load_sketch.hh +++ b/locator/load_sketch.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/local_strategy.cc b/locator/local_strategy.cc index b7f30e4e5f5a..ef4fbc814940 100644 --- a/locator/local_strategy.cc +++ b/locator/local_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/locator/local_strategy.hh b/locator/local_strategy.hh index d1b1a5224a1d..8d0eb914ec37 100644 --- a/locator/local_strategy.hh +++ b/locator/local_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/network_topology_strategy.cc b/locator/network_topology_strategy.cc index 294885f44d20..062c1254d0d0 100644 --- a/locator/network_topology_strategy.cc +++ b/locator/network_topology_strategy.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/locator/network_topology_strategy.hh b/locator/network_topology_strategy.hh index f2c50edd78d4..81761a93a9b5 100644 --- a/locator/network_topology_strategy.hh +++ b/locator/network_topology_strategy.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/production_snitch_base.cc b/locator/production_snitch_base.cc index 6a593b314fe6..fb00dcab3877 100644 --- a/locator/production_snitch_base.cc +++ b/locator/production_snitch_base.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "locator/production_snitch_base.hh" #include "db/system_keyspace.hh" diff --git a/locator/production_snitch_base.hh b/locator/production_snitch_base.hh index 95d4d8e0c730..d52e75426624 100644 --- a/locator/production_snitch_base.hh +++ b/locator/production_snitch_base.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/rack_inferring_snitch.cc b/locator/rack_inferring_snitch.cc index ad68021874e1..2a12b5b33fbd 100644 --- a/locator/rack_inferring_snitch.cc +++ b/locator/rack_inferring_snitch.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/rack_inferring_snitch.hh" diff --git a/locator/rack_inferring_snitch.hh b/locator/rack_inferring_snitch.hh index c6dbbe78ef92..900d228484d9 100644 --- a/locator/rack_inferring_snitch.hh +++ b/locator/rack_inferring_snitch.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/simple_snitch.cc b/locator/simple_snitch.cc index c9dbd9dd7484..326bd18f5220 100644 --- a/locator/simple_snitch.cc +++ b/locator/simple_snitch.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/simple_snitch.hh" diff --git a/locator/simple_snitch.hh b/locator/simple_snitch.hh index 9a47a74c9c82..7757bb27e571 100644 --- a/locator/simple_snitch.hh +++ b/locator/simple_snitch.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/simple_strategy.cc b/locator/simple_strategy.cc index 3446e09386fe..8c9345ce96e9 100644 --- a/locator/simple_strategy.cc +++ b/locator/simple_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/locator/simple_strategy.hh b/locator/simple_strategy.hh index 3f25b4f91764..e55e8de8c9b5 100644 --- a/locator/simple_strategy.hh +++ b/locator/simple_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/snitch_base.cc b/locator/snitch_base.cc index f3b2038631c0..e6ec5157ca85 100644 --- a/locator/snitch_base.cc +++ b/locator/snitch_base.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "locator/snitch_base.hh" diff --git a/locator/snitch_base.hh b/locator/snitch_base.hh index 1f9cb881aec0..8e3d090322da 100644 --- a/locator/snitch_base.hh +++ b/locator/snitch_base.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/tablet_metadata_guard.hh b/locator/tablet_metadata_guard.hh index 3747143ac4a0..402f50e2764e 100644 --- a/locator/tablet_metadata_guard.hh +++ b/locator/tablet_metadata_guard.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/tablet_replication_strategy.hh b/locator/tablet_replication_strategy.hh index 5a339ffc7e64..f2cc2c79f420 100644 --- a/locator/tablet_replication_strategy.hh +++ b/locator/tablet_replication_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/tablet_sharder.hh b/locator/tablet_sharder.hh index e34beee1b5fb..922fea6aa91a 100644 --- a/locator/tablet_sharder.hh +++ b/locator/tablet_sharder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/tablets.cc b/locator/tablets.cc index 23922a27792d..cafe0fe22aa8 100644 --- a/locator/tablets.cc +++ b/locator/tablets.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/tablet_replication_strategy.hh" diff --git a/locator/tablets.hh b/locator/tablets.hh index 26861c203919..f4f6fbeef08b 100644 --- a/locator/tablets.hh +++ b/locator/tablets.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index dc91ccb417b8..37db2fde8c71 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "token_metadata.hh" diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh index dfd289278bb7..038bb0e94452 100644 --- a/locator/token_metadata.hh +++ b/locator/token_metadata.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/token_metadata_fwd.hh b/locator/token_metadata_fwd.hh index 370e15a31a7e..0ca9f9dd078b 100644 --- a/locator/token_metadata_fwd.hh +++ b/locator/token_metadata_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/token_range_splitter.hh b/locator/token_range_splitter.hh index ac55d65fd19f..ac49109d73dc 100644 --- a/locator/token_range_splitter.hh +++ b/locator/token_range_splitter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/locator/topology.cc b/locator/topology.cc index 427849dfd8be..fb21706b0374 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/locator/topology.hh b/locator/topology.hh index 517c2a7ce3bf..9f89c80e7c38 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/types.hh b/locator/types.hh index 3f2783f3fe3b..47dd77b1ac78 100644 --- a/locator/types.hh +++ b/locator/types.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/locator/util.cc b/locator/util.cc index b6b02453415b..ce006f84ad1d 100644 --- a/locator/util.cc +++ b/locator/util.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/util.hh" #include "replica/database.hh" diff --git a/locator/util.hh b/locator/util.hh index 4de6cd0c50a7..e85c04048ea7 100644 --- a/locator/util.hh +++ b/locator/util.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/main.cc b/main.cc index 2468fba4e1b2..d2013cbba4d9 100644 --- a/main.cc +++ b/main.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/map_difference.hh b/map_difference.hh index 8b0f4ab24b3c..cd576701ad63 100644 --- a/map_difference.hh +++ b/map_difference.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/marshal_exception.hh b/marshal_exception.hh index b0cb688a77be..560b1c507756 100644 --- a/marshal_exception.hh +++ b/marshal_exception.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/message/messaging_service.cc b/message/messaging_service.cc index e51173ebdef4..35825e985183 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "gms/inet_address.hh" diff --git a/message/messaging_service.hh b/message/messaging_service.hh index 9b3139e44a7f..ce013ea1e368 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/message/messaging_service_fwd.hh b/message/messaging_service_fwd.hh index 725cfdebc9fa..08d2940cc768 100644 --- a/message/messaging_service_fwd.hh +++ b/message/messaging_service_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/message/msg_addr.hh b/message/msg_addr.hh index d0a6a41a101f..a7ab46b266bc 100644 --- a/message/msg_addr.hh +++ b/message/msg_addr.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/message/rpc_protocol_impl.hh b/message/rpc_protocol_impl.hh index ec502903c412..4b0114a700b3 100644 --- a/message/rpc_protocol_impl.hh +++ b/message/rpc_protocol_impl.hh @@ -1,4 +1,4 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 // Copyright 2021-present ScyllaDB #pragma once diff --git a/multishard_mutation_query.cc b/multishard_mutation_query.cc index 0b0db9086f32..3fbccd4de2e6 100644 --- a/multishard_mutation_query.cc +++ b/multishard_mutation_query.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "schema/schema_registry.hh" diff --git a/multishard_mutation_query.hh b/multishard_mutation_query.hh index 9a6bfaabef69..f2df21da52a3 100644 --- a/multishard_mutation_query.hh +++ b/multishard_mutation_query.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/async_utils.cc b/mutation/async_utils.cc index 3732bc182575..6c9ce51eb69a 100644 --- a/mutation/async_utils.cc +++ b/mutation/async_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/async_utils.hh b/mutation/async_utils.hh index 5903e67dab57..8ec82dfc8e01 100644 --- a/mutation/async_utils.hh +++ b/mutation/async_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/atomic_cell.cc b/mutation/atomic_cell.cc index 3155ae234d47..d687e1ead9c3 100644 --- a/mutation/atomic_cell.cc +++ b/mutation/atomic_cell.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "atomic_cell.hh" diff --git a/mutation/atomic_cell.hh b/mutation/atomic_cell.hh index 90822466a281..b9ff6e7ccf15 100644 --- a/mutation/atomic_cell.hh +++ b/mutation/atomic_cell.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/atomic_cell_hash.hh b/mutation/atomic_cell_hash.hh index 2a59d827670d..c8029734a557 100644 --- a/mutation/atomic_cell_hash.hh +++ b/mutation/atomic_cell_hash.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/atomic_cell_or_collection.hh b/mutation/atomic_cell_or_collection.hh index 4e6362767347..48b612c9e6ff 100644 --- a/mutation/atomic_cell_or_collection.hh +++ b/mutation/atomic_cell_or_collection.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/canonical_mutation.cc b/mutation/canonical_mutation.cc index 51598a84fc5d..ee1b4b997de8 100644 --- a/mutation/canonical_mutation.cc +++ b/mutation/canonical_mutation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "canonical_mutation.hh" diff --git a/mutation/canonical_mutation.hh b/mutation/canonical_mutation.hh index 04338efae0d6..7194e02d68ad 100644 --- a/mutation/canonical_mutation.hh +++ b/mutation/canonical_mutation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/compact_and_expire_result.hh b/mutation/compact_and_expire_result.hh index 14adcf1aa81a..ad3fc57e2446 100644 --- a/mutation/compact_and_expire_result.hh +++ b/mutation/compact_and_expire_result.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/frozen_mutation.cc b/mutation/frozen_mutation.cc index 759b331aa374..802685dd4e27 100644 --- a/mutation/frozen_mutation.cc +++ b/mutation/frozen_mutation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/frozen_mutation.hh b/mutation/frozen_mutation.hh index 5a184bbef104..76ddbd12bd8c 100644 --- a/mutation/frozen_mutation.hh +++ b/mutation/frozen_mutation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/json.hh b/mutation/json.hh index 67bc7fe127be..ed6255e83450 100644 --- a/mutation/json.hh +++ b/mutation/json.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation.cc b/mutation/mutation.cc index 8ceadd37ed90..4d4f6471de9c 100644 --- a/mutation/mutation.cc +++ b/mutation/mutation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/mutation.hh b/mutation/mutation.hh index d6c5975ad71b..811d572f67dd 100644 --- a/mutation/mutation.hh +++ b/mutation/mutation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_cleaner.hh b/mutation/mutation_cleaner.hh index 6cad63f9360f..7b7111a543ce 100644 --- a/mutation/mutation_cleaner.hh +++ b/mutation/mutation_cleaner.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_compactor.hh b/mutation/mutation_compactor.hh index 50dc86629013..a5fbe9138815 100644 --- a/mutation/mutation_compactor.hh +++ b/mutation/mutation_compactor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_consumer.hh b/mutation/mutation_consumer.hh index f86b1b20e154..4512278a0d16 100644 --- a/mutation/mutation_consumer.hh +++ b/mutation/mutation_consumer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_consumer_concepts.hh b/mutation/mutation_consumer_concepts.hh index eddffe036b23..2df95373f6bf 100644 --- a/mutation/mutation_consumer_concepts.hh +++ b/mutation/mutation_consumer_concepts.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_fragment.cc b/mutation/mutation_fragment.cc index 85af13341573..c9b8aea1f527 100644 --- a/mutation/mutation_fragment.cc +++ b/mutation/mutation_fragment.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/mutation_fragment.hh b/mutation/mutation_fragment.hh index fbe33e670552..186767db66bd 100644 --- a/mutation/mutation_fragment.hh +++ b/mutation/mutation_fragment.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_fragment_fwd.hh b/mutation/mutation_fragment_fwd.hh index e3566c2d24f8..6bf811ae814c 100644 --- a/mutation/mutation_fragment_fwd.hh +++ b/mutation/mutation_fragment_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_fragment_stream_validator.cc b/mutation/mutation_fragment_stream_validator.cc index f1c5c41fc823..bbc9068227f8 100644 --- a/mutation/mutation_fragment_stream_validator.cc +++ b/mutation/mutation_fragment_stream_validator.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation/mutation_fragment_stream_validator.hh" diff --git a/mutation/mutation_fragment_stream_validator.hh b/mutation/mutation_fragment_stream_validator.hh index 75edb9e43537..d6a8d9e9eb6e 100644 --- a/mutation/mutation_fragment_stream_validator.hh +++ b/mutation/mutation_fragment_stream_validator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_fragment_v2.hh b/mutation/mutation_fragment_v2.hh index 26b6a68ccb1a..b0b0abb9a319 100644 --- a/mutation/mutation_fragment_v2.hh +++ b/mutation/mutation_fragment_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_partition.cc b/mutation/mutation_partition.cc index 8ec1a1eb25bf..af1e0c19f6be 100644 --- a/mutation/mutation_partition.cc +++ b/mutation/mutation_partition.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/mutation_partition.hh b/mutation/mutation_partition.hh index 69166f4ca233..39ed8adddcca 100644 --- a/mutation/mutation_partition.hh +++ b/mutation/mutation_partition.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_partition_serializer.cc b/mutation/mutation_partition_serializer.cc index 255ec36bfc1f..99052bdbb55e 100644 --- a/mutation/mutation_partition_serializer.cc +++ b/mutation/mutation_partition_serializer.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation_partition_serializer.hh" diff --git a/mutation/mutation_partition_serializer.hh b/mutation/mutation_partition_serializer.hh index 11c4977e86e0..68ad0f99c554 100644 --- a/mutation/mutation_partition_serializer.hh +++ b/mutation/mutation_partition_serializer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_partition_v2.cc b/mutation/mutation_partition_v2.cc index b62470d8ea75..cb5edc12dc78 100644 --- a/mutation/mutation_partition_v2.cc +++ b/mutation/mutation_partition_v2.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/mutation_partition_v2.hh b/mutation/mutation_partition_v2.hh index 83f5023853eb..c281fcb954b2 100644 --- a/mutation/mutation_partition_v2.hh +++ b/mutation/mutation_partition_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_partition_view.cc b/mutation/mutation_partition_view.cc index ebbcc1112736..cc5f53d4d455 100644 --- a/mutation/mutation_partition_view.cc +++ b/mutation/mutation_partition_view.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/mutation/mutation_partition_view.hh b/mutation/mutation_partition_view.hh index f7f81dc765fd..1d46613fc1ef 100644 --- a/mutation/mutation_partition_view.hh +++ b/mutation/mutation_partition_view.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_partition_visitor.hh b/mutation/mutation_partition_visitor.hh index fe2d8a982bb5..97f69e036068 100644 --- a/mutation/mutation_partition_visitor.hh +++ b/mutation/mutation_partition_visitor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_rebuilder.hh b/mutation/mutation_rebuilder.hh index 5519c5d7a415..5007e26377ca 100644 --- a/mutation/mutation_rebuilder.hh +++ b/mutation/mutation_rebuilder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/mutation_source_metadata.hh b/mutation/mutation_source_metadata.hh index 86887a66b7ec..421c627b3eef 100644 --- a/mutation/mutation_source_metadata.hh +++ b/mutation/mutation_source_metadata.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/partition_version.cc b/mutation/partition_version.cc index 949c4f57783c..846488ab16a7 100644 --- a/mutation/partition_version.cc +++ b/mutation/partition_version.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/partition_version.hh b/mutation/partition_version.hh index 87ae46aeb6a3..f2f88c31fa90 100644 --- a/mutation/partition_version.hh +++ b/mutation/partition_version.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/partition_version_list.hh b/mutation/partition_version_list.hh index 11433f0acacd..af61ddadd2fa 100644 --- a/mutation/partition_version_list.hh +++ b/mutation/partition_version_list.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/position_in_partition.hh b/mutation/position_in_partition.hh index fcb9160439e0..4ccc5ff8cce8 100644 --- a/mutation/position_in_partition.hh +++ b/mutation/position_in_partition.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/range_tombstone.cc b/mutation/range_tombstone.cc index b159edb213d1..91808c046214 100644 --- a/mutation/range_tombstone.cc +++ b/mutation/range_tombstone.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "range_tombstone.hh" diff --git a/mutation/range_tombstone.hh b/mutation/range_tombstone.hh index e1c3eb37bd39..646e04e3ecb1 100644 --- a/mutation/range_tombstone.hh +++ b/mutation/range_tombstone.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/range_tombstone_assembler.hh b/mutation/range_tombstone_assembler.hh index 745424b5e8b7..1262568ead3e 100644 --- a/mutation/range_tombstone_assembler.hh +++ b/mutation/range_tombstone_assembler.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/range_tombstone_change_generator.hh b/mutation/range_tombstone_change_generator.hh index b04e96a51252..bd77b23bf87c 100644 --- a/mutation/range_tombstone_change_generator.hh +++ b/mutation/range_tombstone_change_generator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/range_tombstone_list.cc b/mutation/range_tombstone_list.cc index 463f8482dd64..4701959c3d9c 100644 --- a/mutation/range_tombstone_list.cc +++ b/mutation/range_tombstone_list.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation/range_tombstone_list.hh b/mutation/range_tombstone_list.hh index 8ba52b1e2a1f..5e4f3d50aad0 100644 --- a/mutation/range_tombstone_list.hh +++ b/mutation/range_tombstone_list.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/range_tombstone_splitter.hh b/mutation/range_tombstone_splitter.hh index a573ddd77fe4..59976e5d2a43 100644 --- a/mutation/range_tombstone_splitter.hh +++ b/mutation/range_tombstone_splitter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation/tombstone.hh b/mutation/tombstone.hh index 72e5e8959927..a82263ea5015 100644 --- a/mutation/tombstone.hh +++ b/mutation/tombstone.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation_query.cc b/mutation_query.cc index 2c30928daa22..d831d3f19f82 100644 --- a/mutation_query.cc +++ b/mutation_query.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/mutation_query.hh b/mutation_query.hh index fd05cff9c457..157d9509d889 100644 --- a/mutation_query.hh +++ b/mutation_query.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation_writer/feed_writers.cc b/mutation_writer/feed_writers.cc index 8a4c190bd71f..af08df45e3de 100644 --- a/mutation_writer/feed_writers.cc +++ b/mutation_writer/feed_writers.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "feed_writers.hh" diff --git a/mutation_writer/feed_writers.hh b/mutation_writer/feed_writers.hh index 1d816a77f529..6dd6ad22b9f5 100644 --- a/mutation_writer/feed_writers.hh +++ b/mutation_writer/feed_writers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation_writer/multishard_writer.cc b/mutation_writer/multishard_writer.cc index a78a147a5064..ea7f069a33b9 100644 --- a/mutation_writer/multishard_writer.cc +++ b/mutation_writer/multishard_writer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/mutation_writer/multishard_writer.hh b/mutation_writer/multishard_writer.hh index d235bcbcf2da..f743d8ed56cc 100644 --- a/mutation_writer/multishard_writer.hh +++ b/mutation_writer/multishard_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation_writer/partition_based_splitting_writer.cc b/mutation_writer/partition_based_splitting_writer.cc index 2dc4b81e1969..366f8da83797 100644 --- a/mutation_writer/partition_based_splitting_writer.cc +++ b/mutation_writer/partition_based_splitting_writer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation_writer/partition_based_splitting_writer.hh" diff --git a/mutation_writer/partition_based_splitting_writer.hh b/mutation_writer/partition_based_splitting_writer.hh index e5d53fbc9747..6d8ffcc39386 100644 --- a/mutation_writer/partition_based_splitting_writer.hh +++ b/mutation_writer/partition_based_splitting_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation_writer/shard_based_splitting_writer.cc b/mutation_writer/shard_based_splitting_writer.cc index 1ef21e6159cd..8ec22c57ae03 100644 --- a/mutation_writer/shard_based_splitting_writer.cc +++ b/mutation_writer/shard_based_splitting_writer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation_writer/shard_based_splitting_writer.hh" diff --git a/mutation_writer/shard_based_splitting_writer.hh b/mutation_writer/shard_based_splitting_writer.hh index 73bb27b460a1..586be88ba72e 100644 --- a/mutation_writer/shard_based_splitting_writer.hh +++ b/mutation_writer/shard_based_splitting_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation_writer/timestamp_based_splitting_writer.cc b/mutation_writer/timestamp_based_splitting_writer.cc index 3ab052f5449c..d85011db9e43 100644 --- a/mutation_writer/timestamp_based_splitting_writer.cc +++ b/mutation_writer/timestamp_based_splitting_writer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation_writer/timestamp_based_splitting_writer.hh" diff --git a/mutation_writer/timestamp_based_splitting_writer.hh b/mutation_writer/timestamp_based_splitting_writer.hh index e8d6734f7213..e63503acfc08 100644 --- a/mutation_writer/timestamp_based_splitting_writer.hh +++ b/mutation_writer/timestamp_based_splitting_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/mutation_writer/token_group_based_splitting_writer.cc b/mutation_writer/token_group_based_splitting_writer.cc index f9417d5856de..cd435bba5dd1 100644 --- a/mutation_writer/token_group_based_splitting_writer.cc +++ b/mutation_writer/token_group_based_splitting_writer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "mutation_writer/token_group_based_splitting_writer.hh" diff --git a/mutation_writer/token_group_based_splitting_writer.hh b/mutation_writer/token_group_based_splitting_writer.hh index 3595fefbed4a..b4ed475e3c91 100644 --- a/mutation_writer/token_group_based_splitting_writer.hh +++ b/mutation_writer/token_group_based_splitting_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/node_ops/id.hh b/node_ops/id.hh index 616c97334dcd..a2dbf1cea243 100644 --- a/node_ops/id.hh +++ b/node_ops/id.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/node_ops/node_ops_ctl.cc b/node_ops/node_ops_ctl.cc index 22e7c652f548..1eaaf2e333e2 100644 --- a/node_ops/node_ops_ctl.cc +++ b/node_ops/node_ops_ctl.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/config.hh" diff --git a/node_ops/node_ops_ctl.hh b/node_ops/node_ops_ctl.hh index 20d823aebef0..4fcf8aa0cbf9 100644 --- a/node_ops/node_ops_ctl.hh +++ b/node_ops/node_ops_ctl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/node_ops/task_manager_module.cc b/node_ops/task_manager_module.cc index c60bf087281a..7a25b36b9476 100644 --- a/node_ops/task_manager_module.cc +++ b/node_ops/task_manager_module.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/system_keyspace.hh" diff --git a/node_ops/task_manager_module.hh b/node_ops/task_manager_module.hh index 1415d9209dda..04be6161d4fb 100644 --- a/node_ops/task_manager_module.hh +++ b/node_ops/task_manager_module.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/noexcept_traits.hh b/noexcept_traits.hh index f7ead48296f4..c58f71645d38 100644 --- a/noexcept_traits.hh +++ b/noexcept_traits.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/partition_builder.hh b/partition_builder.hh index c35d4a068185..1b5789cc2689 100644 --- a/partition_builder.hh +++ b/partition_builder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/partition_range_compat.hh b/partition_range_compat.hh index 26a437bef660..40a260123a0e 100644 --- a/partition_range_compat.hh +++ b/partition_range_compat.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/partition_slice_builder.cc b/partition_slice_builder.cc index a3775dbd4f1d..1e3e79961595 100644 --- a/partition_slice_builder.cc +++ b/partition_slice_builder.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/partition_slice_builder.hh b/partition_slice_builder.hh index e90f52d6b53d..1b21e27bf10c 100644 --- a/partition_slice_builder.hh +++ b/partition_slice_builder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/partition_snapshot_reader.hh b/partition_snapshot_reader.hh index 62a5bf7f88bd..077b4c394048 100644 --- a/partition_snapshot_reader.hh +++ b/partition_snapshot_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/partition_snapshot_row_cursor.hh b/partition_snapshot_row_cursor.hh index 6d97241f5627..e64ac51b5e65 100644 --- a/partition_snapshot_row_cursor.hh +++ b/partition_snapshot_row_cursor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/protocol_server.hh b/protocol_server.hh index e4c229d25595..37b2c1f34cc3 100644 --- a/protocol_server.hh +++ b/protocol_server.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/querier.cc b/querier.cc index 7d869b9807d6..734cd2ffc240 100644 --- a/querier.cc +++ b/querier.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/querier.hh b/querier.hh index c66250577fb3..f2053392ccdb 100644 --- a/querier.hh +++ b/querier.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/query-request.hh b/query-request.hh index f873dedc95f3..6fdd32e32b23 100644 --- a/query-request.hh +++ b/query-request.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/query-result-reader.hh b/query-result-reader.hh index 453b3e8959ad..485dbcac8a82 100644 --- a/query-result-reader.hh +++ b/query-result-reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/query-result-set.cc b/query-result-set.cc index f0ea8edf4bd5..cfd3f650ef24 100644 --- a/query-result-set.cc +++ b/query-result-set.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "query-result-set.hh" diff --git a/query-result-set.hh b/query-result-set.hh index b02ed5e5f224..20c5f0a08b5b 100644 --- a/query-result-set.hh +++ b/query-result-set.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/query-result-writer.hh b/query-result-writer.hh index f1e30819d7b2..2a228a387ad0 100644 --- a/query-result-writer.hh +++ b/query-result-writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/query-result.hh b/query-result.hh index b97e5a6c58ae..c712ee33112b 100644 --- a/query-result.hh +++ b/query-result.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/query.cc b/query.cc index 799352ac2bbb..dd2e295734f1 100644 --- a/query.cc +++ b/query.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/query_id.hh b/query_id.hh index 6f3b12fd3509..40ccb60cd603 100644 --- a/query_id.hh +++ b/query_id.hh @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/query_ranges_to_vnodes.cc b/query_ranges_to_vnodes.cc index f35591ff5525..c73986f4459c 100644 --- a/query_ranges_to_vnodes.cc +++ b/query_ranges_to_vnodes.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "query_ranges_to_vnodes.hh" diff --git a/query_ranges_to_vnodes.hh b/query_ranges_to_vnodes.hh index bea631a13551..b4b692875a12 100644 --- a/query_ranges_to_vnodes.hh +++ b/query_ranges_to_vnodes.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/query_result_merger.hh b/query_result_merger.hh index a7c26b0bec41..e071006fc2ba 100644 --- a/query_result_merger.hh +++ b/query_result_merger.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/raft/fsm.cc b/raft/fsm.cc index 8def0afe5f80..3d9e59ebb021 100644 --- a/raft/fsm.cc +++ b/raft/fsm.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "fsm.hh" #include diff --git a/raft/fsm.hh b/raft/fsm.hh index 54c8a024ca68..9d1ab9980f85 100644 --- a/raft/fsm.hh +++ b/raft/fsm.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/raft/internal.hh b/raft/internal.hh index e086e819128c..fc014a4bdcd3 100644 --- a/raft/internal.hh +++ b/raft/internal.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/raft/log.cc b/raft/log.cc index 5cb71e8ffd5f..3c8d6d5edf27 100644 --- a/raft/log.cc +++ b/raft/log.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" #include "log.hh" diff --git a/raft/log.hh b/raft/log.hh index 98a0bb9e6817..6916c3a8260d 100644 --- a/raft/log.hh +++ b/raft/log.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/raft/logical_clock.hh b/raft/logical_clock.hh index 65be568d2e7c..12db2fadac47 100644 --- a/raft/logical_clock.hh +++ b/raft/logical_clock.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/raft/raft.cc b/raft/raft.cc index 28a69eda2f46..37599ca0d706 100644 --- a/raft/raft.cc +++ b/raft/raft.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "raft.hh" #include diff --git a/raft/raft.hh b/raft/raft.hh index e616e3aeaaa8..0e8144efc19d 100644 --- a/raft/raft.hh +++ b/raft/raft.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/raft/server.cc b/raft/server.cc index d7774eb9eae2..2a745d3828a4 100644 --- a/raft/server.cc +++ b/raft/server.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "server.hh" diff --git a/raft/server.hh b/raft/server.hh index 40df909bfbb0..a609b10b3221 100644 --- a/raft/server.hh +++ b/raft/server.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once #include diff --git a/raft/tracker.cc b/raft/tracker.cc index e683c64e1e4e..a7902e974c6f 100644 --- a/raft/tracker.cc +++ b/raft/tracker.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" #include "tracker.hh" diff --git a/raft/tracker.hh b/raft/tracker.hh index 1f69d1d2802f..c9c187c2b3aa 100644 --- a/raft/tracker.hh +++ b/raft/tracker.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/read_context.hh b/read_context.hh index 3d3650042059..d7910f8f9671 100644 --- a/read_context.hh +++ b/read_context.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/reader_concurrency_semaphore.cc b/reader_concurrency_semaphore.cc index 2703cde3638b..bd27b892b517 100644 --- a/reader_concurrency_semaphore.cc +++ b/reader_concurrency_semaphore.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/reader_concurrency_semaphore.hh b/reader_concurrency_semaphore.hh index e1191c182149..0daad1d25a4c 100644 --- a/reader_concurrency_semaphore.hh +++ b/reader_concurrency_semaphore.hh @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/reader_permit.hh b/reader_permit.hh index 70e351cd036b..c0804bc8d42e 100644 --- a/reader_permit.hh +++ b/reader_permit.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/clustering_combined.hh b/readers/clustering_combined.hh index 0c7c02b88661..804e56608a6c 100644 --- a/readers/clustering_combined.hh +++ b/readers/clustering_combined.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/combined.cc b/readers/combined.cc index 4dbd2cc991f7..0dd585311d72 100644 --- a/readers/combined.cc +++ b/readers/combined.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/readers/combined.hh b/readers/combined.hh index 3feda0a12068..d01a71fbdc67 100644 --- a/readers/combined.hh +++ b/readers/combined.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/combined_reader_stats.hh b/readers/combined_reader_stats.hh index e7235acbb584..c4d117e93f04 100644 --- a/readers/combined_reader_stats.hh +++ b/readers/combined_reader_stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/compacting.hh b/readers/compacting.hh index 009841ce1376..8a766e251d73 100644 --- a/readers/compacting.hh +++ b/readers/compacting.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/delegating_v2.hh b/readers/delegating_v2.hh index 59ff7617c5a4..ca034daa5a0e 100644 --- a/readers/delegating_v2.hh +++ b/readers/delegating_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/empty_v2.hh b/readers/empty_v2.hh index a51cdc38e231..f56728dc11b0 100644 --- a/readers/empty_v2.hh +++ b/readers/empty_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/evictable.hh b/readers/evictable.hh index 529528f018ca..fcf1f8c5a9e4 100644 --- a/readers/evictable.hh +++ b/readers/evictable.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/filtering.hh b/readers/filtering.hh index 01b42c10607e..ee88c4bf3fad 100644 --- a/readers/filtering.hh +++ b/readers/filtering.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/foreign.hh b/readers/foreign.hh index f4324ebe9f97..f959f50fdc0b 100644 --- a/readers/foreign.hh +++ b/readers/foreign.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/forwardable_v2.hh b/readers/forwardable_v2.hh index 315f1a13fbae..df889e4c8fc4 100644 --- a/readers/forwardable_v2.hh +++ b/readers/forwardable_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/from_fragments_v2.hh b/readers/from_fragments_v2.hh index 0b10491ffeee..153f4091c9e7 100644 --- a/readers/from_fragments_v2.hh +++ b/readers/from_fragments_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/from_mutations_v2.hh b/readers/from_mutations_v2.hh index 8e5b7c483fa3..76a3360b979e 100644 --- a/readers/from_mutations_v2.hh +++ b/readers/from_mutations_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/generating_v2.hh b/readers/generating_v2.hh index 4daca17747ba..4cf8f2476c82 100644 --- a/readers/generating_v2.hh +++ b/readers/generating_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/multi_range.hh b/readers/multi_range.hh index 5f0af184ef73..aaf0e7784506 100644 --- a/readers/multi_range.hh +++ b/readers/multi_range.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/multishard.cc b/readers/multishard.cc index 8fd1430f67eb..c91df2c117df 100644 --- a/readers/multishard.cc +++ b/readers/multishard.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/readers/multishard.hh b/readers/multishard.hh index 8038553f240d..5cf91264b34b 100644 --- a/readers/multishard.hh +++ b/readers/multishard.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/mutation_reader.cc b/readers/mutation_reader.cc index 8bf26c58557d..c3c11aeb5617 100644 --- a/readers/mutation_reader.cc +++ b/readers/mutation_reader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/readers/mutation_reader.hh b/readers/mutation_reader.hh index 00107ced4c0f..81f6ac72ae4a 100644 --- a/readers/mutation_reader.hh +++ b/readers/mutation_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/mutation_reader_fwd.hh b/readers/mutation_reader_fwd.hh index 595203e42e9d..1e911d60b80e 100644 --- a/readers/mutation_reader_fwd.hh +++ b/readers/mutation_reader_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/mutation_readers.cc b/readers/mutation_readers.cc index c461557229b7..9a6c0f56bfb6 100644 --- a/readers/mutation_readers.cc +++ b/readers/mutation_readers.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/readers/mutation_source.hh b/readers/mutation_source.hh index 4dc133afe4ce..9a3c59a147c2 100644 --- a/readers/mutation_source.hh +++ b/readers/mutation_source.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/next_partition_adaptor.hh b/readers/next_partition_adaptor.hh index 6071604e9fd1..fcc9f2d89d63 100644 --- a/readers/next_partition_adaptor.hh +++ b/readers/next_partition_adaptor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/nonforwardable.hh b/readers/nonforwardable.hh index fb7005db42c5..4e8679e66d1b 100644 --- a/readers/nonforwardable.hh +++ b/readers/nonforwardable.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/queue.hh b/readers/queue.hh index e7fc9a56cbe0..486b634aca58 100644 --- a/readers/queue.hh +++ b/readers/queue.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/range_tombstone_change_merger.hh b/readers/range_tombstone_change_merger.hh index 06ca65c0732c..b603716712db 100644 --- a/readers/range_tombstone_change_merger.hh +++ b/readers/range_tombstone_change_merger.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/reversing_v2.hh b/readers/reversing_v2.hh index eeefb0ae0b85..066261466c69 100644 --- a/readers/reversing_v2.hh +++ b/readers/reversing_v2.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/slicing_filtering.hh b/readers/slicing_filtering.hh index 36e2601567bf..50afee799d45 100644 --- a/readers/slicing_filtering.hh +++ b/readers/slicing_filtering.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/readers/upgrading_consumer.hh b/readers/upgrading_consumer.hh index 2bb96f0c9154..2da62c9b0d0b 100644 --- a/readers/upgrading_consumer.hh +++ b/readers/upgrading_consumer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/real_dirty_memory_accounter.hh b/real_dirty_memory_accounter.hh index 6bd74fb65ca7..afea98f84489 100644 --- a/real_dirty_memory_accounter.hh +++ b/real_dirty_memory_accounter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/abstract_command.cc b/redis/abstract_command.cc index f3da524cd9a2..945ff78df76f 100644 --- a/redis/abstract_command.cc +++ b/redis/abstract_command.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "abstract_command.hh" diff --git a/redis/abstract_command.hh b/redis/abstract_command.hh index caccb8851c0d..55252e68b15c 100644 --- a/redis/abstract_command.hh +++ b/redis/abstract_command.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/command_factory.cc b/redis/command_factory.cc index 655eb0adf29f..368eb8f5e7fa 100644 --- a/redis/command_factory.cc +++ b/redis/command_factory.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "redis/command_factory.hh" diff --git a/redis/command_factory.hh b/redis/command_factory.hh index 6c6aa83ba764..002d2d2f65c7 100644 --- a/redis/command_factory.hh +++ b/redis/command_factory.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/commands.cc b/redis/commands.cc index bc7fb227f74c..b6c3e6945f7f 100644 --- a/redis/commands.cc +++ b/redis/commands.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "redis/commands.hh" diff --git a/redis/commands.hh b/redis/commands.hh index 927800f1d53a..25be8f3df604 100644 --- a/redis/commands.hh +++ b/redis/commands.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/controller.cc b/redis/controller.cc index cd237829568f..db6917e8f0f0 100644 --- a/redis/controller.cc +++ b/redis/controller.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/redis/controller.hh b/redis/controller.hh index 69f3b6040f35..27ecd1f42f35 100644 --- a/redis/controller.hh +++ b/redis/controller.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/exceptions.hh b/redis/exceptions.hh index d7863ba0ab0b..1d24b105aaa8 100644 --- a/redis/exceptions.hh +++ b/redis/exceptions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/keyspace_utils.cc b/redis/keyspace_utils.cc index 8628fff8a5f7..2bbff6956a87 100644 --- a/redis/keyspace_utils.cc +++ b/redis/keyspace_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/redis/keyspace_utils.hh b/redis/keyspace_utils.hh index be056a1f4336..14df576a93ba 100644 --- a/redis/keyspace_utils.hh +++ b/redis/keyspace_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/lolwut.cc b/redis/lolwut.cc index 2667508230aa..0c0ad35fcc74 100644 --- a/redis/lolwut.cc +++ b/redis/lolwut.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/redis/lolwut.hh b/redis/lolwut.hh index ae5295d1d9b9..379c35226b35 100644 --- a/redis/lolwut.hh +++ b/redis/lolwut.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/mutation_utils.cc b/redis/mutation_utils.cc index c1e721c663af..96ea4506a39e 100644 --- a/redis/mutation_utils.cc +++ b/redis/mutation_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "redis/mutation_utils.hh" diff --git a/redis/mutation_utils.hh b/redis/mutation_utils.hh index aee736edb623..4910168045e8 100644 --- a/redis/mutation_utils.hh +++ b/redis/mutation_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/options.cc b/redis/options.cc index 08e97c163e89..cfe02b89abf2 100644 --- a/redis/options.cc +++ b/redis/options.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "redis/options.hh" diff --git a/redis/options.hh b/redis/options.hh index b5bb4ff5cd11..c2ff0f3c4ab8 100644 --- a/redis/options.hh +++ b/redis/options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/protocol_parser.rl b/redis/protocol_parser.rl index 86ce5ef4c01f..6369a87330bb 100644 --- a/redis/protocol_parser.rl +++ b/redis/protocol_parser.rl @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/redis/query_processor.cc b/redis/query_processor.cc index dfcadb7393ea..3fa02065eb27 100644 --- a/redis/query_processor.cc +++ b/redis/query_processor.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "redis/query_processor.hh" diff --git a/redis/query_processor.hh b/redis/query_processor.hh index 7f415cd681ee..c498f204b5ab 100644 --- a/redis/query_processor.hh +++ b/redis/query_processor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/query_utils.cc b/redis/query_utils.cc index c6c30554948e..713356d16310 100644 --- a/redis/query_utils.cc +++ b/redis/query_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/redis/query_utils.hh b/redis/query_utils.hh index bd75ad3e266c..e584ac08e82b 100644 --- a/redis/query_utils.hh +++ b/redis/query_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/reply.hh b/redis/reply.hh index de979f60e16f..a0a6cfbef021 100644 --- a/redis/reply.hh +++ b/redis/reply.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/request.hh b/redis/request.hh index 9fe43891fe24..b787caa9ca3c 100644 --- a/redis/request.hh +++ b/redis/request.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/server.cc b/redis/server.cc index 5f83e3a22e87..b9f508ea8cff 100644 --- a/redis/server.cc +++ b/redis/server.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "redis/server.hh" diff --git a/redis/server.hh b/redis/server.hh index ee6b478343c1..6fd5da658877 100644 --- a/redis/server.hh +++ b/redis/server.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/stats.cc b/redis/stats.cc index 54b5d18e3959..868718c4c52d 100644 --- a/redis/stats.cc +++ b/redis/stats.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "redis/stats.hh" diff --git a/redis/stats.hh b/redis/stats.hh index 9e6b615a01e0..6b866513406d 100644 --- a/redis/stats.hh +++ b/redis/stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/redis/version.hh b/redis/version.hh index 1f3b7ed43a10..cb254d4154d1 100644 --- a/redis/version.hh +++ b/redis/version.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/release.cc b/release.cc index bbb3cd80b509..20a41d5c4bb9 100644 --- a/release.cc +++ b/release.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/release.hh b/release.hh index 1047ea51e958..ed99db9ace35 100644 --- a/release.hh +++ b/release.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/repair/decorated_key_with_hash.hh b/repair/decorated_key_with_hash.hh index 9229a815bda1..07657cb197ef 100644 --- a/repair/decorated_key_with_hash.hh +++ b/repair/decorated_key_with_hash.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/repair/repair.cc b/repair/repair.cc index 65f1fd6bdeb3..fff23be9f9ac 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "repair.hh" diff --git a/repair/repair.hh b/repair/repair.hh index d4225f1fae5d..61efc6c2f49c 100644 --- a/repair/repair.hh +++ b/repair/repair.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/repair/row.hh b/repair/row.hh index b94ba714b1c0..8cd0c818723d 100644 --- a/repair/row.hh +++ b/repair/row.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/repair/row_level.cc b/repair/row_level.cc index 95b7a5088b67..cb8843faaa84 100644 --- a/repair/row_level.cc +++ b/repair/row_level.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/repair/row_level.hh b/repair/row_level.hh index 7f367e95a3d7..decf12e5f6b3 100644 --- a/repair/row_level.hh +++ b/repair/row_level.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/repair/sync_boundary.hh b/repair/sync_boundary.hh index 59db7d2b4c74..575449ce3802 100644 --- a/repair/sync_boundary.hh +++ b/repair/sync_boundary.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/repair/table_check.cc b/repair/table_check.cc index 2fe6d199f737..3db7be410f7b 100644 --- a/repair/table_check.cc +++ b/repair/table_check.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "replica/database.hh" diff --git a/repair/table_check.hh b/repair/table_check.hh index 587ba87b5a83..33e136d5bdea 100644 --- a/repair/table_check.hh +++ b/repair/table_check.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh index 6631d5a62652..01611468a7f3 100644 --- a/repair/task_manager_module.hh +++ b/repair/task_manager_module.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/compaction_group.hh b/replica/compaction_group.hh index 71e08b5ac3b7..abfa173d3aaf 100644 --- a/replica/compaction_group.hh +++ b/replica/compaction_group.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/replica/data_dictionary_impl.hh b/replica/data_dictionary_impl.hh index 002364f73a14..804b3dc59777 100644 --- a/replica/data_dictionary_impl.hh +++ b/replica/data_dictionary_impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/database.cc b/replica/database.cc index ca88ab5992f5..87da7731d3c4 100644 --- a/replica/database.cc +++ b/replica/database.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/replica/database.hh b/replica/database.hh index 08dd0b4a1c10..0e1341082f7b 100644 --- a/replica/database.hh +++ b/replica/database.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/database_fwd.hh b/replica/database_fwd.hh index 1e517027e681..c4b95a81cb48 100644 --- a/replica/database_fwd.hh +++ b/replica/database_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/dirty_memory_manager.cc b/replica/dirty_memory_manager.cc index 258dd3da8f42..e46c8440fde3 100644 --- a/replica/dirty_memory_manager.cc +++ b/replica/dirty_memory_manager.cc @@ -1,5 +1,5 @@ // Copyright (C) 2012-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #include "utils/assert.hh" diff --git a/replica/dirty_memory_manager.hh b/replica/dirty_memory_manager.hh index 389ffbb3c767..0c72fffa4407 100644 --- a/replica/dirty_memory_manager.hh +++ b/replica/dirty_memory_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/distributed_loader.cc b/replica/distributed_loader.cc index ad61093e9f30..431389d070a5 100644 --- a/replica/distributed_loader.cc +++ b/replica/distributed_loader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/replica/distributed_loader.hh b/replica/distributed_loader.hh index 3a4559ac4df2..7f102f2b2de1 100644 --- a/replica/distributed_loader.hh +++ b/replica/distributed_loader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/exceptions.cc b/replica/exceptions.cc index c36b9134853c..dfb3eb06a946 100644 --- a/replica/exceptions.cc +++ b/replica/exceptions.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/replica/exceptions.hh b/replica/exceptions.hh index 09302616805e..32e28ec5fb4c 100644 --- a/replica/exceptions.hh +++ b/replica/exceptions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/global_table_ptr.hh b/replica/global_table_ptr.hh index 98bfe2a917f2..7e76a8860df7 100644 --- a/replica/global_table_ptr.hh +++ b/replica/global_table_ptr.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/memtable-sstable.hh b/replica/memtable-sstable.hh index 1b5cacdef288..c2f76d726a7d 100644 --- a/replica/memtable-sstable.hh +++ b/replica/memtable-sstable.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/replica/memtable.cc b/replica/memtable.cc index 129103900730..d5d48214c776 100644 --- a/replica/memtable.cc +++ b/replica/memtable.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/replica/memtable.hh b/replica/memtable.hh index 01fa1ce45d71..de15bbe4662c 100644 --- a/replica/memtable.hh +++ b/replica/memtable.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/mutation_dump.cc b/replica/mutation_dump.cc index 965c3aad8b18..5e8690009c02 100644 --- a/replica/mutation_dump.cc +++ b/replica/mutation_dump.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0) */ #include "multishard_mutation_query.hh" diff --git a/replica/mutation_dump.hh b/replica/mutation_dump.hh index f7d7eec7b5e5..dea9807d4779 100644 --- a/replica/mutation_dump.hh +++ b/replica/mutation_dump.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0) */ #pragma once diff --git a/replica/query.hh b/replica/query.hh index 27040bc15a40..720683a957e7 100644 --- a/replica/query.hh +++ b/replica/query.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/query_state.hh b/replica/query_state.hh index ab6e4c3359ae..cd83cbec2b48 100644 --- a/replica/query_state.hh +++ b/replica/query_state.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/schema_describe_helper.hh b/replica/schema_describe_helper.hh index cf31f5a49c26..7b02618ff5c1 100644 --- a/replica/schema_describe_helper.hh +++ b/replica/schema_describe_helper.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/table.cc b/replica/table.cc index 63b244edfcfa..111fdace12b7 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/replica/tablet_mutation_builder.hh b/replica/tablet_mutation_builder.hh index 62b49d730233..ff7d9e98f230 100644 --- a/replica/tablet_mutation_builder.hh +++ b/replica/tablet_mutation_builder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/replica/tablets.cc b/replica/tablets.cc index 4f93458f93f6..e0d3ae129c96 100644 --- a/replica/tablets.cc +++ b/replica/tablets.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/replica/tablets.hh b/replica/tablets.hh index 1f10115b28e1..f0fc90f3096e 100644 --- a/replica/tablets.hh +++ b/replica/tablets.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/reversibly_mergeable.hh b/reversibly_mergeable.hh index da88a7391248..ab65e6ba33e1 100644 --- a/reversibly_mergeable.hh +++ b/reversibly_mergeable.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/row_cache.cc b/row_cache.cc index 14ee954773be..9d31a6d1c694 100644 --- a/row_cache.cc +++ b/row_cache.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "row_cache.hh" diff --git a/row_cache.hh b/row_cache.hh index eedfa732716f..e64b504ffcc4 100644 --- a/row_cache.hh +++ b/row_cache.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/rust/inc/src/lib.rs b/rust/inc/src/lib.rs index 7e10e04aec61..93e5be0fd357 100644 --- a/rust/inc/src/lib.rs +++ b/rust/inc/src/lib.rs @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #[cxx::bridge(namespace = "rust")] diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 065b49a2e542..7ea208a2941f 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ extern crate inc; diff --git a/rust/wasmtime_bindings/src/lib.rs b/rust/wasmtime_bindings/src/lib.rs index 973e51009a82..025035828fda 100644 --- a/rust/wasmtime_bindings/src/lib.rs +++ b/rust/wasmtime_bindings/src/lib.rs @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ use anyhow::Context; diff --git a/rust/wasmtime_bindings/src/memory_creator.rs b/rust/wasmtime_bindings/src/memory_creator.rs index 6481218d1bf8..a17d8aef8f80 100644 --- a/rust/wasmtime_bindings/src/memory_creator.rs +++ b/rust/wasmtime_bindings/src/memory_creator.rs @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ use anyhow::{anyhow, Result}; diff --git a/rust/wasmtime_bindings/src/test_memory_creator.rs b/rust/wasmtime_bindings/src/test_memory_creator.rs index 93cd6c74ba0a..f34546c5f565 100644 --- a/rust/wasmtime_bindings/src/test_memory_creator.rs +++ b/rust/wasmtime_bindings/src/test_memory_creator.rs @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ use crate::memory_creator::ScyllaMemoryCreator; diff --git a/schema/caching_options.cc b/schema/caching_options.cc index afae32a51d81..c932937851fb 100644 --- a/schema/caching_options.cc +++ b/schema/caching_options.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "caching_options.hh" diff --git a/schema/caching_options.hh b/schema/caching_options.hh index 74af5c7bd703..ba8ff5954b85 100644 --- a/schema/caching_options.hh +++ b/schema/caching_options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/schema/schema.cc b/schema/schema.cc index 71237ffb13fb..204c5654b701 100644 --- a/schema/schema.cc +++ b/schema/schema.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/schema/schema.hh b/schema/schema.hh index ecbb80fe5da2..469140b08a81 100644 --- a/schema/schema.hh +++ b/schema/schema.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/schema/schema_builder.hh b/schema/schema_builder.hh index dab6764b4b78..782a63edff9a 100644 --- a/schema/schema_builder.hh +++ b/schema/schema_builder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/schema/schema_fwd.hh b/schema/schema_fwd.hh index 07bee886a9a5..9d150c98fbfd 100644 --- a/schema/schema_fwd.hh +++ b/schema/schema_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/schema/schema_registry.cc b/schema/schema_registry.cc index 19c2678d562b..0bbd412a89d2 100644 --- a/schema/schema_registry.cc +++ b/schema/schema_registry.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/schema/schema_registry.hh b/schema/schema_registry.hh index 7453c3ca3216..bf925e95d3db 100644 --- a/schema/schema_registry.hh +++ b/schema/schema_registry.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/schema_mutations.cc b/schema_mutations.cc index b29a44e33ace..30b2c377f383 100644 --- a/schema_mutations.cc +++ b/schema_mutations.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "schema_mutations.hh" diff --git a/schema_mutations.hh b/schema_mutations.hh index 79b6be661db7..aba2f2bc9ff0 100644 --- a/schema_mutations.hh +++ b/schema_mutations.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/schema_upgrader.hh b/schema_upgrader.hh index d2ada0480fee..51e41c45d88d 100644 --- a/schema_upgrader.hh +++ b/schema_upgrader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/scripts/base36-uuid.py b/scripts/base36-uuid.py index 078386448817..a3608976d757 100755 --- a/scripts/base36-uuid.py +++ b/scripts/base36-uuid.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/scripts/coverage.py b/scripts/coverage.py index 4068fc7485b1..b0a0b9908da9 100755 --- a/scripts/coverage.py +++ b/scripts/coverage.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # diff --git a/scripts/cpp-name-format.py b/scripts/cpp-name-format.py index 3696074f909a..dfad5428f852 100755 --- a/scripts/cpp-name-format.py +++ b/scripts/cpp-name-format.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/scripts/create-relocatable-package.py b/scripts/create-relocatable-package.py index e30f8bd43f3b..81a292bbcbca 100755 --- a/scripts/create-relocatable-package.py +++ b/scripts/create-relocatable-package.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/scripts/jobs b/scripts/jobs index e5bad583f60b..c657d1d56e76 100755 --- a/scripts/jobs +++ b/scripts/jobs @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # Guess and print out a good number of compiler jobs to diff --git a/scripts/merge-compdb.py b/scripts/merge-compdb.py index 44e1378c4700..0fb4869460d9 100755 --- a/scripts/merge-compdb.py +++ b/scripts/merge-compdb.py @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # diff --git a/scripts/nodetool-toppartitions b/scripts/nodetool-toppartitions index b692cca7483d..b03a802d1457 100755 --- a/scripts/nodetool-toppartitions +++ b/scripts/nodetool-toppartitions @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/scripts/open-coredump.sh b/scripts/open-coredump.sh index 31f0d2c9c67f..23bdd3598069 100755 --- a/scripts/open-coredump.sh +++ b/scripts/open-coredump.sh @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # diff --git a/scripts/refresh-submodules.sh b/scripts/refresh-submodules.sh index 7acedc811d18..1e8c77b7fc19 100755 --- a/scripts/refresh-submodules.sh +++ b/scripts/refresh-submodules.sh @@ -6,7 +6,7 @@ # # Copyright (C) 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # set -euo pipefail diff --git a/scripts/tablet-mon.py b/scripts/tablet-mon.py index 4f7eadd2dd77..c339011044ca 100755 --- a/scripts/tablet-mon.py +++ b/scripts/tablet-mon.py @@ -2,7 +2,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This is a tool for live-monitoring the state of tablets and load balancing dynamics in a Scylla cluster. # diff --git a/scylla_post_install.sh b/scylla_post_install.sh index 504b82103bac..f3b2ee9e0427 100755 --- a/scylla_post_install.sh +++ b/scylla_post_install.sh @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # if [ ! -d /run/systemd/system ]; then diff --git a/seastarx.hh b/seastarx.hh index 1c733836e062..6fdf0d0f514d 100644 --- a/seastarx.hh +++ b/seastarx.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/serialization_visitors.hh b/serialization_visitors.hh index 9e0491c4feaa..4ee88676e12f 100644 --- a/serialization_visitors.hh +++ b/serialization_visitors.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/serializer.cc b/serializer.cc index e00c4e026048..175b163cda84 100644 --- a/serializer.cc +++ b/serializer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "serializer_impl.hh" diff --git a/serializer.hh b/serializer.hh index 4ae308178287..67d1ec2de64f 100644 --- a/serializer.hh +++ b/serializer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/serializer_impl.hh b/serializer_impl.hh index 0594bf8e8735..5e63f0cf18e1 100644 --- a/serializer_impl.hh +++ b/serializer_impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/address_map.hh b/service/address_map.hh index 4bac45a2c913..e13c01f0d596 100644 --- a/service/address_map.hh +++ b/service/address_map.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/broadcast_tables/experimental/lang.cc b/service/broadcast_tables/experimental/lang.cc index 910984c697ac..dccbbe5a3c9a 100644 --- a/service/broadcast_tables/experimental/lang.cc +++ b/service/broadcast_tables/experimental/lang.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/service/broadcast_tables/experimental/lang.hh b/service/broadcast_tables/experimental/lang.hh index ed0858365bef..91b8067a5317 100644 --- a/service/broadcast_tables/experimental/lang.hh +++ b/service/broadcast_tables/experimental/lang.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/broadcast_tables/experimental/query_result.hh b/service/broadcast_tables/experimental/query_result.hh index b3ed12cfe698..5034fb3b1b46 100644 --- a/service/broadcast_tables/experimental/query_result.hh +++ b/service/broadcast_tables/experimental/query_result.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/cache_hitrate_calculator.hh b/service/cache_hitrate_calculator.hh index ff0d7a559d84..4a4f364996fb 100644 --- a/service/cache_hitrate_calculator.hh +++ b/service/cache_hitrate_calculator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/client_state.cc b/service/client_state.cc index 265623dea182..76ce2d03c542 100644 --- a/service/client_state.cc +++ b/service/client_state.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "client_state.hh" diff --git a/service/client_state.hh b/service/client_state.hh index b3c22f002bd2..f5998fdca6bc 100644 --- a/service/client_state.hh +++ b/service/client_state.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/endpoint_lifecycle_subscriber.hh b/service/endpoint_lifecycle_subscriber.hh index 01677b4a544e..84a00db69bcc 100644 --- a/service/endpoint_lifecycle_subscriber.hh +++ b/service/endpoint_lifecycle_subscriber.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/load_broadcaster.hh b/service/load_broadcaster.hh index 609c241aaaae..abbce7eb00db 100644 --- a/service/load_broadcaster.hh +++ b/service/load_broadcaster.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/load_meter.hh b/service/load_meter.hh index a8eb3f1eadf5..4da1e97ee047 100644 --- a/service/load_meter.hh +++ b/service/load_meter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/maintenance_mode.hh b/service/maintenance_mode.hh index 6c713cec6eaf..7837e4b6acb0 100644 --- a/service/maintenance_mode.hh +++ b/service/maintenance_mode.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/mapreduce_service.cc b/service/mapreduce_service.cc index 4f06665fa5de..275ed040e825 100644 --- a/service/mapreduce_service.cc +++ b/service/mapreduce_service.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service/mapreduce_service.hh" diff --git a/service/mapreduce_service.hh b/service/mapreduce_service.hh index 69e1a7b080c5..486be5a48ed2 100644 --- a/service/mapreduce_service.hh +++ b/service/mapreduce_service.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/memory_limiter.hh b/service/memory_limiter.hh index 3361dfd569c0..392291e6b6a5 100644 --- a/service/memory_limiter.hh +++ b/service/memory_limiter.hh @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 * * Copyright (C) 2021-present ScyllaDB * diff --git a/service/migration_listener.hh b/service/migration_listener.hh index 20868feed379..89b3eeada632 100644 --- a/service/migration_listener.hh +++ b/service/migration_listener.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/migration_manager.cc b/service/migration_manager.cc index 70e10295b0ed..21d0e730c5b0 100644 --- a/service/migration_manager.cc +++ b/service/migration_manager.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/service/migration_manager.hh b/service/migration_manager.hh index 11bb08c806ea..db030d6099ef 100644 --- a/service/migration_manager.hh +++ b/service/migration_manager.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/misc_services.cc b/service/misc_services.cc index f562a0b27869..a2e6d558c332 100644 --- a/service/misc_services.cc +++ b/service/misc_services.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/service/pager/paging_state.cc b/service/pager/paging_state.cc index 6bf67d66f32b..d8a2a0344024 100644 --- a/service/pager/paging_state.cc +++ b/service/pager/paging_state.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "bytes.hh" diff --git a/service/pager/paging_state.hh b/service/pager/paging_state.hh index ee9cf1c11706..2d4bc0e6a6db 100644 --- a/service/pager/paging_state.hh +++ b/service/pager/paging_state.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/pager/query_pager.hh b/service/pager/query_pager.hh index c0779eeccfc6..3127683eed40 100644 --- a/service/pager/query_pager.hh +++ b/service/pager/query_pager.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/pager/query_pagers.cc b/service/pager/query_pagers.cc index 1607a1ed9992..cfe6316e845c 100644 --- a/service/pager/query_pagers.cc +++ b/service/pager/query_pagers.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "query_pagers.hh" diff --git a/service/pager/query_pagers.hh b/service/pager/query_pagers.hh index 6f8a6fd5e5bb..0fddb1ddf6bf 100644 --- a/service/pager/query_pagers.hh +++ b/service/pager/query_pagers.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/paxos/cas_request.hh b/service/paxos/cas_request.hh index 7e1c72cd1748..0f056682c94e 100644 --- a/service/paxos/cas_request.hh +++ b/service/paxos/cas_request.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/paxos/paxos_state.cc b/service/paxos/paxos_state.cc index 62606a3295a4..f1d1f1870882 100644 --- a/service/paxos/paxos_state.cc +++ b/service/paxos/paxos_state.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include #include diff --git a/service/paxos/paxos_state.hh b/service/paxos/paxos_state.hh index 70101f6a14f6..908267b6612c 100644 --- a/service/paxos/paxos_state.hh +++ b/service/paxos/paxos_state.hh @@ -4,7 +4,7 @@ * Modified by ScyllaDB */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once #include "seastar/core/semaphore.hh" diff --git a/service/paxos/prepare_response.cc b/service/paxos/prepare_response.cc index bc75e2906d27..7fdbde4e2467 100644 --- a/service/paxos/prepare_response.cc +++ b/service/paxos/prepare_response.cc @@ -4,7 +4,7 @@ * Modified by ScyllaDB */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "prepare_response.hh" diff --git a/service/paxos/prepare_response.hh b/service/paxos/prepare_response.hh index a6b20680e371..7b7a108202e1 100644 --- a/service/paxos/prepare_response.hh +++ b/service/paxos/prepare_response.hh @@ -4,7 +4,7 @@ * Modified by ScyllaDB */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/paxos/prepare_summary.cc b/service/paxos/prepare_summary.cc index 0b4fb5fc48ad..b4387e37d5c9 100644 --- a/service/paxos/prepare_summary.cc +++ b/service/paxos/prepare_summary.cc @@ -4,7 +4,7 @@ * Modified by ScyllaDB */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "service/paxos/prepare_summary.hh" diff --git a/service/paxos/prepare_summary.hh b/service/paxos/prepare_summary.hh index 9c211c98341a..7cf24d097605 100644 --- a/service/paxos/prepare_summary.hh +++ b/service/paxos/prepare_summary.hh @@ -4,7 +4,7 @@ * Modified by ScyllaDB */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/paxos/proposal.cc b/service/paxos/proposal.cc index 1382f5c36c02..16215e0742af 100644 --- a/service/paxos/proposal.cc +++ b/service/paxos/proposal.cc @@ -4,7 +4,7 @@ * Modified by ScyllaDB */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "proposal.hh" diff --git a/service/paxos/proposal.hh b/service/paxos/proposal.hh index ce02afc525e7..0b3a2d28b9d1 100644 --- a/service/paxos/proposal.hh +++ b/service/paxos/proposal.hh @@ -4,7 +4,7 @@ * Modified by ScyllaDB */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index 0e89d7532a3a..ad782ded1c59 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "qos_common.hh" diff --git a/service/qos/qos_common.hh b/service/qos/qos_common.hh index bfb6812f3b77..16efa26037c9 100644 --- a/service/qos/qos_common.hh +++ b/service/qos/qos_common.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/qos/qos_configuration_change_subscriber.hh b/service/qos/qos_configuration_change_subscriber.hh index 9311402cdb3a..0804e4609d37 100644 --- a/service/qos/qos_configuration_change_subscriber.hh +++ b/service/qos/qos_configuration_change_subscriber.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/qos/raft_service_level_distributed_data_accessor.cc b/service/qos/raft_service_level_distributed_data_accessor.cc index 61f24f8da49c..588081f37988 100644 --- a/service/qos/raft_service_level_distributed_data_accessor.cc +++ b/service/qos/raft_service_level_distributed_data_accessor.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "raft_service_level_distributed_data_accessor.hh" diff --git a/service/qos/raft_service_level_distributed_data_accessor.hh b/service/qos/raft_service_level_distributed_data_accessor.hh index b76a01051f91..311559860540 100644 --- a/service/qos/raft_service_level_distributed_data_accessor.hh +++ b/service/qos/raft_service_level_distributed_data_accessor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index 78ea503e5856..4f15b98047f3 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "cql3/util.hh" diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index dbac8c4a3105..4afc634b6d7f 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/qos/standard_service_level_distributed_data_accessor.cc b/service/qos/standard_service_level_distributed_data_accessor.cc index 198736cbe415..c7f7171561e1 100644 --- a/service/qos/standard_service_level_distributed_data_accessor.cc +++ b/service/qos/standard_service_level_distributed_data_accessor.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "standard_service_level_distributed_data_accessor.hh" diff --git a/service/qos/standard_service_level_distributed_data_accessor.hh b/service/qos/standard_service_level_distributed_data_accessor.hh index 7777588c67b6..a308ffd63b2e 100644 --- a/service/qos/standard_service_level_distributed_data_accessor.hh +++ b/service/qos/standard_service_level_distributed_data_accessor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/query_state.hh b/service/query_state.hh index 93048d2ad6b8..d448ad825cc8 100644 --- a/service/query_state.hh +++ b/service/query_state.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef SERVICE_QUERY_STATE_HH diff --git a/service/raft/discovery.cc b/service/raft/discovery.cc index 8bef0889bd8e..854e1159da00 100644 --- a/service/raft/discovery.cc +++ b/service/raft/discovery.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" #include "service/raft/discovery.hh" diff --git a/service/raft/discovery.hh b/service/raft/discovery.hh index d8a16e5b2e83..0540752f488c 100644 --- a/service/raft/discovery.hh +++ b/service/raft/discovery.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once #include "service/raft/group0_fwd.hh" diff --git a/service/raft/group0_fwd.hh b/service/raft/group0_fwd.hh index 867dd1d09080..825a43900867 100644 --- a/service/raft/group0_fwd.hh +++ b/service/raft/group0_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/group0_state_id_handler.cc b/service/raft/group0_state_id_handler.cc index 13db19db00d0..dff564c6318c 100644 --- a/service/raft/group0_state_id_handler.cc +++ b/service/raft/group0_state_id_handler.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service/raft/group0_state_id_handler.hh" #include "db/config.hh" diff --git a/service/raft/group0_state_id_handler.hh b/service/raft/group0_state_id_handler.hh index a8c66b037c06..746c0c5adcbe 100644 --- a/service/raft/group0_state_id_handler.hh +++ b/service/raft/group0_state_id_handler.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/group0_state_machine.cc b/service/raft/group0_state_machine.cc index 7e9ee9cda5a1..3c472ed09cfc 100644 --- a/service/raft/group0_state_machine.cc +++ b/service/raft/group0_state_machine.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service/raft/group0_state_machine.hh" #include "mutation/atomic_cell.hh" diff --git a/service/raft/group0_state_machine.hh b/service/raft/group0_state_machine.hh index 8dc8144cd02a..27220c0cd9da 100644 --- a/service/raft/group0_state_machine.hh +++ b/service/raft/group0_state_machine.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/group0_state_machine_merger.cc b/service/raft/group0_state_machine_merger.cc index 37e26f0d807b..ea33098fd046 100644 --- a/service/raft/group0_state_machine_merger.cc +++ b/service/raft/group0_state_machine_merger.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/config.hh" #include "db/system_keyspace.hh" diff --git a/service/raft/group0_state_machine_merger.hh b/service/raft/group0_state_machine_merger.hh index 394b6f479170..4066c052e6c1 100644 --- a/service/raft/group0_state_machine_merger.hh +++ b/service/raft/group0_state_machine_merger.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/join_node.hh b/service/raft/join_node.hh index 20c071f30a42..18546a6dfa1f 100644 --- a/service/raft/join_node.hh +++ b/service/raft/join_node.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index 349e238c5509..089a0a7e2c53 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include #include diff --git a/service/raft/raft_group0.hh b/service/raft/raft_group0.hh index a4e54eda15ce..5cd05a30607d 100644 --- a/service/raft/raft_group0.hh +++ b/service/raft/raft_group0.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/raft_group0_client.cc b/service/raft/raft_group0_client.cc index 199343000b54..c52284b43f57 100644 --- a/service/raft/raft_group0_client.cc +++ b/service/raft/raft_group0_client.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/service/raft/raft_group0_client.hh b/service/raft/raft_group0_client.hh index 36123a4fc639..b15529ee747c 100644 --- a/service/raft/raft_group0_client.hh +++ b/service/raft/raft_group0_client.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/raft_group_registry.cc b/service/raft/raft_group_registry.cc index f60a3e2f4e77..ca8fb0a9685e 100644 --- a/service/raft/raft_group_registry.cc +++ b/service/raft/raft_group_registry.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service/raft/raft_group_registry.hh" #include "raft/raft.hh" diff --git a/service/raft/raft_group_registry.hh b/service/raft/raft_group_registry.hh index 69369d859cc8..b2e47e46c7d2 100644 --- a/service/raft/raft_group_registry.hh +++ b/service/raft/raft_group_registry.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/raft_rpc.cc b/service/raft/raft_rpc.cc index 62529f75fb6f..643e97bbe3bb 100644 --- a/service/raft/raft_rpc.cc +++ b/service/raft/raft_rpc.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service/raft/raft_rpc.hh" #include diff --git a/service/raft/raft_rpc.hh b/service/raft/raft_rpc.hh index e20424184ab3..2429de5a8b8b 100644 --- a/service/raft/raft_rpc.hh +++ b/service/raft/raft_rpc.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/raft_state_machine.hh b/service/raft/raft_state_machine.hh index e4599f7f6d09..0e0435103b62 100644 --- a/service/raft/raft_state_machine.hh +++ b/service/raft/raft_state_machine.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/raft_sys_table_storage.cc b/service/raft/raft_sys_table_storage.cc index db3592f5ee42..5f8c2a050098 100644 --- a/service/raft/raft_sys_table_storage.cc +++ b/service/raft/raft_sys_table_storage.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service/raft/raft_sys_table_storage.hh" diff --git a/service/raft/raft_sys_table_storage.hh b/service/raft/raft_sys_table_storage.hh index fda8c3070216..b9c63484bb3c 100644 --- a/service/raft/raft_sys_table_storage.hh +++ b/service/raft/raft_sys_table_storage.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/raft/raft_timeout.hh b/service/raft/raft_timeout.hh index b00693a4abd1..9e4777e11425 100644 --- a/service/raft/raft_timeout.hh +++ b/service/raft/raft_timeout.hh @@ -1,5 +1,5 @@ // Copyright (C) 2024-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/service/session.cc b/service/session.cc index e00907aed5bb..d85b56ad2f73 100644 --- a/service/session.cc +++ b/service/session.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "service/session.hh" diff --git a/service/session.hh b/service/session.hh index 64e8309a3f28..703d7a8b7fe2 100644 --- a/service/session.hh +++ b/service/session.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/state_id.hh b/service/state_id.hh index 85141deebde9..ef3ee053c979 100644 --- a/service/state_id.hh +++ b/service/state_id.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 34b1d64e757b..38e7991cbdfa 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/service/storage_proxy.hh b/service/storage_proxy.hh index 1783589a5126..6689b68c5d8f 100644 --- a/service/storage_proxy.hh +++ b/service/storage_proxy.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/storage_proxy_stats.hh b/service/storage_proxy_stats.hh index 544c1194b440..e8ca2bd5c865 100644 --- a/service/storage_proxy_stats.hh +++ b/service/storage_proxy_stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/storage_service.cc b/service/storage_service.cc index 1504d5f2dec8..167e0bdecc6c 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -6,7 +6,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "storage_service.hh" diff --git a/service/storage_service.hh b/service/storage_service.hh index 6febb14fc161..7d540ae38cbf 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -6,7 +6,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/tablet_allocator.cc b/service/tablet_allocator.cc index 1ef81a4ac96a..f212a1b44084 100644 --- a/service/tablet_allocator.cc +++ b/service/tablet_allocator.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/tablets.hh" diff --git a/service/tablet_allocator.hh b/service/tablet_allocator.hh index 2036b9ea3d9e..ac645db1dc4d 100644 --- a/service/tablet_allocator.hh +++ b/service/tablet_allocator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/tablet_allocator_fwd.hh b/service/tablet_allocator_fwd.hh index 8873cd02a491..753425cdce8a 100644 --- a/service/tablet_allocator_fwd.hh +++ b/service/tablet_allocator_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index 99ddb6c3997d..eddd9bc167c0 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "locator/tablets.hh" diff --git a/service/task_manager_module.hh b/service/task_manager_module.hh index 331009f81806..1fbce3aec3d2 100644 --- a/service/task_manager_module.hh +++ b/service/task_manager_module.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index 60e907a795ab..eb374d4b4616 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/service/topology_coordinator.hh b/service/topology_coordinator.hh index e20b024a9b14..0a7ce0f5c43d 100644 --- a/service/topology_coordinator.hh +++ b/service/topology_coordinator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/topology_guard.hh b/service/topology_guard.hh index 1012990e1586..65cbc9a030d6 100644 --- a/service/topology_guard.hh +++ b/service/topology_guard.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/topology_mutation.cc b/service/topology_mutation.cc index c9be2916afe1..e51416f72c4e 100644 --- a/service/topology_mutation.cc +++ b/service/topology_mutation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/service/topology_mutation.hh b/service/topology_mutation.hh index eb5514355cb7..25a4bd43fdae 100644 --- a/service/topology_mutation.hh +++ b/service/topology_mutation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service/topology_state_machine.cc b/service/topology_state_machine.cc index 4cc8ccaae5ca..fd31e5a538f5 100644 --- a/service/topology_state_machine.cc +++ b/service/topology_state_machine.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "topology_state_machine.hh" diff --git a/service/topology_state_machine.hh b/service/topology_state_machine.hh index 2d245c0da283..0d29d66c7f0d 100644 --- a/service/topology_state_machine.hh +++ b/service/topology_state_machine.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/service/view_update_backlog_broker.hh b/service/view_update_backlog_broker.hh index fd035b05dfda..30db2c8410e2 100644 --- a/service/view_update_backlog_broker.hh +++ b/service/view_update_backlog_broker.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/service_permit.hh b/service_permit.hh index 3eaf8f0e5487..0934d9e86acd 100644 --- a/service_permit.hh +++ b/service_permit.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/setup.py b/setup.py index 44f65b41ac23..c60d1253ef83 100644 --- a/setup.py +++ b/setup.py @@ -5,7 +5,7 @@ description='NoSQL data store using the seastar framework, compatible with Apache Cassandra', url='https://github.com/scylladb/scylla', download_url='https://github.com/scylladb/scylla/tags', - license='AGPL', + license='ScyllaDB-Source-Available-1.0', platforms='any', packages=find_packages(), include_package_data=True, diff --git a/shell.nix b/shell.nix index 770fee58c73b..e8cb7843707f 100644 --- a/shell.nix +++ b/shell.nix @@ -2,7 +2,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 args: diff --git a/sstables/binary_search.hh b/sstables/binary_search.hh index a33f63cf6ccb..be63a0b18539 100644 --- a/sstables/binary_search.hh +++ b/sstables/binary_search.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/checksum_utils.hh b/sstables/checksum_utils.hh index 37399dd344fc..b227f0daac49 100644 --- a/sstables/checksum_utils.hh +++ b/sstables/checksum_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/checksummed_data_source.cc b/sstables/checksummed_data_source.cc index 2f206fc8bda6..87aab4c187e8 100644 --- a/sstables/checksummed_data_source.cc +++ b/sstables/checksummed_data_source.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables/checksummed_data_source.hh b/sstables/checksummed_data_source.hh index f23ae18f0a80..30fab99bac6f 100644 --- a/sstables/checksummed_data_source.hh +++ b/sstables/checksummed_data_source.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/column_translation.hh b/sstables/column_translation.hh index 6f11c51f0952..6f47ebb04fee 100644 --- a/sstables/column_translation.hh +++ b/sstables/column_translation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/component_type.hh b/sstables/component_type.hh index 56496f71ea80..7de680f8f5a6 100644 --- a/sstables/component_type.hh +++ b/sstables/component_type.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/compress.cc b/sstables/compress.cc index 2601845221e9..a85e1d7d9d17 100644 --- a/sstables/compress.cc +++ b/sstables/compress.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables/compress.hh b/sstables/compress.hh index 419465df5199..8302d0a17f31 100644 --- a/sstables/compress.hh +++ b/sstables/compress.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/consumer.hh b/sstables/consumer.hh index bc9a208c44cc..070c4417b53c 100644 --- a/sstables/consumer.hh +++ b/sstables/consumer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/data_source_types.hh b/sstables/data_source_types.hh index e8786cfee870..b795a47ca928 100644 --- a/sstables/data_source_types.hh +++ b/sstables/data_source_types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/disk_types.hh b/sstables/disk_types.hh index af69b0032574..60d7b86d4217 100644 --- a/sstables/disk_types.hh +++ b/sstables/disk_types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/downsampling.hh b/sstables/downsampling.hh index 20efe4e2f72f..9c59e19e7677 100644 --- a/sstables/downsampling.hh +++ b/sstables/downsampling.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/sstables/exceptions.hh b/sstables/exceptions.hh index 5b9e50c13f39..b16d54a82b17 100644 --- a/sstables/exceptions.hh +++ b/sstables/exceptions.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/file_writer.hh b/sstables/file_writer.hh index 1c4412df500c..21366f2df90f 100644 --- a/sstables/file_writer.hh +++ b/sstables/file_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/filter.hh b/sstables/filter.hh index 7d033f35e274..c75e9f537abc 100644 --- a/sstables/filter.hh +++ b/sstables/filter.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/generation_type.hh b/sstables/generation_type.hh index a8bcb6b2ef2f..0ae1de105753 100644 --- a/sstables/generation_type.hh +++ b/sstables/generation_type.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/hyperloglog.hh b/sstables/hyperloglog.hh index 2f573e19cf75..4103b4c5e826 100644 --- a/sstables/hyperloglog.hh +++ b/sstables/hyperloglog.hh @@ -10,7 +10,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/sstables/index_entry.hh b/sstables/index_entry.hh index 625655fb107b..082e41b6b3df 100644 --- a/sstables/index_entry.hh +++ b/sstables/index_entry.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/index_reader.hh b/sstables/index_reader.hh index cc6f737f1214..b6a5ac6569e9 100644 --- a/sstables/index_reader.hh +++ b/sstables/index_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/integrity_checked_file_impl.cc b/sstables/integrity_checked_file_impl.cc index f1387a3b4b9c..02513b1c5698 100644 --- a/sstables/integrity_checked_file_impl.cc +++ b/sstables/integrity_checked_file_impl.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "integrity_checked_file_impl.hh" diff --git a/sstables/integrity_checked_file_impl.hh b/sstables/integrity_checked_file_impl.hh index 295ba75d6db5..755d24d20474 100644 --- a/sstables/integrity_checked_file_impl.hh +++ b/sstables/integrity_checked_file_impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/key.hh b/sstables/key.hh index f32426a55b60..52cb298f66d3 100644 --- a/sstables/key.hh +++ b/sstables/key.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/kl/reader.cc b/sstables/kl/reader.cc index d7979b717f0e..70428f50b4a6 100644 --- a/sstables/kl/reader.cc +++ b/sstables/kl/reader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "sstables/consumer.hh" diff --git a/sstables/kl/reader.hh b/sstables/kl/reader.hh index b70ac200f393..82917349b630 100644 --- a/sstables/kl/reader.hh +++ b/sstables/kl/reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/liveness_info.hh b/sstables/liveness_info.hh index 6bc9aec236e8..7c01c8cbb613 100644 --- a/sstables/liveness_info.hh +++ b/sstables/liveness_info.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/m_format_read_helpers.cc b/sstables/m_format_read_helpers.cc index 974e5762fba1..7059d3fce1b9 100644 --- a/sstables/m_format_read_helpers.cc +++ b/sstables/m_format_read_helpers.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "vint-serialization.hh" diff --git a/sstables/m_format_read_helpers.hh b/sstables/m_format_read_helpers.hh index d2ed29b965b1..a5c961150c17 100644 --- a/sstables/m_format_read_helpers.hh +++ b/sstables/m_format_read_helpers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/metadata_collector.cc b/sstables/metadata_collector.cc index e6e596467d92..dfef9b2d3b4a 100644 --- a/sstables/metadata_collector.cc +++ b/sstables/metadata_collector.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/log.hh" diff --git a/sstables/metadata_collector.hh b/sstables/metadata_collector.hh index 14968c60a3f3..c72e64843bc8 100644 --- a/sstables/metadata_collector.hh +++ b/sstables/metadata_collector.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/sstables/mutation_fragment_filter.hh b/sstables/mutation_fragment_filter.hh index 5b7d90e87b36..f967895e19b5 100644 --- a/sstables/mutation_fragment_filter.hh +++ b/sstables/mutation_fragment_filter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/mx/bsearch_clustered_cursor.hh b/sstables/mx/bsearch_clustered_cursor.hh index a11fa155a37b..d40a5e48eda3 100644 --- a/sstables/mx/bsearch_clustered_cursor.hh +++ b/sstables/mx/bsearch_clustered_cursor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/mx/parsers.hh b/sstables/mx/parsers.hh index 2d7def6574a9..9eb85767a256 100644 --- a/sstables/mx/parsers.hh +++ b/sstables/mx/parsers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/mx/partition_reversing_data_source.cc b/sstables/mx/partition_reversing_data_source.cc index 3d35f7547492..7ea61d8ebf2f 100644 --- a/sstables/mx/partition_reversing_data_source.cc +++ b/sstables/mx/partition_reversing_data_source.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables/mx/partition_reversing_data_source.hh b/sstables/mx/partition_reversing_data_source.hh index c2836bd7257c..10a8f9f1a08c 100644 --- a/sstables/mx/partition_reversing_data_source.hh +++ b/sstables/mx/partition_reversing_data_source.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/mx/reader.cc b/sstables/mx/reader.cc index f53e954cd048..e8636bf6a3f4 100644 --- a/sstables/mx/reader.cc +++ b/sstables/mx/reader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "reader.hh" diff --git a/sstables/mx/reader.hh b/sstables/mx/reader.hh index 50b7029013cc..f0ab3cca8c87 100644 --- a/sstables/mx/reader.hh +++ b/sstables/mx/reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/mx/types.hh b/sstables/mx/types.hh index 2f80c1f18be1..610d81035f68 100644 --- a/sstables/mx/types.hh +++ b/sstables/mx/types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/mx/writer.cc b/sstables/mx/writer.cc index ef8cdd145043..b331cfc4216c 100644 --- a/sstables/mx/writer.cc +++ b/sstables/mx/writer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "sstables/mx/writer.hh" diff --git a/sstables/mx/writer.hh b/sstables/mx/writer.hh index c56c8832307e..e5f776e6faf1 100644 --- a/sstables/mx/writer.hh +++ b/sstables/mx/writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/open_info.hh b/sstables/open_info.hh index d32a475ae5d3..323e78490a17 100644 --- a/sstables/open_info.hh +++ b/sstables/open_info.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/partition_index_cache.hh b/sstables/partition_index_cache.hh index e7aa7dfa206f..a6b87cfd1876 100644 --- a/sstables/partition_index_cache.hh +++ b/sstables/partition_index_cache.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/partition_index_cache_stats.hh b/sstables/partition_index_cache_stats.hh index 0dbed49be721..0f9c37f00c9d 100644 --- a/sstables/partition_index_cache_stats.hh +++ b/sstables/partition_index_cache_stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/prepended_input_stream.cc b/sstables/prepended_input_stream.cc index 19953302b846..7aeec5868afe 100644 --- a/sstables/prepended_input_stream.cc +++ b/sstables/prepended_input_stream.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "prepended_input_stream.hh" diff --git a/sstables/prepended_input_stream.hh b/sstables/prepended_input_stream.hh index 038ca7a49e56..576ca446b929 100644 --- a/sstables/prepended_input_stream.hh +++ b/sstables/prepended_input_stream.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/processing_result_generator.hh b/sstables/processing_result_generator.hh index f10a8e38602d..6755a9392d2b 100644 --- a/sstables/processing_result_generator.hh +++ b/sstables/processing_result_generator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/progress_monitor.hh b/sstables/progress_monitor.hh index aaaaa5cfb33e..2c4fb2a0b952 100644 --- a/sstables/progress_monitor.hh +++ b/sstables/progress_monitor.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/promoted_index_blocks_reader.hh b/sstables/promoted_index_blocks_reader.hh index 0855b78124b7..87487d2e6091 100644 --- a/sstables/promoted_index_blocks_reader.hh +++ b/sstables/promoted_index_blocks_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/random_access_reader.cc b/sstables/random_access_reader.cc index a6230be5bb42..e53e652cdebc 100644 --- a/sstables/random_access_reader.cc +++ b/sstables/random_access_reader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables/random_access_reader.hh b/sstables/random_access_reader.hh index c240c5a7e693..467e1f7de169 100644 --- a/sstables/random_access_reader.hh +++ b/sstables/random_access_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/scanning_clustered_index_cursor.hh b/sstables/scanning_clustered_index_cursor.hh index af9ad0fe2003..287d585dc6d2 100644 --- a/sstables/scanning_clustered_index_cursor.hh +++ b/sstables/scanning_clustered_index_cursor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/segmented_compress_params.hh b/sstables/segmented_compress_params.hh index 9f587178168c..721aca358303 100644 --- a/sstables/segmented_compress_params.hh +++ b/sstables/segmented_compress_params.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/sstables/shareable_components.hh b/sstables/shareable_components.hh index 2916d0fe2d52..b3940468e782 100644 --- a/sstables/shareable_components.hh +++ b/sstables/shareable_components.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/shared_sstable.hh b/sstables/shared_sstable.hh index 497fb07c3df4..1489315f4b93 100644 --- a/sstables/shared_sstable.hh +++ b/sstables/shared_sstable.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_directory.cc b/sstables/sstable_directory.cc index 5c751e171a68..bae64522c34a 100644 --- a/sstables/sstable_directory.cc +++ b/sstables/sstable_directory.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables/sstable_directory.hh b/sstables/sstable_directory.hh index e1421f5ab310..697ef5bc826f 100644 --- a/sstables/sstable_directory.hh +++ b/sstables/sstable_directory.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_mutation_reader.cc b/sstables/sstable_mutation_reader.cc index 400c7de2bbdb..b569ccabb5e8 100644 --- a/sstables/sstable_mutation_reader.cc +++ b/sstables/sstable_mutation_reader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "sstable_mutation_reader.hh" diff --git a/sstables/sstable_mutation_reader.hh b/sstables/sstable_mutation_reader.hh index 4988be9ef1a4..b01eefb01a54 100644 --- a/sstables/sstable_mutation_reader.hh +++ b/sstables/sstable_mutation_reader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_set.cc b/sstables/sstable_set.cc index cb21722f6521..d6fe0e55cab3 100644 --- a/sstables/sstable_set.cc +++ b/sstables/sstable_set.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables/sstable_set.hh b/sstables/sstable_set.hh index 201d69ec8545..141503168d43 100644 --- a/sstables/sstable_set.hh +++ b/sstables/sstable_set.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_set_impl.hh b/sstables/sstable_set_impl.hh index ea223e84b9e0..0c4deab86a5a 100644 --- a/sstables/sstable_set_impl.hh +++ b/sstables/sstable_set_impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_version.cc b/sstables/sstable_version.cc index 26c2bc797984..b85c9f48d2dd 100644 --- a/sstables/sstable_version.cc +++ b/sstables/sstable_version.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "sstable_version.hh" diff --git a/sstables/sstable_version.hh b/sstables/sstable_version.hh index d4441cc5e6f3..5fffffcc11fe 100644 --- a/sstables/sstable_version.hh +++ b/sstables/sstable_version.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_version_k_l.hh b/sstables/sstable_version_k_l.hh index ab1fd8460987..3a888ff47998 100644 --- a/sstables/sstable_version_k_l.hh +++ b/sstables/sstable_version_k_l.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_version_m.hh b/sstables/sstable_version_m.hh index 1caa74a4d1dc..848237b5ba68 100644 --- a/sstables/sstable_version_m.hh +++ b/sstables/sstable_version_m.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstable_writer.hh b/sstables/sstable_writer.hh index ae5d9a2f5b3a..90635532b50a 100644 --- a/sstables/sstable_writer.hh +++ b/sstables/sstable_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstables.cc b/sstables/sstables.cc index 283022d27896..0b0fe6d14915 100644 --- a/sstables/sstables.cc +++ b/sstables/sstables.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/log.hh" diff --git a/sstables/sstables.hh b/sstables/sstables.hh index 79d1383bcaf1..5a63e168b3d7 100644 --- a/sstables/sstables.hh +++ b/sstables/sstables.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstables_manager.cc b/sstables/sstables_manager.cc index 371757089257..81e070966faf 100644 --- a/sstables/sstables_manager.cc +++ b/sstables/sstables_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables/sstables_manager.hh b/sstables/sstables_manager.hh index e792f7e2a919..d851fce2e3f5 100644 --- a/sstables/sstables_manager.hh +++ b/sstables/sstables_manager.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/sstables_registry.hh b/sstables/sstables_registry.hh index e6e6bb4b03ad..e360c4d636a7 100644 --- a/sstables/sstables_registry.hh +++ b/sstables/sstables_registry.hh @@ -1,5 +1,5 @@ // Copyright (C) 2024-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/sstables/stats.hh b/sstables/stats.hh index 2fc6f98c8c84..47b639b91661 100644 --- a/sstables/stats.hh +++ b/sstables/stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/storage.cc b/sstables/storage.cc index 45f016affd0d..41ca0a93e8b6 100644 --- a/sstables/storage.cc +++ b/sstables/storage.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "storage.hh" diff --git a/sstables/storage.hh b/sstables/storage.hh index b571656bec47..206217b33599 100644 --- a/sstables/storage.hh +++ b/sstables/storage.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/types.hh b/sstables/types.hh index a4b6d742a940..51dd55868449 100644 --- a/sstables/types.hh +++ b/sstables/types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/types_fwd.hh b/sstables/types_fwd.hh index 48aa8d342b89..5f55ea487ff2 100644 --- a/sstables/types_fwd.hh +++ b/sstables/types_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/version.hh b/sstables/version.hh index b8e128b35e19..ebacef3241b8 100644 --- a/sstables/version.hh +++ b/sstables/version.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/writer.cc b/sstables/writer.cc index 85d7ea944af0..f44e89aca3c8 100644 --- a/sstables/writer.cc +++ b/sstables/writer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "sstables.hh" diff --git a/sstables/writer.hh b/sstables/writer.hh index a6a2df769eaf..d0b006cd9f69 100644 --- a/sstables/writer.hh +++ b/sstables/writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables/writer_impl.hh b/sstables/writer_impl.hh index 1f3b03fc56f3..49110a97b9e4 100644 --- a/sstables/writer_impl.hh +++ b/sstables/writer_impl.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/sstables_loader.cc b/sstables_loader.cc index c4497917d77f..d06d7bc00e22 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/sstables_loader.hh b/sstables_loader.hh index a2896f4b356e..8dfc63d27f4e 100644 --- a/sstables_loader.hh +++ b/sstables_loader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/streaming/consumer.cc b/streaming/consumer.cc index d2e437d65e41..f49e4dfac0ec 100644 --- a/streaming/consumer.cc +++ b/streaming/consumer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/streaming/consumer.hh b/streaming/consumer.hh index d4ad1f6306dc..dabdf3aad17b 100644 --- a/streaming/consumer.hh +++ b/streaming/consumer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/streaming/prepare_message.hh b/streaming/prepare_message.hh index 745c202eae7c..f0d25ab64c99 100644 --- a/streaming/prepare_message.hh +++ b/streaming/prepare_message.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/progress_info.cc b/streaming/progress_info.cc index eecf94ba3dbf..b4bfbd99ff1b 100644 --- a/streaming/progress_info.cc +++ b/streaming/progress_info.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/streaming/progress_info.hh b/streaming/progress_info.hh index b03db6298442..359e506408e4 100644 --- a/streaming/progress_info.hh +++ b/streaming/progress_info.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/session_info.cc b/streaming/session_info.cc index 54e127ae54aa..e2192afb0ced 100644 --- a/streaming/session_info.cc +++ b/streaming/session_info.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/streaming/session_info.hh b/streaming/session_info.hh index fc7aa9c2ffa8..476768e0627c 100644 --- a/streaming/session_info.hh +++ b/streaming/session_info.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_coordinator.cc b/streaming/stream_coordinator.cc index 7630fd6fb777..22e0c5079868 100644 --- a/streaming/stream_coordinator.cc +++ b/streaming/stream_coordinator.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_session_state.hh" diff --git a/streaming/stream_coordinator.hh b/streaming/stream_coordinator.hh index 20f4c0633956..40203d7a015c 100644 --- a/streaming/stream_coordinator.hh +++ b/streaming/stream_coordinator.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_detail.hh b/streaming/stream_detail.hh index 8a7c0729d996..0d294665172e 100644 --- a/streaming/stream_detail.hh +++ b/streaming/stream_detail.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_event.hh b/streaming/stream_event.hh index d0063e078617..2f6f58ee640d 100644 --- a/streaming/stream_event.hh +++ b/streaming/stream_event.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_event_handler.hh b/streaming/stream_event_handler.hh index f9dff1205c27..814c729317e8 100644 --- a/streaming/stream_event_handler.hh +++ b/streaming/stream_event_handler.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_exception.hh b/streaming/stream_exception.hh index 2021480a645e..6a6d3dc4d2db 100644 --- a/streaming/stream_exception.hh +++ b/streaming/stream_exception.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_fwd.hh b/streaming/stream_fwd.hh index a29e570a3160..5d58ed0ab4b7 100644 --- a/streaming/stream_fwd.hh +++ b/streaming/stream_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/streaming/stream_manager.cc b/streaming/stream_manager.cc index 1bf2ab108428..23730a794722 100644 --- a/streaming/stream_manager.cc +++ b/streaming/stream_manager.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include diff --git a/streaming/stream_manager.hh b/streaming/stream_manager.hh index 4fc26dd1ba45..5b577aad7369 100644 --- a/streaming/stream_manager.hh +++ b/streaming/stream_manager.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_mutation_fragments_cmd.hh b/streaming/stream_mutation_fragments_cmd.hh index 70dd7656b78b..c160e5e73f95 100644 --- a/streaming/stream_mutation_fragments_cmd.hh +++ b/streaming/stream_mutation_fragments_cmd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/streaming/stream_plan.cc b/streaming/stream_plan.cc index 936760182c73..f197962c2c95 100644 --- a/streaming/stream_plan.cc +++ b/streaming/stream_plan.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_plan.hh" diff --git a/streaming/stream_plan.hh b/streaming/stream_plan.hh index e36857eef937..0ac8a4df9451 100644 --- a/streaming/stream_plan.hh +++ b/streaming/stream_plan.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_reason.hh b/streaming/stream_reason.hh index 87b4f8adc8a2..cb5b0e312685 100644 --- a/streaming/stream_reason.hh +++ b/streaming/stream_reason.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/streaming/stream_receive_task.cc b/streaming/stream_receive_task.cc index c3192ba7a223..12ed8c1f367f 100644 --- a/streaming/stream_receive_task.cc +++ b/streaming/stream_receive_task.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_session.hh" diff --git a/streaming/stream_receive_task.hh b/streaming/stream_receive_task.hh index 79aa42436101..5a2037b42b25 100644 --- a/streaming/stream_receive_task.hh +++ b/streaming/stream_receive_task.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_request.cc b/streaming/stream_request.cc index eb67f64e1c10..36e63d1d06c1 100644 --- a/streaming/stream_request.cc +++ b/streaming/stream_request.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_request.hh" diff --git a/streaming/stream_request.hh b/streaming/stream_request.hh index 9520725b18cf..469b398b7169 100644 --- a/streaming/stream_request.hh +++ b/streaming/stream_request.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_result_future.cc b/streaming/stream_result_future.cc index c399ce67fef5..e825d9b9abcf 100644 --- a/streaming/stream_result_future.cc +++ b/streaming/stream_result_future.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_result_future.hh" diff --git a/streaming/stream_result_future.hh b/streaming/stream_result_future.hh index 2a8fc64ca5fe..69abec1d8363 100644 --- a/streaming/stream_result_future.hh +++ b/streaming/stream_result_future.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_session.cc b/streaming/stream_session.cc index 6e52011465fc..4619f2b09242 100644 --- a/streaming/stream_session.cc +++ b/streaming/stream_session.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "locator/token_metadata.hh" diff --git a/streaming/stream_session.hh b/streaming/stream_session.hh index e1f19bb3c641..3b8b06079e75 100644 --- a/streaming/stream_session.hh +++ b/streaming/stream_session.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_session_state.cc b/streaming/stream_session_state.cc index f992b6bf32ff..aabfb293dd22 100644 --- a/streaming/stream_session_state.cc +++ b/streaming/stream_session_state.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_session_state.hh" diff --git a/streaming/stream_session_state.hh b/streaming/stream_session_state.hh index 1714db6d5942..b711960d7f04 100644 --- a/streaming/stream_session_state.hh +++ b/streaming/stream_session_state.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_state.hh b/streaming/stream_state.hh index 6bc01727db66..24dc87bd3c36 100644 --- a/streaming/stream_state.hh +++ b/streaming/stream_state.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_summary.cc b/streaming/stream_summary.cc index 34155e03cd82..cb45a6d8d5aa 100644 --- a/streaming/stream_summary.cc +++ b/streaming/stream_summary.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_summary.hh" diff --git a/streaming/stream_summary.hh b/streaming/stream_summary.hh index b4dc612f12ce..3241d065404f 100644 --- a/streaming/stream_summary.hh +++ b/streaming/stream_summary.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_task.cc b/streaming/stream_task.cc index cd6e06e48c27..200ca31d9cea 100644 --- a/streaming/stream_task.cc +++ b/streaming/stream_task.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "streaming/stream_task.hh" diff --git a/streaming/stream_task.hh b/streaming/stream_task.hh index 3784fcb34b10..8847584ae1f1 100644 --- a/streaming/stream_task.hh +++ b/streaming/stream_task.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/streaming/stream_transfer_task.cc b/streaming/stream_transfer_task.cc index 6f6c27f97716..ce1616cbb542 100644 --- a/streaming/stream_transfer_task.cc +++ b/streaming/stream_transfer_task.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/log.hh" diff --git a/streaming/stream_transfer_task.hh b/streaming/stream_transfer_task.hh index c25a3f6f1d90..93aa3f34272a 100644 --- a/streaming/stream_transfer_task.hh +++ b/streaming/stream_transfer_task.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/supervisor.hh b/supervisor.hh index 40964f4f7f84..dd9ee510113b 100644 --- a/supervisor.hh +++ b/supervisor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/table_helper.cc b/table_helper.cc index c6230711c300..39733e9db231 100644 --- a/table_helper.cc +++ b/table_helper.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/table_helper.hh b/table_helper.hh index 4e9217c8e96b..e3c8ac932efc 100644 --- a/table_helper.hh +++ b/table_helper.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tasks/task_handler.cc b/tasks/task_handler.cc index eaf1e008d7ad..667b4db69d38 100644 --- a/tasks/task_handler.cc +++ b/tasks/task_handler.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/timeout_clock.hh" diff --git a/tasks/task_handler.hh b/tasks/task_handler.hh index 7aee8274307f..75dfad2b4000 100644 --- a/tasks/task_handler.hh +++ b/tasks/task_handler.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tasks/task_manager.cc b/tasks/task_manager.cc index 1da8f6b444bc..48cbb3004e72 100644 --- a/tasks/task_manager.cc +++ b/tasks/task_manager.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/tasks/task_manager.hh b/tasks/task_manager.hh index 3e4b8d80768a..33d04cf80b2f 100644 --- a/tasks/task_manager.hh +++ b/tasks/task_manager.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tasks/test_module.hh b/tasks/test_module.hh index 0c835f4d6a90..5eaf2f7e91e4 100644 --- a/tasks/test_module.hh +++ b/tasks/test_module.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef SCYLLA_BUILD_MODE_RELEASE diff --git a/tasks/types.hh b/tasks/types.hh index 20e76b1ed644..090ca740b747 100644 --- a/tasks/types.hh +++ b/tasks/types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tasks/virtual_task_hint.hh b/tasks/virtual_task_hint.hh index 661e0f074510..35ad7685c1a0 100644 --- a/tasks/virtual_task_hint.hh +++ b/tasks/virtual_task_hint.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test.py b/test.py index 9d652506df73..e6ac057ecc12 100755 --- a/test.py +++ b/test.py @@ -4,7 +4,7 @@ # Copyright (C) 2015-present ScyllaDB # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse import asyncio diff --git a/test/alternator/conftest.py b/test/alternator/conftest.py index 685141a3e468..7ea204015151 100644 --- a/test/alternator/conftest.py +++ b/test/alternator/conftest.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # This file contains "test fixtures", a pytest concept described in # https://docs.pytest.org/en/latest/fixture.html. diff --git a/test/alternator/test_authorization.py b/test/alternator/test_authorization.py index fad7f351f4e4..6e9b468f144a 100644 --- a/test/alternator/test_authorization.py +++ b/test/alternator/test_authorization.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for authorization diff --git a/test/alternator/test_backup.py b/test/alternator/test_backup.py index 9ace8e88e352..f9068cd3a43f 100644 --- a/test/alternator/test_backup.py +++ b/test/alternator/test_backup.py @@ -1,6 +1,6 @@ # Copyright 2022 ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the different types of backup and restore features in # DynamoDB. diff --git a/test/alternator/test_batch.py b/test/alternator/test_batch.py index 9dc49e9685de..366b70a48258 100644 --- a/test/alternator/test_batch.py +++ b/test/alternator/test_batch.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for batch operations - BatchWriteItem, BatchGetItem. # Note that various other tests in other files also use these operations, diff --git a/test/alternator/test_condition_expression.py b/test/alternator/test_condition_expression.py index 846c249c416e..11a00a8edb80 100644 --- a/test/alternator/test_condition_expression.py +++ b/test/alternator/test_condition_expression.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the ConditionExpression parameter which makes certain operations # (PutItem, UpdateItem and DeleteItem) conditional on the existing attribute diff --git a/test/alternator/test_cors.py b/test/alternator/test_cors.py index a445d0709653..393aa037736f 100644 --- a/test/alternator/test_cors.py +++ b/test/alternator/test_cors.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for CORS (Cross-origin resource sharing) protocol support. # If the request has the "Origin" header specifying where the script which diff --git a/test/alternator/test_cql_rbac.py b/test/alternator/test_cql_rbac.py index c2f96fe64fa0..aa66af3143c1 100644 --- a/test/alternator/test_cql_rbac.py +++ b/test/alternator/test_cql_rbac.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for how CQL's Role-Based Access Control (RBAC) commands - CREATE ROLE, # GRANT, REVOKE, etc., can be used on Alternator for authentication and for diff --git a/test/alternator/test_describe_endpoints.py b/test/alternator/test_describe_endpoints.py index 53dfb148047d..e526e1ad0bc9 100644 --- a/test/alternator/test_describe_endpoints.py +++ b/test/alternator/test_describe_endpoints.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Test for the DescribeEndpoints operation diff --git a/test/alternator/test_describe_table.py b/test/alternator/test_describe_table.py index 551578584493..df2660ad0098 100644 --- a/test/alternator/test_describe_table.py +++ b/test/alternator/test_describe_table.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the DescribeTable operation. # Some attributes used only by a specific major feature will be tested diff --git a/test/alternator/test_expected.py b/test/alternator/test_expected.py index e61aec636f1e..cc9bf01511ac 100644 --- a/test/alternator/test_expected.py +++ b/test/alternator/test_expected.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the "Expected" parameter used to make certain operations (PutItem, # UpdateItem and DeleteItem) conditional on the existing attribute values. diff --git a/test/alternator/test_filter_expression.py b/test/alternator/test_filter_expression.py index 2f2ded5c8d06..408818943499 100644 --- a/test/alternator/test_filter_expression.py +++ b/test/alternator/test_filter_expression.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the FilterExpression parameter of the Query and Scan operations. # FilterExpression is a newer version of the older "QueryFilter" and diff --git a/test/alternator/test_gsi.py b/test/alternator/test_gsi.py index 841dd09dd119..cfbbc769eb96 100644 --- a/test/alternator/test_gsi.py +++ b/test/alternator/test_gsi.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests of GSI (Global Secondary Indexes) # diff --git a/test/alternator/test_gsi_updatetable.py b/test/alternator/test_gsi_updatetable.py index 6666d4a069ed..d38bc5c3360d 100644 --- a/test/alternator/test_gsi_updatetable.py +++ b/test/alternator/test_gsi_updatetable.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests of for UpdateTable's GlobalSecondaryIndexUpdates option for modifying # the GSIs (Global Secondary Indexes) on an existing table: Adding a GSI to diff --git a/test/alternator/test_health.py b/test/alternator/test_health.py index 6606b490e5cd..3b5645c6f014 100644 --- a/test/alternator/test_health.py +++ b/test/alternator/test_health.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the health check diff --git a/test/alternator/test_item.py b/test/alternator/test_item.py index a268689f8981..0e7de526ed3e 100644 --- a/test/alternator/test_item.py +++ b/test/alternator/test_item.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the CRUD item operations: PutItem, GetItem, UpdateItem, DeleteItem diff --git a/test/alternator/test_key_condition_expression.py b/test/alternator/test_key_condition_expression.py index 466762d14378..575c24d4d597 100644 --- a/test/alternator/test_key_condition_expression.py +++ b/test/alternator/test_key_condition_expression.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the KeyConditionExpression parameter of the Query operation. # KeyConditionExpression is a newer version of the older "KeyConditions" diff --git a/test/alternator/test_key_conditions.py b/test/alternator/test_key_conditions.py index a134f44258bd..d812d98b618a 100644 --- a/test/alternator/test_key_conditions.py +++ b/test/alternator/test_key_conditions.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the KeyConditions parameter of the Query operation. # KeyConditions is the older version of the newer "KeyConditionExpression" diff --git a/test/alternator/test_limits.py b/test/alternator/test_limits.py index 746b968e2122..1e0c857d6f88 100644 --- a/test/alternator/test_limits.py +++ b/test/alternator/test_limits.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for various limits, which did not fit naturally into other test files diff --git a/test/alternator/test_lsi.py b/test/alternator/test_lsi.py index b8353ed35fed..c2d1ec9bef2f 100644 --- a/test/alternator/test_lsi.py +++ b/test/alternator/test_lsi.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests of LSI (Local Secondary Indexes) # diff --git a/test/alternator/test_manual_requests.py b/test/alternator/test_manual_requests.py index a59e51954751..13b6fc318847 100644 --- a/test/alternator/test_manual_requests.py +++ b/test/alternator/test_manual_requests.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for manual requests - not necessarily generated # by boto3, in order to allow non-validated input to get through diff --git a/test/alternator/test_metrics.py b/test/alternator/test_metrics.py index 0ddce7ad7bc6..9ba9008b0334 100644 --- a/test/alternator/test_metrics.py +++ b/test/alternator/test_metrics.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################## # Tests for Scylla's metrics (see docs/dev/metrics.md) for Alternator diff --git a/test/alternator/test_nested.py b/test/alternator/test_nested.py index 64977a08bd2c..aad5bd30ab1d 100644 --- a/test/alternator/test_nested.py +++ b/test/alternator/test_nested.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Test for operations on items with *nested* attributes. diff --git a/test/alternator/test_number.py b/test/alternator/test_number.py index 5fa4514b3eaa..611545c9e5b9 100644 --- a/test/alternator/test_number.py +++ b/test/alternator/test_number.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the number type. Numbers in DynamoDB have an unusual definition - # they are a floating-point type with 38 decimal digits of precision and diff --git a/test/alternator/test_projection_expression.py b/test/alternator/test_projection_expression.py index 9dd4db0af396..395d2f69bfd7 100644 --- a/test/alternator/test_projection_expression.py +++ b/test/alternator/test_projection_expression.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the various operations (GetItem, Query, Scan) with a # ProjectionExpression parameter. diff --git a/test/alternator/test_provisioned_throughput.py b/test/alternator/test_provisioned_throughput.py index 87d6dd3ab288..37976383e090 100644 --- a/test/alternator/test_provisioned_throughput.py +++ b/test/alternator/test_provisioned_throughput.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Test for ProvisionedThroughput # ProvisionedThroughput is part of a table definition diff --git a/test/alternator/test_query.py b/test/alternator/test_query.py index 2c9364e09dae..7812e31f9dce 100644 --- a/test/alternator/test_query.py +++ b/test/alternator/test_query.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the Query operation # Some of the Query features are tested in separate files: diff --git a/test/alternator/test_query_filter.py b/test/alternator/test_query_filter.py index 8e156c90c0ef..4cc4d1879377 100644 --- a/test/alternator/test_query_filter.py +++ b/test/alternator/test_query_filter.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the QueryFilter parameter of the Query operation. # QueryFilter is the older version of the newer FilterExpression syntax, diff --git a/test/alternator/test_returnconsumedcapacity.py b/test/alternator/test_returnconsumedcapacity.py index 77bc025d1f8e..4bb520d4eefc 100644 --- a/test/alternator/test_returnconsumedcapacity.py +++ b/test/alternator/test_returnconsumedcapacity.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the ReturnConsumedCapacity header diff --git a/test/alternator/test_returnvalues.py b/test/alternator/test_returnvalues.py index 68f87eeb1b28..f75805628b71 100644 --- a/test/alternator/test_returnvalues.py +++ b/test/alternator/test_returnvalues.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the ReturnValues parameter for the different update operations # (PutItem, UpdateItem, DeleteItem). diff --git a/test/alternator/test_scan.py b/test/alternator/test_scan.py index e0b377495f50..c25703eef6a5 100644 --- a/test/alternator/test_scan.py +++ b/test/alternator/test_scan.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the Scan operation diff --git a/test/alternator/test_scylla.py b/test/alternator/test_scylla.py index e20f55125734..463b6fd43a7c 100644 --- a/test/alternator/test_scylla.py +++ b/test/alternator/test_scylla.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # This file contains tests which check Scylla-specific features that do # not exist on AWS. So all these tests are skipped when running with "--aws". diff --git a/test/alternator/test_streams.py b/test/alternator/test_streams.py index da47396a3844..b9af0c6bd4e6 100644 --- a/test/alternator/test_streams.py +++ b/test/alternator/test_streams.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for stream operations: ListStreams, DescribeStream, GetShardIterator, # GetRecords. diff --git a/test/alternator/test_system_tables.py b/test/alternator/test_system_tables.py index 7f8e024c0ce0..73e8f8262c34 100644 --- a/test/alternator/test_system_tables.py +++ b/test/alternator/test_system_tables.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for accessing alternator-only system tables (from Scylla). diff --git a/test/alternator/test_table.py b/test/alternator/test_table.py index 57f1ddd8f848..489bd3d9c895 100644 --- a/test/alternator/test_table.py +++ b/test/alternator/test_table.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for basic table operations: CreateTable, DeleteTable, ListTables. # Also some basic tests for UpdateTable - although UpdateTable usually diff --git a/test/alternator/test_tag.py b/test/alternator/test_tag.py index 956c9677196a..54a9aab8c33c 100644 --- a/test/alternator/test_tag.py +++ b/test/alternator/test_tag.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for Tagging: # 1. TagResource - tagging a table with a (key, value) pair diff --git a/test/alternator/test_tracing.py b/test/alternator/test_tracing.py index aa0feb2dbe53..d8b67f0e5f44 100644 --- a/test/alternator/test_tracing.py +++ b/test/alternator/test_tracing.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################## # Tests for Scylla's "tracing" feature (see docs/dev/tracing.md) for Alternator diff --git a/test/alternator/test_ttl.py b/test/alternator/test_ttl.py index e09750c31dbe..901e100319f8 100644 --- a/test/alternator/test_ttl.py +++ b/test/alternator/test_ttl.py @@ -1,6 +1,6 @@ # Copyright 2021 ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the Time To Live (TTL) feature for item expiration. diff --git a/test/alternator/test_update_expression.py b/test/alternator/test_update_expression.py index 39fa04a6c3f8..ea3c74f1007b 100644 --- a/test/alternator/test_update_expression.py +++ b/test/alternator/test_update_expression.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for the UpdateItem operations with an UpdateExpression parameter diff --git a/test/alternator/util.py b/test/alternator/util.py index 9ad78bb0e40e..2ea2aa470e44 100644 --- a/test/alternator/util.py +++ b/test/alternator/util.py @@ -1,6 +1,6 @@ # Copyright 2019-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Various utility functions which are useful for multiple tests diff --git a/test/auth_cluster/conftest.py b/test/auth_cluster/conftest.py index 875c695d1438..8378e6c3211b 100644 --- a/test/auth_cluster/conftest.py +++ b/test/auth_cluster/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use diff --git a/test/auth_cluster/test_auth_no_quorum.py b/test/auth_cluster/test_auth_no_quorum.py index 6d1a8ae27d08..474d21206176 100644 --- a/test/auth_cluster/test_auth_no_quorum.py +++ b/test/auth_cluster/test_auth_no_quorum.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/auth_cluster/test_auth_raft_command_split.py b/test/auth_cluster/test_auth_raft_command_split.py index 92c54dcd6803..fbda9a928c4d 100644 --- a/test/auth_cluster/test_auth_raft_command_split.py +++ b/test/auth_cluster/test_auth_raft_command_split.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/auth_cluster/test_auth_v2_migration.py b/test/auth_cluster/test_auth_v2_migration.py index 9e5d3803ceda..fa8087f512ea 100644 --- a/test/auth_cluster/test_auth_v2_migration.py +++ b/test/auth_cluster/test_auth_v2_migration.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/auth_cluster/test_maintenance_socket.py b/test/auth_cluster/test_maintenance_socket.py index e51491b44ff5..eedfe85d3bfa 100644 --- a/test/auth_cluster/test_maintenance_socket.py +++ b/test/auth_cluster/test_maintenance_socket.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.auth import PlainTextAuthProvider diff --git a/test/auth_cluster/test_raft_service_levels.py b/test/auth_cluster/test_raft_service_levels.py index 08d9b5b36b16..2e715f1bc0ac 100644 --- a/test/auth_cluster/test_raft_service_levels.py +++ b/test/auth_cluster/test_raft_service_levels.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest import time diff --git a/test/boost/UUID_test.cc b/test/boost/UUID_test.cc index b7303137af42..d52c2ae0e76c 100644 --- a/test/boost/UUID_test.cc +++ b/test/boost/UUID_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/address_map_test.cc b/test/boost/address_map_test.cc index 8b3e30e2b4c3..f8b0fb6da4b1 100644 --- a/test/boost/address_map_test.cc +++ b/test/boost/address_map_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/aggregate_fcts_test.cc b/test/boost/aggregate_fcts_test.cc index e9a6a86de2b5..c58460297f89 100644 --- a/test/boost/aggregate_fcts_test.cc +++ b/test/boost/aggregate_fcts_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/allocation_strategy_test.cc b/test/boost/allocation_strategy_test.cc index e8c0186a17e3..252f316383b3 100644 --- a/test/boost/allocation_strategy_test.cc +++ b/test/boost/allocation_strategy_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/alternator_unit_test.cc b/test/boost/alternator_unit_test.cc index 5b5e1a7f4d2c..93e3f5864eed 100644 --- a/test/boost/alternator_unit_test.cc +++ b/test/boost/alternator_unit_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE alternator diff --git a/test/boost/anchorless_list_test.cc b/test/boost/anchorless_list_test.cc index 41ebb432bd58..f65c81fe4c3c 100644 --- a/test/boost/anchorless_list_test.cc +++ b/test/boost/anchorless_list_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/auth_passwords_test.cc b/test/boost/auth_passwords_test.cc index efc0aa7f1a2a..3307125184e1 100644 --- a/test/boost/auth_passwords_test.cc +++ b/test/boost/auth_passwords_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/auth_resource_test.cc b/test/boost/auth_resource_test.cc index 1d234c80d13d..9f7c836cecfa 100644 --- a/test/boost/auth_resource_test.cc +++ b/test/boost/auth_resource_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/auth_test.cc b/test/boost/auth_test.cc index 06ef23647722..5a48e721a476 100644 --- a/test/boost/auth_test.cc +++ b/test/boost/auth_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/aws_error_injection_test.cc b/test/boost/aws_error_injection_test.cc index 220707159737..5713553536a5 100644 --- a/test/boost/aws_error_injection_test.cc +++ b/test/boost/aws_error_injection_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/log.hh" diff --git a/test/boost/aws_errors_test.cc b/test/boost/aws_errors_test.cc index f423e58f034d..1190d1e10041 100644 --- a/test/boost/aws_errors_test.cc +++ b/test/boost/aws_errors_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE object_storage diff --git a/test/boost/batchlog_manager_test.cc b/test/boost/batchlog_manager_test.cc index 10418afc3e2f..fdd3be54585c 100644 --- a/test/boost/batchlog_manager_test.cc +++ b/test/boost/batchlog_manager_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/big_decimal_test.cc b/test/boost/big_decimal_test.cc index cb7a84933374..1f744a1edd72 100644 --- a/test/boost/big_decimal_test.cc +++ b/test/boost/big_decimal_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE big_decimal diff --git a/test/boost/bloom_filter_test.cc b/test/boost/bloom_filter_test.cc index a11d745e1a23..8cbfe877c87e 100644 --- a/test/boost/bloom_filter_test.cc +++ b/test/boost/bloom_filter_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/bptree_test.cc b/test/boost/bptree_test.cc index 139e85fe3e44..5f0f7066de9e 100644 --- a/test/boost/bptree_test.cc +++ b/test/boost/bptree_test.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/bptree_validation.hh b/test/boost/bptree_validation.hh index 9c3ee8866f93..7c0921f856dc 100644 --- a/test/boost/bptree_validation.hh +++ b/test/boost/bptree_validation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/broken_sstable_test.cc b/test/boost/broken_sstable_test.cc index 80a04f4e6ed5..5b220f8a0f6f 100644 --- a/test/boost/broken_sstable_test.cc +++ b/test/boost/broken_sstable_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/btree_test.cc b/test/boost/btree_test.cc index 0666f698ba11..f50d8641ba95 100644 --- a/test/boost/btree_test.cc +++ b/test/boost/btree_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/btree_validation.hh b/test/boost/btree_validation.hh index f7fa74b5acbc..6ce805142d31 100644 --- a/test/boost/btree_validation.hh +++ b/test/boost/btree_validation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/bytes_ostream_test.cc b/test/boost/bytes_ostream_test.cc index f7471ebcbeb4..cc65d89a2d40 100644 --- a/test/boost/bytes_ostream_test.cc +++ b/test/boost/bytes_ostream_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/cache_algorithm_test.cc b/test/boost/cache_algorithm_test.cc index 09a66a6ca78a..de97618e9d07 100644 --- a/test/boost/cache_algorithm_test.cc +++ b/test/boost/cache_algorithm_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/cache_mutation_reader_test.cc b/test/boost/cache_mutation_reader_test.cc index b6182fe3efe3..a7e2bfe3e51b 100644 --- a/test/boost/cache_mutation_reader_test.cc +++ b/test/boost/cache_mutation_reader_test.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/cached_file_test.cc b/test/boost/cached_file_test.cc index 57271d5a148a..8190fdd3091a 100644 --- a/test/boost/cached_file_test.cc +++ b/test/boost/cached_file_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/caching_options_test.cc b/test/boost/caching_options_test.cc index a1e1cf3d39fe..0b52bb983986 100644 --- a/test/boost/caching_options_test.cc +++ b/test/boost/caching_options_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/canonical_mutation_test.cc b/test/boost/canonical_mutation_test.cc index 46027f155024..fe5e731c10c2 100644 --- a/test/boost/canonical_mutation_test.cc +++ b/test/boost/canonical_mutation_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/cartesian_product_test.cc b/test/boost/cartesian_product_test.cc index bb57bc274119..4df23b6ee325 100644 --- a/test/boost/cartesian_product_test.cc +++ b/test/boost/cartesian_product_test.cc @@ -20,7 +20,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/castas_fcts_test.cc b/test/boost/castas_fcts_test.cc index 87535a43a669..c253e669f826 100644 --- a/test/boost/castas_fcts_test.cc +++ b/test/boost/castas_fcts_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/cdc_generation_test.cc b/test/boost/cdc_generation_test.cc index 7c4d38a5f358..f54a3d752e95 100644 --- a/test/boost/cdc_generation_test.cc +++ b/test/boost/cdc_generation_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/cdc_test.cc b/test/boost/cdc_test.cc index afe09c8cab38..79faa267f851 100644 --- a/test/boost/cdc_test.cc +++ b/test/boost/cdc_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/cell_locker_test.cc b/test/boost/cell_locker_test.cc index d495d098f85c..5d801ad02a64 100644 --- a/test/boost/cell_locker_test.cc +++ b/test/boost/cell_locker_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/checksum_utils_test.cc b/test/boost/checksum_utils_test.cc index 4c5f260770b5..5cdfe06b6fca 100644 --- a/test/boost/checksum_utils_test.cc +++ b/test/boost/checksum_utils_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/chunked_managed_vector_test.cc b/test/boost/chunked_managed_vector_test.cc index b3a41b607dd2..00e568532733 100644 --- a/test/boost/chunked_managed_vector_test.cc +++ b/test/boost/chunked_managed_vector_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/chunked_vector_test.cc b/test/boost/chunked_vector_test.cc index f2e5ba411caa..262e3e2a3202 100644 --- a/test/boost/chunked_vector_test.cc +++ b/test/boost/chunked_vector_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/clustering_ranges_walker_test.cc b/test/boost/clustering_ranges_walker_test.cc index 89758c558e53..4f129e0a59dd 100644 --- a/test/boost/clustering_ranges_walker_test.cc +++ b/test/boost/clustering_ranges_walker_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/collection_stress.hh b/test/boost/collection_stress.hh index 1cdad8f852be..efaa52eb18f3 100644 --- a/test/boost/collection_stress.hh +++ b/test/boost/collection_stress.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/column_mapping_test.cc b/test/boost/column_mapping_test.cc index 63c2f76edd98..4f0b2224c7d5 100644 --- a/test/boost/column_mapping_test.cc +++ b/test/boost/column_mapping_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/commitlog_cleanup_test.cc b/test/boost/commitlog_cleanup_test.cc index 145b91a331f9..6df6ed79acb4 100644 --- a/test/boost/commitlog_cleanup_test.cc +++ b/test/boost/commitlog_cleanup_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/commitlog_test.cc b/test/boost/commitlog_test.cc index da511558e07c..c1f3d983880b 100644 --- a/test/boost/commitlog_test.cc +++ b/test/boost/commitlog_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/compaction_group_test.cc b/test/boost/compaction_group_test.cc index 01460b57ac2f..c2c41fc3b3f3 100644 --- a/test/boost/compaction_group_test.cc +++ b/test/boost/compaction_group_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/compound_test.cc b/test/boost/compound_test.cc index 5caa97c969f1..be714476195a 100644 --- a/test/boost/compound_test.cc +++ b/test/boost/compound_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/compress_test.cc b/test/boost/compress_test.cc index ad8c4d65c6a6..d9784c089447 100644 --- a/test/boost/compress_test.cc +++ b/test/boost/compress_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/config_test.cc b/test/boost/config_test.cc index 4457d0909ee4..b99d0e5ebf87 100644 --- a/test/boost/config_test.cc +++ b/test/boost/config_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/continuous_data_consumer_test.cc b/test/boost/continuous_data_consumer_test.cc index a95034c1818a..523061089034 100644 --- a/test/boost/continuous_data_consumer_test.cc +++ b/test/boost/continuous_data_consumer_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "vint-serialization.hh" diff --git a/test/boost/counter_test.cc b/test/boost/counter_test.cc index ead470adf029..fe586134e6d9 100644 --- a/test/boost/counter_test.cc +++ b/test/boost/counter_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "counters.hh" diff --git a/test/boost/cql_auth_query_test.cc b/test/boost/cql_auth_query_test.cc index d27058a051ec..0c77d00f8161 100644 --- a/test/boost/cql_auth_query_test.cc +++ b/test/boost/cql_auth_query_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/cql_auth_syntax_test.cc b/test/boost/cql_auth_syntax_test.cc index e6cb6172c0fd..2c0314a4b3a8 100644 --- a/test/boost/cql_auth_syntax_test.cc +++ b/test/boost/cql_auth_syntax_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/cql_functions_test.cc b/test/boost/cql_functions_test.cc index df3577f0ef8a..4e78b0b6043f 100644 --- a/test/boost/cql_functions_test.cc +++ b/test/boost/cql_functions_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/cql_query_group_test.cc b/test/boost/cql_query_group_test.cc index 7f091489dfd1..8e6854ffa152 100644 --- a/test/boost/cql_query_group_test.cc +++ b/test/boost/cql_query_group_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/cql_query_large_test.cc b/test/boost/cql_query_large_test.cc index b15c3b74ae51..a8876b761692 100644 --- a/test/boost/cql_query_large_test.cc +++ b/test/boost/cql_query_large_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/cql_query_like_test.cc b/test/boost/cql_query_like_test.cc index 72810c57ef90..d83199faf9e9 100644 --- a/test/boost/cql_query_like_test.cc +++ b/test/boost/cql_query_like_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/cql_query_test.cc b/test/boost/cql_query_test.cc index 576c95daddeb..cbc7a0663498 100644 --- a/test/boost/cql_query_test.cc +++ b/test/boost/cql_query_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/crc_test.cc b/test/boost/crc_test.cc index 05f00406b092..11af0981272e 100644 --- a/test/boost/crc_test.cc +++ b/test/boost/crc_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/data_listeners_test.cc b/test/boost/data_listeners_test.cc index accfa872a32e..a9a5ae695188 100644 --- a/test/boost/data_listeners_test.cc +++ b/test/boost/data_listeners_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/database_test.cc b/test/boost/database_test.cc index e30c5ed2d72e..5933d2600c1b 100644 --- a/test/boost/database_test.cc +++ b/test/boost/database_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/dirty_memory_manager_test.cc b/test/boost/dirty_memory_manager_test.cc index 35c36e84d4e6..83ce99b0b983 100644 --- a/test/boost/dirty_memory_manager_test.cc +++ b/test/boost/dirty_memory_manager_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/double_decker_test.cc b/test/boost/double_decker_test.cc index 03d836e1c87e..511005484245 100644 --- a/test/boost/double_decker_test.cc +++ b/test/boost/double_decker_test.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/duration_test.cc b/test/boost/duration_test.cc index 387da40eb6c4..33cb6d9862eb 100644 --- a/test/boost/duration_test.cc +++ b/test/boost/duration_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/dynamic_bitset_test.cc b/test/boost/dynamic_bitset_test.cc index ff6f4e59630c..abaab94473a3 100644 --- a/test/boost/dynamic_bitset_test.cc +++ b/test/boost/dynamic_bitset_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/enum_option_test.cc b/test/boost/enum_option_test.cc index 933204f48c99..c3f7935d8949 100644 --- a/test/boost/enum_option_test.cc +++ b/test/boost/enum_option_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/enum_set_test.cc b/test/boost/enum_set_test.cc index 90a695601705..956aad84bb31 100644 --- a/test/boost/enum_set_test.cc +++ b/test/boost/enum_set_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/error_injection_test.cc b/test/boost/error_injection_test.cc index 382f47973aff..ba4c166902f1 100644 --- a/test/boost/error_injection_test.cc +++ b/test/boost/error_injection_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/estimated_histogram_test.cc b/test/boost/estimated_histogram_test.cc index 06390e66a167..2322df8590a2 100644 --- a/test/boost/estimated_histogram_test.cc +++ b/test/boost/estimated_histogram_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/exception_container_test.cc b/test/boost/exception_container_test.cc index 994689470172..e0c333933f42 100644 --- a/test/boost/exception_container_test.cc +++ b/test/boost/exception_container_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/exception_container.hh" diff --git a/test/boost/exceptions_fallback_test.cc b/test/boost/exceptions_fallback_test.cc index 9f4627a4a45b..eafc0eabcf57 100644 --- a/test/boost/exceptions_fallback_test.cc +++ b/test/boost/exceptions_fallback_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define NO_OPTIMIZED_EXCEPTION_HANDLING diff --git a/test/boost/exceptions_optimized_test.cc b/test/boost/exceptions_optimized_test.cc index 6db2b591483a..2301cf253fe0 100644 --- a/test/boost/exceptions_optimized_test.cc +++ b/test/boost/exceptions_optimized_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #if defined(NO_OPTIMIZED_EXCEPTION_HANDLING) diff --git a/test/boost/exceptions_test.inc.cc b/test/boost/exceptions_test.inc.cc index 4cdfa15d96c6..8d9d2338cdcc 100644 --- a/test/boost/exceptions_test.inc.cc +++ b/test/boost/exceptions_test.inc.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // Common definitions of test cases used in diff --git a/test/boost/expr_test.cc b/test/boost/expr_test.cc index 8f92b43a5df7..108a771f006e 100644 --- a/test/boost/expr_test.cc +++ b/test/boost/expr_test.cc @@ -1,5 +1,5 @@ // Copyright (C) 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #include "cql3/column_identifier.hh" #include "cql3/util.hh" diff --git a/test/boost/extensions_test.cc b/test/boost/extensions_test.cc index 6f1929b0aab6..24be9721621b 100644 --- a/test/boost/extensions_test.cc +++ b/test/boost/extensions_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/filtering_test.cc b/test/boost/filtering_test.cc index 364f02f4f7e3..a9f663362007 100644 --- a/test/boost/filtering_test.cc +++ b/test/boost/filtering_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/flush_queue_test.cc b/test/boost/flush_queue_test.cc index 294aee25bf47..d113aa575754 100644 --- a/test/boost/flush_queue_test.cc +++ b/test/boost/flush_queue_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/fragmented_temporary_buffer_test.cc b/test/boost/fragmented_temporary_buffer_test.cc index 1551e643c4d8..b7dce7668804 100644 --- a/test/boost/fragmented_temporary_buffer_test.cc +++ b/test/boost/fragmented_temporary_buffer_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/frozen_mutation_test.cc b/test/boost/frozen_mutation_test.cc index 990add53ce23..df36c22f03fa 100644 --- a/test/boost/frozen_mutation_test.cc +++ b/test/boost/frozen_mutation_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/generic_server_test.cc b/test/boost/generic_server_test.cc index 163613c418b1..2fac2a0d7c58 100644 --- a/test/boost/generic_server_test.cc +++ b/test/boost/generic_server_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/gossiping_property_file_snitch_test.cc b/test/boost/gossiping_property_file_snitch_test.cc index 547d57e78b6d..ce554459c6ff 100644 --- a/test/boost/gossiping_property_file_snitch_test.cc +++ b/test/boost/gossiping_property_file_snitch_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/group0_cmd_merge_test.cc b/test/boost/group0_cmd_merge_test.cc index 8099f7ce7329..3219d6e3118c 100644 --- a/test/boost/group0_cmd_merge_test.cc +++ b/test/boost/group0_cmd_merge_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/group0_test.cc b/test/boost/group0_test.cc index 8dc2cdd487e3..2b10ef48f3a6 100644 --- a/test/boost/group0_test.cc +++ b/test/boost/group0_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/hash_test.cc b/test/boost/hash_test.cc index d7faf67444be..13f8fee34088 100644 --- a/test/boost/hash_test.cc +++ b/test/boost/hash_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/hashers_test.cc b/test/boost/hashers_test.cc index 6d3e43b84603..c2294ca75109 100644 --- a/test/boost/hashers_test.cc +++ b/test/boost/hashers_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/timeout_clock.hh" diff --git a/test/boost/hint_test.cc b/test/boost/hint_test.cc index 9a19658b43c1..441647500632 100644 --- a/test/boost/hint_test.cc +++ b/test/boost/hint_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/idl_test.cc b/test/boost/idl_test.cc index 79756bec195a..7499b541c2fd 100644 --- a/test/boost/idl_test.cc +++ b/test/boost/idl_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/index_reader_test.cc b/test/boost/index_reader_test.cc index 9b8f7d80448b..bf71f9ec27d8 100644 --- a/test/boost/index_reader_test.cc +++ b/test/boost/index_reader_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/index_with_paging_test.cc b/test/boost/index_with_paging_test.cc index abb0aef4fec6..e397e51998fa 100644 --- a/test/boost/index_with_paging_test.cc +++ b/test/boost/index_with_paging_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/input_stream_test.cc b/test/boost/input_stream_test.cc index 2d77bcebb968..d62afbe1061d 100644 --- a/test/boost/input_stream_test.cc +++ b/test/boost/input_stream_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/intrusive_array_test.cc b/test/boost/intrusive_array_test.cc index fb349ec4e585..ebee1464051d 100644 --- a/test/boost/intrusive_array_test.cc +++ b/test/boost/intrusive_array_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/json_cql_query_test.cc b/test/boost/json_cql_query_test.cc index e1829971a831..884d9e08fde5 100644 --- a/test/boost/json_cql_query_test.cc +++ b/test/boost/json_cql_query_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/json_test.cc b/test/boost/json_test.cc index d4aa094a3e68..55b15c886ae6 100644 --- a/test/boost/json_test.cc +++ b/test/boost/json_test.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE json diff --git a/test/boost/keys_test.cc b/test/boost/keys_test.cc index ff5d5d3a1287..b51f39d227bb 100644 --- a/test/boost/keys_test.cc +++ b/test/boost/keys_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/large_paging_state_test.cc b/test/boost/large_paging_state_test.cc index 1f743d7701c9..57598cf58e98 100644 --- a/test/boost/large_paging_state_test.cc +++ b/test/boost/large_paging_state_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/like_matcher_test.cc b/test/boost/like_matcher_test.cc index 448af2eb238d..2045ca95bf9e 100644 --- a/test/boost/like_matcher_test.cc +++ b/test/boost/like_matcher_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/limiting_data_source_test.cc b/test/boost/limiting_data_source_test.cc index 2358f7d2834e..f9e389538203 100644 --- a/test/boost/limiting_data_source_test.cc +++ b/test/boost/limiting_data_source_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/boost/linearizing_input_stream_test.cc b/test/boost/linearizing_input_stream_test.cc index 6e9218ba522b..fd3660a2d5d2 100644 --- a/test/boost/linearizing_input_stream_test.cc +++ b/test/boost/linearizing_input_stream_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/lister_test.cc b/test/boost/lister_test.cc index d0bdd7cb5366..6a2594839e36 100644 --- a/test/boost/lister_test.cc +++ b/test/boost/lister_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/loading_cache_test.cc b/test/boost/loading_cache_test.cc index 61ee7ba75b3a..d99c28167df5 100644 --- a/test/boost/loading_cache_test.cc +++ b/test/boost/loading_cache_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc index f3784bb17353..c41ded826c89 100644 --- a/test/boost/locator_topology_test.cc +++ b/test/boost/locator_topology_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/boost/log_heap_test.cc b/test/boost/log_heap_test.cc index baa6f8d93c47..75fef46897ef 100644 --- a/test/boost/log_heap_test.cc +++ b/test/boost/log_heap_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/logalloc_test.cc b/test/boost/logalloc_test.cc index 04f8be057bde..1458151c8ab6 100644 --- a/test/boost/logalloc_test.cc +++ b/test/boost/logalloc_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/managed_bytes_test.cc b/test/boost/managed_bytes_test.cc index 361d3777c813..6aacd176ed02 100644 --- a/test/boost/managed_bytes_test.cc +++ b/test/boost/managed_bytes_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/managed_vector_test.cc b/test/boost/managed_vector_test.cc index c5da2d937847..071783012946 100644 --- a/test/boost/managed_vector_test.cc +++ b/test/boost/managed_vector_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/map_difference_test.cc b/test/boost/map_difference_test.cc index 6171e9d99684..17c8a159e1e7 100644 --- a/test/boost/map_difference_test.cc +++ b/test/boost/map_difference_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/memtable_test.cc b/test/boost/memtable_test.cc index 50f5975cc9f4..9a65682f4b68 100644 --- a/test/boost/memtable_test.cc +++ b/test/boost/memtable_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/multishard_combining_reader_as_mutation_source_test.cc b/test/boost/multishard_combining_reader_as_mutation_source_test.cc index 0d6e03daf3ad..f5fe28faee54 100644 --- a/test/boost/multishard_combining_reader_as_mutation_source_test.cc +++ b/test/boost/multishard_combining_reader_as_mutation_source_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/multishard_mutation_query_test.cc b/test/boost/multishard_mutation_query_test.cc index eb6f44173c32..c17a0e141aaa 100644 --- a/test/boost/multishard_mutation_query_test.cc +++ b/test/boost/multishard_mutation_query_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/boost/murmur_hash_test.cc b/test/boost/murmur_hash_test.cc index d3b1dc4b7df2..5fd61e952313 100644 --- a/test/boost/murmur_hash_test.cc +++ b/test/boost/murmur_hash_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/mutation_fragment_test.cc b/test/boost/mutation_fragment_test.cc index 0eb9632b519a..859ebefa44f1 100644 --- a/test/boost/mutation_fragment_test.cc +++ b/test/boost/mutation_fragment_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/mutation_query_test.cc b/test/boost/mutation_query_test.cc index 9f4c1f70fb1b..e8c0cfc23040 100644 --- a/test/boost/mutation_query_test.cc +++ b/test/boost/mutation_query_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/mutation_reader_another_test.cc b/test/boost/mutation_reader_another_test.cc index 8bffc422d164..1baf80c5f651 100644 --- a/test/boost/mutation_reader_another_test.cc +++ b/test/boost/mutation_reader_another_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/mutation_reader_test.cc b/test/boost/mutation_reader_test.cc index eab30fd5fdf3..e9280e4d4120 100644 --- a/test/boost/mutation_reader_test.cc +++ b/test/boost/mutation_reader_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/mutation_test.cc b/test/boost/mutation_test.cc index 7a2b40997b20..9367ba422c30 100644 --- a/test/boost/mutation_test.cc +++ b/test/boost/mutation_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/mutation_writer_test.cc b/test/boost/mutation_writer_test.cc index d9d1e4fcfd5b..0871f092b81e 100644 --- a/test/boost/mutation_writer_test.cc +++ b/test/boost/mutation_writer_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/mvcc_test.cc b/test/boost/mvcc_test.cc index d1f094e289b8..6378204d0d51 100644 --- a/test/boost/mvcc_test.cc +++ b/test/boost/mvcc_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index a33ac754ef8e..e2ff9a1aef1a 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/nonwrapping_interval_test.cc b/test/boost/nonwrapping_interval_test.cc index ec2e4c097774..fc9f5b1c37c9 100644 --- a/test/boost/nonwrapping_interval_test.cc +++ b/test/boost/nonwrapping_interval_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/observable_test.cc b/test/boost/observable_test.cc index 44dc4d735560..b95cc4d475ba 100644 --- a/test/boost/observable_test.cc +++ b/test/boost/observable_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE observable_test diff --git a/test/boost/partitioner_test.cc b/test/boost/partitioner_test.cc index 9b4419223d0e..8ca1b0f8a0d8 100644 --- a/test/boost/partitioner_test.cc +++ b/test/boost/partitioner_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/pretty_printers_test.cc b/test/boost/pretty_printers_test.cc index 1715b8bcd1d9..795e76cbb4e0 100644 --- a/test/boost/pretty_printers_test.cc +++ b/test/boost/pretty_printers_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE utils diff --git a/test/boost/querier_cache_test.cc b/test/boost/querier_cache_test.cc index f7d66897e5b0..92a27970acd2 100644 --- a/test/boost/querier_cache_test.cc +++ b/test/boost/querier_cache_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/query_processor_test.cc b/test/boost/query_processor_test.cc index 6a56f90c263b..c9a7b0a7b160 100644 --- a/test/boost/query_processor_test.cc +++ b/test/boost/query_processor_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/radix_tree_printer.hh b/test/boost/radix_tree_printer.hh index f08eeb6b3082..64ea957b74a1 100644 --- a/test/boost/radix_tree_printer.hh +++ b/test/boost/radix_tree_printer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/radix_tree_test.cc b/test/boost/radix_tree_test.cc index daea6753dcff..8f07b8ec182d 100644 --- a/test/boost/radix_tree_test.cc +++ b/test/boost/radix_tree_test.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/range_assert.hh b/test/boost/range_assert.hh index 4fc6f2da0045..1926f889b16e 100644 --- a/test/boost/range_assert.hh +++ b/test/boost/range_assert.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/range_tombstone_list_assertions.hh b/test/boost/range_tombstone_list_assertions.hh index 7c279e711b69..ab4fde429d17 100644 --- a/test/boost/range_tombstone_list_assertions.hh +++ b/test/boost/range_tombstone_list_assertions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/range_tombstone_list_test.cc b/test/boost/range_tombstone_list_test.cc index dfc99c9ded64..58f989013cee 100644 --- a/test/boost/range_tombstone_list_test.cc +++ b/test/boost/range_tombstone_list_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/rate_limiter_test.cc b/test/boost/rate_limiter_test.cc index ccb789052ed7..0971a5785e6d 100644 --- a/test/boost/rate_limiter_test.cc +++ b/test/boost/rate_limiter_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/reader_concurrency_semaphore_test.cc b/test/boost/reader_concurrency_semaphore_test.cc index 8b52bb35549b..2681f298e7cf 100644 --- a/test/boost/reader_concurrency_semaphore_test.cc +++ b/test/boost/reader_concurrency_semaphore_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/boost/recent_entries_map_test.cc b/test/boost/recent_entries_map_test.cc index 379b8c949720..99dd7cd1a027 100644 --- a/test/boost/recent_entries_map_test.cc +++ b/test/boost/recent_entries_map_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/repair_test.cc b/test/boost/repair_test.cc index da4186d6e247..521cd925313b 100644 --- a/test/boost/repair_test.cc +++ b/test/boost/repair_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "replica/memtable.hh" diff --git a/test/boost/restrictions_test.cc b/test/boost/restrictions_test.cc index a632f20746de..f075b29d43db 100644 --- a/test/boost/restrictions_test.cc +++ b/test/boost/restrictions_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/result_utils_test.cc b/test/boost/result_utils_test.cc index 2a0cefebc1b1..59ff4009e222 100644 --- a/test/boost/result_utils_test.cc +++ b/test/boost/result_utils_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/reusable_buffer_test.cc b/test/boost/reusable_buffer_test.cc index b54f04c2448f..a5a8f1fbf581 100644 --- a/test/boost/reusable_buffer_test.cc +++ b/test/boost/reusable_buffer_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/random_utils.hh" diff --git a/test/boost/role_manager_test.cc b/test/boost/role_manager_test.cc index bcf8fb9fa892..e6052f21cd7e 100644 --- a/test/boost/role_manager_test.cc +++ b/test/boost/role_manager_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/row_cache_test.cc b/test/boost/row_cache_test.cc index 109996666c00..365aacfb78ab 100644 --- a/test/boost/row_cache_test.cc +++ b/test/boost/row_cache_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/rust_test.cc b/test/boost/rust_test.cc index 83e59ed12b41..285a2e98670f 100644 --- a/test/boost/rust_test.cc +++ b/test/boost/rust_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/s3_test.cc b/test/boost/s3_test.cc index 55ea71f753fb..89c20446cc9c 100644 --- a/test/boost/s3_test.cc +++ b/test/boost/s3_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/schema_change_test.cc b/test/boost/schema_change_test.cc index c3caf3b38781..10a8394e3417 100644 --- a/test/boost/schema_change_test.cc +++ b/test/boost/schema_change_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/schema_changes_test.cc b/test/boost/schema_changes_test.cc index 385fcef6fa40..ec56578215c7 100644 --- a/test/boost/schema_changes_test.cc +++ b/test/boost/schema_changes_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/schema_loader_test.cc b/test/boost/schema_loader_test.cc index 21b0ffd57cb1..19344bd6f161 100644 --- a/test/boost/schema_loader_test.cc +++ b/test/boost/schema_loader_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/log.hh" diff --git a/test/boost/schema_registry_test.cc b/test/boost/schema_registry_test.cc index 153734ab0b33..ead71c7efa42 100644 --- a/test/boost/schema_registry_test.cc +++ b/test/boost/schema_registry_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/secondary_index_test.cc b/test/boost/secondary_index_test.cc index 1700c5828713..0ac2cd8b51cd 100644 --- a/test/boost/secondary_index_test.cc +++ b/test/boost/secondary_index_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/serialization_test.cc b/test/boost/serialization_test.cc index b45b870ac404..285148026b8a 100644 --- a/test/boost/serialization_test.cc +++ b/test/boost/serialization_test.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE test-serialization diff --git a/test/boost/serialized_action_test.cc b/test/boost/serialized_action_test.cc index 5c243700c23a..664ec9cefe5a 100644 --- a/test/boost/serialized_action_test.cc +++ b/test/boost/serialized_action_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/service_level_controller_test.cc b/test/boost/service_level_controller_test.cc index ea3469bdd905..cbd361cf09f6 100644 --- a/test/boost/service_level_controller_test.cc +++ b/test/boost/service_level_controller_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/sessions_test.cc b/test/boost/sessions_test.cc index a7bf2347dba9..d80a4b185a4f 100644 --- a/test/boost/sessions_test.cc +++ b/test/boost/sessions_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/small_vector_test.cc b/test/boost/small_vector_test.cc index 9cb96c03289c..7ddcaa6c58a5 100644 --- a/test/boost/small_vector_test.cc +++ b/test/boost/small_vector_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE small_vector diff --git a/test/boost/snitch_reset_test.cc b/test/boost/snitch_reset_test.cc index 8b9d5ccfa0e3..3dbdeaa8c2b0 100644 --- a/test/boost/snitch_reset_test.cc +++ b/test/boost/snitch_reset_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/sorting_test.cc b/test/boost/sorting_test.cc index dd198046c1a0..ba6cd04d06df 100644 --- a/test/boost/sorting_test.cc +++ b/test/boost/sorting_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/sstable_3_x_test.cc b/test/boost/sstable_3_x_test.cc index 3ec42995ec8c..9dd3dbe30464 100644 --- a/test/boost/sstable_3_x_test.cc +++ b/test/boost/sstable_3_x_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/boost/sstable_compaction_test.cc b/test/boost/sstable_compaction_test.cc index ad9f4dc3a712..ffc0877780f2 100644 --- a/test/boost/sstable_compaction_test.cc +++ b/test/boost/sstable_compaction_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/sstable_conforms_to_mutation_source_test.cc b/test/boost/sstable_conforms_to_mutation_source_test.cc index 79598d5c2b3a..15e746d6bdeb 100644 --- a/test/boost/sstable_conforms_to_mutation_source_test.cc +++ b/test/boost/sstable_conforms_to_mutation_source_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/sstable_datafile_test.cc b/test/boost/sstable_datafile_test.cc index 254f655b332d..ac4478cbc8e5 100644 --- a/test/boost/sstable_datafile_test.cc +++ b/test/boost/sstable_datafile_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/boost/sstable_directory_test.cc b/test/boost/sstable_directory_test.cc index 5a390ba48d1c..5c9596e08b78 100644 --- a/test/boost/sstable_directory_test.cc +++ b/test/boost/sstable_directory_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/sstable_generation_test.cc b/test/boost/sstable_generation_test.cc index 4ef67f1e1d0c..5c59227532aa 100644 --- a/test/boost/sstable_generation_test.cc +++ b/test/boost/sstable_generation_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE sstable-generation diff --git a/test/boost/sstable_move_test.cc b/test/boost/sstable_move_test.cc index dd183759cd43..e4cc89202a49 100644 --- a/test/boost/sstable_move_test.cc +++ b/test/boost/sstable_move_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/sstable_mutation_test.cc b/test/boost/sstable_mutation_test.cc index 05c3d430c5b4..4ac3f433de95 100644 --- a/test/boost/sstable_mutation_test.cc +++ b/test/boost/sstable_mutation_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/sstable_partition_index_cache_test.cc b/test/boost/sstable_partition_index_cache_test.cc index 35c17f2d0ad0..08e48e6e138d 100644 --- a/test/boost/sstable_partition_index_cache_test.cc +++ b/test/boost/sstable_partition_index_cache_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/sstable_set_test.cc b/test/boost/sstable_set_test.cc index aa7fb3a86671..92cadb6cfce4 100644 --- a/test/boost/sstable_set_test.cc +++ b/test/boost/sstable_set_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/sstable_test.cc b/test/boost/sstable_test.cc index ba6bcd2c0751..83ca9cc765c6 100644 --- a/test/boost/sstable_test.cc +++ b/test/boost/sstable_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/sstable_test.hh b/test/boost/sstable_test.hh index 3dc0eb4d3935..cded038902f4 100644 --- a/test/boost/sstable_test.hh +++ b/test/boost/sstable_test.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/stall_free_test.cc b/test/boost/stall_free_test.cc index 8b98fcbd4fbe..32623a864f89 100644 --- a/test/boost/stall_free_test.cc +++ b/test/boost/stall_free_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/statement_restrictions_test.cc b/test/boost/statement_restrictions_test.cc index 6826167bc011..8f6c2a2df071 100644 --- a/test/boost/statement_restrictions_test.cc +++ b/test/boost/statement_restrictions_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/storage_proxy_test.cc b/test/boost/storage_proxy_test.cc index 64adabfcc229..eb53af5bc073 100644 --- a/test/boost/storage_proxy_test.cc +++ b/test/boost/storage_proxy_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/string_format_test.cc b/test/boost/string_format_test.cc index 9f8aa56d85e3..b382a9b4c80c 100644 --- a/test/boost/string_format_test.cc +++ b/test/boost/string_format_test.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE string_format diff --git a/test/boost/summary_test.cc b/test/boost/summary_test.cc index 741a85fabae2..f6c19509a0ad 100644 --- a/test/boost/summary_test.cc +++ b/test/boost/summary_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index df6cec348025..9e0aaa13bccf 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/tagged_integer_test.cc b/test/boost/tagged_integer_test.cc index 0b07b2877afd..4d3557a58b1f 100644 --- a/test/boost/tagged_integer_test.cc +++ b/test/boost/tagged_integer_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/token_metadata_test.cc b/test/boost/token_metadata_test.cc index c35b54e9a681..6f62ad4a1575 100644 --- a/test/boost/token_metadata_test.cc +++ b/test/boost/token_metadata_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/top_k_test.cc b/test/boost/top_k_test.cc index 27a8304fc609..fe689e1cfdfd 100644 --- a/test/boost/top_k_test.cc +++ b/test/boost/top_k_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/total_order_check.hh b/test/boost/total_order_check.hh index d6d8b52ddf10..83123902319c 100644 --- a/test/boost/total_order_check.hh +++ b/test/boost/total_order_check.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/tracing_test.cc b/test/boost/tracing_test.cc index d0d14bacfb89..e27b75c558eb 100644 --- a/test/boost/tracing_test.cc +++ b/test/boost/tracing_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/transport_test.cc b/test/boost/transport_test.cc index a58b9764fc4a..8b20bc7004ec 100644 --- a/test/boost/transport_test.cc +++ b/test/boost/transport_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/tree_test_key.hh b/test/boost/tree_test_key.hh index 8d886e560339..3bb156a77186 100644 --- a/test/boost/tree_test_key.hh +++ b/test/boost/tree_test_key.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/boost/types_test.cc b/test/boost/types_test.cc index 130332ea5247..05b3c8577338 100644 --- a/test/boost/types_test.cc +++ b/test/boost/types_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/user_function_test.cc b/test/boost/user_function_test.cc index 4be5cdd0fc47..b1f7adf931b0 100644 --- a/test/boost/user_function_test.cc +++ b/test/boost/user_function_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/user_types_test.cc b/test/boost/user_types_test.cc index 9968c63785fb..6ebc79ad088a 100644 --- a/test/boost/user_types_test.cc +++ b/test/boost/user_types_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/utf8_test.cc b/test/boost/utf8_test.cc index d0ecdd0bf8eb..9c553e49495a 100644 --- a/test/boost/utf8_test.cc +++ b/test/boost/utf8_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/view_build_test.cc b/test/boost/view_build_test.cc index 48b08daaa39e..db53e8d31bda 100644 --- a/test/boost/view_build_test.cc +++ b/test/boost/view_build_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/view_complex_test.cc b/test/boost/view_complex_test.cc index 913c7e4b951c..d09a134c2b76 100644 --- a/test/boost/view_complex_test.cc +++ b/test/boost/view_complex_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/view_schema_ckey_test.cc b/test/boost/view_schema_ckey_test.cc index e67cc425fb90..9d6c105f3476 100644 --- a/test/boost/view_schema_ckey_test.cc +++ b/test/boost/view_schema_ckey_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/view_schema_pkey_test.cc b/test/boost/view_schema_pkey_test.cc index 6ad60528b018..5a9aa8f8e96a 100644 --- a/test/boost/view_schema_pkey_test.cc +++ b/test/boost/view_schema_pkey_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/view_schema_test.cc b/test/boost/view_schema_test.cc index 33a98328eddd..c7ac134dad63 100644 --- a/test/boost/view_schema_test.cc +++ b/test/boost/view_schema_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/boost/vint_serialization_test.cc b/test/boost/vint_serialization_test.cc index ac2e520273b1..b551e8b1e011 100644 --- a/test/boost/vint_serialization_test.cc +++ b/test/boost/vint_serialization_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/boost/virtual_reader_test.cc b/test/boost/virtual_reader_test.cc index d6fb66687116..0dfb4e03b1db 100644 --- a/test/boost/virtual_reader_test.cc +++ b/test/boost/virtual_reader_test.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/virtual_table_mutation_source_test.cc b/test/boost/virtual_table_mutation_source_test.cc index c924925a9f31..142b37bc7223 100644 --- a/test/boost/virtual_table_mutation_source_test.cc +++ b/test/boost/virtual_table_mutation_source_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/boost/virtual_table_test.cc b/test/boost/virtual_table_test.cc index 655b3986a6d6..160c1a8427ca 100644 --- a/test/boost/virtual_table_test.cc +++ b/test/boost/virtual_table_test.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" diff --git a/test/boost/wasm_alloc_test.cc b/test/boost/wasm_alloc_test.cc index 27914f5cee57..e0daeac30b6d 100644 --- a/test/boost/wasm_alloc_test.cc +++ b/test/boost/wasm_alloc_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "lang/wasm.hh" diff --git a/test/boost/wasm_test.cc b/test/boost/wasm_test.cc index 4f26bc4abc0e..50f36737e922 100644 --- a/test/boost/wasm_test.cc +++ b/test/boost/wasm_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "lang/wasm.hh" diff --git a/test/boost/wrapping_interval_test.cc b/test/boost/wrapping_interval_test.cc index 7c197e2d5b33..b6b120dc7e9f 100644 --- a/test/boost/wrapping_interval_test.cc +++ b/test/boost/wrapping_interval_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE core diff --git a/test/broadcast_tables/conftest.py b/test/broadcast_tables/conftest.py index 4c693f495714..c67c52d6fc75 100644 --- a/test/broadcast_tables/conftest.py +++ b/test/broadcast_tables/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.cql_repl.conftest import * diff --git a/test/broadcast_tables/test_broadcast_tables.py b/test/broadcast_tables/test_broadcast_tables.py index 729013b2f879..39ac130cfec8 100644 --- a/test/broadcast_tables/test_broadcast_tables.py +++ b/test/broadcast_tables/test_broadcast_tables.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import random import string diff --git a/test/cql/cassandra_batch_test.cql b/test/cql/cassandra_batch_test.cql index 5279a1aa0d9e..63c3260a3c97 100644 --- a/test/cql/cassandra_batch_test.cql +++ b/test/cql/cassandra_batch_test.cql @@ -9,7 +9,7 @@ -- -- Modified by ScyllaDB -- --- SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +-- SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) -- setup CREATE KEYSPACE k WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}; diff --git a/test/cql/cassandra_batch_test.result b/test/cql/cassandra_batch_test.result index cdf7298d471b..3849c8ab16a8 100644 --- a/test/cql/cassandra_batch_test.result +++ b/test/cql/cassandra_batch_test.result @@ -9,7 +9,7 @@ > -- > -- Modified by ScyllaDB > -- -> -- SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +> -- SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) > > -- setup > CREATE KEYSPACE k WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}; diff --git a/test/cql/cassandra_cql_test.cql b/test/cql/cassandra_cql_test.cql index 5bd7077a726d..feaff2778397 100644 --- a/test/cql/cassandra_cql_test.cql +++ b/test/cql/cassandra_cql_test.cql @@ -9,7 +9,7 @@ -- -- Modified by ScyllaDB -- --- SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +-- SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) -- setup CREATE KEYSPACE ks1 WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1} diff --git a/test/cql/cassandra_cql_test.result b/test/cql/cassandra_cql_test.result index 7dc47e510448..3319bbc4815d 100644 --- a/test/cql/cassandra_cql_test.result +++ b/test/cql/cassandra_cql_test.result @@ -9,7 +9,7 @@ > -- > -- Modified by ScyllaDB > -- -> -- SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) +> -- SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) > > -- setup > CREATE KEYSPACE ks1 WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1} diff --git a/test/cqlpy/cassandra_tests/porting.py b/test/cqlpy/cassandra_tests/porting.py index 9794b4f96455..b7d885561fe7 100644 --- a/test/cqlpy/cassandra_tests/porting.py +++ b/test/cqlpy/cassandra_tests/porting.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # This file contains various utility functions which are useful for porting diff --git a/test/cqlpy/conftest.py b/test/cqlpy/conftest.py index f8ff7393b3ed..f862d157cec4 100644 --- a/test/cqlpy/conftest.py +++ b/test/cqlpy/conftest.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use. A "fixture" is some diff --git a/test/cqlpy/fetch_scylla.py b/test/cqlpy/fetch_scylla.py index eecc06184f13..9de54dc1e25d 100755 --- a/test/cqlpy/fetch_scylla.py +++ b/test/cqlpy/fetch_scylla.py @@ -1,6 +1,6 @@ #!/usr/bin/python # Copyright (C) 2024-present ScyllaDB -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # Fetch from ScyllaDB's S3 bucket (downloads.scylladb.com) a pre-compiled # version of Scylla for a desired release, ready to be run by tests (run.py). diff --git a/test/cqlpy/nodetool.py b/test/cqlpy/nodetool.py index d97bc4fdb7f7..d35a92c75cbe 100644 --- a/test/cqlpy/nodetool.py +++ b/test/cqlpy/nodetool.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ################################################################## # This file provides a few nodetool-compatible commands that may be useful diff --git a/test/cqlpy/rest_api.py b/test/cqlpy/rest_api.py index b8afa118f4dc..c911c998ff16 100644 --- a/test/cqlpy/rest_api.py +++ b/test/cqlpy/rest_api.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # This file provides utility to REST API requests. diff --git a/test/cqlpy/test_aggregate.py b/test/cqlpy/test_aggregate.py index 79ed4491e71f..87f5f501b19f 100644 --- a/test/cqlpy/test_aggregate.py +++ b/test/cqlpy/test_aggregate.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests aggregation functions COUNT(), MIN(), MAX(), SUM() diff --git a/test/cqlpy/test_allow_filtering.py b/test/cqlpy/test_allow_filtering.py index 9b6bd88a3b25..d4ded488e2b7 100644 --- a/test/cqlpy/test_allow_filtering.py +++ b/test/cqlpy/test_allow_filtering.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the "ALLOW FILTERING" specification on SELECT requests. diff --git a/test/cqlpy/test_alter_table.py b/test/cqlpy/test_alter_table.py index 330fbca5eaeb..77a0967bf4d8 100644 --- a/test/cqlpy/test_alter_table.py +++ b/test/cqlpy/test_alter_table.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for alter table statement diff --git a/test/cqlpy/test_bad_grammar.py b/test/cqlpy/test_bad_grammar.py index a166fd412754..0a53a4b54cd1 100644 --- a/test/cqlpy/test_bad_grammar.py +++ b/test/cqlpy/test_bad_grammar.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # A collection of tests for grammar that should be rejected, but wasn't at # some point in the past. diff --git a/test/cqlpy/test_batch.py b/test/cqlpy/test_batch.py index 300bd65c5c45..ad9ce6cfbb85 100644 --- a/test/cqlpy/test_batch.py +++ b/test/cqlpy/test_batch.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for batch operations ############################################################################# diff --git a/test/cqlpy/test_bloom_filter.py b/test/cqlpy/test_bloom_filter.py index 5d9cfa922978..fec0e78fea92 100644 --- a/test/cqlpy/test_bloom_filter.py +++ b/test/cqlpy/test_bloom_filter.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest from . import nodetool diff --git a/test/cqlpy/test_cast.py b/test/cqlpy/test_cast.py index 982c6285c3fb..6593f8c456f4 100644 --- a/test/cqlpy/test_cast.py +++ b/test/cqlpy/test_cast.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################### # Tests for CQL casting, e.g `blob_column = (blob)(int)123` diff --git a/test/cqlpy/test_cast_data.py b/test/cqlpy/test_cast_data.py index c5ab231bc1ce..f1ca1a9862f5 100644 --- a/test/cqlpy/test_cast_data.py +++ b/test/cqlpy/test_cast_data.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################### # Tests for data casts, e.g `SELECT CAST (... AS int) FROM tbl` diff --git a/test/cqlpy/test_cdc.py b/test/cqlpy/test_cdc.py index 1385b38a62ad..c7a5a74d164c 100644 --- a/test/cqlpy/test_cdc.py +++ b/test/cqlpy/test_cdc.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 from cassandra.cluster import ConsistencyLevel from cassandra.query import SimpleStatement diff --git a/test/cqlpy/test_clustering_order.py b/test/cqlpy/test_clustering_order.py index 2dbe9016a8e5..8c3a84f9bc21 100644 --- a/test/cqlpy/test_clustering_order.py +++ b/test/cqlpy/test_clustering_order.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for clustering key ordering, namely the WITH CLUSTERING ORDER BY diff --git a/test/cqlpy/test_compaction.py b/test/cqlpy/test_compaction.py index eea391ba4fc9..fdcf7899fd1e 100644 --- a/test/cqlpy/test_compaction.py +++ b/test/cqlpy/test_compaction.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest import requests diff --git a/test/cqlpy/test_compaction_strategy_validation.py b/test/cqlpy/test_compaction_strategy_validation.py index cc444fcfa3a2..e11a20ab3e84 100644 --- a/test/cqlpy/test_compaction_strategy_validation.py +++ b/test/cqlpy/test_compaction_strategy_validation.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for compaction strategy validation diff --git a/test/cqlpy/test_counter.py b/test/cqlpy/test_counter.py index ed6e17c2ee16..37aeaccadd9a 100644 --- a/test/cqlpy/test_counter.py +++ b/test/cqlpy/test_counter.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################### # Tests for various operations on COUNTER columns. diff --git a/test/cqlpy/test_describe.py b/test/cqlpy/test_describe.py index d2191a941922..618c90a0aac2 100644 --- a/test/cqlpy/test_describe.py +++ b/test/cqlpy/test_describe.py @@ -1,7 +1,7 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################### # Tests for server-side describe diff --git a/test/cqlpy/test_distinct.py b/test/cqlpy/test_distinct.py index 7f3b43d01257..ea07e3c8356c 100644 --- a/test/cqlpy/test_distinct.py +++ b/test/cqlpy/test_distinct.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the SELECT DISTINCT feature diff --git a/test/cqlpy/test_empty.py b/test/cqlpy/test_empty.py index 52de4d33084c..3cdcf42fda81 100644 --- a/test/cqlpy/test_empty.py +++ b/test/cqlpy/test_empty.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for empty values (especially, but not just, empty strings) diff --git a/test/cqlpy/test_filtering.py b/test/cqlpy/test_filtering.py index 5b6bf273a559..4bc70fbc0aed 100644 --- a/test/cqlpy/test_filtering.py +++ b/test/cqlpy/test_filtering.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the SELECT requests with various filtering expressions. diff --git a/test/cqlpy/test_frozen_collection.py b/test/cqlpy/test_frozen_collection.py index 272aedb833c9..4d84ebcfb746 100644 --- a/test/cqlpy/test_frozen_collection.py +++ b/test/cqlpy/test_frozen_collection.py @@ -1,6 +1,6 @@ # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for the handling of frozen collections. Note that Cassandra diff --git a/test/cqlpy/test_group_by.py b/test/cqlpy/test_group_by.py index d96352039659..dce31ba56db4 100644 --- a/test/cqlpy/test_group_by.py +++ b/test/cqlpy/test_group_by.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for SELECT's GROUP BY feature diff --git a/test/cqlpy/test_json.py b/test/cqlpy/test_json.py index 34af9e61bf15..31fda8c9aa32 100644 --- a/test/cqlpy/test_json.py +++ b/test/cqlpy/test_json.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for JSON support in Scylla. Note that Cassandra also had diff --git a/test/cqlpy/test_key_length.py b/test/cqlpy/test_key_length.py index 403739674156..5ae0748f8bf8 100644 --- a/test/cqlpy/test_key_length.py +++ b/test/cqlpy/test_key_length.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Although regular column values are only limited by 31 bits (2GB), diff --git a/test/cqlpy/test_keyspace.py b/test/cqlpy/test_keyspace.py index 10abb68a3067..4dbfde2054a2 100644 --- a/test/cqlpy/test_keyspace.py +++ b/test/cqlpy/test_keyspace.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for basic keyspace operations: CREATE KEYSPACE, DROP KEYSPACE, # ALTER KEYSPACE diff --git a/test/cqlpy/test_large_cells_rows.py b/test/cqlpy/test_large_cells_rows.py index 841cdf469f3a..7f467739cd46 100644 --- a/test/cqlpy/test_large_cells_rows.py +++ b/test/cqlpy/test_large_cells_rows.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 from .util import new_test_table diff --git a/test/cqlpy/test_limit.py b/test/cqlpy/test_limit.py index 90074366a32b..7094578ab946 100644 --- a/test/cqlpy/test_limit.py +++ b/test/cqlpy/test_limit.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for SELECT's LIMIT feature diff --git a/test/cqlpy/test_logs.py b/test/cqlpy/test_logs.py index 2bebd487c6a3..ceac41852b47 100644 --- a/test/cqlpy/test_logs.py +++ b/test/cqlpy/test_logs.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Most cqlpy tests should limit themselves to the CQL API provided by diff --git a/test/cqlpy/test_lwt.py b/test/cqlpy/test_lwt.py index a811648b966e..edcda7bfdaa1 100644 --- a/test/cqlpy/test_lwt.py +++ b/test/cqlpy/test_lwt.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for Light-Weight Transactions (LWT) support in Scylla. diff --git a/test/cqlpy/test_materialized_view.py b/test/cqlpy/test_materialized_view.py index ed32088be91f..d1b85bafbb17 100644 --- a/test/cqlpy/test_materialized_view.py +++ b/test/cqlpy/test_materialized_view.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for materialized views diff --git a/test/cqlpy/test_native_functions.py b/test/cqlpy/test_native_functions.py index a4737b90c2a3..a2fec4fbf694 100644 --- a/test/cqlpy/test_native_functions.py +++ b/test/cqlpy/test_native_functions.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################### # Tests for various native (built-in) scalar functions that can be used in diff --git a/test/cqlpy/test_native_transport.py b/test/cqlpy/test_native_transport.py index 5106e1d5f030..8397ca57c024 100644 --- a/test/cqlpy/test_native_transport.py +++ b/test/cqlpy/test_native_transport.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest import time diff --git a/test/cqlpy/test_non_deterministic_functions.py b/test/cqlpy/test_non_deterministic_functions.py index 0fa86d8a78af..2da323b99abf 100644 --- a/test/cqlpy/test_non_deterministic_functions.py +++ b/test/cqlpy/test_non_deterministic_functions.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Non-deterministic CQL functions should be evaluated just before query diff --git a/test/cqlpy/test_null.py b/test/cqlpy/test_null.py index d0a4cced7017..86cb7422ba94 100644 --- a/test/cqlpy/test_null.py +++ b/test/cqlpy/test_null.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for finer points of the meaning of "null" in various places diff --git a/test/cqlpy/test_paging.py b/test/cqlpy/test_paging.py index b5424cb44844..c212b6acd359 100644 --- a/test/cqlpy/test_paging.py +++ b/test/cqlpy/test_paging.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 from .util import new_test_table from cassandra.query import SimpleStatement diff --git a/test/cqlpy/test_permissions.py b/test/cqlpy/test_permissions.py index a0e7ba887107..8e0e98a849f1 100644 --- a/test/cqlpy/test_permissions.py +++ b/test/cqlpy/test_permissions.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for managing permissions diff --git a/test/cqlpy/test_prepare.py b/test/cqlpy/test_prepare.py index 1def4774a107..2c8131bafbbe 100644 --- a/test/cqlpy/test_prepare.py +++ b/test/cqlpy/test_prepare.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for preparing various kinds of statements. When a client asks to prepare diff --git a/test/cqlpy/test_range_and_slice.py b/test/cqlpy/test_range_and_slice.py index 0e6317022d72..0ed538f01495 100644 --- a/test/cqlpy/test_range_and_slice.py +++ b/test/cqlpy/test_range_and_slice.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests the calculation of partition range and slice. diff --git a/test/cqlpy/test_restrictions.py b/test/cqlpy/test_restrictions.py index 72a8dcb0cd3e..7fc8df5e868b 100644 --- a/test/cqlpy/test_restrictions.py +++ b/test/cqlpy/test_restrictions.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the SELECT requests with various restriction (WHERE) expressions. diff --git a/test/cqlpy/test_scan.py b/test/cqlpy/test_scan.py index 3a14b34ac559..f41e5e172fb4 100644 --- a/test/cqlpy/test_scan.py +++ b/test/cqlpy/test_scan.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for scanning SELECT requests (which read many rows and/or many diff --git a/test/cqlpy/test_secondary_index.py b/test/cqlpy/test_secondary_index.py index 9e5eefaf3106..6cfc36416d93 100644 --- a/test/cqlpy/test_secondary_index.py +++ b/test/cqlpy/test_secondary_index.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for secondary indexes diff --git a/test/cqlpy/test_select_from_mutation_fragments.py b/test/cqlpy/test_select_from_mutation_fragments.py index 2eccf10a0871..38978bbcc286 100644 --- a/test/cqlpy/test_select_from_mutation_fragments.py +++ b/test/cqlpy/test_select_from_mutation_fragments.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests concerning the SELECT * FROM MUTATION_FRAGMENTS($table) statement, which allows dumping # the underlying mutation fragment data stream, for a table. diff --git a/test/cqlpy/test_service_levels.py b/test/cqlpy/test_service_levels.py index 50b41ce4549f..f4307ec2c3a1 100644 --- a/test/cqlpy/test_service_levels.py +++ b/test/cqlpy/test_service_levels.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the service levels infrastructure. Service levels can be attached diff --git a/test/cqlpy/test_shedding.py b/test/cqlpy/test_shedding.py index 8284addac066..6abb8c3884d3 100644 --- a/test/cqlpy/test_shedding.py +++ b/test/cqlpy/test_shedding.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the shedding mechanisms in the CQL layer diff --git a/test/cqlpy/test_ssl.py b/test/cqlpy/test_ssl.py index 02a599888c5b..35ea11955da1 100644 --- a/test/cqlpy/test_ssl.py +++ b/test/cqlpy/test_ssl.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for CQL over SSL (TLS). These tests are skipped when the tests are diff --git a/test/cqlpy/test_sstable.py b/test/cqlpy/test_sstable.py index aa7d3545afd9..7745190dbc1e 100644 --- a/test/cqlpy/test_sstable.py +++ b/test/cqlpy/test_sstable.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # The cqlpy test framework is about testing CQL functionality, so diff --git a/test/cqlpy/test_sstable_compression.py b/test/cqlpy/test_sstable_compression.py index 5f519116ab4a..07242e5c401d 100644 --- a/test/cqlpy/test_sstable_compression.py +++ b/test/cqlpy/test_sstable_compression.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for configuration of compressed sstables diff --git a/test/cqlpy/test_sstable_validation.py b/test/cqlpy/test_sstable_validation.py index 216556251e5d..ab643c89659e 100644 --- a/test/cqlpy/test_sstable_validation.py +++ b/test/cqlpy/test_sstable_validation.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for sstable validation. diff --git a/test/cqlpy/test_static.py b/test/cqlpy/test_static.py index 418ed9b1edaf..6ea48189d767 100644 --- a/test/cqlpy/test_static.py +++ b/test/cqlpy/test_static.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for static-column support in Scylla. diff --git a/test/cqlpy/test_system_tables.py b/test/cqlpy/test_system_tables.py index a7708f0cee23..cf432e8e18a0 100644 --- a/test/cqlpy/test_system_tables.py +++ b/test/cqlpy/test_system_tables.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for the content of system tables. Many of these tables have diff --git a/test/cqlpy/test_tablets.py b/test/cqlpy/test_tablets.py index 29284f9f80f3..88912a08b962 100644 --- a/test/cqlpy/test_tablets.py +++ b/test/cqlpy/test_tablets.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Some tests for the new "tablets"-based replication, replicating the old diff --git a/test/cqlpy/test_tombstone_limit.py b/test/cqlpy/test_tombstone_limit.py index 80551e709388..7660d022a7cd 100644 --- a/test/cqlpy/test_tombstone_limit.py +++ b/test/cqlpy/test_tombstone_limit.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for query_tombstone_page_limit diff --git a/test/cqlpy/test_tools.py b/test/cqlpy/test_tools.py index 2a13bc03f9ad..cdad32084574 100644 --- a/test/cqlpy/test_tools.py +++ b/test/cqlpy/test_tools.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the tools hosted by scylla diff --git a/test/cqlpy/test_ttl.py b/test/cqlpy/test_ttl.py index 84b267c30c83..1757e8e20978 100644 --- a/test/cqlpy/test_ttl.py +++ b/test/cqlpy/test_ttl.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for Scylla's ttl feature - USING TTL and DEFAULT_TIME_TO_LIVE diff --git a/test/cqlpy/test_type_date.py b/test/cqlpy/test_type_date.py index e4180b2051de..9db7d1550be1 100644 --- a/test/cqlpy/test_type_date.py +++ b/test/cqlpy/test_type_date.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Test involving the "date" column type. diff --git a/test/cqlpy/test_type_decimal.py b/test/cqlpy/test_type_decimal.py index b21408679b88..42893efd3612 100644 --- a/test/cqlpy/test_type_decimal.py +++ b/test/cqlpy/test_type_decimal.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Test involving the "decimal" column type. diff --git a/test/cqlpy/test_type_duration.py b/test/cqlpy/test_type_duration.py index 4c9e16a43e24..f86b74198e9b 100644 --- a/test/cqlpy/test_type_duration.py +++ b/test/cqlpy/test_type_duration.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Test involving the "duration" column type. diff --git a/test/cqlpy/test_type_string.py b/test/cqlpy/test_type_string.py index d1f7d204ec12..2b6a4c04114d 100644 --- a/test/cqlpy/test_type_string.py +++ b/test/cqlpy/test_type_string.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for string-like types - ascii, blob, and text (a.k.a varchar). diff --git a/test/cqlpy/test_type_time.py b/test/cqlpy/test_type_time.py index f50ca971ae81..650977b1c2f0 100644 --- a/test/cqlpy/test_type_time.py +++ b/test/cqlpy/test_type_time.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Test involving the "time" column type. diff --git a/test/cqlpy/test_type_timestamp.py b/test/cqlpy/test_type_timestamp.py index aad2db28fcbb..8a924ac24342 100644 --- a/test/cqlpy/test_type_timestamp.py +++ b/test/cqlpy/test_type_timestamp.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests involving the "timestamp" column type. diff --git a/test/cqlpy/test_type_uuid.py b/test/cqlpy/test_type_uuid.py index 8180dba2b693..ae7cacb52b58 100644 --- a/test/cqlpy/test_type_uuid.py +++ b/test/cqlpy/test_type_uuid.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Test involving the "uuid" column type. diff --git a/test/cqlpy/test_uda.py b/test/cqlpy/test_uda.py index 7692cbd39b1a..0a0ebf8a5df6 100644 --- a/test/cqlpy/test_uda.py +++ b/test/cqlpy/test_uda.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for user-defined aggregates (UDA) diff --git a/test/cqlpy/test_udf.py b/test/cqlpy/test_udf.py index 4aa490ad58db..5a1f8553052e 100644 --- a/test/cqlpy/test_udf.py +++ b/test/cqlpy/test_udf.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for user-defined functions (UDF) diff --git a/test/cqlpy/test_unset.py b/test/cqlpy/test_unset.py index eaa7debf580f..bf3d5c4e9e2f 100644 --- a/test/cqlpy/test_unset.py +++ b/test/cqlpy/test_unset.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the UNSET_VALUE value introduced in CQL version 4. Unset values diff --git a/test/cqlpy/test_use.py b/test/cqlpy/test_use.py index 4d6f11232436..d36a2aa4a745 100644 --- a/test/cqlpy/test_use.py +++ b/test/cqlpy/test_use.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the "USE" statement, which modifies the default keyspace used diff --git a/test/cqlpy/test_using_service_level.py b/test/cqlpy/test_using_service_level.py index 42e54da7a66b..56dc5de5e9ce 100644 --- a/test/cqlpy/test_using_service_level.py +++ b/test/cqlpy/test_using_service_level.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for USING SERVICE LEVEL extension diff --git a/test/cqlpy/test_using_timeout.py b/test/cqlpy/test_using_timeout.py index 8ea22d3c699f..a3b2166474fd 100644 --- a/test/cqlpy/test_using_timeout.py +++ b/test/cqlpy/test_using_timeout.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # Tests for USING TIMEOUT extension diff --git a/test/cqlpy/test_using_timestamp.py b/test/cqlpy/test_using_timestamp.py index dca7472dc3c4..46ee6d587cae 100644 --- a/test/cqlpy/test_using_timestamp.py +++ b/test/cqlpy/test_using_timestamp.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Various tests for USING TIMESTAMP support in Scylla. Note that Cassandra diff --git a/test/cqlpy/test_utf8.py b/test/cqlpy/test_utf8.py index f9eb9e674dcc..7f1939b32433 100644 --- a/test/cqlpy/test_utf8.py +++ b/test/cqlpy/test_utf8.py @@ -2,7 +2,7 @@ # -*- coding: utf-8 -*- # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for finer points of UTF-8 support. The issue of *invalid* UTF-8 input diff --git a/test/cqlpy/test_validation.py b/test/cqlpy/test_validation.py index fd196504af40..4126d8563cbb 100644 --- a/test/cqlpy/test_validation.py +++ b/test/cqlpy/test_validation.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for the type validation in CQL. For example, it should not be diff --git a/test/cqlpy/test_virtual_tables.py b/test/cqlpy/test_virtual_tables.py index 71e570910848..8c0b3f358ce3 100644 --- a/test/cqlpy/test_virtual_tables.py +++ b/test/cqlpy/test_virtual_tables.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest from . import nodetool diff --git a/test/cqlpy/test_wasm.py b/test/cqlpy/test_wasm.py index ba0d5707102f..09b97ec34c5b 100644 --- a/test/cqlpy/test_wasm.py +++ b/test/cqlpy/test_wasm.py @@ -1,7 +1,7 @@ # -*- coding: utf-8 -*- # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ############################################################################# # Tests for user defined functions defined with WebAssembly backend diff --git a/test/cqlpy/util.py b/test/cqlpy/util.py index 027bb9086885..14b611e027c0 100644 --- a/test/cqlpy/util.py +++ b/test/cqlpy/util.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 ################################################################## # Various utility functions which are useful for multiple tests. diff --git a/test/lib/alternator_test_env.cc b/test/lib/alternator_test_env.cc index 9a3a64f741c9..64198b61129a 100644 --- a/test/lib/alternator_test_env.cc +++ b/test/lib/alternator_test_env.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/alternator_test_env.hh" diff --git a/test/lib/alternator_test_env.hh b/test/lib/alternator_test_env.hh index 0c3954003b00..abb3fd167748 100644 --- a/test/lib/alternator_test_env.hh +++ b/test/lib/alternator_test_env.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/cql_assertions.cc b/test/lib/cql_assertions.cc index 347e10cf74e1..c035a55a14c6 100644 --- a/test/lib/cql_assertions.cc +++ b/test/lib/cql_assertions.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/lib/cql_assertions.hh b/test/lib/cql_assertions.hh index 9cfe8e1ffa4a..8392c2470852 100644 --- a/test/lib/cql_assertions.hh +++ b/test/lib/cql_assertions.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index 3b21d951b009..140a283c39e6 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/lib/cql_test_env.hh b/test/lib/cql_test_env.hh index 8ab126d044ab..040684ca82dd 100644 --- a/test/lib/cql_test_env.hh +++ b/test/lib/cql_test_env.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/data_model.cc b/test/lib/data_model.cc index 195e5fc87277..4afab72ea0e1 100644 --- a/test/lib/data_model.cc +++ b/test/lib/data_model.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/lib/data_model.hh b/test/lib/data_model.hh index dce0151aba3c..8497c6b05af0 100644 --- a/test/lib/data_model.hh +++ b/test/lib/data_model.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/dummy_sharder.cc b/test/lib/dummy_sharder.cc index 21fcbbcfcea1..4c61391f1fa8 100644 --- a/test/lib/dummy_sharder.cc +++ b/test/lib/dummy_sharder.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/lib/dummy_sharder.hh b/test/lib/dummy_sharder.hh index b1e5571588df..cb6076a5db60 100644 --- a/test/lib/dummy_sharder.hh +++ b/test/lib/dummy_sharder.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/eventually.hh b/test/lib/eventually.hh index a746f5d74d90..1dbc76bc40db 100644 --- a/test/lib/eventually.hh +++ b/test/lib/eventually.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/exception_utils.cc b/test/lib/exception_utils.cc index 4872b2c6f4e8..c9f017784dab 100644 --- a/test/lib/exception_utils.cc +++ b/test/lib/exception_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/exception_utils.hh" diff --git a/test/lib/exception_utils.hh b/test/lib/exception_utils.hh index e51b1051a376..83b7036034b9 100644 --- a/test/lib/exception_utils.hh +++ b/test/lib/exception_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/expr_test_utils.cc b/test/lib/expr_test_utils.cc index 9645265c3452..1453e7a3d38e 100644 --- a/test/lib/expr_test_utils.cc +++ b/test/lib/expr_test_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "expr_test_utils.hh" diff --git a/test/lib/expr_test_utils.hh b/test/lib/expr_test_utils.hh index a7e326988cee..40f56eaff831 100644 --- a/test/lib/expr_test_utils.hh +++ b/test/lib/expr_test_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/failure_injecting_allocation_strategy.hh b/test/lib/failure_injecting_allocation_strategy.hh index b44f62dbb35b..d0db6431b499 100644 --- a/test/lib/failure_injecting_allocation_strategy.hh +++ b/test/lib/failure_injecting_allocation_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/fragment_scatterer.hh b/test/lib/fragment_scatterer.hh index 3ae53f7b2ebe..6acbbaf69e62 100644 --- a/test/lib/fragment_scatterer.hh +++ b/test/lib/fragment_scatterer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/index_reader_assertions.hh b/test/lib/index_reader_assertions.hh index 56eee074bdd5..97b3871fd4c6 100644 --- a/test/lib/index_reader_assertions.hh +++ b/test/lib/index_reader_assertions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/key_utils.cc b/test/lib/key_utils.cc index 6d4da71623b3..b9f1e0c86b38 100644 --- a/test/lib/key_utils.cc +++ b/test/lib/key_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/key_utils.hh" diff --git a/test/lib/key_utils.hh b/test/lib/key_utils.hh index af88bb0f9a6e..31625cc68313 100644 --- a/test/lib/key_utils.hh +++ b/test/lib/key_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/log.cc b/test/lib/log.cc index 6dcf6039bc5c..1bdc2b34b390 100644 --- a/test/lib/log.cc +++ b/test/lib/log.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/log.hh" diff --git a/test/lib/log.hh b/test/lib/log.hh index d4756f27448b..e543002d6cdc 100644 --- a/test/lib/log.hh +++ b/test/lib/log.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once #include diff --git a/test/lib/make_random_string.hh b/test/lib/make_random_string.hh index 46f5a62bab6b..59c0b2898e6d 100644 --- a/test/lib/make_random_string.hh +++ b/test/lib/make_random_string.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/memtable_snapshot_source.hh b/test/lib/memtable_snapshot_source.hh index 7534aaf3fbbf..772184d89dae 100644 --- a/test/lib/memtable_snapshot_source.hh +++ b/test/lib/memtable_snapshot_source.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/mutation_assertions.hh b/test/lib/mutation_assertions.hh index ffe1c20498eb..d71055275161 100644 --- a/test/lib/mutation_assertions.hh +++ b/test/lib/mutation_assertions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/mutation_reader_assertions.hh b/test/lib/mutation_reader_assertions.hh index fd91bcfe4f07..566a432ca776 100644 --- a/test/lib/mutation_reader_assertions.hh +++ b/test/lib/mutation_reader_assertions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/mutation_source_test.cc b/test/lib/mutation_source_test.cc index 381c8221377b..5523c95d6328 100644 --- a/test/lib/mutation_source_test.cc +++ b/test/lib/mutation_source_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/lib/mutation_source_test.hh b/test/lib/mutation_source_test.hh index 07b0987ac0c0..db19d5e71ae1 100644 --- a/test/lib/mutation_source_test.hh +++ b/test/lib/mutation_source_test.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/random_schema.cc b/test/lib/random_schema.cc index f26bc136a9e4..9b731b0abd81 100644 --- a/test/lib/random_schema.cc +++ b/test/lib/random_schema.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/lib/random_schema.hh b/test/lib/random_schema.hh index 2db2fc2e4eb8..555fc51f2f39 100644 --- a/test/lib/random_schema.hh +++ b/test/lib/random_schema.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/random_utils.hh b/test/lib/random_utils.hh index f60586294852..386beba386bb 100644 --- a/test/lib/random_utils.hh +++ b/test/lib/random_utils.hh @@ -2,7 +2,7 @@ * Copyright (C) 2018-present ScyllaDB */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/reader_concurrency_semaphore.hh b/test/lib/reader_concurrency_semaphore.hh index 23648a630615..ba8aeb3ee345 100644 --- a/test/lib/reader_concurrency_semaphore.hh +++ b/test/lib/reader_concurrency_semaphore.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/reader_lifecycle_policy.hh b/test/lib/reader_lifecycle_policy.hh index 1929d9907383..ebb0ba741054 100644 --- a/test/lib/reader_lifecycle_policy.hh +++ b/test/lib/reader_lifecycle_policy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/result_set_assertions.cc b/test/lib/result_set_assertions.cc index e113c354f3ee..5dd557b49dec 100644 --- a/test/lib/result_set_assertions.cc +++ b/test/lib/result_set_assertions.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/lib/result_set_assertions.hh b/test/lib/result_set_assertions.hh index f31f2b4fb254..948a032d3ef7 100644 --- a/test/lib/result_set_assertions.hh +++ b/test/lib/result_set_assertions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/scylla_test_case.hh b/test/lib/scylla_test_case.hh index 04cc74cc312d..59acf290ad3c 100644 --- a/test/lib/scylla_test_case.hh +++ b/test/lib/scylla_test_case.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/scylla_tests_cmdline_options.hh b/test/lib/scylla_tests_cmdline_options.hh index 53158d038b80..999ca1ddbc28 100644 --- a/test/lib/scylla_tests_cmdline_options.hh +++ b/test/lib/scylla_tests_cmdline_options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/select_statement_utils.hh b/test/lib/select_statement_utils.hh index 5f3377b41244..54a15215218c 100644 --- a/test/lib/select_statement_utils.hh +++ b/test/lib/select_statement_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/simple_position_reader_queue.hh b/test/lib/simple_position_reader_queue.hh index ecfc57fb0215..d42ac02cfcb7 100644 --- a/test/lib/simple_position_reader_queue.hh +++ b/test/lib/simple_position_reader_queue.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/simple_schema.hh b/test/lib/simple_schema.hh index 551cf9d25468..9ad6609fcbb4 100644 --- a/test/lib/simple_schema.hh +++ b/test/lib/simple_schema.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/sstable_run_based_compaction_strategy_for_tests.cc b/test/lib/sstable_run_based_compaction_strategy_for_tests.cc index 50439fb6ce32..d7bff18c89a8 100644 --- a/test/lib/sstable_run_based_compaction_strategy_for_tests.cc +++ b/test/lib/sstable_run_based_compaction_strategy_for_tests.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/lib/sstable_run_based_compaction_strategy_for_tests.hh b/test/lib/sstable_run_based_compaction_strategy_for_tests.hh index 24fa42806dbe..6b412832d44f 100644 --- a/test/lib/sstable_run_based_compaction_strategy_for_tests.hh +++ b/test/lib/sstable_run_based_compaction_strategy_for_tests.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/sstable_test_env.hh b/test/lib/sstable_test_env.hh index 9e687539a1f3..aaf14dadf24a 100644 --- a/test/lib/sstable_test_env.hh +++ b/test/lib/sstable_test_env.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/sstable_utils.cc b/test/lib/sstable_utils.cc index bd895848d2a1..67522bf870e8 100644 --- a/test/lib/sstable_utils.cc +++ b/test/lib/sstable_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/sstable_utils.hh" diff --git a/test/lib/sstable_utils.hh b/test/lib/sstable_utils.hh index 1eab34f2e04d..a15f62a976f9 100644 --- a/test/lib/sstable_utils.hh +++ b/test/lib/sstable_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/test_services.cc b/test/lib/test_services.cc index 888ca3f05b3b..f1c47fc4cecb 100644 --- a/test/lib/test_services.cc +++ b/test/lib/test_services.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_tests_cmdline_options.hh" diff --git a/test/lib/test_utils.cc b/test/lib/test_utils.cc index 28b3f060977b..7133daa43de9 100644 --- a/test/lib/test_utils.cc +++ b/test/lib/test_utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/test_utils.hh" diff --git a/test/lib/test_utils.hh b/test/lib/test_utils.hh index 2db228bc356c..38a956f220e5 100644 --- a/test/lib/test_utils.hh +++ b/test/lib/test_utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/lib/tmpdir.cc b/test/lib/tmpdir.cc index 412965d0d36e..738dcdf90fd1 100644 --- a/test/lib/tmpdir.cc +++ b/test/lib/tmpdir.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/tmpdir.hh" diff --git a/test/lib/tmpdir.hh b/test/lib/tmpdir.hh index f9a7c9db3985..4b5a793bc866 100644 --- a/test/lib/tmpdir.hh +++ b/test/lib/tmpdir.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/manual/ec2_snitch_test.cc b/test/manual/ec2_snitch_test.cc index cfed7e931cd1..98889213429f 100644 --- a/test/manual/ec2_snitch_test.cc +++ b/test/manual/ec2_snitch_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/manual/enormous_table_scan_test.cc b/test/manual/enormous_table_scan_test.cc index 862d3151439f..8ff34cca6a66 100644 --- a/test/manual/enormous_table_scan_test.cc +++ b/test/manual/enormous_table_scan_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/manual/gce_snitch_test.cc b/test/manual/gce_snitch_test.cc index 042c6198a54b..6e77b9ccc2f7 100644 --- a/test/manual/gce_snitch_test.cc +++ b/test/manual/gce_snitch_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /// \brief Tests the GoogleCloudSnitch. diff --git a/test/manual/gossip.cc b/test/manual/gossip.cc index 17dd5f0c3493..a124aec8c1db 100644 --- a/test/manual/gossip.cc +++ b/test/manual/gossip.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/manual/hint_test.cc b/test/manual/hint_test.cc index 72e29c3fcf13..729e951647e1 100644 --- a/test/manual/hint_test.cc +++ b/test/manual/hint_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/manual/message.cc b/test/manual/message.cc index 9f51ae9b420d..66560e646c0d 100644 --- a/test/manual/message.cc +++ b/test/manual/message.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/manual/partition_data_test.cc b/test/manual/partition_data_test.cc index 686f4c057c13..80732ae9d832 100644 --- a/test/manual/partition_data_test.cc +++ b/test/manual/partition_data_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE partition_data diff --git a/test/manual/row_locker_test.cc b/test/manual/row_locker_test.cc index c80ce9bdf597..b45b8e1fddf7 100644 --- a/test/manual/row_locker_test.cc +++ b/test/manual/row_locker_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/manual/sstable_scan_footprint_test.cc b/test/manual/sstable_scan_footprint_test.cc index eebc28452a96..8851901dbc6f 100644 --- a/test/manual/sstable_scan_footprint_test.cc +++ b/test/manual/sstable_scan_footprint_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/cql_test_env.hh" diff --git a/test/manual/streaming_histogram_test.cc b/test/manual/streaming_histogram_test.cc index f194e7142362..adb461122183 100644 --- a/test/manual/streaming_histogram_test.cc +++ b/test/manual/streaming_histogram_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #define BOOST_TEST_MODULE core diff --git a/test/nodetool/conftest.py b/test/nodetool/conftest.py index 7e366fa19d93..4322086c23f1 100644 --- a/test/nodetool/conftest.py +++ b/test/nodetool/conftest.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import os diff --git a/test/nodetool/rest_api_mock.py b/test/nodetool/rest_api_mock.py index 3157a7808c73..a5f567c1ebde 100644 --- a/test/nodetool/rest_api_mock.py +++ b/test/nodetool/rest_api_mock.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import aiohttp diff --git a/test/nodetool/test_autocompaction.py b/test/nodetool/test_autocompaction.py index fc3480715364..6a910b473ade 100644 --- a/test/nodetool/test_autocompaction.py +++ b/test/nodetool/test_autocompaction.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_backup.py b/test/nodetool/test_backup.py index 854aedb1bc72..60806116b093 100644 --- a/test/nodetool/test_backup.py +++ b/test/nodetool/test_backup.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_binary.py b/test/nodetool/test_binary.py index df6b939b8183..37c81b1fc2b5 100644 --- a/test/nodetool/test_binary.py +++ b/test/nodetool/test_binary.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_check_and_repair_cdc_streams.py b/test/nodetool/test_check_and_repair_cdc_streams.py index 250fc82e7d92..94cc97333b07 100644 --- a/test/nodetool/test_check_and_repair_cdc_streams.py +++ b/test/nodetool/test_check_and_repair_cdc_streams.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_cleanup.py b/test/nodetool/test_cleanup.py index 6319c4f015da..8d2a3e80ffd1 100644 --- a/test/nodetool/test_cleanup.py +++ b/test/nodetool/test_cleanup.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_compact.py b/test/nodetool/test_compact.py index 19341ffab7c0..e1b2a89224e8 100644 --- a/test/nodetool/test_compact.py +++ b/test/nodetool/test_compact.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_compactionhistory.py b/test/nodetool/test_compactionhistory.py index 44117e042533..9d41da80492c 100644 --- a/test/nodetool/test_compactionhistory.py +++ b/test/nodetool/test_compactionhistory.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import datetime diff --git a/test/nodetool/test_compactionstats.py b/test/nodetool/test_compactionstats.py index 01a892923860..aafa0481991c 100644 --- a/test/nodetool/test_compactionstats.py +++ b/test/nodetool/test_compactionstats.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_describecluster.py b/test/nodetool/test_describecluster.py index 94a440763d56..e60c98187f38 100644 --- a/test/nodetool/test_describecluster.py +++ b/test/nodetool/test_describecluster.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_describering.py b/test/nodetool/test_describering.py index 4b1583751293..dbc543dc6547 100644 --- a/test/nodetool/test_describering.py +++ b/test/nodetool/test_describering.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_drain.py b/test/nodetool/test_drain.py index 48f92f3dec47..2b00dce55e1e 100644 --- a/test/nodetool/test_drain.py +++ b/test/nodetool/test_drain.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_flush.py b/test/nodetool/test_flush.py index e9fee2b1f403..bf78910a1d76 100644 --- a/test/nodetool/test_flush.py +++ b/test/nodetool/test_flush.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_getendpoints.py b/test/nodetool/test_getendpoints.py index e5647e18b78a..cbc6c6a41c0f 100644 --- a/test/nodetool/test_getendpoints.py +++ b/test/nodetool/test_getendpoints.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_gossip.py b/test/nodetool/test_gossip.py index 51a1186d5d81..246009a4d0c2 100644 --- a/test/nodetool/test_gossip.py +++ b/test/nodetool/test_gossip.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_gossipinfo.py b/test/nodetool/test_gossipinfo.py index 05024f415ea3..d51b8ca1d7c7 100644 --- a/test/nodetool/test_gossipinfo.py +++ b/test/nodetool/test_gossipinfo.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_help.py b/test/nodetool/test_help.py index 1a438a649085..53e5b71128e8 100644 --- a/test/nodetool/test_help.py +++ b/test/nodetool/test_help.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_info.py b/test/nodetool/test_info.py index 60b33b384874..6e74ac020b6d 100644 --- a/test/nodetool/test_info.py +++ b/test/nodetool/test_info.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import random diff --git a/test/nodetool/test_logging.py b/test/nodetool/test_logging.py index f8960d11a1a1..3c817fb88eb4 100644 --- a/test/nodetool/test_logging.py +++ b/test/nodetool/test_logging.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_netstats.py b/test/nodetool/test_netstats.py index 28f9d86fab5a..7129ddee98ee 100644 --- a/test/nodetool/test_netstats.py +++ b/test/nodetool/test_netstats.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_nodeops.py b/test/nodetool/test_nodeops.py index 90aa0c422e7f..343c235e72d5 100644 --- a/test/nodetool/test_nodeops.py +++ b/test/nodetool/test_nodeops.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_nodetool.py b/test/nodetool/test_nodetool.py index e107427bb80a..51806f212e79 100644 --- a/test/nodetool/test_nodetool.py +++ b/test/nodetool/test_nodetool.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request, expected_requests_manager diff --git a/test/nodetool/test_proxyhistograms.py b/test/nodetool/test_proxyhistograms.py index 9b98853a370b..41db1a8f5938 100644 --- a/test/nodetool/test_proxyhistograms.py +++ b/test/nodetool/test_proxyhistograms.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_rebuild.py b/test/nodetool/test_rebuild.py index 2b0061698255..6d86bd51408d 100644 --- a/test/nodetool/test_rebuild.py +++ b/test/nodetool/test_rebuild.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_refresh.py b/test/nodetool/test_refresh.py index 8c35641a2f50..a6a0f8085622 100644 --- a/test/nodetool/test_refresh.py +++ b/test/nodetool/test_refresh.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_repair.py b/test/nodetool/test_repair.py index cd7a1d3220f9..56df5566ac9f 100644 --- a/test/nodetool/test_repair.py +++ b/test/nodetool/test_repair.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_resetlocalschema.py b/test/nodetool/test_resetlocalschema.py index e562eb27887c..9bd128d3ad46 100644 --- a/test/nodetool/test_resetlocalschema.py +++ b/test/nodetool/test_resetlocalschema.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_restore.py b/test/nodetool/test_restore.py index 4b7889194b59..93a5e3ced93c 100644 --- a/test/nodetool/test_restore.py +++ b/test/nodetool/test_restore.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/test_ring.py b/test/nodetool/test_ring.py index 40efeb47a2d6..544d51b4be05 100644 --- a/test/nodetool/test_ring.py +++ b/test/nodetool/test_ring.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from typing import NamedTuple diff --git a/test/nodetool/test_scrub.py b/test/nodetool/test_scrub.py index 2c796b1e00af..bd27d585e910 100644 --- a/test/nodetool/test_scrub.py +++ b/test/nodetool/test_scrub.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import enum diff --git a/test/nodetool/test_snapshot.py b/test/nodetool/test_snapshot.py index f5a4f9a7f54b..f80e7bfeedef 100644 --- a/test/nodetool/test_snapshot.py +++ b/test/nodetool/test_snapshot.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.utils import check_nodetool_fails_with from test.nodetool.rest_api_mock import expected_request, approximate_value diff --git a/test/nodetool/test_sstable.py b/test/nodetool/test_sstable.py index 67d80528b862..44a4acd21c08 100644 --- a/test/nodetool/test_sstable.py +++ b/test/nodetool/test_sstable.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.utils import check_nodetool_fails_with from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_status.py b/test/nodetool/test_status.py index 06c50eaa4811..a48c2490a3c5 100644 --- a/test/nodetool/test_status.py +++ b/test/nodetool/test_status.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # diff --git a/test/nodetool/test_stop.py b/test/nodetool/test_stop.py index 666a9ba9b317..acf19ef2d812 100644 --- a/test/nodetool/test_stop.py +++ b/test/nodetool/test_stop.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.utils import check_nodetool_fails_with from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_tablehistograms.py b/test/nodetool/test_tablehistograms.py index 315a053a97d7..de7bdd9b9872 100644 --- a/test/nodetool/test_tablehistograms.py +++ b/test/nodetool/test_tablehistograms.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_tablestats.py b/test/nodetool/test_tablestats.py index 91eeb87f46ab..230bbdb532b3 100644 --- a/test/nodetool/test_tablestats.py +++ b/test/nodetool/test_tablestats.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import json diff --git a/test/nodetool/test_tasks.py b/test/nodetool/test_tasks.py index 35f3de0e4ee3..d9052c2898fe 100644 --- a/test/nodetool/test_tasks.py +++ b/test/nodetool/test_tasks.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_toppartitions.py b/test/nodetool/test_toppartitions.py index c4f239170e62..90c633f70cc8 100644 --- a/test/nodetool/test_toppartitions.py +++ b/test/nodetool/test_toppartitions.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from typing import NamedTuple diff --git a/test/nodetool/test_traceprobability.py b/test/nodetool/test_traceprobability.py index 286ac605885d..87e6db93cf17 100644 --- a/test/nodetool/test_traceprobability.py +++ b/test/nodetool/test_traceprobability.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_upgradesstables.py b/test/nodetool/test_upgradesstables.py index 8ee1ca6ab1fe..a33bab4ccb3b 100644 --- a/test/nodetool/test_upgradesstables.py +++ b/test/nodetool/test_upgradesstables.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # diff --git a/test/nodetool/test_version.py b/test/nodetool/test_version.py index d090692ba275..95eb40981474 100644 --- a/test/nodetool/test_version.py +++ b/test/nodetool/test_version.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.nodetool.rest_api_mock import expected_request diff --git a/test/nodetool/test_viewbuildstatus.py b/test/nodetool/test_viewbuildstatus.py index 15b7e1dd9779..63d6c308d710 100644 --- a/test/nodetool/test_viewbuildstatus.py +++ b/test/nodetool/test_viewbuildstatus.py @@ -1,7 +1,7 @@ # # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/nodetool/utils.py b/test/nodetool/utils.py index 612f0f0d2221..86f594cbfae8 100644 --- a/test/nodetool/utils.py +++ b/test/nodetool/utils.py @@ -1,7 +1,7 @@ # # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import itertools diff --git a/test/perf/entry_point.hh b/test/perf/entry_point.hh index ce8566660891..c8beb3349657 100644 --- a/test/perf/entry_point.hh +++ b/test/perf/entry_point.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/perf/logalloc.cc b/test/perf/logalloc.cc index e8c9142fd864..7ca1ac17648a 100644 --- a/test/perf/logalloc.cc +++ b/test/perf/logalloc.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/memory_footprint_test.cc b/test/perf/memory_footprint_test.cc index e37466ee512d..680964b6b668 100644 --- a/test/perf/memory_footprint_test.cc +++ b/test/perf/memory_footprint_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/perf/perf.cc b/test/perf/perf.cc index 6d38659db68f..06dbc7611eb0 100644 --- a/test/perf/perf.cc +++ b/test/perf/perf.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "perf.hh" diff --git a/test/perf/perf.hh b/test/perf/perf.hh index a48155bcace5..3266102c7d25 100644 --- a/test/perf/perf.hh +++ b/test/perf/perf.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/perf/perf_alternator.cc b/test/perf/perf_alternator.cc index 1bb7165ee4db..35870de72940 100644 --- a/test/perf/perf_alternator.cc +++ b/test/perf/perf_alternator.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_big_decimal.cc b/test/perf/perf_big_decimal.cc index 5b31d0dcb4f9..d36b2861a5a8 100644 --- a/test/perf/perf_big_decimal.cc +++ b/test/perf/perf_big_decimal.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_cache_eviction.cc b/test/perf/perf_cache_eviction.cc index b5495094e059..12d7a975453c 100644 --- a/test/perf/perf_cache_eviction.cc +++ b/test/perf/perf_cache_eviction.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_checksum.cc b/test/perf/perf_checksum.cc index 2e6c7b8e0fc2..0d9724ee92be 100644 --- a/test/perf/perf_checksum.cc +++ b/test/perf/perf_checksum.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "sstables/checksum_utils.hh" diff --git a/test/perf/perf_collection.cc b/test/perf/perf_collection.cc index 7196b7a28681..7a6098824c88 100644 --- a/test/perf/perf_collection.cc +++ b/test/perf/perf_collection.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_commitlog.cc b/test/perf/perf_commitlog.cc index 71047ae68562..7a9b753c4934 100644 --- a/test/perf/perf_commitlog.cc +++ b/test/perf/perf_commitlog.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_cql_parser.cc b/test/perf/perf_cql_parser.cc index aa1eb21deac0..58d727ffda7c 100644 --- a/test/perf/perf_cql_parser.cc +++ b/test/perf/perf_cql_parser.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/perf/perf.hh" diff --git a/test/perf/perf_fast_forward.cc b/test/perf/perf_fast_forward.cc index 7d6ffb0bb419..cb5510ce324a 100644 --- a/test/perf/perf_fast_forward.cc +++ b/test/perf/perf_fast_forward.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_fast_forward_report.py b/test/perf/perf_fast_forward_report.py index 289cbdbed331..028dd3d3c327 100755 --- a/test/perf/perf_fast_forward_report.py +++ b/test/perf/perf_fast_forward_report.py @@ -4,7 +4,7 @@ # Copyright (C) 2019-present ScyllaDB # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse diff --git a/test/perf/perf_hash.cc b/test/perf/perf_hash.cc index 2b51be1d7708..a41ad355fb9c 100644 --- a/test/perf/perf_hash.cc +++ b/test/perf/perf_hash.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/murmur_hash.hh" diff --git a/test/perf/perf_idl.cc b/test/perf/perf_idl.cc index 2a48549a9b9c..51380e7abeef 100644 --- a/test/perf/perf_idl.cc +++ b/test/perf/perf_idl.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_mutation.cc b/test/perf/perf_mutation.cc index f14bfa912b93..ff56cae5565b 100644 --- a/test/perf/perf_mutation.cc +++ b/test/perf/perf_mutation.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "replica/database.hh" diff --git a/test/perf/perf_mutation_fragment.cc b/test/perf/perf_mutation_fragment.cc index 73d19e727bb0..bf7d2f1c06f5 100644 --- a/test/perf/perf_mutation_fragment.cc +++ b/test/perf/perf_mutation_fragment.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_mutation_readers.cc b/test/perf/perf_mutation_readers.cc index 9f9999c6fce8..cdd5b0eb1372 100644 --- a/test/perf/perf_mutation_readers.cc +++ b/test/perf/perf_mutation_readers.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/test/perf/perf_row_cache_reads.cc b/test/perf/perf_row_cache_reads.cc index 836bea6ef637..0b006101fc09 100644 --- a/test/perf/perf_row_cache_reads.cc +++ b/test/perf/perf_row_cache_reads.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_row_cache_update.cc b/test/perf/perf_row_cache_update.cc index ad8f3be3463f..3c775c162159 100644 --- a/test/perf/perf_row_cache_update.cc +++ b/test/perf/perf_row_cache_update.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_s3_client.cc b/test/perf/perf_s3_client.cc index e0bbdd8f35b8..f045f81fd257 100644 --- a/test/perf/perf_s3_client.cc +++ b/test/perf/perf_s3_client.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_simple_query.cc b/test/perf/perf_simple_query.cc index f4951bb58b36..18ce46336b30 100644 --- a/test/perf/perf_simple_query.cc +++ b/test/perf/perf_simple_query.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/perf/perf_sstable.cc b/test/perf/perf_sstable.cc index bd8002f7a873..e66bff64437c 100644 --- a/test/perf/perf_sstable.cc +++ b/test/perf/perf_sstable.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/perf_sstable.hh b/test/perf/perf_sstable.hh index 7d92595e63d2..a2f814b97fc2 100644 --- a/test/perf/perf_sstable.hh +++ b/test/perf/perf_sstable.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/perf/perf_tablets.cc b/test/perf/perf_tablets.cc index f5af96c9404a..d9edff0c03d2 100644 --- a/test/perf/perf_tablets.cc +++ b/test/perf/perf_tablets.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/perf/perf_vint.cc b/test/perf/perf_vint.cc index c2fafd160452..a898ec85a014 100644 --- a/test/perf/perf_vint.cc +++ b/test/perf/perf_vint.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/perf/tablet_load_balancing.cc b/test/perf/tablet_load_balancing.cc index 0212bef3e2b5..9a9e5178cb9a 100644 --- a/test/perf/tablet_load_balancing.cc +++ b/test/perf/tablet_load_balancing.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/pylib/artifact_registry.py b/test/pylib/artifact_registry.py index ee1a02618b74..80fe7c68c7e5 100644 --- a/test/pylib/artifact_registry.py +++ b/test/pylib/artifact_registry.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from typing import Protocol from typing import Callable, Coroutine, List, Dict, Optional diff --git a/test/pylib/async_cql.py b/test/pylib/async_cql.py index 04e5d39e690f..f26733781ddf 100644 --- a/test/pylib/async_cql.py +++ b/test/pylib/async_cql.py @@ -1,6 +1,6 @@ # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 """ async_cql: diff --git a/test/pylib/coverage_utils.py b/test/pylib/coverage_utils.py index a0358462e8e7..ac88cbf7ac81 100755 --- a/test/pylib/coverage_utils.py +++ b/test/pylib/coverage_utils.py @@ -4,7 +4,7 @@ # Copyright (C) 2024-present ScyllaDB # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import argparse import asyncio diff --git a/test/pylib/cql_repl/conftest.py b/test/pylib/cql_repl/conftest.py index f2c4898e317d..7ae93658dbcc 100644 --- a/test/pylib/cql_repl/conftest.py +++ b/test/pylib/cql_repl/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ diff --git a/test/pylib/cql_repl/cql_repl.py b/test/pylib/cql_repl/cql_repl.py index ef352fb57227..a85902c2713e 100644 --- a/test/pylib/cql_repl/cql_repl.py +++ b/test/pylib/cql_repl/cql_repl.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra import ConsistencyLevel # type: ignore from cassandra.query import SimpleStatement # type: ignore diff --git a/test/pylib/db/model.py b/test/pylib/db/model.py index f24e313e6489..89f410fc8e88 100644 --- a/test/pylib/db/model.py +++ b/test/pylib/db/model.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from datetime import datetime diff --git a/test/pylib/db/writer.py b/test/pylib/db/writer.py index 55ab18dfd787..21da995b6644 100644 --- a/test/pylib/db/writer.py +++ b/test/pylib/db/writer.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import sqlite3 diff --git a/test/pylib/host_registry.py b/test/pylib/host_registry.py index a35dec1633fb..9bd04076820d 100644 --- a/test/pylib/host_registry.py +++ b/test/pylib/host_registry.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import errno import fcntl diff --git a/test/pylib/internal_types.py b/test/pylib/internal_types.py index 38e583c2606a..90bb5ca71a3b 100644 --- a/test/pylib/internal_types.py +++ b/test/pylib/internal_types.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """Internal types for handling Scylla test servers. """ diff --git a/test/pylib/lcov_utils.py b/test/pylib/lcov_utils.py index 41007f6c5fe8..28a8a194f042 100644 --- a/test/pylib/lcov_utils.py +++ b/test/pylib/lcov_utils.py @@ -3,7 +3,7 @@ # Copyright (C) 2024-present ScyllaDB # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import unidiff from typing import ( diff --git a/test/pylib/log_browsing.py b/test/pylib/log_browsing.py index 7cc132c352e9..95c04fd4a2fe 100644 --- a/test/pylib/log_browsing.py +++ b/test/pylib/log_browsing.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import asyncio from concurrent.futures import ThreadPoolExecutor diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index 0a4edb7231fc..c4feb21538df 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """Manager client. Communicates with Manager server via socket. diff --git a/test/pylib/minio_server.py b/test/pylib/minio_server.py index caf2e9f613e7..865522c0ec1f 100755 --- a/test/pylib/minio_server.py +++ b/test/pylib/minio_server.py @@ -2,7 +2,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """Minio server for testing. Provides helpers to setup and manage minio server for testing. diff --git a/test/pylib/random_tables.py b/test/pylib/random_tables.py index 1191cfda47bf..ff212d5987ab 100644 --- a/test/pylib/random_tables.py +++ b/test/pylib/random_tables.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """This module provides helper classes to manage CQL tables, perform random schema changes, and verify expected current schema. diff --git a/test/pylib/repair.py b/test/pylib/repair.py index a748207fd7ab..85e6f239816a 100644 --- a/test/pylib/repair.py +++ b/test/pylib/repair.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.util import wait_for_cql_and_get_hosts diff --git a/test/pylib/report_plugin.py b/test/pylib/report_plugin.py index 5779d738e7f2..864f7eef3614 100644 --- a/test/pylib/report_plugin.py +++ b/test/pylib/report_plugin.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import allure diff --git a/test/pylib/resource_gather.py b/test/pylib/resource_gather.py index a13835e521ec..1219282cc86e 100644 --- a/test/pylib/resource_gather.py +++ b/test/pylib/resource_gather.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/pylib/rest_client.py b/test/pylib/rest_client.py index 34e5793859c6..4835cdc837f7 100644 --- a/test/pylib/rest_client.py +++ b/test/pylib/rest_client.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """Asynchronous helper for Scylla REST API operations. """ diff --git a/test/pylib/s3_proxy.py b/test/pylib/s3_proxy.py index d3a709218907..1be878c08a23 100644 --- a/test/pylib/s3_proxy.py +++ b/test/pylib/s3_proxy.py @@ -2,7 +2,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # S3 proxy server to inject retryable errors for fuzzy testing. diff --git a/test/pylib/s3_server_mock.py b/test/pylib/s3_server_mock.py index 9cbebfdb9e9e..7dbc65571ada 100644 --- a/test/pylib/s3_server_mock.py +++ b/test/pylib/s3_server_mock.py @@ -2,7 +2,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # Mock S3 server to inject errors for testing. diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index b06ff3f2ebbe..1322da534a9d 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """Scylla clusters for testing. Provides helpers to setup and manage clusters of Scylla servers for testing. diff --git a/test/pylib/tablets.py b/test/pylib/tablets.py index fc94048499d9..9d52e7b21bdd 100644 --- a/test/pylib/tablets.py +++ b/test/pylib/tablets.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/pylib/util.py b/test/pylib/util.py index 4df8a108825a..ae39f4395f09 100644 --- a/test/pylib/util.py +++ b/test/pylib/util.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from collections.abc import Coroutine import threading diff --git a/test/raft/discovery_test.cc b/test/raft/discovery_test.cc index ccf1389a6b2c..dbd1806de4c2 100644 --- a/test/raft/discovery_test.cc +++ b/test/raft/discovery_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE raft #include "test/raft/helpers.hh" diff --git a/test/raft/etcd_test.cc b/test/raft/etcd_test.cc index 8431a2b63dd5..a6de402c2399 100644 --- a/test/raft/etcd_test.cc +++ b/test/raft/etcd_test.cc @@ -17,7 +17,7 @@ * under the License. */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // Port of etcd Raft implementation unit tests diff --git a/test/raft/failure_detector_test.cc b/test/raft/failure_detector_test.cc index 23d2678f5f77..e6def1e61b84 100644 --- a/test/raft/failure_detector_test.cc +++ b/test/raft/failure_detector_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/raft/fsm_test.cc b/test/raft/fsm_test.cc index 662f1b73cf1b..22c477dc4121 100644 --- a/test/raft/fsm_test.cc +++ b/test/raft/fsm_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #define BOOST_TEST_MODULE raft diff --git a/test/raft/future_set.hh b/test/raft/future_set.hh index f95ebc6c5882..0c3fc3d4324d 100644 --- a/test/raft/future_set.hh +++ b/test/raft/future_set.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/raft/generator.hh b/test/raft/generator.hh index dfd00cbd9479..88b524597f83 100644 --- a/test/raft/generator.hh +++ b/test/raft/generator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/raft/helpers.cc b/test/raft/helpers.cc index eab2b63ea8fd..f7c2b626b0ca 100644 --- a/test/raft/helpers.cc +++ b/test/raft/helpers.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // diff --git a/test/raft/helpers.hh b/test/raft/helpers.hh index fbf04b9c75e3..cce976ae35e1 100644 --- a/test/raft/helpers.hh +++ b/test/raft/helpers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // diff --git a/test/raft/logical_timer.hh b/test/raft/logical_timer.hh index d019d4b76fd8..0310422c386a 100644 --- a/test/raft/logical_timer.hh +++ b/test/raft/logical_timer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/raft/many_test.cc b/test/raft/many_test.cc index b5368a5fe8ab..d1696e4514ce 100644 --- a/test/raft/many_test.cc +++ b/test/raft/many_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // Test Raft library with many candidates diff --git a/test/raft/raft_sys_table_storage_test.cc b/test/raft/raft_sys_table_storage_test.cc index 170a8efcf0ab..7f693583e420 100644 --- a/test/raft/raft_sys_table_storage_test.cc +++ b/test/raft/raft_sys_table_storage_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/raft/randomized_nemesis_test.cc b/test/raft/randomized_nemesis_test.cc index c9f52146cf11..8545846773fb 100644 --- a/test/raft/randomized_nemesis_test.cc +++ b/test/raft/randomized_nemesis_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/test/raft/replication.cc b/test/raft/replication.cc index eabb8449fe64..4db83a17c2a0 100644 --- a/test/raft/replication.cc +++ b/test/raft/replication.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #undef SEASTAR_TESTING_MAIN diff --git a/test/raft/replication.hh b/test/raft/replication.hh index 4de8590d90ce..15f0a09e1220 100644 --- a/test/raft/replication.hh +++ b/test/raft/replication.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/raft/replication_test.cc b/test/raft/replication_test.cc index 3f71a146b779..86459367c232 100644 --- a/test/raft/replication_test.cc +++ b/test/raft/replication_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "replication.hh" diff --git a/test/raft/ticker.hh b/test/raft/ticker.hh index e6432d1fac66..d5beb74bbc44 100644 --- a/test/raft/ticker.hh +++ b/test/raft/ticker.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/test/redis/conftest.py b/test/redis/conftest.py index de30a7c8d2d3..4eb9cfbc253e 100644 --- a/test/redis/conftest.py +++ b/test/redis/conftest.py @@ -1,6 +1,6 @@ # Copyright 2020-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # This file contains "test fixtures", a pytest concept described in # https://docs.pytest.org/en/latest/fixture.html. diff --git a/test/redis/test_hashes.py b/test/redis/test_hashes.py index c4bc1171f2bf..87f5bceeaf19 100644 --- a/test/redis/test_hashes.py +++ b/test/redis/test_hashes.py @@ -1,5 +1,5 @@ # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/redis/test_raw_cmd.py b/test/redis/test_raw_cmd.py index 999276dbaa2b..4a8f24755e55 100644 --- a/test/redis/test_raw_cmd.py +++ b/test/redis/test_raw_cmd.py @@ -2,7 +2,7 @@ # Copyright (C) 2019-present ScyllaDB # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import socket diff --git a/test/redis/test_strings.py b/test/redis/test_strings.py index bdd1d934dc14..43f202347cc1 100644 --- a/test/redis/test_strings.py +++ b/test/redis/test_strings.py @@ -2,7 +2,7 @@ # Copyright (C) 2019 pengjian.uestc @ gmail.com # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/redis/util.py b/test/redis/util.py index f51c5964e65f..16ab0411c893 100644 --- a/test/redis/util.py +++ b/test/redis/util.py @@ -2,7 +2,7 @@ # Copyright (C) 2019 pengjian.uestc @ gmail.com # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import string diff --git a/test/rest_api/conftest.py b/test/rest_api/conftest.py index c1895f755bcb..84dda82925f1 100644 --- a/test/rest_api/conftest.py +++ b/test/rest_api/conftest.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use. A "fixture" is some diff --git a/test/rest_api/test_column_family.py b/test/rest_api/test_column_family.py index 45a1b31b8915..591bda2b590b 100644 --- a/test/rest_api/test_column_family.py +++ b/test/rest_api/test_column_family.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest import sys diff --git a/test/rest_api/test_compaction_manager.py b/test/rest_api/test_compaction_manager.py index 0fe488d5096c..034f61b6535a 100644 --- a/test/rest_api/test_compaction_manager.py +++ b/test/rest_api/test_compaction_manager.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest import sys diff --git a/test/rest_api/test_compactionhistory.py b/test/rest_api/test_compactionhistory.py index 6cf9c60175df..7ed28efc556c 100644 --- a/test/rest_api/test_compactionhistory.py +++ b/test/rest_api/test_compactionhistory.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import requests import sys diff --git a/test/rest_api/test_failure_detector.py b/test/rest_api/test_failure_detector.py index 2e35da646f27..d7f03978e5f7 100644 --- a/test/rest_api/test_failure_detector.py +++ b/test/rest_api/test_failure_detector.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import re import time diff --git a/test/rest_api/test_gossiper.py b/test/rest_api/test_gossiper.py index 5c108e58af2e..0f88e38485ca 100644 --- a/test/rest_api/test_gossiper.py +++ b/test/rest_api/test_gossiper.py @@ -1,6 +1,6 @@ # Copyright 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest import sys diff --git a/test/rest_api/test_storage_service.py b/test/rest_api/test_storage_service.py index d61a2779d7cd..711df2bdcb6e 100644 --- a/test/rest_api/test_storage_service.py +++ b/test/rest_api/test_storage_service.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest import sys diff --git a/test/rest_api/test_system.py b/test/rest_api/test_system.py index 322dca8c3db9..06469db2f136 100644 --- a/test/rest_api/test_system.py +++ b/test/rest_api/test_system.py @@ -1,6 +1,6 @@ # Copyright 2021-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 def test_system_uptime_ms(rest_api): resp = rest_api.send('GET', "system/uptime_ms") diff --git a/test/scylla_gdb/conftest.py b/test/scylla_gdb/conftest.py index a2570365cf39..e79d2511c663 100644 --- a/test/scylla_gdb/conftest.py +++ b/test/scylla_gdb/conftest.py @@ -1,6 +1,6 @@ # Copyright 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use. A "fixture" is some diff --git a/test/topology/conftest.py b/test/topology/conftest.py index b23c12ef5799..6fd7d3745db1 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use diff --git a/test/topology/test_automatic_cleanup.py b/test/topology/test_automatic_cleanup.py index 619130547690..8ad8f7838960 100644 --- a/test/topology/test_automatic_cleanup.py +++ b/test/topology/test_automatic_cleanup.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient from test.pylib.scylla_cluster import ReplaceConfig diff --git a/test/topology/test_change_ip.py b/test/topology/test_change_ip.py index 2ce18711d17e..fff5326dd37d 100644 --- a/test/topology/test_change_ip.py +++ b/test/topology/test_change_ip.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test clusters can restart fine after an IP address change. diff --git a/test/topology/test_cluster_features.py b/test/topology/test_cluster_features.py index ccfa5785f4cc..91b236bf99e2 100644 --- a/test/topology/test_cluster_features.py +++ b/test/topology/test_cluster_features.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Tests the cluster feature functionality. diff --git a/test/topology/test_concurrent_schema.py b/test/topology/test_concurrent_schema.py index b6e8dbc4932c..3c26dec33ae2 100644 --- a/test/topology/test_concurrent_schema.py +++ b/test/topology/test_concurrent_schema.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import logging diff --git a/test/topology/test_coordinator_queue_management.py b/test/topology/test_coordinator_queue_management.py index 2f1caded23f5..65ab64ca9ebd 100644 --- a/test/topology/test_coordinator_queue_management.py +++ b/test/topology/test_coordinator_queue_management.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient from test.pylib.util import wait_for_first_completed diff --git a/test/topology/test_global_ignore_nodes.py b/test/topology/test_global_ignore_nodes.py index ba6c9a7e3223..e2a549f19c9f 100644 --- a/test/topology/test_global_ignore_nodes.py +++ b/test/topology/test_global_ignore_nodes.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient from test.pylib.scylla_cluster import ReplaceConfig diff --git a/test/topology/test_gossiper.py b/test/topology/test_gossiper.py index 33e040306669..9af2d3912f96 100644 --- a/test/topology/test_gossiper.py +++ b/test/topology/test_gossiper.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology/test_mutation_schema_change.py b/test/topology/test_mutation_schema_change.py index 3e2a18e3deea..6705fe58e70e 100644 --- a/test/topology/test_mutation_schema_change.py +++ b/test/topology/test_mutation_schema_change.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Reproducer for a failure during lwt operation due to missing of a column mapping in schema history table. diff --git a/test/topology/test_mv.py b/test/topology/test_mv.py index c90552c031b8..40953308acd3 100644 --- a/test/topology/test_mv.py +++ b/test/topology/test_mv.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Tests for materialized views that need a three-node cluster. diff --git a/test/topology/test_random_tables.py b/test/topology/test_random_tables.py index 2cf92d39d1d8..d0c832c55781 100644 --- a/test/topology/test_random_tables.py +++ b/test/topology/test_random_tables.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import time diff --git a/test/topology/test_replace_alive_node.py b/test/topology/test_replace_alive_node.py index 619e3c447025..610847634db5 100644 --- a/test/topology/test_replace_alive_node.py +++ b/test/topology/test_replace_alive_node.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.scylla_cluster import ReplaceConfig from test.pylib.manager_client import ManagerClient diff --git a/test/topology/test_snapshot.py b/test/topology/test_snapshot.py index a705af645829..3ee2e9104562 100644 --- a/test/topology/test_snapshot.py +++ b/test/topology/test_snapshot.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test snapshot transfer by forcing threshold and performing schema changes diff --git a/test/topology/test_start_bootstrapped_with_invalid_seed.py b/test/topology/test_start_bootstrapped_with_invalid_seed.py index 44faf5e4a167..fc126e5d3db0 100644 --- a/test/topology/test_start_bootstrapped_with_invalid_seed.py +++ b/test/topology/test_start_bootstrapped_with_invalid_seed.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology/test_tls.py b/test/topology/test_tls.py index 77d752e30665..417f5649bfdc 100644 --- a/test/topology/test_tls.py +++ b/test/topology/test_tls.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology/test_topology_failure_recovery.py b/test/topology/test_topology_failure_recovery.py index 404b498d1a56..e49f8d57c7b1 100644 --- a/test/topology/test_topology_failure_recovery.py +++ b/test/topology/test_topology_failure_recovery.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient from test.pylib.internal_types import ServerInfo diff --git a/test/topology/test_topology_rejoin.py b/test/topology/test_topology_rejoin.py index afc25c3309e2..6a4327a808bc 100644 --- a/test/topology/test_topology_rejoin.py +++ b/test/topology/test_topology_rejoin.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test rejoin of a server after it was stopped suddenly (crash-like) diff --git a/test/topology/test_topology_remove_decom.py b/test/topology/test_topology_remove_decom.py index 617f84ed120e..d50c175277da 100644 --- a/test/topology/test_topology_remove_decom.py +++ b/test/topology/test_topology_remove_decom.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test consistency of schema changes with topology changes. diff --git a/test/topology/test_topology_schema.py b/test/topology/test_topology_schema.py index 4b7c4617b34b..6577da599d4f 100644 --- a/test/topology/test_topology_schema.py +++ b/test/topology/test_topology_schema.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test consistency of schema changes with server hard stop. diff --git a/test/topology/util.py b/test/topology/util.py index 61f173d5fb35..aa3e4769bcf5 100644 --- a/test/topology/util.py +++ b/test/topology/util.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test consistency of schema changes with topology changes. diff --git a/test/topology_custom/conftest.py b/test/topology_custom/conftest.py index 74f19717c3b4..374b0244b2e7 100644 --- a/test/topology_custom/conftest.py +++ b/test/topology_custom/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use diff --git a/test/topology_custom/test_boot_after_ip_change.py b/test/topology_custom/test_boot_after_ip_change.py index 2bcb8fe5cf45..713d61fb883c 100644 --- a/test/topology_custom/test_boot_after_ip_change.py +++ b/test/topology_custom/test_boot_after_ip_change.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import time import pytest diff --git a/test/topology_custom/test_change_replication_factor_1_to_0.py b/test/topology_custom/test_change_replication_factor_1_to_0.py index f5b10012e11a..937f9d29b4b4 100644 --- a/test/topology_custom/test_change_replication_factor_1_to_0.py +++ b/test/topology_custom/test_change_replication_factor_1_to_0.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging import pytest diff --git a/test/topology_custom/test_change_rpc_address.py b/test/topology_custom/test_change_rpc_address.py index edd3a951d915..222ec168ca8a 100644 --- a/test/topology_custom/test_change_rpc_address.py +++ b/test/topology_custom/test_change_rpc_address.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ diff --git a/test/topology_custom/test_commitlog.py b/test/topology_custom/test_commitlog.py index ee14b4b136a8..237da6c1915a 100644 --- a/test/topology_custom/test_commitlog.py +++ b/test/topology_custom/test_commitlog.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_commitlog_segment_data_resurrection.py b/test/topology_custom/test_commitlog_segment_data_resurrection.py index a9bd41cd51b8..acafe673aece 100644 --- a/test/topology_custom/test_commitlog_segment_data_resurrection.py +++ b/test/topology_custom/test_commitlog_segment_data_resurrection.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_compacting_reader_tombstone_gc.py b/test/topology_custom/test_compacting_reader_tombstone_gc.py index e2ff8af8787b..5c8597b1edf9 100644 --- a/test/topology_custom/test_compacting_reader_tombstone_gc.py +++ b/test/topology_custom/test_compacting_reader_tombstone_gc.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_data_resurrection_after_cleanup.py b/test/topology_custom/test_data_resurrection_after_cleanup.py index c72376526466..98c21fc87b80 100644 --- a/test/topology_custom/test_data_resurrection_after_cleanup.py +++ b/test/topology_custom/test_data_resurrection_after_cleanup.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_decommission.py b/test/topology_custom/test_decommission.py index df968090f3a3..bb8019cca1d1 100644 --- a/test/topology_custom/test_decommission.py +++ b/test/topology_custom/test_decommission.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging diff --git a/test/topology_custom/test_deprecating_cluster_features.py b/test/topology_custom/test_deprecating_cluster_features.py index efd64b7c3c49..e68cebdba878 100644 --- a/test/topology_custom/test_deprecating_cluster_features.py +++ b/test/topology_custom/test_deprecating_cluster_features.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import time diff --git a/test/topology_custom/test_different_group0_ids.py b/test/topology_custom/test_different_group0_ids.py index 4c667856f28d..9ba514b65d44 100644 --- a/test/topology_custom/test_different_group0_ids.py +++ b/test/topology_custom/test_different_group0_ids.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_gossiper_orphan_remover.py b/test/topology_custom/test_gossiper_orphan_remover.py index 492506966481..11a1a1ea64f4 100644 --- a/test/topology_custom/test_gossiper_orphan_remover.py +++ b/test/topology_custom/test_gossiper_orphan_remover.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_group0_schema_versioning.py b/test/topology_custom/test_group0_schema_versioning.py index 56d661981466..f1d399e27c88 100644 --- a/test/topology_custom/test_group0_schema_versioning.py +++ b/test/topology_custom/test_group0_schema_versioning.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import time diff --git a/test/topology_custom/test_hints.py b/test/topology_custom/test_hints.py index b794376888d4..bbb8cd209224 100644 --- a/test/topology_custom/test_hints.py +++ b/test/topology_custom/test_hints.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import pytest diff --git a/test/topology_custom/test_initial_token.py b/test/topology_custom/test_initial_token.py index fdbd9ddb5ae3..a67fc33f7666 100644 --- a/test/topology_custom/test_initial_token.py +++ b/test/topology_custom/test_initial_token.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/topology_custom/test_ip_mappings.py b/test/topology_custom/test_ip_mappings.py index 913a3aa61210..3d481921154f 100644 --- a/test/topology_custom/test_ip_mappings.py +++ b/test/topology_custom/test_ip_mappings.py @@ -1,6 +1,6 @@ # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_long_join.py b/test/topology_custom/test_long_join.py index c37fd8931744..9f8fa20d637c 100644 --- a/test/topology_custom/test_long_join.py +++ b/test/topology_custom/test_long_join.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/topology_custom/test_lwt_semaphore.py b/test/topology_custom/test_lwt_semaphore.py index 68a4d9cd1563..e1f4b6a00114 100644 --- a/test/topology_custom/test_lwt_semaphore.py +++ b/test/topology_custom/test_lwt_semaphore.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_maintenance_mode.py b/test/topology_custom/test_maintenance_mode.py index 7962ac0cdc17..9e83b8859e78 100644 --- a/test/topology_custom/test_maintenance_mode.py +++ b/test/topology_custom/test_maintenance_mode.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.protocol import ConfigurationException diff --git a/test/topology_custom/test_major_compaction.py b/test/topology_custom/test_major_compaction.py index 672c91de0c29..f004ad649b9d 100644 --- a/test/topology_custom/test_major_compaction.py +++ b/test/topology_custom/test_major_compaction.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/topology_custom/test_multidc.py b/test/topology_custom/test_multidc.py index 786128a81166..7e13059ab2ab 100644 --- a/test/topology_custom/test_multidc.py +++ b/test/topology_custom/test_multidc.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging import sys diff --git a/test/topology_custom/test_mv_admission_control.py b/test/topology_custom/test_mv_admission_control.py index ffa51c2db4ae..32eecf9e98e6 100644 --- a/test/topology_custom/test_mv_admission_control.py +++ b/test/topology_custom/test_mv_admission_control.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_mv_backlog.py b/test/topology_custom/test_mv_backlog.py index eb47f69f8a60..c3a5e2e0ce10 100644 --- a/test/topology_custom/test_mv_backlog.py +++ b/test/topology_custom/test_mv_backlog.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_mv_building.py b/test/topology_custom/test_mv_building.py index 29b288784e56..9ee3bc2e31c0 100644 --- a/test/topology_custom/test_mv_building.py +++ b/test/topology_custom/test_mv_building.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import pytest diff --git a/test/topology_custom/test_mv_delete_partitions.py b/test/topology_custom/test_mv_delete_partitions.py index 6364aa5973b9..35a2add62f72 100644 --- a/test/topology_custom/test_mv_delete_partitions.py +++ b/test/topology_custom/test_mv_delete_partitions.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_mv_fail_building.py b/test/topology_custom/test_mv_fail_building.py index 8661128bb2bd..21cf2dbbb5b6 100644 --- a/test/topology_custom/test_mv_fail_building.py +++ b/test/topology_custom/test_mv_fail_building.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import pytest diff --git a/test/topology_custom/test_mv_read_concurrency.py b/test/topology_custom/test_mv_read_concurrency.py index 2fad506a312d..f4d68cae49dd 100644 --- a/test/topology_custom/test_mv_read_concurrency.py +++ b/test/topology_custom/test_mv_read_concurrency.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_mv_tablets_empty_ip.py b/test/topology_custom/test_mv_tablets_empty_ip.py index 42cab2b70dad..a3d00f4a6fb2 100644 --- a/test/topology_custom/test_mv_tablets_empty_ip.py +++ b/test/topology_custom/test_mv_tablets_empty_ip.py @@ -2,7 +2,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import pytest diff --git a/test/topology_custom/test_mv_topology_change.py b/test/topology_custom/test_mv_topology_change.py index 7250aa758feb..f3654679cc7a 100644 --- a/test/topology_custom/test_mv_topology_change.py +++ b/test/topology_custom/test_mv_topology_change.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import pytest diff --git a/test/topology_custom/test_no_removed_node_event_on_ip_change.py b/test/topology_custom/test_no_removed_node_event_on_ip_change.py index 6cd08e2d602e..de3294c18694 100644 --- a/test/topology_custom/test_no_removed_node_event_on_ip_change.py +++ b/test/topology_custom/test_no_removed_node_event_on_ip_change.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging diff --git a/test/topology_custom/test_node_ops_metrics.py b/test/topology_custom/test_node_ops_metrics.py index 5c922372acfc..413b95889394 100644 --- a/test/topology_custom/test_node_ops_metrics.py +++ b/test/topology_custom/test_node_ops_metrics.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import pytest diff --git a/test/topology_custom/test_old_ip_notification_repro.py b/test/topology_custom/test_old_ip_notification_repro.py index 917deeddac80..3a1084d67346 100644 --- a/test/topology_custom/test_old_ip_notification_repro.py +++ b/test/topology_custom/test_old_ip_notification_repro.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging diff --git a/test/topology_custom/test_query_rebounce.py b/test/topology_custom/test_query_rebounce.py index fc4d6b4d3016..efc4046449c0 100644 --- a/test/topology_custom/test_query_rebounce.py +++ b/test/topology_custom/test_query_rebounce.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_raft_fix_broken_snapshot.py b/test/topology_custom/test_raft_fix_broken_snapshot.py index 89eaa29a00f5..4d3b0d69db39 100644 --- a/test/topology_custom/test_raft_fix_broken_snapshot.py +++ b/test/topology_custom/test_raft_fix_broken_snapshot.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/topology_custom/test_raft_no_quorum.py b/test/topology_custom/test_raft_no_quorum.py index ee5b3e4b868e..1294d71fd9fa 100644 --- a/test/topology_custom/test_raft_no_quorum.py +++ b/test/topology_custom/test_raft_no_quorum.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging diff --git a/test/topology_custom/test_raft_recovery_basic.py b/test/topology_custom/test_raft_recovery_basic.py index bebd8d5947ed..4dc413907ff2 100644 --- a/test/topology_custom/test_raft_recovery_basic.py +++ b/test/topology_custom/test_raft_recovery_basic.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_raft_recovery_majority_loss.py b/test/topology_custom/test_raft_recovery_majority_loss.py index 2c8b4b8a53a4..6110461cc355 100644 --- a/test/topology_custom/test_raft_recovery_majority_loss.py +++ b/test/topology_custom/test_raft_recovery_majority_loss.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_raft_recovery_stuck.py b/test/topology_custom/test_raft_recovery_stuck.py index f56c60b23bd8..2b7ef69d81b3 100644 --- a/test/topology_custom/test_raft_recovery_stuck.py +++ b/test/topology_custom/test_raft_recovery_stuck.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_raft_snapshot_request.py b/test/topology_custom/test_raft_snapshot_request.py index 29841be2dffe..f81373a7dee9 100644 --- a/test/topology_custom/test_raft_snapshot_request.py +++ b/test/topology_custom/test_raft_snapshot_request.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_raft_snapshot_truncation.py b/test/topology_custom/test_raft_snapshot_truncation.py index 85c756e90c91..dc7624860f9f 100644 --- a/test/topology_custom/test_raft_snapshot_truncation.py +++ b/test/topology_custom/test_raft_snapshot_truncation.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_read_repair.py b/test/topology_custom/test_read_repair.py index 2e52b393d03a..fe2f5abddc9a 100644 --- a/test/topology_custom/test_read_repair.py +++ b/test/topology_custom/test_read_repair.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import datetime diff --git a/test/topology_custom/test_remove_alive_node.py b/test/topology_custom/test_remove_alive_node.py index f6ebaaed7d9d..8f6cad73a10c 100644 --- a/test/topology_custom/test_remove_alive_node.py +++ b/test/topology_custom/test_remove_alive_node.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.scylla_cluster import ReplaceConfig from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_remove_rpc_client_with_pending_requests.py b/test/topology_custom/test_remove_rpc_client_with_pending_requests.py index b7ebaadde2a8..4516c36cc759 100644 --- a/test/topology_custom/test_remove_rpc_client_with_pending_requests.py +++ b/test/topology_custom/test_remove_rpc_client_with_pending_requests.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import logging diff --git a/test/topology_custom/test_repair.py b/test/topology_custom/test_repair.py index 8dce70c09193..612a451cc3fe 100644 --- a/test/topology_custom/test_repair.py +++ b/test/topology_custom/test_repair.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging diff --git a/test/topology_custom/test_replace.py b/test/topology_custom/test_replace.py index 8111c61e3ce5..093b07f6e275 100644 --- a/test/topology_custom/test_replace.py +++ b/test/topology_custom/test_replace.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test replacing node in different scenarios diff --git a/test/topology_custom/test_replace_ignore_nodes.py b/test/topology_custom/test_replace_ignore_nodes.py index de6737b01413..c336bdb29cd4 100644 --- a/test/topology_custom/test_replace_ignore_nodes.py +++ b/test/topology_custom/test_replace_ignore_nodes.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import time import pytest diff --git a/test/topology_custom/test_replace_with_encryption.py b/test/topology_custom/test_replace_with_encryption.py index 6a3fd302cc8d..7c49859c67c5 100644 --- a/test/topology_custom/test_replace_with_encryption.py +++ b/test/topology_custom/test_replace_with_encryption.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_replace_with_same_ip_twice.py b/test/topology_custom/test_replace_with_same_ip_twice.py index 08ff740a9dcf..cd08cb92d7f9 100644 --- a/test/topology_custom/test_replace_with_same_ip_twice.py +++ b/test/topology_custom/test_replace_with_same_ip_twice.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import time from test.pylib.scylla_cluster import ReplaceConfig diff --git a/test/topology_custom/test_reversed_queries_during_simulated_upgrade_process.py b/test/topology_custom/test_reversed_queries_during_simulated_upgrade_process.py index 6455f6f49aa9..c72624628460 100644 --- a/test/topology_custom/test_reversed_queries_during_simulated_upgrade_process.py +++ b/test/topology_custom/test_reversed_queries_during_simulated_upgrade_process.py @@ -1,6 +1,6 @@ # Copyright 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import asyncio import pytest from itertools import zip_longest diff --git a/test/topology_custom/test_select_from_mutation_fragments.py b/test/topology_custom/test_select_from_mutation_fragments.py index dea33fe56101..b002dc28365f 100644 --- a/test/topology_custom/test_select_from_mutation_fragments.py +++ b/test/topology_custom/test_select_from_mutation_fragments.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This file configures pytest for all tests in this directory, and also diff --git a/test/topology_custom/test_shutdown_hang.py b/test/topology_custom/test_shutdown_hang.py index 8cb8dc7f1730..74b42d40d30f 100644 --- a/test/topology_custom/test_shutdown_hang.py +++ b/test/topology_custom/test_shutdown_hang.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_custom/test_snitch_change.py b/test/topology_custom/test_snitch_change.py index 004ab9d01b57..47882fdbf58a 100644 --- a/test/topology_custom/test_snitch_change.py +++ b/test/topology_custom/test_snitch_change.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest diff --git a/test/topology_custom/test_table_desc_read_barrier.py b/test/topology_custom/test_table_desc_read_barrier.py index 58fdf2a17351..9df4807a48e4 100644 --- a/test/topology_custom/test_table_desc_read_barrier.py +++ b/test/topology_custom/test_table_desc_read_barrier.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging diff --git a/test/topology_custom/test_tablets.py b/test/topology_custom/test_tablets.py index 79172caca1df..5e0b6d89b44f 100644 --- a/test/topology_custom/test_tablets.py +++ b/test/topology_custom/test_tablets.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.protocol import ConfigurationException, InvalidRequest, SyntaxException from cassandra.query import SimpleStatement, ConsistencyLevel diff --git a/test/topology_custom/test_tablets_cql.py b/test/topology_custom/test_tablets_cql.py index 6c6c6920a213..5979e316eefd 100644 --- a/test/topology_custom/test_tablets_cql.py +++ b/test/topology_custom/test_tablets_cql.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio import logging diff --git a/test/topology_custom/test_tablets_migration.py b/test/topology_custom/test_tablets_migration.py index 678668f59f4e..486b93915563 100644 --- a/test/topology_custom/test_tablets_migration.py +++ b/test/topology_custom/test_tablets_migration.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.query import SimpleStatement, ConsistencyLevel from test.pylib.manager_client import ManagerClient diff --git a/test/topology_custom/test_topology_failure_recovery.py b/test/topology_custom/test_topology_failure_recovery.py index 37082fc29089..d69e917d175b 100644 --- a/test/topology_custom/test_topology_failure_recovery.py +++ b/test/topology_custom/test_topology_failure_recovery.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient from test.pylib.scylla_cluster import ReplaceConfig diff --git a/test/topology_custom/test_topology_remove_garbage_group0.py b/test/topology_custom/test_topology_remove_garbage_group0.py index e67fc260152f..8dac8d78e977 100644 --- a/test/topology_custom/test_topology_remove_garbage_group0.py +++ b/test/topology_custom/test_topology_remove_garbage_group0.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test removenode with node with node no longer member diff --git a/test/topology_custom/test_topology_smp.py b/test/topology_custom/test_topology_smp.py index 1251c9005422..cf50aca6fea2 100644 --- a/test/topology_custom/test_topology_smp.py +++ b/test/topology_custom/test_topology_smp.py @@ -1,7 +1,7 @@ # # Copyright (C) 2022-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test functionality on the cluster with different values of the --smp parameter on the nodes. diff --git a/test/topology_custom/test_truncate_with_tablets.py b/test/topology_custom/test_truncate_with_tablets.py index da500c75150c..e7042f525437 100644 --- a/test/topology_custom/test_truncate_with_tablets.py +++ b/test/topology_custom/test_truncate_with_tablets.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.query import SimpleStatement, ConsistencyLevel from cassandra.protocol import InvalidRequest diff --git a/test/topology_custom/test_view_build_status.py b/test/topology_custom/test_view_build_status.py index c5e0279abc13..2e070a6fe5f6 100644 --- a/test/topology_custom/test_view_build_status.py +++ b/test/topology_custom/test_view_build_status.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest import time diff --git a/test/topology_custom/test_writes_to_previous_cdc_generations.py b/test/topology_custom/test_writes_to_previous_cdc_generations.py index cc046ed5a549..7eb0ca0ba890 100644 --- a/test/topology_custom/test_writes_to_previous_cdc_generations.py +++ b/test/topology_custom/test_writes_to_previous_cdc_generations.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient, ServerInfo from test.pylib.util import wait_for, wait_for_cql_and_get_hosts diff --git a/test/topology_custom/test_zero_token_nodes_topology_ops.py b/test/topology_custom/test_zero_token_nodes_topology_ops.py index 5dcf0dc84cc8..68622de13665 100644 --- a/test/topology_custom/test_zero_token_nodes_topology_ops.py +++ b/test/topology_custom/test_zero_token_nodes_topology_ops.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest import logging diff --git a/test/topology_experimental_raft/conftest.py b/test/topology_experimental_raft/conftest.py index 74f19717c3b4..374b0244b2e7 100644 --- a/test/topology_experimental_raft/conftest.py +++ b/test/topology_experimental_raft/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use diff --git a/test/topology_experimental_raft/test_alternator.py b/test/topology_experimental_raft/test_alternator.py index 524b19d61b82..3f5287df6b3d 100644 --- a/test/topology_experimental_raft/test_alternator.py +++ b/test/topology_experimental_raft/test_alternator.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # Multi-node tests for Alternator. diff --git a/test/topology_experimental_raft/test_blocked_bootstrap.py b/test/topology_experimental_raft/test_blocked_bootstrap.py index 641ddfc35738..ac71c31b7465 100644 --- a/test/topology_experimental_raft/test_blocked_bootstrap.py +++ b/test/topology_experimental_raft/test_blocked_bootstrap.py @@ -1,6 +1,6 @@ # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.scylla_cluster import ReplaceConfig from test.pylib.manager_client import ManagerClient diff --git a/test/topology_experimental_raft/test_cdc_generation_clearing.py b/test/topology_experimental_raft/test_cdc_generation_clearing.py index a65c04dc3e91..d2d03a868dad 100644 --- a/test/topology_experimental_raft/test_cdc_generation_clearing.py +++ b/test/topology_experimental_raft/test_cdc_generation_clearing.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.rest_client import inject_error from test.pylib.manager_client import ManagerClient diff --git a/test/topology_experimental_raft/test_cdc_generation_publishing.py b/test/topology_experimental_raft/test_cdc_generation_publishing.py index 96e724f098c5..2df6fa982244 100644 --- a/test/topology_experimental_raft/test_cdc_generation_publishing.py +++ b/test/topology_experimental_raft/test_cdc_generation_publishing.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient, ServerInfo from test.pylib.rest_client import inject_error diff --git a/test/topology_experimental_raft/test_crash_coordinator_before_streaming.py b/test/topology_experimental_raft/test_crash_coordinator_before_streaming.py index b61b0337c0fd..004b46f5a82d 100644 --- a/test/topology_experimental_raft/test_crash_coordinator_before_streaming.py +++ b/test/topology_experimental_raft/test_crash_coordinator_before_streaming.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging import re diff --git a/test/topology_experimental_raft/test_fencing.py b/test/topology_experimental_raft/test_fencing.py index b88f71979037..34fe6f6670f8 100644 --- a/test/topology_experimental_raft/test_fencing.py +++ b/test/topology_experimental_raft/test_fencing.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient from test.pylib.random_tables import RandomTables, Column, IntType, CounterType diff --git a/test/topology_experimental_raft/test_mv_tablets.py b/test/topology_experimental_raft/test_mv_tablets.py index 621183b35c39..985f59c466f2 100644 --- a/test/topology_experimental_raft/test_mv_tablets.py +++ b/test/topology_experimental_raft/test_mv_tablets.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # Tests for interaction of materialized views with *tablets* diff --git a/test/topology_experimental_raft/test_mv_tablets_replace.py b/test/topology_experimental_raft/test_mv_tablets_replace.py index 86a3680c9ff8..c6bdd8128a44 100644 --- a/test/topology_experimental_raft/test_mv_tablets_replace.py +++ b/test/topology_experimental_raft/test_mv_tablets_replace.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from typing import List diff --git a/test/topology_experimental_raft/test_node_isolation.py b/test/topology_experimental_raft/test_node_isolation.py index 9bb06719b784..16518b89a984 100644 --- a/test/topology_experimental_raft/test_node_isolation.py +++ b/test/topology_experimental_raft/test_node_isolation.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging import pytest diff --git a/test/topology_experimental_raft/test_not_enough_token_owners.py b/test/topology_experimental_raft/test_not_enough_token_owners.py index b909b625f5ce..c3e3493ab383 100644 --- a/test/topology_experimental_raft/test_not_enough_token_owners.py +++ b/test/topology_experimental_raft/test_not_enough_token_owners.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest import logging diff --git a/test/topology_experimental_raft/test_raft_cluster_features.py b/test/topology_experimental_raft/test_raft_cluster_features.py index b4f5bd1ab3a8..d37fa239a048 100644 --- a/test/topology_experimental_raft/test_raft_cluster_features.py +++ b/test/topology_experimental_raft/test_raft_cluster_features.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Tests that are specific to the raft-based cluster feature implementation. diff --git a/test/topology_experimental_raft/test_raft_ignore_nodes.py b/test/topology_experimental_raft/test_raft_ignore_nodes.py index aea22d0a299e..6659e2001fc1 100644 --- a/test/topology_experimental_raft/test_raft_ignore_nodes.py +++ b/test/topology_experimental_raft/test_raft_ignore_nodes.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import time import pytest diff --git a/test/topology_experimental_raft/test_restart_cluster.py b/test/topology_experimental_raft/test_restart_cluster.py index f8566b70f020..fa01b4b2c4a6 100644 --- a/test/topology_experimental_raft/test_restart_cluster.py +++ b/test/topology_experimental_raft/test_restart_cluster.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # """ Test clusters can restart fine after all nodes are stopped gracefully diff --git a/test/topology_experimental_raft/test_tablet_repair_scheduler.py b/test/topology_experimental_raft/test_tablet_repair_scheduler.py index f3689d1ffdf8..b3eec4959edd 100644 --- a/test/topology_experimental_raft/test_tablet_repair_scheduler.py +++ b/test/topology_experimental_raft/test_tablet_repair_scheduler.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.manager_client import ManagerClient diff --git a/test/topology_experimental_raft/test_tablets.py b/test/topology_experimental_raft/test_tablets.py index db79e64a8114..acf2a0454fd0 100644 --- a/test/topology_experimental_raft/test_tablets.py +++ b/test/topology_experimental_raft/test_tablets.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.query import SimpleStatement, ConsistencyLevel diff --git a/test/topology_experimental_raft/test_tablets_intranode.py b/test/topology_experimental_raft/test_tablets_intranode.py index ca50dcb523b8..cefd63607fe0 100644 --- a/test/topology_experimental_raft/test_tablets_intranode.py +++ b/test/topology_experimental_raft/test_tablets_intranode.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.query import SimpleStatement, ConsistencyLevel from cassandra.cluster import Session, ConsistencyLevel diff --git a/test/topology_experimental_raft/test_tablets_merge.py b/test/topology_experimental_raft/test_tablets_merge.py index 3fa38afb6b10..2cdf0e0e5528 100644 --- a/test/topology_experimental_raft/test_tablets_merge.py +++ b/test/topology_experimental_raft/test_tablets_merge.py @@ -1,7 +1,7 @@ # # Copyright (C) 2025-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.query import SimpleStatement, ConsistencyLevel diff --git a/test/topology_experimental_raft/test_tablets_removenode.py b/test/topology_experimental_raft/test_tablets_removenode.py index b46ce44cd105..f34264519812 100644 --- a/test/topology_experimental_raft/test_tablets_removenode.py +++ b/test/topology_experimental_raft/test_tablets_removenode.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from cassandra.query import SimpleStatement, ConsistencyLevel # type: ignore diff --git a/test/topology_experimental_raft/test_tombstone_gc.py b/test/topology_experimental_raft/test_tombstone_gc.py index 3bf996369610..fda85ddcc7ee 100644 --- a/test/topology_experimental_raft/test_tombstone_gc.py +++ b/test/topology_experimental_raft/test_tombstone_gc.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio from datetime import datetime, timezone diff --git a/test/topology_experimental_raft/test_topology_ops.py b/test/topology_experimental_raft/test_topology_ops.py index 9b739b677044..22fe0f60288c 100644 --- a/test/topology_experimental_raft/test_topology_ops.py +++ b/test/topology_experimental_raft/test_topology_ops.py @@ -1,7 +1,7 @@ # # Copyright (C) 2023-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.pylib.scylla_cluster import ReplaceConfig from test.pylib.manager_client import ManagerClient diff --git a/test/topology_experimental_raft/test_topology_recovery_basic.py b/test/topology_experimental_raft/test_topology_recovery_basic.py index f920099fa656..5bd7d9c4a9f9 100644 --- a/test/topology_experimental_raft/test_topology_recovery_basic.py +++ b/test/topology_experimental_raft/test_topology_recovery_basic.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_experimental_raft/test_topology_recovery_majority_loss.py b/test/topology_experimental_raft/test_topology_recovery_majority_loss.py index ebe90e8b7d55..9edb6e9b2179 100644 --- a/test/topology_experimental_raft/test_topology_recovery_majority_loss.py +++ b/test/topology_experimental_raft/test_topology_recovery_majority_loss.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_experimental_raft/test_topology_upgrade.py b/test/topology_experimental_raft/test_topology_upgrade.py index 5ef80de6b82f..d71c2dcdd80c 100644 --- a/test/topology_experimental_raft/test_topology_upgrade.py +++ b/test/topology_experimental_raft/test_topology_upgrade.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_experimental_raft/test_topology_upgrade_not_stuck_after_recent_removal.py b/test/topology_experimental_raft/test_topology_upgrade_not_stuck_after_recent_removal.py index d36a40e3cf5c..563fdcb3d41f 100644 --- a/test/topology_experimental_raft/test_topology_upgrade_not_stuck_after_recent_removal.py +++ b/test/topology_experimental_raft/test_topology_upgrade_not_stuck_after_recent_removal.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/topology_experimental_raft/test_zero_token_nodes_multidc.py b/test/topology_experimental_raft/test_zero_token_nodes_multidc.py index 3c8d40f3381d..b5d833d1cf07 100644 --- a/test/topology_experimental_raft/test_zero_token_nodes_multidc.py +++ b/test/topology_experimental_raft/test_zero_token_nodes_multidc.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import logging import pytest diff --git a/test/topology_experimental_raft/test_zero_token_nodes_no_replication.py b/test/topology_experimental_raft/test_zero_token_nodes_no_replication.py index c183ab2e6217..381cbef6f181 100644 --- a/test/topology_experimental_raft/test_zero_token_nodes_no_replication.py +++ b/test/topology_experimental_raft/test_zero_token_nodes_no_replication.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import pytest import logging diff --git a/test/topology_random_failures/cluster_events.py b/test/topology_random_failures/cluster_events.py index 6c446b16be5d..b09a2f50d4e2 100644 --- a/test/topology_random_failures/cluster_events.py +++ b/test/topology_random_failures/cluster_events.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from __future__ import annotations diff --git a/test/topology_random_failures/conftest.py b/test/topology_random_failures/conftest.py index 53eb20ebd9ab..b8a860666bba 100644 --- a/test/topology_random_failures/conftest.py +++ b/test/topology_random_failures/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use diff --git a/test/topology_random_failures/error_injections.py b/test/topology_random_failures/error_injections.py index 99c2658bd77d..55fafab3da99 100644 --- a/test/topology_random_failures/error_injections.py +++ b/test/topology_random_failures/error_injections.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # - New items should be added to the end of the list diff --git a/test/topology_random_failures/test_random_failures.py b/test/topology_random_failures/test_random_failures.py index 72ec8df1b1cf..c74c1fc507e8 100644 --- a/test/topology_random_failures/test_random_failures.py +++ b/test/topology_random_failures/test_random_failures.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from __future__ import annotations diff --git a/test/topology_tasks/conftest.py b/test/topology_tasks/conftest.py index 53eb20ebd9ab..b8a860666bba 100644 --- a/test/topology_tasks/conftest.py +++ b/test/topology_tasks/conftest.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # # This file configures pytest for all tests in this directory, and also # defines common test fixtures for all of them to use diff --git a/test/topology_tasks/task_manager_client.py b/test/topology_tasks/task_manager_client.py index 135ea900a17a..00f5af3e8f4c 100644 --- a/test/topology_tasks/task_manager_client.py +++ b/test/topology_tasks/task_manager_client.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from test.topology_tasks.task_manager_types import TaskID, TaskStats, TaskStatus diff --git a/test/topology_tasks/task_manager_types.py b/test/topology_tasks/task_manager_types.py index f798f286b0bb..3726da72c93e 100644 --- a/test/topology_tasks/task_manager_types.py +++ b/test/topology_tasks/task_manager_types.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from enum import StrEnum diff --git a/test/topology_tasks/test_node_ops_tasks.py b/test/topology_tasks/test_node_ops_tasks.py index 261bfac26a84..dc2d2ea742a0 100644 --- a/test/topology_tasks/test_node_ops_tasks.py +++ b/test/topology_tasks/test_node_ops_tasks.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # from functools import partial diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index 3636bfc584c2..fec3a8c75bfc 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -1,7 +1,7 @@ # # Copyright (C) 2024-present ScyllaDB # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # import asyncio diff --git a/test/unit/cross_shard_barrier_test.cc b/test/unit/cross_shard_barrier_test.cc index de0dbda0ef05..482f4020e458 100644 --- a/test/unit/cross_shard_barrier_test.cc +++ b/test/unit/cross_shard_barrier_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/unit/lsa_async_eviction_test.cc b/test/unit/lsa_async_eviction_test.cc index 62b20db50582..4a7fc8a834a2 100644 --- a/test/unit/lsa_async_eviction_test.cc +++ b/test/unit/lsa_async_eviction_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/unit/lsa_sync_eviction_test.cc b/test/unit/lsa_sync_eviction_test.cc index a95a446fb2ef..dfbcf6dab7f4 100644 --- a/test/unit/lsa_sync_eviction_test.cc +++ b/test/unit/lsa_sync_eviction_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/unit/row_cache_alloc_stress_test.cc b/test/unit/row_cache_alloc_stress_test.cc index f49a6f2da87a..6531e5e7edf5 100644 --- a/test/unit/row_cache_alloc_stress_test.cc +++ b/test/unit/row_cache_alloc_stress_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/test/unit/row_cache_stress_test.cc b/test/unit/row_cache_stress_test.cc index cee7d1588c47..2f292a8dcfd7 100644 --- a/test/unit/row_cache_stress_test.cc +++ b/test/unit/row_cache_stress_test.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/timeout_config.cc b/timeout_config.cc index c07b162987ea..0f5641b0f9b3 100644 --- a/timeout_config.cc +++ b/timeout_config.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "timeout_config.hh" diff --git a/timeout_config.hh b/timeout_config.hh index 018ef07b577e..7d1b10b36bac 100644 --- a/timeout_config.hh +++ b/timeout_config.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/timestamp.hh b/timestamp.hh index ca6f5fc259a5..85be2782be64 100644 --- a/timestamp.hh +++ b/timestamp.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tombstone_gc-internals.hh b/tombstone_gc-internals.hh index 97c65d232530..a5282f6f43c9 100644 --- a/tombstone_gc-internals.hh +++ b/tombstone_gc-internals.hh @@ -1,5 +1,5 @@ // Copyright (C) 2024-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #include "tombstone_gc.hh" #include diff --git a/tombstone_gc.cc b/tombstone_gc.cc index 2b60bd3ef9c3..d4df8a6e95a8 100644 --- a/tombstone_gc.cc +++ b/tombstone_gc.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/tombstone_gc.hh b/tombstone_gc.hh index 88cd0eb5d246..878d512d3ec2 100644 --- a/tombstone_gc.hh +++ b/tombstone_gc.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tombstone_gc_extension.hh b/tombstone_gc_extension.hh index ea2fb6c168e4..715492a4979c 100644 --- a/tombstone_gc_extension.hh +++ b/tombstone_gc_extension.hh @@ -2,7 +2,7 @@ * Copyright 2021-present ScyllaDB */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tombstone_gc_options.cc b/tombstone_gc_options.cc index a5afb934bdda..4be8263ce3c9 100644 --- a/tombstone_gc_options.cc +++ b/tombstone_gc_options.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/tombstone_gc_options.hh b/tombstone_gc_options.hh index 61152bbcf489..9456f59ccf44 100644 --- a/tombstone_gc_options.hh +++ b/tombstone_gc_options.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/build_info.hh b/tools/build_info.hh index a1c057344cfa..3abe17d835e9 100644 --- a/tools/build_info.hh +++ b/tools/build_info.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/entry_point.hh b/tools/entry_point.hh index ddfb822fb05a..fe2ef0ee73b8 100644 --- a/tools/entry_point.hh +++ b/tools/entry_point.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/format_printers.hh b/tools/format_printers.hh index 2b2dcf261ab4..1f33e8f53ec1 100644 --- a/tools/format_printers.hh +++ b/tools/format_printers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/json_writer.hh b/tools/json_writer.hh index 0e1e5c679f3b..3a757601082f 100644 --- a/tools/json_writer.hh +++ b/tools/json_writer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/load_system_tablets.cc b/tools/load_system_tablets.cc index 29f26519c5a0..21c53a81c04c 100644 --- a/tools/load_system_tablets.cc +++ b/tools/load_system_tablets.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "tools/load_system_tablets.hh" diff --git a/tools/load_system_tablets.hh b/tools/load_system_tablets.hh index 978364fe71a0..6ddfdb5ecfae 100644 --- a/tools/load_system_tablets.hh +++ b/tools/load_system_tablets.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/lua_sstable_consumer.cc b/tools/lua_sstable_consumer.cc index c02a35d0f583..2be97e117d06 100644 --- a/tools/lua_sstable_consumer.cc +++ b/tools/lua_sstable_consumer.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/tools/lua_sstable_consumer.hh b/tools/lua_sstable_consumer.hh index 4b47f2fbfe29..b89cac834fca 100644 --- a/tools/lua_sstable_consumer.hh +++ b/tools/lua_sstable_consumer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/read_mutation.cc b/tools/read_mutation.cc index 14b55808a8ae..8348c6568d37 100644 --- a/tools/read_mutation.cc +++ b/tools/read_mutation.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "tools/read_mutation.hh" diff --git a/tools/read_mutation.hh b/tools/read_mutation.hh index 0a37bbd7fda8..6515b449e56a 100644 --- a/tools/read_mutation.hh +++ b/tools/read_mutation.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/schema_loader.cc b/tools/schema_loader.cc index ee80a2cafc63..c416938dd560 100644 --- a/tools/schema_loader.cc +++ b/tools/schema_loader.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/tools/schema_loader.hh b/tools/schema_loader.hh index a840c53d5d46..4c480311b387 100644 --- a/tools/schema_loader.hh +++ b/tools/schema_loader.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/scylla-nodetool.cc b/tools/scylla-nodetool.cc index b85210f043b1..bf1cf0a68294 100644 --- a/tools/scylla-nodetool.cc +++ b/tools/scylla-nodetool.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/tools/scylla-sstable-scripts/dump.lua b/tools/scylla-sstable-scripts/dump.lua index 1db8e0ad4089..1cd22b098e96 100644 --- a/tools/scylla-sstable-scripts/dump.lua +++ b/tools/scylla-sstable-scripts/dump.lua @@ -1,7 +1,7 @@ -- -- Copyright (C) 2022-present ScyllaDB -- --- SPDX-License-Identifier: AGPL-3.0-or-later +-- SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -- -- Dumps the content of the sstable(s). diff --git a/tools/scylla-sstable-scripts/find-incomplete-clustering-row-keys.lua b/tools/scylla-sstable-scripts/find-incomplete-clustering-row-keys.lua index 54c7f33a6012..c5197f67a143 100644 --- a/tools/scylla-sstable-scripts/find-incomplete-clustering-row-keys.lua +++ b/tools/scylla-sstable-scripts/find-incomplete-clustering-row-keys.lua @@ -1,7 +1,7 @@ -- -- Copyright (C) 2022-present ScyllaDB -- --- SPDX-License-Identifier: AGPL-3.0-or-later +-- SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -- -- Finds clustering rows which have incomplete (prefix) keys. diff --git a/tools/scylla-sstable-scripts/fragment-stats.lua b/tools/scylla-sstable-scripts/fragment-stats.lua index 9eda2290e122..f0b6ea277596 100644 --- a/tools/scylla-sstable-scripts/fragment-stats.lua +++ b/tools/scylla-sstable-scripts/fragment-stats.lua @@ -1,7 +1,7 @@ -- -- Copyright (C) 2022-present ScyllaDB -- --- SPDX-License-Identifier: AGPL-3.0-or-later +-- SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -- -- Creates simple statistics of the fragments in the sstable diff --git a/tools/scylla-sstable-scripts/keys.lua b/tools/scylla-sstable-scripts/keys.lua index c9d58fb2f337..7c57ee719885 100644 --- a/tools/scylla-sstable-scripts/keys.lua +++ b/tools/scylla-sstable-scripts/keys.lua @@ -1,7 +1,7 @@ -- -- Copyright (C) 2024-present ScyllaDB -- --- SPDX-License-Identifier: AGPL-3.0-or-later +-- SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -- -- Dumps all keys from an sstable diff --git a/tools/scylla-sstable-scripts/largest-key.lua b/tools/scylla-sstable-scripts/largest-key.lua index 64a2b747dc0b..db501094a541 100644 --- a/tools/scylla-sstable-scripts/largest-key.lua +++ b/tools/scylla-sstable-scripts/largest-key.lua @@ -1,7 +1,7 @@ -- -- Copyright (C) 2024-present ScyllaDB -- --- SPDX-License-Identifier: AGPL-3.0-or-later +-- SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -- -- Find the largest key in an sstable diff --git a/tools/scylla-sstable-scripts/slice.lua b/tools/scylla-sstable-scripts/slice.lua index b212e7d1c7bc..c7b71d49055d 100644 --- a/tools/scylla-sstable-scripts/slice.lua +++ b/tools/scylla-sstable-scripts/slice.lua @@ -1,7 +1,7 @@ -- -- Copyright (C) 2022-present ScyllaDB -- --- SPDX-License-Identifier: AGPL-3.0-or-later +-- SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -- -- Filters and dumps the content of the sstable(s). diff --git a/tools/scylla-sstable.cc b/tools/scylla-sstable.cc index 58b4313d9656..31f33c3c01c2 100644 --- a/tools/scylla-sstable.cc +++ b/tools/scylla-sstable.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/tools/scylla-types.cc b/tools/scylla-types.cc index 1f7f1b81d6b3..394f900c3d1b 100644 --- a/tools/scylla-types.cc +++ b/tools/scylla-types.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/tools/sstable_consumer.hh b/tools/sstable_consumer.hh index dd45ae1e06a7..cd8546efe523 100644 --- a/tools/sstable_consumer.hh +++ b/tools/sstable_consumer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tools/testing/dist-check/dist-check.sh b/tools/testing/dist-check/dist-check.sh index 31b6d2bd83ee..6f5f61d5508d 100755 --- a/tools/testing/dist-check/dist-check.sh +++ b/tools/testing/dist-check/dist-check.sh @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # PROGRAM=$(basename $0) diff --git a/tools/testing/dist-check/docker.io/rockylinux-9.sh b/tools/testing/dist-check/docker.io/rockylinux-9.sh index dddcc9b79058..4c8ae3483051 100755 --- a/tools/testing/dist-check/docker.io/rockylinux-9.sh +++ b/tools/testing/dist-check/docker.io/rockylinux-9.sh @@ -5,7 +5,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # source "$(dirname $0)/util.sh" diff --git a/tools/testing/dist-check/docker.io/util.sh b/tools/testing/dist-check/docker.io/util.sh index e80087f42d15..3b8966ce10c8 100644 --- a/tools/testing/dist-check/docker.io/util.sh +++ b/tools/testing/dist-check/docker.io/util.sh @@ -3,7 +3,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # PROGRAM=$(basename $0) diff --git a/tools/utils.cc b/tools/utils.cc index 805e099ca1b1..c36526a0b1d3 100644 --- a/tools/utils.cc +++ b/tools/utils.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/tools/utils.hh b/tools/utils.hh index 75addb58e8ea..d110715ad44a 100644 --- a/tools/utils.hh +++ b/tools/utils.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tracing/trace_keyspace_helper.cc b/tracing/trace_keyspace_helper.cc index 221987a9bb88..286e5459b5f9 100644 --- a/tracing/trace_keyspace_helper.cc +++ b/tracing/trace_keyspace_helper.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include #include "types/types.hh" diff --git a/tracing/trace_keyspace_helper.hh b/tracing/trace_keyspace_helper.hh index 4a70fa8966c5..b56e6593b28f 100644 --- a/tracing/trace_keyspace_helper.hh +++ b/tracing/trace_keyspace_helper.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/tracing/trace_state.cc b/tracing/trace_state.cc index 8d1880e91952..36dfa0934ce2 100644 --- a/tracing/trace_state.cc +++ b/tracing/trace_state.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include #include "cql3/statements/prepared_statement.hh" diff --git a/tracing/trace_state.hh b/tracing/trace_state.hh index 1c6ea7a2ccf8..e37f6be3ba2f 100644 --- a/tracing/trace_state.hh +++ b/tracing/trace_state.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/tracing/traced_file.cc b/tracing/traced_file.cc index caf1186c85f5..65ac67de5bb5 100644 --- a/tracing/traced_file.cc +++ b/tracing/traced_file.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/tracing/traced_file.hh b/tracing/traced_file.hh index f64aa6ca3133..d2783b16f55f 100644 --- a/tracing/traced_file.hh +++ b/tracing/traced_file.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/tracing/tracing.cc b/tracing/tracing.cc index deddaa036573..d12d8af4ee9d 100644 --- a/tracing/tracing.cc +++ b/tracing/tracing.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include #include diff --git a/tracing/tracing.hh b/tracing/tracing.hh index 12cd40303144..66267bc05130 100644 --- a/tracing/tracing.hh +++ b/tracing/tracing.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/transport/controller.cc b/transport/controller.cc index 3a6045b39688..e3b3fd435a34 100644 --- a/transport/controller.cc +++ b/transport/controller.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/transport/controller.hh b/transport/controller.hh index e035e0ff6d28..30d02077b466 100644 --- a/transport/controller.hh +++ b/transport/controller.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/transport/cql_protocol_extension.cc b/transport/cql_protocol_extension.cc index 97282dd0e137..69629763cd68 100644 --- a/transport/cql_protocol_extension.cc +++ b/transport/cql_protocol_extension.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/transport/cql_protocol_extension.hh b/transport/cql_protocol_extension.hh index b2729557e0c9..af354b510b91 100644 --- a/transport/cql_protocol_extension.hh +++ b/transport/cql_protocol_extension.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/transport/event.cc b/transport/event.cc index b15994d1daa9..a97ba5d994da 100644 --- a/transport/event.cc +++ b/transport/event.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/transport/event.hh b/transport/event.hh index 80303e6e46ea..7e0fa3ce25eb 100644 --- a/transport/event.hh +++ b/transport/event.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/transport/event_notifier.cc b/transport/event_notifier.cc index fa9ece27be56..66f5d1c713b9 100644 --- a/transport/event_notifier.cc +++ b/transport/event_notifier.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "transport/server.hh" diff --git a/transport/messages/result_message.cc b/transport/messages/result_message.cc index 116210359381..228751923653 100644 --- a/transport/messages/result_message.cc +++ b/transport/messages/result_message.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "result_message.hh" diff --git a/transport/messages/result_message.hh b/transport/messages/result_message.hh index 467930bc2dc8..e1fdb77e5ce7 100644 --- a/transport/messages/result_message.hh +++ b/transport/messages/result_message.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/transport/messages/result_message_base.hh b/transport/messages/result_message_base.hh index c90ce12fe2f3..761c0424b51a 100644 --- a/transport/messages/result_message_base.hh +++ b/transport/messages/result_message_base.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/transport/messages_fwd.hh b/transport/messages_fwd.hh index 1103ae1ce747..740bd34168e2 100644 --- a/transport/messages_fwd.hh +++ b/transport/messages_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/transport/request.hh b/transport/request.hh index 5b126ec5c6b9..5063637d7c33 100644 --- a/transport/request.hh +++ b/transport/request.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/transport/response.hh b/transport/response.hh index 81823a30e866..694a521342a8 100644 --- a/transport/response.hh +++ b/transport/response.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/transport/server.cc b/transport/server.cc index 853476d542b5..e4e863b6c077 100644 --- a/transport/server.cc +++ b/transport/server.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "server.hh" diff --git a/transport/server.hh b/transport/server.hh index 52cf61b050e1..527ea508e744 100644 --- a/transport/server.hh +++ b/transport/server.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/collection.hh b/types/collection.hh index 845d91f4465d..9dcdf59dce3a 100644 --- a/types/collection.hh +++ b/types/collection.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/list.hh b/types/list.hh index c1f547d8139f..9f575f1380dc 100644 --- a/types/list.hh +++ b/types/list.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/listlike_partial_deserializing_iterator.hh b/types/listlike_partial_deserializing_iterator.hh index 6426664ebd8d..069a9c853076 100644 --- a/types/listlike_partial_deserializing_iterator.hh +++ b/types/listlike_partial_deserializing_iterator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/map.hh b/types/map.hh index a55358566def..b09d9b242366 100644 --- a/types/map.hh +++ b/types/map.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/set.hh b/types/set.hh index 18807bfdf1be..bd3ea8c4cb30 100644 --- a/types/set.hh +++ b/types/set.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/tuple.hh b/types/tuple.hh index c06e709539b3..f5bf61b2707c 100644 --- a/types/tuple.hh +++ b/types/tuple.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/types.cc b/types/types.cc index a42c40c1858e..62928c8543d9 100644 --- a/types/types.cc +++ b/types/types.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/types/types.hh b/types/types.hh index a203ef9f082b..bae9ac89b221 100644 --- a/types/types.hh +++ b/types/types.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/types/user.hh b/types/user.hh index e358511d1711..5c1856d61715 100644 --- a/types/user.hh +++ b/types/user.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/unified/build_unified.sh b/unified/build_unified.sh index 36d5b6676811..09b720b329f3 100755 --- a/unified/build_unified.sh +++ b/unified/build_unified.sh @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # print_usage() { diff --git a/unified/install.sh b/unified/install.sh index 7d8a6716546d..1a09ff67106a 100755 --- a/unified/install.sh +++ b/unified/install.sh @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # set -e diff --git a/unified/uninstall.sh b/unified/uninstall.sh index 13a1836a046e..5884b024dd80 100755 --- a/unified/uninstall.sh +++ b/unified/uninstall.sh @@ -4,7 +4,7 @@ # # -# SPDX-License-Identifier: AGPL-3.0-or-later +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 # set -e diff --git a/unimplemented.cc b/unimplemented.cc index 5b6f6f69f646..39f142e9aa6f 100644 --- a/unimplemented.cc +++ b/unimplemented.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/unimplemented.hh b/unimplemented.hh index 903994f3d7d9..b3e96c6c1253 100644 --- a/unimplemented.hh +++ b/unimplemented.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/UUID.hh b/utils/UUID.hh index 4aacc8f70629..f03e0e4efd30 100644 --- a/utils/UUID.hh +++ b/utils/UUID.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // This class is the parts of java.util.UUID that we need diff --git a/utils/UUID_gen.cc b/utils/UUID_gen.cc index 974c2aca6c25..00817553fbb8 100644 --- a/utils/UUID_gen.cc +++ b/utils/UUID_gen.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "UUID_gen.hh" diff --git a/utils/UUID_gen.hh b/utils/UUID_gen.hh index bec257ac296d..9be245ec0867 100644 --- a/utils/UUID_gen.hh +++ b/utils/UUID_gen.hh @@ -6,7 +6,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "utils/assert.hh" diff --git a/utils/abi/eh_ia64.hh b/utils/abi/eh_ia64.hh index 8713e8d33c79..947e0941ad3c 100644 --- a/utils/abi/eh_ia64.hh +++ b/utils/abi/eh_ia64.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/allocation_strategy.hh b/utils/allocation_strategy.hh index 3d088f60873f..e7f588eadc98 100644 --- a/utils/allocation_strategy.hh +++ b/utils/allocation_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/amortized_reserve.hh b/utils/amortized_reserve.hh index f3357357a921..5c15868ac529 100644 --- a/utils/amortized_reserve.hh +++ b/utils/amortized_reserve.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/anchorless_list.hh b/utils/anchorless_list.hh index 84f4b14e1972..b174f29aa801 100644 --- a/utils/anchorless_list.hh +++ b/utils/anchorless_list.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/array-search.cc b/utils/array-search.cc index 84713ac21d53..b0f88338552d 100644 --- a/utils/array-search.cc +++ b/utils/array-search.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/array-search.hh b/utils/array-search.hh index 0bafd9e9bd00..89db17ff9d73 100644 --- a/utils/array-search.hh +++ b/utils/array-search.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/ascii.cc b/utils/ascii.cc index 83400ac511e0..fcf11edef92d 100644 --- a/utils/ascii.cc +++ b/utils/ascii.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "ascii.hh" diff --git a/utils/ascii.hh b/utils/ascii.hh index 925e3dd5258d..eded0ccabb22 100644 --- a/utils/ascii.hh +++ b/utils/ascii.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/assert.hh b/utils/assert.hh index 9b733ec40353..6f6081bb7776 100644 --- a/utils/assert.hh +++ b/utils/assert.hh @@ -1,5 +1,5 @@ // Copyright 2024-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/utils/atomic_vector.hh b/utils/atomic_vector.hh index 4e8fe6b9f1cf..1cfb98ea2b63 100644 --- a/utils/atomic_vector.hh +++ b/utils/atomic_vector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/aws_sigv4.cc b/utils/aws_sigv4.cc index 014302e27564..e3797b5e7367 100644 --- a/utils/aws_sigv4.cc +++ b/utils/aws_sigv4.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/aws_sigv4.hh b/utils/aws_sigv4.hh index cb3683623ffc..c55d1eb73b5f 100644 --- a/utils/aws_sigv4.hh +++ b/utils/aws_sigv4.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/base64.cc b/utils/base64.cc index a9a23b936b2e..11523f6c5618 100644 --- a/utils/base64.cc +++ b/utils/base64.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "base64.hh" diff --git a/utils/base64.hh b/utils/base64.hh index cbd1eaf961c9..3b3ffc825d9d 100644 --- a/utils/base64.hh +++ b/utils/base64.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/big_decimal.cc b/utils/big_decimal.cc index 4dde78abe2b2..b83c86d18da6 100644 --- a/utils/big_decimal.cc +++ b/utils/big_decimal.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/utils/big_decimal.hh b/utils/big_decimal.hh index f670752738f4..e063de33f957 100644 --- a/utils/big_decimal.hh +++ b/utils/big_decimal.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/bit_cast.hh b/utils/bit_cast.hh index 28f2e47fee7e..1266f428ae4e 100644 --- a/utils/bit_cast.hh +++ b/utils/bit_cast.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/bloom_calculations.cc b/utils/bloom_calculations.cc index ffe5cf8e473f..9d01411c8ac3 100644 --- a/utils/bloom_calculations.cc +++ b/utils/bloom_calculations.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "bloom_calculations.hh" diff --git a/utils/bloom_calculations.hh b/utils/bloom_calculations.hh index ed05bc26ffbc..316b267d7a3e 100644 --- a/utils/bloom_calculations.hh +++ b/utils/bloom_calculations.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/bloom_filter.cc b/utils/bloom_filter.cc index 711cc33aafba..5d23567f3139 100644 --- a/utils/bloom_filter.cc +++ b/utils/bloom_filter.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "i_filter.hh" diff --git a/utils/bloom_filter.hh b/utils/bloom_filter.hh index ef92ca64a2da..2cbc720308ab 100644 --- a/utils/bloom_filter.hh +++ b/utils/bloom_filter.hh @@ -8,7 +8,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once #include "i_filter.hh" diff --git a/utils/bounded_stats_deque.hh b/utils/bounded_stats_deque.hh index 5e666e5a669f..9d41e6a859cc 100644 --- a/utils/bounded_stats_deque.hh +++ b/utils/bounded_stats_deque.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/bptree.hh b/utils/bptree.hh index b03324f5c437..c0d97b371865 100644 --- a/utils/bptree.hh +++ b/utils/bptree.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/buffer_input_stream.cc b/utils/buffer_input_stream.cc index f73bf966b963..dfeb425e23c4 100644 --- a/utils/buffer_input_stream.cc +++ b/utils/buffer_input_stream.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "buffer_input_stream.hh" diff --git a/utils/buffer_input_stream.hh b/utils/buffer_input_stream.hh index 6c5fcc1c5689..f656c5d9ae91 100644 --- a/utils/buffer_input_stream.hh +++ b/utils/buffer_input_stream.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/buffer_view-to-managed_bytes_view.hh b/utils/buffer_view-to-managed_bytes_view.hh index e1461d0562a6..5b06ef6c02ec 100644 --- a/utils/buffer_view-to-managed_bytes_view.hh +++ b/utils/buffer_view-to-managed_bytes_view.hh @@ -1,5 +1,5 @@ // Copyright 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/utils/cached_file.hh b/utils/cached_file.hh index e4bf8a6b2a32..334d86f83a73 100644 --- a/utils/cached_file.hh +++ b/utils/cached_file.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/cached_file_stats.hh b/utils/cached_file_stats.hh index 42ebbfb4ed8e..f571e99c8ba4 100644 --- a/utils/cached_file_stats.hh +++ b/utils/cached_file_stats.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/chunked_vector.hh b/utils/chunked_vector.hh index aa6b31076c3b..b13cc45eb7cb 100644 --- a/utils/chunked_vector.hh +++ b/utils/chunked_vector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/class_registrator.hh b/utils/class_registrator.hh index a6360dea0842..36884c453a59 100644 --- a/utils/class_registrator.hh +++ b/utils/class_registrator.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/clmul.hh b/utils/clmul.hh index 793d55a66386..0f7f9ee74aca 100644 --- a/utils/clmul.hh +++ b/utils/clmul.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 * */ diff --git a/utils/coarse_steady_clock.hh b/utils/coarse_steady_clock.hh index ea6ab7e4fff2..96dd665db0e7 100644 --- a/utils/coarse_steady_clock.hh +++ b/utils/coarse_steady_clock.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/collection-concepts.hh b/utils/collection-concepts.hh index 38c7dc388ca2..6f3747a1ea86 100644 --- a/utils/collection-concepts.hh +++ b/utils/collection-concepts.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/compact-radix-tree.hh b/utils/compact-radix-tree.hh index 61e9b0b9865d..5051b5ec44df 100644 --- a/utils/compact-radix-tree.hh +++ b/utils/compact-radix-tree.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/config_file.cc b/utils/config_file.cc index 3f4ad12f4f00..49595c90fe5b 100644 --- a/utils/config_file.cc +++ b/utils/config_file.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/config_file.hh b/utils/config_file.hh index 7216a380a2bf..771fa77e4a3d 100644 --- a/utils/config_file.hh +++ b/utils/config_file.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/config_file_impl.hh b/utils/config_file_impl.hh index 6307654641cd..2f4c1024248a 100644 --- a/utils/config_file_impl.hh +++ b/utils/config_file_impl.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/contiguous_shared_buffer.hh b/utils/contiguous_shared_buffer.hh index 7726cde77ca3..b5d47297d7e9 100644 --- a/utils/contiguous_shared_buffer.hh +++ b/utils/contiguous_shared_buffer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/coroutine.hh b/utils/coroutine.hh index c551ed6b132a..c220038a8bb7 100644 --- a/utils/coroutine.hh +++ b/utils/coroutine.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/crc.hh b/utils/crc.hh index d4cea52319c8..a8a4e3dcea69 100644 --- a/utils/crc.hh +++ b/utils/crc.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 * * A crc32 calculation for __PPC64__ uses the code from https://github.com/antonblanchard/crc32-vpmsum * written by Anton Blanchard , IBM diff --git a/utils/cross-shard-barrier.hh b/utils/cross-shard-barrier.hh index e058d6128362..5ddea916f79b 100644 --- a/utils/cross-shard-barrier.hh +++ b/utils/cross-shard-barrier.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/data_input.hh b/utils/data_input.hh index 30eeb1391a0f..8dbd80f3ed66 100644 --- a/utils/data_input.hh +++ b/utils/data_input.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef UTILS_DATA_INPUT_HH_ diff --git a/utils/digest_algorithm.hh b/utils/digest_algorithm.hh index b710d165540e..23b8d9ca68d9 100644 --- a/utils/digest_algorithm.hh +++ b/utils/digest_algorithm.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/digester.hh b/utils/digester.hh index fc9102191b2a..0d2185388e10 100644 --- a/utils/digester.hh +++ b/utils/digester.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/directories.cc b/utils/directories.cc index d172b3b26229..1b6f1299f091 100644 --- a/utils/directories.cc +++ b/utils/directories.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/directories.hh b/utils/directories.hh index 280b21fef3ff..8e717d2b024b 100644 --- a/utils/directories.hh +++ b/utils/directories.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/disk-error-handler.cc b/utils/disk-error-handler.cc index 2826aa73b99c..dd39d4b32b21 100644 --- a/utils/disk-error-handler.cc +++ b/utils/disk-error-handler.cc @@ -2,7 +2,7 @@ * Copyright 2016-present ScyllaDB **/ -/* SPDX-License-Identifier: AGPL-3.0-or-later +/* SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "db/extensions.hh" diff --git a/utils/disk-error-handler.hh b/utils/disk-error-handler.hh index c9b95fe1df7b..7e661d70fc02 100644 --- a/utils/disk-error-handler.hh +++ b/utils/disk-error-handler.hh @@ -2,7 +2,7 @@ * Copyright 2016-present ScyllaDB **/ -/* SPDX-License-Identifier: AGPL-3.0-or-later +/* SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/div_ceil.hh b/utils/div_ceil.hh index 3ed71499a40a..e0966afda813 100644 --- a/utils/div_ceil.hh +++ b/utils/div_ceil.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/double-decker.hh b/utils/double-decker.hh index 7e3850ba1fae..712019c2112c 100644 --- a/utils/double-decker.hh +++ b/utils/double-decker.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/dynamic_bitset.cc b/utils/dynamic_bitset.cc index 6fb98fb38dff..eef6813ecc92 100644 --- a/utils/dynamic_bitset.cc +++ b/utils/dynamic_bitset.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/dynamic_bitset.hh b/utils/dynamic_bitset.hh index da8a280d26aa..2493239e6399 100644 --- a/utils/dynamic_bitset.hh +++ b/utils/dynamic_bitset.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/entangled.hh b/utils/entangled.hh index bf8cdfcc52de..7227572d111a 100644 --- a/utils/entangled.hh +++ b/utils/entangled.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/enum_option.hh b/utils/enum_option.hh index 1204b7e7cbaf..fa1446ee1a55 100644 --- a/utils/enum_option.hh +++ b/utils/enum_option.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // TODO: upstream this to Boost. diff --git a/utils/error_injection.cc b/utils/error_injection.cc index a62ca77bea79..13016dcdfaf3 100644 --- a/utils/error_injection.cc +++ b/utils/error_injection.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/error_injection.hh" diff --git a/utils/error_injection.hh b/utils/error_injection.hh index 3cbb2472de78..125525cc2097 100644 --- a/utils/error_injection.hh +++ b/utils/error_injection.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/estimated_histogram.hh b/utils/estimated_histogram.hh index bc4eb370a1e5..e6ff598c97bc 100644 --- a/utils/estimated_histogram.hh +++ b/utils/estimated_histogram.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/exception_container.hh b/utils/exception_container.hh index 691dc0664af2..a994f66b76cf 100644 --- a/utils/exception_container.hh +++ b/utils/exception_container.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/exceptions.cc b/utils/exceptions.cc index 099d6107ef46..0a4b023e98c9 100644 --- a/utils/exceptions.cc +++ b/utils/exceptions.cc @@ -2,7 +2,7 @@ * Copyright 2015-present ScyllaDB */ -/* SPDX-License-Identifier: AGPL-3.0-or-later +/* SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/exceptions.hh b/utils/exceptions.hh index a8281b2701ac..950e54d11e32 100644 --- a/utils/exceptions.hh +++ b/utils/exceptions.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/exponential_backoff_retry.hh b/utils/exponential_backoff_retry.hh index 4ccb50d1aae0..9f26080b21ca 100644 --- a/utils/exponential_backoff_retry.hh +++ b/utils/exponential_backoff_retry.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/extremum_tracking.hh b/utils/extremum_tracking.hh index 7f18884d8379..c32344937ff3 100644 --- a/utils/extremum_tracking.hh +++ b/utils/extremum_tracking.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/file_lock.cc b/utils/file_lock.cc index 56cc14bce797..830e1498c13f 100644 --- a/utils/file_lock.cc +++ b/utils/file_lock.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/file_lock.hh b/utils/file_lock.hh index 559cead89462..95e3548dc04a 100644 --- a/utils/file_lock.hh +++ b/utils/file_lock.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/flush_queue.hh b/utils/flush_queue.hh index f8910873df34..5256541827c8 100644 --- a/utils/flush_queue.hh +++ b/utils/flush_queue.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/fragment_range.hh b/utils/fragment_range.hh index d233cdbf090d..6b3ccd36d7bf 100644 --- a/utils/fragment_range.hh +++ b/utils/fragment_range.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/fragmented_temporary_buffer.hh b/utils/fragmented_temporary_buffer.hh index 3e0cb93bc992..495f1f8e4427 100644 --- a/utils/fragmented_temporary_buffer.hh +++ b/utils/fragmented_temporary_buffer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/gz/crc_combine.cc b/utils/gz/crc_combine.cc index b28b0dc23960..936d72805a08 100644 --- a/utils/gz/crc_combine.cc +++ b/utils/gz/crc_combine.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 * */ diff --git a/utils/gz/crc_combine.hh b/utils/gz/crc_combine.hh index 30e1a7f09e6d..1eba3aa1e05b 100644 --- a/utils/gz/crc_combine.hh +++ b/utils/gz/crc_combine.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 * */ diff --git a/utils/gz/crc_combine_table.cc b/utils/gz/crc_combine_table.cc index 6439802a6b82..652711626114 100644 --- a/utils/gz/crc_combine_table.cc +++ b/utils/gz/crc_combine_table.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 * */ diff --git a/utils/gz/crc_combine_table.hh b/utils/gz/crc_combine_table.hh index 1f967a1c818e..ab1befdd5083 100644 --- a/utils/gz/crc_combine_table.hh +++ b/utils/gz/crc_combine_table.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 * */ diff --git a/utils/hash.hh b/utils/hash.hh index 18ff8e5ec84c..e2f1b44c807d 100644 --- a/utils/hash.hh +++ b/utils/hash.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #ifndef UTILS_HASH_HH_ diff --git a/utils/hashers.cc b/utils/hashers.cc index d89a8372d497..d66cf9fc2045 100644 --- a/utils/hashers.cc +++ b/utils/hashers.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/hashers.hh" diff --git a/utils/hashers.hh b/utils/hashers.hh index b0c2fb02fe99..91ce2f1c955d 100644 --- a/utils/hashers.hh +++ b/utils/hashers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/hashing.hh b/utils/hashing.hh index 29e1c09a1b2a..336f005fc85d 100644 --- a/utils/hashing.hh +++ b/utils/hashing.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/histogram.hh b/utils/histogram.hh index ffb1cc890a90..bddaf8cd8de7 100644 --- a/utils/histogram.hh +++ b/utils/histogram.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/histogram_metrics_helper.cc b/utils/histogram_metrics_helper.cc index 5b2f2f9469db..59813d741294 100644 --- a/utils/histogram_metrics_helper.cc +++ b/utils/histogram_metrics_helper.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "histogram_metrics_helper.hh" diff --git a/utils/histogram_metrics_helper.hh b/utils/histogram_metrics_helper.hh index ecf50def0992..2c1061c81aab 100644 --- a/utils/histogram_metrics_helper.hh +++ b/utils/histogram_metrics_helper.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/http.hh b/utils/http.hh index e7219c2a8881..bf12e007ec4c 100644 --- a/utils/http.hh +++ b/utils/http.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/human_readable.cc b/utils/human_readable.cc index 20eaddb5ddca..2fd189e4666f 100644 --- a/utils/human_readable.cc +++ b/utils/human_readable.cc @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/utils/human_readable.hh b/utils/human_readable.hh index 990143f93209..692909d80495 100644 --- a/utils/human_readable.hh +++ b/utils/human_readable.hh @@ -1,5 +1,5 @@ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/utils/i_filter.cc b/utils/i_filter.cc index a5fb73310e5c..9464a128cf55 100644 --- a/utils/i_filter.cc +++ b/utils/i_filter.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/log.hh" diff --git a/utils/i_filter.hh b/utils/i_filter.hh index 336c5f07cc43..2a39b912fda6 100644 --- a/utils/i_filter.hh +++ b/utils/i_filter.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/immutable-collection.hh b/utils/immutable-collection.hh index 25a917db2ce5..11e733fc1627 100644 --- a/utils/immutable-collection.hh +++ b/utils/immutable-collection.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/input_stream.hh b/utils/input_stream.hh index 8bea9ba359fd..eccac25e8f87 100644 --- a/utils/input_stream.hh +++ b/utils/input_stream.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/int_range.hh b/utils/int_range.hh index eeebc3c1fdac..c50a76aca053 100644 --- a/utils/int_range.hh +++ b/utils/int_range.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/intrusive-array.hh b/utils/intrusive-array.hh index 1236f2cce707..5ea84e33cded 100644 --- a/utils/intrusive-array.hh +++ b/utils/intrusive-array.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/intrusive_btree.hh b/utils/intrusive_btree.hh index 940f720175f9..1224c8cb1b8f 100644 --- a/utils/intrusive_btree.hh +++ b/utils/intrusive_btree.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/large_bitset.cc b/utils/large_bitset.cc index 950ee4b08323..9df5518810ce 100644 --- a/utils/large_bitset.cc +++ b/utils/large_bitset.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/utils/large_bitset.hh b/utils/large_bitset.hh index 010e0b87cb0f..453ba1ecf33f 100644 --- a/utils/large_bitset.hh +++ b/utils/large_bitset.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // A bitset containing a very large number of bits, so it uses fragmented diff --git a/utils/latency.hh b/utils/latency.hh index 15e5d309997e..6560400d901e 100644 --- a/utils/latency.hh +++ b/utils/latency.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/lexicographical_compare.hh b/utils/lexicographical_compare.hh index 36ba4ae0e3aa..03b45a68ed88 100644 --- a/utils/lexicographical_compare.hh +++ b/utils/lexicographical_compare.hh @@ -1,5 +1,5 @@ // Copyright 2023-present ScyllaDB -// SPDX-License-Identifier: AGPL-3.0-or-later +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 #pragma once diff --git a/utils/like_matcher.cc b/utils/like_matcher.cc index 8fe7c8300ca1..f2ce1f9fb225 100644 --- a/utils/like_matcher.cc +++ b/utils/like_matcher.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/like_matcher.hh b/utils/like_matcher.hh index 29b939b4e935..32ceb435272a 100644 --- a/utils/like_matcher.hh +++ b/utils/like_matcher.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/limiting_data_source.cc b/utils/limiting_data_source.cc index 7a94a70e5f7d..30e8c3b48f93 100644 --- a/utils/limiting_data_source.cc +++ b/utils/limiting_data_source.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "limiting_data_source.hh" diff --git a/utils/limiting_data_source.hh b/utils/limiting_data_source.hh index 57a7d12d6ecc..f8a7a785818c 100644 --- a/utils/limiting_data_source.hh +++ b/utils/limiting_data_source.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/linearizing_input_stream.hh b/utils/linearizing_input_stream.hh index 5b01fa2090ba..2433877f8a0e 100644 --- a/utils/linearizing_input_stream.hh +++ b/utils/linearizing_input_stream.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/lister.hh b/utils/lister.hh index faccc0bf8200..35fcca9b6c34 100644 --- a/utils/lister.hh +++ b/utils/lister.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/loading_cache.hh b/utils/loading_cache.hh index 11ff2345ced7..e5af274a0560 100644 --- a/utils/loading_cache.hh +++ b/utils/loading_cache.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/loading_shared_values.hh b/utils/loading_shared_values.hh index 4b3a1e030cf3..1ab5a41cc55a 100644 --- a/utils/loading_shared_values.hh +++ b/utils/loading_shared_values.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/log.hh b/utils/log.hh index 829d2e446098..4cb5bc2fcc83 100644 --- a/utils/log.hh +++ b/utils/log.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/log_heap.hh b/utils/log_heap.hh index f9a2f3e1412b..dd7125b69ee4 100644 --- a/utils/log_heap.hh +++ b/utils/log_heap.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/logalloc.cc b/utils/logalloc.cc index cfcb98a8b630..e84552b0d1f5 100644 --- a/utils/logalloc.cc +++ b/utils/logalloc.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/logalloc.hh b/utils/logalloc.hh index 8e8e3bdcf3a9..52937d5f20f0 100644 --- a/utils/logalloc.hh +++ b/utils/logalloc.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/lru.hh b/utils/lru.hh index 699bd22487fa..baa8681820b1 100644 --- a/utils/lru.hh +++ b/utils/lru.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/lsa/chunked_managed_vector.hh b/utils/lsa/chunked_managed_vector.hh index 883800471bed..e1f06a4b5119 100644 --- a/utils/lsa/chunked_managed_vector.hh +++ b/utils/lsa/chunked_managed_vector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/lsa/weak_ptr.hh b/utils/lsa/weak_ptr.hh index aef298f7ef68..297301aedf9b 100644 --- a/utils/lsa/weak_ptr.hh +++ b/utils/lsa/weak_ptr.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/managed_bytes.cc b/utils/managed_bytes.cc index 577bb54f0ed8..d78b4bd39a73 100644 --- a/utils/managed_bytes.cc +++ b/utils/managed_bytes.cc @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/managed_bytes.hh b/utils/managed_bytes.hh index 610236d31cf8..fe3f4f5475f2 100644 --- a/utils/managed_bytes.hh +++ b/utils/managed_bytes.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/managed_ref.hh b/utils/managed_ref.hh index ffd78eb090f5..439377d85965 100644 --- a/utils/managed_ref.hh +++ b/utils/managed_ref.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/managed_vector.hh b/utils/managed_vector.hh index cbb894841145..758c7e1a688a 100644 --- a/utils/managed_vector.hh +++ b/utils/managed_vector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/maybe_yield.hh b/utils/maybe_yield.hh index 0b69089702aa..b0d1b6b961d5 100644 --- a/utils/maybe_yield.hh +++ b/utils/maybe_yield.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/memory_data_sink.hh b/utils/memory_data_sink.hh index 917ffae68149..da0412be34de 100644 --- a/utils/memory_data_sink.hh +++ b/utils/memory_data_sink.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/memory_limit_reached.hh b/utils/memory_limit_reached.hh index cc644f682c88..804df5de710c 100644 --- a/utils/memory_limit_reached.hh +++ b/utils/memory_limit_reached.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/multiprecision_int.cc b/utils/multiprecision_int.cc index 4214902b837a..1f7a3f902088 100644 --- a/utils/multiprecision_int.cc +++ b/utils/multiprecision_int.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/multiprecision_int.hh b/utils/multiprecision_int.hh index cc689a2c99fb..6fc46aa4e70a 100644 --- a/utils/multiprecision_int.hh +++ b/utils/multiprecision_int.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/murmur_hash.cc b/utils/murmur_hash.cc index 9fc6d72b576a..62490fa0604c 100644 --- a/utils/murmur_hash.cc +++ b/utils/murmur_hash.cc @@ -6,7 +6,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "murmur_hash.hh" diff --git a/utils/murmur_hash.hh b/utils/murmur_hash.hh index 5a31ca486086..0ebb2029e559 100644 --- a/utils/murmur_hash.hh +++ b/utils/murmur_hash.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/mutable_view.hh b/utils/mutable_view.hh index eae9dd66e60e..dcd05c3651e5 100644 --- a/utils/mutable_view.hh +++ b/utils/mutable_view.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/neat-object-id.hh b/utils/neat-object-id.hh index 4def533f2fd6..872ada8566c4 100644 --- a/utils/neat-object-id.hh +++ b/utils/neat-object-id.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/observable.hh b/utils/observable.hh index bc1d58cc452b..855a11796a6b 100644 --- a/utils/observable.hh +++ b/utils/observable.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/on_internal_error.cc b/utils/on_internal_error.cc index 1f80057074c8..26eeec3e5c15 100644 --- a/utils/on_internal_error.cc +++ b/utils/on_internal_error.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/on_internal_error.hh b/utils/on_internal_error.hh index d217c9c619b6..1c10254f1afa 100644 --- a/utils/on_internal_error.hh +++ b/utils/on_internal_error.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // Seastar's on_internal_error() is a replacement for SCYLLA_ASSERT(). Instead of diff --git a/utils/overloaded_functor.hh b/utils/overloaded_functor.hh index f30bab2ba4a3..208e3ff5f570 100644 --- a/utils/overloaded_functor.hh +++ b/utils/overloaded_functor.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/phased_barrier.hh b/utils/phased_barrier.hh index 548a4db5240c..06a9ad994500 100644 --- a/utils/phased_barrier.hh +++ b/utils/phased_barrier.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/preempt.hh b/utils/preempt.hh index 513cd012951e..61b1519568dc 100644 --- a/utils/preempt.hh +++ b/utils/preempt.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/pretty_printers.cc b/utils/pretty_printers.cc index cd6ba10d64d8..700b10e89830 100644 --- a/utils/pretty_printers.cc +++ b/utils/pretty_printers.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/assert.hh" diff --git a/utils/pretty_printers.hh b/utils/pretty_printers.hh index df12e814f599..4709e5bf08fb 100644 --- a/utils/pretty_printers.hh +++ b/utils/pretty_printers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/rate_limiter.cc b/utils/rate_limiter.cc index cf9f76f673eb..20ff8f86f6da 100644 --- a/utils/rate_limiter.cc +++ b/utils/rate_limiter.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "rate_limiter.hh" diff --git a/utils/rate_limiter.hh b/utils/rate_limiter.hh index 8a4ac2199ac7..b54797f18890 100644 --- a/utils/rate_limiter.hh +++ b/utils/rate_limiter.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/recent_entries_map.hh b/utils/recent_entries_map.hh index 0418d87efbfb..30be0426936f 100644 --- a/utils/recent_entries_map.hh +++ b/utils/recent_entries_map.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/result.hh b/utils/result.hh index c1a547f73676..ba73bcf8d513 100644 --- a/utils/result.hh +++ b/utils/result.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/result_combinators.hh b/utils/result_combinators.hh index f793f4b64a71..be55e711f2b5 100644 --- a/utils/result_combinators.hh +++ b/utils/result_combinators.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/result_loop.hh b/utils/result_loop.hh index f041ace2d5d9..2cbebf33622a 100644 --- a/utils/result_loop.hh +++ b/utils/result_loop.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/result_try.hh b/utils/result_try.hh index 7226a1c80f22..c0813aa16e7c 100644 --- a/utils/result_try.hh +++ b/utils/result_try.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/reusable_buffer.hh b/utils/reusable_buffer.hh index c3a34d1d03c6..995c2c45f848 100644 --- a/utils/reusable_buffer.hh +++ b/utils/reusable_buffer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/rjson.cc b/utils/rjson.cc index b871f1baaa67..ce34920ecf58 100644 --- a/utils/rjson.cc +++ b/utils/rjson.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "rjson.hh" diff --git a/utils/rjson.hh b/utils/rjson.hh index b78eea6b4e2b..1a08f6292e4a 100644 --- a/utils/rjson.hh +++ b/utils/rjson.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/rpc_utils.hh b/utils/rpc_utils.hh index a4b8753c0088..1fb39027eb0c 100644 --- a/utils/rpc_utils.hh +++ b/utils/rpc_utils.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/runtime.cc b/utils/runtime.cc index 18da0c64ca68..a8b86fa47688 100644 --- a/utils/runtime.cc +++ b/utils/runtime.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/runtime.hh" diff --git a/utils/runtime.hh b/utils/runtime.hh index 46f01b123eac..ecce87b7683e 100644 --- a/utils/runtime.hh +++ b/utils/runtime.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/s3/aws_error.cc b/utils/s3/aws_error.cc index 132a49d7577b..db5a98556018 100644 --- a/utils/s3/aws_error.cc +++ b/utils/s3/aws_error.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #if __has_include() diff --git a/utils/s3/aws_error.hh b/utils/s3/aws_error.hh index 1622838344a1..9521d92cea44 100644 --- a/utils/s3/aws_error.hh +++ b/utils/s3/aws_error.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/s3/client.cc b/utils/s3/client.cc index deb887aa53b4..477ecb353069 100644 --- a/utils/s3/client.cc +++ b/utils/s3/client.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/s3/client.hh b/utils/s3/client.hh index 329aff9fe2ef..40752fc8cfef 100644 --- a/utils/s3/client.hh +++ b/utils/s3/client.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/s3/client_fwd.hh b/utils/s3/client_fwd.hh index 2f3664825fe0..3c208bfdd522 100644 --- a/utils/s3/client_fwd.hh +++ b/utils/s3/client_fwd.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/s3/creds.hh b/utils/s3/creds.hh index 4afe9dda0442..289ba29c1fab 100644 --- a/utils/s3/creds.hh +++ b/utils/s3/creds.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/s3/retry_strategy.cc b/utils/s3/retry_strategy.cc index 7bc347fa7620..155ee6f4529a 100644 --- a/utils/s3/retry_strategy.cc +++ b/utils/s3/retry_strategy.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "retry_strategy.hh" diff --git a/utils/s3/retry_strategy.hh b/utils/s3/retry_strategy.hh index fc53a92f8d12..0ffc4495c5ad 100644 --- a/utils/s3/retry_strategy.hh +++ b/utils/s3/retry_strategy.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/sequenced_set.hh b/utils/sequenced_set.hh index 19f2498408bc..bc358d8daf67 100644 --- a/utils/sequenced_set.hh +++ b/utils/sequenced_set.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/serialization.hh b/utils/serialization.hh index eacda2fb0de3..2a5f34082040 100644 --- a/utils/serialization.hh +++ b/utils/serialization.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // The following is a redesigned subset of Java's DataOutput, diff --git a/utils/serialized_action.hh b/utils/serialized_action.hh index 48f4b2a1e3a2..e1ce0c2d7285 100644 --- a/utils/serialized_action.hh +++ b/utils/serialized_action.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/simple_hashers.hh b/utils/simple_hashers.hh index 06aff002d394..07fd7e201630 100644 --- a/utils/simple_hashers.hh +++ b/utils/simple_hashers.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/small_vector.hh b/utils/small_vector.hh index c4afdc34e9e3..8ac37ffc5da3 100644 --- a/utils/small_vector.hh +++ b/utils/small_vector.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/sorting.hh b/utils/sorting.hh index da7b7dd9a98b..c842b82e299b 100644 --- a/utils/sorting.hh +++ b/utils/sorting.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/stall_free.hh b/utils/stall_free.hh index 04fb3fc655b5..5194cb6b7b48 100644 --- a/utils/stall_free.hh +++ b/utils/stall_free.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/streaming_histogram.hh b/utils/streaming_histogram.hh index 39bd7631cdf5..cd55ac281610 100644 --- a/utils/streaming_histogram.hh +++ b/utils/streaming_histogram.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/tagged_integer.hh b/utils/tagged_integer.hh index 26c47d35e2c3..dfdc40dbc680 100644 --- a/utils/tagged_integer.hh +++ b/utils/tagged_integer.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/to_string.cc b/utils/to_string.cc index 2360d528b7ef..3a4313ce61db 100644 --- a/utils/to_string.cc +++ b/utils/to_string.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "utils/to_string.hh" diff --git a/utils/to_string.hh b/utils/to_string.hh index f987cc259ff4..39b8da0f04e2 100644 --- a/utils/to_string.hh +++ b/utils/to_string.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/top_k.hh b/utils/top_k.hh index bf0739e4be97..dd0fe97ee0be 100644 --- a/utils/top_k.hh +++ b/utils/top_k.hh @@ -19,7 +19,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/tuple_utils.hh b/utils/tuple_utils.hh index 06a2106299d6..e8f74ee769ec 100644 --- a/utils/tuple_utils.hh +++ b/utils/tuple_utils.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/utils/unconst.hh b/utils/unconst.hh index 7aa617ac76f9..1dedd6dc1423 100644 --- a/utils/unconst.hh +++ b/utils/unconst.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/updateable_value.cc b/utils/updateable_value.cc index 426ff7d2aeb7..673c8b4d5bb4 100644 --- a/utils/updateable_value.cc +++ b/utils/updateable_value.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/updateable_value.hh b/utils/updateable_value.hh index 5b94613c8714..8945145217a8 100644 --- a/utils/updateable_value.hh +++ b/utils/updateable_value.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/user_provided_param.hh b/utils/user_provided_param.hh index f79dcd408656..08fdac8da358 100644 --- a/utils/user_provided_param.hh +++ b/utils/user_provided_param.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include diff --git a/utils/utf8.cc b/utils/utf8.cc index 087ff334032c..2ec45826c482 100644 --- a/utils/utf8.cc +++ b/utils/utf8.cc @@ -6,7 +6,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ /* diff --git a/utils/utf8.hh b/utils/utf8.hh index ad09cd478ac9..1a6b86b574fa 100644 --- a/utils/utf8.hh +++ b/utils/utf8.hh @@ -6,7 +6,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/uuid.cc b/utils/uuid.cc index 9817c3f46b8e..f5edc1ab60fb 100644 --- a/utils/uuid.cc +++ b/utils/uuid.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ diff --git a/utils/value_or_reference.hh b/utils/value_or_reference.hh index 4a5fe6aeb565..bdb5830252f7 100644 --- a/utils/value_or_reference.hh +++ b/utils/value_or_reference.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/variant_element.hh b/utils/variant_element.hh index b761bb2210bd..de0782130bf0 100644 --- a/utils/variant_element.hh +++ b/utils/variant_element.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/vle.hh b/utils/vle.hh index 0590c8373dd6..949b3ea66ac5 100644 --- a/utils/vle.hh +++ b/utils/vle.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/utils/xx_hasher.hh b/utils/xx_hasher.hh index 1345c388e919..446bcb78e2e3 100644 --- a/utils/xx_hasher.hh +++ b/utils/xx_hasher.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/validation.cc b/validation.cc index f9e4f5deecd7..1dadd0045791 100644 --- a/validation.cc +++ b/validation.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "validation.hh" diff --git a/validation.hh b/validation.hh index 95bba8521cd0..89c690f7303b 100644 --- a/validation.hh +++ b/validation.hh @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once diff --git a/version.hh b/version.hh index 18f27eb14911..67282088a0af 100644 --- a/version.hh +++ b/version.hh @@ -4,7 +4,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/view_info.hh b/view_info.hh index bce853d82369..49a02d333582 100644 --- a/view_info.hh +++ b/view_info.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #pragma once diff --git a/vint-serialization.cc b/vint-serialization.cc index b0117498a1c5..aa96bb3b8300 100644 --- a/vint-serialization.cc +++ b/vint-serialization.cc @@ -5,7 +5,7 @@ */ /* - * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) + * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #include "vint-serialization.hh" diff --git a/vint-serialization.hh b/vint-serialization.hh index f32092829dd4..003e02aa6198 100644 --- a/vint-serialization.hh +++ b/vint-serialization.hh @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ // diff --git a/zstd.cc b/zstd.cc index f5afb2d8d786..00c50ba2af80 100644 --- a/zstd.cc +++ b/zstd.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include From dc375b8cd3e8c7e85d5b9199cc4ff77d2c4bebfa Mon Sep 17 00:00:00 2001 From: Ferenc Szili Date: Thu, 12 Dec 2024 10:57:38 +0100 Subject: [PATCH 034/397] test: enable test_truncate_with_coordinator_crash This test was added in PR #19789 but was disabled with xfail because of the bug with way truncate saved the commit log replay positions. More specifically, the replay positions for shards that had no mutations were saved to system.truncated with shard_id == 0, regardless for which shard it was actually saved for (see #21719). The bug was fixed in #21722, so this change removes the xfail tag from the test. Closes scylladb/scylladb#21902 --- test/topology_custom/test_truncate_with_tablets.py | 1 - 1 file changed, 1 deletion(-) diff --git a/test/topology_custom/test_truncate_with_tablets.py b/test/topology_custom/test_truncate_with_tablets.py index e7042f525437..014ab30d5c2a 100644 --- a/test/topology_custom/test_truncate_with_tablets.py +++ b/test/topology_custom/test_truncate_with_tablets.py @@ -168,7 +168,6 @@ async def test_truncate_while_node_restart(manager: ManagerClient): assert row[0].count == 0 -@pytest.mark.xfail(reason="issue #21719") @pytest.mark.asyncio @skip_mode('release', 'error injections are not supported in release mode') async def test_truncate_with_coordinator_crash(manager: ManagerClient): From 2a31a82ae2dff0ed4859ec778a3c634e4d8197d4 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 2 Dec 2024 11:11:26 +0800 Subject: [PATCH 035/397] .github: Ensure header generation before include analysis When running clang-include-cleaner, the tool performs static analysis by "compiling" specified source files. Previously, non-existent included headers caused the tool to skip source files, reducing the effectiveness of unused include detection. Problem: - Header files like 'rust/wasmtime_bindings.hh' were not pre-generated - Compilation errors led to skipping source file analysis ``` /__w/scylladb/scylladb/lang/wasm.hh:15:10: fatal error: 'rust/wasmtime_bindings.hh' file not found 15 | #include "rust/wasmtime_bindings.hh" | ^~~~~~~~~~~~~~~~~~~~~~~~~~~ Skipping file /__w/scylladb/scylladb/lang/wasm.hh due to compiler errors. clang-include-cleaner expects to work on compilable source code. 1 error generated. ``` - This significantly reduced clang-include-cleaner's coverage Solution: - Build the `wasmtime_bindings` target to generate required header files - Ensure all necessary headers are created before running static analysis - Enable full source file checking for unused includes By generating headers before analysis, we prevent skipping of source files and improve the comprehensiveness of our include cleaner workflow. Signed-off-by: Kefu Chai Closes scylladb/scylladb#21739 --- .github/workflows/iwyu.yaml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.github/workflows/iwyu.yaml b/.github/workflows/iwyu.yaml index 10dbe62e7db5..1c11011a0c96 100644 --- a/.github/workflows/iwyu.yaml +++ b/.github/workflows/iwyu.yaml @@ -45,6 +45,10 @@ jobs: -G Ninja \ -B $BUILD_DIR \ -S . + - run: | + cmake \ + --build $BUILD_DIR \ + --target wasmtime_bindings - name: Build headers run: | swagger_targets='' From dcdc60fffde8e536c2406ccb8961cc60ff620d2d Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 19 Dec 2024 08:43:43 +0200 Subject: [PATCH 036/397] test: test_topology_compare_endpoints: use host_id:s This is the last call site requiring the compare_endpoints flavour for inet_address. Once this test is converted to use host_id:s instead, compare_endpoints and sort_by_proximity can be simplified to support only host_id:s. Signed-off-by: Benny Halevy --- locator/topology.hh | 2 +- test/boost/network_topology_strategy_test.cc | 14 ++++---------- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/locator/topology.hh b/locator/topology.hh index 9f89c80e7c38..51d1e34878dd 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -455,7 +455,7 @@ private: friend class token_metadata_impl; public: - void test_compare_endpoints(const inet_address& address, const inet_address& a1, const inet_address& a2) const; + void test_compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const; }; } // namespace locator diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index e2ff9a1aef1a..a7c9b00fe3c6 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -941,7 +941,7 @@ SEASTAR_TEST_CASE(test_invalid_dcs) { namespace locator { -void topology::test_compare_endpoints(const inet_address& address, const inet_address& a1, const inet_address& a2) const { +void topology::test_compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const { std::optional expected; const auto& loc = get_location(address); const auto& loc1 = get_location(a1); @@ -1012,7 +1012,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_compare_endpoints) { std::generate_n(std::back_inserter(nodes), NODES, [&, i = 0u]() mutable { return make_address(++i); }); - auto bogus_address = inet_address((127u << 24) | static_cast(NODES + 1)); semaphore sem(1); shared_token_metadata stm([&sem] () noexcept { return get_units(sem, 1); }, tm_cfg); @@ -1020,9 +1019,9 @@ SEASTAR_THREAD_TEST_CASE(test_topology_compare_endpoints) { auto& topo = tm.get_topology(); generate_topology(topo, datacenters, nodes); - const auto& address = tm.get_endpoint_for_host_id(nodes[tests::random::get_int(0, NODES-1)]); - const auto& a1 = tm.get_endpoint_for_host_id(nodes[tests::random::get_int(0, NODES-1)]); - const auto& a2 = tm.get_endpoint_for_host_id(nodes[tests::random::get_int(0, NODES-1)]); + const auto& address = nodes[tests::random::get_int(0, NODES-1)]; + const auto& a1 = nodes[tests::random::get_int(0, NODES-1)]; + const auto& a2 = nodes[tests::random::get_int(0, NODES-1)]; topo.test_compare_endpoints(address, address, address); topo.test_compare_endpoints(address, address, a1); @@ -1030,11 +1029,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_compare_endpoints) { topo.test_compare_endpoints(address, a1, a1); topo.test_compare_endpoints(address, a1, a2); topo.test_compare_endpoints(address, a2, a1); - - topo.test_compare_endpoints(bogus_address, bogus_address, bogus_address); - topo.test_compare_endpoints(address, bogus_address, bogus_address); - topo.test_compare_endpoints(address, a1, bogus_address); - topo.test_compare_endpoints(address, bogus_address, a2); return make_ready_future<>(); }).get(); } From 1c5b0eca41e706ec5fcb26ff72e55dba90cd3f50 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 19 Dec 2024 08:46:59 +0200 Subject: [PATCH 037/397] locator/topology: retire sort_by_proximity/compare_endpoints for inet_address Those are not used anymore now that the last call site for compare_endpoints by inet_address is converted to use host_id. Signed-off-by: Benny Halevy --- locator/topology.cc | 11 +---------- locator/topology.hh | 9 +-------- 2 files changed, 2 insertions(+), 18 deletions(-) diff --git a/locator/topology.cc b/locator/topology.cc index fb21706b0374..159fa797ac77 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -566,14 +566,6 @@ const endpoint_dc_rack& topology::get_location(const inet_address& ep) const { return endpoint_dc_rack::default_location; } -void topology::sort_by_proximity(inet_address address, inet_address_vector_replica_set& addresses) const { - if (_sort_by_proximity) { - std::sort(addresses.begin(), addresses.end(), [this, &address](inet_address& a1, inet_address& a2) { - return compare_endpoints(address, a1, a2) < 0; - }); - } -} - void topology::sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const { if (_sort_by_proximity) { std::sort(addresses.begin(), addresses.end(), [this, &address](locator::host_id& a1, locator::host_id& a2) { @@ -582,8 +574,7 @@ void topology::sort_by_proximity(locator::host_id address, host_id_vector_replic } } -template -std::weak_ordering topology::compare_endpoints(const T& address, const T& a1, const T& a2) const { +std::weak_ordering topology::compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const { const auto& loc = get_location(address); const auto& loc1 = get_location(a1); const auto& loc2 = get_location(a2); diff --git a/locator/topology.hh b/locator/topology.hh index 51d1e34878dd..c0d0ce1550cb 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -353,12 +353,6 @@ public: return std::count_if(endpoints.begin(), endpoints.end(), get_local_dc_filter()); } - /** - * This method will sort the List by proximity to the given - * address. - */ - void sort_by_proximity(inet_address address, inet_address_vector_replica_set& addresses) const; - /** * This method will sort the List by proximity to the given * host_id. @@ -418,8 +412,7 @@ private: * 2. Nodes in the same RACK as the reference node * 3. Nodes in the same DC as the reference node */ - template - std::weak_ordering compare_endpoints(const T& address, const T& a1, const T& a2) const; + std::weak_ordering compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const; unsigned _shard; config _cfg; From 67b7015cedbcb09c7760f6503892ce774be846b7 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 19 Dec 2024 09:32:09 +0200 Subject: [PATCH 038/397] test: network_topology_strategy_test: add test_topology_sort_by_proximity Before further changes are made to sort_by_proximity add a unit test for it. Signed-off-by: Benny Halevy --- locator/topology.hh | 1 + test/boost/network_topology_strategy_test.cc | 69 ++++++++++++++++++-- 2 files changed, 66 insertions(+), 4 deletions(-) diff --git a/locator/topology.hh b/locator/topology.hh index c0d0ce1550cb..6864f6de94d4 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -449,6 +449,7 @@ private: friend class token_metadata_impl; public: void test_compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const; + void test_sort_by_proximity(const locator::host_id& address, const host_id_vector_replica_set& nodes) const; }; } // namespace locator diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index a7c9b00fe3c6..22e79799a054 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -9,6 +9,7 @@ #include #include #include "gms/inet_address.hh" +#include "inet_address_vectors.hh" #include "locator/types.hh" #include "utils/assert.hh" #include "utils/UUID_gen.hh" @@ -21,6 +22,7 @@ #include "utils/log.hh" #include "gms/gossiper.hh" #include "schema/schema_builder.hh" +#include #include #include #include @@ -840,8 +842,7 @@ static void test_equivalence(const shared_token_metadata& stm, const locator::to } } - -void generate_topology(topology& topo, const std::unordered_map datacenters, const std::vector& nodes) { +void generate_topology(topology& topo, const std::unordered_map datacenters, const host_id_vector_replica_set& nodes) { auto& e1 = seastar::testing::local_random_engine; std::unordered_map racks_per_dc; @@ -885,7 +886,7 @@ SEASTAR_THREAD_TEST_CASE(testCalculateEndpoints) { { "rf5_2", 5 }, { "rf5_3", 5 }, }; - std::vector nodes; + host_id_vector_replica_set nodes; nodes.reserve(NODES); std::generate_n(std::back_inserter(nodes), NODES, [i = 0u]() mutable { return host_id{utils::UUID(0, ++i)}; @@ -982,6 +983,24 @@ void topology::test_compare_endpoints(const locator::host_id& address, const loc } } +void topology::test_sort_by_proximity(const locator::host_id& address, const host_id_vector_replica_set& nodes) const { + auto sorted_nodes = nodes; + sort_by_proximity(address, sorted_nodes); + std::unordered_set nodes_set(nodes.begin(), nodes.end()); + std::unordered_set sorted_nodes_set(sorted_nodes.begin(), sorted_nodes.end()); + // Test that no nodes were lost by sort_by_proximity + BOOST_REQUIRE_EQUAL(nodes_set, sorted_nodes_set); + // Verify that the reference address is sorted as first + // if it is part of the input vector + if (std::ranges::find(nodes, address) != nodes.end()) { + BOOST_REQUIRE_EQUAL(sorted_nodes[0], address); + } + // Test sort monotonicity + for (size_t i = 1; i < sorted_nodes.size(); ++i) { + BOOST_REQUIRE(compare_endpoints(address, sorted_nodes[i-1], sorted_nodes[i]) <= 0); + } +} + } // namespace locator SEASTAR_THREAD_TEST_CASE(test_topology_compare_endpoints) { @@ -997,7 +1016,7 @@ SEASTAR_THREAD_TEST_CASE(test_topology_compare_endpoints) { { "rf2", 2 }, { "rf3", 3 }, }; - std::vector nodes; + host_id_vector_replica_set nodes; nodes.reserve(NODES); auto make_address = [] (unsigned i) { @@ -1033,6 +1052,48 @@ SEASTAR_THREAD_TEST_CASE(test_topology_compare_endpoints) { }).get(); } +SEASTAR_THREAD_TEST_CASE(test_topology_sort_by_proximity) { + using map_type = std::unordered_map; + map_type datacenters; + size_t num_dcs = tests::random::get_int(1, 3); + for (size_t i = 0; i < num_dcs; ++i) { + size_t rf = tests::random::get_int(3, 5); + datacenters.emplace(format("dc{}", i), rf); + } + size_t num_nodes = std::ranges::fold_left(datacenters | std::views::transform(std::mem_fn(&map_type::value_type::second)), size_t(0), std::plus{}); + host_id_vector_replica_set nodes; + auto make_address = [] (unsigned i) { + return host_id{utils::UUID(0, i)}; + }; + nodes.reserve(num_nodes); + std::generate_n(std::back_inserter(nodes), num_nodes, [&, i = 0u]() mutable { + return make_address(++i); + }); + + locator::token_metadata::config tm_cfg; + auto my_address = gms::inet_address("localhost"); + tm_cfg.topo_cfg.this_endpoint = my_address; + tm_cfg.topo_cfg.this_cql_address = my_address; + tm_cfg.topo_cfg.this_host_id = nodes[0]; + tm_cfg.topo_cfg.local_dc_rack = locator::endpoint_dc_rack::default_location; + semaphore sem(1); + shared_token_metadata stm([&sem] () noexcept { return get_units(sem, 1); }, tm_cfg); + stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { + generate_topology(tm.get_topology(), datacenters, nodes); + return make_ready_future(); + }).get(); + + auto tmptr = stm.get(); + const auto& topology = stm.get()->get_topology(); + auto it = nodes.begin() + tests::random::get_int(0, num_nodes - 1); + auto address = *it; + topology.test_sort_by_proximity(address, nodes); + + // remove the reference node from the nodes list + nodes.erase(it); + topology.test_sort_by_proximity(address, nodes); +} + SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { inet_address ip1("192.168.0.1"); inet_address ip2("192.168.0.2"); From 37a25d3af40d622294e1718bdc85b27b59a93e53 Mon Sep 17 00:00:00 2001 From: Wojciech Mitros Date: Mon, 9 Dec 2024 13:09:59 +0100 Subject: [PATCH 039/397] mv: avoid stalls when calculating affected clustering ranges Currently, when finishing db::view::calculate_affected_clustering_ranges we deoverlap, transform and copy all ranges prepared before. This is all done within a single continuation and can cause stalls. We fix this by adding yields after each transform and moving elements to the final vector one by one instead of copying them all at the end. After this change, the longest continuation in this code will be deoverlapping the initial ranges (and one transform). While it has a relatively high computational complexity (we sort all ranges), it should execute quickly because we're operating on views there and we don't need to copy the actual bytes. If we encounter a stall there, we'll need to implement an asynchronous `deoverlap` method. Fixes scylladb/scylladb#21843 Closes scylladb/scylladb#21846 --- db/view/view.cc | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/db/view/view.cc b/db/view/view.cc index a199a6a28cd6..57ef4958d1ea 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -1674,13 +1674,14 @@ future calculate_affected_clustering_ranges(data_d // content, in case the view includes a column that is not included in // this mutation. - //FIXME: Unfortunate copy. - co_return interval::deoverlap(std::move(row_ranges), cmp) - | std::views::transform([] (auto&& v) { - return std::move(v).transform([] (auto&& ckv) { return clustering_key_prefix(ckv); }); - }) - | std::ranges::to(); - + query::clustering_row_ranges result_ranges; + auto deoverlapped_ranges = interval::deoverlap(std::move(row_ranges), cmp); + result_ranges.reserve(deoverlapped_ranges.size()); + for (auto&& r : deoverlapped_ranges) { + result_ranges.emplace_back(std::move(r).transform([] (auto&& ckv) { return clustering_key_prefix(ckv); })); + co_await coroutine::maybe_yield(); + } + co_return result_ranges; } bool needs_static_row(const mutation_partition& mp, const std::vector& views) { From a4440392d79d1ea23eefe4cf9312955f27531257 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Thu, 19 Dec 2024 14:07:27 +0200 Subject: [PATCH 040/397] build: update dependencies for features to be ported from enterprise ldap/slapd/toxiproxy/cyrus-sasl - for ldap authentication and authorization git-lfs/bolt - for profile-guided optimization lz4-static - for dictionary based network compression jwt - for Oauth/GCP connectivity (for key management) openkmip - for kmip testing fipscheck - for FIPS validation Frozen toolchain regenerated, with optimized clang from https://devpkg.scylladb.com/clang/clang-18.1.8-Fedora-40-aarch64.tar.gz https://devpkg.scylladb.com/clang/clang-18.1.8-Fedora-40-x86_64.tar.gz --- install-dependencies.sh | 24 ++++++++++++++++++++++++ tools/toolchain/image | 2 +- 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/install-dependencies.sh b/install-dependencies.sh index 77b34361048d..e820385be0ad 100755 --- a/install-dependencies.sh +++ b/install-dependencies.sh @@ -49,6 +49,9 @@ debian_base_packages=( librapidxml-dev libcrypto++-dev libxxhash-dev + slapd + ldap-utils + libcpp-jwt-dev ) fedora_packages=( @@ -68,6 +71,7 @@ fedora_packages=( systemd-devel cryptopp-devel git + git-lfs python sudo patchelf @@ -97,6 +101,7 @@ fedora_packages=( xxhash-devel makeself libzstd-static libzstd-devel + lz4-static lz4-devel rpm-build devscripts debhelper @@ -112,6 +117,17 @@ fedora_packages=( binaryen lcov java-11-openjdk-devel # for tools/java + + llvm-bolt + moreutils + iproute + llvm + openldap-servers + openldap-devel + toxiproxy + cyrus-sasl + fipscheck + cpp-jwt-devel ) # lld is not available on s390x, see @@ -142,6 +158,7 @@ declare -A pip_packages=( [treelib]= [allure-pytest]= [pytest-xdist]= + [pykmip]= ) pip_symlinks=( @@ -159,6 +176,9 @@ centos_packages=( scylla-python34-pyparsing20 systemd-devel pigz + openldap-servers + openldap-devel + cpp-jwt-devel ) # 1) glibc 2.30-3 has sys/sdt.h (systemtap include) @@ -316,8 +336,11 @@ if [ "$ID" = "ubuntu" ] || [ "$ID" = "debian" ]; then else apt-get -y install libsystemd-dev antlr3 libyaml-cpp-dev fi + apt-get -y install libssl-dev + echo -e "Configure example:\n\t./configure.py --enable-dpdk --mode=release --static-boost --static-yaml-cpp --compiler=/opt/scylladb/bin/g++-7 --cflags=\"-I/opt/scylladb/include -L/opt/scylladb/lib/x86-linux-gnu/\" --ldflags=\"-Wl,-rpath=/opt/scylladb/lib\"" elif [ "$ID" = "fedora" ]; then + fedora_packages+=(openssl-devel) if rpm -q --quiet yum-utils; then echo echo "This script will install dnf-utils package, witch will conflict with currently installed package: yum-utils" @@ -344,6 +367,7 @@ elif [ "$ID" = "fedora" ]; then fi fi elif [ "$ID" = "centos" ]; then + centos_packages+=(openssl-devel) dnf install -y "${centos_packages[@]}" echo -e "Configure example:\n\tpython3.4 ./configure.py --enable-dpdk --mode=release --static-boost --compiler=/opt/scylladb/bin/g++-7.3 --python python3.4 --ldflag=-Wl,-rpath=/opt/scylladb/lib64 --cflags=-I/opt/scylladb/include --with-antlr3=/opt/scylladb/bin/antlr3" elif [ "$ID" == "arch" ]; then diff --git a/tools/toolchain/image b/tools/toolchain/image index 85e10d163011..00ee4fe5356c 100644 --- a/tools/toolchain/image +++ b/tools/toolchain/image @@ -1 +1 @@ -docker.io/scylladb/scylla-toolchain:fedora-40-20241022 +docker.io/scylladb/scylla-toolchain:fedora-40-20241219 From 93be8f3a0cbc1013d79c388472fcedf45b882185 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Fri, 13 Dec 2024 12:24:53 +0800 Subject: [PATCH 041/397] db,sstables: migate boost::range::stable_partition to std library now that we are allowed to use C++23. we now have the luxury of using `std::ranges::stable_partition`. in this change, we: - replace `boost::range::stable_parition()` to `std::ranges::stable_parition()` - since `std::ranges::stable_parition()` returns a subrange instead of an iterator, change the names of variables which were previously used for holding the return value of `boost::range::stable_partition()` accordingly for better readability. - remove unused `#include` of boost headers Signed-off-by: Kefu Chai Closes scylladb/scylladb#21911 --- db/consistency_level.cc | 19 +++++++++---------- sstables/mx/writer.cc | 5 ++--- sstables/sstable_mutation_reader.cc | 3 +-- 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/db/consistency_level.cc b/db/consistency_level.cc index 84bc7d46ebc7..f86783381154 100644 --- a/db/consistency_level.cc +++ b/db/consistency_level.cc @@ -11,7 +11,6 @@ #include "db/consistency_level.hh" #include "db/consistency_level_validations.hh" -#include #include #include #include "exceptions/exceptions.hh" @@ -252,10 +251,10 @@ filter_for_query(consistency_level cl, if (read_repair == read_repair_decision::DC_LOCAL || is_datacenter_local(cl)) { const auto& topo = erm.get_topology(); - auto it = boost::range::stable_partition(live_endpoints, topo.get_local_dc_filter()); - local_count = std::distance(live_endpoints.begin(), it); + auto non_local_endpoints = std::ranges::stable_partition(live_endpoints, topo.get_local_dc_filter()); + local_count = std::distance(live_endpoints.begin(), non_local_endpoints.begin()); if (is_datacenter_local(cl)) { - live_endpoints.erase(it, live_endpoints.end()); + live_endpoints.erase(non_local_endpoints.begin(), non_local_endpoints.end()); } } @@ -275,19 +274,19 @@ filter_for_query(consistency_level cl, // selected this way aren't enough to satisfy CL requirements select the // remaining ones according to the load-balancing strategy as before. if (!preferred_endpoints.empty()) { - const auto it = boost::stable_partition(live_endpoints, [&preferred_endpoints] (const locator::host_id& a) { + const auto preferred = std::ranges::stable_partition(live_endpoints, [&preferred_endpoints] (const locator::host_id& a) { return std::find(preferred_endpoints.cbegin(), preferred_endpoints.cend(), a) == preferred_endpoints.end(); }); - const size_t selected = std::distance(it, live_endpoints.end()); + const size_t selected = std::ranges::distance(preferred); if (selected >= bf) { if (extra) { - *extra = selected == bf ? live_endpoints.front() : *(it + bf); + *extra = selected == bf ? live_endpoints.front() : *(preferred.begin() + bf); } - return host_id_vector_replica_set(it, it + bf); + return host_id_vector_replica_set(preferred.begin(), preferred.begin() + bf); } else if (selected) { selected_endpoints.reserve(bf); - std::move(it, live_endpoints.end(), std::back_inserter(selected_endpoints)); - live_endpoints.erase(it, live_endpoints.end()); + std::ranges::move(preferred, std::back_inserter(selected_endpoints)); + live_endpoints.erase(preferred.begin(), preferred.end()); } } diff --git a/sstables/mx/writer.cc b/sstables/mx/writer.cc index b331cfc4216c..7d0668835a93 100644 --- a/sstables/mx/writer.cc +++ b/sstables/mx/writer.cc @@ -24,7 +24,6 @@ #include #include #include -#include logging::logger slogger("mc_writer"); @@ -419,8 +418,8 @@ sstable_schema make_sstable_schema(const schema& s, const encoding_stats& enc_st // For static and regular columns, we write all simple columns first followed by collections // These containers have columns partitioned by atomicity auto pred = [] (const std::reference_wrapper& column) { return column.get().is_atomic(); }; - boost::range::stable_partition(sst_sch.regular_columns, pred); - boost::range::stable_partition(sst_sch.static_columns, pred); + std::ranges::stable_partition(sst_sch.regular_columns, pred); + std::ranges::stable_partition(sst_sch.static_columns, pred); return sst_sch; } diff --git a/sstables/sstable_mutation_reader.cc b/sstables/sstable_mutation_reader.cc index b569ccabb5e8..869d706f031e 100644 --- a/sstables/sstable_mutation_reader.cc +++ b/sstables/sstable_mutation_reader.cc @@ -13,7 +13,6 @@ #include "utils/fragment_range.hh" #include "utils/to_string.hh" -#include #include namespace sstables { @@ -155,7 +154,7 @@ std::vector column_translation::state::build( schema_mismatch }); } - boost::range::stable_partition(cols, [](const column_info& column) { return !column.is_collection; }); + std::ranges::stable_partition(cols, [](const column_info& column) { return !column.is_collection; }); } return cols; } From df36985fc3b2206e79bef121ff4a43cbe19ac6e8 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 9 Dec 2024 08:57:53 +0800 Subject: [PATCH 042/397] raft: do not include unused headers these unused includes are identified by clang-include-cleaner. after auditing the source files, all of the reports have been confirmed. Signed-off-by: Kefu Chai Closes scylladb/scylladb#21838 --- raft/logical_clock.hh | 2 +- raft/raft.hh | 3 --- raft/tracker.hh | 1 - 3 files changed, 1 insertion(+), 5 deletions(-) diff --git a/raft/logical_clock.hh b/raft/logical_clock.hh index 12db2fadac47..1646c4d34bda 100644 --- a/raft/logical_clock.hh +++ b/raft/logical_clock.hh @@ -8,7 +8,7 @@ #pragma once #include -#include +#include namespace raft { diff --git a/raft/raft.hh b/raft/raft.hh index 0e8144efc19d..65f3afd28146 100644 --- a/raft/raft.hh +++ b/raft/raft.hh @@ -8,15 +8,12 @@ #pragma once #include "utils/assert.hh" -#include #include #include #include #include -#include #include #include -#include #include #include "bytes_ostream.hh" #include "internal.hh" diff --git a/raft/tracker.hh b/raft/tracker.hh index c9c187c2b3aa..ea9016763ddf 100644 --- a/raft/tracker.hh +++ b/raft/tracker.hh @@ -8,7 +8,6 @@ #pragma once #include "utils/assert.hh" -#include #include #include "raft.hh" From f6ebd445e4e5cb3f826040dc3ee375466e220391 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Mon, 9 Sep 2024 11:09:54 +0200 Subject: [PATCH 043/397] test_tablets.py: limit concurrency in test_tablet_storage_freeing Apparently the python driver can't deal with the current concurrency sometimes. Lower it from 1000 to 100. Fixes scylladb/scylladb#20489 Closes scylladb/scylladb#20494 --- test/topology_experimental_raft/test_tablets.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/test/topology_experimental_raft/test_tablets.py b/test/topology_experimental_raft/test_tablets.py index acf2a0454fd0..441f11323428 100644 --- a/test/topology_experimental_raft/test_tablets.py +++ b/test/topology_experimental_raft/test_tablets.py @@ -23,7 +23,7 @@ from collections import defaultdict from collections.abc import Iterable from contextlib import asynccontextmanager - +import itertools logger = logging.getLogger(__name__) @@ -1294,7 +1294,10 @@ async def test_tablet_storage_freeing(manager: ManagerClient): await cql.run_async("CREATE TABLE test.test (pk int PRIMARY KEY, v text) WITH compression = {'sstable_compression': ''};") insert_stmt = cql.prepare("INSERT INTO test.test (pk, v) VALUES (?, ?);") payload = "a"*10000 - await asyncio.gather(*[cql.run_async(insert_stmt, [k, payload]) for k in range(n_partitions)]) + + max_concurrency = 100 + for batch in itertools.batched(range(n_partitions), max_concurrency): + await asyncio.gather(*[cql.run_async(insert_stmt, [k, payload]) for k in batch]) await manager.api.keyspace_flush(servers[0].ip_addr, "test") logger.info("Start second node.") From 16397d8cbabe37c6ae18727d62a328f9f4c9d03c Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Sat, 14 Dec 2024 10:15:44 +0800 Subject: [PATCH 044/397] message: do not include unused header In commit bfee93c7, repair verbs were moved to IDL. During this refactoring, the `gc_clock.hh` header became unused as its references were relocated. `clang-include-cleaner` helped identify this unnecessary include, which is now removed to clean up the codebase. Signed-off-by: Kefu Chai Closes scylladb/scylladb#21919 --- message/messaging_service.hh | 1 - 1 file changed, 1 deletion(-) diff --git a/message/messaging_service.hh b/message/messaging_service.hh index ce013ea1e368..ce81a2025e82 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -16,7 +16,6 @@ #include "gms/inet_address.hh" #include #include -#include "gc_clock.hh" #include "interval.hh" #include "schema/schema_fwd.hh" #include "streaming/stream_fwd.hh" From 6914892a1b3c95805f745b4d85e531852aa675b7 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 9 Dec 2024 10:28:02 +0800 Subject: [PATCH 045/397] repair: do not include unused headers these unused includes are identified by clang-include-cleaner. after auditing the source files, all of the reports have been confirmed. Signed-off-by: Kefu Chai Closes scylladb/scylladb#21837 --- repair/repair.hh | 7 ------- repair/row.hh | 1 - repair/row_level.hh | 2 -- repair/writer.hh | 1 - 4 files changed, 11 deletions(-) diff --git a/repair/repair.hh b/repair/repair.hh index 61efc6c2f49c..a65c4acc9299 100644 --- a/repair/repair.hh +++ b/repair/repair.hh @@ -8,19 +8,13 @@ #pragma once -#include - #include #include -#include #include -#include #include #include #include -#include -#include #include "gms/inet_address.hh" #include "locator/abstract_replication_strategy.hh" @@ -31,7 +25,6 @@ #include "utils/stall_free.hh" #include "repair/sync_boundary.hh" #include "tasks/types.hh" -#include "schema/schema.hh" #include "gms/gossip_address_map.hh" namespace tasks { diff --git a/repair/row.hh b/repair/row.hh index 8cd0c818723d..00a34fa9a8e3 100644 --- a/repair/row.hh +++ b/repair/row.hh @@ -10,7 +10,6 @@ #include #include "mutation/frozen_mutation.hh" #include -#include #include "repair/decorated_key_with_hash.hh" #include "repair/hash.hh" #include "repair/sync_boundary.hh" diff --git a/repair/row_level.hh b/repair/row_level.hh index decf12e5f6b3..8421287527e3 100644 --- a/repair/row_level.hh +++ b/repair/row_level.hh @@ -8,8 +8,6 @@ #pragma once -#include - #include #include "gms/gossip_address_map.hh" #include "gms/inet_address.hh" diff --git a/repair/writer.hh b/repair/writer.hh index b6d558470499..242e39149bc9 100644 --- a/repair/writer.hh +++ b/repair/writer.hh @@ -7,7 +7,6 @@ #include "streaming/stream_reason.hh" #include "repair/decorated_key_with_hash.hh" #include "readers/upgrading_consumer.hh" -#include using namespace seastar; From 24283d9dd0a1bcda86be5685440eba0990a98fb3 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 16 Dec 2024 12:37:30 +0800 Subject: [PATCH 046/397] test/topology: rename manager_internal to manager_client instead of reusing the variable name and overriding the parameter, use a new name for the return value of `manager_internal()` for better readability. Signed-off-by: Kefu Chai Closes scylladb/scylladb#21932 --- test/topology/conftest.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/test/topology/conftest.py b/test/topology/conftest.py index 6fd7d3745db1..65bf15aa48d0 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -187,9 +187,9 @@ async def manager(request, manager_internal, record_property, build_mode): # this should be consistent with scylla_cluster.py handler name in _before_test method test_py_log_test = suite_testpy_log.parent / f"{test_case_name}.log" - manager_internal = manager_internal() # set up client object in fixture with scope function - await manager_internal.before_test(test_case_name, test_log) - yield manager_internal + manager_client = manager_internal() # set up client object in fixture with scope function + await manager_client.before_test(test_case_name, test_log) + yield manager_client # `request.node.stash` contains a report stored in `pytest_runtest_makereport` from where we can retrieve # test failure. report = request.node.stash[PHASE_REPORT_KEY] @@ -200,7 +200,7 @@ async def manager(request, manager_internal, record_property, build_mode): # Then add property to the XML report with the path to the directory, so it can be visible in Jenkins failed_test_dir_path = tmp_dir / build_mode / "failed_test" / f"{test_case_name}" failed_test_dir_path.mkdir(parents=True, exist_ok=True) - await manager_internal.gather_related_logs( + await manager_client.gather_related_logs( failed_test_dir_path, {'pytest.log': test_log, 'test_py.log': test_py_log_test} ) @@ -212,8 +212,8 @@ async def manager(request, manager_internal, record_property, build_mode): full_url = f"failed_test_logs" record_property("TEST_LOGS", full_url) - cluster_status = await manager_internal.after_test(test_case_name, not failed) - await manager_internal.stop() # Stop client session and close driver after each test + cluster_status = await manager_client.after_test(test_case_name, not failed) + await manager_client.stop() # Stop client session and close driver after each test if cluster_status["server_broken"]: pytest.fail(f"test case {test_case_name} leave unfinished tasks on Scylla server. Server marked as broken") From 0141906c4a7bc95d6d69f77c35650a2fb69bac96 Mon Sep 17 00:00:00 2001 From: Asias He Date: Tue, 17 Dec 2024 10:10:29 +0800 Subject: [PATCH 047/397] repair: Enable small table optimization for RBNO rebuild Similar to 9ace1916163b6d59aa5d05ece52384c12be191bd (repair: Enable small table optimization for RBNO bootstrap and decommission), this patch enables small table optimization for RBNO rebuild. This is useful for rebuild ops which is used for building an empty DC. Fixes: #21951 Closes scylladb/scylladb#21952 --- repair/repair.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/repair/repair.cc b/repair/repair.cc index fff23be9f9ac..356582eec9d8 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -1497,6 +1497,7 @@ future<> repair::data_sync_repair_task_impl::run() { "system_traces" }; if (_reason == streaming::stream_reason::bootstrap || + _reason == streaming::stream_reason::rebuild || _reason == streaming::stream_reason::decommission) { small_table_optimization = small_table_optimization_enabled_ks.contains(keyspace); } From 74c5aabd23af1c7834166201f043e0fc49df5183 Mon Sep 17 00:00:00 2001 From: Yaron Kaikov Date: Tue, 2 Jul 2024 15:11:30 +0300 Subject: [PATCH 048/397] build_docker: add option for building container based on Ubuntu Pro Today our container is based on ubuntu:22.04, we need to build another container based on Ubuntu Pro for FIPS support (currently the latest one is 20.04) The default docker build process doesn't change, if FIPS is required I have added `--type pro` to build a supported container. To enable FIPS there is a need to attach an Ubuntu Pro subscription (it will be done as part of https://github.com/scylladb/scylla-pkg/issues/4186) Closes scylladb/scylladb#21974 --- dist/docker/debian/build_docker.sh | 26 +++++++++++++++++++------- dist/docker/pro-attach-config.yaml | 4 ++++ 2 files changed, 23 insertions(+), 7 deletions(-) create mode 100644 dist/docker/pro-attach-config.yaml diff --git a/dist/docker/debian/build_docker.sh b/dist/docker/debian/build_docker.sh index 2f7a2603ca94..9939e4c62986 100755 --- a/dist/docker/debian/build_docker.sh +++ b/dist/docker/debian/build_docker.sh @@ -13,6 +13,7 @@ version="$(sed 's/-/~/' +enable_services: +- fips +- fips-updates From a7fc566c7e45fdc1c1f2ad0a209a315a858cb430 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Fri, 20 Dec 2024 12:12:16 +0100 Subject: [PATCH 049/397] node_ops: filter topology_requests entries Currently node_ops_virtual_task shows stats of all system.topology_request entries. However, the table also contains info about non-node_ops requests, e.g. truncate. Filter the entries used by node_ops_virtual_task by their type. With this change bootstrap of the first node will not be visible. Update the test accordingly. --- db/system_keyspace.cc | 6 ++++-- node_ops/task_manager_module.cc | 2 +- test/topology_tasks/test_node_ops_tasks.py | 16 +++++----------- 3 files changed, 10 insertions(+), 14 deletions(-) diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 075c7a8f462a..6bb0c1c4ad69 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -3416,12 +3416,14 @@ future system_keyspace::get_topology_r future system_keyspace::get_topology_request_entries(db_clock::time_point end_time_limit) { // Running requests. auto rs_running = co_await execute_cql( - format("SELECT * FROM system.{} WHERE done = false ALLOW FILTERING", TOPOLOGY_REQUESTS)); + format("SELECT * FROM system.{} WHERE done = false AND request_type IN ('{}', '{}', '{}', '{}', '{}') ALLOW FILTERING", TOPOLOGY_REQUESTS, + service::topology_request::join, service::topology_request::replace, service::topology_request::rebuild, service::topology_request::leave, service::topology_request::remove)); // Requests which finished after end_time_limit. auto rs_done = co_await execute_cql( - format("SELECT * FROM system.{} WHERE end_time > {} ALLOW FILTERING", TOPOLOGY_REQUESTS, end_time_limit.time_since_epoch().count())); + format("SELECT * FROM system.{} WHERE end_time > {} AND request_type IN ('{}', '{}', '{}', '{}', '{}') ALLOW FILTERING", TOPOLOGY_REQUESTS, end_time_limit.time_since_epoch().count(), + service::topology_request::join, service::topology_request::replace, service::topology_request::rebuild, service::topology_request::leave, service::topology_request::remove)); topology_requests_entries m; for (const auto& row: *rs_done) { diff --git a/node_ops/task_manager_module.cc b/node_ops/task_manager_module.cc index 7a25b36b9476..056bc139d5fe 100644 --- a/node_ops/task_manager_module.cc +++ b/node_ops/task_manager_module.cc @@ -114,7 +114,7 @@ future> node_ops_virtual_task::contains( } auto entry = co_await _ss._sys_ks.local().get_topology_request_entry(task_id.uuid(), false); - co_return bool(entry.id) ? empty_hint : std::nullopt; + co_return bool(entry.id) && entry.request_type ? empty_hint : std::nullopt; } future node_ops_virtual_task::is_abortable(tasks::virtual_task_hint) const { diff --git a/test/topology_tasks/test_node_ops_tasks.py b/test/topology_tasks/test_node_ops_tasks.py index dc2d2ea742a0..ee0de2d39052 100644 --- a/test/topology_tasks/test_node_ops_tasks.py +++ b/test/topology_tasks/test_node_ops_tasks.py @@ -72,20 +72,14 @@ def check_regular_task_status(task: TaskStatus, expected_state: str, expected_ty async def check_bootstrap_tasks_tree(tm: TaskManagerClient, module_name: str, servers: list[ServerInfo], previous_vts: list[TaskID] = []) -> tuple[list[ServerInfo], list[TaskID]]: - virtual_tasks = await get_new_virtual_tasks_statuses(tm, module_name, servers, previous_vts, len(servers)) - - # No streaming task for first node bootstrap. - bootstrap_with_streaming = [status.id for status in virtual_tasks if status.children_ids] - assert len(bootstrap_with_streaming) == len(virtual_tasks) - 1, "All but one tasks should have children" + # Bootstrap of the first node is omitted. + virtual_tasks = await get_new_virtual_tasks_statuses(tm, module_name, servers, previous_vts, len(servers) - 1) for virtual_task in virtual_tasks: - if virtual_task.id in bootstrap_with_streaming: - check_virtual_task_status(virtual_task, "done", "bootstrap", 1) + check_virtual_task_status(virtual_task, "done", "bootstrap", 1) - child = await tm.get_task_status(virtual_task.children_ids[0]["node"], virtual_task.children_ids[0]["task_id"]) - check_regular_task_status(child, "done", "bootstrap: streaming", "node", virtual_task.id, 0) - else: - check_virtual_task_status(virtual_task, "done", "bootstrap", 0) + child = await tm.get_task_status(virtual_task.children_ids[0]["node"], virtual_task.children_ids[0]["task_id"]) + check_regular_task_status(child, "done", "bootstrap: streaming", "node", virtual_task.id, 0) return (servers, [vt.id for vt in virtual_tasks]) From ee4bd287fd62ec9b7187afdfe91db8d37a5701fb Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Fri, 20 Dec 2024 12:25:48 +0100 Subject: [PATCH 050/397] node_ops: rename a method that get node ops entries --- db/system_keyspace.cc | 2 +- db/system_keyspace.hh | 2 +- node_ops/task_manager_module.cc | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 6bb0c1c4ad69..957f69a24246 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -3413,7 +3413,7 @@ future system_keyspace::get_topology_r co_return topology_request_row_to_entry(id, row); } -future system_keyspace::get_topology_request_entries(db_clock::time_point end_time_limit) { +future system_keyspace::get_node_ops_request_entries(db_clock::time_point end_time_limit) { // Running requests. auto rs_running = co_await execute_cql( format("SELECT * FROM system.{} WHERE done = false AND request_type IN ('{}', '{}', '{}', '{}', '{}') ALLOW FILTERING", TOPOLOGY_REQUESTS, diff --git a/db/system_keyspace.hh b/db/system_keyspace.hh index cb7e0a4bc656..f468e7fbab04 100644 --- a/db/system_keyspace.hh +++ b/db/system_keyspace.hh @@ -643,7 +643,7 @@ public: future get_topology_request_state(utils::UUID id, bool require_entry); topology_requests_entry topology_request_row_to_entry(utils::UUID id, const cql3::untyped_result_set_row& row); future get_topology_request_entry(utils::UUID id, bool require_entry); - future get_topology_request_entries(db_clock::time_point end_time_limit); + future get_node_ops_request_entries(db_clock::time_point end_time_limit); public: future> get_service_levels_version(); diff --git a/node_ops/task_manager_module.cc b/node_ops/task_manager_module.cc index 056bc139d5fe..c3dd83fcf696 100644 --- a/node_ops/task_manager_module.cc +++ b/node_ops/task_manager_module.cc @@ -56,7 +56,7 @@ static std::set get_pending_ids(service::topology& topology) { static future get_entries(db::system_keyspace& sys_ks, service::topology& topology, std::chrono::seconds ttl) { // Started requests. - auto entries = co_await sys_ks.get_topology_request_entries(db_clock::now() - ttl); + auto entries = co_await sys_ks.get_node_ops_request_entries(db_clock::now() - ttl); // Pending requests. for (auto& id : get_pending_ids(topology)) { From da7301679b83c535a8c4c5c419349862cacd7ec1 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Fri, 20 Dec 2024 12:26:35 +0100 Subject: [PATCH 051/397] test: truncate the table before node ops task checks Truncate a table before testing node ops tasks to check if the truncate request won't be considered by node_ops_virtual_task. --- test/topology_tasks/test_node_ops_tasks.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/test/topology_tasks/test_node_ops_tasks.py b/test/topology_tasks/test_node_ops_tasks.py index ee0de2d39052..204513a2f129 100644 --- a/test/topology_tasks/test_node_ops_tasks.py +++ b/test/topology_tasks/test_node_ops_tasks.py @@ -203,6 +203,13 @@ async def test_node_ops_tasks_tree(manager: ManagerClient): servers = [await manager.server_add() for _ in range(3)] assert module_name in await tm.list_modules(servers[0].ip_addr), "node_ops module wasn't registered" + cql = manager.get_cql() + await cql.run_async("CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 1}") + await cql.run_async("CREATE TABLE test.test (pk int PRIMARY KEY, c int);") + await cql.run_async(f"INSERT INTO test.test (pk, c) VALUES ({1}, {1});") + await cql.run_async(f"TRUNCATE test.test;") + + servers, vt_ids = await check_bootstrap_tasks_tree(tm, module_name, servers) servers, vt_ids = await check_replace_tasks_tree(manager, tm, module_name, servers, vt_ids) servers, vt_ids = await check_rebuild_tasks_tree(manager, tm, module_name, servers, vt_ids) From 2a9f34bb8535c44a58f1038ef5b4e53721e877c3 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 16 Dec 2024 15:22:08 +0800 Subject: [PATCH 052/397] test/pytest.ini: put `repair` marker declaration back During the consolidation of per-suite pytest.ini files (commit 8bf62a086f), the 'repair' marker was inadvertently dropped. This led to pytest warnings for tests using the @pytest.mark.repair decorator. This patch restores the marker declaration to eliminate the distracting PytestUnknownMarkWarning: ``` test/topology_experimental_raft/test_tablets.py:396 /home/kefu/dev/scylladb/test/topology_experimental_raft/test_tablets.py:396: PytestUnknownMarkWarning: Unknown pytest.mark.repair - is this a typo? You can register custom marks to avoid this warning - for details, see https://docs.pytest.org/en/stable/how-to/mark.html @pytest.mark.repair ``` Restoring the marker allows tests to use the 'repair' mark without generating warnings. Signed-off-by: Kefu Chai Closes scylladb/scylladb#21931 --- test/pytest.ini | 1 + 1 file changed, 1 insertion(+) diff --git a/test/pytest.ini b/test/pytest.ini index 5a54cd42d16c..f2f5a4580262 100644 --- a/test/pytest.ini +++ b/test/pytest.ini @@ -9,6 +9,7 @@ markers = replication_factor: replication factor for RandomTables without_scylla: run without attaching to a scylla process enable_tablets: create keyspace with tablets enabled or disabled + repair: tests for repair norecursedirs = manual perf lib # Ignore warnings about HTTPS requests without certificate verification # (see issue #15287). Pytest breaks urllib3.disable_warnings() in conftest.py, From 48f7ef1c302165ec7432f6a8985c0b66e3ad026c Mon Sep 17 00:00:00 2001 From: Amnon Heiman Date: Wed, 18 Dec 2024 14:52:53 +0200 Subject: [PATCH 053/397] alternator/executor.cc: Add WCU for update_item This patch adds WCU support for update_item. The way Alternator modifies values means we don't always have the full item sizes. When there is a read-before-write, the code in rmw_operation takes care of the object size. When updating a value without read-before-write, we will make a rough estimation of the value's size. This is better than simply taking 1 (as we do with delete) and is also more Alternator-like. --- alternator/executor.cc | 58 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/alternator/executor.cc b/alternator/executor.cc index 30bc2c071cf4..53dc9088830f 100644 --- a/alternator/executor.cc +++ b/alternator/executor.cc @@ -2807,6 +2807,47 @@ static bool check_needs_read_before_write(const attribute_path_mapname.GetStringLength(); // Size of the key + size += estimate_value_size(it->value); // Size of the value + } + } + return size; +} + class update_item_operation : public rmw_operation { public: // Some information parsed during the constructor to check for input @@ -2894,6 +2935,20 @@ update_item_operation::update_item_operation(service::storage_proxy& proxy, rjso throw api_error::validation( format("UpdateItem does not allow both old-style AttributeUpdates and new-style ConditionExpression to be given together")); } + if (_pk.representation().size() > 2) { + // ScyllaDB uses two extra bytes compared to DynamoDB for the key bytes length + _consumed_capacity._total_bytes += _pk.representation().size() - 2; + } + if (_ck.representation().size() > 2) { + // ScyllaDB uses two extra bytes compared to DynamoDB for the key bytes length + _consumed_capacity._total_bytes += _ck.representation().size() - 2; + } + if (expression_attribute_names) { + _consumed_capacity._total_bytes += estimate_value_size(*expression_attribute_names); + } + if (expression_attribute_values) { + _consumed_capacity._total_bytes += estimate_value_size(*expression_attribute_values); + } } // These are the cases where update_item_operation::apply() needs to use @@ -3128,6 +3183,9 @@ static bool hierarchy_actions( std::optional update_item_operation::apply(std::unique_ptr previous_item, api::timestamp_type ts) const { + if (_consumed_capacity._total_bytes == 0) { + _consumed_capacity._total_bytes = 1; + } if (!verify_expected(_request, previous_item.get()) || !verify_condition_expression(_condition_expression, previous_item.get())) { if (previous_item && _returnvalues_on_condition_check_failure == From 1c29726477646e3e4b30f67a732b5a7e9474cebb Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Fri, 20 Dec 2024 09:24:55 +0100 Subject: [PATCH 054/397] replica: do not set tablet_task_info if it isn't valid Currently, in tablet_map_to_mutation, repair's and migration's tablet_task_info is always set. Do not set the tablet_task_info if there is no running operation. Closes scylladb/scylladb#22005 --- replica/tablets.cc | 6 +++-- test/pylib/repair.py | 10 ++++++++ test/topology_tasks/test_tablet_tasks.py | 30 +++++++++++++++++++++++- 3 files changed, 43 insertions(+), 3 deletions(-) diff --git a/replica/tablets.cc b/replica/tablets.cc index e0d3ae129c96..b7d75451fbe2 100644 --- a/replica/tablets.cc +++ b/replica/tablets.cc @@ -135,11 +135,13 @@ tablet_map_to_mutation(const tablet_map& tablets, table_id id, const sstring& ke auto last_token = tablets.get_last_token(tid); auto ck = clustering_key::from_single_value(*s, data_value(dht::token::to_int64(last_token)).serialize_nonnull()); m.set_clustered_cell(ck, "replicas", make_list_value(replica_set_type, replicas_to_data_value(tablet.replicas)), ts); - if (features.tablet_migration_virtual_task) { + if (features.tablet_migration_virtual_task && tablet.migration_task_info.is_valid()) { m.set_clustered_cell(ck, "migration_task_info", tablet_task_info_to_data_value(tablet.migration_task_info), ts); } if (features.tablet_repair_scheduler) { - m.set_clustered_cell(ck, "repair_task_info", tablet_task_info_to_data_value(tablet.repair_task_info), ts); + if (tablet.repair_task_info.is_valid()) { + m.set_clustered_cell(ck, "repair_task_info", tablet_task_info_to_data_value(tablet.repair_task_info), ts); + } if (tablet.repair_time != db_clock::time_point{}) { m.set_clustered_cell(ck, "repair_time", data_value(tablet.repair_time), ts); } diff --git a/test/pylib/repair.py b/test/pylib/repair.py index 85e6f239816a..ca77adf382a5 100644 --- a/test/pylib/repair.py +++ b/test/pylib/repair.py @@ -44,3 +44,13 @@ async def create_table_insert_data_for_repair(manager, rf = 3 , tablets = 8, fas logging.info(f'Got hosts={hosts}'); table_id = await manager.get_table_id("test", "test") return (servers, cql, hosts, table_id) + +async def get_tablet_task_id(cql, host, table_id, token): + rows = await cql.run_async(f"SELECT last_token, repair_task_info from system.tablets where table_id = {table_id}", host=host) + for row in rows: + if row.last_token == token: + if row.repair_task_info == None: + return None + else: + return str(row.repair_task_info.tablet_task_id) + return None diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index fec3a8c75bfc..6a4395cbaa7c 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -9,7 +9,7 @@ from test.pylib.internal_types import ServerInfo from test.pylib.manager_client import ManagerClient -from test.pylib.repair import create_table_insert_data_for_repair +from test.pylib.repair import create_table_insert_data_for_repair, get_tablet_task_id from test.pylib.tablets import get_all_tablet_replicas from test.topology.conftest import skip_mode from test.topology_experimental_raft.test_tablets import inject_error_on @@ -280,3 +280,31 @@ async def check(type, log, mark): src = replicas[0].replicas[0] dst = (src[0], 1 - src[1]) await asyncio.gather(move_tablet(src, dst), check("intranode_migration", log, mark)) + +@pytest.mark.asyncio +@skip_mode('release', 'error injections are not supported in release mode') +async def test_repair_task_info_is_none_when_no_running_repair(manager: ManagerClient): + module_name = "tablets" + tm = TaskManagerClient(manager.api) + token = -1 + + servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager) + assert module_name in await tm.list_modules(servers[0].ip_addr), "tablets module wasn't registered" + + async def check_none(): + tablet_task_id = await get_tablet_task_id(cql, hosts[0], table_id, token) + assert tablet_task_id is None + + await check_none() + + async def repair_task(): + await enable_injection(manager, servers, "repair_tablet_fail_on_rpc_call") + await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token) + + async def wait_and_check_none(): + task = (await wait_tasks_created(tm, servers[0], module_name, 1,"user_repair"))[0] + await disable_injection(manager, servers, "repair_tablet_fail_on_rpc_call") + status = await tm.wait_for_task(servers[0].ip_addr, task.task_id) + await check_none() + + await asyncio.gather(repair_task(), wait_and_check_none()) From dbe4ac746583615e6520ea28d034da6b1a68bb9f Mon Sep 17 00:00:00 2001 From: Yaniv Michael Kaul Date: Thu, 19 Dec 2024 14:25:10 +0200 Subject: [PATCH 055/397] LICENSE-ScyllaDB-Source-Available.md: fix markdown Codespell complained. Signed-off-by: Yaniv Kaul Closes scylladb/scylladb#21980 --- LICENSE-ScyllaDB-Source-Available.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/LICENSE-ScyllaDB-Source-Available.md b/LICENSE-ScyllaDB-Source-Available.md index ff7f058e35a4..1867f760f963 100644 --- a/LICENSE-ScyllaDB-Source-Available.md +++ b/LICENSE-ScyllaDB-Source-Available.md @@ -18,7 +18,7 @@ The terms "**You**" or "**Licensee**" refer to any individual accessing or using 2) create a modified version of the Software (each, a "**Licensed Work**"); provided however, that each such Licensed Work keeps all or substantially all of the functions and features of the Software, and/or using all or substantially all of the source code of the Software. You hereby agree that all the Licensed Work are, upon creation, considered Licensed Work of the Licensor, shall be the sole property of the Licensor and its assignees, and the Licensor and its assignees shall be the sole owner of all rights of any kind or nature, in connection with such Licensed Work. You hereby irrevocably and unconditionally assign to the Licensor all the Licensed Work and any part thereof. This License applies separately for each version of the Licensed Work, which shall be considered "Software" for the purpose of this Agreement. -**License Limitations, Restrictions and Obligations:** The license grant above is subject to the following limitations, restrictions, and obligations. If Licensee’s Use of the Software does not comply with the above license grant or the terms of this section (including exceeding the Usage Limit set forth below), Licensee must: (i) refrain from any Use of the Software; and (ii) purchase a [commercial paid licens](https://www.scylladb.com/scylladb-proprietary-software-license-agreement/)e from the Licensor. +**License Limitations, Restrictions and Obligations:** The license grant above is subject to the following limitations, restrictions, and obligations. If Licensee’s Use of the Software does not comply with the above license grant or the terms of this section (including exceeding the Usage Limit set forth below), Licensee must: (i) refrain from any Use of the Software; and (ii) purchase a [commercial paid license](https://www.scylladb.com/scylladb-proprietary-software-license-agreement/) from the Licensor. * **Updates:** You shall be solely responsible for providing all equipment, systems, assets, access, and ancillary goods and services needed to access and Use the Software. Licensor may modify or update the Software at any time, without notification, in its sole and absolute discretion. After the effective date of each such update, Licensor shall bear no obligation to run, provide or support legacy versions of the Software. * **"Usage Limit":** Licensee's total overall available storage across all deployments and clusters of the Software and the Licensed Work under this License shall not exceed 10TB and/or an upper limit of 50 VCPUs (hyper threads). From f1a0613a394dbf9eccad704964bbcedc2d42542f Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Fri, 20 Dec 2024 17:15:43 +0800 Subject: [PATCH 056/397] mutation: remove unused function `prefixed()` is a static function in `mutation_partition_v2.cc`. and this function is not used in this translation unit. so let's remove it. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22006 --- mutation/mutation_partition_v2.cc | 7 ------- 1 file changed, 7 deletions(-) diff --git a/mutation/mutation_partition_v2.cc b/mutation/mutation_partition_v2.cc index cb5edc12dc78..e49debcaf374 100644 --- a/mutation/mutation_partition_v2.cc +++ b/mutation/mutation_partition_v2.cc @@ -755,13 +755,6 @@ void mutation_partition_v2::for_each_row(const schema& schema, const query::clus } } -// Transforms given range of printable into a range of strings where each element -// in the original range is prefxied with given string. -template -static auto prefixed(const sstring& prefix, const RangeOfPrintable& r) { - return r | std::views::transform([&] (auto&& e) { return format("{}{}", prefix, e); }); -} - auto fmt::formatter::format(const mutation_partition_v2::printer& p, fmt::format_context& ctx) const -> decltype(ctx.out()) { const auto indent = ""; From eb62593f2c319ec2a7ee56310a5ab53ea8de5e60 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Tue, 17 Dec 2024 23:18:43 +0200 Subject: [PATCH 057/397] treewide: use angle brackets when including seastar headers We treat Seastar as a "system" library, and those are included with angle brackets. Closes scylladb/scylladb#21959 --- auth/standard_role_manager.cc | 4 ++-- cql3/statements/describe_statement.cc | 2 +- db/schema_applier.hh | 2 +- gms/gossiper.cc | 4 ++-- locator/topology.cc | 2 +- service/paxos/paxos_state.cc | 2 +- service/paxos/paxos_state.hh | 2 +- service/qos/service_level_controller.cc | 8 ++++---- service/qos/service_level_controller.hh | 2 +- test/boost/service_level_controller_test.cc | 2 +- test/boost/tablets_test.cc | 2 +- 11 files changed, 16 insertions(+), 16 deletions(-) diff --git a/auth/standard_role_manager.cc b/auth/standard_role_manager.cc index 989f06e8a0e1..8a3d4df43058 100644 --- a/auth/standard_role_manager.cc +++ b/auth/standard_role_manager.cc @@ -30,8 +30,8 @@ #include "db/consistency_level_type.hh" #include "exceptions/exceptions.hh" #include "utils/log.hh" -#include "seastar/core/loop.hh" -#include "seastar/coroutine/maybe_yield.hh" +#include +#include #include "service/raft/raft_group0_client.hh" #include "utils/class_registrator.hh" #include "service/migration_manager.hh" diff --git a/cql3/statements/describe_statement.cc b/cql3/statements/describe_statement.cc index 595c414d3548..9a257ee90561 100644 --- a/cql3/statements/describe_statement.cc +++ b/cql3/statements/describe_statement.cc @@ -18,7 +18,7 @@ #include "exceptions/exceptions.hh" #include #include -#include "seastar/coroutine/exception.hh" +#include #include "service/client_state.hh" #include "types/types.hh" #include "cql3/query_processor.hh" diff --git a/db/schema_applier.hh b/db/schema_applier.hh index fef6af4d906f..f96b14dc8ff7 100644 --- a/db/schema_applier.hh +++ b/db/schema_applier.hh @@ -10,7 +10,7 @@ #pragma once #include "mutation/mutation.hh" -#include "seastar/core/future.hh" +#include #include "service/storage_proxy.hh" #include "query-result-set.hh" diff --git a/gms/gossiper.cc b/gms/gossiper.cc index 1426d13525cc..4862d06fd983 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -43,8 +43,8 @@ #include #include "gms/generation-number.hh" #include "locator/token_metadata.hh" -#include "seastar/core/shard_id.hh" -#include "seastar/rpc/rpc_types.hh" +#include +#include #include "utils/assert.hh" #include "utils/exceptions.hh" #include "utils/error_injection.hh" diff --git a/locator/topology.cc b/locator/topology.cc index 159fa797ac77..7f6a1c0ecaa3 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -12,7 +12,7 @@ #include #include -#include "seastar/core/shard_id.hh" +#include #include "utils/log.hh" #include "locator/topology.hh" #include "locator/production_snitch_base.hh" diff --git a/service/paxos/paxos_state.cc b/service/paxos/paxos_state.cc index f1d1f1870882..f26b32c3f493 100644 --- a/service/paxos/paxos_state.cc +++ b/service/paxos/paxos_state.cc @@ -10,7 +10,7 @@ #include #include #include -#include "seastar/coroutine/exception.hh" +#include #include "service/storage_proxy.hh" #include "service/paxos/proposal.hh" #include "service/paxos/paxos_state.hh" diff --git a/service/paxos/paxos_state.hh b/service/paxos/paxos_state.hh index 908267b6612c..22719bac3f84 100644 --- a/service/paxos/paxos_state.hh +++ b/service/paxos/paxos_state.hh @@ -7,7 +7,7 @@ * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ #pragma once -#include "seastar/core/semaphore.hh" +#include #include "service/paxos/proposal.hh" #include "utils/log.hh" #include "utils/digest_algorithm.hh" diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index 4f15b98047f3..3b32843422e6 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -21,10 +21,10 @@ #include "db/system_keyspace.hh" #include #include -#include "seastar/core/future.hh" -#include "seastar/core/semaphore.hh" -#include "seastar/core/shard_id.hh" -#include "seastar/coroutine/maybe_yield.hh" +#include +#include +#include +#include #include "service/qos/raft_service_level_distributed_data_accessor.hh" #include "service/qos/standard_service_level_distributed_data_accessor.hh" #include "service_level_controller.hh" diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 4afc634b6d7f..46b208279153 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -9,7 +9,7 @@ #pragma once #include -#include "seastar/util/bool_class.hh" +#include #include #include #include diff --git a/test/boost/service_level_controller_test.cc b/test/boost/service_level_controller_test.cc index cbd361cf09f6..453470d2a465 100644 --- a/test/boost/service_level_controller_test.cc +++ b/test/boost/service_level_controller_test.cc @@ -13,7 +13,7 @@ #include #include -#include "seastar/core/future.hh" +#include #include "seastarx.hh" #include "test/lib/scylla_test_case.hh" #include "test/lib/test_utils.hh" diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 9e0aaa13bccf..b352baff39db 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -8,7 +8,7 @@ -#include "seastar/core/shard_id.hh" +#include #include "test/lib/scylla_test_case.hh" #include "test/lib/random_utils.hh" #include From 10c79a4d4745ad29e13f550c2a3637233cf2d301 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 16 Dec 2024 16:45:43 +0800 Subject: [PATCH 058/397] test/pylib: do not check for self.cmd when tearing down ScyllaServer we already check `self.cmd` for null at the very beginning of the `ScyllaServer.stop()`, and in the `try` block, we don't reset `self.cmd`, hence there is no need to check it again. Signed-off-by: Kefu Chai Closes scylladb/scylladb#21936 --- test/pylib/scylla_cluster.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 1322da534a9d..457d228690ae 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -714,8 +714,7 @@ async def stop(self) -> None: else: await self.cmd.wait() finally: - if self.cmd: - self.logger.info("stopped %s in %s", self, self.workdir.name) + self.logger.info("stopped %s in %s", self, self.workdir.name) self.cmd = None @stop_event From f8ce49ebe95e14181b525a038a94fc49320f2503 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Thu, 19 Dec 2024 16:34:00 +0200 Subject: [PATCH 059/397] cql3: implement NOT IN Where the grammar supports IN, we add NOT IN. This includes the WHERE clause and LWT IF clause. Evaluation of NOT IN follows from IN. In statement_restrictions analysis, they are different, as NOT IN doesn't enable any clever query plan and must filter. Some tests are added. An error message was changed ('in' changed to 'IN'), so some tests are adjusted. Closes scylladb/scylladb#21992 --- cql3/Cql.g | 15 ++++++ cql3/expr/expression.cc | 55 ++++++++++++++++++++- cql3/expr/expression.hh | 2 +- cql3/expr/prepare_expr.cc | 4 +- cql3/restrictions/statement_restrictions.cc | 2 +- docs/cql/dml/select.rst | 2 +- test/boost/expr_test.cc | 50 +++++++++++++++++++ test/cqlpy/test_filtering.py | 29 ++++++++++- test/cqlpy/test_lwt.py | 26 ++++++++++ 9 files changed, 176 insertions(+), 9 deletions(-) diff --git a/cql3/Cql.g b/cql3/Cql.g index 67513ab154e7..e09d155ffa2a 100644 --- a/cql3/Cql.g +++ b/cql3/Cql.g @@ -1785,6 +1785,13 @@ columnCondition returns [uexpression e] oper_t::IN, std::move(values)); } + | K_NOT K_IN + values=singleColumnInValuesOrMarkerExpr { + e = binary_operator( + std::move(key), + oper_t::NOT_IN, + std::move(values)); + } ) ; @@ -1839,6 +1846,14 @@ relation returns [uexpression e] .style = collection_constructor::style_type::list, .elements = std::move(in_values) }); } + | name=cident K_NOT K_IN marker1=marker + { $e = binary_operator(unresolved_identifier{std::move(name)}, oper_t::NOT_IN, std::move(marker1)); } + | name=cident K_NOT K_IN in_values=singleColumnInValues + { $e = binary_operator(unresolved_identifier{std::move(name)}, oper_t::NOT_IN, + collection_constructor { + .style = collection_constructor::style_type::list, + .elements = std::move(in_values) + }); } | name=cident K_CONTAINS { rt = oper_t::CONTAINS; } (K_KEY { rt = oper_t::CONTAINS_KEY; })? t=term { $e = binary_operator(unresolved_identifier{std::move(name)}, rt, std::move(t)); } | name=cident '[' key=term ']' type=relationType t=term { $e = binary_operator(subscript{.val = unresolved_identifier{std::move(name)}, .sub = std::move(key)}, type, std::move(t)); } diff --git a/cql3/expr/expression.cc b/cql3/expr/expression.cc index 3e9335a995c8..f79ce6ddfff5 100644 --- a/cql3/expr/expression.cc +++ b/cql3/expr/expression.cc @@ -329,6 +329,8 @@ bool_or_null limits(const expression& lhs, oper_t op, null_handling_style null_h } else { return list_contains_null(*sides_bytes.second); } + case oper_t::NOT_IN: + on_internal_error(expr_logger, "NOT IN operator on limits(), despite being rejected via !is_slice()"); } } } @@ -492,6 +494,50 @@ bool_or_null is_one_of(const expression& lhs, const expression& rhs, const evalu return false; } +/// True iff the column value is NOT in the set defined by rhs. Differs from !is_one_of() in handling NULLs. +bool_or_null is_none_of(const expression& lhs, const expression& rhs, const evaluation_inputs& inputs, null_handling_style null_handling) { + std::pair sides_bytes = + evaluate_binop_sides(lhs, rhs, oper_t::IN, inputs); + if (!sides_bytes.first || !sides_bytes.second) { + switch (null_handling) { + case null_handling_style::sql: + return bool_or_null::null(); + case null_handling_style::lwt_nulls: + if (!sides_bytes.second) { + return true; + } else { + return !list_contains_null(*sides_bytes.second); + } + } + } + + auto [lhs_bytes, rhs_bytes] = std::move(sides_bytes); + + expression lhs_constant = constant(raw_value::make_value(std::move(*lhs_bytes)), type_of(lhs)); + utils::chunked_vector list_elems = get_list_elements(raw_value::make_value(std::move(*rhs_bytes))); + bool saw_null = false; + for (const managed_bytes_opt& elem : list_elems) { + auto cmp_result = equal(lhs_constant, elem, inputs); + if (cmp_result.is_null()) { + // If we match another element in the list, we can return `false` with + // confidence. If we don't, we return NULL to indicate we don't know. + saw_null = true; + } else if (cmp_result.is_true()) { + return false; + } + } + if (saw_null) { + switch (null_handling) { + case null_handling_style::sql: + return bool_or_null::null(); + case null_handling_style::lwt_nulls: + // In LWT, `3 NOT IN (NULL, 5)` counts as true + return true; + } + } + return true; +} + bool is_not_null(const expression& lhs, const expression& rhs, const evaluation_inputs& inputs) { cql3::raw_value lhs_val = evaluate(lhs, inputs); cql3::raw_value rhs_val = evaluate(rhs, inputs); @@ -609,12 +655,12 @@ auto fmt::formatter::format(const cql3::expr::e out = fmt::format_to(out, " [[lwt_nulls]]"); } } else { - if (opr.op == oper_t::IN && is(opr.rhs)) { + if ((opr.op == oper_t::IN || opr.op == oper_t::NOT_IN) && is(opr.rhs)) { tuple_constructor rhs_tuple { .elements = as(opr.rhs).elements }; out = fmt::format_to(out, "{} {} {}", to_printer(opr.lhs), opr.op, to_printer(rhs_tuple)); - } else if (opr.op == oper_t::IN && is(opr.rhs) && as(opr.rhs).type->without_reversed().is_list()) { + } else if ((opr.op == oper_t::IN || opr.op == oper_t::NOT_IN) && is(opr.rhs) && as(opr.rhs).type->without_reversed().is_list()) { tuple_constructor rhs_tuple; const list_type_impl* list_typ = dynamic_cast(&as(opr.rhs).type->without_reversed()); for (const managed_bytes_opt& elem : get_list_elements(as(opr.rhs).value)) { @@ -1065,6 +1111,9 @@ cql3::raw_value do_evaluate(const binary_operator& binop, const evaluation_input case oper_t::IN: binop_result = is_one_of(binop.lhs, binop.rhs, inputs, binop.null_handling); break; + case oper_t::NOT_IN: + binop_result = is_none_of(binop.lhs, binop.rhs, inputs, binop.null_handling); + break; case oper_t::IS_NOT: binop_result = is_not_null(binop.lhs, binop.rhs, inputs); break; @@ -2305,6 +2354,8 @@ std::string_view fmt::formatter::to_string(const cql3::expr: return ">="; case oper_t::IN: return "IN"; + case oper_t::NOT_IN: + return "NOT IN"; case oper_t::CONTAINS: return "CONTAINS"; case oper_t::CONTAINS_KEY: diff --git a/cql3/expr/expression.hh b/cql3/expr/expression.hh index a9273a682ea3..e4ca2535dcaf 100644 --- a/cql3/expr/expression.hh +++ b/cql3/expr/expression.hh @@ -221,7 +221,7 @@ const column_value& get_subscripted_column(const subscript&); /// Only columns can be subscripted in CQL, so we can expect that the subscripted expression is a column_value. const column_value& get_subscripted_column(const expression&); -enum class oper_t { EQ, NEQ, LT, LTE, GTE, GT, IN, CONTAINS, CONTAINS_KEY, IS_NOT, LIKE }; +enum class oper_t { EQ, NEQ, LT, LTE, GTE, GT, IN, NOT_IN, CONTAINS, CONTAINS_KEY, IS_NOT, LIKE }; /// Describes the nature of clustering-key comparisons. Useful for implementing SCYLLA_CLUSTERING_BOUND. enum class comparison_order : char { diff --git a/cql3/expr/prepare_expr.cc b/cql3/expr/prepare_expr.cc index 7a5f9adae747..7e3f52484a0b 100644 --- a/cql3/expr/prepare_expr.cc +++ b/cql3/expr/prepare_expr.cc @@ -1450,9 +1450,9 @@ static lw_shared_ptr get_lhs_receiver(const expression& pr static lw_shared_ptr get_rhs_receiver(lw_shared_ptr& lhs_receiver, oper_t oper) { const data_type lhs_type = lhs_receiver->type->underlying_type(); - if (oper == oper_t::IN) { + if (oper == oper_t::IN || oper == oper_t::NOT_IN) { data_type rhs_receiver_type = list_type_impl::get_instance(std::move(lhs_type), false); - auto in_name = ::make_shared(format("in({})", lhs_receiver->name->text()), true); + auto in_name = ::make_shared(format("{}({})", oper, lhs_receiver->name->text()), true); return make_lw_shared(lhs_receiver->ks_name, lhs_receiver->cf_name, in_name, diff --git a/cql3/restrictions/statement_restrictions.cc b/cql3/restrictions/statement_restrictions.cc index 19aabbcba584..cae6d629cc0c 100644 --- a/cql3/restrictions/statement_restrictions.cc +++ b/cql3/restrictions/statement_restrictions.cc @@ -99,7 +99,7 @@ extern bool index_supports_some_column( inline bool needs_filtering(oper_t op) { return (op == oper_t::CONTAINS) || (op == oper_t::CONTAINS_KEY) || (op == oper_t::LIKE) || - (op == oper_t::IS_NOT) || (op == oper_t::NEQ) ; + (op == oper_t::IS_NOT) || (op == oper_t::NEQ) || (op == oper_t::NOT_IN); } inline auto find_needs_filtering(const expression& e) { diff --git a/docs/cql/dml/select.rst b/docs/cql/dml/select.rst index b7c174e91b4d..4f6d08800fb3 100644 --- a/docs/cql/dml/select.rst +++ b/docs/cql/dml/select.rst @@ -29,7 +29,7 @@ Querying data from data is done using a ``SELECT`` statement: relation: `column_name` `operator` `term` : '(' `column_name` ( ',' `column_name` )* ')' `operator` `tuple_literal` : TOKEN '(' `column_name` ( ',' `column_name` )* ')' `operator` `term` - operator: '=' | '<' | '>' | '<=' | '>=' | IN | CONTAINS | CONTAINS KEY + operator: '=' | '<' | '>' | '<=' | '>=' | IN | NOT IN | CONTAINS | CONTAINS KEY ordering_clause: `column_name` [ ASC | DESC ] ( ',' `column_name` [ ASC | DESC ] )* timeout: `duration` diff --git a/test/boost/expr_test.cc b/test/boost/expr_test.cc index 108a771f006e..66d888003138 100644 --- a/test/boost/expr_test.cc +++ b/test/boost/expr_test.cc @@ -2822,6 +2822,33 @@ BOOST_AUTO_TEST_CASE(evaluate_binary_operator_in) { test_evaluate_binop_null(oper_t::IN, make_int_const(5), in_list); } +BOOST_AUTO_TEST_CASE(evaluate_binary_operator_not_in) { + // IN expects a list as its rhs, sets are not allowed + expression in_list = make_int_list_const({1, 3, 5}); + + expression false_not_in_binop = binary_operator(make_int_const(3), oper_t::NOT_IN, in_list); + BOOST_REQUIRE_EQUAL(evaluate(false_not_in_binop, evaluation_inputs{}), make_bool_raw(false)); + + expression true_not_in_binop = binary_operator(make_int_const(2), oper_t::NOT_IN, in_list); + BOOST_REQUIRE_EQUAL(evaluate(true_not_in_binop, evaluation_inputs{}), make_bool_raw(true)); + + expression empty_not_in_list = binary_operator(make_empty_const(int32_type), oper_t::NOT_IN, in_list); + BOOST_REQUIRE_EQUAL(evaluate(empty_not_in_list, evaluation_inputs{}), make_bool_raw(true)); + + expression list_with_empty = + make_list_const({make_int_const(1), make_empty_const(int32_type), make_int_const(3)}, int32_type); + expression empty_not_in_list_with_empty = binary_operator(make_empty_const(int32_type), oper_t::NOT_IN, list_with_empty); + BOOST_REQUIRE_EQUAL(evaluate(empty_not_in_list_with_empty, evaluation_inputs{}), make_bool_raw(false)); + + expression existing_int_not_in_list_with_empty = binary_operator(make_int_const(3), oper_t::NOT_IN, list_with_empty); + BOOST_REQUIRE_EQUAL(evaluate(existing_int_not_in_list_with_empty, evaluation_inputs{}), make_bool_raw(false)); + + expression nonexisting_not_int_in_list_with_empty = binary_operator(make_int_const(321), oper_t::NOT_IN, list_with_empty); + BOOST_REQUIRE_EQUAL(evaluate(nonexisting_not_int_in_list_with_empty, evaluation_inputs{}), make_bool_raw(true)); + + test_evaluate_binop_null(oper_t::NOT_IN, make_int_const(5), in_list); +} + // Tests ` IN (123, ?, 789)` where the bind variable has value 456 BOOST_AUTO_TEST_CASE(evaluate_binary_operator_in_list_with_bind_variable) { schema_ptr table_schema = @@ -2845,6 +2872,29 @@ BOOST_AUTO_TEST_CASE(evaluate_binary_operator_in_list_with_bind_variable) { BOOST_REQUIRE_EQUAL(evaluate(empty_in_list, inputs), make_bool_raw(false)); } +// Tests ` IN (123, ?, 789)` where the bind variable has value 456 +BOOST_AUTO_TEST_CASE(evaluate_binary_operator_not_in_list_with_bind_variable) { + schema_ptr table_schema = + schema_builder("test_ks", "test_cf").with_column("pk", int32_type, column_kind::partition_key).build(); + + expression in_list = collection_constructor{ + .style = collection_constructor::style_type::list, + .elements = {make_int_const(123), bind_variable{.bind_index = 0, .receiver = make_receiver(int32_type)}, + make_int_const(789)}, + .type = list_type_impl::get_instance(int32_type, true)}; + + auto [inputs, inputs_data] = make_evaluation_inputs(table_schema, {{"pk", make_int_raw(111)}}, {make_int_raw(456)}); + + expression false_not_in_binop = binary_operator(make_int_const(456), oper_t::NOT_IN, in_list); + BOOST_REQUIRE_EQUAL(evaluate(false_not_in_binop, inputs), make_bool_raw(false)); + + expression true_not_in_binop = binary_operator(make_int_const(-100), oper_t::NOT_IN, in_list); + BOOST_REQUIRE_EQUAL(evaluate(true_not_in_binop, inputs), make_bool_raw(true)); + + expression empty_not_in_list = binary_operator(make_empty_const(int32_type), oper_t::NOT_IN, in_list); + BOOST_REQUIRE_EQUAL(evaluate(empty_not_in_list, inputs), make_bool_raw(true)); +} + BOOST_AUTO_TEST_CASE(evaluate_binary_operator_list_contains) { expression list_val = make_int_list_const({1, 3, 5}); diff --git a/test/cqlpy/test_filtering.py b/test/cqlpy/test_filtering.py index 4bc70fbc0aed..65b479535bb2 100644 --- a/test/cqlpy/test_filtering.py +++ b/test/cqlpy/test_filtering.py @@ -152,6 +152,31 @@ def test_filtering_with_in_relation(cql, test_keyspace, cassandra_bug): res = cql.execute(f"select * from {table} where v in (5,7) ALLOW FILTERING") assert set(res) == set([(2,3,4,5), (4,5,6,7)]) +# Test that NOT IN restrictions are supported with filtering and return the +# correct results. +def test_filtering_with_not_in_relation(cql, test_keyspace): + schema = 'p1 int, p2 int, c int, v int, primary key ((p1, p2),c)' + with new_test_table(cql, test_keyspace, schema) as table: + cql.execute(f"INSERT INTO {table} (p1, p2, c, v) VALUES (1, 2, 3, 4)") + cql.execute(f"INSERT INTO {table} (p1, p2, c, v) VALUES (2, 3, 4, 5)") + cql.execute(f"INSERT INTO {table} (p1, p2, c, v) VALUES (3, 4, 5, 6)") + cql.execute(f"INSERT INTO {table} (p1, p2, c, v) VALUES (4, 5, 6, 7)") + cql.execute(f"INSERT INTO {table} (p1, p2, c, v) VALUES (5, 6, 7, NULL)") + res = cql.execute(f"select * from {table} where p1 NOT IN (2,4) ALLOW FILTERING") + assert set(res) == set([(1,2,3,4), (3,4,5,6), (5,6,7,None)]) + res = cql.execute(f"select * from {table} where p2 NOT IN (2,4) ALLOW FILTERING") + assert set(res) == set([(2,3,4,5), (4,5,6,7), (5,6,7,None)]) + res = cql.execute(f"select * from {table} where c NOT IN (3,5) ALLOW FILTERING") + assert set(res) == set([(2,3,4,5), (4,5,6,7), (5,6,7,None)]) + res = cql.execute(f"select * from {table} where v NOT IN (5,7) ALLOW FILTERING") + assert set(res) == set([(1,2,3,4), (3,4,5,6)]) + with pytest.raises(InvalidRequest, match='ALLOW FILTERING'): + cql.execute(f"select * from {table} where p1 NOT IN (2,4)") + # Since NULL is unknown, it could have matched p1, so NOT IN returns NULL too. + res = cql.execute(f"select * from {table} where v NOT IN (5,NULL) ALLOW FILTERING") + assert set(res) == set() + + # Test that subscripts in expressions work as expected. They should only work # on map columns, and must have the correct type. Test that they also work # as expected for null or unset subscripts. @@ -445,13 +470,13 @@ def test_multi_column_relation_tuples_null_check(cql, test_keyspace): #This test shows how Scylla handles invalid number of items in one of the tuples of multi-column relation. #If the number of items is lesser than required, Scylla throws "Expected {} elements in value tuple, but got {}" #But if the number of items is greater than required, Scylla throws different error: -#"Invalid list literal for in((b,c,d)): value (1, 2, 3, 4) is not of type frozen>" +#"Invalid list literal for IN((b,c,d)): value (1, 2, 3, 4) is not of type frozen>" # Reproduces #13217 def test_multi_column_relation_wrong_number_of_items_in_tuple(cql, test_keyspace): with new_test_table(cql, test_keyspace, 'a int, b int, c int, d int, primary key (a, b, c, d)') as table: with pytest.raises(InvalidRequest, match='elements in value tuple, but got'): cql.execute(f'SELECT * FROM {table} WHERE a = 0 AND (b, c, d) IN ((1, 2), (2, 1, 4))') - with pytest.raises(InvalidRequest, match='Invalid list literal for in'): + with pytest.raises(InvalidRequest, match='Invalid list literal for IN'): cql.execute(f'SELECT * FROM {table} WHERE a = 0 AND (b, c, d) IN ((1, 2, 3, 4, 5), (2, 1, 4))') # Test for a bug found while developing the fix for #10357. The bug was that if a regular column was selected diff --git a/test/cqlpy/test_lwt.py b/test/cqlpy/test_lwt.py index edcda7bfdaa1..5da8e4531231 100644 --- a/test/cqlpy/test_lwt.py +++ b/test/cqlpy/test_lwt.py @@ -102,3 +102,29 @@ def test_lwt_with_batch_conflict_2(cql, table1): # Scylla returns a separate row for each of the two conditions. for r in rs: assert r.applied == False + +# Test NOT IN condition in LWT IF clause +# +# Cassandra rejects this with "line 1:84 no viable alternative at input 'NOT' (...AND c=1 IF r [NOT]...)"", +# indicating its grammar only supports this for WHERE, not IF. +def test_lwt_not_in(cql, table1, cassandra_bug): + p = unique_key_int() + cql.execute(f'INSERT INTO {table1}(p, c, r) values ({p}, 1, 1)') + rs = list(cql.execute(f'UPDATE {table1} SET r=2 WHERE p={p} AND c=1 IF r NOT IN (1, 2)')) + for r in rs: + assert r.applied == False + # Check that we look at the entire list, not just the first element + rs = list(cql.execute(f'UPDATE {table1} SET r=2 WHERE p={p} AND c=1 IF r NOT IN (2, 1)')) + for r in rs: + assert r.applied == False + rs = list(cql.execute(f'UPDATE {table1} SET r=2 WHERE p={p} AND c=1 IF r NOT IN (7, 8)')) + for r in rs: + assert r.applied == True + # LWT IF conditions don't treat NULL as a special value + rs = list(cql.execute(f'UPDATE {table1} SET r=NULL WHERE p={p} AND c=1 IF r NOT IN (NULL, 7, 8)')) + for r in rs: + assert r.applied == True + # Similar, but now show that NULL input fails the condition + rs = list(cql.execute(f'UPDATE {table1} SET r=NULL WHERE p={p} AND c=1 IF r NOT IN (NULL, 7, 8)')) + for r in rs: + assert r.applied == False From 200f0bb2192607c37c23c5c2f252701fc6624dfa Mon Sep 17 00:00:00 2001 From: Piotr Smaron Date: Mon, 16 Dec 2024 12:11:10 +0100 Subject: [PATCH 060/397] alternator: use get_datacenters() in get_network_topology_options() Currently, `get_network_topology_options()` is using gossip data and iterates over topology using IPs and not host IDs, which may result in operating on inconsistent data. This method's implemenations has been changed to instead use `get_datacenters()`, which should always return consistent data. Fixes: scylladb/scylladb#21490 Closes scylladb/scylladb#21940 --- alternator/executor.cc | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/alternator/executor.cc b/alternator/executor.cc index 30bc2c071cf4..2ade02c89726 100644 --- a/alternator/executor.cc +++ b/alternator/executor.cc @@ -4794,11 +4794,9 @@ future executor::describe_endpoints(client_state& static std::map get_network_topology_options(service::storage_proxy& sp, gms::gossiper& gossiper, int rf) { std::map options; - sstring rf_str = std::to_string(rf); - auto& topology = sp.get_token_metadata_ptr()->get_topology(); - for (const gms::inet_address& addr : gossiper.get_live_members()) { - options.emplace(topology.get_datacenter(addr), rf_str); - }; + for (const auto& dc : sp.get_token_metadata_ptr()->get_topology().get_datacenters()) { + options.emplace(dc, std::to_string(rf)); + } return options; } From 03461d6a54b07b085796ca806133a033548841f5 Mon Sep 17 00:00:00 2001 From: Takuya ASADA Date: Thu, 5 Sep 2024 05:37:28 +0900 Subject: [PATCH 061/397] test: compile unit tests into a single executable To reduce test executable size and speed up compilation time, compile unit tests into a single executable. Here is a file size comparison of the unit test executable: - Before applying the patch $ du -h --exclude='*.o' --exclude='*.o.d' build/release/test/boost/ build/debug/test/boost/ 11G build/release/test/boost/ 29G build/debug/test/boost/ - After applying the patch du -h --exclude='*.o' --exclude='*.o.d' build/release/test/boost/ build/debug/test/boost/ 5.5G build/release/test/boost/ 19G build/debug/test/boost/ It reduces executable sizes 5.5GB on release, and 10GB on debug. Closes #9155 Closes scylladb/scylladb#21443 --- configure.py | 131 ++++++------ replica/database.hh | 4 +- test.py | 110 +++++++++- test/boost/CMakeLists.txt | 199 +++++++----------- test/boost/aggregate_fcts_test.cc | 7 +- test/boost/auth_test.cc | 8 +- test/boost/batchlog_manager_test.cc | 6 +- test/boost/cache_algorithm_test.cc | 4 + test/boost/castas_fcts_test.cc | 7 +- test/boost/cdc_test.cc | 16 +- test/boost/column_mapping_test.cc | 7 +- test/boost/combined_tests.cc | 9 + test/boost/commitlog_cleanup_test.cc | 8 +- test/boost/commitlog_test.cc | 7 +- test/boost/cql_auth_query_test.cc | 8 +- test/boost/cql_functions_test.cc | 7 +- test/boost/cql_query_group_test.cc | 7 +- test/boost/cql_query_large_test.cc | 7 +- test/boost/cql_query_like_test.cc | 7 +- test/boost/cql_query_test.cc | 10 +- test/boost/data_listeners_test.cc | 7 +- test/boost/database_test.cc | 25 ++- test/boost/error_injection_test.cc | 7 +- test/boost/extensions_test.cc | 7 +- test/boost/filtering_test.cc | 7 +- test/boost/group0_cmd_merge_test.cc | 7 +- test/boost/group0_test.cc | 7 +- test/boost/index_with_paging_test.cc | 7 +- test/boost/json_cql_query_test.cc | 7 +- test/boost/large_paging_state_test.cc | 7 +- test/boost/loading_cache_test.cc | 7 +- test/boost/memtable_test.cc | 7 +- ...ombining_reader_as_mutation_source_test.cc | 7 +- test/boost/multishard_mutation_query_test.cc | 16 +- test/boost/mutation_reader_test.cc | 7 +- test/boost/mutation_writer_test.cc | 7 +- test/boost/network_topology_strategy_test.cc | 10 +- test/boost/per_partition_rate_limit_test.cc | 9 +- test/boost/querier_cache_test.cc | 8 +- test/boost/query_processor_test.cc | 7 +- .../reader_concurrency_semaphore_test.cc | 15 +- test/boost/repair_test.cc | 7 +- test/boost/restrictions_test.cc | 8 +- test/boost/role_manager_test.cc | 7 +- test/boost/row_cache_test.cc | 11 +- test/boost/schema_change_test.cc | 7 +- test/boost/schema_registry_test.cc | 7 +- test/boost/secondary_index_test.cc | 7 +- test/boost/sessions_test.cc | 7 +- test/boost/sstable_compaction_test.cc | 8 +- test/boost/sstable_directory_test.cc | 8 +- test/boost/sstable_set_test.cc | 7 +- test/boost/statement_restrictions_test.cc | 7 +- test/boost/storage_proxy_test.cc | 8 +- test/boost/tablets_test.cc | 6 +- test/boost/tracing_test.cc | 7 +- test/boost/user_function_test.cc | 7 +- test/boost/user_types_test.cc | 7 +- test/boost/view_build_test.cc | 7 +- test/boost/view_complex_test.cc | 7 +- test/boost/view_schema_ckey_test.cc | 7 +- test/boost/view_schema_pkey_test.cc | 7 +- test/boost/view_schema_test.cc | 7 +- test/boost/virtual_reader_test.cc | 7 +- test/boost/virtual_table_test.cc | 7 +- 65 files changed, 660 insertions(+), 272 deletions(-) create mode 100644 test/boost/combined_tests.cc diff --git a/configure.py b/configure.py index 6d05746b3bee..30df1cb02ba6 100755 --- a/configure.py +++ b/configure.py @@ -430,92 +430,67 @@ def find_ninja(): } scylla_tests = set([ + 'test/boost/combined_tests', 'test/boost/UUID_test', - 'test/boost/aggregate_fcts_test', 'test/boost/allocation_strategy_test', 'test/boost/alternator_unit_test', 'test/boost/anchorless_list_test', 'test/boost/auth_passwords_test', 'test/boost/auth_resource_test', - 'test/boost/auth_test', - 'test/boost/batchlog_manager_test', 'test/boost/big_decimal_test', 'test/boost/bloom_filter_test', 'test/boost/bptree_test', 'test/boost/broken_sstable_test', 'test/boost/btree_test', 'test/boost/bytes_ostream_test', - 'test/boost/cache_algorithm_test', 'test/boost/cache_mutation_reader_test', 'test/boost/cached_file_test', 'test/boost/caching_options_test', 'test/boost/canonical_mutation_test', 'test/boost/cartesian_product_test', - 'test/boost/castas_fcts_test', 'test/boost/cdc_generation_test', - 'test/boost/cdc_test', 'test/boost/cell_locker_test', 'test/boost/checksum_utils_test', 'test/boost/chunked_managed_vector_test', 'test/boost/chunked_vector_test', 'test/boost/clustering_ranges_walker_test', - 'test/boost/column_mapping_test', - 'test/boost/commitlog_cleanup_test', - 'test/boost/commitlog_test', 'test/boost/compaction_group_test', 'test/boost/compound_test', 'test/boost/compress_test', 'test/boost/config_test', 'test/boost/continuous_data_consumer_test', 'test/boost/counter_test', - 'test/boost/cql_auth_query_test', 'test/boost/cql_auth_syntax_test', - 'test/boost/cql_functions_test', - 'test/boost/cql_query_group_test', - 'test/boost/cql_query_large_test', - 'test/boost/cql_query_like_test', - 'test/boost/cql_query_test', 'test/boost/crc_test', - 'test/boost/data_listeners_test', - 'test/boost/database_test', 'test/boost/dirty_memory_manager_test', 'test/boost/double_decker_test', 'test/boost/duration_test', 'test/boost/dynamic_bitset_test', 'test/boost/enum_option_test', 'test/boost/enum_set_test', - 'test/boost/error_injection_test', 'test/boost/estimated_histogram_test', 'test/boost/exception_container_test', 'test/boost/exceptions_fallback_test', 'test/boost/exceptions_optimized_test', 'test/boost/expr_test', - 'test/boost/extensions_test', - 'test/boost/filtering_test', 'test/boost/flush_queue_test', 'test/boost/fragmented_temporary_buffer_test', 'test/boost/frozen_mutation_test', 'test/boost/generic_server_test', 'test/boost/gossiping_property_file_snitch_test', - 'test/boost/group0_cmd_merge_test', - 'test/boost/group0_test', 'test/boost/hash_test', 'test/boost/hashers_test', 'test/boost/hint_test', 'test/boost/idl_test', 'test/boost/index_reader_test', - 'test/boost/index_with_paging_test', 'test/boost/input_stream_test', 'test/boost/intrusive_array_test', - 'test/boost/json_cql_query_test', 'test/boost/json_test', 'test/boost/keys_test', - 'test/boost/large_paging_state_test', 'test/boost/like_matcher_test', 'test/boost/limiting_data_source_test', 'test/boost/linearizing_input_stream_test', 'test/boost/lister_test', - 'test/boost/loading_cache_test', 'test/boost/locator_topology_test', 'test/boost/log_heap_test', 'test/boost/logalloc_standard_allocator_segment_pool_backend_test', @@ -523,88 +498,54 @@ def find_ninja(): 'test/boost/managed_bytes_test', 'test/boost/managed_vector_test', 'test/boost/map_difference_test', - 'test/boost/memtable_test', - 'test/boost/multishard_combining_reader_as_mutation_source_test', - 'test/boost/multishard_mutation_query_test', 'test/boost/murmur_hash_test', 'test/boost/mutation_fragment_test', 'test/boost/mutation_query_test', 'test/boost/mutation_reader_another_test', - 'test/boost/mutation_reader_test', 'test/boost/mutation_test', - 'test/boost/mutation_writer_test', 'test/boost/mvcc_test', - 'test/boost/network_topology_strategy_test', 'test/boost/nonwrapping_interval_test', 'test/boost/observable_test', 'test/boost/partitioner_test', - 'test/boost/per_partition_rate_limit_test', 'test/boost/pretty_printers_test', - 'test/boost/querier_cache_test', - 'test/boost/query_processor_test', 'test/boost/radix_tree_test', 'test/boost/range_tombstone_list_test', 'test/boost/rate_limiter_test', - 'test/boost/reader_concurrency_semaphore_test', 'test/boost/recent_entries_map_test', - 'test/boost/repair_test', - 'test/boost/restrictions_test', 'test/boost/result_utils_test', 'test/boost/reusable_buffer_test', - 'test/boost/role_manager_test', - 'test/boost/row_cache_test', 'test/boost/rust_test', 'test/boost/s3_test', 'test/boost/aws_errors_test', 'test/boost/aws_error_injection_test', - 'test/boost/schema_change_test', 'test/boost/schema_changes_test', 'test/boost/schema_loader_test', - 'test/boost/schema_registry_test', - 'test/boost/secondary_index_test', 'test/boost/serialization_test', 'test/boost/serialized_action_test', 'test/boost/service_level_controller_test', - 'test/boost/sessions_test', 'test/boost/small_vector_test', 'test/boost/snitch_reset_test', 'test/boost/sorting_test', 'test/boost/sstable_3_x_test', - 'test/boost/sstable_compaction_test', 'test/boost/sstable_conforms_to_mutation_source_test', 'test/boost/sstable_datafile_test', - 'test/boost/sstable_directory_test', 'test/boost/sstable_generation_test', 'test/boost/sstable_move_test', 'test/boost/sstable_mutation_test', 'test/boost/sstable_partition_index_cache_test', 'test/boost/sstable_resharding_test', - 'test/boost/sstable_set_test', 'test/boost/sstable_test', 'test/boost/stall_free_test', - 'test/boost/statement_restrictions_test', - 'test/boost/storage_proxy_test', 'test/boost/string_format_test', 'test/boost/summary_test', - 'test/boost/tablets_test', 'test/boost/tagged_integer_test', 'test/boost/token_metadata_test', 'test/boost/top_k_test', - 'test/boost/tracing_test', 'test/boost/transport_test', 'test/boost/types_test', - 'test/boost/user_function_test', - 'test/boost/user_types_test', 'test/boost/utf8_test', - 'test/boost/view_build_test', - 'test/boost/view_complex_test', - 'test/boost/view_schema_ckey_test', - 'test/boost/view_schema_pkey_test', - 'test/boost/view_schema_test', 'test/boost/vint_serialization_test', - 'test/boost/virtual_reader_test', 'test/boost/virtual_table_mutation_source_test', - 'test/boost/virtual_table_test', 'test/boost/wasm_alloc_test', 'test/boost/wasm_test', 'test/boost/wrapping_interval_test', @@ -1455,8 +1396,72 @@ def find_ninja(): deps[t] = [t + '.cc'] + scylla_tests_dependencies + perf_tests_seastar_deps deps[t] += ['test/perf/perf.cc', 'seastar/tests/perf/linux_perf_event.cc'] -deps['test/boost/mutation_reader_test'] += ['test/lib/dummy_sharder.cc' ] -deps['test/boost/multishard_combining_reader_as_mutation_source_test'] += ['test/lib/dummy_sharder.cc' ] +deps['test/boost/combined_tests'] += [ + 'test/boost/aggregate_fcts_test.cc', + 'test/boost/auth_test.cc', + 'test/boost/batchlog_manager_test.cc', + 'test/boost/cache_algorithm_test.cc', + 'test/boost/castas_fcts_test.cc', + 'test/boost/cdc_test.cc', + 'test/boost/column_mapping_test.cc', + 'test/boost/commitlog_cleanup_test.cc', + 'test/boost/commitlog_test.cc', + 'test/boost/cql_auth_query_test.cc', + 'test/boost/cql_functions_test.cc', + 'test/boost/cql_query_group_test.cc', + 'test/boost/cql_query_large_test.cc', + 'test/boost/cql_query_like_test.cc', + 'test/boost/cql_query_test.cc', + 'test/boost/database_test.cc', + 'test/boost/data_listeners_test.cc', + 'test/boost/error_injection_test.cc', + 'test/boost/extensions_test.cc', + 'test/boost/filtering_test.cc', + 'test/boost/group0_cmd_merge_test.cc', + 'test/boost/group0_test.cc', + 'test/boost/index_with_paging_test.cc', + 'test/boost/json_cql_query_test.cc', + 'test/boost/large_paging_state_test.cc', + 'test/boost/loading_cache_test.cc', + 'test/boost/memtable_test.cc', + 'test/boost/multishard_combining_reader_as_mutation_source_test.cc', + 'test/boost/multishard_mutation_query_test.cc', + 'test/boost/mutation_reader_test.cc', + 'test/boost/mutation_writer_test.cc', + 'test/boost/network_topology_strategy_test.cc', + 'test/boost/per_partition_rate_limit_test.cc', + 'test/boost/querier_cache_test.cc', + 'test/boost/query_processor_test.cc', + 'test/boost/reader_concurrency_semaphore_test.cc', + 'test/boost/repair_test.cc', + 'test/boost/restrictions_test.cc', + 'test/boost/role_manager_test.cc', + 'test/boost/row_cache_test.cc', + 'test/boost/schema_change_test.cc', + 'test/boost/schema_registry_test.cc', + 'test/boost/secondary_index_test.cc', + 'test/boost/sessions_test.cc', + 'test/boost/sstable_compaction_test.cc', + 'test/boost/sstable_directory_test.cc', + 'test/boost/sstable_set_test.cc', + 'test/boost/statement_restrictions_test.cc', + 'test/boost/storage_proxy_test.cc', + 'test/boost/tablets_test.cc', + 'test/boost/tracing_test.cc', + 'test/boost/user_function_test.cc', + 'test/boost/user_types_test.cc', + 'test/boost/view_build_test.cc', + 'test/boost/view_complex_test.cc', + 'test/boost/view_schema_ckey_test.cc', + 'test/boost/view_schema_pkey_test.cc', + 'test/boost/view_schema_test.cc', + 'test/boost/virtual_reader_test.cc', + 'test/boost/virtual_table_test.cc', + 'tools/schema_loader.cc', + 'tools/read_mutation.cc', + 'test/lib/expr_test_utils.cc', + 'test/lib/dummy_sharder.cc', +] deps['test/boost/bytes_ostream_test'] = [ "test/boost/bytes_ostream_test.cc", @@ -1496,8 +1501,6 @@ def find_ninja(): deps['test/boost/schema_loader_test'] += ['tools/schema_loader.cc', 'tools/read_mutation.cc'] deps['test/boost/rust_test'] += ['rust/inc/src/lib.rs'] -deps['test/boost/group0_cmd_merge_test'] += ['test/lib/expr_test_utils.cc'] - deps['test/raft/replication_test'] = ['test/raft/replication_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies deps['test/raft/raft_server_test'] = ['test/raft/raft_server_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies deps['test/raft/randomized_nemesis_test'] = ['test/raft/randomized_nemesis_test.cc', 'direct_failure_detector/failure_detector.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies diff --git a/replica/database.hh b/replica/database.hh index 0e1341082f7b..91ef1010197c 100644 --- a/replica/database.hh +++ b/replica/database.hh @@ -141,7 +141,7 @@ class mutation_reordered_with_truncate_exception : public std::exception {}; class column_family_test; class table_for_tests; -class database_test; +class database_test_wrapper; using sstable_list = sstables::sstable_list; class sigquit_handler; @@ -1384,7 +1384,7 @@ class db_user_types_storage; // use table::shard_for_reads()/table::shard_for_writes() for data class database : public peering_sharded_service { - friend class ::database_test; + friend class ::database_test_wrapper; public: enum class table_kind { system, diff --git a/test.py b/test.py index e6ac057ecc12..a1e9269544ec 100755 --- a/test.py +++ b/test.py @@ -352,6 +352,12 @@ async def add_test(self, shortname, casename) -> None: @property def pattern(self) -> str: + # This should only match individual tests and not combined_tests.cc + # file of the combined test. + # It is because combined_tests.cc itself does not contain any tests. + # To keep the code simple, we have avoided this by renaming + # combined test file to “_tests.cc” instead of changing the match + # pattern. return "*_test.cc" @@ -362,11 +368,47 @@ class BoostTestSuite(UnitTestSuite): # --list_content. Static to share across all modes. _case_cache: Dict[str, List[str]] = dict() + _exec_name_cache: Dict[str, str] = dict() + + def _generate_cache(self) -> None: + # Apply combined test only for test/boost + if self.name != 'boost': + return + exe = path_to(self.mode, "test", self.name, 'combined_tests') + res = subprocess.run( + [exe, '--list_content'], + check=True, + capture_output=True, + env=dict(os.environ, + **{"ASAN_OPTIONS": "halt_on_error=0"}), + ) + testname = None + fqname = None + for line in res.stderr.decode().splitlines(): + if not line.startswith(' '): + testname = line.strip().rstrip('*') + fqname = os.path.join(self.mode, self.name, testname) + self._exec_name_cache[fqname] = 'combined_tests' + self._case_cache[fqname] = [] + else: + casename = line.strip().rstrip('*') + if casename.startswith('_'): + continue + self._case_cache[fqname].append(casename) + def __init__(self, path, cfg: dict, options: argparse.Namespace, mode) -> None: super().__init__(path, cfg, options, mode) + self._generate_cache() async def create_test(self, shortname: str, casename: str, suite, args) -> None: - exe = path_to(suite.mode, "test", suite.name, shortname) + fqname = os.path.join(self.mode, self.name, shortname) + if fqname in self._exec_name_cache: + execname = self._exec_name_cache[fqname] + combined_test = True + else: + execname = None + combined_test = False + exe = path_to(suite.mode, "test", suite.name, execname if combined_test else shortname) if not os.access(exe, os.X_OK): print(palette.warn(f"Boost test executable {exe} not found.")) return @@ -374,6 +416,8 @@ async def create_test(self, shortname: str, casename: str, suite, args) -> None: allows_compaction_groups = self.all_can_run_compaction_groups_except != None and shortname not in self.all_can_run_compaction_groups_except if options.parallel_cases and (shortname not in self.no_parallel_cases) and casename is None: fqname = os.path.join(self.mode, self.name, shortname) + # since combined tests are preloaded to self._case_cache, this will + # only run in non-combined test mode if fqname not in self._case_cache: process = await asyncio.create_subprocess_exec( exe, *['--list_content'], @@ -398,16 +442,36 @@ async def create_test(self, shortname: str, casename: str, suite, args) -> None: case_list = self._case_cache[fqname] if len(case_list) == 1: - test = BoostTest(self.next_id((shortname, self.suite_key)), shortname, suite, args, None, allows_compaction_groups) + test = BoostTest(self.next_id((shortname, self.suite_key)), shortname, suite, args, None, allows_compaction_groups, execname) self.tests.append(test) else: for case in case_list: - test = BoostTest(self.next_id((shortname, self.suite_key, case)), shortname, suite, args, case, allows_compaction_groups) + test = BoostTest(self.next_id((shortname, self.suite_key, case)), shortname, suite, args, case, allows_compaction_groups, execname) self.tests.append(test) else: - test = BoostTest(self.next_id((shortname, self.suite_key)), shortname, suite, args, casename, allows_compaction_groups) + test = BoostTest(self.next_id((shortname, self.suite_key)), shortname, suite, args, casename, allows_compaction_groups, execname) self.tests.append(test) + async def add_test(self, shortname, casename) -> None: + """Create a UnitTest class with possibly custom command line + arguments and add it to the list of tests""" + fqname = os.path.join(self.mode, self.name, shortname) + if fqname in self._exec_name_cache: + execname = self._exec_name_cache[fqname] + combined_test = True + else: + combined_test = False + # Skip tests which are not configured, and hence are not built + if os.path.join("test", self.name, execname if combined_test else shortname) not in self.options.tests: + return + + # Default seastar arguments, if not provided in custom test options, + # are two cores and 2G of RAM + args = self.custom_args.get(shortname, ["-c2 -m2G"]) + args = merge_cmdline_options(args, self.options.extra_scylla_cmdline_options) + for a in args: + await self.create_test(shortname, casename, self, a) + def junit_tests(self) -> Iterable['Test']: """Boost tests produce an own XML output, so are not included in a junit report""" return [] @@ -714,16 +778,38 @@ async def run(self, options) -> Test: TestPath = collections.namedtuple('TestPath', ['suite_name', 'test_name', 'case_name']) -class BoostTest(UnitTest): +class BoostTest(Test): """A unit test which can produce its own XML output""" + standard_args = shlex.split("--overprovisioned --unsafe-bypass-fsync 1 " + "--kernel-page-cache 1 " + "--blocked-reactor-notify-ms 2000000 --collectd 0 " + "--max-networking-io-control-blocks=100 ") + def __init__(self, test_no: int, shortname: str, suite, args: str, - casename: Optional[str], allows_compaction_groups : bool) -> None: + casename: Optional[str], allows_compaction_groups : bool, execname: Optional[str]) -> None: boost_args = [] + combined_test = True if execname else False + _shortname = shortname if casename: shortname += '.' + casename - boost_args += ['--run_test=' + casename] - super().__init__(test_no, shortname, suite, args) + if combined_test: + boost_args += ['--run_test=' + _shortname + '/' + casename] + else: + boost_args += ['--run_test=' + casename] + else: + if combined_test: + boost_args += ['--run_test=' + _shortname] + + super().__init__(test_no, shortname, suite) + if combined_test: + self.path = path_to(self.mode, "test", suite.name, execname) + else: + self.path = path_to(self.mode, "test", suite.name, shortname.split('.')[0]) + self.args = shlex.split(args) + UnitTest.standard_args + if self.mode == "coverage": + self.env.update(coverage.env(self.path)) + self.xmlout = os.path.join(suite.options.tmpdir, self.mode, "xml", self.uname + ".xunit.xml") boost_args += ['--report_level=no', '--logger=HRF,test_suite:XML,test_suite,' + self.xmlout] @@ -784,12 +870,18 @@ async def run(self, options): self.args += ['--random-seed', options.random_seed] if self.allows_compaction_groups and options.x_log2_compaction_groups: self.args += [ "--x-log2-compaction-groups", str(options.x_log2_compaction_groups) ] - return await super().run(options) + self.success = await run_test(self, options, env=self.env) + logging.info("Test %s %s", self.uname, "succeeded" if self.success else "failed ") + return self def write_junit_failure_report(self, xml_res: ET.Element) -> None: """Does not write junit report for Jenkins legacy reasons""" assert False + def print_summary(self) -> None: + print("Output of {} {}:".format(self.path, " ".join(self.args))) + print(read_log(self.log_filename)) + class CQLApprovalTest(Test): """Run a sequence of CQL commands against a standalone Scylla""" diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index 59b8627a8c48..6addd8a9f262 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -1,7 +1,5 @@ add_scylla_test(UUID_test KIND BOOST) -add_scylla_test(aggregate_fcts_test - KIND SEASTAR) add_scylla_test(allocation_strategy_test KIND BOOST) add_scylla_test(alternator_unit_test @@ -14,11 +12,6 @@ add_scylla_test(auth_passwords_test LIBRARIES auth) add_scylla_test(auth_resource_test KIND BOOST) -add_scylla_test(auth_test - KIND SEASTAR - LIBRARIES cql3) -add_scylla_test(batchlog_manager_test - KIND SEASTAR) add_scylla_test(big_decimal_test KIND BOOST LIBRARIES utils) @@ -34,8 +27,6 @@ add_scylla_test(btree_test add_scylla_test(bytes_ostream_test KIND BOOST LIBRARIES Seastar::seastar_testing) -add_scylla_test(cache_algorithm_test - KIND SEASTAR) add_scylla_test(cache_mutation_reader_test KIND SEASTAR) add_scylla_test(cached_file_test @@ -46,13 +37,9 @@ add_scylla_test(canonical_mutation_test KIND SEASTAR) add_scylla_test(cartesian_product_test KIND BOOST) -add_scylla_test(castas_fcts_test - KIND SEASTAR) add_scylla_test(cdc_generation_test KIND BOOST LIBRARIES Seastar::seastar_testing) -add_scylla_test(cdc_test - KIND SEASTAR) add_scylla_test(cell_locker_test KIND SEASTAR) add_scylla_test(checksum_utils_test @@ -63,12 +50,6 @@ add_scylla_test(chunked_vector_test KIND BOOST) add_scylla_test(clustering_ranges_walker_test KIND SEASTAR) -add_scylla_test(column_mapping_test - KIND SEASTAR) -add_scylla_test(commitlog_cleanup_test - KIND SEASTAR) -add_scylla_test(commitlog_test - KIND SEASTAR) add_scylla_test(compaction_group_test KIND SEASTAR) add_scylla_test(compound_test @@ -81,28 +62,11 @@ add_scylla_test(continuous_data_consumer_test KIND SEASTAR) add_scylla_test(counter_test KIND SEASTAR) -add_scylla_test(cql_auth_query_test - KIND SEASTAR) add_scylla_test(cql_auth_syntax_test KIND BOOST LIBRARIES cql3) -add_scylla_test(cql_functions_test - KIND SEASTAR) -add_scylla_test(cql_query_group_test - KIND SEASTAR) -add_scylla_test(cql_query_large_test - KIND SEASTAR) -add_scylla_test(cql_query_like_test - KIND SEASTAR) -add_scylla_test(cql_query_test - KIND SEASTAR - LIBRARIES cql3) add_scylla_test(crc_test KIND BOOST) -add_scylla_test(data_listeners_test - KIND SEASTAR) -add_scylla_test(database_test - KIND SEASTAR) add_scylla_test(dirty_memory_manager_test KIND SEASTAR) add_scylla_test(double_decker_test @@ -116,8 +80,6 @@ add_scylla_test(enum_option_test KIND BOOST) add_scylla_test(enum_set_test KIND BOOST) -add_scylla_test(error_injection_test - KIND SEASTAR) add_scylla_test(estimated_histogram_test KIND BOOST) add_scylla_test(exception_container_test @@ -129,10 +91,6 @@ add_scylla_test(exceptions_optimized_test add_scylla_test(expr_test KIND BOOST LIBRARIES cql3) -add_scylla_test(extensions_test - KIND SEASTAR) -add_scylla_test(filtering_test - KIND SEASTAR) add_scylla_test(flush_queue_test KIND SEASTAR) add_scylla_test(fragmented_temporary_buffer_test @@ -143,10 +101,6 @@ add_scylla_test(generic_server_test KIND SEASTAR) add_scylla_test(gossiping_property_file_snitch_test KIND SEASTAR) -add_scylla_test(group0_cmd_merge_test - KIND SEASTAR) -add_scylla_test(group0_test - KIND SEASTAR) add_scylla_test(hash_test KIND SEASTAR) add_scylla_test(hashers_test @@ -158,23 +112,17 @@ add_scylla_test(idl_test LIBRARIES idl) add_scylla_test(index_reader_test KIND SEASTAR) -add_scylla_test(index_with_paging_test - KIND SEASTAR) add_scylla_test(input_stream_test KIND BOOST LIBRARIES Seastar::seastar_testing) add_scylla_test(intrusive_array_test KIND SEASTAR) -add_scylla_test(json_cql_query_test - KIND SEASTAR) add_scylla_test(json_test KIND BOOST LIBRARIES cql3) add_scylla_test(keys_test KIND BOOST LIBRARIES idl schema) -add_scylla_test(large_paging_state_test - KIND SEASTAR) add_scylla_test(like_matcher_test KIND BOOST LIBRARIES utils) @@ -184,8 +132,6 @@ add_scylla_test(linearizing_input_stream_test KIND BOOST) add_scylla_test(lister_test KIND SEASTAR) -add_scylla_test(loading_cache_test - KIND SEASTAR) add_scylla_test(locator_topology_test KIND SEASTAR) add_scylla_test(log_heap_test @@ -201,12 +147,6 @@ add_scylla_test(managed_vector_test KIND SEASTAR) add_scylla_test(map_difference_test KIND BOOST) -add_scylla_test(memtable_test - KIND SEASTAR) -add_scylla_test(multishard_combining_reader_as_mutation_source_test - KIND SEASTAR) -add_scylla_test(multishard_mutation_query_test - KIND SEASTAR) add_scylla_test(murmur_hash_test KIND BOOST) add_scylla_test(mutation_fragment_test @@ -215,52 +155,30 @@ add_scylla_test(mutation_query_test KIND SEASTAR) add_scylla_test(mutation_reader_another_test KIND SEASTAR) -add_scylla_test(mutation_reader_test - KIND SEASTAR) add_scylla_test(mutation_test KIND SEASTAR) -add_scylla_test(mutation_writer_test - KIND SEASTAR) add_scylla_test(mvcc_test KIND SEASTAR) -add_scylla_test(network_topology_strategy_test - KIND SEASTAR) add_scylla_test(nonwrapping_interval_test KIND BOOST) add_scylla_test(observable_test KIND BOOST) add_scylla_test(partitioner_test KIND SEASTAR) -add_scylla_test(per_partition_rate_limit_test - KIND SEASTAR) add_scylla_test(pretty_printers_test KIND BOOST) -add_scylla_test(querier_cache_test - KIND SEASTAR) -add_scylla_test(query_processor_test - KIND SEASTAR) add_scylla_test(radix_tree_test KIND SEASTAR) add_scylla_test(range_tombstone_list_test KIND BOOST) add_scylla_test(rate_limiter_test KIND SEASTAR) -add_scylla_test(reader_concurrency_semaphore_test - KIND SEASTAR) add_scylla_test(recent_entries_map_test KIND SEASTAR) -add_scylla_test(repair_test - KIND SEASTAR) -add_scylla_test(restrictions_test - KIND SEASTAR) add_scylla_test(result_utils_test KIND SEASTAR) add_scylla_test(reusable_buffer_test KIND SEASTAR) -add_scylla_test(role_manager_test - KIND SEASTAR) -add_scylla_test(row_cache_test - KIND SEASTAR) add_scylla_test(rust_test KIND BOOST LIBRARIES inc) @@ -270,25 +188,17 @@ add_scylla_test(aws_errors_test KIND BOOST) add_scylla_test(aws_error_injection_test KIND SEASTAR) -add_scylla_test(schema_change_test - KIND SEASTAR) add_scylla_test(schema_changes_test KIND SEASTAR) add_scylla_test(schema_loader_test KIND SEASTAR LIBRARIES tools) -add_scylla_test(schema_registry_test - KIND SEASTAR) -add_scylla_test(secondary_index_test - KIND SEASTAR) add_scylla_test(serialization_test KIND BOOST) add_scylla_test(serialized_action_test KIND SEASTAR) add_scylla_test(service_level_controller_test KIND SEASTAR) -add_scylla_test(sessions_test - KIND SEASTAR) add_scylla_test(small_vector_test KIND SEASTAR) add_scylla_test(snitch_reset_test @@ -297,14 +207,10 @@ add_scylla_test(sorting_test KIND SEASTAR) add_scylla_test(sstable_3_x_test KIND SEASTAR) -add_scylla_test(sstable_compaction_test - KIND SEASTAR) add_scylla_test(sstable_conforms_to_mutation_source_test KIND SEASTAR) add_scylla_test(sstable_datafile_test KIND SEASTAR) -add_scylla_test(sstable_directory_test - KIND SEASTAR) add_scylla_test(sstable_generation_test KIND BOOST) add_scylla_test(sstable_move_test @@ -315,64 +221,34 @@ add_scylla_test(sstable_partition_index_cache_test KIND SEASTAR) add_scylla_test(sstable_resharding_test KIND SEASTAR) -add_scylla_test(sstable_set_test - KIND SEASTAR) add_scylla_test(sstable_test KIND SEASTAR) add_scylla_test(stall_free_test KIND SEASTAR) -add_scylla_test(statement_restrictions_test - KIND SEASTAR - LIBRARIES cql3) -add_scylla_test(storage_proxy_test - KIND SEASTAR) add_scylla_test(string_format_test KIND BOOST) add_scylla_test(summary_test KIND BOOST) -add_scylla_test(tablets_test - KIND SEASTAR) add_scylla_test(tagged_integer_test KIND SEASTAR) add_scylla_test(token_metadata_test KIND SEASTAR) add_scylla_test(top_k_test KIND BOOST) -add_scylla_test(tracing_test - KIND SEASTAR) add_scylla_test(transport_test KIND SEASTAR) add_scylla_test(types_test KIND SEASTAR) -add_scylla_test(user_function_test - KIND SEASTAR - LIBRARIES idl) -add_scylla_test(user_types_test - KIND SEASTAR) add_scylla_test(utf8_test KIND BOOST LIBRARIES utils) -add_scylla_test(view_build_test - KIND SEASTAR) -add_scylla_test(view_complex_test - KIND SEASTAR) -add_scylla_test(view_schema_ckey_test - KIND SEASTAR) -add_scylla_test(view_schema_pkey_test - KIND SEASTAR) -add_scylla_test(view_schema_test - KIND SEASTAR) add_scylla_test(vint_serialization_test KIND BOOST - LIBRARIES - scylla-main - utils) -add_scylla_test(virtual_reader_test - KIND SEASTAR) + LIBRARIES + scylla-main + utils) add_scylla_test(virtual_table_mutation_source_test KIND SEASTAR) -add_scylla_test(virtual_table_test - KIND SEASTAR) add_scylla_test(wasm_alloc_test KIND SEASTAR) add_scylla_test(wasm_test @@ -382,6 +258,75 @@ add_scylla_test(wrapping_interval_test add_scylla_test(address_map_test KIND SEASTAR) +add_scylla_test(combined_tests + KIND SEASTAR + SOURCES + combined_tests.cc + aggregate_fcts_test.cc + auth_test.cc + batchlog_manager_test.cc + cache_algorithm_test.cc + castas_fcts_test.cc + cdc_test.cc + column_mapping_test.cc + commitlog_cleanup_test.cc + commitlog_test.cc + cql_auth_query_test.cc + cql_functions_test.cc + cql_query_group_test.cc + cql_query_large_test.cc + cql_query_like_test.cc + cql_query_test.cc + database_test.cc + data_listeners_test.cc + error_injection_test.cc + extensions_test.cc + filtering_test.cc + group0_cmd_merge_test.cc + group0_test.cc + index_with_paging_test.cc + json_cql_query_test.cc + large_paging_state_test.cc + loading_cache_test.cc + memtable_test.cc + multishard_combining_reader_as_mutation_source_test.cc + multishard_mutation_query_test.cc + mutation_reader_test.cc + mutation_writer_test.cc + network_topology_strategy_test.cc + per_partition_rate_limit_test.cc + querier_cache_test.cc + query_processor_test.cc + reader_concurrency_semaphore_test.cc + repair_test.cc + role_manager_test.cc + restrictions_test.cc + row_cache_test.cc + schema_change_test.cc + schema_registry_test.cc + secondary_index_test.cc + sessions_test.cc + sstable_compaction_test.cc + sstable_directory_test.cc + sstable_set_test.cc + statement_restrictions_test.cc + storage_proxy_test.cc + tablets_test.cc + tracing_test.cc + user_function_test.cc + user_types_test.cc + view_build_test.cc + view_complex_test.cc + view_schema_ckey_test.cc + view_schema_pkey_test.cc + view_schema_test.cc + virtual_reader_test.cc + virtual_table_test.cc + LIBRARIES + cql3 + idl + tools + utils) if(EXISTS "${CMAKE_CURRENT_SOURCE_DIR}/suite.yaml") set(scylla_tests "${scylla_tests}" PARENT_SCOPE) endif() diff --git a/test/boost/aggregate_fcts_test.cc b/test/boost/aggregate_fcts_test.cc index c58460297f89..ccd266119b8d 100644 --- a/test/boost/aggregate_fcts_test.cc +++ b/test/boost/aggregate_fcts_test.cc @@ -12,7 +12,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -20,6 +21,8 @@ #include "types/set.hh" +BOOST_AUTO_TEST_SUITE(aggregate_fcts_test) + namespace { void create_table(cql_test_env& e) { @@ -267,3 +270,5 @@ SEASTAR_TEST_CASE(test_minmax_on_set) { } }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/auth_test.cc b/test/boost/auth_test.cc index 5a48e721a476..f68083f4f5b0 100644 --- a/test/boost/auth_test.cc +++ b/test/boost/auth_test.cc @@ -21,7 +21,9 @@ #include "cql3/CqlParser.hpp" #include "exceptions/exceptions.hh" #include "service/raft/raft_group0_client.hh" -#include "test/lib/scylla_test_case.hh" + +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" #include "test/lib/exception_utils.hh" @@ -34,6 +36,8 @@ #include "db/config.hh" +BOOST_AUTO_TEST_SUITE(auth_test) + cql_test_config auth_on(bool with_authorizer = true) { cql_test_config cfg; if (with_authorizer) { @@ -407,3 +411,5 @@ SEASTAR_TEST_CASE(test_try_describe_schema_with_internals_and_passwords_as_anony exception_predicate::message_equals("DESCRIBE SCHEMA WITH INTERNALS AND PASSWORDS can only be issued by a superuser")); }, auth_on(true)); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/batchlog_manager_test.cc b/test/boost/batchlog_manager_test.cc index fdd3be54585c..74de28c88448 100644 --- a/test/boost/batchlog_manager_test.cc +++ b/test/boost/batchlog_manager_test.cc @@ -12,7 +12,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include @@ -24,6 +25,8 @@ #include "message/messaging_service.hh" #include "service/storage_proxy.hh" +BOOST_AUTO_TEST_SUITE(batchlog_manager_test) + static atomic_cell make_atomic_cell(data_type dt, bytes value) { return atomic_cell::make_live(*dt, 0, std::move(value)); }; @@ -66,3 +69,4 @@ SEASTAR_TEST_CASE(test_execute_batch) { }); } +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cache_algorithm_test.cc b/test/boost/cache_algorithm_test.cc index de97618e9d07..e1f1369c376d 100644 --- a/test/boost/cache_algorithm_test.cc +++ b/test/boost/cache_algorithm_test.cc @@ -7,11 +7,14 @@ */ #include +#undef SEASTAR_TESTING_MAIN #include #include "test/lib/cql_test_env.hh" #include "test/lib/log.hh" #include "db/config.hh" +BOOST_AUTO_TEST_SUITE(cache_algorithm_test) + // These tests are slow, and tuned to a particular amount of memory // (and --memory is ignored in debug mode). // Hence they are not run in debug. @@ -217,3 +220,4 @@ SEASTAR_TEST_CASE(test_index_is_cached_in_big_partition_workload) { } #endif +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/castas_fcts_test.cc b/test/boost/castas_fcts_test.cc index c253e669f826..c3bcd88f966b 100644 --- a/test/boost/castas_fcts_test.cc +++ b/test/boost/castas_fcts_test.cc @@ -14,7 +14,8 @@ #include "utils/big_decimal.hh" #include "exceptions/exceptions.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" #include "test/lib/exception_utils.hh" @@ -22,6 +23,8 @@ #include #include "transport/messages/result_message.hh" +BOOST_AUTO_TEST_SUITE(castas_fcts_test) + namespace { template @@ -615,3 +618,5 @@ SEASTAR_TEST_CASE(test_identity_casts) { } // FIXME: Add test with user-defined functions after they are available. + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cdc_test.cc b/test/boost/cdc_test.cc index 79faa267f851..75f74d8b2794 100644 --- a/test/boost/cdc_test.cc +++ b/test/boost/cdc_test.cc @@ -8,7 +8,9 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include +#include #include #include #include @@ -42,6 +44,8 @@ api::timestamp_type find_timestamp(const mutation&); utils::UUID generate_timeuuid(api::timestamp_type); } +BOOST_AUTO_TEST_SUITE(cdc_test) + SEASTAR_THREAD_TEST_CASE(test_find_mutation_timestamp) { do_with_cql_env_thread([] (cql_test_env& e) { cquery_nofail(e, "CREATE TYPE ut (a int, b int)"); @@ -373,6 +377,8 @@ SEASTAR_THREAD_TEST_CASE(test_cdc_log_schema) { }).get(); } +} // cdc_test namespace + static std::vector> to_bytes(const cql_transport::messages::result_message::rows& rows) { auto rs = rows.rs().result_set().rows(); std::vector> results; @@ -426,6 +432,8 @@ static auto select_log(cql_test_env& e, const sstring& table_name) { return rows; }; +namespace cdc_test { + SEASTAR_THREAD_TEST_CASE(test_primary_key_logging) { do_with_cql_env_thread([](cql_test_env& e) { cquery_nofail(e, "CREATE TABLE ks.tbl (pk int, pk2 int, ck int, ck2 int, s int STATIC, val int, PRIMARY KEY((pk, pk2), ck, ck2)) WITH cdc = {'enabled':'true'}"); @@ -1305,6 +1313,8 @@ SEASTAR_THREAD_TEST_CASE(test_update_insert_delete_distinction) { }).get(); } +} // namespace cdc_test + static std::vector> get_result(cql_test_env& e, const std::vector& col_types, const sstring& query) { auto deser = [] (const data_type& t, const bytes_opt& b) -> data_value { @@ -1330,6 +1340,8 @@ static std::vector> get_result(cql_test_env& e, return res; } +namespace cdc_test { + SEASTAR_THREAD_TEST_CASE(test_change_splitting) { do_with_cql_env_thread([](cql_test_env& e) { using oper_ut = std::underlying_type_t; @@ -2077,3 +2089,5 @@ SEASTAR_THREAD_TEST_CASE(test_image_deleted_column) { perform_test(true); }).get(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/column_mapping_test.cc b/test/boost/column_mapping_test.cc index 4f0b2224c7d5..87c8ab96d28e 100644 --- a/test/boost/column_mapping_test.cc +++ b/test/boost/column_mapping_test.cc @@ -6,7 +6,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -15,6 +16,8 @@ #include "db/schema_tables.hh" #include "transport/messages/result_message.hh" +BOOST_AUTO_TEST_SUITE(column_mapping_test) + SEASTAR_TEST_CASE(test_column_mapping_persistence) { return do_with_cql_env_thread([] (cql_test_env& e) { // Check that column mapping history is empty initially @@ -84,3 +87,5 @@ SEASTAR_TEST_CASE(test_column_mapping_ttl_check) { BOOST_REQUIRE(ttl_val > 0); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/combined_tests.cc b/test/boost/combined_tests.cc new file mode 100644 index 000000000000..fa4a06c216ed --- /dev/null +++ b/test/boost/combined_tests.cc @@ -0,0 +1,9 @@ +/* + * Copyright (C) 2024-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: AGPL-3.0-or-later + */ + +#include "test/lib/scylla_test_case.hh" diff --git a/test/boost/commitlog_cleanup_test.cc b/test/boost/commitlog_cleanup_test.cc index 6df6ed79acb4..c348c66f1ff4 100644 --- a/test/boost/commitlog_cleanup_test.cc +++ b/test/boost/commitlog_cleanup_test.cc @@ -7,13 +7,17 @@ */ #include -#include "test/lib/scylla_test_case.hh" + +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "db/commitlog/commitlog_replayer.hh" #include "db/commitlog/commitlog.hh" #include "db/config.hh" #include "db/system_keyspace.hh" +BOOST_AUTO_TEST_SUITE(commitlog_cleanup_test) + // Test that `canonical_token_range(tr)` contains the same tokens as `tr`. SEASTAR_TEST_CASE(test_canonical_token_range) { const int64_t arbitrary_token = -42; @@ -219,3 +223,5 @@ SEASTAR_TEST_CASE(test_commitlog_cleanup_record_gc) { BOOST_REQUIRE_EQUAL(get_num_records(), 1); }, cfg); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/commitlog_test.cc b/test/boost/commitlog_test.cc index c1f3d983880b..c3ea81cfcd38 100644 --- a/test/boost/commitlog_test.cc +++ b/test/boost/commitlog_test.cc @@ -19,7 +19,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include #include @@ -47,6 +48,8 @@ #include "test/lib/key_utils.hh" #include "test/lib/test_utils.hh" +BOOST_AUTO_TEST_SUITE(commitlog_test) + using namespace db; static future<> cl_test(commitlog::config cfg, noncopyable_function (commitlog&)> f) { @@ -2056,3 +2059,5 @@ SEASTAR_TEST_CASE(test_commitlog_buffer_size_counter) { co_await log.shutdown(); co_await log.clear(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cql_auth_query_test.cc b/test/boost/cql_auth_query_test.cc index 0c77d00f8161..15c0dbabfc54 100644 --- a/test/boost/cql_auth_query_test.cc +++ b/test/boost/cql_auth_query_test.cc @@ -14,7 +14,9 @@ #include #include #include -#include "test/lib/scylla_test_case.hh" + +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/test_utils.hh" #include "auth/authenticated_user.hh" @@ -27,6 +29,8 @@ #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" +BOOST_AUTO_TEST_SUITE(cql_auth_query_test) + static const auto alice = std::string_view("alice"); static const auto bob = std::string_view("bob"); @@ -339,3 +343,5 @@ SEASTAR_TEST_CASE(modify_table_with_index) { {{int32_type->decompose(14)}}); }, db_config_with_auth()); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cql_functions_test.cc b/test/boost/cql_functions_test.cc index 4e78b0b6043f..def66d186b76 100644 --- a/test/boost/cql_functions_test.cc +++ b/test/boost/cql_functions_test.cc @@ -16,7 +16,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -30,6 +31,8 @@ #include "types/set.hh" #include "schema/schema_builder.hh" +BOOST_AUTO_TEST_SUITE(cql_functions_test) + using namespace std::literals::chrono_literals; SEASTAR_TEST_CASE(test_functions) { @@ -408,3 +411,5 @@ SEASTAR_TEST_CASE(test_aggregate_functions_map_type) { ).test_min_max_count(); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cql_query_group_test.cc b/test/boost/cql_query_group_test.cc index 8e6854ffa152..47179f714e1e 100644 --- a/test/boost/cql_query_group_test.cc +++ b/test/boost/cql_query_group_test.cc @@ -14,7 +14,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -22,6 +23,8 @@ #include #include "test/lib/exception_utils.hh" +BOOST_AUTO_TEST_SUITE(cql_query_group_test) + using namespace std::literals::chrono_literals; SEASTAR_TEST_CASE(test_group_by_syntax) { @@ -233,3 +236,5 @@ SEASTAR_TEST_CASE(test_group_by_null_clustering) { return make_ready_future<>(); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cql_query_large_test.cc b/test/boost/cql_query_large_test.cc index a8876b761692..aef1f4e49cc5 100644 --- a/test/boost/cql_query_large_test.cc +++ b/test/boost/cql_query_large_test.cc @@ -14,7 +14,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -29,6 +30,8 @@ #include "compaction/compaction_manager.hh" #include "schema/schema_builder.hh" +BOOST_AUTO_TEST_SUITE(cql_query_large_test) + using namespace std::literals::chrono_literals; SEASTAR_TEST_CASE(test_large_partitions) { @@ -195,3 +198,5 @@ SEASTAR_TEST_CASE(test_insert_large_collection_values) { }); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cql_query_like_test.cc b/test/boost/cql_query_like_test.cc index d83199faf9e9..0d921d98ae6c 100644 --- a/test/boost/cql_query_like_test.cc +++ b/test/boost/cql_query_like_test.cc @@ -12,7 +12,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -20,6 +21,8 @@ #include #include "test/lib/exception_utils.hh" +BOOST_AUTO_TEST_SUITE(cql_query_like_test) + using namespace std::literals::chrono_literals; namespace { @@ -178,3 +181,5 @@ SEASTAR_TEST_CASE(test_like_operator_on_token) { exception_predicate::message_contains("token function")); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/cql_query_test.cc b/test/boost/cql_query_test.cc index cbc7a0663498..008b07525086 100644 --- a/test/boost/cql_query_test.cc +++ b/test/boost/cql_query_test.cc @@ -19,7 +19,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -53,6 +54,7 @@ #include "replica/schema_describe_helper.hh" +BOOST_AUTO_TEST_SUITE(cql_query_test) using namespace std::literals::chrono_literals; @@ -4735,6 +4737,8 @@ SEASTAR_TEST_CASE(test_impossible_where) { }); } +} // cql_query_test namespace + // FIXME: copy-pasta static bool has_more_pages(::shared_ptr res) { auto rows = dynamic_pointer_cast(res); @@ -4758,6 +4762,8 @@ static lw_shared_ptr extract_paging_state(::shared return make_lw_shared(*paging_state); }; +namespace cql_query_test { + SEASTAR_THREAD_TEST_CASE(test_query_limit) { cql_test_config cfg; @@ -5921,3 +5927,5 @@ SEASTAR_TEST_CASE(test_schema_change_events) { BOOST_REQUIRE(dynamic_pointer_cast(res)); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/data_listeners_test.cc b/test/boost/data_listeners_test.cc index a9a5ae695188..42f3be2a51b7 100644 --- a/test/boost/data_listeners_test.cc +++ b/test/boost/data_listeners_test.cc @@ -8,13 +8,16 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/log.hh" #include "readers/filtering.hh" #include "db/data_listeners.hh" +BOOST_AUTO_TEST_SUITE(data_listeners_test) + using namespace std::chrono_literals; class table_listener : public db::data_listener { @@ -111,3 +114,5 @@ SEASTAR_TEST_CASE(test_dlistener_t2) { BOOST_REQUIRE_EQUAL(0, res.write); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/database_test.cc b/test/boost/database_test.cc index 5933d2600c1b..10a2381d678c 100644 --- a/test/boost/database_test.cc +++ b/test/boost/database_test.cc @@ -13,7 +13,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include #include @@ -54,10 +55,10 @@ using namespace std::chrono_literals; using namespace sstables; -class database_test { +class database_test_wrapper { replica::database& _db; public: - explicit database_test(replica::database& db) : _db(db) { } + explicit database_test_wrapper(replica::database& db) : _db(db) { } reader_concurrency_semaphore& get_user_read_concurrency_semaphore() { return _db._read_concurrency_sem; @@ -99,6 +100,8 @@ future<> do_with_cql_env_and_compaction_groups(std::function(); cfg->auto_snapshot.set(false); @@ -1125,11 +1128,11 @@ SEASTAR_THREAD_TEST_CASE(reader_concurrency_semaphore_selection_test) { destroy_scheduling_group(unknown_scheduling_group).get(); }); - const auto user_semaphore = std::mem_fn(&database_test::get_user_read_concurrency_semaphore); - const auto system_semaphore = std::mem_fn(&database_test::get_system_read_concurrency_semaphore); - const auto streaming_semaphore = std::mem_fn(&database_test::get_streaming_read_concurrency_semaphore); + const auto user_semaphore = std::mem_fn(&database_test_wrapper::get_user_read_concurrency_semaphore); + const auto system_semaphore = std::mem_fn(&database_test_wrapper::get_system_read_concurrency_semaphore); + const auto streaming_semaphore = std::mem_fn(&database_test_wrapper::get_streaming_read_concurrency_semaphore); - std::vector>> scheduling_group_and_expected_semaphore{ + std::vector>> scheduling_group_and_expected_semaphore{ {default_scheduling_group(), system_semaphore} }; @@ -1146,7 +1149,7 @@ SEASTAR_THREAD_TEST_CASE(reader_concurrency_semaphore_selection_test) { do_with_cql_env_and_compaction_groups([&scheduling_group_and_expected_semaphore] (cql_test_env& e) { auto& db = e.local_db(); - database_test tdb(db); + database_test_wrapper tdb(db); for (const auto& [sched_group, expected_sem_getter] : scheduling_group_and_expected_semaphore) { with_scheduling_group(sched_group, [&db, sched_group = sched_group, expected_sem_ptr = &expected_sem_getter(tdb)] { auto& sem = db.get_reader_concurrency_semaphore(); @@ -1196,7 +1199,7 @@ SEASTAR_THREAD_TEST_CASE(max_result_size_for_query_selection_test) { do_with_cql_env_and_compaction_groups([&scheduling_group_and_expected_max_result_size] (cql_test_env& e) { auto& db = e.local_db(); - database_test tdb(db); + database_test_wrapper tdb(db); for (const auto& [sched_group, expected_max_size] : scheduling_group_and_expected_max_result_size) { with_scheduling_group(sched_group, [&db, sched_group = sched_group, expected_max_size = expected_max_size] { const auto max_size = db.get_query_max_result_size(); @@ -1418,7 +1421,7 @@ SEASTAR_TEST_CASE(database_drop_column_family_clears_querier_cache) { auto q = query::querier( tbl.as_mutation_source(), tbl.schema(), - database_test(db).get_user_read_concurrency_semaphore().make_tracking_only_permit(s, "test", db::no_timeout, {}), + database_test_wrapper(db).get_user_read_concurrency_semaphore().make_tracking_only_permit(s, "test", db::no_timeout, {}), query::full_partition_range, s->full_slice(), nullptr); @@ -1513,3 +1516,5 @@ SEASTAR_TEST_CASE(mutation_dump_generated_schema_deterministic_id_version) { return make_ready_future<>(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/error_injection_test.cc b/test/boost/error_injection_test.cc index ba4c166902f1..3341a293c107 100644 --- a/test/boost/error_injection_test.cc +++ b/test/boost/error_injection_test.cc @@ -9,7 +9,8 @@ #include #include "test/lib/cql_test_env.hh" #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "utils/error_injection.hh" #include "db/timeout_clock.hh" @@ -19,6 +20,8 @@ #include "utils/log.hh" #include +BOOST_AUTO_TEST_SUITE(error_injection_test) + using namespace std::literals::chrono_literals; static logging::logger flogger("error_injection_test"); @@ -500,3 +503,5 @@ SEASTAR_TEST_CASE(test_inject_cql) { }); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/extensions_test.cc b/test/boost/extensions_test.cc index 24be9721621b..d3e33f58bd2c 100644 --- a/test/boost/extensions_test.cc +++ b/test/boost/extensions_test.cc @@ -12,7 +12,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -26,6 +27,8 @@ #include "transport/messages/result_message.hh" #include "utils/overloaded_functor.hh" +BOOST_AUTO_TEST_SUITE(extensions_test) + class dummy_ext : public schema_extension { public: dummy_ext(bytes b) : _bytes(b) {} @@ -206,3 +209,5 @@ SEASTAR_TEST_CASE(test_extension_remove) { }); }, ::make_shared(ext)); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/filtering_test.cc b/test/boost/filtering_test.cc index a9f663362007..961692458535 100644 --- a/test/boost/filtering_test.cc +++ b/test/boost/filtering_test.cc @@ -15,7 +15,8 @@ #include #include "test/lib/eventually.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -26,6 +27,8 @@ #include "types/set.hh" #include "types/map.hh" +BOOST_AUTO_TEST_SUITE(filtering_test) + using namespace std::literals::chrono_literals; @@ -1152,3 +1155,5 @@ SEASTAR_TEST_CASE(test_filtering) { }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/group0_cmd_merge_test.cc b/test/boost/group0_cmd_merge_test.cc index 3219d6e3118c..7c7ed5acb7cc 100644 --- a/test/boost/group0_cmd_merge_test.cc +++ b/test/boost/group0_cmd_merge_test.cc @@ -8,7 +8,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "db/config.hh" @@ -24,6 +25,8 @@ #include "utils/error_injection.hh" #include "test/lib/expr_test_utils.hh" +BOOST_AUTO_TEST_SUITE(group0_cmd_merge_test) + const auto OLD_TIMEUUID = utils::UUID_gen::get_time_UUID(std::chrono::system_clock::time_point::min()); static service::group0_command create_command(utils::UUID id) { @@ -120,3 +123,5 @@ SEASTAR_TEST_CASE(test_group0_cmd_merge) { }, cfg); #endif } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/group0_test.cc b/test/boost/group0_test.cc index 2b10ef48f3a6..5a6fc27f7daf 100644 --- a/test/boost/group0_test.cc +++ b/test/boost/group0_test.cc @@ -6,7 +6,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_assertions.hh" #include @@ -32,6 +33,8 @@ static future get_history_size(cql_test_env& e) { co_return (co_await fetch_rows(e, "select * from system.group0_history")).size(); } +BOOST_AUTO_TEST_SUITE(group0_test) + SEASTAR_TEST_CASE(test_abort_server_on_background_error) { #ifndef SCYLLA_ENABLE_ERROR_INJECTION std::cerr << "Skipping test as it depends on error injection. Please run in mode where it's enabled (debug,dev).\n"; @@ -341,3 +344,5 @@ SEASTAR_TEST_CASE(test_group0_batch) { co_await std::move(mc1).commit(rclient, as, ::service::raft_timeout{}); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/index_with_paging_test.cc b/test/boost/index_with_paging_test.cc index e397e51998fa..7b4e52f522b5 100644 --- a/test/boost/index_with_paging_test.cc +++ b/test/boost/index_with_paging_test.cc @@ -6,7 +6,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" #include "test/lib/eventually.hh" @@ -14,6 +15,8 @@ #include "cql3/query_processor.hh" #include "transport/messages/result_message.hh" +BOOST_AUTO_TEST_SUITE(index_with_paging_test) + SEASTAR_TEST_CASE(test_index_with_paging) { return do_with_cql_env_thread([] (auto& e) { e.execute_cql("CREATE TABLE tab (pk int, ck text, v int, v2 int, v3 text, PRIMARY KEY (pk, ck))").get(); @@ -102,3 +105,5 @@ SEASTAR_TEST_CASE(test_index_with_paging_with_base_short_read_no_ck) { }); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/json_cql_query_test.cc b/test/boost/json_cql_query_test.cc index 884d9e08fde5..0a5a89bfd8be 100644 --- a/test/boost/json_cql_query_test.cc +++ b/test/boost/json_cql_query_test.cc @@ -14,7 +14,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -27,6 +28,8 @@ #include "types/list.hh" #include "utils/rjson.hh" +BOOST_AUTO_TEST_SUITE(json_cql_query_test) + using namespace std::literals::chrono_literals; SEASTAR_TEST_CASE(test_select_json_types) { @@ -788,3 +791,5 @@ SEASTAR_TEST_CASE(test_unpack_decimal){ assert_that(msg).is_rows().with_rows({{int32_type->decompose(1), lt_val}}); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/large_paging_state_test.cc b/test/boost/large_paging_state_test.cc index 57598cf58e98..0cbcc8df783f 100644 --- a/test/boost/large_paging_state_test.cc +++ b/test/boost/large_paging_state_test.cc @@ -9,12 +9,15 @@ #include "utils/assert.hh" #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "transport/messages/result_message.hh" #include "types/types.hh" +BOOST_AUTO_TEST_SUITE(large_paging_state_test) + static lw_shared_ptr extract_paging_state(::shared_ptr res) { auto rows = dynamic_pointer_cast(res); auto paging_state = rows->rs().get_metadata().paging_state(); @@ -114,3 +117,5 @@ SEASTAR_TEST_CASE(test_use_high_bits_of_remaining_rows_in_paging_state_filtering } }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/loading_cache_test.cc b/test/boost/loading_cache_test.cc index d99c28167df5..691ecfc23e94 100644 --- a/test/boost/loading_cache_test.cc +++ b/test/boost/loading_cache_test.cc @@ -20,7 +20,8 @@ #include "seastarx.hh" #include "test/lib/eventually.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/tmpdir.hh" #include "test/lib/log.hh" @@ -31,6 +32,8 @@ #include #include +BOOST_AUTO_TEST_SUITE(loading_cache_test) + /// Get a random integer in the [0, max) range. /// \param max bound of the random value range /// \return The uniformly distributed random integer from the [0, \ref max) range. @@ -851,3 +854,5 @@ SEASTAR_TEST_CASE(test_prepared_statement_small_cache) { } }, small_cache_config); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/memtable_test.cc b/test/boost/memtable_test.cc index 9a65682f4b68..daa7d85c51d1 100644 --- a/test/boost/memtable_test.cc +++ b/test/boost/memtable_test.cc @@ -12,7 +12,8 @@ #include "db/config.hh" #include "utils/assert.hh" #include "utils/UUID_gen.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "schema/schema_builder.hh" #include @@ -42,6 +43,8 @@ #include "db/config.hh" #include "service/storage_service.hh" +BOOST_AUTO_TEST_SUITE(memtable_test) + using namespace std::literals::chrono_literals; static api::timestamp_type next_timestamp() { @@ -1481,3 +1484,5 @@ SEASTAR_TEST_CASE(test_ext_config_exceptions_in_flush_on_sstable_open) { [] { throw db::extension_storage_misconfigured(get_name()); } ); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/multishard_combining_reader_as_mutation_source_test.cc b/test/boost/multishard_combining_reader_as_mutation_source_test.cc index f5fe28faee54..c3543e3f3b47 100644 --- a/test/boost/multishard_combining_reader_as_mutation_source_test.cc +++ b/test/boost/multishard_combining_reader_as_mutation_source_test.cc @@ -12,7 +12,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/mutation_source_test.hh" #include "test/lib/cql_test_env.hh" @@ -23,6 +24,8 @@ #include "schema/schema_registry.hh" #include "readers/forwardable_v2.hh" +BOOST_AUTO_TEST_SUITE(multishard_combining_reader_as_mutation_source_test) + // It has to be a container that does not invalidate pointers static std::list keep_alive_sharder; @@ -154,3 +157,5 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_with_tiny_buffer_rever return make_ready_future<>(); }).get(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/multishard_mutation_query_test.cc b/test/boost/multishard_mutation_query_test.cc index c17a0e141aaa..46ebd328f368 100644 --- a/test/boost/multishard_mutation_query_test.cc +++ b/test/boost/multishard_mutation_query_test.cc @@ -28,7 +28,9 @@ #include "db/paxos_grace_seconds_extension.hh" #include "db/per_partition_rate_limit_extension.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include +#include #include #include @@ -189,6 +191,8 @@ static void require_eventually_empty_caches(distributed& db, tests::require(eventually_true(aggregated_population_is_zero)); } +BOOST_AUTO_TEST_SUITE(multishard_mutation_query_test) + // Best run with SMP>=2 SEASTAR_THREAD_TEST_CASE(test_abandoned_read) { do_with_cql_env_thread([] (cql_test_env& env) -> future<> { @@ -222,6 +226,8 @@ SEASTAR_THREAD_TEST_CASE(test_abandoned_read) { }, cql_config_with_extensions()).get(); } +} // multishard_mutation_query_test namespace + static std::vector read_all_partitions_one_by_one(distributed& db, schema_ptr s, std::vector pkeys, const query::partition_slice& slice) { const auto& sharder = s->get_sharder(); @@ -527,6 +533,8 @@ void check_results_are_equal(std::vector& results1, std::vector=2 SEASTAR_THREAD_TEST_CASE(test_read_all) { do_with_cql_env_thread([] (cql_test_env& env) -> future<> { @@ -809,6 +817,8 @@ SEASTAR_THREAD_TEST_CASE(test_read_reversed) { }, cql_config_with_extensions()).get(); } +} // multishard_mutation_query_test namespace + namespace { class buffer_ostream { @@ -1107,6 +1117,8 @@ run_fuzzy_test_workload(fuzzy_test_config cfg, distributed& d } // namespace +namespace multishard_mutation_query_test { + SEASTAR_THREAD_TEST_CASE(fuzzy_test) { auto cql_cfg = cql_config_with_extensions(); cql_cfg.db_config->enable_commitlog(false); @@ -1158,3 +1170,5 @@ SEASTAR_THREAD_TEST_CASE(fuzzy_test) { return make_ready_future<>(); }, cql_cfg).get(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/mutation_reader_test.cc b/test/boost/mutation_reader_test.cc index e9280e4d4120..4f364b15b8ac 100644 --- a/test/boost/mutation_reader_test.cc +++ b/test/boost/mutation_reader_test.cc @@ -22,7 +22,8 @@ #include #include "sstables/generation_type.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/eventually.hh" #include "test/lib/mutation_assertions.hh" @@ -66,6 +67,8 @@ #include "readers/evictable.hh" #include "readers/queue.hh" +BOOST_AUTO_TEST_SUITE(mutation_reader_test) + static schema_ptr make_schema() { return schema_builder("ks", "cf") .with_column("pk", bytes_type, column_kind::partition_key) @@ -4623,3 +4626,5 @@ SEASTAR_TEST_CASE(test_multishard_reader_buffer_hint_small_partitions) { } }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/mutation_writer_test.cc b/test/boost/mutation_writer_test.cc index 0871f092b81e..b137bab337b1 100644 --- a/test/boost/mutation_writer_test.cc +++ b/test/boost/mutation_writer_test.cc @@ -8,7 +8,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include #include @@ -36,6 +37,8 @@ #include "readers/generating_v2.hh" #include "readers/combined.hh" +BOOST_AUTO_TEST_SUITE(mutation_writer_test) + using namespace mutation_writer; struct generate_error_tag { }; @@ -590,3 +593,5 @@ SEASTAR_THREAD_TEST_CASE(test_token_group_based_splitting_mutation_writer) { assert_that_segregator_produces_correct_data(buckets, muts, semaphore.make_permit(), random_schema); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index 22e79799a054..304ed980d2f0 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -16,7 +16,8 @@ #include "utils/sequenced_set.hh" #include "utils/to_string.hh" #include "locator/network_topology_strategy.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include #include "utils/log.hh" @@ -422,6 +423,7 @@ void heavy_origin_test() { full_ring_check(ring_points, config_options, ars_ptr, stm.get()); } +BOOST_AUTO_TEST_SUITE(network_topology_strategy_test) SEASTAR_THREAD_TEST_CASE(NetworkTopologyStrategy_simple) { return simple_test(); @@ -940,6 +942,8 @@ SEASTAR_TEST_CASE(test_invalid_dcs) { }); } +} // namespace network_topology_strategy_test + namespace locator { void topology::test_compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const { @@ -1003,6 +1007,8 @@ void topology::test_sort_by_proximity(const locator::host_id& address, const hos } // namespace locator +namespace network_topology_strategy_test { + SEASTAR_THREAD_TEST_CASE(test_topology_compare_endpoints) { locator::token_metadata::config tm_cfg; auto my_address = gms::inet_address("localhost"); @@ -1210,3 +1216,5 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack_v2); BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack_v2); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/per_partition_rate_limit_test.cc b/test/boost/per_partition_rate_limit_test.cc index 63828026740e..6371371b61b3 100644 --- a/test/boost/per_partition_rate_limit_test.cc +++ b/test/boost/per_partition_rate_limit_test.cc @@ -1,7 +1,8 @@ #include #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -9,6 +10,8 @@ #include "mutation/mutation.hh" #include "service/storage_proxy.hh" +BOOST_AUTO_TEST_SUITE(per_partition_rate_limit_test) + SEASTAR_TEST_CASE(test_internal_operation_filtering) { return do_with_cql_env_thread([] (cql_test_env& e) -> future<> { // The test requires at least two shards @@ -106,4 +109,6 @@ SEASTAR_TEST_CASE(test_internal_operation_filtering) { return make_ready_future<>(); }); -} \ No newline at end of file +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/querier_cache_test.cc b/test/boost/querier_cache_test.cc index 92a27970acd2..289b3859ff0f 100644 --- a/test/boost/querier_cache_test.cc +++ b/test/boost/querier_cache_test.cc @@ -20,12 +20,16 @@ #include #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include +#include #include #include "readers/from_mutations_v2.hh" #include "readers/empty_v2.hh" +BOOST_AUTO_TEST_SUITE(querier_cache_test) + using namespace std::chrono_literals; class dummy_result_builder { @@ -835,3 +839,5 @@ SEASTAR_THREAD_TEST_CASE(test_semaphore_mismatch) { .no_evictions(); } } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/query_processor_test.cc b/test/boost/query_processor_test.cc index c9a7b0a7b160..96b6e42247f3 100644 --- a/test/boost/query_processor_test.cc +++ b/test/boost/query_processor_test.cc @@ -14,7 +14,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" #include "test/lib/test_utils.hh" @@ -25,6 +26,8 @@ #include "cql3/query_processor.hh" #include "cql3/untyped_result_set.hh" +BOOST_AUTO_TEST_SUITE(query_processor_test) + SEASTAR_TEST_CASE(test_execute_internal_insert) { return do_with_cql_env([] (auto& e) { auto& qp = e.local_qp(); @@ -354,3 +357,5 @@ SEASTAR_TEST_CASE(test_select_full_scan_metrics) { BOOST_CHECK_EQUAL(stat_ps8, qp.get_cql_stats().select_partition_range_scan); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/reader_concurrency_semaphore_test.cc b/test/boost/reader_concurrency_semaphore_test.cc index 2681f298e7cf..613e8431b852 100644 --- a/test/boost/reader_concurrency_semaphore_test.cc +++ b/test/boost/reader_concurrency_semaphore_test.cc @@ -25,7 +25,8 @@ #include #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include #include "readers/empty_v2.hh" @@ -33,6 +34,8 @@ #include "replica/database.hh" // new_reader_base_cost is there :( #include "db/config.hh" +BOOST_AUTO_TEST_SUITE(reader_concurrency_semaphore_test) + SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_clear_inactive_reads) { simple_schema s; std::vector permits; @@ -683,6 +686,8 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_stop_waits_on_permits } +} // reader_concurrency_semaphore_test namespace + static void require_can_admit(schema_ptr schema, reader_concurrency_semaphore& semaphore, bool expected_can_admit, const char* description, seastar::compat::source_location sl = seastar::compat::source_location::current()) { testlog.trace("Running admission scenario {}, with exepcted_can_admit={}", description, expected_can_admit); @@ -711,6 +716,8 @@ static void require_can_admit(schema_ptr schema, reader_concurrency_semaphore& s } }; +namespace reader_concurrency_semaphore_test { + SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_admission) { simple_schema s; const auto schema = s.schema(); @@ -1209,6 +1216,8 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_set_resources) { permit3_fut.get(); } +} // namespace reader_concurrency_semaphore_test + namespace { class allocating_reader { @@ -1345,6 +1354,8 @@ class allocating_reader { } //anonymous namespace +namespace reader_concurrency_semaphore_test { + // Check that the memory consumption limiting mechanism doesn't leak any // resources or cause any internal consistencies in the semaphore. SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_memory_limit_no_leaks) { @@ -2139,3 +2150,5 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_live_update_cpu_concu } require_can_admit(true, "!need_cpu"); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/repair_test.cc b/test/boost/repair_test.cc index 521cd925313b..9302d7a5212d 100644 --- a/test/boost/repair_test.cc +++ b/test/boost/repair_test.cc @@ -18,11 +18,14 @@ #include "test/lib/cql_test_env.hh" #include "service/storage_proxy.hh" #include "test/lib/reader_concurrency_semaphore.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/sstable_utils.hh" #include "readers/mutation_fragment_v1_stream.hh" #include "schema/schema_registry.hh" +BOOST_AUTO_TEST_SUITE(repair_test) + // Helper mutation_fragment_queue that stores the received stream of // mutation_fragments in a passed in deque of mutation_fragment_v2. // This allows easy reader construction to verify what was sent to the queue @@ -269,3 +272,5 @@ SEASTAR_TEST_CASE(repair_rows_size_considers_external_memory) { BOOST_REQUIRE_EQUAL(row_with_boundary.size(), fmf_size + boundary.pk.external_memory_usage() + boundary.position.external_memory_usage() + sizeof(repair_row)); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/restrictions_test.cc b/test/boost/restrictions_test.cc index f075b29d43db..fa6d26f93c2d 100644 --- a/test/boost/restrictions_test.cc +++ b/test/boost/restrictions_test.cc @@ -8,7 +8,9 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include +#include #include "cql3/cql_config.hh" #include "cql3/values.hh" @@ -21,6 +23,8 @@ #include "types/map.hh" #include "types/set.hh" +BOOST_AUTO_TEST_SUITE(restrictions_test) + namespace { using seastar::compat::source_location; @@ -1046,3 +1050,5 @@ SEASTAR_THREAD_TEST_CASE(strict_is_not_null_in_views_default_value) { auto cfg = make_shared(); BOOST_REQUIRE(cfg->strict_is_not_null_in_views() == db::tri_mode_restriction_t::mode::WARN); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/role_manager_test.cc b/test/boost/role_manager_test.cc index e6052f21cd7e..b2fc911929c1 100644 --- a/test/boost/role_manager_test.cc +++ b/test/boost/role_manager_test.cc @@ -11,10 +11,13 @@ #include "auth/standard_role_manager.hh" #include "service/raft/raft_group0_client.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/test_utils.hh" #include "test/lib/cql_test_env.hh" +BOOST_AUTO_TEST_SUITE(role_manager_test) + auto make_manager(cql_test_env& env) { auto stop_role_manager = [] (auth::standard_role_manager* m) { m->stop().get(); @@ -240,3 +243,5 @@ SEASTAR_TEST_CASE(alter_role) { }); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/row_cache_test.cc b/test/boost/row_cache_test.cc index 365aacfb78ab..4a5ef390672a 100644 --- a/test/boost/row_cache_test.cc +++ b/test/boost/row_cache_test.cc @@ -13,7 +13,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/mutation_assertions.hh" #include "test/lib/mutation_reader_assertions.hh" #include "test/lib/mutation_source_test.hh" @@ -126,6 +127,8 @@ void verify_has(row_cache& cache, const mutation& m) { assert_that(std::move(reader)).next_mutation().is_equal_to(m); } +BOOST_AUTO_TEST_SUITE(row_cache_test) + SEASTAR_TEST_CASE(test_cache_delegates_to_underlying) { return seastar::async([] { auto s = make_schema(); @@ -2133,6 +2136,8 @@ SEASTAR_TEST_CASE(test_tombstone_merging_in_partial_partition) { }); } +} // row_cache_test namespace + static void consume_all(mutation_reader& rd) { while (auto mfopt = rd().get()) {} } @@ -2160,6 +2165,8 @@ static void apply(row_cache& cache, memtable_snapshot_source& underlying, replic cache.update(row_cache::external_updater([&] { underlying.apply(std::move(mt1)); }), m).get(); } +namespace row_cache_test { + SEASTAR_TEST_CASE(test_readers_get_all_data_after_eviction) { return seastar::async([] { simple_schema table; @@ -4892,3 +4899,5 @@ SEASTAR_THREAD_TEST_CASE(test_reproduce_18045) { auto close_rd = deferred_close(rd); read_mutation_from_mutation_reader(rd).get(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/schema_change_test.cc b/test/boost/schema_change_test.cc index 10a8394e3417..3b6d8ba40f5b 100644 --- a/test/boost/schema_change_test.cc +++ b/test/boost/schema_change_test.cc @@ -10,7 +10,8 @@ #include #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include @@ -32,6 +33,8 @@ #include "test/lib/log.hh" #include "cdc/cdc_extension.hh" +BOOST_AUTO_TEST_SUITE(schema_change_test) + static cql_test_config run_with_raft_recovery_config() { cql_test_config c; c.run_with_raft_recovery = true; @@ -1158,3 +1161,5 @@ SEASTAR_TEST_CASE(test_system_schema_version_is_stable) { BOOST_REQUIRE_EQUAL(s->version(), table_schema_version(utils::UUID("9621f170-f101-3459-a8d3-f342c83ad86e"))); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/schema_registry_test.cc b/test/boost/schema_registry_test.cc index ead71c7efa42..49f2726dced0 100644 --- a/test/boost/schema_registry_test.cc +++ b/test/boost/schema_registry_test.cc @@ -9,7 +9,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "data_dictionary/user_types_metadata.hh" #include "schema/schema_registry.hh" @@ -23,6 +24,8 @@ #include "test/lib/cql_test_env.hh" #include "gms/feature_service.hh" +BOOST_AUTO_TEST_SUITE(schema_registry_test) + static bytes random_column_name() { return to_bytes(to_hex(make_blob(32))); } @@ -254,3 +257,5 @@ SEASTAR_THREAD_TEST_CASE(test_table_is_attached) { BOOST_REQUIRE_THROW(learned_s1->table(), replica::no_such_column_family); }).get(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/secondary_index_test.cc b/test/boost/secondary_index_test.cc index 0ac2cd8b51cd..463b2fc15d6f 100644 --- a/test/boost/secondary_index_test.cc +++ b/test/boost/secondary_index_test.cc @@ -11,7 +11,8 @@ #include "test/lib/cql_assertions.hh" #include "test/lib/eventually.hh" #include "test/lib/exception_utils.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/select_statement_utils.hh" #include "transport/messages/result_message.hh" #include "service/pager/paging_state.hh" @@ -22,6 +23,8 @@ #include "utils/assert.hh" #include "utils/error_injection.hh" +BOOST_AUTO_TEST_SUITE(secondary_index_test) + using namespace std::chrono_literals; SEASTAR_TEST_CASE(test_secondary_index_regular_column_query) { @@ -2002,3 +2005,5 @@ SEASTAR_TEST_CASE(test_returning_failure_from_ghost_rows_deletion) { } }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/sessions_test.cc b/test/boost/sessions_test.cc index d80a4b185a4f..e6494bda6335 100644 --- a/test/boost/sessions_test.cc +++ b/test/boost/sessions_test.cc @@ -7,7 +7,8 @@ */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/random_utils.hh" #include "test/lib/cql_test_env.hh" @@ -15,6 +16,8 @@ #include "service/session.hh" +BOOST_AUTO_TEST_SUITE(sessions_test) + using namespace service; SEASTAR_TEST_CASE(test_default_session_always_exists) { @@ -92,3 +95,5 @@ SEASTAR_TEST_CASE(test_session_closing) { BOOST_REQUIRE_THROW(mgr.enter_session(id3), std::runtime_error); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/sstable_compaction_test.cc b/test/boost/sstable_compaction_test.cc index ffc0877780f2..e2d545716db7 100644 --- a/test/boost/sstable_compaction_test.cc +++ b/test/boost/sstable_compaction_test.cc @@ -20,7 +20,8 @@ #include "sstables/sstables.hh" #include "sstables/compress.hh" #include "compaction/compaction.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "schema/schema.hh" #include "schema/schema_builder.hh" @@ -57,7 +58,6 @@ #include #include #include -#include #include "test/lib/test_services.hh" #include "test/lib/cql_test_env.hh" #include "test/lib/reader_concurrency_semaphore.hh" @@ -72,6 +72,8 @@ #include "utils/assert.hh" #include "utils/pretty_printers.hh" +BOOST_AUTO_TEST_SUITE(sstable_compaction_test) + namespace fs = std::filesystem; using namespace sstables; @@ -6090,3 +6092,5 @@ SEASTAR_TEST_CASE(splitting_compaction_test) { std::runtime_error); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/sstable_directory_test.cc b/test/boost/sstable_directory_test.cc index 5c9596e08b78..84881b07c5c7 100644 --- a/test/boost/sstable_directory_test.cc +++ b/test/boost/sstable_directory_test.cc @@ -12,7 +12,9 @@ #include #include #include "sstables/generation_type.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include +#include #include "sstables/shared_sstable.hh" #include "sstables/sstable_directory.hh" #include "replica/distributed_loader.hh" @@ -163,6 +165,8 @@ static void with_sstable_directory(sharded& db, func(sstdir); } +BOOST_AUTO_TEST_SUITE(sstable_directory_test) + SEASTAR_TEST_CASE(sstable_directory_test_table_simple_empty_directory_scan) { return sstables::test_env::do_with_async([] (test_env& env) { auto& dir = env.tempdir(); @@ -838,3 +842,5 @@ SEASTAR_TEST_CASE(test_pending_log_garbage_collection) { } }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/sstable_set_test.cc b/test/boost/sstable_set_test.cc index 92cadb6cfce4..cef9b11d779c 100644 --- a/test/boost/sstable_set_test.cc +++ b/test/boost/sstable_set_test.cc @@ -7,7 +7,8 @@ */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "db/config.hh" @@ -22,6 +23,8 @@ #include "test/lib/sstable_utils.hh" #include "readers/from_mutations_v2.hh" +BOOST_AUTO_TEST_SUITE(sstable_set_test) + using namespace sstables; static sstables::sstable_set make_sstable_set(schema_ptr schema, lw_shared_ptr all = {}, bool use_level_metadata = true) { @@ -197,3 +200,5 @@ SEASTAR_TEST_CASE(test_tablet_sstable_set_copy_ctor) { }, std::move(cfg)); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/statement_restrictions_test.cc b/test/boost/statement_restrictions_test.cc index 8f6c2a2df071..4f0a5ba814de 100644 --- a/test/boost/statement_restrictions_test.cc +++ b/test/boost/statement_restrictions_test.cc @@ -7,7 +7,8 @@ */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include @@ -20,6 +21,8 @@ #include "test/lib/cql_test_env.hh" #include "test/lib/test_utils.hh" +BOOST_AUTO_TEST_SUITE(statement_restrictions_test) + using namespace cql3; namespace { @@ -515,3 +518,5 @@ BOOST_AUTO_TEST_CASE(expression_extract_column_restrictions) { assert_expr_vec_eq(restrictions::extract_single_column_restrictions_for_column(big_where_expr, col_r3), {}); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/storage_proxy_test.cc b/test/boost/storage_proxy_test.cc index eb53af5bc073..9c004c039514 100644 --- a/test/boost/storage_proxy_test.cc +++ b/test/boost/storage_proxy_test.cc @@ -9,13 +9,17 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include +#include #include "test/lib/cql_test_env.hh" #include "service/storage_proxy.hh" #include "query_ranges_to_vnodes.hh" #include "schema/schema_builder.hh" +BOOST_AUTO_TEST_SUITE(storage_proxy_test) + // Returns random keys sorted in ring order. // The schema must have a single bytes_type partition key column. static std::vector make_ring(schema_ptr s, int n_keys) { @@ -131,3 +135,5 @@ SEASTAR_THREAD_TEST_CASE(test_split_stats) { stats1->register_metrics_for("DC1", ep1); stats2->register_metrics_for("DC1", ep1); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index b352baff39db..4eda80679a65 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -9,7 +9,8 @@ #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/random_utils.hh" #include #include @@ -37,6 +38,8 @@ #include +BOOST_AUTO_TEST_SUITE(tablets_test) + using namespace locator; using namespace replica; using namespace service; @@ -3649,3 +3652,4 @@ SEASTAR_TEST_CASE(test_recognition_of_deprecated_name_for_resize_transition) { BOOST_REQUIRE_EQUAL(service::transition_state_from_string("tablet resize finalization"), transition_state::tablet_resize_finalization); return make_ready_future<>(); } +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/tracing_test.cc b/test/boost/tracing_test.cc index e27b75c558eb..b4c04528d963 100644 --- a/test/boost/tracing_test.cc +++ b/test/boost/tracing_test.cc @@ -6,13 +6,16 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "tracing/tracing.hh" #include "tracing/trace_state.hh" #include "test/lib/cql_test_env.hh" +BOOST_AUTO_TEST_SUITE(tracing_test) + future<> do_with_tracing_env(std::function(cql_test_env&)> func, cql_test_config cfg_in = {}) { return do_with_cql_env_thread([func](auto &env) { sharded& tracing = tracing::tracing::tracing_instance(); @@ -80,3 +83,5 @@ SEASTAR_TEST_CASE(tracing_slow_query_fast_mode) { return make_ready_future<>(); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/user_function_test.cc b/test/boost/user_function_test.cc index b1f7adf931b0..2abed853a8a2 100644 --- a/test/boost/user_function_test.cc +++ b/test/boost/user_function_test.cc @@ -6,7 +6,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include #include "test/lib/cql_assertions.hh" @@ -23,6 +24,8 @@ #include "test/lib/exception_utils.hh" #include "test/lib/test_utils.hh" +BOOST_AUTO_TEST_SUITE(user_function_test) + using ire = exceptions::invalid_request_exception; using exception_predicate::message_equals; using exception_predicate::message_contains; @@ -1103,3 +1106,5 @@ SEASTAR_TEST_CASE(test_user_function_filtering) { std::runtime_error, message_contains("User function cannot be executed in this context")); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/user_types_test.cc b/test/boost/user_types_test.cc index 6ebc79ad088a..3236ada1c351 100644 --- a/test/boost/user_types_test.cc +++ b/test/boost/user_types_test.cc @@ -6,7 +6,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -21,6 +22,8 @@ #include #include +BOOST_AUTO_TEST_SUITE(user_types_test) + // Specifies that the given 'cql' query fails with the 'msg' message. // Requires a cql_test_env. The caller must be inside thread. #define REQUIRE_INVALID(e, cql, msg) \ @@ -671,3 +674,5 @@ SEASTAR_TEST_CASE(test_cql3_name_without_frozen) { BOOST_REQUIRE(list_type_ptr->cql3_type_name_without_frozen() == list_type_name); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/view_build_test.cc b/test/boost/view_build_test.cc index db53e8d31bda..bf65a25193d7 100644 --- a/test/boost/view_build_test.cc +++ b/test/boost/view_build_test.cc @@ -17,7 +17,8 @@ #include "db/config.hh" #include "cql3/query_options.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include @@ -38,6 +39,8 @@ #include "readers/from_mutations_v2.hh" #include "readers/evictable.hh" +BOOST_AUTO_TEST_SUITE(view_build_test) + using namespace std::literals::chrono_literals; schema_ptr test_table_schema() { @@ -1007,3 +1010,5 @@ SEASTAR_THREAD_TEST_CASE(test_view_update_generator_buffering_with_empty_mutatio vuc.consume_end_of_stream(); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/view_complex_test.cc b/test/boost/view_complex_test.cc index d09a134c2b76..f4ca4b3d0048 100644 --- a/test/boost/view_complex_test.cc +++ b/test/boost/view_complex_test.cc @@ -14,13 +14,16 @@ #include "db/view/view_builder.hh" #include "compaction/compaction_manager.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" #include "test/lib/eventually.hh" #include "db/config.hh" +BOOST_AUTO_TEST_SUITE(view_complex_test) + using namespace std::literals::chrono_literals; // This test checks various cases where a base table row disappears - or does @@ -1748,3 +1751,5 @@ SEASTAR_TEST_CASE(test_3362_row_deletion_2) { }); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/view_schema_ckey_test.cc b/test/boost/view_schema_ckey_test.cc index 9d6c105f3476..dfe654db818d 100644 --- a/test/boost/view_schema_ckey_test.cc +++ b/test/boost/view_schema_ckey_test.cc @@ -10,12 +10,15 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" #include "test/lib/eventually.hh" +BOOST_AUTO_TEST_SUITE(view_schema_ckey_test) + using namespace std::literals::chrono_literals; SEASTAR_TEST_CASE(test_delete_single_column_in_view_clustering_key) { @@ -491,3 +494,5 @@ SEASTAR_TEST_CASE(test_no_clustering_key_2) { }); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/view_schema_pkey_test.cc b/test/boost/view_schema_pkey_test.cc index 5a9aa8f8e96a..01abd7ef8da2 100644 --- a/test/boost/view_schema_pkey_test.cc +++ b/test/boost/view_schema_pkey_test.cc @@ -13,11 +13,14 @@ #include "db/view/view_builder.hh" #include "test/lib/eventually.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" +BOOST_AUTO_TEST_SUITE(view_schema_pkey_test) + using namespace std::literals::chrono_literals; SEASTAR_TEST_CASE(test_compound_partition_key) { @@ -712,3 +715,5 @@ SEASTAR_TEST_CASE(test_base_non_pk_columns_in_view_partition_key_are_non_emtpy) } }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/view_schema_test.cc b/test/boost/view_schema_test.cc index c7ac134dad63..bb95894760ed 100644 --- a/test/boost/view_schema_test.cc +++ b/test/boost/view_schema_test.cc @@ -16,7 +16,8 @@ #include "db/view/node_view_update_backlog.hh" #include "db/view/view_builder.hh" -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -27,6 +28,8 @@ #include "types/list.hh" #include "types/map.hh" +BOOST_AUTO_TEST_SUITE(view_schema_test) + using namespace std::literals::chrono_literals; // CQL usually folds identifier names - keyspace, table and column names - @@ -3268,3 +3271,5 @@ SEASTAR_TEST_CASE(test_mv_allow_some_column_drops) { BOOST_REQUIRE_THROW(e.execute_cql("alter table cf2 drop d").get(), exceptions::invalid_request_exception); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/virtual_reader_test.cc b/test/boost/virtual_reader_test.cc index 0dfb4e03b1db..938da924da51 100644 --- a/test/boost/virtual_reader_test.cc +++ b/test/boost/virtual_reader_test.cc @@ -13,7 +13,8 @@ #include #include -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include "test/lib/cql_test_env.hh" #include "test/lib/cql_assertions.hh" @@ -23,6 +24,8 @@ #include "db/view/view_builder.hh" #include +BOOST_AUTO_TEST_SUITE(virtual_reader_test) + using namespace std::literals::chrono_literals; SEASTAR_TEST_CASE(test_query_size_estimates_virtual_table) { @@ -212,3 +215,5 @@ SEASTAR_TEST_CASE(test_query_view_built_progress_virtual_table) { assert_that(rs).is_rows().with_size(0); }); } + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/boost/virtual_table_test.cc b/test/boost/virtual_table_test.cc index 160c1a8427ca..d8a3e589e5a0 100644 --- a/test/boost/virtual_table_test.cc +++ b/test/boost/virtual_table_test.cc @@ -8,7 +8,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include "test/lib/scylla_test_case.hh" +#undef SEASTAR_TESTING_MAIN +#include #include #include "test/lib/cql_test_env.hh" #include "db/virtual_table.hh" @@ -53,6 +54,8 @@ class test_table : public virtual_table { } +BOOST_AUTO_TEST_SUITE(virtual_table_test) + SEASTAR_TEST_CASE(test_set_cell) { auto table = db::test_table(); table.test_set_cell(); @@ -100,3 +103,5 @@ SEASTAR_THREAD_TEST_CASE(test_system_config_table_no_live_update) { ); }).get(); } + +BOOST_AUTO_TEST_SUITE_END() From cd2a2bd02143598cc6e88ad2b99f6ed7301355ba Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Fri, 20 Dec 2024 14:09:10 +0800 Subject: [PATCH 062/397] repair: correct misspelling of "corespondent" replace "corespondent" with "corresponding" in a logging message. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22003 --- repair/repair.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/repair/repair.cc b/repair/repair.cc index 356582eec9d8..12d9f46b3723 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -293,7 +293,7 @@ static std::vector get_neighbors( // If same host is listed twice, don't add it again later neighbor_set.erase(*endpoint); } else { - rlogger.warn("Provided host ip {} has no corespondent host id", ip); + rlogger.warn("Provided host ip {} has no corresponding host id", ip); } } // Nodes which aren't neighbors for this range are ignored. From 68b0b442fd485db4ec65bd3f9f7b8c423207bfcf Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Tue, 17 Dec 2024 17:22:16 +0200 Subject: [PATCH 063/397] locator: refactor sort_by_proximity Extract can_sort_by_proximity() out so it can be used later by storage_proxy, and introduce do_sort_by_proximity that sorts unconditionally. Signed-off-by: Benny Halevy --- locator/topology.cc | 12 ++++++++---- locator/topology.hh | 14 ++++++++++++-- test/boost/network_topology_strategy_test.cc | 2 +- 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/locator/topology.cc b/locator/topology.cc index 7f6a1c0ecaa3..a46b843f31b8 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -567,13 +567,17 @@ const endpoint_dc_rack& topology::get_location(const inet_address& ep) const { } void topology::sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const { - if (_sort_by_proximity) { - std::sort(addresses.begin(), addresses.end(), [this, &address](locator::host_id& a1, locator::host_id& a2) { - return compare_endpoints(address, a1, a2) < 0; - }); + if (can_sort_by_proximity()) { + do_sort_by_proximity(address, addresses); } } +void topology::do_sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const { + std::sort(addresses.begin(), addresses.end(), [this, &address](const locator::host_id& a1, const locator::host_id& a2) { + return compare_endpoints(address, a1, a2) < 0; + }); +} + std::weak_ordering topology::compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const { const auto& loc = get_location(address); const auto& loc1 = get_location(a1); diff --git a/locator/topology.hh b/locator/topology.hh index 6864f6de94d4..1521aae481c2 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -353,12 +353,22 @@ public: return std::count_if(endpoints.begin(), endpoints.end(), get_local_dc_filter()); } + bool can_sort_by_proximity() const noexcept { + return _sort_by_proximity; + } + /** - * This method will sort the List by proximity to the given - * host_id. + * This method will sort the addresses list by proximity to the given host_id, + * if `can_sort_by_proximity()`. */ void sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const; + /** + * Unconditionally sort the addresses list by proximity to the given host_id, + * assuming `can_sort_by_proximity`. + */ + void do_sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const; + // Executes a function for each node in a state other than "none" and "left". void for_each_node(std::function func) const; diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index 304ed980d2f0..8d3df681d183 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -989,7 +989,7 @@ void topology::test_compare_endpoints(const locator::host_id& address, const loc void topology::test_sort_by_proximity(const locator::host_id& address, const host_id_vector_replica_set& nodes) const { auto sorted_nodes = nodes; - sort_by_proximity(address, sorted_nodes); + do_sort_by_proximity(address, sorted_nodes); std::unordered_set nodes_set(nodes.begin(), nodes.end()); std::unordered_set sorted_nodes_set(sorted_nodes.begin(), sorted_nodes.end()); // Test that no nodes were lost by sort_by_proximity From 7c1eaa427ef20024eedf384af628c7a0fa199c02 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Wed, 16 Oct 2024 14:55:45 +0300 Subject: [PATCH 064/397] sstables_loader: Wrap get_endpoints() Preparational patch. Next will add more code to get_endpoints() that will need to work for both if/else branches, this change helps having less churn later. Signed-off-by: Pavel Emelyanov --- sstables_loader.cc | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sstables_loader.cc b/sstables_loader.cc index 015fc0ea92a9..97a34cf79c01 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -162,6 +162,8 @@ class sstable_streamer { protected: virtual host_id_vector_replica_set get_primary_endpoints(const dht::token& token) const; future<> stream_sstables(const dht::partition_range&, std::vector, std::function on_streamed); +private: + host_id_vector_replica_set get_all_endpoints(const dht::token& token) const; }; class tablet_sstable_streamer : public sstable_streamer { @@ -192,6 +194,10 @@ class tablet_sstable_streamer : public sstable_streamer { }; host_id_vector_replica_set sstable_streamer::get_endpoints(const dht::token& token) const { + return get_all_endpoints(token); +} + +host_id_vector_replica_set sstable_streamer::get_all_endpoints(const dht::token& token) const { if (_primary_replica_only) { return get_primary_endpoints(token); } From 30aac0d1da71a7b0b64e676784754e4f026bdb3e Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Wed, 16 Oct 2024 15:03:46 +0300 Subject: [PATCH 065/397] sstables_loader: Introduce streaming scope Currently load-and-stream sends mutations to whatever node is considered to be a "replica" for it. One exception is the "primary-replica-only" flag that can be requested by the user. This patch introduces a "scope" parameter that limits streaming part in where it can stream the data to with 4 options: - all -- current way of doing things, stream to wherever needed - dc -- only stream to nodes that live in the same datacenter - rack -- only stream to nodes that live in the same rack - node -- only "stream" to current node It's not yet configurable and streamer object initializes itself with "all" mode. Will be changed later. Signed-off-by: Pavel Emelyanov --- sstables_loader.cc | 16 +++++++++++++++- sstables_loader.hh | 1 + 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sstables_loader.cc b/sstables_loader.cc index 97a34cf79c01..65f059c1269d 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -126,6 +126,7 @@ using unlink_sstables = bool_class; class sstable_streamer { protected: + using stream_scope = sstables_loader::stream_scope; netw::messaging_service& _ms; replica::database& _db; replica::table& _table; @@ -133,6 +134,7 @@ class sstable_streamer { std::vector _sstables; const primary_replica_only _primary_replica_only; const unlink_sstables _unlink_sstables; + const stream_scope _stream_scope; public: sstable_streamer(netw::messaging_service& ms, replica::database& db, ::table_id table_id, std::vector sstables, primary_replica_only primary, unlink_sstables unlink) : _ms(ms) @@ -142,6 +144,7 @@ class sstable_streamer { , _sstables(std::move(sstables)) , _primary_replica_only(primary) , _unlink_sstables(unlink) + , _stream_scope(stream_scope::all) { // By sorting SSTables by their primary key, we allow SSTable runs to be // incrementally streamed. @@ -194,7 +197,18 @@ class tablet_sstable_streamer : public sstable_streamer { }; host_id_vector_replica_set sstable_streamer::get_endpoints(const dht::token& token) const { - return get_all_endpoints(token); + return get_all_endpoints(token) | std::views::filter([&topo = _erm->get_topology(), scope = _stream_scope] (const auto& ep) { + switch (scope) { + case stream_scope::all: + return true; + case stream_scope::dc: + return topo.get_datacenter(ep) == topo.get_datacenter(); + case stream_scope::rack: + return topo.get_location(ep) == topo.get_location(); + case stream_scope::node: + return topo.is_me(ep); + } + }) | std::ranges::to(); } host_id_vector_replica_set sstable_streamer::get_all_endpoints(const dht::token& token) const { diff --git a/sstables_loader.hh b/sstables_loader.hh index 8dfc63d27f4e..da1fdc89d07f 100644 --- a/sstables_loader.hh +++ b/sstables_loader.hh @@ -35,6 +35,7 @@ class view_builder; // system. Built on top of the distributed_loader functionality. class sstables_loader : public seastar::peering_sharded_service { public: + enum class stream_scope { all, dc, rack, node }; class task_manager_module : public tasks::task_manager::module { public: task_manager_module(tasks::task_manager& tm) noexcept : tasks::task_manager::module(tm, "sstables_loader") {} From 93aed22cd54d75510091fa81fa28acdc73c6c220 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Mon, 9 Dec 2024 17:46:23 +0300 Subject: [PATCH 066/397] streamer: Disable scoped streaming of primary replica only There's been some discussions of how primary replica only streaming schould interact with the scope. There are two options how to consider this combination: - find where the primary replica is and handle it if it's within the requested sope - within the requested scope find the primary replica for that subset of nodes, then handle it There's also some itermediate solution: suppoer "primary replica in DC" and reject all other combinations. Until decided which way is correct, let's disable this configuration. Signed-off-by: Pavel Emelyanov --- sstables_loader.cc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sstables_loader.cc b/sstables_loader.cc index 65f059c1269d..ccc641d7e3b3 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -146,6 +146,9 @@ class sstable_streamer { , _unlink_sstables(unlink) , _stream_scope(stream_scope::all) { + if (_primary_replica_only && _stream_scope != stream_scope::all) { + throw std::runtime_error("Scoped streaming of primary replica only is not supported yet"); + } // By sorting SSTables by their primary key, we allow SSTable runs to be // incrementally streamed. // Overlapping run fragments can have their content deduplicated, reducing From e8201a789726c974309d35a1d3c539f590d48e52 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 3 Dec 2024 15:00:25 +0300 Subject: [PATCH 067/397] sstables_loader: Filter tablets based on scope Loading and streaming tablets has pre-filtering loop that walks the tablet map sorts sstables into three lists: - fully contained in one of map ranges - partially overlapping with the map - not intersecting with the map Sstables from the 3rd list is immediately dropped from the process and for the remaining two core load-and-stream happens. This filtering deserves more care from the newly introduced scope. When a tablet replica set doesn't get in the scope, the whole entry can be disregarded, because load-and-stream will only do its "load" part anyway and all mutations from it will be ignored. Signed-off-by: Pavel Emelyanov --- sstables_loader.cc | 34 +++++++++++++++++++++++++++++++++- 1 file changed, 33 insertions(+), 1 deletion(-) diff --git a/sstables_loader.cc b/sstables_loader.cc index ccc641d7e3b3..3f71a8580507 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -197,6 +197,8 @@ class tablet_sstable_streamer : public sstable_streamer { // FIXME: fully contained sstables can be optimized. return stream_sstables(pr, std::move(sstables), std::move(on_streamed)); } + + bool tablet_in_scope(locator::tablet_id) const; }; host_id_vector_replica_set sstable_streamer::get_endpoints(const dht::token& token) const { @@ -242,11 +244,41 @@ future<> sstable_streamer::stream(std::function on_streamed) { co_await stream_sstables(full_partition_range, std::move(_sstables), std::move(on_streamed)); } +bool tablet_sstable_streamer::tablet_in_scope(locator::tablet_id tid) const { + if (_stream_scope == stream_scope::all) { + return true; + } + + const auto& topo = _erm->get_topology(); + for (const auto& r : _tablet_map.get_tablet_info(tid).replicas) { + switch (_stream_scope) { + case stream_scope::node: + if (topo.is_me(r.host)) { + return true; + } + break; + case stream_scope::rack: + if (topo.get_location(r.host) == topo.get_location()) { + return true; + } + break; + case stream_scope::dc: + if (topo.get_datacenter(r.host) == topo.get_datacenter()) { + return true; + } + break; + case stream_scope::all: // checked above already, but still need it here + return true; + } + } + return false; +} + future<> tablet_sstable_streamer::stream(std::function on_streamed) { // sstables are sorted by first key in reverse order. auto sstable_it = _sstables.rbegin(); - for (auto tablet_id : _tablet_map.tablet_ids()) { + for (auto tablet_id : _tablet_map.tablet_ids() | std::views::filter([this] (auto tid) { return tablet_in_scope(tid); })) { auto tablet_range = _tablet_map.get_token_range(tablet_id); auto sstable_token_range = [] (const sstables::shared_sstable& sst) { From 960041d4b46d85483a7ce63ddd51af5d5f86ac90 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Wed, 16 Oct 2024 15:26:30 +0300 Subject: [PATCH 068/397] sstables_loader: Propagate scope from API down Semi-mechanical change that adds newly introduced "scope" parameter to all the functions between API methods and the low-level streamer object. No real functional changes. API methods set it to "all" to keep existing behavior. Signed-off-by: Pavel Emelyanov --- api/storage_service.cc | 4 ++-- sstables_loader.cc | 28 +++++++++++++++------------- sstables_loader.hh | 6 +++--- 3 files changed, 20 insertions(+), 18 deletions(-) diff --git a/api/storage_service.cc b/api/storage_service.cc index 2496b2d8c491..8bc1cb21c9dd 100644 --- a/api/storage_service.cc +++ b/api/storage_service.cc @@ -479,7 +479,7 @@ void set_sstables_loader(http_context& ctx, routes& r, sharded& return sst_loader.invoke_on(coordinator, [ks = std::move(ks), cf = std::move(cf), load_and_stream, primary_replica_only] (sstables_loader& loader) { - return loader.load_new_sstables(ks, cf, load_and_stream, primary_replica_only); + return loader.load_new_sstables(ks, cf, load_and_stream, primary_replica_only, sstables_loader::stream_scope::all); }).then_wrapped([] (auto&& f) { if (f.failed()) { auto msg = fmt::format("Failed to load new sstables: {}", f.get_exception()); @@ -505,7 +505,7 @@ void set_sstables_loader(http_context& ctx, routes& r, sharded& auto sstables = parsed.GetArray() | std::views::transform([] (const auto& s) { return sstring(rjson::to_string_view(s)); }) | std::ranges::to(); - auto task_id = co_await sst_loader.local().download_new_sstables(keyspace, table, prefix, std::move(sstables), endpoint, bucket); + auto task_id = co_await sst_loader.local().download_new_sstables(keyspace, table, prefix, std::move(sstables), endpoint, bucket, sstables_loader::stream_scope::all); co_return json::json_return_type(fmt::to_string(task_id)); }); diff --git a/sstables_loader.cc b/sstables_loader.cc index 3f71a8580507..23761ca3c9b5 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -136,7 +136,7 @@ class sstable_streamer { const unlink_sstables _unlink_sstables; const stream_scope _stream_scope; public: - sstable_streamer(netw::messaging_service& ms, replica::database& db, ::table_id table_id, std::vector sstables, primary_replica_only primary, unlink_sstables unlink) + sstable_streamer(netw::messaging_service& ms, replica::database& db, ::table_id table_id, std::vector sstables, primary_replica_only primary, unlink_sstables unlink, stream_scope scope) : _ms(ms) , _db(db) , _table(db.find_column_family(table_id)) @@ -144,7 +144,7 @@ class sstable_streamer { , _sstables(std::move(sstables)) , _primary_replica_only(primary) , _unlink_sstables(unlink) - , _stream_scope(stream_scope::all) + , _stream_scope(scope) { if (_primary_replica_only && _stream_scope != stream_scope::all) { throw std::runtime_error("Scoped streaming of primary replica only is not supported yet"); @@ -175,8 +175,8 @@ class sstable_streamer { class tablet_sstable_streamer : public sstable_streamer { const locator::tablet_map& _tablet_map; public: - tablet_sstable_streamer(netw::messaging_service& ms, replica::database& db, ::table_id table_id, std::vector sstables, primary_replica_only primary, unlink_sstables unlink) - : sstable_streamer(ms, db, table_id, std::move(sstables), primary, unlink) + tablet_sstable_streamer(netw::messaging_service& ms, replica::database& db, ::table_id table_id, std::vector sstables, primary_replica_only primary, unlink_sstables unlink, stream_scope scope) + : sstable_streamer(ms, db, table_id, std::move(sstables), primary, unlink, scope) , _tablet_map(_erm->get_token_metadata().tablets().get_tablet_map(table_id)) { } @@ -449,13 +449,13 @@ static std::unique_ptr make_sstable_streamer(bool uses_tablets } future<> sstables_loader::load_and_stream(sstring ks_name, sstring cf_name, - ::table_id table_id, std::vector sstables, bool primary, bool unlink, + ::table_id table_id, std::vector sstables, bool primary, bool unlink, stream_scope scope, std::function on_streamed) { // streamer guarantees topology stability, for correctness, by holding effective_replication_map // throughout its lifetime. auto streamer = make_sstable_streamer(_db.local().find_column_family(table_id).uses_tablets(), _messaging, _db.local(), table_id, std::move(sstables), - primary_replica_only(primary), unlink_sstables(unlink)); + primary_replica_only(primary), unlink_sstables(unlink), scope); co_await streamer->stream(on_streamed); } @@ -464,7 +464,7 @@ future<> sstables_loader::load_and_stream(sstring ks_name, sstring cf_name, // All the global operations are going to happen here, and just the reloading happens // in there. future<> sstables_loader::load_new_sstables(sstring ks_name, sstring cf_name, - bool load_and_stream, bool primary_replica_only) { + bool load_and_stream, bool primary_replica_only, stream_scope scope) { if (_loading_new_sstables) { throw std::runtime_error("Already loading SSTables. Try again later"); } else { @@ -492,8 +492,8 @@ future<> sstables_loader::load_new_sstables(sstring ks_name, sstring cf_name, .load_bloom_filter = false, }; std::tie(table_id, sstables_on_shards) = co_await replica::distributed_loader::get_sstables_from_upload_dir(_db, ks_name, cf_name, cfg); - co_await container().invoke_on_all([&sstables_on_shards, ks_name, cf_name, table_id, primary_replica_only] (sstables_loader& loader) mutable -> future<> { - co_await loader.load_and_stream(ks_name, cf_name, table_id, std::move(sstables_on_shards[this_shard_id()]), primary_replica_only, true, {}); + co_await container().invoke_on_all([&sstables_on_shards, ks_name, cf_name, table_id, primary_replica_only, scope] (sstables_loader& loader) mutable -> future<> { + co_await loader.load_and_stream(ks_name, cf_name, table_id, std::move(sstables_on_shards[this_shard_id()]), primary_replica_only, true, scope, {}); }); } else { co_await replica::distributed_loader::process_upload_dir(_db, _view_builder, ks_name, cf_name); @@ -517,6 +517,7 @@ class sstables_loader::download_task_impl : public tasks::task_manager::task::im sstring _ks; sstring _cf; sstring _prefix; + sstables_loader::stream_scope _scope; std::vector _sstables; std::vector _num_sstables_processed; @@ -526,7 +527,7 @@ class sstables_loader::download_task_impl : public tasks::task_manager::task::im public: download_task_impl(tasks::task_manager::module_ptr module, sharded& loader, sstring endpoint, sstring bucket, - sstring ks, sstring cf, sstring prefix, std::vector sstables) noexcept + sstring ks, sstring cf, sstring prefix, std::vector sstables, sstables_loader::stream_scope scope) noexcept : tasks::task_manager::task::impl(module, tasks::task_id::create_random_id(), 0, "node", ks, "", "", tasks::task_id::create_null_id()) , _loader(loader) , _endpoint(std::move(endpoint)) @@ -534,6 +535,7 @@ class sstables_loader::download_task_impl : public tasks::task_manager::task::im , _ks(std::move(ks)) , _cf(std::move(cf)) , _prefix(std::move(prefix)) + , _scope(scope) , _sstables(std::move(sstables)) , _num_sstables_processed(smp::count) { @@ -597,7 +599,7 @@ future<> sstables_loader::download_task_impl::run() { }); co_await _loader.invoke_on_all([this, &sstables_on_shards, table_id] (sstables_loader& loader) mutable -> future<> { - co_await loader.load_and_stream(_ks, _cf, table_id, std::move(sstables_on_shards[this_shard_id()]), false, false, [this] (unsigned num_streamed) { + co_await loader.load_and_stream(_ks, _cf, table_id, std::move(sstables_on_shards[this_shard_id()]), false, false, _scope, [this] (unsigned num_streamed) { _num_sstables_processed[this_shard_id()] += num_streamed; }); }); @@ -643,12 +645,12 @@ future<> sstables_loader::stop() { future sstables_loader::download_new_sstables(sstring ks_name, sstring cf_name, sstring prefix, std::vector sstables, - sstring endpoint, sstring bucket) { + sstring endpoint, sstring bucket, stream_scope scope) { if (!_storage_manager.is_known_endpoint(endpoint)) { throw std::invalid_argument(format("endpoint {} not found", endpoint)); } llog.info("Restore sstables from {}({}) to {}", endpoint, prefix, ks_name); - auto task = co_await _task_manager_module->make_and_start_task({}, container(), std::move(endpoint), std::move(bucket), std::move(ks_name), std::move(cf_name), std::move(prefix), std::move(sstables)); + auto task = co_await _task_manager_module->make_and_start_task({}, container(), std::move(endpoint), std::move(bucket), std::move(ks_name), std::move(cf_name), std::move(prefix), std::move(sstables), scope); co_return task->id(); } diff --git a/sstables_loader.hh b/sstables_loader.hh index da1fdc89d07f..b11fde2f1ad3 100644 --- a/sstables_loader.hh +++ b/sstables_loader.hh @@ -59,7 +59,7 @@ private: future<> load_and_stream(sstring ks_name, sstring cf_name, table_id, std::vector sstables, - bool primary_replica_only, bool unlink_sstables, + bool primary_replica_only, bool unlink_sstables, stream_scope scope, std::function on_streamed); public: @@ -85,14 +85,14 @@ public: * @return a future<> when the operation finishes. */ future<> load_new_sstables(sstring ks_name, sstring cf_name, - bool load_and_stream, bool primary_replica_only); + bool load_and_stream, bool primary_replica_only, stream_scope scope); /** * Download new SSTables not currently tracked by the system from object store */ future download_new_sstables(sstring ks_name, sstring cf_name, sstring prefix, std::vector sstables, - sstring endpoint, sstring bucket); + sstring endpoint, sstring bucket, stream_scope scope); class download_task_impl; }; From a24dc022555f057271d0a088733036b91c4db7c6 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Wed, 16 Oct 2024 15:32:41 +0300 Subject: [PATCH 069/397] api: New "scope" API param to load-and-stream calls There are two of those -- the POST /storage_service/keyspace that loads and streams new sstables from /upload and POST /storage_service/restore that does the same, but gets sstables from object store. The new optional parameter allow users to tun the streaming phase behavior. The test/pylib client part is also updated here. Signed-off-by: Pavel Emelyanov --- api/api-doc/storage_service.json | 9 +++++++++ api/storage_service.cc | 19 ++++++++++++++++++- .../nodetool-commands/restore.rst | 15 +++++++++++++++ test/pylib/rest_client.py | 4 +++- tools/scylla-nodetool.cc | 4 ++++ 5 files changed, 49 insertions(+), 2 deletions(-) diff --git a/api/api-doc/storage_service.json b/api/api-doc/storage_service.json index 187c0670cf15..e43bb3caf243 100644 --- a/api/api-doc/storage_service.json +++ b/api/api-doc/storage_service.json @@ -881,6 +881,15 @@ "allowMultiple":false, "type":"string", "paramType":"query" + }, + { + "name":"scope", + "description":"Defines the set of nodes to which mutations can be streamed", + "required":false, + "allowMultiple":false, + "type":"string", + "paramType":"query", + "enum": ["all", "dc", "rack", "node"] } ] } diff --git a/api/storage_service.cc b/api/storage_service.cc index 8bc1cb21c9dd..0c245dbdce99 100644 --- a/api/storage_service.cc +++ b/api/storage_service.cc @@ -463,6 +463,22 @@ void unset_repair(http_context& ctx, routes& r) { ss::force_terminate_all_repair_sessions_new.unset(r); } +static sstables_loader::stream_scope parse_stream_scope(const sstring& scope_str) { + using namespace ss::ns_start_restore; + auto sc = scope_str.empty() ? scope::all : str2scope(scope_str); + + switch (sc) { + case scope::all: return sstables_loader::stream_scope::all; + case scope::dc: return sstables_loader::stream_scope::dc; + case scope::rack: return sstables_loader::stream_scope::rack; + case scope::node: return sstables_loader::stream_scope::node; + case scope::NUM_ITEMS: + break; + } + + throw httpd::bad_param_exception("invalid scope parameter value"); +} + void set_sstables_loader(http_context& ctx, routes& r, sharded& sst_loader) { ss::load_new_ss_tables.set(r, [&ctx, &sst_loader](std::unique_ptr req) { auto ks = validate_keyspace(ctx, req); @@ -495,6 +511,7 @@ void set_sstables_loader(http_context& ctx, routes& r, sharded& auto table = req->get_query_param("table"); auto bucket = req->get_query_param("bucket"); auto prefix = req->get_query_param("prefix"); + auto scope = parse_stream_scope(req->get_query_param("scope")); // TODO: the http_server backing the API does not use content streaming // should use it for better performance @@ -505,7 +522,7 @@ void set_sstables_loader(http_context& ctx, routes& r, sharded& auto sstables = parsed.GetArray() | std::views::transform([] (const auto& s) { return sstring(rjson::to_string_view(s)); }) | std::ranges::to(); - auto task_id = co_await sst_loader.local().download_new_sstables(keyspace, table, prefix, std::move(sstables), endpoint, bucket, sstables_loader::stream_scope::all); + auto task_id = co_await sst_loader.local().download_new_sstables(keyspace, table, prefix, std::move(sstables), endpoint, bucket, scope); co_return json::json_return_type(fmt::to_string(task_id)); }); diff --git a/docs/operating-scylla/nodetool-commands/restore.rst b/docs/operating-scylla/nodetool-commands/restore.rst index b6a4f15f04ea..f691492d5a11 100644 --- a/docs/operating-scylla/nodetool-commands/restore.rst +++ b/docs/operating-scylla/nodetool-commands/restore.rst @@ -19,6 +19,7 @@ Syntax --keyspace --table [--nowait] + [--scope ] ... Example @@ -43,8 +44,22 @@ Options * ``--keyspace`` - Name of the keyspace to load SSTables into * ``--table`` - Name of the table to load SSTables into * ``--nowait`` - Don't wait on the restore process +* ``--scope `` - Use specified load-and-stream scope * ```` - Remainder of keys of the TOC (Table of Contents) components of SSTables to restore, relative to the specified prefix +The `scope` parameter describes the subset of cluster nodes where you want to load data: + +* `node` - On the local node. +* `rack` - On the local rack. +* `dc` - In the datacenter (DC) where the local node lives. +* `all` (default) - Everywhere across the cluster. + +To fully restore a cluster, you should combine the ``scope`` parameter with the correct list of +SStables to restore to each node. +On one extreme, one node is given all SStables with the scope ``all``; on the other extreme, all +nodes are restoring only their own SStables with the scope ``node``. In between, you can choose +a subset of nodes to restore only SStables that belong to the rack or DC. + See also :doc:`Nodetool backup ` diff --git a/test/pylib/rest_client.py b/test/pylib/rest_client.py index 4835cdc837f7..0298023b6f4e 100644 --- a/test/pylib/rest_client.py +++ b/test/pylib/rest_client.py @@ -328,13 +328,15 @@ async def backup(self, node_ip: str, ks: str, table: str, tag: str, dest: str, b "snapshot": tag} return await self.client.post_json(f"/storage_service/backup", host=node_ip, params=params) - async def restore(self, node_ip: str, ks: str, cf: str, dest: str, bucket: str, prefix: str, sstables: list[str]) -> str: + async def restore(self, node_ip: str, ks: str, cf: str, dest: str, bucket: str, prefix: str, sstables: list[str], scope: str = None) -> str: """Restore keyspace:table from backup""" params = {"keyspace": ks, "table": cf, "endpoint": dest, "bucket": bucket, "prefix": prefix} + if scope is not None: + params['scope'] = scope return await self.client.post_json(f"/storage_service/restore", host=node_ip, params=params, json=sstables) async def take_snapshot(self, node_ip: str, ks: str, tag: str) -> None: diff --git a/tools/scylla-nodetool.cc b/tools/scylla-nodetool.cc index bf1cf0a68294..6b7133ba6a28 100644 --- a/tools/scylla-nodetool.cc +++ b/tools/scylla-nodetool.cc @@ -1536,6 +1536,9 @@ void restore_operation(scylla_rest_client& client, const bpo::variables_map& vm) if (!vm.contains("sstables")) { throw std::invalid_argument("missing required possitional argument: sstables"); } + if (vm.contains("scope")) { + params["scope"] = vm["scope"].as(); + } sstring sstables_body = std::invoke([&vm] { std::stringstream output; rjson::streaming_writer writer(output); @@ -3943,6 +3946,7 @@ For more information, see: {}" typed_option("keyspace", "Name of a keyspace to copy SSTables to"), typed_option("table", "Name of a table to copy SSTables to"), typed_option<>("nowait", "Don't wait on the restore process"), + typed_option("scope", "Load-and-stream scope (node, rack or dc)"), }, { typed_option>("sstables", "The object keys of the TOC component of the SSTables to be restored", -1), From 972ff80fada849005c83180dacc71414f9810fbe Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Fri, 18 Oct 2024 11:02:44 +0300 Subject: [PATCH 070/397] test: Add scope-streaming test (for restore from backup) - create - a cluster with given topology - keyspace with tablets and given rf value - table with some data - backup - flush all nodes - kick backup API on every node - re-create keyspace and table - drop it first - create again with the same parameters and schema, but don't populate table with data - restore - collect nodes to contact and corresponding list of TOCs according to the preferred "scope" - ask selected nodes to restore, limiting its streaming scope and providing the specific list of sstables - check - select mutation fragments from all nodes for random keys - make sure that the number of non-empty responses equals the expected rf value Specific topologies, RFs and stream scopes used are: rf = 1, nodes = 3, racks = 1, dcs = 1, scope = node rf = 3, nodes = 5, racks = 1, dcs = 1, scope = node rf = 1, nodes = 4, racks = 2, dcs = 1, scope = rack rf = 3, nodes = 6, racks = 2, dcs = 1, scope = rack rf = 3, nodes = 6, racks = 3, dcs = 1, scope = rack rf = 2, nodes = 8, racks = 4, dcs = 2, scope = dc nodes and racks are evenly distributed in racks and dcs respectively in the last topo RF effectively becomes 4 (2 in each dc) Signed-off-by: Pavel Emelyanov --- test/object_store/test_backup.py | 127 +++++++++++++++++++++++++++++++ 1 file changed, 127 insertions(+) diff --git a/test/object_store/test_backup.py b/test/object_store/test_backup.py index 5e222a3cb929..6dcd91a6113b 100644 --- a/test/object_store/test_backup.py +++ b/test/object_store/test_backup.py @@ -2,13 +2,20 @@ import os import logging +import asyncio import pytest +import time +import random from test.pylib.manager_client import ManagerClient from test.object_store.conftest import format_tuples from test.object_store.conftest import get_s3_resource from test.topology.conftest import skip_mode +from test.topology.util import wait_for_cql_and_get_hosts +from test.pylib.rest_client import read_barrier from test.pylib.util import unique_name +from cassandra.cluster import ConsistencyLevel, Session +from cassandra.query import SimpleStatement # type: ignore # pylint: disable=no-name-in-module logger = logging.getLogger(__name__) @@ -317,3 +324,123 @@ async def test_simple_backup_and_restore(manager: ManagerClient, s3_server): async def test_abort_simple_backup_and_restore(manager: ManagerClient, s3_server): '''check that restoring from backed up snapshot for a keyspace:table works''' await do_test_simple_backup_and_restore(manager, s3_server, True) + + +# Helper class to parametrize the test below +class topo: + def __init__(self, rf, nodes, racks, dcs): + self.rf = rf + self.nodes = nodes + self.racks = racks + self.dcs = dcs + +@pytest.mark.asyncio +@pytest.mark.parametrize("topology", [ + topo(rf = 1, nodes = 3, racks = 1, dcs = 1), + topo(rf = 3, nodes = 5, racks = 1, dcs = 1), + topo(rf = 1, nodes = 4, racks = 2, dcs = 1), + topo(rf = 3, nodes = 6, racks = 2, dcs = 1), + topo(rf = 3, nodes = 6, racks = 3, dcs = 1), + topo(rf = 2, nodes = 8, racks = 4, dcs = 2) + ]) +async def test_restore_with_streaming_scopes(manager: ManagerClient, s3_server, topology): + '''Check that restoring of a cluster with stream scopes works''' + + logger.info(f'Start cluster with {topology.nodes} nodes in {topology.dcs} DCs, {topology.racks} racks') + cfg = { 'object_storage_config_file': str(s3_server.config_file), 'task_ttl_in_seconds': 300 } + cmd = [ '--logger-log-level', 'sstables_loader=debug:sstable_directory=trace:snapshots=trace:s3=trace:sstable=debug:http=debug' ] + servers = [] + host_ids = {} + for s in range(topology.nodes): + dc = f'dc{s % topology.dcs}' + rack = f'rack{s % topology.racks}' + s = await manager.server_add(config=cfg, cmdline=cmd, property_file={'dc': dc, 'rack': rack}) + logger.info(f'Created node {s.ip_addr} in {dc}.{rack}') + servers.append(s) + host_ids[s.server_id] = await manager.get_host_id(s.server_id) + + await manager.api.disable_tablet_balancing(servers[0].ip_addr) + cql = manager.get_cql() + + logger.info(f'Create keyspace, rf={topology.rf}') + keys = range(256) + ks = 'ks' + cf = 'cf' + replication_opts = format_tuples({'class': 'NetworkTopologyStrategy', 'replication_factor': f'{topology.rf}'}) + cql.execute((f"CREATE KEYSPACE {ks} WITH REPLICATION = {replication_opts};")) + + schema = f"CREATE TABLE {ks}.{cf} ( pk int primary key, value text );" + cql.execute(schema) + for k in keys: + cql.execute(f"INSERT INTO {ks}.{cf} ( pk, value ) VALUES ({k}, '{k}');") + + logger.info(f'Collect sstables lists') + sstables = [] + for s in servers: + await manager.api.flush_keyspace(s.ip_addr, ks) + workdir = await manager.server_get_workdir(s.server_id) + cf_dir = os.listdir(f'{workdir}/data/{ks}')[0] + tocs = [ f.name for f in os.scandir(f'{workdir}/data/{ks}/{cf_dir}') if f.is_file() and f.name.endswith('TOC.txt') ] + logger.info(f'Collected sstables from {s.ip_addr}:{cf_dir}: {tocs}') + sstables += tocs + + snap_name = unique_name('backup_') + logger.info(f'Backup to {snap_name}') + prefix = f'{cf}/{snap_name}' + async def do_backup(s): + await manager.api.take_snapshot(s.ip_addr, ks, snap_name) + tid = await manager.api.backup(s.ip_addr, ks, cf, snap_name, s3_server.address, s3_server.bucket_name, prefix) + await manager.api.wait_task(s.ip_addr, tid) + + await asyncio.gather(*(do_backup(s) for s in servers)) + + logger.info(f'Re-initialize keyspace') + cql.execute(f'DROP KEYSPACE {ks}') + cql.execute((f"CREATE KEYSPACE {ks} WITH REPLICATION = {replication_opts};")) + cql.execute(schema) + + logger.info(f'Restore') + async def do_restore(s, toc_names, scope): + logger.info(f'Restore {s.ip_addr} with {toc_names}, scope={scope}') + tid = await manager.api.restore(s.ip_addr, ks, cf, s3_server.address, s3_server.bucket_name, prefix, toc_names, scope) + await manager.api.wait_task(s.ip_addr, tid) + + if topology.dcs > 1: + scope = 'dc' + r_servers = servers[:topology.dcs] + elif topology.racks > 1: + scope = 'rack' + r_servers = servers[:topology.racks] + else: + scope = 'node' + r_servers = servers + + await asyncio.gather(*(do_restore(s, sstables, scope) for s in r_servers)) + + logger.info(f'Check the data is back') + async def check_mutations(server, key): + host = await wait_for_cql_and_get_hosts(cql, [server], time.time() + 30) + await read_barrier(manager.api, server.ip_addr) # scylladb/scylladb#18199 + res = await cql.run_async(f"SELECT partition_region FROM MUTATION_FRAGMENTS({ks}.{cf}) WHERE pk={key}", host=host[0]) + for fragment in res: + if fragment.partition_region == 0: # partition start + return True + return False + + for k in random.sample(keys, 17): + res = await asyncio.gather(*(check_mutations(s, k) for s in servers)) + assert res.count(True) == topology.rf * topology.dcs + + logger.info(f'Validate streaming directions') + for i, s in enumerate(r_servers): + log = await manager.server_open_log(s.server_id) + res = await log.grep(r'INFO.*sstables_loader - load_and_stream:.*target_node=([0-9a-z-]+),.*num_bytes_sent=([0-9]+)') + streamed_to = set([ str(host_ids[s.server_id]) ] + [ r[1].group(1) for r in res ]) + scope_nodes = set([ str(host_ids[s.server_id]) ]) + # See comment near merge_tocs() above for explanation of servers list filtering below + if scope == 'rack': + scope_nodes.update([ str(host_ids[s.server_id]) for s in servers[i::topology.racks] ]) + elif scope == 'dc': + scope_nodes.update([ str(host_ids[s.server_id]) for s in servers[i::topology.dcs] ]) + logger.info(f'{s.ip_addr} streamed to {streamed_to}, expected {scope_nodes}') + assert streamed_to == scope_nodes From 866326efe46f7c4089d3420d69243334a82ebeca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 4 Dec 2024 17:28:38 +0100 Subject: [PATCH 071/397] utils: add stream_compressor Adds utilities for "advanced" methods of compression with lz4 and zstd -- with streaming (a history buffer persisted across messages) and/or precomputed dictionaries. This patch is mostly just glue needed to use the underlying libraries with discontiguous input and output buffers, and for reusing the same compressor context objects across messages. It doesn't contain any innovations of its own. There is one "design decision" in the patch. The block format of LZ4 doesn't contain the length of the compressed blocks. At decompression time, that length must be delivered to the decompressor by a channel separate to the compressed block itself. In `lz4_cstream`, we deal with that by prepending a variable-length integer containing the compressed size to each compressed block. This is suboptimal for single-fragment messages, since the user of lz4_cstream is likely going to remember the length of the whole message anyway, which makes the length prepended to the block redundant. But a loss of 1 byte is probably acceptable for most uses. --- CMakeLists.txt | 6 +- cmake/Findlz4.cmake | 60 ++++ cmake/Findzstd.cmake | 26 +- configure.py | 5 +- test/boost/CMakeLists.txt | 5 + test/boost/stream_compressor_test.cc | 166 +++++++++ utils/CMakeLists.txt | 1 + utils/stream_compressor.cc | 505 +++++++++++++++++++++++++++ utils/stream_compressor.hh | 187 ++++++++++ 9 files changed, 953 insertions(+), 8 deletions(-) create mode 100644 cmake/Findlz4.cmake create mode 100644 test/boost/stream_compressor_test.cc create mode 100644 utils/stream_compressor.cc create mode 100644 utils/stream_compressor.hh diff --git a/CMakeLists.txt b/CMakeLists.txt index 543762b2af78..7ffbabf7189c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -146,6 +146,7 @@ find_package(RapidJSON REQUIRED) find_package(xxHash REQUIRED) find_package(yaml-cpp REQUIRED) find_package(zstd REQUIRED) +find_package(lz4 REQUIRED) set(scylla_gen_build_dir "${CMAKE_BINARY_DIR}/gen") file(MAKE_DIRECTORY "${scylla_gen_build_dir}") @@ -213,7 +214,10 @@ target_link_libraries(scylla-main Seastar::seastar Snappy::snappy systemd - ZLIB::ZLIB) + ZLIB::ZLIB + lz4::lz4_static + zstd::zstd_static +) option(Scylla_CHECK_HEADERS "Add check-headers target for checking the self-containness of headers") diff --git a/cmake/Findlz4.cmake b/cmake/Findlz4.cmake new file mode 100644 index 000000000000..3361a14b4a81 --- /dev/null +++ b/cmake/Findlz4.cmake @@ -0,0 +1,60 @@ +# +# Copyright 2024-present ScyllaDB +# + +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# + +find_package (PkgConfig REQUIRED) + +pkg_search_module (PC_lz4 QUIET liblz4) + +find_library (lz4_STATIC_LIBRARY + NAMES liblz4.a + HINTS + ${PC_lz4_STATIC_LIBDIR} + ${PC_lz4_STATIC_LIBRARY_DIRS}) + +find_library (lz4_LIBRARY + NAMES lz4 + HINTS + ${PC_lz4_LIBDIR} + ${PC_lz4_LIBRARY_DIRS}) + +find_path (lz4_INCLUDE_DIR + NAMES lz4.h + HINTS + ${PC_lz4_STATIC_INCLUDEDIR} + ${PC_lz4_STATIC_INCLUDE_DIRS}) + +mark_as_advanced ( + lz4_STATIC_LIBRARY + lz4_LIBRARY + lz4_INCLUDE_DIR) + +include (FindPackageHandleStandardArgs) + +find_package_handle_standard_args (lz4 + REQUIRED_VARS + lz4_STATIC_LIBRARY + lz4_LIBRARY + lz4_INCLUDE_DIR + VERSION_VAR PC_lz4_STATIC_VERSION) + +if (lz4_FOUND) + if (NOT (TARGET lz4::lz4_static)) + add_library (lz4::lz4_static UNKNOWN IMPORTED) + set_target_properties (lz4::lz4_static + PROPERTIES + IMPORTED_LOCATION ${lz4_STATIC_LIBRARY} + INTERFACE_INCLUDE_DIRECTORIES ${lz4_INCLUDE_DIR}) + endif () + if (NOT (TARGET lz4::lz4)) + add_library (lz4::lz4 UNKNOWN IMPORTED) + set_target_properties (lz4::lz4 + PROPERTIES + IMPORTED_LOCATION ${lz4_LIBRARY} + INTERFACE_INCLUDE_DIRECTORIES ${lz4_INCLUDE_DIR}) + endif () +endif () diff --git a/cmake/Findzstd.cmake b/cmake/Findzstd.cmake index 22be8e04b3f0..653ed21ef08f 100644 --- a/cmake/Findzstd.cmake +++ b/cmake/Findzstd.cmake @@ -8,7 +8,13 @@ find_package (PkgConfig REQUIRED) -pkg_check_modules (PC_zstd QUIET libzstd) +pkg_search_module (PC_zstd QUIET libzstd) + +find_library (zstd_STATIC_LIBRARY + NAMES libzstd.a + HINTS + ${PC_zstd_STATIC_LIBDIR} + ${PC_zstd_STATIC_LIBRARY_DIRS}) find_library (zstd_LIBRARY NAMES zstd @@ -19,10 +25,11 @@ find_library (zstd_LIBRARY find_path (zstd_INCLUDE_DIR NAMES zstd.h HINTS - ${PC_zstd_INCLUDEDIR} - ${PC_zstd_INCLUDE_DIRS}) + ${PC_zstd_STATIC_INCLUDEDIR} + ${PC_zstd_STATIC_INCLUDE_DIRS}) mark_as_advanced ( + zstd_STATIC_LIBRARY zstd_LIBRARY zstd_INCLUDE_DIR) @@ -30,13 +37,20 @@ include (FindPackageHandleStandardArgs) find_package_handle_standard_args (zstd REQUIRED_VARS + zstd_STATIC_LIBRARY zstd_LIBRARY zstd_INCLUDE_DIR - VERSION_VAR PC_zstd_VERSION) + VERSION_VAR PC_zstd_STATIC_VERSION) if (zstd_FOUND) - set (zstd_LIBRARIES ${zstd_LIBRARY}) - set (zstd_INCLUDE_DIRS ${zstd_INCLUDE_DIR}) + if (NOT (TARGET zstd::zstd_static)) + add_library (zstd::zstd_static UNKNOWN IMPORTED) + + set_target_properties (zstd::zstd_static + PROPERTIES + IMPORTED_LOCATION ${zstd_STATIC_LIBRARY} + INTERFACE_INCLUDE_DIRECTORIES ${zstd_INCLUDE_DIR}) + endif () if (NOT (TARGET zstd::libzstd)) add_library (zstd::libzstd UNKNOWN IMPORTED) diff --git a/configure.py b/configure.py index 30df1cb02ba6..b50dae3283ba 100755 --- a/configure.py +++ b/configure.py @@ -536,6 +536,7 @@ def find_ninja(): 'test/boost/sstable_resharding_test', 'test/boost/sstable_test', 'test/boost/stall_free_test', + 'test/boost/stream_compressor_test', 'test/boost/string_format_test', 'test/boost/summary_test', 'test/boost/tagged_integer_test', @@ -774,6 +775,7 @@ def find_ninja(): 'utils/histogram_metrics_helper.cc', 'utils/on_internal_error.cc', 'utils/pretty_printers.cc', + 'utils/stream_compressor.cc', 'converting_mutation_partition_applier.cc', 'readers/combined.cc', 'readers/multishard.cc', @@ -1829,11 +1831,12 @@ def query_seastar_flags(pc_file, use_shared_libs, link_static_cxx=False): pkgs.append('lua53' if have_pkg('lua53') else 'lua') -libs = ' '.join([maybe_static(args.staticyamlcpp, '-lyaml-cpp'), '-latomic', '-llz4', '-lz', '-lsnappy', +libs = ' '.join([maybe_static(args.staticyamlcpp, '-lyaml-cpp'), '-latomic', '-lz', '-lsnappy', ' -lstdc++fs', ' -lcrypt', ' -lcryptopp', ' -lpthread', # Must link with static version of libzstd, since # experimental APIs that we use are only present there. maybe_static(True, '-lzstd'), + maybe_static(True, '-llz4'), maybe_static(args.staticboost, '-lboost_date_time -lboost_regex -licuuc -licui18n'), '-lxxhash', '-ldeflate', diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index 6addd8a9f262..d03903d55f5e 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -225,6 +225,11 @@ add_scylla_test(sstable_test KIND SEASTAR) add_scylla_test(stall_free_test KIND SEASTAR) +add_scylla_test(stream_compressor_test + KIND BOOST + LIBRARIES + Seastar::seastar_testing + utils) add_scylla_test(string_format_test KIND BOOST) add_scylla_test(summary_test diff --git a/test/boost/stream_compressor_test.cc b/test/boost/stream_compressor_test.cc new file mode 100644 index 000000000000..11e428910523 --- /dev/null +++ b/test/boost/stream_compressor_test.cc @@ -0,0 +1,166 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#define BOOST_TEST_MODULE core +#include +#include "utils/stream_compressor.hh" +#include "test/lib/random_utils.hh" +#include +#include "bytes.hh" + +template +concept RpcBuf = std::same_as || std::same_as; + +template +bytes rpc_buf_to_bytes(const Buf& data) { + if (auto src = std::get_if>(&data.bufs)) { + return bytes(reinterpret_cast(src->get()), src->size()); + } + auto src = std::get>>(data.bufs).data(); + auto out = bytes(bytes::initialized_later{}, data.size); + size_t i = 0; + while (i < data.size) { + std::memcpy(&out[i], src->get(), src->size()); + i += src->size(); + ++src; + } + return out; +} + +template +Buf convert_rpc_buf(BufFrom data) { + Buf b; + b.size = data.size; + b.bufs = std::move(data.bufs); + return b; +} + +// Compute some interesting splits of a message into pieces. +// Pieces of size 0, 1, and whole message are particularly interesting -- they should cover most edge cases. +template +std::vector compute_splits(bytes b) { + std::vector out; + { + // Mostly 1-byte pieces, some 0-byte pieces. + std::vector> frags; + frags.push_back(temporary_buffer()); + for (size_t i = 0; i < b.size();) { + for (size_t k = std::min(b.size(), i + 2); i < k; ++i) { + frags.push_back(temporary_buffer(reinterpret_cast(&b[i]), 1)); + } + frags.push_back(temporary_buffer()); + } + out.emplace_back(std::move(frags), b.size()); + } + { + // Mostly 0-byte pieces, some 1-byte pieces. + std::vector> frags; + frags.push_back(temporary_buffer()); + for (size_t i = 0; i < b.size(); ++i) { + frags.push_back(temporary_buffer(reinterpret_cast(&b[i]), 1)); + for (size_t k = 0; k < 2; ++k) { + frags.push_back(temporary_buffer()); + } + } + out.emplace_back(std::move(frags), b.size()); + } + { + // Whole message. + out.emplace_back(temporary_buffer(reinterpret_cast(b.data()), b.size())); + } + return out; +} + +void test_compressor_pair_basic_correctness(utils::stream_compressor& compressor, utils::stream_decompressor& decompressor) { + // Generate some messages. + for (const auto& message : { + tests::random::get_bytes(0), + tests::random::get_bytes(1), + tests::random::get_bytes(2), + tests::random::get_bytes(2000), + }) + // Test both with and without streaming. + for (bool end_of_frame : {false, true}) + // Split input into pieces. + for (const auto& input : compute_splits(message)) + // Test chunk sizes smaller, equal, and larger to the message. + for (const auto& chunk_size : {1, 5000}) + // Use each compressor multiple times to make sure it is returned to a proper state after each use. + for (int repeat = 0; repeat < 3; ++repeat) + { + auto compressed = utils::compress_impl(0, input, compressor, end_of_frame, chunk_size); + auto rcv_buf = convert_rpc_buf(std::move(compressed)); + auto decompressed = utils::decompress_impl(rcv_buf, decompressor, end_of_frame, chunk_size); + BOOST_REQUIRE_EQUAL(message, rpc_buf_to_bytes(std::move(decompressed))); + } +} + +BOOST_AUTO_TEST_CASE(test_correctness) { + { + utils::raw_stream stream; + test_compressor_pair_basic_correctness(stream, stream); + } + { + utils::lz4_dstream dstream{}; + utils::lz4_cstream cstream{}; + test_compressor_pair_basic_correctness(cstream, dstream); + } + { + utils::lz4_dstream dstream{2}; + utils::lz4_cstream cstream{2}; + test_compressor_pair_basic_correctness(cstream, dstream); + } + { + utils::zstd_dstream dstream{}; + utils::zstd_cstream cstream{}; + test_compressor_pair_basic_correctness(cstream, dstream); + } +} + +void test_recovery_after_oom_one(utils::stream_decompressor& dstream, utils::stream_compressor& cstream) { + // Check that compressors and decompressors handle OOM properly and can be reused afterwards. + for (int repeat = 0; repeat < 3; ++repeat) { + auto message = tests::random::get_bytes(256); + rpc::snd_buf compressed; + { + auto message_buf = rpc::snd_buf(temporary_buffer(reinterpret_cast(message.data()), message.size())); + memory::with_allocation_failures([&] { + try { + compressed = utils::compress_impl(0, message_buf, cstream, true, 64); + } catch (const std::runtime_error&) { + throw std::bad_alloc(); + } + }); + } + rpc::rcv_buf decompressed; + { + auto compressed_2 = convert_rpc_buf(std::move(compressed)); + memory::with_allocation_failures([&] { + try { + decompressed = utils::decompress_impl(compressed_2, dstream, true, 64); + } catch (const std::runtime_error&) { + throw std::bad_alloc(); + } + }); + } + BOOST_REQUIRE_EQUAL(message, rpc_buf_to_bytes(decompressed)); + } +} + +BOOST_AUTO_TEST_CASE(test_recovery_after_oom) { + { + utils::lz4_dstream dstream{}; + utils::lz4_cstream cstream{}; + test_recovery_after_oom_one(dstream, cstream); + } + { + utils::zstd_dstream dstream{}; + utils::zstd_cstream cstream{}; + test_recovery_after_oom_one(dstream, cstream); + } +} diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index e6a544413d80..f81029213312 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -46,6 +46,7 @@ target_sources(utils utf8.cc uuid.cc aws_sigv4.cc + stream_compressor.cc s3/aws_error.cc s3/client.cc s3/retry_strategy.cc) diff --git a/utils/stream_compressor.cc b/utils/stream_compressor.cc new file mode 100644 index 000000000000..3f7c371cef3b --- /dev/null +++ b/utils/stream_compressor.cc @@ -0,0 +1,505 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "utils/stream_compressor.hh" + +#include +#include +#include +#include +#include +#include +#include "utils/small_vector.hh" +#include "seastarx.hh" +#include "utils/crc.hh" + +#define ZSTD_STATIC_LINKING_ONLY +#include + +namespace utils { + +namespace { + +size_t varint_length(uint8_t first_byte) { + return 1 + std::countr_zero(first_byte); +} + +uint32_t varint_decode(uint64_t all_bytes) { + size_t n_bytes = 1 + std::countr_zero(all_bytes); + size_t ignored_msb = (64 - 8 * n_bytes); + return ((all_bytes << ignored_msb) >> ignored_msb) >> n_bytes; +} + +std::pair varint_encode(uint32_t x) { + size_t n_bytes = (std::max(std::bit_width(x), 1) + 6) / 7; + return {n_bytes, ((x << 1) | 1) << (n_bytes - 1)}; +} + +} // namespace + +size_t raw_stream::copy(ZSTD_outBuffer* out, ZSTD_inBuffer* in) { + size_t n = std::min(out->size - out->pos, in->size - in->pos); + memcpy((char*)out->dst + out->pos, (const char*)in->src + in->pos, n); + in->pos += n; + out->pos += n; + return 0; +} + +void raw_stream::decompress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, bool) { + copy(out, in); +} + +size_t raw_stream::compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDirective end) { + return copy(out, in); +} + +void raw_stream::reset() noexcept { +} + +// Throw if ret is an ZSTD error code. +static void check_zstd(size_t ret, const char* text) { + if (ZSTD_isError(ret)) { + throw std::runtime_error(fmt::format("{} error: {}", text, ZSTD_getErrorName(ret))); + } +} + +zstd_dstream::zstd_dstream() { + _ctx.reset(ZSTD_createDStream()); + check_zstd(ZSTD_DCtx_setParameter(_ctx.get(), ZSTD_d_format, ZSTD_f_zstd1_magicless), "ZSTD_CCtx_setParameter(.., ZSTD_c_format, ZSTD_f_zstd1_magicless)"); + if (!_ctx) { + throw std::bad_alloc(); + } +} + +void zstd_dstream::reset() noexcept { + ZSTD_DCtx_reset(_ctx.get(), ZSTD_reset_session_only); + _in_progress = false; +} + +void zstd_dstream::decompress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, bool end_of_frame) { + if (!_in_progress && in->pos == in->size) { + // Without this early return, we would start the decompression of the next frame. + // ZSTD_decompressStream() would return something positive, and our truncation + // checking logic could wrongly claim truncation. + return; + } + size_t ret = ZSTD_decompressStream(_ctx.get(), out, in); + check_zstd(ret, "ZSTD_decompressStream"); + _in_progress = bool(ret); + if (in->pos == in->size && end_of_frame && _in_progress) { + // Should never happen in practice, barring a bug/corruption. + // It's not the compressor's job to verify data integrity, + // so we could as well not check for this. + // + // But we do it as a good practice. + // To an extent, it keeps the effects of a corruption localized to the corrupted message. + // This could make some possible debugging easier. + // (If we didn't check for truncation, a bad message could leave leftovers in the compressor, + // which would cause weirdness when decompressing the next -- perhaps correct -- message). + throw std::runtime_error("truncated ZSTD frame"); + } +} +void zstd_dstream::set_dict(const ZSTD_DDict* dict) { + _dict = dict; + ZSTD_DCtx_refDDict(_ctx.get(), _dict); +} + +zstd_cstream::zstd_cstream() { + _ctx.reset(ZSTD_createCStream()); + if (!_ctx) { + throw std::bad_alloc(); + } + // For now, we hardcode a 128 kiB window and the lowest compression level here. + check_zstd(ZSTD_initCStream(_ctx.get(), 1), "ZSTD_initCStream(.., 1)"); + check_zstd(ZSTD_CCtx_setParameter(_ctx.get(), ZSTD_c_format, ZSTD_f_zstd1_magicless), "ZSTD_CCtx_setParameter(.., ZSTD_c_format, ZSTD_f_zstd1_magicless)"); + check_zstd(ZSTD_CCtx_setParameter(_ctx.get(), ZSTD_c_contentSizeFlag, 0), "ZSTD_CCtx_setParameter(.., ZSTD_c_contentSizeFlag, 0)"); + check_zstd(ZSTD_CCtx_setParameter(_ctx.get(), ZSTD_c_checksumFlag, 0), "ZSTD_CCtx_setParameter(.., ZSTD_c_checksumFlag, 0)"); + check_zstd(ZSTD_CCtx_setParameter(_ctx.get(), ZSTD_c_dictIDFlag, 0), "ZSTD_CCtx_setParameter(.., ZSTD_c_dictIDFlag, 0)"); + check_zstd(ZSTD_CCtx_setParameter(_ctx.get(), ZSTD_c_windowLog, 17), "ZSTD_CCtx_setParameter(.., ZSTD_c_windowLog, 17)"); +} + +size_t zstd_cstream::compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDirective end) { + size_t ret = ZSTD_compressStream2(_ctx.get(), out, in, end); + check_zstd(ret, "ZSTD_compressStream2"); + return ret; +} + +void zstd_cstream::reset() noexcept { + ZSTD_CCtx_reset(_ctx.get(), ZSTD_reset_session_only); +} +void zstd_cstream::set_dict(const ZSTD_CDict* dict) { + _dict = dict; + ZSTD_CCtx_refCDict(_ctx.get(), _dict); +} + +// Used as a intermediary buffer by lz4_cstream and lz4_dstream; +// Since this buffer is shared, lz4_cstream and lz4_dstream must be used synchronously. +// That is, when the processing of a frame (the string of data between `reset()`s) starts, +// it has to be finished before another compressor instance attempts to use it. +// +// If there ever is a need to process multiple frames concurrently, the dependency on this +// buffer has to be eliminated, and each concurrent stream must be given a separate buffer. +// +// This makes for a bad/dangerous API, but it's only used in this file, so it's okay for now. +static thread_local std::array _lz4_scratch; + +lz4_cstream::lz4_cstream(size_t window_size) + : _buf(window_size) +{ + reset(); +} + +void lz4_cstream::reset() noexcept { + _scratch_beg = _scratch_end = _buf_pos = 0; + LZ4_initStream(&_ctx, sizeof(_ctx)); + LZ4_attach_dictionary(&_ctx, _dict); +} + +void lz4_cstream::resetFast() noexcept { + _scratch_beg = _scratch_end = _buf_pos = 0; + LZ4_resetStream_fast(&_ctx); + LZ4_attach_dictionary(&_ctx, _dict); +} + +// When new data arrives in `in`, we copy an arbitrary amount of it to `_buf`, +// (the amount is arbirary, but it has to fit contiguously in `_buf`), +// compress the new block from `_buf` to `_lz4_scratch`, +// then we copy everything from `_lz4_scratch` to `out`. +// Repeat until `in` is empty. +size_t lz4_cstream::compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDirective end) { + if (_scratch_end == _scratch_beg && in->size != in->pos) { + // If we already copied everything we compressed to `out`, + // and there is still some data in `in`, + // we are going to compress a new block from `in` to `_lz4_scratch`. + + if (_buf_pos == _buf.size()) { + // The ring buffer wraps around here. + _buf_pos = 0; + } + // We will compress the biggest prefix of `in` that fits contiguously inside `buf`. + // In principle, this is sligthly suboptimal -- ideally, if `in` is smaller than the contiguous space in `buf`, + // we should only copy `in` to `buf` and wait with the compressor call until future `in`s + // fill the contiguous space entirely, or `end` is `ZSTD_e_flush` or `ZSTD_e_end`. + // But for streaming LZ4 it doesn't really make a difference. + size_t n = std::min(_buf.size() - _buf_pos, in->size - in->pos); + // The compressed block mustn't fill the entire ring buffer. + // It must be at least 1 byte shorter. It's a dumb quirk of lz4. Perhaps it should be called a bug. + n = std::min(max_lz4_window_size - 1, n); + std::memcpy(_buf.data() + _buf_pos, static_cast(in->src) + in->pos, n); + in->pos += n; + // The first header_size bytes contain the length of the compressed block, so we compress to _lz4_scratch.data() + sizeof(uint64_t). + int x = LZ4_compress_fast_continue(&_ctx, _buf.data() + _buf_pos, _lz4_scratch.data() + sizeof(uint64_t), n, _lz4_scratch.size(), 1); + if (x < 0) { + throw std::runtime_error(fmt::format( + "LZ4_compress_fast_continue failed with negative return value {}. " + "LZ4 shouldn't fail. This indicates a bug or a corruption.", + x)); + } + auto [header_size, header_value] = varint_encode(x); + _scratch_end = sizeof(uint64_t) + x; + _scratch_beg = sizeof(uint64_t) - header_size; + _buf_pos += n; + // We prepend to every block its compressed length. + // This is necessary, because LZ4 needs to know it, but it doesn't store it by itself. + seastar::write_le(_lz4_scratch.data(), header_value << (8 * (sizeof(uint64_t) - header_size))); + } + if (_scratch_end > _scratch_beg) { + // If we compressed a block to _lz4_scratch, but we still haven't copied it all to `out`, + // we copy as much as possible to `out`. + size_t n = std::min(_scratch_end - _scratch_beg, out->size - out->pos); + std::memcpy(static_cast(out->dst) + out->pos, _lz4_scratch.data() + _scratch_beg, n); + _scratch_beg += n; + out->pos += n; + } + if (_scratch_beg == _scratch_end && in->size == in->pos && end == ZSTD_e_end) { + // If we have no more data in `_lz4_scratch`, there is no more data in `in`, and `end` + // says that this is the last block in the stream, we reset the stream state. + resetFast(); + // Note that the we currently don't emit anything to mark end of stream. + // We rely on the outer layer to notify the decompressor about it manually. + // This is inconsistent with what zstd's API does. We could fix that for elegance points. + }; + // Return a non-zero value if there is still some data lingering in the compressor's + // internal buffers (`_lz4_scratch`), which has to be copied out. + // This will prompt the caller to call us again, even if `in` is empty. + return _scratch_beg != _scratch_end; +} +// The passed dict must live until it is unset by another set_dict(). (Or until the compressor is destroyed). +// The pointer can be null, this will unset the current dict. +void lz4_cstream::set_dict(const LZ4_stream_t* dict) { + _dict = dict; + resetFast(); +} + +lz4_dstream::lz4_dstream(size_t window_size) + : _buf(window_size) +{} + +void lz4_dstream::reset() noexcept { + _scratch_pos = _buf_end = _buf_beg = 0; + LZ4_setStreamDecode(&_ctx, reinterpret_cast(_dict.size() ? _dict.data() : nullptr), _dict.size()); +} + +void lz4_dstream::decompress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, bool end_of_frame) { + if (_buf_beg == _buf_end) { + // If we have no decompressed data that wasn't yet output, + // we will decompress a new block. + + if (_buf_end == _buf.size()) { + // The ring buffer wraps around here. + _buf_end = _buf_beg = 0; + } + // First, we have to ingest the first few bytes of input data, which contain + // the post-compression size of the following block. + size_t header_size = 1; + if (_scratch_pos < header_size) { + size_t n = std::min(header_size - _scratch_pos, in->size - in->pos); + std::memcpy(_lz4_scratch.data() + _scratch_pos, static_cast(in->src) + in->pos, n); + _scratch_pos += n; + in->pos += n; + } + if (_scratch_pos >= header_size) { + header_size = varint_length(static_cast(_lz4_scratch[0])); + } + if (_scratch_pos < header_size) { + size_t n = std::min(header_size - _scratch_pos, in->size - in->pos); + std::memcpy(_lz4_scratch.data() + _scratch_pos, static_cast(in->src) + in->pos, n); + _scratch_pos += n; + in->pos += n; + } + // If we know the first header_size bytes, we can read the compressed size and ingest that many bytes, + // so we have a full compressed block in contiguous memory. + if (_scratch_pos >= header_size) { + auto x = varint_decode(seastar::read_le(_lz4_scratch.data())); + if (x + header_size > _lz4_scratch.size()) { + throw std::runtime_error(fmt::format("Oversized LZ4 block: {} bytes.", x + header_size)); + } + size_t n = std::min(x + header_size - _scratch_pos, in->size - in->pos); + std::memcpy(_lz4_scratch.data() + _scratch_pos, static_cast(in->src) + in->pos, n); + _scratch_pos += n; + in->pos += n; + if (_scratch_pos == x + header_size) { + // Now the full compressed block is in contiguous memory. + // We can decompress it to `_buf` and clear `_lz4_scratch`. + size_t n_buf = _buf.size() - _buf_end; + int ret = LZ4_decompress_safe_continue(&_ctx, _lz4_scratch.data() + header_size, _buf.data() + _buf_end, x, n_buf); + if (ret < 0) { + throw std::runtime_error(fmt::format("LZ4_decompress_safe_continue failed with negative return value {}. This indicates a corruption of the RPC connection.", ret)); + } + _buf_end += ret; + _scratch_pos = 0; + } + } + } + if (_buf_end > _buf_beg) { + // If we have some decompressed data that wasn't yet output, + // we output as much as possible. + size_t n = std::min(_buf_end - _buf_beg, out->size - out->pos); + std::memcpy(static_cast(out->dst) + out->pos, _buf.data() + _buf_beg, n); + out->pos += n; + _buf_beg += n; + } + if (end_of_frame && _buf_beg == _buf_end && in->size == in->pos) { + // If there is no data in internal buffers to be output, there is no more data in `in`, + // and `in` was the last block in the stream, we reset the stream state. + if (_scratch_pos != 0) { + throw std::runtime_error(fmt::format("Truncated LZ4 frame.")); + } + reset(); + } + // Sanity check. + assert(_buf_end >= _buf_beg); +} +// The passed dict must live until it is unset by another set_dict(). (Or until the decompressor is destroyed). +// The span can be empty, this will unset the current dict. +void lz4_dstream::set_dict(std::span dict) { + _dict = dict; + reset(); +} + +rpc::snd_buf compress_impl(size_t head_space, const rpc::snd_buf& data, stream_compressor& compressor, bool end_of_frame, size_t chunk_size) try { + const auto size = data.size; + + auto src = std::get_if>(&data.bufs); + if (!src) { + src = std::get>>(data.bufs).data(); + } + + size_t size_left = size; + size_t size_compressed = 0; + + ZSTD_inBuffer inbuf = {}; + ZSTD_outBuffer outbuf = {}; + + small_vector, 16> dst_buffers; + + // Note: we always allocate chunk_size here, then we resize it to fit at the end. + // Maybe that's a waste of cycles, and we should allocate a buffer that's about as big + // as the input, and not shrink at the end? + dst_buffers.emplace_back(std::max(head_space, chunk_size)); + outbuf.dst = dst_buffers.back().get_write(); + outbuf.size = dst_buffers.back().size(); + outbuf.pos = head_space; + + // Stream compressors can handle frames of size 0, + // but when a Zstd compressor is called multiple times (around 10, I think?) + // with an empty input, it tries to be helpful and emits a "no progress" error. + // + // To avoid that, we handle empty frames we special-case empty frames with the `if` + // below, and we just return an empty buffer as the result, + // without putting the empty message through the compressor. + if (size > 0) { + while (true) { + if (size_left && inbuf.pos == inbuf.size) { + size_left -= src->size(); + inbuf.src = src->get(); + inbuf.size = src->size(); + inbuf.pos = 0; + ++src; + continue; + } + if (outbuf.pos == outbuf.size) { + size_compressed += outbuf.pos; + dst_buffers.emplace_back(chunk_size); + outbuf.dst = dst_buffers.back().get_write(); + outbuf.size = dst_buffers.back().size(); + outbuf.pos = 0; + continue; + } + size_t ret = compressor.compress(&outbuf, &inbuf, size_left ? ZSTD_e_continue : (end_of_frame ? ZSTD_e_end : ZSTD_e_flush)); + if (!size_left // No more input chunks. + && inbuf.pos == inbuf.size // No more data in the last chunk. + && ret == 0 // No data remaining in compressor's internal buffers. + ) { + break; + } + } + } + size_compressed += outbuf.pos; + dst_buffers.back().trim(outbuf.pos); + + // In this routine, we always allocate fragments of size chunk_size, even + // if the message is tiny. + // Hence, at this point, dst_buffers contains up to (128 kiB) unused memory. + // When there are many concurrent RPC messages, this waste might add up to + // a considerable overhead. + // Let's pay some cycles to shrink the underlying allocation to fit, to + // avoid any problems with that unseen memory usage. + dst_buffers.back() = dst_buffers.back().clone(); + + if (dst_buffers.size() == 1) { + return rpc::snd_buf(std::move(dst_buffers.front())); + } + return rpc::snd_buf({std::make_move_iterator(dst_buffers.begin()), std::make_move_iterator(dst_buffers.end())}, size_compressed); +} catch (...) { + compressor.reset(); + throw; +} + +template +requires std::same_as || std::same_as +uint32_t crc_impl_generic(const T& data) noexcept { + auto size = data.size; + + auto it = std::get_if>(&data.bufs); + if (!it) { + it = std::get>>(data.bufs).data(); + } + + utils::crc32 crc; + while (size > 0) { + crc.process(reinterpret_cast(it->get()), it->size()); + size -= it->size(); + ++it; + } + return crc.get(); +} + +uint32_t crc_impl(const rpc::snd_buf& data) noexcept { + return crc_impl_generic(data); +} + +uint32_t crc_impl(const rpc::rcv_buf& data) noexcept { + return crc_impl_generic(data); +} + +rpc::rcv_buf decompress_impl(const seastar::rpc::rcv_buf& data, stream_decompressor& decompressor, bool end_of_frame, size_t chunk_size) try { + // As a special-case, if we take empty input, we immediately return an empty output, + // without going through the actual compressors. + // + // This is due to a quirk of Zstd, see the matching comment in `decompress_impl` + // for details. + if (data.size == 0) { + return rpc::rcv_buf(); + } + + size_t size_decompressed = 0; + + size_t input_remaining = data.size; + auto it = std::get_if>(&data.bufs); + if (!it) { + it = std::get>>(data.bufs).data(); + } + + small_vector, 16> dst_buffers; + + ZSTD_inBuffer inbuf = {}; + ZSTD_outBuffer outbuf = {}; + while (true) { + if (input_remaining && inbuf.pos == inbuf.size) { + inbuf.src = it->get(); + inbuf.size = it->size(); + input_remaining -= it->size(); + ++it; + inbuf.pos = 0; + continue; + } + if (outbuf.pos == outbuf.size) { + size_decompressed += outbuf.pos; + dst_buffers.emplace_back(chunk_size); + outbuf.dst = dst_buffers.back().get_write(); + outbuf.size = dst_buffers.back().size(); + outbuf.pos = 0; + continue; + } + decompressor.decompress(&outbuf, &inbuf, end_of_frame && !input_remaining); + if (!input_remaining // No more input chunks. + && inbuf.pos == inbuf.size // No more data in the last input chunk. + && outbuf.pos < outbuf.size // No more data in decompressor's internal buffers. + ) { + break; + } + } + + size_decompressed += outbuf.pos; + if (!dst_buffers.empty()) { + dst_buffers.back().trim(outbuf.pos); + // In this routine, we always allocate fragments of size chunk_size, even + // if the message is tiny. + // Hence, at this point, dst_buffers contains up to (128 kiB) unused memory. + // When there are many concurrent RPC messages, this waste might add up to + // a considerable overhead. + // Let's pay some cycles to shrink the underlying allocation to fit, to + // avoid any problems with that unseen memory usage. + + // We could avoid the CPU cost if we prepended the decompressed size to + // the message (thus growing the messages by 1-2 bytes) and used that to + // allocate exactly the right buffer. + // But I don't know if it's worth the 2 bytes. + dst_buffers.back() = dst_buffers.back().clone(); + } + if (dst_buffers.size() == 1) { + return rpc::rcv_buf(std::move(dst_buffers.front())); + } + return rpc::rcv_buf({std::make_move_iterator(dst_buffers.begin()), std::make_move_iterator(dst_buffers.end())}, size_decompressed); +} catch (...) { + decompressor.reset(); + throw; +} + +} // namespace utils diff --git a/utils/stream_compressor.hh b/utils/stream_compressor.hh new file mode 100644 index 000000000000..f309b7b53592 --- /dev/null +++ b/utils/stream_compressor.hh @@ -0,0 +1,187 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include +#include + +#define ZSTD_STATIC_LINKING_ONLY +#include +#define LZ4_STATIC_LINKING_ONLY +#include + +namespace utils { + +// The pairs (zstd_dstream, zstd_cstream) and (raw_stream, raw_stream) +// implement a common compressor interface, with similar semantics as the streaming interface of zstd. +// (That's why zstd_cstream and zstd_dstream are only thin wrappers, and why all take ZSTD_inBuffer/ZSTD_outBuffer args). +// The main difference with zstd's interface is that we communicate errors by exception rather than +// by error code. + +struct stream_compressor { + virtual size_t compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDirective end) = 0; + // After compress() throws, the compressor is left in an undefined state. + // In this state, it mustn't be used for compression. + // However, reset() can be called to reset the internal state and recycle the compressor. + virtual void reset() noexcept = 0; + virtual ~stream_compressor() {} +}; + +struct stream_decompressor { + virtual void decompress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, bool end_of_frame) = 0; + // After decompress() throws, the decompressor is left in an undefined state. + // In this state, it mustn't be used for decompression. + // However, reset() can be called to reset the internal state and recycle the decompressor. + virtual void reset() noexcept = 0; + virtual ~stream_decompressor() {} +}; + +// Implements a streaming compression interface similar to ZSTD_CStream/ZSTD_DStream, +// but the "compression" is just memcpy. +struct raw_stream final : public stream_compressor, public stream_decompressor { + static size_t copy(ZSTD_outBuffer* out, ZSTD_inBuffer* in); + void decompress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, bool) override; + size_t compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDirective end) override; + void reset() noexcept override; +}; + +// Thin wrapper over ZSTD_DStream. +class zstd_dstream final : public stream_decompressor { + struct ctx_deleter { + void operator()(ZSTD_DStream* stream) const noexcept { + ZSTD_freeDStream(stream); + } + }; + std::unique_ptr _ctx; + const ZSTD_DDict* _dict; + bool _in_progress = false; +public: + zstd_dstream(); + void reset() noexcept override; + void decompress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, bool end_of_frame) override; + // The passed dict must live until it is unset by another set_dict(). (Or until the compressor is destroyed). + // The pointer can be null, this will unset the current dict. + void set_dict(const ZSTD_DDict* dict); +}; + +// Thin wrapper over ZSTD_CStream. +class zstd_cstream final : public stream_compressor { + struct ctx_deleter { + void operator()(ZSTD_CStream* stream) const noexcept { + ZSTD_freeCStream(stream); + } + }; + std::unique_ptr _ctx; + const ZSTD_CDict* _dict; +public: + zstd_cstream(); + size_t compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDirective end) override; + void reset() noexcept override; + // The passed dict must live until it is unset by another set_dict(). (Or until the compressor is destroyed). + // The pointer can be null, this will unset the current dict. + void set_dict(const ZSTD_CDict* dict); +}; + +seastar::rpc::snd_buf compress_impl(size_t head_space, const seastar::rpc::snd_buf& data, stream_compressor& compressor, bool end_of_frame, size_t chunk_size); +seastar::rpc::rcv_buf decompress_impl(const seastar::rpc::rcv_buf& data, stream_decompressor& decompressor, bool end_of_frame, size_t chunk_size); +uint32_t crc_impl(const seastar::rpc::snd_buf& data) noexcept; +uint32_t crc_impl(const seastar::rpc::rcv_buf& data) noexcept; + +// Size of the history buffer maintained by both sides of the compressed connection. +// Governs the memory usage and effectiveness of streaming LZ4 compression. +// +// There is no value in making it greater than 64 kiB, because LZ4 doesn't support +// greater history sizes, at least in the official releases of LZ4. +// But it might be set smaller to reduce memory usage at the cost of lowered +// compression strength. +// +// If LZ4 streaming compression turns out effective, we should make this live-updatable +// and check the effectiveness of various sizes in practice. +// +// Must be equal on both sides of the connection. Currently this is achieved by making it +// a constant. If we want to make it live-updatable, changes in window size will have to +// be made a part of lz4_cstream's/lz4_dstream's internal protocol. +constexpr size_t max_lz4_window_size = 64 * 1024; + +// Implements a streaming compression interface similar to ZSTD_CStream. +class lz4_cstream final : public stream_compressor { + // A ring buffer with recent stream history. + // + // To implement streaming compression, LZ4 doesn't copy the history to its own buffer. + // Instead, we maintain our own history buffer, and the LZ4 compressor only stores a view + // of the most recent contiguous 64 kiB chunk from that buffer. + // + // Thus the "contiguity" during decompression has to match the "contiguity" during compression. + // That is, for every block, the contiguous sum (up to 64 kiB) of most recent views passed to + // LZ4_decompress_safe_continue must be at least as long as the contiguous sum (up to 64 kiB) + // of most recent views passed to LZ4_compress_fast_continue for that same block. + // + // Thus there are some rules/schemes which have to be obeyed when maintaining the history buffer. + // + // We use a scheme which LZ4 calls "synchronized". + // The two history ringbuffers on both sides of the stream are in "lockstep", + // meaning that every compression call with compressor's _buf as source, + // has a matching decompression call with decompressor's _buf as target, + // with the same length and offset in _buf. + std::vector _buf; + // The current position in the ringbuffer _buf. New input will be appened at this position. + size_t _buf_pos = 0; + // This pair describes the compressed data in `_lz4_scratch`, which is pending output. + // We have to copy it out before we can compress new data to the scratch buffer. + size_t _scratch_beg = 0; + size_t _scratch_end = 0; + + LZ4_stream_t _ctx; + const LZ4_stream_t* _dict = nullptr; + +public: + lz4_cstream(size_t window_size = max_lz4_window_size); + void reset() noexcept override; + void resetFast() noexcept; + // When new data arrives in `in`, we copy an arbitrary amount of it to `_buf`, + // (the amount is arbirary, but it has to fit contiguously in `_buf`), + // compress the new block from `_buf` to `_lz4_scratch`, + // then we copy everything from `_lz4_scratch` to `out`. + // Repeat until `in` is empty. + size_t compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDirective end) override; + // The passed dict must live until it is unset by another set_dict(). (Or until the compressor is destroyed). + // The pointer can be null, this will unset the current dict. + void set_dict(const LZ4_stream_t* dict); +}; + +// Implements a streaming compression interface similar to ZSTD_DStream. +class lz4_dstream final : public stream_decompressor { + // See the _buf comment in lz4_cstream. + std::vector _buf; + // The write position in `_buf`. New input will be decompressed to this offset. + // It's updated in lockstep with `_buf_pos` of the compressor. + size_t _buf_end = 0; + // The read position in `_buf`. The chunk between `_buf_beg` and `_buf_end` is the data + // that was decompressed, but hasn't been copied to caller's `out` yet. + // We have to copy it out before we can overwrite it with new decompressed data. + size_t _buf_beg = 0; + // The amount of data accumulated in `_lz4_scratch`. Data accumulates in `_lz4_scratch` until + // a full LZ4 block (with prepended length) is available — only then we can decompress it to `_buf. + size_t _scratch_pos = 0; + + LZ4_streamDecode_t _ctx; + std::span _dict; + +public: + lz4_dstream(size_t window_size = max_lz4_window_size); + void reset() noexcept override; + void decompress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, bool has_more_fragments) override; + // The passed dict must live until it is unset by another set_dict(). (Or until the decompressor is destroyed). + // The span can be empty, this will unset the current dict. + void set_dict(std::span dict); +}; + +} // namespace utils From d301c29af5bcedeb90b93bf6f325003bc22eab62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Sat, 23 Mar 2024 11:11:40 +0100 Subject: [PATCH 072/397] utils: introduce alien_worker Introduces a util which launches a new OS thread and accepts callables for concurrent execution. Meant to be created once at startup and used until shutdown, for running nonpreemptible, 3rd party, non-interactive code. Note: this new utility is almost identical to wasm::alien_thread_runner. Maybe we should unify them. --- configure.py | 1 + utils/CMakeLists.txt | 1 + utils/alien_worker.cc | 60 +++++++++++++++++++++++++++++++++++++ utils/alien_worker.hh | 70 +++++++++++++++++++++++++++++++++++++++++++ 4 files changed, 132 insertions(+) create mode 100644 utils/alien_worker.cc create mode 100644 utils/alien_worker.hh diff --git a/configure.py b/configure.py index b50dae3283ba..800165d283a7 100755 --- a/configure.py +++ b/configure.py @@ -761,6 +761,7 @@ def find_ninja(): 'row_cache.cc', 'schema_mutations.cc', 'generic_server.cc', + 'utils/alien_worker.cc', 'utils/array-search.cc', 'utils/base64.cc', 'utils/logalloc.cc', diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index f81029213312..38bf87aa5e16 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -5,6 +5,7 @@ add_library(utils STATIC) target_sources(utils PRIVATE UUID_gen.cc + alien_worker.cc arch/powerpc/crc32-vpmsum/crc32_wrapper.cc arch/powerpc/crc32-vpmsum/crc32.S array-search.cc diff --git a/utils/alien_worker.cc b/utils/alien_worker.cc new file mode 100644 index 000000000000..bdfccda29b88 --- /dev/null +++ b/utils/alien_worker.cc @@ -0,0 +1,60 @@ +/* + * Copyright (C) 2024-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "utils/alien_worker.hh" +#include + +using namespace seastar; + +namespace utils { + +std::thread alien_worker::spawn(seastar::logger& log, int niceness) { + sigset_t newset; + sigset_t oldset; + sigfillset(&newset); + auto r = ::pthread_sigmask(SIG_SETMASK, &newset, &oldset); + assert(r == 0); + auto thread = std::thread([this, &log, niceness] () noexcept { + errno = 0; + int nice_value = nice(niceness); + if (nice_value == -1 && errno != 0) { + log.warn("Unable to renice worker thread (system error number {}); the thread will compete with reactor, which can cause latency spikes. Try adding CAP_SYS_NICE", errno); + } + + while (true) { + std::unique_lock lk(_mut); + _cv.wait(lk, [this] { return !_pending.empty() || !_running; }); + if (!_running) { + return; + } + auto f = std::move(_pending.front()); + _pending.pop(); + lk.unlock(); + f(); + } + }); + r = ::pthread_sigmask(SIG_SETMASK, &oldset, nullptr); + assert(r == 0); + return thread; +} + +alien_worker::alien_worker(seastar::logger& log, int niceness) + : _thread(spawn(log, niceness)) +{} + +alien_worker::~alien_worker() { + { + std::unique_lock lk(_mut); + _running = false; + } + _cv.notify_one(); + _thread.join(); +} + +} // namespace utils + diff --git a/utils/alien_worker.hh b/utils/alien_worker.hh new file mode 100644 index 000000000000..db645e1d6ed8 --- /dev/null +++ b/utils/alien_worker.hh @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2024-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include + +#include + +namespace seastar { + class logger; +} // namespace seastar + +namespace utils { + +// Spawns a new OS thread, which can be used as a worker for running nonpreemptible 3rd party code. +// Callables can be sent to the thread for execution via submit(). +class alien_worker { + bool _running = true; + std::mutex _mut; + std::condition_variable _cv; + std::queue> _pending; + // Note: initialization of _thread uses other fields, so it must be performed last. + std::thread _thread; + + std::thread spawn(seastar::logger&, int niceness); +public: + alien_worker(seastar::logger&, int niceness); + ~alien_worker(); + // The worker captures `this`, so `this` must have a stable address. + alien_worker(const alien_worker&) = delete; + alien_worker(alien_worker&&) = delete; + + // Submits a new callable to the thread for execution. + // This callable will run on a different OS thread, + // concurrently with the current thread, so be careful not to cause a data race. + // Avoid capturing references in the callable if possible, and if you do, + // be extremely careful about their concurrent uses. + template + seastar::future submit(seastar::noncopyable_function f) { + auto p = seastar::promise(); + auto fut = p.get_future(); + auto wrapper = [p = std::move(p), f = std::move(f), shard = seastar::this_shard_id(), &alien = seastar::engine().alien()] () mutable noexcept { + try { + auto v = f(); + seastar::alien::run_on(alien, shard, [v = std::move(v), p = std::move(p)] () mutable noexcept { + p.set_value(std::move(v)); + }); + } catch (...) { + seastar::alien::run_on(alien, shard, [p = std::move(p), ep = std::current_exception()] () mutable noexcept { + p.set_exception(ep); + }); + } + }; + { + std::unique_lock lk(_mut); + _pending.push(std::move(wrapper)); + } + _cv.notify_one(); + return fut; + } +}; + +} // namespace utils From 9de52b1c98529d9bc0b56fa78c55aaa5edbed7c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Thu, 7 Dec 2023 19:41:02 +0100 Subject: [PATCH 073/397] utils: introduce reservoir_sampling We are planning to improve some usages of compression in Scylla (in which we compress small blocks of data) by pre-training compression dictionaries on similar data seen so far. For example, many RPC messages have similar structure (and likely similar data), so the similarity could be exploited for better compression. This can be achieved e.g. by training a dictionary on the RPC traffic, and compressing subsequent RPC messages against that dictionary. To work well, the training should be fed a representative sample of the compressible data. Such a sample can be approached by taking a random subset (of some given reasonable size) of the data, with uniform probability. For our purposes, we need an online algorithm for this -- one which can select the random k-subset from a stream of arbitrary size (e.g. all RPC traffic over an hour), while requiring only the necessary minimum of memory. This is a known problem, called "reservoir sampling". This PR introduces `reservoir_sampler`, which implements an optimal algorithm for reservoir sampling. Additionally, it introduces `page_sampler` -- a wrapper for `reservoir_sampler`, which uses it to select a random sample of pages from a stream of bytes. --- configure.py | 2 + test/boost/CMakeLists.txt | 2 + test/boost/reservoir_sampling_test.cc | 171 ++++++++++++++++++++++ utils/reservoir_sampling.hh | 198 ++++++++++++++++++++++++++ 4 files changed, 373 insertions(+) create mode 100644 test/boost/reservoir_sampling_test.cc create mode 100644 utils/reservoir_sampling.hh diff --git a/configure.py b/configure.py index 800165d283a7..c9760d8dbbc9 100755 --- a/configure.py +++ b/configure.py @@ -512,6 +512,7 @@ def find_ninja(): 'test/boost/range_tombstone_list_test', 'test/boost/rate_limiter_test', 'test/boost/recent_entries_map_test', + 'test/boost/reservoir_sampling_test', 'test/boost/result_utils_test', 'test/boost/reusable_buffer_test', 'test/boost/rust_test', @@ -1343,6 +1344,7 @@ def find_ninja(): 'test/boost/observable_test', 'test/boost/wrapping_interval_test', 'test/boost/range_tombstone_list_test', + 'test/boost/reservoir_sampling_test', 'test/boost/serialization_test', 'test/boost/small_vector_test', 'test/boost/top_k_test', diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index d03903d55f5e..eba64b9a59a5 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -179,6 +179,8 @@ add_scylla_test(result_utils_test KIND SEASTAR) add_scylla_test(reusable_buffer_test KIND SEASTAR) +add_scylla_test(reservoir_sampling_test + KIND BOOST) add_scylla_test(rust_test KIND BOOST LIBRARIES inc) diff --git a/test/boost/reservoir_sampling_test.cc b/test/boost/reservoir_sampling_test.cc new file mode 100644 index 000000000000..fcc207ddc4a8 --- /dev/null +++ b/test/boost/reservoir_sampling_test.cc @@ -0,0 +1,171 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#define BOOST_TEST_MODULE core + +#include +#include +#include +#include +#include +#include +#include "utils/bit_cast.hh" +#include "utils/reservoir_sampling.hh" + +BOOST_AUTO_TEST_CASE(test_reservoir_sampling_probability) { + // We sample K elements from the sequence 1..N. + // We do this REPEATS times, and check that the number of times + // we observed each sample is within expectations. + + constexpr int REPEATS = 100000; + constexpr int K = 3; + constexpr int N = 5; + // There are 10 possible 3-element subsets of a 5-element set. + // Thus the expected count of each sample is REPEATS / 10 = 10000. + // The probability that the actual count falls outside the below + // bounds should be smaller than (1 - 1e-23). + constexpr int MIN_EXPECTED_COUNT = 9000; + constexpr int MAX_EXPECTED_COUNT = 11000; + + using sample = std::array; + auto results = std::unordered_map>(); + + for (int repeat = 0; repeat < REPEATS; ++repeat) { + sample storage{}; + auto rs = utils::reservoir_sampler(storage.size(), repeat); + // Sample K elements from the sequence 1..N + for (int i = rs.next_replace(); i < N; i = rs.next_replace()) { + storage.at(rs.replace()) = i; + } + // Increment the count for this sample. + std::ranges::sort(storage); + results.insert({storage, 0}).first->second += 1; + } + + // The code below iterates over all possible samples. + sample wksp; + auto first = wksp.begin(); + auto last = wksp.end(); + // Fill wksp with first possible sample. + std::iota(first, last, 0); + size_t n_samples = 0; + while (true) { + n_samples += 1; + const auto& sample_count = results.insert({wksp, 0}).first->second; + // These comparisons should be almost impossible to fail. + BOOST_REQUIRE_GE(sample_count, MIN_EXPECTED_COUNT); + BOOST_REQUIRE_LE(sample_count, MAX_EXPECTED_COUNT); + // Advance wksp to next possible sample. + auto mt = last; + --mt; + while (mt > first && *mt == N-(last-mt)) { + --mt; + } + if (mt == first && *mt == N-(last-mt)) { + break; + } + ++(*mt); + while (++mt != last) { + *mt = *(mt-1) + 1; + } + } + // Check that no invalid samples were generated. + BOOST_REQUIRE_EQUAL(n_samples, results.size()); +} + +BOOST_AUTO_TEST_CASE(test_reservoir_sampling_zero_size) { + // Special case with sample size of 0. + auto rs = utils::reservoir_sampler(0, std::random_device()()); + BOOST_REQUIRE_EQUAL(rs.next_replace(), -1); +} + +BOOST_AUTO_TEST_CASE(test_page_sampling_probability) { + // We sample K pages from the sequence 1..N. + // We do this REPEATS times, and check that the number of times + // we observed each page is within expectations. + + constexpr int REPEATS = 100000; + constexpr int K = 3; + constexpr int N = 5; + // There are 10 possible 3-element subsets of a 5-element set. + // Thus the expected count of each sample is REPEATS / 10 = 10000. + // The probability that the actual count falls outside the below + // bounds should be smaller than (1 - 1e-23). + constexpr int MIN_EXPECTED_COUNT = 9000; + constexpr int MAX_EXPECTED_COUNT = 11000; + + using sample = std::array; + auto results = std::unordered_map>(); + + constexpr size_t PAGE_SIZE = sizeof(int); + constexpr size_t BLOCK_SIZE = 3; + + auto data = std::array{}; + std::iota(data.begin(), data.end(), 0); + + for (int repeat = 0; repeat < REPEATS; ++repeat) { + sample storage{}; + auto ps = utils::page_sampler(PAGE_SIZE, storage.size(), repeat); + + // Sample K elements from the sequence 1..N + auto stream = std::as_bytes(std::span(data)).subspan(0); + while (stream.size()) { + auto block = stream.first(std::min(BLOCK_SIZE, stream.size())); + stream = stream.subspan(block.size()); + while (block.size()) { + if (auto cmd = ps.ingest_some(block)) { + BOOST_REQUIRE_EQUAL(cmd->data.size(), PAGE_SIZE); + storage.at(cmd->slot) = read_unaligned(cmd->data.data()); + } + } + } + // Increment the count for this sample. + std::ranges::sort(storage); + results.insert({storage, 0}).first->second += 1; + } + + // The code below iterates over all possible samples. + sample wksp; + auto first = wksp.begin(); + auto last = wksp.end(); + // Fill wksp with first possible sample. + std::iota(first, last, 0); + size_t n_samples = 0; + while (true) { + n_samples += 1; + const auto& sample_count = results.insert({wksp, 0}).first->second; + // These comparisons should be almost impossible to fail. + BOOST_REQUIRE_GE(sample_count, MIN_EXPECTED_COUNT); + BOOST_REQUIRE_LE(sample_count, MAX_EXPECTED_COUNT); + // Advance wksp to next possible sample. + auto mt = last; + --mt; + while (mt > first && *mt == N-(last-mt)) { + --mt; + } + if (mt == first && *mt == N-(last-mt)) { + break; + } + ++(*mt); + while (++mt != last) { + *mt = *(mt-1) + 1; + } + } + // Check that no invalid samples were generated. + BOOST_REQUIRE_EQUAL(n_samples, results.size()); +} + +BOOST_AUTO_TEST_CASE(test_page_sampling_zero_size) { + // Special case with sample size of 0. + auto ps = utils::page_sampler(4, 0, std::random_device()()); + auto data = std::array(); + auto stream = std::as_bytes(std::span(data)).subspan(0); + while (stream.size()) { + BOOST_REQUIRE(!ps.ingest_some(stream)); + } +} diff --git a/utils/reservoir_sampling.hh b/utils/reservoir_sampling.hh new file mode 100644 index 000000000000..84da17d767c7 --- /dev/null +++ b/utils/reservoir_sampling.hh @@ -0,0 +1,198 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace utils { + +// Selects a random sample of a given size from a stream, +// with uniform probability. +// +// For example: in the below usage, at the beginning of every loop, +// `storage` contains a uniformly random sample (of size `min(sample_size, i)`) +// of the values observed so far. +// +// ``` +// value_type generate_value() {...} +// +// const int sample_size = 10; +// auto rs = utils::reservoir_sampler(sample_size); +// std::vector storage; +// storage.reserve(sample_size); +// +// for (size_t i = 0; true; ++i) { +// auto value = generate_value(); +// // rs.next_replace() is the index (in the stream) of the next element +// // selected for the sample. +// if (i == rs.next_replace()) { +// // rs.replace() advances next_replace() and returns the slot +// // (in the storage) of the replaced element. +// uint64_t idx = rs.replace(); +// if (idx == storage.size()) { +// storage.push(std::move(value)); +// } else { +// storage[idx] = std::move(value); +// } +// } +// } +// ``` +class reservoir_sampler { + // The index of the next element picked for the sample. + uint64_t _next = 0; + // The max capacity of the sample. + uint64_t _size; + // Conceptually: + // Every element in the stream is associated with a random number in range [0;1]. + // The ones with the lowest numbers are the ones picked for the sample. + // _w is the greatest random number among the ones currently in the sample. + double _w = 0; + // The random number generator. + // Perhaps it should be passed through the constructor, + // but currently we only pass the seed to avoid a template. + std::default_random_engine _rng; + + // Random double in [0;1]. + double random() { + return std::uniform_real_distribution(0, 1)(_rng); + } +public: + reservoir_sampler(uint64_t sample_size, uint64_t random_seed) + : _size(sample_size) + , _rng(random_seed) + { + // We handle the special case of an empty sample + // by advancing the selection to infinity. + if (_size == 0) { + _next = std::numeric_limits::max(); + } + } + // The index of the next element which has been selected into the sample. + uint64_t next_replace() const noexcept { + return _next; + } + // Returns the slot (in the sample) which should be overwritten by the selected + // element, and advances the selection. + // + // Mustn't be called if sample_size was 0. That would be nonsensical. + uint64_t replace() { + assert(_size != 0); + // The algorithm used below is "Algorithm L" + // from "Reservoir-sampling algorithms of time complexity O(n(1 + log(N/n)))", Kim-Hung Li 1994 + if (_next < _size) { + auto retval = _next++; + if (_next == _size) { + _w = std::exp(std::log(random()) / _size); + _next += std::log(random())/std::log(1-_w); + } + return retval; + } + _w *= std::exp(std::log(random()) / _size); + _next += 1 + std::log(random())/std::log(1-_w); + uint64_t replaced = std::uniform_int_distribution(0, _size - 1)(_rng); + return replaced; + } +}; + +// Splits a stream of bytes into pages, +// and selects a random sample of them. +// +// For example: in the below usage, at the beginning of every loop, +// `storage` contains a uniformly random sample of the pages +// ingested so far. +// +// ``` +// std::span generate_some_bytes() {...} +// +// const int pages_in_sample = 10; +// const int bytes_in_page = 4096; +// +// auto ps = utils::page_sampler(bytes_in_page, pages_in_sample); +// +// std::vector storage; +// storage.reserve(pages_in_sample * bytes_in_page); +// +// while (true) { +// auto value = generate_some_bytes(); +// while (value.size()) { +// if (auto cmd = ps.ingest_some(value)) { +// auto pos = cmd->slot * bytes_in_page; +// if (pos >= storage.size()) { +// storage.resize(pos + bytes_in_page); +// } +// memcpy(&storage[pos], cmd->data.data(), cmd->data.size()); +// } +// } +// } +// ``` +class page_sampler { + // Contents of the next sampled page. + std::vector _page; + // Index in the stream of the next sampled page. + uint64_t _page_idx = -1; + // How many bytes we have to copy until we finish the next sampled page. + uint64_t _bytes_to_copy = 0; + // How many bytes we have to skip until we start the next sampled page. + uint64_t _bytes_to_skip = 0; + // Chooses the pages to be sampled. + reservoir_sampler _rs; + + void move_to_next_page() { + auto new_idx = _rs.next_replace(); + _bytes_to_skip = (new_idx - _page_idx - 1) * _page.size(); + _bytes_to_copy = _page.size(); + _page_idx = new_idx; + } +public: + page_sampler(uint64_t page_size, uint64_t pages_in_sample, uint64_t random_seed) + : _page(page_size) + , _rs(pages_in_sample, random_seed) + { + move_to_next_page(); + } + + struct replace_cmd { + // This slot in the sample which should be overwritten with `data`. + uint64_t slot; + // Contents of the page selected into the sample. + // Invalidated by the next call to ingest_some() or the destruction of the page_sampler (obviously). + std::span data; + }; + // Processes some bytes from span `x` (and advances the front of `x` beyond + // the processed bytes). + // If this completes the next page to be added to the sample, this page is returned. + std::optional ingest_some(std::span& x) { + if (_bytes_to_skip) [[likely]] { + auto n = std::min(_bytes_to_skip, x.size()); + _bytes_to_skip -= n; + x = x.subspan(n); + return std::nullopt; + } else { + auto n = std::min(_bytes_to_copy, x.size()); + std::memcpy(_page.data() + (_page.size() - _bytes_to_copy), x.data(), n); + _bytes_to_copy -= n; + x = x.subspan(n); + if (_bytes_to_copy == 0) { + uint64_t replaced_slot = _rs.replace(); + move_to_next_page(); + return replace_cmd{replaced_slot, _page}; + } else { + return std::nullopt; + } + } + } +}; + +} // namespace utils From 5294762ac756e7926a61fa3f3211b300f2b392a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 4 Dec 2024 17:37:05 +0100 Subject: [PATCH 074/397] utils: add dict_trainer --- configure.py | 2 + test/boost/CMakeLists.txt | 2 + test/boost/dict_trainer_test.cc | 174 +++++++++++++++++++++++++++ utils/CMakeLists.txt | 1 + utils/dict_trainer.cc | 201 ++++++++++++++++++++++++++++++++ utils/dict_trainer.hh | 123 +++++++++++++++++++ 6 files changed, 503 insertions(+) create mode 100644 test/boost/dict_trainer_test.cc create mode 100644 utils/dict_trainer.cc create mode 100644 utils/dict_trainer.hh diff --git a/configure.py b/configure.py index c9760d8dbbc9..e918249b0810 100755 --- a/configure.py +++ b/configure.py @@ -462,6 +462,7 @@ def find_ninja(): 'test/boost/counter_test', 'test/boost/cql_auth_syntax_test', 'test/boost/crc_test', + 'test/boost/dict_trainer_test', 'test/boost/dirty_memory_manager_test', 'test/boost/double_decker_test', 'test/boost/duration_test', @@ -770,6 +771,7 @@ def find_ninja(): 'utils/buffer_input_stream.cc', 'utils/limiting_data_source.cc', 'utils/updateable_value.cc', + 'utils/dict_trainer.cc', 'utils/directories.cc', 'gms/generation-number.cc', 'utils/rjson.cc', diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index eba64b9a59a5..4c866d7631b0 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -67,6 +67,8 @@ add_scylla_test(cql_auth_syntax_test LIBRARIES cql3) add_scylla_test(crc_test KIND BOOST) +add_scylla_test(dict_trainer_test + KIND SEASTAR) add_scylla_test(dirty_memory_manager_test KIND SEASTAR) add_scylla_test(double_decker_test diff --git a/test/boost/dict_trainer_test.cc b/test/boost/dict_trainer_test.cc new file mode 100644 index 000000000000..ce1e0d8b92a1 --- /dev/null +++ b/test/boost/dict_trainer_test.cc @@ -0,0 +1,174 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include +#include "test/lib/scylla_test_case.hh" +#include "utils/dict_trainer.hh" +#include "test/lib/random_utils.hh" +#define ZSTD_STATIC_LINKING_ONLY +#include + +using namespace seastar; +using namespace std::chrono_literals; + +static auto bytes_to_page(bytes_view b) { + auto view = std::span(reinterpret_cast(b.data()), b.size()); + return utils::dict_sampler::page_type(view.begin(), view.end()); +} + +// 1. Compute several random page. Each page is incompressible by itself. +// 2. Feed a stream with many copies of each page to the sampler. +// 3. Train a dictionary on the collected sample. +// 4. Check that a message composed of the chosen pages compresses very well. +SEASTAR_THREAD_TEST_CASE(test_zdict_train) { + // Compute some random pages and concatenate them to form the test message. + int pagesize = 1024; + int n_unique_pages = 64; + auto unique_pages = std::vector(); + auto message = std::vector(); + for (int i = 0; i < n_unique_pages; ++i) { + unique_pages.push_back(bytes_to_page(tests::random::get_bytes(pagesize))); + message.insert(message.end(), unique_pages.back().begin(), unique_pages.back().end()); + } + + // Feed the pages to the sampler, many times. + utils::dict_sampler dt; + seastar::abort_source as; + auto fut = dt.sample(utils::dict_sampler::request{ + .min_sampling_duration = seastar::sleep(1s), + .min_sampling_bytes = 0, + .page_size = 1000, + .sample_size = 1 * 1024 * 1024, + }, as); + for (const auto& up : unique_pages) { + for (int k = 0; k < 1024; ++k) { + auto p = up; + memcpy(p.data(), &k, sizeof(k)); + dt.ingest(p); + } + } + seastar::manual_clock::advance(2s); + auto pages = fut.get(); + + // Train on the sample. + auto dict = utils::zdict_train(pages, {.max_dict_size=128000}); + + // A reasonable dict should contain the repetitive pages verbatim. + BOOST_CHECK_GT(dict.size(), n_unique_pages * pagesize); + // A zstd dictionary should start with ZSTD_MAGIC_DICTIONARY magic. + BOOST_REQUIRE_EQUAL(ZSTD_MAGIC_DICTIONARY, seastar::read_le((char*)dict.data())); + + // We passed multiple copies of the same sample to training. + // A reasonable dictionary based on that will allow us to compress yet another copy of this sample + // (otherwise uncompressible) perfectly. + auto cdict = std::unique_ptr(ZSTD_createCDict(dict.data(), dict.size(), 3), ZSTD_freeCDict); + auto cctx = std::unique_ptr(ZSTD_createCCtx(), ZSTD_freeCCtx); + std::vector buf1(message.size() * 2); + std::vector buf2(message.size()); + size_t compressed_size = ZSTD_compress_usingCDict(cctx.get(), buf1.data(), buf1.size(), message.data(), message.size(), cdict.get()); + BOOST_REQUIRE(!ZSTD_isError(compressed_size)); + // The compressed size should be very small -- no actual data, only backreferences to the dict. + BOOST_REQUIRE_LT(compressed_size, n_unique_pages * 64); + + // Sanity check. Check that the compressed data decompresses properly. + auto ddict = std::unique_ptr(ZSTD_createDDict(dict.data(), dict.size()), ZSTD_freeDDict); + auto dctx = std::unique_ptr(ZSTD_createDCtx(), ZSTD_freeDCtx); + size_t decompressed_size = ZSTD_decompress_usingDDict(dctx.get(), buf2.data(), buf2.size(), buf1.data(), compressed_size, ddict.get()); + BOOST_REQUIRE_EQUAL(decompressed_size, message.size()); + BOOST_REQUIRE(std::ranges::equal(message, std::span(buf2).first(decompressed_size))); +} + +SEASTAR_THREAD_TEST_CASE(test_zstd_max_dict_size) { + auto pages = std::vector(); + for (size_t i = 0; i < 1024; ++i) { + pages.push_back(bytes_to_page(tests::random::get_bytes(1024))); + } + auto dict = utils::zdict_train(pages, {.max_dict_size = 1024 * 128}); + BOOST_REQUIRE_EQUAL(dict.size(), 1024 * 128); + dict = utils::zdict_train(pages, {.max_dict_size = 1024}); + BOOST_REQUIRE_EQUAL(dict.size(), 1024); +} + +// Check that `fut` is aborted when `as` is triggered. +static void check_future_is_aborted(future<> fut, abort_source& as) { + auto ex = std::make_exception_ptr(std::runtime_error("Cancelling!")); + as.request_abort_ex(ex); + fut.wait(); + BOOST_REQUIRE(fut.failed()); + fut.get_exception(); + // The below should work in the ideal world, but unfortunately + // seastar semaphores ignore the aborting exception and propagate + // their own one... + // BOOST_REQUIRE(fut.get_exception() == ex); +} + +static void test_min_bytes_condition_impl(bool should_abort) { + utils::dict_sampler dt; + seastar::abort_source as; + auto fut = dt.sample(utils::dict_sampler::request{ + .min_sampling_duration = seastar::sleep(1s), + .min_sampling_bytes = 8096, + .page_size = 1024, + .sample_size = 4096, + }, as); + BOOST_REQUIRE(!fut.available()); + + manual_clock::advance(2s); + thread::yield(); + BOOST_REQUIRE(!fut.available()); + + dt.ingest(std::array()); + thread::yield(); + BOOST_REQUIRE(!fut.available()); + + if (should_abort) { + check_future_is_aborted(std::move(fut).discard_result(), as); + return; + } + + dt.ingest(std::array()); + fut.get(); +} + +// Test that min_sampling_bytes is respected, and can be aborted. +SEASTAR_THREAD_TEST_CASE(test_min_bytes_condition) { + test_min_bytes_condition_impl(false); + test_min_bytes_condition_impl(true); +} + +static void test_min_duration_condition_impl(bool should_abort) { + utils::dict_sampler dt; + seastar::abort_source as; + auto fut = dt.sample(utils::dict_sampler::request{ + .min_sampling_duration = seastar::sleep_abortable(1s, as), + .min_sampling_bytes = 8096, + .page_size = 1024, + .sample_size = 4096, + }, as); + + dt.ingest(std::array()); + thread::yield(); + BOOST_REQUIRE(!fut.available()); + + if (should_abort) { + check_future_is_aborted(std::move(fut).discard_result(), as); + return; + } + + manual_clock::advance(2s); + thread::yield(); + fut.get(); +} + +// Test that min_sampling_duration is respected, and can be aborted. +SEASTAR_THREAD_TEST_CASE(test_min_duration_condition) { + test_min_duration_condition_impl(false); + test_min_duration_condition_impl(true); +} diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 38bf87aa5e16..952c9e9a14c9 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -17,6 +17,7 @@ target_sources(utils buffer_input_stream.cc build_id.cc config_file.cc + dict_trainer.cc directories.cc disk-error-handler.cc dynamic_bitset.cc diff --git a/utils/dict_trainer.cc b/utils/dict_trainer.cc new file mode 100644 index 000000000000..c3639eb03e9a --- /dev/null +++ b/utils/dict_trainer.cc @@ -0,0 +1,201 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include +#include +#include "utils/dict_trainer.hh" +#include "utils/alien_worker.hh" +#include "utils/shared_dict.hh" +#include "utils/hashers.hh" +#include + +using namespace seastar; + + +namespace utils { + +seastar::logger dict_trainer_logger("dict_training"); + +future> dict_sampler::sample(request req, abort_source& as) { + auto ensure_reset = defer([this] { + dict_trainer_logger.debug("Sampling finished."); + reset(); + }); + + assert(!_sampling); + assert(req.page_size); + + _storage.reserve(req.sample_size / req.page_size); + _page_sampler = page_sampler(req.page_size, req.sample_size / req.page_size, /* hardcoded random seed */ 0); + _bytes_remaining = req.min_sampling_bytes; + _min_bytes_satisifed.signal(_bytes_remaining == 0); + _sampling = true; + dict_trainer_logger.debug("Sampling until the requested amount of time passes..."); + co_await std::move(req.min_sampling_duration); + dict_trainer_logger.debug("Sampling until sampled data size threshold is met..."); + co_await _min_bytes_satisifed.wait(as); + co_return std::move(_storage); +} + +void dict_sampler::reset() noexcept { + *this = dict_sampler(); +} + +void dict_sampler::ingest(std::span x) { + if (!_sampling) { + return; + } + const size_t sz = x.size(); + while (x.size()) { + if (auto cmd = _page_sampler.ingest_some(x)) { + if (cmd->slot >= _storage.size()) { + _storage.push_back(page_type(cmd->data.begin(), cmd->data.end())); + } else { + _storage[cmd->slot].assign(cmd->data.begin(), cmd->data.end()); + } + } + } + auto bytes_remaining_before = _bytes_remaining; + _bytes_remaining -= std::min(sz, _bytes_remaining); + if (_bytes_remaining == 0 && bytes_remaining_before != 0) { + _min_bytes_satisifed.signal(); + } +} + +dict_sampler::dict_type zdict_train(std::span samples, zdict_train_config cfg) { + auto sample_sizes = std::vector(); + sample_sizes.reserve(samples.size()); + for (const auto& sample : samples) { + sample_sizes.push_back(sample.size()); + } + + auto input = std::vector(); + input.reserve(std::accumulate(sample_sizes.begin(), sample_sizes.end(), 0)); + for (const auto& sample : samples) { + input.insert(input.end(), sample.begin(), sample.end()); + } + + auto ret = dict_sampler::dict_type(cfg.max_dict_size); + auto dictsize = ZDICT_trainFromBuffer(ret.data(), ret.size(), input.data(), sample_sizes.data(), sample_sizes.size()); + if (ZDICT_isError(dictsize)) { + const char* errname = ZDICT_getErrorName(dictsize); + dict_trainer_logger.error("ZDICT_trainFromBuffer: {}", errname); + throw std::runtime_error(fmt::format("ZDICT_trainFromBuffer: {}", errname)); + } + + ret.resize(dictsize); + return ret; +} + +void dict_training_loop::pause() { + dict_trainer_logger.debug("dict_training_loop::pause(), called"); + if (!std::exchange(_paused, true)) { + dict_trainer_logger.debug("dict_training_loop::pause(), pausing"); + _pause.consume(); + _pause_as.request_abort(); + } +} + +void dict_training_loop::unpause() { + dict_trainer_logger.debug("dict_training_loop::unpause(), called"); + if (std::exchange(_paused, false)) { + dict_trainer_logger.debug("dict_training_loop::unpause(), unpausing"); + _pause.signal(); + } +} + +void dict_training_loop::cancel() noexcept { + dict_trainer_logger.debug("dict_training_loop::cancel(), called"); + _cancelled.request_abort(); + _pause_as.request_abort(); +} + +seastar::future<> dict_training_loop::start( + dict_sampler& ds, + std::function(dict_sampler::dict_type)> emit, + utils::updateable_value min_time_seconds, + utils::updateable_value min_bytes, + utils::alien_worker& worker +) { + dict_trainer_logger.debug("dict_training_loop::start(), called"); + std::default_random_engine rng(0); + while (!_cancelled.abort_requested()) { + try { + _pause_as = seastar::abort_source(); + dict_trainer_logger.debug("dict_training_loop: waiting on _pause..."); + auto units = co_await get_units(_pause, 1, _cancelled); + dict_trainer_logger.debug("dict_training_loop: sampling..."); + auto sample = co_await ds.sample({ + .min_sampling_duration = seastar::sleep_abortable(std::chrono::seconds(min_time_seconds), _pause_as), + .min_sampling_bytes = min_bytes, + }, _pause_as); + dict_trainer_logger.debug("dict_training_loop: training..."); + // The order of samples coming from dict_sampler is unspecified. + // In particular, they could have a correlation with time. + // + // But the zdict trainer silently expects samples to be shuffled, + // because of how it does its train-test split. + // + // It shouldn't matter in practice, but can matter in a synthetic test + // with a small amount of training data. + std::shuffle(sample.begin(), sample.end(), rng); + auto dict_data = co_await worker.submit([sample = std::move(sample)] { + return zdict_train(sample, {}); + }); + dict_trainer_logger.debug("dict_training_loop: publishing..."); + co_await emit(dict_data); + dict_trainer_logger.debug("dict_training_loop: published..."); + } catch (...) { + if (_cancelled.abort_requested()) { + dict_trainer_logger.debug("dict_training_loop: cancelled"); + } else if (_paused) { + dict_trainer_logger.debug("dict_training_loop: paused"); + } else { + dict_trainer_logger.error("Failed to train a dictionary: {}.", std::current_exception()); + } + } + } +} + +static sha256_type get_sha256(std::span in) { + sha256_hasher hasher; + hasher.update(reinterpret_cast(in.data()), in.size()); + auto b = hasher.finalize(); + auto out = sha256_type(); + assert(b.size() == out.size()); + std::memcpy(&out, b.data(), b.size()); + return out; +} + +shared_dict::shared_dict(std::span d, uint64_t timestamp, UUID origin_node, int zstd_compression_level) + : id{ + .timestamp = timestamp, + .origin_node = origin_node, + .content_sha256 = get_sha256(d) + } + , data(d.begin(), d.end()) + , zstd_ddict(ZSTD_createDDict_byReference(data.data(), data.size()), ZSTD_freeDDict) + , zstd_cdict(ZSTD_createCDict_byReference(data.data(), data.size(), zstd_compression_level), ZSTD_freeCDict) + , lz4_cdict(LZ4_createStream(), LZ4_freeStream) +{ + size_t lz4_dict_size = std::min(data.size(), max_lz4_dict_size); + lz4_ddict = std::span(data).last(lz4_dict_size); + LZ4_loadDict(lz4_cdict.get(), reinterpret_cast(lz4_ddict.data()), lz4_ddict.size()); + // Note: zstd dictionary builder puts the most valuable (frequent) samples + // at the end of the buffer (to minimize the size of backreference offsets), + // and it puts entropy tables (useless for lz4) at the front. + // + // So for lz4, which can only use dictionaries of size at most 64 kiB + // we should take the last 64 kiB. + lz4_ddict = std::span(data).last(lz4_dict_size); +} + +} // namespace utils + diff --git a/utils/dict_trainer.hh b/utils/dict_trainer.hh new file mode 100644 index 000000000000..6e6e15adec71 --- /dev/null +++ b/utils/dict_trainer.hh @@ -0,0 +1,123 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "seastar/core/sharded.hh" +#include "utils/enum_option.hh" +#include "utils/reservoir_sampling.hh" +#include "utils/updateable_value.hh" +#include +#include +#include + +namespace utils { + +extern seastar::logger dict_trainer_logger; + +class alien_worker; + +// A utility for training static compression dictionaries. +// +// It's a combination of a reservoir sampler (utils::page_sampler) and a 3rd party +// training library (zstd). +class dict_sampler { +public: + using page_type = std::vector; + using dict_type = std::vector; +private: + std::vector _storage; + size_t _bytes_remaining = 0; + seastar::semaphore _min_bytes_satisifed{0}; + utils::page_sampler _page_sampler{0, 0, 0}; + bool _sampling = false; + + void reset() noexcept; +public: + + + // Describes a config for sampling and training. + struct request { + // Sampling will last at least until this future resolves. + seastar::future<> min_sampling_duration = seastar::make_ready_future<>(); + // Sampling will ingest at least this much before training. + size_t min_sampling_bytes = 1024 * 1024 * 1024; + // Shouldn't matter too much. + size_t page_size = 8096; + // A reasonable sample size is ~100x the dict size. + size_t sample_size = 16 * 1024 * 1024; + }; + + // Begin sampling and training. + // After enough time passes and enough data is input via `ingest()`, + // the training library will be called on the gathered sample, and then the + // returned future will resolve to the resulting dict. + // + // If the abort source is triggered before the min_sampling_bytes threshold is met, + // the sampling will be canceled and the returned future will resolve to the aborting exception. + // In reasonable use cases, min_sampling_duration should be abortable withe the same abort source. + seastar::future> sample(request, seastar::abort_source&); + + // When in the sampling phase, this will feed the data to the sampler. + // Otherwise, it's a no-op. + void ingest(std::span x); + + bool is_sampling() { return _sampling; } +}; + +struct zdict_train_config { + // 110 kiB is zstd's recommended default. + size_t max_dict_size = 110 * 1024; +}; +// Outside of tests, this should only be called on non-reactor threads. Calling it on a reactor thread will +// cause giant stalls. +dict_sampler::dict_type zdict_train(std::span samples, zdict_train_config cfg); + +class walltime_compressor_tracker; +class shared_dict; + +class dict_training_loop { + bool _paused = true; + seastar::abort_source _cancelled; + seastar::semaphore _pause{0}; + seastar::abort_source _pause_as; +public: + struct when { + enum class type { + NEVER, + WHEN_LEADER, + ALWAYS, + COUNT, + }; + static constexpr std::string_view names[] = { + "never", + "when_leader", + "always", + }; + static_assert(std::size(names) == static_cast(type::COUNT)); + // Implements enum_option. + static std::unordered_map map() { + std::unordered_map ret; + for (size_t i = 0; i < std::size(names); ++i) { + ret.insert({std::string(names[i]), type(i)}); + } + return ret; + } + }; + void pause(); + void unpause(); + void cancel() noexcept; + seastar::future<> start( + dict_sampler&, + std::function(dict_sampler::dict_type)> emit, + utils::updateable_value min_time_seconds, + utils::updateable_value min_bytes, + utils::alien_worker&); +}; + +} // namespace utils From 0fd10507843bbf2696b86196c6c66008a4b204a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 4 Dec 2024 17:32:22 +0100 Subject: [PATCH 075/397] utils: add advanced_rpc_compressor Adds glue needed to pass lz4 and zstd with streaming and/or dictionaries as the network traffic compressors for Seastar's RPC servers. The main jobs of this glue are: 1. Implementing the API expected by Seastar from RPC compressors. 2. Expose metrics about the effectiveness of the compression. 3. Allow dynamically switching algorithms and dictionaries on a running connection, without any extra waits. The biggest design decision here is that the choice of algorithm and dictionary is negotiated by both sides of the connection, not dictated unilaterally by the sender. The negotiation algorithm is fairly complicated (a TLA+ model validating it is included in the commit). Unilateral compression choice would be much simpler. However, negotiation avoids re-sending the same dictionary over every connection in the cluster after dictionary updates (with one-way communication, it's the only reliable way to ensure that our receiver possesses the dictionary we are about to start using), lets receivers ask for a cheaper compression mode if they want, and lets them refuse to update a dictionary if they don't think they have enough free memory for that. In hindsight, those properties probably weren't worth the extra complexity and extra development effort. Zstd can be quite expensive, so this patch also includes a mechanism which temporarily downgrades the compressor from zstd to lz4 if zstd has been using too much CPU in a given slice of time. But it should be noted that this can't be treated as a reliable "protection" from negative performance effects of zstd, since a downgrade can happen on the sender side, and receivers are at the mercy of senders. --- configure.py | 2 + .../advanced_rpc_compression_negotiation.tla | 150 +++++ test/boost/CMakeLists.txt | 2 + test/boost/advanced_rpc_compressor_test.cc | 291 +++++++++ utils/CMakeLists.txt | 1 + utils/advanced_rpc_compressor.cc | 574 ++++++++++++++++++ utils/advanced_rpc_compressor.hh | 365 +++++++++++ utils/advanced_rpc_compressor_protocol.hh | 47 ++ utils/refcounted.hh | 44 ++ utils/shared_dict.hh | 58 ++ 10 files changed, 1534 insertions(+) create mode 100644 docs/dev/advanced_rpc_compression_negotiation.tla create mode 100644 test/boost/advanced_rpc_compressor_test.cc create mode 100644 utils/advanced_rpc_compressor.cc create mode 100644 utils/advanced_rpc_compressor.hh create mode 100644 utils/advanced_rpc_compressor_protocol.hh create mode 100644 utils/refcounted.hh create mode 100644 utils/shared_dict.hh diff --git a/configure.py b/configure.py index e918249b0810..bee7f5111b07 100755 --- a/configure.py +++ b/configure.py @@ -432,6 +432,7 @@ def find_ninja(): scylla_tests = set([ 'test/boost/combined_tests', 'test/boost/UUID_test', + 'test/boost/advanced_rpc_compressor_test', 'test/boost/allocation_strategy_test', 'test/boost/alternator_unit_test', 'test/boost/anchorless_list_test', @@ -992,6 +993,7 @@ def find_ninja(): 'utils/s3/aws_error.cc', 'utils/s3/client.cc', 'utils/s3/retry_strategy.cc', + 'utils/advanced_rpc_compressor.cc', 'gms/version_generator.cc', 'gms/versioned_value.cc', 'gms/gossiper.cc', diff --git a/docs/dev/advanced_rpc_compression_negotiation.tla b/docs/dev/advanced_rpc_compression_negotiation.tla new file mode 100644 index 000000000000..90b4f89ab9ba --- /dev/null +++ b/docs/dev/advanced_rpc_compression_negotiation.tla @@ -0,0 +1,150 @@ + +---------------------------- MODULE compression ---------------------------- + +EXTENDS Integers, Sequences, TLC + +CONSTANTS Dictionary, MaxNrUpdates + +VARIABLES +senderRecentDict, senderCommittedDict, senderCurrentDict, +senderProtocolEpoch, senderHasUpdate, senderHasCommit, +receiverRecentDict, receiverCommittedDict, receiverCurrentDict, +receiverProtocolEpoch, receiverHasUpdate, receiverHasCommit, +senderToReceiver, receiverToSender, good, +nrUpdates + +receiverVars == <> +senderVars == <> +vars == << + receiverRecentDict, receiverCommittedDict, receiverCurrentDict, receiverHasUpdate, receiverHasCommit, receiverProtocolEpoch, + senderRecentDict, senderCommittedDict, senderCurrentDict, senderHasUpdate, senderHasCommit, senderProtocolEpoch, + senderToReceiver, receiverToSender, good, nrUpdates>> + +Init == \E x \in Dictionary : +/\ senderRecentDict = x +/\ senderCommittedDict = x +/\ senderCurrentDict = x +/\ receiverRecentDict = x +/\ receiverCommittedDict = x +/\ receiverCurrentDict = x +/\ senderHasUpdate = FALSE +/\ receiverHasUpdate = FALSE +/\ senderHasCommit = FALSE +/\ receiverHasCommit = FALSE +/\ senderProtocolEpoch = 0 +/\ receiverProtocolEpoch = 0 +/\ senderToReceiver = <<>> +/\ receiverToSender = <<>> +/\ good = TRUE +/\ nrUpdates = 0 + +AnnounceDictionarySender(dict) == +/\ senderRecentDict' = dict +/\ senderProtocolEpoch' = senderProtocolEpoch + 1 +/\ senderHasUpdate' = TRUE +/\ senderHasCommit' = FALSE +/\ nrUpdates' = nrUpdates + 1 +/\ nrUpdates < MaxNrUpdates +/\ UNCHANGED <> + +AnnounceDictionaryReceiver(dict) == +/\ receiverRecentDict' = dict +/\ receiverHasUpdate' = TRUE +/\ receiverHasCommit' = FALSE +/\ nrUpdates' = nrUpdates + 1 +/\ nrUpdates < MaxNrUpdates +/\ UNCHANGED <> + +receiverDicts == <> +senderDicts == <> + +SenderSend == +\/ /\ senderHasCommit + /\ senderHasCommit' = FALSE + /\ senderCurrentDict' = senderCommittedDict + /\ senderToReceiver' = Append(senderToReceiver, <<"COMMIT", senderCommittedDict, senderProtocolEpoch>>) + /\ UNCHANGED <> +\/ /\ senderHasUpdate + /\ senderHasUpdate' = FALSE + /\ senderCommittedDict' = senderRecentDict + /\ senderToReceiver' = Append(senderToReceiver, <<"UPDATE", senderRecentDict, senderProtocolEpoch>>) + /\ UNCHANGED <> + +ReceiverSend == +\/ /\ receiverHasCommit + /\ receiverHasCommit' = FALSE + /\ receiverToSender' = Append(receiverToSender, <<"COMMIT", receiverCommittedDict, receiverProtocolEpoch>>) + /\ UNCHANGED <> +\/ /\ receiverHasUpdate + /\ receiverHasUpdate' = FALSE + /\ receiverToSender' = Append(receiverToSender, <<"UPDATE", receiverRecentDict, receiverProtocolEpoch>>) + /\ UNCHANGED <> + +Send == +\/ ReceiverSend +\/ SenderSend + +SenderRecv(msg) == +\/ /\ msg[1] = "UPDATE" + /\ senderProtocolEpoch' = senderProtocolEpoch + 1 + /\ senderHasUpdate' = TRUE + /\ senderHasCommit' = FALSE + /\ UNCHANGED <> +\/ /\ msg[1] = "COMMIT" + /\ msg[3] = senderProtocolEpoch + /\ senderCommittedDict' = IF msg[2] = senderCommittedDict THEN senderCommittedDict ELSE senderCurrentDict + /\ senderHasCommit' = TRUE + /\ UNCHANGED <> +\/ /\ msg[1] = "COMMIT" + /\ ~(msg[3] = senderProtocolEpoch) + /\ UNCHANGED <> + +ReceiverRecv(msg) == +\/ /\ msg[1] = "UPDATE" + /\ receiverHasCommit' = TRUE + /\ receiverHasUpdate' = FALSE + /\ receiverCommittedDict' = IF msg[2] = receiverRecentDict THEN receiverRecentDict ELSE receiverCommittedDict + /\ receiverProtocolEpoch' = msg[3] + /\ UNCHANGED <> +\/ /\ msg[1] = "COMMIT" + /\ good' = ((msg[2] = receiverCurrentDict) \/ (msg[2] = receiverCommittedDict)) + /\ receiverCurrentDict' = IF (msg[2] = receiverCommittedDict) THEN receiverCommittedDict ELSE receiverCurrentDict + /\ UNCHANGED <> + +Receive == +\/ /\ Len(senderToReceiver) > 0 + /\ ReceiverRecv(Head(senderToReceiver)) + /\ senderToReceiver' = Tail(senderToReceiver) + /\ UNCHANGED <> +\/ /\ Len(receiverToSender) > 0 + /\ SenderRecv(Head(receiverToSender)) + /\ receiverToSender' = Tail(receiverToSender) + /\ UNCHANGED <> + +Next == +\/ \E dict \in Dictionary : AnnounceDictionarySender(dict) +\/ \E dict \in Dictionary : AnnounceDictionaryReceiver(dict) +\/ Send +\/ Receive + +Spec == Init /\ [][Next]_vars +FairSpec == Init /\ [][Next]_vars /\ WF_vars(Send) /\ WF_vars(Receive) + +AnnouncePossible == nrUpdates < MaxNrUpdates => \A d \in Dictionary: (ENABLED(AnnounceDictionaryReceiver(d)) /\ ENABLED(AnnounceDictionarySender(d))) +Good == good + +Invariants == +/\ ~(senderHasCommit /\ senderHasUpdate) +/\ ~(receiverHasCommit /\ receiverHasUpdate) +/\ Good +/\ AnnouncePossible + +Settles == <>[](senderRecentDict = receiverRecentDict) ~> [](senderCurrentDict = senderRecentDict) + +Properties == +/\ Settles + +============================================================================= +\* Modification History +\* Last modified Wed May 22 17:05:46 CEST 2024 by michal +\* Created Sat May 18 20:01:15 CEST 2024 by michal diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index 4c866d7631b0..c6f9290a7c92 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -1,5 +1,7 @@ add_scylla_test(UUID_test KIND BOOST) +add_scylla_test(advanced_rpc_compressor_test + KIND SEASTAR) add_scylla_test(allocation_strategy_test KIND BOOST) add_scylla_test(alternator_unit_test diff --git a/test/boost/advanced_rpc_compressor_test.cc b/test/boost/advanced_rpc_compressor_test.cc new file mode 100644 index 000000000000..85c34da517e9 --- /dev/null +++ b/test/boost/advanced_rpc_compressor_test.cc @@ -0,0 +1,291 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "seastar/core/manual_clock.hh" +#include "seastar/util/closeable.hh" +#include "test/lib/random_utils.hh" +#include "test/lib/scylla_test_case.hh" +#include "utils/advanced_rpc_compressor.hh" +#include "utils/advanced_rpc_compressor_protocol.hh" + +using namespace seastar; +using namespace std::chrono_literals; + +static utils::dict_ptr make_dict(uint64_t timestamp, std::vector content = {}) { + return make_lw_shared(make_foreign(make_lw_shared(utils::shared_dict(std::move(content), timestamp, {})))); +} + +SEASTAR_THREAD_TEST_CASE(test_control_protocol_sanity) { + condition_variable cv; + auto dict_1 = make_dict(1); + auto dict_2 = make_dict(2); + utils::control_protocol alice(cv); + utils::control_protocol bob(cv); + auto settle = [&] { + bool run = true; + while (run) { + run = false; + if (auto msg = alice.produce_control_header()) { + bob.consume_control_header(*msg); + run = true; + } + if (auto msg = bob.produce_control_header()) { + alice.consume_control_header(*msg); + run = true; + } + } + }; + alice.announce_dict(dict_1); + bob.announce_dict(dict_2); + settle(); + BOOST_REQUIRE(alice.sender_current_dict().id == utils::shared_dict::dict_id()); + BOOST_REQUIRE(bob.sender_current_dict().id == utils::shared_dict::dict_id()); + alice.announce_dict(dict_2); + settle(); + BOOST_REQUIRE(alice.sender_current_dict().id == (**dict_2).id); + BOOST_REQUIRE(bob.sender_current_dict().id == (**dict_2).id); + alice.announce_dict(nullptr); + settle(); + BOOST_REQUIRE(alice.sender_current_dict().id == (**dict_2).id); + BOOST_REQUIRE(bob.sender_current_dict().id == (**dict_2).id); + bob.announce_dict(nullptr); + settle(); + BOOST_REQUIRE(alice.sender_current_dict().id == utils::shared_dict::dict_id()); + BOOST_REQUIRE(bob.sender_current_dict().id == utils::shared_dict::dict_id()); +} + +temporary_buffer bytes_view_to_temporary_buffer(bytes_view bv) { + return temporary_buffer(reinterpret_cast(bv.data()), bv.size()); +} + +template +concept RpcBuf = std::same_as || std::same_as; + +template +bytes rpc_buf_to_bytes(const Buf& data) { + if (auto src = std::get_if>(&data.bufs)) { + return bytes(reinterpret_cast(src->get()), src->size()); + } + auto src = std::get>>(data.bufs).data(); + auto out = bytes(bytes::initialized_later{}, data.size); + size_t i = 0; + while (i < data.size) { + std::memcpy(&out[i], src->get(), src->size()); + i += src->size(); + ++src; + } + return out; +} + +template +Buf convert_rpc_buf(BufFrom data) { + Buf b; + b.size = data.size; + b.bufs = std::move(data.bufs); + return b; +} + +class tracker_without_clock final : public utils::advanced_rpc_compressor::tracker { + virtual uint64_t get_steady_nanos() const override { + return 0; + } +public: + using tracker::tracker; +}; + +SEASTAR_THREAD_TEST_CASE(test_tracker_basic_sanity) { + for (const bool checksumming : {false, true}) + for (const auto& zstd_cpu_limit : {0.0, 1.0}) { + auto cfg = utils::advanced_rpc_compressor::tracker::config{ + .zstd_quota_fraction{zstd_cpu_limit}, + .algo_config = utils::updateable_value{ + {utils::compression_algorithm::type::ZSTD, utils::compression_algorithm::type::LZ4}, + }, + .checksumming = utils::updateable_value{checksumming}, + }; + tracker_without_clock tracker{cfg}; + auto feature_string = tracker.supported(); + auto server_compressor = tracker.negotiate(feature_string, true, [] { return make_ready_future<>(); }); + auto close_server_compressor = deferred_close(*server_compressor); + auto client_compressor = tracker.negotiate(server_compressor->name(), false, [] { return make_ready_future<>(); }); + auto close_client_compressor = deferred_close(*client_compressor); + + for (const auto [a, b] : { + std::make_pair(std::ref(server_compressor), std::ref(client_compressor)), + std::make_pair(std::ref(client_compressor), std::ref(server_compressor)), + }) + for (int repeat = 0; repeat < 10; ++repeat) { + auto message = tests::random::get_bytes(100000) + bytes(size_t(100000), bytes::value_type(0)); + constexpr int head_space = 4; + auto compressed = a->compress(head_space, rpc::snd_buf{bytes_view_to_temporary_buffer(message)}); + + compressed.front().trim_front(head_space); + compressed.size -= head_space; + + // Mess with the header deserializer by prepending an empty fragment to `compressed`. + if (auto src = std::get_if>(&compressed.bufs)) { + auto vec = std::vector>(); + vec.push_back(std::move(*src)); + compressed.bufs = std::move(vec); + } + auto &vec = std::get>>(compressed.bufs); + vec.insert(vec.begin(), temporary_buffer()); + + auto decompressed = b->decompress(convert_rpc_buf(std::move(compressed))); + BOOST_REQUIRE_EQUAL(message, rpc_buf_to_bytes(decompressed)); + } + } +} + +SEASTAR_THREAD_TEST_CASE(test_tracker_dict_sanity) { + for (const auto& algo : {utils::compression_algorithm::type::ZSTD, utils::compression_algorithm::type::LZ4}) { + auto cfg = utils::advanced_rpc_compressor::tracker::config{ + .zstd_quota_fraction{1.0}, + .algo_config = utils::updateable_value{ + {utils::compression_algorithm::type::RAW}, + }, + }; + tracker_without_clock tracker{cfg}; + auto feature_string = tracker.supported(); + auto server_compressor = tracker.negotiate(feature_string, true, [] { return make_ready_future<>(); }); + auto close_server_compressor = deferred_close(*server_compressor); + auto client_compressor = tracker.negotiate(server_compressor->name(), false, [] { return make_ready_future<>(); }); + auto close_client_compressor = deferred_close(*client_compressor); + + auto message = tests::random::get_bytes(8192); + auto message_view = std::span(reinterpret_cast(message.data()), message.size()); + // We will send messages which perfectly match the dict. + // If dict negotiation succeeds as expected, this should result in very small messages. + auto dict = make_dict(1, {message_view.begin(), message_view.end()}); + tracker.announce_dict(dict); + tracker.set_supported_algos(utils::compression_algorithm_set::singleton(algo)); + + // Arbitrary number of repeats. + for (int repeat = 0; repeat < 10; ++repeat) + for (const auto [a, b] : { + std::make_pair(std::ref(server_compressor), std::ref(client_compressor)), + std::make_pair(std::ref(client_compressor), std::ref(server_compressor)), + }) { + constexpr int head_space = 4; + auto compressed = a->compress(head_space, rpc::snd_buf{bytes_view_to_temporary_buffer(message)}); + // The dict negotiation should have settled after a few repeats. + if (repeat >= 5) { + // `100` here is an arbitrary "small size". + BOOST_REQUIRE_LE(compressed.size, 100); + } + compressed.front().trim_front(head_space); + compressed.size -= head_space; + auto decompressed = b->decompress(convert_rpc_buf(std::move(compressed))); + BOOST_REQUIRE_EQUAL(message, rpc_buf_to_bytes(decompressed)); + } + } +} + +SEASTAR_THREAD_TEST_CASE(test_tracker_cpu_limit_shortterm) { + constexpr int quota = 10; + constexpr int quota_refresh = 128; + auto cfg = utils::advanced_rpc_compressor::tracker::config{ + .zstd_quota_fraction{float(quota) / quota_refresh}, + .zstd_quota_refresh_ms{quota_refresh}, + .algo_config = utils::updateable_value{ + {utils::compression_algorithm::type::ZSTD, utils::compression_algorithm::type::LZ4}, + }, + }; + + struct manual_clock_tracker : utils::advanced_rpc_compressor::tracker_with_clock { + using tracker_with_clock::tracker_with_clock; + virtual uint64_t get_steady_nanos() const override { + manual_clock::advance(std::chrono::milliseconds(1)); + return std::chrono::nanoseconds(manual_clock::now().time_since_epoch()).count(); + } + }; + manual_clock_tracker tracker{cfg}; + + auto feature_string = tracker.supported(); + auto server_compressor = tracker.negotiate(feature_string, true, [] { return make_ready_future<>(); }); + auto close_server_compressor = deferred_close(*server_compressor); + auto client_compressor = tracker.negotiate(server_compressor->name(), false, [] { return make_ready_future<>(); }); + auto close_client_compressor = deferred_close(*client_compressor); + + // Settle negotiations + for (int i = 0; i < 3; ++i) { + auto msg = server_compressor->compress(0, rpc::snd_buf{0}); + client_compressor->decompress(convert_rpc_buf(std::move(msg))); + msg = client_compressor->compress(0, rpc::snd_buf{0}); + server_compressor->decompress(convert_rpc_buf(std::move(msg))); + } + // Refresh quotas. + manual_clock::advance(std::chrono::milliseconds(1000)); + + for (int repeat = 0; repeat < 5; ++repeat) { + auto before = tracker.get_stats()[utils::compression_algorithm(utils::compression_algorithm::type::ZSTD).idx()]; + // Do many compressions. + for (int i = 0; i < 30; ++i) { + client_compressor->compress(0, rpc::snd_buf{0}); + } + // Check that the quota is respected. + auto after = tracker.get_stats()[utils::compression_algorithm(utils::compression_algorithm::type::ZSTD).idx()]; + BOOST_REQUIRE_EQUAL(std::chrono::nanoseconds(after.compression_cpu_nanos - before.compression_cpu_nanos), std::chrono::milliseconds(quota)); + // Refresh quotas. + manual_clock::advance(std::chrono::milliseconds(1000)); + } +} + +SEASTAR_THREAD_TEST_CASE(test_tracker_cpu_limit_longterm) { + constexpr static auto step = std::chrono::milliseconds(10); + constexpr static auto limit = 0.1; + auto cfg = utils::advanced_rpc_compressor::tracker::config{ + .zstd_quota_fraction{1}, + .zstd_quota_refresh_ms{1}, + .zstd_longterm_quota_fraction{limit}, + .zstd_longterm_quota_refresh_ms{1000}, + .algo_config = utils::updateable_value{ + {utils::compression_algorithm::type::ZSTD, utils::compression_algorithm::type::LZ4}, + }, + }; + + struct manual_clock_tracker : utils::advanced_rpc_compressor::tracker_with_clock { + using tracker_with_clock::tracker_with_clock; + virtual uint64_t get_steady_nanos() const override { + manual_clock::advance(step); + return std::chrono::nanoseconds(manual_clock::now().time_since_epoch()).count(); + } + }; + manual_clock_tracker tracker{cfg}; + + auto feature_string = tracker.supported(); + auto server_compressor = tracker.negotiate(feature_string, true, [] { return make_ready_future<>(); }); + auto close_server_compressor = deferred_close(*server_compressor); + auto client_compressor = tracker.negotiate(server_compressor->name(), false, [] { return make_ready_future<>(); }); + auto close_client_compressor = deferred_close(*client_compressor); + + // Settle negotiations + for (int i = 0; i < 3; ++i) { + auto msg = server_compressor->compress(0, rpc::snd_buf{0}); + client_compressor->decompress(convert_rpc_buf(std::move(msg))); + msg = client_compressor->compress(0, rpc::snd_buf{0}); + server_compressor->decompress(convert_rpc_buf(std::move(msg))); + } + + constexpr int n_compressions = 1000; + auto used_before = tracker.get_stats()[utils::compression_algorithm(utils::compression_algorithm::type::ZSTD).idx()]; + auto clock_before = manual_clock::now(); + // Do many compressions. + for (int i = 0; i < n_compressions; ++i) { + client_compressor->compress(0, rpc::snd_buf{0}); + } + // Check that the quota is respected. + auto used_after = tracker.get_stats()[utils::compression_algorithm(utils::compression_algorithm::type::ZSTD).idx()]; + auto clock_after = manual_clock::now(); + + auto used = std::chrono::nanoseconds(used_after.compression_cpu_nanos - used_before.compression_cpu_nanos); + auto elapsed = clock_after - clock_before; + + BOOST_REQUIRE_GE(used, elapsed * limit * 0.9); + BOOST_REQUIRE_LE(used, elapsed * limit * 1.1); +} diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 952c9e9a14c9..77caf61615c7 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -5,6 +5,7 @@ add_library(utils STATIC) target_sources(utils PRIVATE UUID_gen.cc + advanced_rpc_compressor.cc alien_worker.cc arch/powerpc/crc32-vpmsum/crc32_wrapper.cc arch/powerpc/crc32-vpmsum/crc32.S diff --git a/utils/advanced_rpc_compressor.cc b/utils/advanced_rpc_compressor.cc new file mode 100644 index 000000000000..8c88239cecc7 --- /dev/null +++ b/utils/advanced_rpc_compressor.cc @@ -0,0 +1,574 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include +#include "log.hh" +#include "utils/advanced_rpc_compressor.hh" +#include "utils/advanced_rpc_compressor_protocol.hh" +#include "stream_compressor.hh" +#include "utils/dict_trainer.hh" +#include "seastar/core/on_internal_error.hh" + +namespace utils { + +logging::logger arc_logger("advanced_rpc_compressor"); + +static const shared_dict null_dict; + +control_protocol::control_protocol(condition_variable& cv) + : _needs_progress(cv) +{ +} + +compression_algorithm control_protocol::sender_current_algorithm() const noexcept { + return _sender_current_algo; +} + +const shared_dict& control_protocol::sender_current_dict() const noexcept { + return _sender_current_dict ? **_sender_current_dict : null_dict; +} + +const shared_dict& control_protocol::receiver_current_dict() const noexcept { + return _receiver_current_dict ? **_receiver_current_dict : null_dict; +} + +static shared_dict::dict_id get_dict_id(dict_ptr d) { + return d ? (**d).id : null_dict.id; +} + +void control_protocol_frame::one_side::serialize(std::span out_span) { + char* out = reinterpret_cast(out_span.data()); + seastar::write_le(&out[0], header); + seastar::write_le(&out[1], epoch); + seastar::write_le(&out[9], algo.value()); + seastar::write_le(&out[10], dict.origin_node.get_least_significant_bits()); + seastar::write_le(&out[18], dict.origin_node.get_most_significant_bits()); + seastar::write_le(&out[26], dict.timestamp); + std::memcpy(&out[34], dict.content_sha256.data(), dict.content_sha256.size()); + static_assert(serialized_size == 66); +} + +control_protocol_frame::one_side control_protocol_frame::one_side::deserialize(std::span in_span) { + const char* in = reinterpret_cast(in_span.data()); + control_protocol_frame::one_side ret; + ret.header = static_cast(seastar::read_le(&in[0])); + ret.epoch = seastar::read_le(&in[1]); + ret.algo = compression_algorithm_set::from_value(seastar::read_le(&in[9])); + ret.dict.origin_node = UUID(seastar::read_le(&in[18]), seastar::read_le(&in[10])); + ret.dict.timestamp = seastar::read_le(&in[26]); + std::memcpy(ret.dict.content_sha256.data(), &in[34], 32); + static_assert(serialized_size == 66); + return ret; +} + +void control_protocol_frame::serialize(std::span out) { + sender.serialize(out.subspan<0, one_side::serialized_size>()); + receiver.serialize(out.subspan()); +}; + +control_protocol_frame control_protocol_frame::deserialize(std::span in) { + control_protocol_frame pf; + pf.sender = one_side::deserialize(in.subspan<0, one_side::serialized_size>()); + pf.receiver = one_side::deserialize(in.subspan()); + return pf; +} + +void control_protocol::announce_dict(dict_ptr d) noexcept { + _sender_recent_dict = d; + _sender_protocol_epoch += 1; + _sender_has_update = true; + _sender_has_commit = false; + _receiver_recent_dict = d; + _receiver_has_update = true; + _receiver_has_commit = false; + _needs_progress.signal(); +} +void control_protocol::set_supported_algos(compression_algorithm_set algos) noexcept { + _algos = algos; + _sender_protocol_epoch += 1; + _sender_has_update = true; + _sender_has_commit = false; + _receiver_has_update = true; + _needs_progress.signal(); +} + +void control_protocol::consume_control_header(control_protocol_frame cpf) { + if (cpf.receiver.header == control_protocol_frame::UPDATE) { + _sender_protocol_epoch += 1; + _sender_has_update = true; + _sender_has_commit = false; + _needs_progress.signal(); + } else if (cpf.receiver.header == control_protocol_frame::COMMIT && cpf.receiver.epoch == _sender_protocol_epoch) { + _sender_has_commit = true; + assert(!_sender_has_update); + if (get_dict_id(_sender_committed_dict) != cpf.receiver.dict) { + _sender_committed_dict = _sender_current_dict; + } + _sender_committed_algo = cpf.receiver.algo.intersection(_algos).heaviest(); + _needs_progress.signal(); + } + if (cpf.sender.header == control_protocol_frame::UPDATE) { + _receiver_has_commit = true; + _receiver_has_update = false; + if (cpf.sender.dict == get_dict_id(_receiver_recent_dict)) { + _receiver_committed_dict = _receiver_recent_dict; + } + _receiver_protocol_epoch = cpf.sender.epoch; + _needs_progress.signal(); + } else if (cpf.sender.header == control_protocol_frame::COMMIT) { + if (cpf.sender.dict == get_dict_id(_receiver_committed_dict)) { + _receiver_current_dict = _receiver_committed_dict; + } else { + assert(cpf.sender.dict == get_dict_id(_receiver_current_dict)); + } + } +} + +std::optional control_protocol::produce_control_header() { + control_protocol_frame pf; + if (!(_sender_has_commit || _sender_has_update || _receiver_has_commit || _receiver_has_update)) [[likely]] { + return std::nullopt; + } + if (_sender_has_commit) { + _sender_has_commit = false; + assert(!_sender_has_update); + _sender_current_dict = _sender_committed_dict; + _sender_current_algo = _sender_committed_algo; + pf.sender.header = control_protocol_frame::COMMIT; + pf.sender.dict = get_dict_id(_sender_current_dict); + pf.sender.algo = compression_algorithm_set::singleton(_sender_current_algo); + pf.sender.epoch = _sender_protocol_epoch; + } else if (_sender_has_update) { + _sender_has_update = false; + _sender_committed_dict = _sender_recent_dict; + pf.sender.header = control_protocol_frame::UPDATE; + pf.sender.dict = get_dict_id(_sender_recent_dict); + pf.sender.algo = compression_algorithm_set::singleton(_sender_current_algo); + pf.sender.epoch = _sender_protocol_epoch; + } + if (_receiver_has_commit) { + _receiver_has_commit = false; + pf.receiver.header = control_protocol_frame::COMMIT; + pf.receiver.dict = get_dict_id(_receiver_committed_dict); + pf.receiver.algo = _algos; + pf.receiver.epoch = _receiver_protocol_epoch; + } else if (_receiver_has_update) { + _receiver_has_update = false; + pf.receiver.header = control_protocol_frame::UPDATE; + pf.receiver.dict = get_dict_id(_receiver_recent_dict); + pf.receiver.algo = _algos; + pf.receiver.epoch = _receiver_protocol_epoch; + } + return pf; +} + +// Converting the list obtained from config.cc to a more workable form. +compression_algorithm_set algo_list_to_set(std::span> v) { + auto out = compression_algorithm_set::singleton(compression_algorithm::type::RAW); + for (const auto& i : v) { + out = out.sum(compression_algorithm_set::singleton(compression_algorithm(i))); + } + return out; +} + +static raw_stream the_raw_stream; + +advanced_rpc_compressor::advanced_rpc_compressor( + tracker& fac, + std::function()> send_empty_frame) + : _tracker(fac) + , _control(_needs_progress) + , _send_empty_frame(std::move(send_empty_frame)) + , _progress_fiber(start_progress_fiber()) +{ + _idx =_tracker->register_compressor(this); +} + +future<> advanced_rpc_compressor::start_progress_fiber() { + while (true) { + co_await _needs_progress.when(); + co_await _send_empty_frame(); + } +} + +future<> advanced_rpc_compressor::close() noexcept { + _needs_progress.broken(); + return std::move(_progress_fiber).handle_exception([] (const auto& ep) {}); +} + +advanced_rpc_compressor::~advanced_rpc_compressor() { + _tracker->unregister_compressor(_idx); +} + +// Note: whenever a backwards-incompatible change to the compressor protocol/format +// is made, the COMPRESSOR_NAME has to change. +// +const static sstring COMPRESSOR_NAME = "SCYLLA_V3"; + +compression_algorithm advanced_rpc_compressor::get_algo_for_next_msg(size_t msgsize) { + auto algo = _control.sender_current_algorithm(); + if (algo == compression_algorithm::type::ZSTD + && (_tracker->cpu_limit_exceeded() + || msgsize < _tracker->_cfg.zstd_min_msg_size.get() + || msgsize > _tracker->_cfg.zstd_max_msg_size.get()) + ) { + algo = compression_algorithm::type::LZ4; + } + return algo; +} + +sstring advanced_rpc_compressor::name() const { + return COMPRESSOR_NAME; +} + +const sstring& advanced_rpc_compressor::tracker::supported() const { + return COMPRESSOR_NAME; +} + +std::unique_ptr advanced_rpc_compressor::tracker::negotiate( + sstring feature, + bool is_server, + std::function()> send_empty_frame) +{ + if (feature != COMPRESSOR_NAME) { + return nullptr; + } + auto c = std::make_unique(*this, std::move(send_empty_frame)); + c->_control.set_supported_algos(algo_list_to_set(_cfg.algo_config.get())); + c->_control.announce_dict(_most_recent_dict); + return c; +} + + +advanced_rpc_compressor::tracker::tracker(config cfg) + : _cfg(cfg) + , _algo_config_observer(_cfg.algo_config.observe([this] (const auto& x) { + set_supported_algos(algo_list_to_set(x)); + })) +{ + if (_cfg.register_metrics) { + register_metrics(); + } +} + +advanced_rpc_compressor::tracker::~tracker() { +} + +void advanced_rpc_compressor::tracker::attach_to_dict_sampler(dict_sampler* dt) noexcept { + _dict_sampler = dt; +} + +void advanced_rpc_compressor::tracker::set_supported_algos(compression_algorithm_set algos) noexcept { + for (const auto c : _compressors) { + c->_control.set_supported_algos(algos); + } +} + +size_t advanced_rpc_compressor::tracker::register_compressor(advanced_rpc_compressor* c) { + _compressors.push_back(c); + c->_control.announce_dict(_most_recent_dict); + return _compressors.size() - 1; +} + +void advanced_rpc_compressor::tracker::unregister_compressor(size_t i) { + assert(_compressors.size() && i < _compressors.size()); + std::swap(_compressors[i], _compressors.back()); + _compressors[i]->_idx = i; + _compressors.pop_back(); +} + +void advanced_rpc_compressor::tracker::register_metrics() { + namespace sm = seastar::metrics; + sm::label algo_label("algorithm"); + for (int i = 0; i < static_cast(compression_algorithm::type::COUNT); ++i) { + auto stats = &_stats[i]; + auto label = algo_label(compression_algorithm(i).name()); + _metrics.add_group("rpc_compression", { + sm::make_counter("bytes_sent", stats->bytes_sent, sm::description("bytes written to RPC connections, before compression"), {label}), + sm::make_counter("compressed_bytes_sent", stats->compressed_bytes_sent, sm::description("bytes written to RPC connections, after compression"), {label}), + sm::make_counter("compressed_bytes_received", stats->compressed_bytes_received, sm::description("bytes read from RPC connections, before decompression"), {label}), + sm::make_counter("messages_received", stats->messages_received, sm::description("RPC messages received"), {label}), + sm::make_counter("messages_sent", stats->messages_sent, sm::description("RPC messages sent"), {label}), + sm::make_counter("bytes_received", stats->bytes_received, sm::description("bytes read from RPC connections, after decompression"), {label}), + sm::make_counter("compression_cpu_nanos", stats->compression_cpu_nanos, sm::description("nanoseconds spent on compression"), {label}), + sm::make_counter("decompression_cpu_nanos", stats->decompression_cpu_nanos, sm::description("nanoseconds spent on decompression"), {label}), + }); + } +} + +uint64_t advanced_rpc_compressor::tracker::get_total_nanos_spent() const noexcept { + return _stats[static_cast(compression_algorithm::type::ZSTD)].decompression_cpu_nanos + + _stats[static_cast(compression_algorithm::type::ZSTD)].compression_cpu_nanos + + _stats[static_cast(compression_algorithm::type::LZ4)].decompression_cpu_nanos + + _stats[static_cast(compression_algorithm::type::LZ4)].compression_cpu_nanos; +} + +void advanced_rpc_compressor::tracker::maybe_refresh_zstd_quota(uint64_t now) noexcept { + using std::chrono::nanoseconds, std::chrono::milliseconds; + if (now >= _short_period_start + nanoseconds(milliseconds(_cfg.zstd_quota_refresh_ms)).count()) { + _short_period_start = now; + _nanos_used_before_this_short_period = get_total_nanos_spent(); + } + if (now >= _long_period_start + nanoseconds(milliseconds(_cfg.zstd_longterm_quota_refresh_ms)).count()) { + _long_period_start = now; + _nanos_used_before_this_long_period = get_total_nanos_spent(); + } +} + +bool advanced_rpc_compressor::tracker::cpu_limit_exceeded() const noexcept { + using std::chrono::nanoseconds, std::chrono::milliseconds; + uint64_t used_short = get_total_nanos_spent() - _nanos_used_before_this_short_period; + uint64_t used_long = get_total_nanos_spent() - _nanos_used_before_this_long_period; + uint64_t limit_short = nanoseconds(milliseconds(_cfg.zstd_quota_refresh_ms.get())).count() * _cfg.zstd_quota_fraction; + uint64_t limit_long = nanoseconds(milliseconds(_cfg.zstd_longterm_quota_refresh_ms.get())).count() * _cfg.zstd_longterm_quota_fraction; + return used_long >= limit_long || used_short >= limit_short; +} + +std::span advanced_rpc_compressor::tracker::get_stats() const noexcept { + return _stats; +} + +stream_compressor& advanced_rpc_compressor::get_compressor(compression_algorithm algo) { + switch (algo.get()) { + case compression_algorithm::type::LZ4: return get_global_lz4_cstream(); + case compression_algorithm::type::ZSTD: return get_global_zstd_cstream(); + case compression_algorithm::type::RAW: return the_raw_stream; + default: __builtin_unreachable(); + } +} + +stream_decompressor& advanced_rpc_compressor::get_decompressor(compression_algorithm algo) { + switch (algo.get()) { + case compression_algorithm::type::LZ4: return get_global_lz4_dstream(); + case compression_algorithm::type::ZSTD: return get_global_zstd_dstream(); + case compression_algorithm::type::RAW: return the_raw_stream; + default: __builtin_unreachable(); + } +} + +rpc::snd_buf advanced_rpc_compressor::compress(size_t head_space, rpc::snd_buf data) { + const size_t checksum_size = _tracker->_cfg.checksumming.get() ? sizeof(uint32_t) : 0; + const uint32_t crc = checksum_size ? crc_impl(data) : -1; + + auto now = _tracker->get_steady_nanos(); + _tracker->maybe_refresh_zstd_quota(now); + + auto algo = get_algo_for_next_msg(data.size); + + auto& stats = _tracker->_stats[algo.idx()]; + auto update_time_stats = defer([&, nanos_before = now] { + stats.compression_cpu_nanos += _tracker->get_steady_nanos() - nanos_before; + }); + + _tracker->ingest(data); + + auto protocol_header = _control.produce_control_header(); + const size_t protocol_header_size = protocol_header ? control_protocol_frame::serialized_size : 0; + + auto uncompressed_size = data.size; + auto compressed = std::invoke([&] { + try { + return compress_impl(head_space + 1 + checksum_size + protocol_header_size, std::move(data), get_compressor(algo), true, rpc::snd_buf::chunk_size); + } catch (...) { + arc_logger.error("Error during decompression with algorithm {}: {}. ", algo.name(), std::current_exception()); + throw; + } + }); + + // Write the algorithm type to the first byte after the external head_space. + // Note: compress_impl guarantees that the head space (including our byte, as we passed head_space + 1) is in the first fragment, + // so what we are doing below is legal. + auto dst = std::get_if>(&compressed.bufs); + if (!dst) { + dst = std::get>>(compressed.bufs).data(); + } + static_assert(compression_algorithm::count() <= 0x3f); // We have 6 bits for algorithm ID, 2 bits for flags. + dst->get_write()[head_space] = (algo.idx() & 0x3f) | (protocol_header ? 0x80 : 0x00) | (checksum_size ? 0x40 : 0x00); + if (checksum_size) { + write_le(&dst->get_write()[head_space + 1], crc); + } + if (protocol_header) { + auto out_data = reinterpret_cast(dst->get_write() + head_space + 1 + checksum_size); + constexpr size_t out_size = control_protocol_frame::serialized_size; + auto out = std::span(out_data, out_size); + protocol_header->serialize(out); + } + + stats.bytes_sent += uncompressed_size; + stats.compressed_bytes_sent += compressed.size - head_space; + stats.messages_sent += 1; + return compressed; +} + +template +requires std::is_trivially_copyable_v +T read_from_rcv_buf(rpc::rcv_buf& data) { + if (data.size < sizeof(T)) { + throw std::runtime_error("Truncated compressed RPC frame"); + } + auto it = std::get_if>(&data.bufs); + if (!it) { + it = std::get>>(data.bufs).data(); + } + std::array out; + auto out_span = std::as_writable_bytes(std::span(out)).subspan(0); + while (out_span.size()) { + size_t n = std::min(out_span.size(), it->size()); + // Make a special case for n==0, to avoid calling memcpy(src=..., it->get()=nullptr, n=0). The nullptr bothers UBSAN. + if (n) { + std::memcpy(static_cast(out_span.data()), it->get(), n); + out_span = out_span.subspan(n); + it->trim_front(n); + data.size -= n; + } + ++it; + } + return out[0]; +} + +rpc::rcv_buf advanced_rpc_compressor::decompress(rpc::rcv_buf data) { + const uint8_t header_byte = read_from_rcv_buf(data); + const bool has_checksum = header_byte & 0x40; + const bool has_control_frame = header_byte & 0x80; + + uint32_t expected_crc = -1; + if (has_checksum) { + expected_crc = seastar::le_to_cpu(read_from_rcv_buf(data)); + } + + if (has_control_frame) { + auto control_protocol_frame_bytes = read_from_rcv_buf>(data); + _control.consume_control_header(control_protocol_frame::deserialize(control_protocol_frame_bytes)); + } + + // Will throw if the enum value is unknown. + auto algo = compression_algorithm(header_byte & 0x3f); + + auto& stats = _tracker->_stats[algo.idx()]; + auto update_time_stats = defer([&, nanos_before = _tracker->get_steady_nanos()] { + stats.decompression_cpu_nanos += _tracker->get_steady_nanos() - nanos_before; + }); + auto compressed_size = data.size; + auto decompressed = std::invoke([&] { + try { + return decompress_impl(data, get_decompressor(algo), true, rpc::snd_buf::chunk_size); + } catch (...) { + arc_logger.error("Error during compression with algorithm {}: {}. ", algo.name(), std::current_exception()); + throw; + } + }); + if (has_checksum) { + const uint32_t actual_crc = crc_impl(decompressed); + if (expected_crc != actual_crc) { + seastar::on_internal_error(arc_logger, fmt::format("RPC compression checksum error (expected: {:x}, got: {:x}). This indicates a bug. Set `internode_compression: none` and restart the nodes to regain stability, then report the bug.", expected_crc, actual_crc)); + } + } + _tracker->ingest(decompressed); + stats.compressed_bytes_received += compressed_size; + stats.bytes_received += decompressed.size; + stats.messages_received += 1; + return decompressed; +} + +zstd_dstream& advanced_rpc_compressor::get_global_zstd_dstream() { + auto& dstream = _tracker->get_global_zstd_dstream(); + dstream.set_dict(_control.receiver_current_dict().zstd_ddict.get()); + return _tracker->get_global_zstd_dstream(); +} + +zstd_cstream& advanced_rpc_compressor::get_global_zstd_cstream() { + auto& cstream = _tracker->get_global_zstd_cstream(); + cstream.set_dict(_control.sender_current_dict().zstd_cdict.get()); + return _tracker->get_global_zstd_cstream(); +} + +lz4_dstream& advanced_rpc_compressor::get_global_lz4_dstream() { + auto& dstream = _tracker->get_global_lz4_dstream(); + dstream.set_dict(_control.receiver_current_dict().lz4_ddict); + return dstream; +} + +lz4_cstream& advanced_rpc_compressor::get_global_lz4_cstream() { + auto& cstream = _tracker->get_global_lz4_cstream(); + cstream.set_dict(_control.sender_current_dict().lz4_cdict.get()); + return cstream; +} + +zstd_dstream& advanced_rpc_compressor::tracker::get_global_zstd_dstream() { + if (!_global_zstd_dstream) { + _global_zstd_dstream = std::make_unique(); + } + return *_global_zstd_dstream; +} + +zstd_cstream& advanced_rpc_compressor::tracker::get_global_zstd_cstream() { + if (!_global_zstd_cstream) { + _global_zstd_cstream = std::make_unique(); + } + return *_global_zstd_cstream; +} + +lz4_dstream& advanced_rpc_compressor::tracker::get_global_lz4_dstream() { + if (!_global_lz4_dstream) { + _global_lz4_dstream = std::make_unique(); + } + return *_global_lz4_dstream; +} + +lz4_cstream& advanced_rpc_compressor::tracker::get_global_lz4_cstream() { + if (!_global_lz4_cstream) { + _global_lz4_cstream = std::make_unique(); + } + return *_global_lz4_cstream; +} + +template +requires std::same_as || std::same_as +void advanced_rpc_compressor::tracker::ingest_generic(const T& data) { + if (_dict_sampler && _dict_sampler->is_sampling()) { + if (const auto* src = std::get_if>(&data.bufs)) { + _dict_sampler->ingest({reinterpret_cast(src->get()), src->size()}); + } else { + const auto& frags = std::get>>(data.bufs); + for (const auto& frag : frags) { + _dict_sampler->ingest({reinterpret_cast(frag.get()), frag.size()}); + } + } + } +} + +void advanced_rpc_compressor::tracker::ingest(const rpc::snd_buf& data) { + ingest_generic(data); +} + +void advanced_rpc_compressor::tracker::ingest(const rpc::rcv_buf& data) { + ingest_generic(data); +} + +void advanced_rpc_compressor::tracker::announce_dict(dict_ptr d) { + _most_recent_dict = d; + for (const auto c : _compressors) { + c->_control.announce_dict(_most_recent_dict); + } +} + +future<> announce_dict_to_shards(seastar::sharded& sharded_tracker, utils::shared_dict shared_dict) { + arc_logger.debug("Announcing new dictionary: ts={}, origin={}", shared_dict.id.timestamp, shared_dict.id.origin_node); + auto dict = make_lw_shared(std::move(shared_dict)); + auto foreign_ptrs = std::vector>(); + for (size_t i = 0; i < smp::count; ++i) { + foreign_ptrs.push_back(make_foreign(dict)); + } + co_await sharded_tracker.invoke_on_all([&foreign_ptrs] (auto& tracker) { + tracker.announce_dict(make_lw_shared(std::move(foreign_ptrs[this_shard_id()]))); + }); +} + +} // namespace utils diff --git a/utils/advanced_rpc_compressor.hh b/utils/advanced_rpc_compressor.hh new file mode 100644 index 000000000000..2cfc10ccc875 --- /dev/null +++ b/utils/advanced_rpc_compressor.hh @@ -0,0 +1,365 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include "utils/refcounted.hh" +#include "utils/updateable_value.hh" +#include "utils/enum_option.hh" + +namespace utils { + +class dict_sampler; +class lz4_cstream; +class lz4_dstream; +class zstd_cstream; +class zstd_dstream; +class stream_compressor; +class stream_decompressor; +class shared_dict; +using dict_ptr = lw_shared_ptr>>; +class control_protocol_frame; + +// An enum wrapper, describing supported RPC compression algorithms. +// Always contains a valid value —- the constructors won't allow +// an invalid/unknown enum variant to be constructed. +struct compression_algorithm { + using underlying = uint8_t; + enum class type : underlying { + RAW, + LZ4, + ZSTD, + COUNT, + } _value; + // Construct from an integer. + // Used to deserialize the algorithm from the first byte of the frame. + constexpr compression_algorithm(underlying x) { + if (x < 0 || x >= static_cast(type::COUNT)) { + throw std::runtime_error(fmt::format("Invalid value {} for enum compression_algorithm", static_cast(x))); + } + _value = static_cast(x); + } + // Construct from `type`. Makes sure that `type` has a valid value. + constexpr compression_algorithm(type x) : compression_algorithm(static_cast(x)) {} + + // These names are used in multiple places: + // RPC negotiation, in metric labels, and config. + static constexpr std::string_view names[] = { + "raw", + "lz4", + "zstd", + }; + static_assert(std::size(names) == static_cast(compression_algorithm::type::COUNT)); + + // Implements enum_option. + static auto map() { + std::unordered_map ret; + for (size_t i = 0; i < std::size(names); ++i) { + ret.insert(std::make_pair(std::string(names[i]), compression_algorithm(i).get())); + } + return ret; + } + + constexpr std::string_view name() const noexcept { return names[idx()]; } + constexpr underlying idx() const noexcept { return static_cast(_value); } + constexpr type get() const noexcept { return _value; } + constexpr static size_t count() { return static_cast(type::COUNT); }; + bool operator<=>(const compression_algorithm &) const = default; +}; + + +// Represents a set of compression algorithms. +// Backed by a bitset. +// Used for convenience during algorithm negotiations. +class compression_algorithm_set { + uint8_t _bitset; + static_assert(std::numeric_limits::digits > compression_algorithm::count()); + constexpr compression_algorithm_set(uint8_t v) noexcept : _bitset(v) {} +public: + // Returns a set containing the given algorithm and all algorithms weaker (smaller in the enum order) + // than it. + constexpr static compression_algorithm_set this_or_lighter(compression_algorithm algo) noexcept { + auto x = 1 << (algo.idx()); + return {x + (x - 1)}; + } + // Returns the strongest (greatest in the enum order) algorithm in the set. + constexpr compression_algorithm heaviest() const { + return {std::bit_width(_bitset) - 1}; + } + // The usual set operations. + constexpr static compression_algorithm_set singleton(compression_algorithm algo) noexcept { + return {1 << algo.idx()}; + } + constexpr compression_algorithm_set intersection(compression_algorithm_set o) const noexcept { + return {_bitset & o._bitset}; + } + constexpr compression_algorithm_set difference(compression_algorithm_set o) const noexcept { + return {_bitset &~ o._bitset}; + } + constexpr compression_algorithm_set sum(compression_algorithm_set o) const noexcept { + return {_bitset | o._bitset}; + } + constexpr bool contains(compression_algorithm algo) const noexcept { + return _bitset & (1 << algo.idx()); + } + constexpr bool operator==(const compression_algorithm_set&) const = default; + // Returns the contained bitset. Used for serialization. + constexpr uint8_t value() const noexcept { + return _bitset; + } + // Reconstructs the set from the output of `value()`. Used for deserialization. + constexpr static compression_algorithm_set from_value(uint8_t bitset) { + compression_algorithm_set x = bitset; + x.heaviest(); // This is a validation check. It will throw if the bitset contains some illegal/unknown bits. + return x; + } +}; + +using algo_config = std::vector>; + +// See docs/dev/advanced_rpc_compression.md, +// section `Negotiation` for more information about the protocol. +struct control_protocol { + // The sender increments its protocol epoch every time it proposes to commit to a different + // algorithm. + // The epoch is echoed back by the receiver to match proposals with accepts. + uint64_t _sender_protocol_epoch = 0; + uint64_t _receiver_protocol_epoch = 0; + + // To send a control frame to the peer, we set one of these flags and signal _needs_progress. + // This will cause at least one RPC message to be sent promptly. We prepend our frame to + // the next RPC message. + + // These two flags are mutually exclusive. + bool _sender_has_update = false; + bool _sender_has_commit = false; + // These two flags are mutually exclusive. + bool _receiver_has_update = false; + bool _receiver_has_commit = false; + + dict_ptr _sender_recent_dict = nullptr; + dict_ptr _sender_committed_dict = nullptr; + dict_ptr _sender_current_dict = nullptr; + dict_ptr _receiver_recent_dict = nullptr; + dict_ptr _receiver_committed_dict = nullptr; + dict_ptr _receiver_current_dict = nullptr; + compression_algorithm _sender_current_algo = compression_algorithm::type::RAW; + compression_algorithm _sender_committed_algo = compression_algorithm::type::RAW; + compression_algorithm_set _algos = compression_algorithm_set::singleton(compression_algorithm::type::RAW); + + // When signalled, an empty message will be sent over this connection soon. + // Used to guarantee progress of algorithm negotiations. + condition_variable& _needs_progress; +public: + control_protocol(condition_variable&); + // These functions handle the control (negotiation) protocol. + std::optional produce_control_header(); + void consume_control_header(control_protocol_frame); + void announce_dict(dict_ptr) noexcept; + void set_supported_algos(compression_algorithm_set algos) noexcept; + compression_algorithm sender_current_algorithm() const noexcept; + const shared_dict& sender_current_dict() const noexcept; + const shared_dict& receiver_current_dict() const noexcept; +}; + +class advanced_rpc_compressor final : public rpc::compressor { +public: + class tracker; + template class tracker_with_clock; +private: + // Pointer/reference to the tracker, which contains stats that we need to update, + // and limits that we need to respect. + // + // The `refcounted` is just a precaution against a misuse of the APIs. + refcounted::ref _tracker; + + // Index of the compressor inside the tracker. + // Used to unregister the compressor on destruction. + size_t _idx = -1; + + // State of the negotiation protocol. + control_protocol _control; + + // Used by _control to send its messages to other side of the connection. + condition_variable _needs_progress; + std::function()> _send_empty_frame; + future<> _progress_fiber; + + // These return global compression contexts (for non-streaming compression modes), lazily initializing them. + zstd_dstream& get_global_zstd_dstream(); + zstd_cstream& get_global_zstd_cstream(); + lz4_dstream& get_global_lz4_dstream(); + lz4_cstream& get_global_lz4_cstream(); + + // Calls the appropriate get_*_cstream() function. + stream_compressor& get_compressor(compression_algorithm); + // Calls the appropriate get_*_dstream() function. + stream_decompressor& get_decompressor(compression_algorithm); + + // Decides the algorithm used for the next message, based + // on the state of the negotiation and the size of the message. + compression_algorithm get_algo_for_next_msg(size_t msgsize); + + // Starts a worker fiber responsible for sending _control's messages. + future<> start_progress_fiber(); +public: + advanced_rpc_compressor( + tracker& fac, + std::function()> send_empty_frame + ); + ~advanced_rpc_compressor(); + + // The public interface of rpc::compressor. + rpc::snd_buf compress(size_t head_space, rpc::snd_buf data) override; + rpc::rcv_buf decompress(rpc::rcv_buf data) override; + sstring name() const override; + future<> close() noexcept override; +}; + +// Tracker holds one of these for every compression mode/algorithm. +// They are used for displaying metrics, and for implementing CPU/memory usage limits. +struct per_algorithm_stats { + uint64_t bytes_sent = 0; + uint64_t compressed_bytes_sent = 0; + uint64_t messages_sent = 0; + uint64_t compression_cpu_nanos = 0; + uint64_t bytes_received = 0; + uint64_t compressed_bytes_received = 0; + uint64_t messages_received = 0; + uint64_t decompression_cpu_nanos = 0; +}; + +// The tracker contains everything which is shared between compressor instances: +// stats, metrics, limits, reusable non-streaming compressors. +// +// Class `tracker` itself contains clock-independent functionality. +// Clock-dependent functionality is split into `tracker_with_clock`, to minimize template pollution. +// Alternatively, we could wrap clocks into some virtual interface. +// +// Tracker is referenced by all compressors, so we inherit from `refcounted` to +// prevent a misuse of the API (dangling references). +class advanced_rpc_compressor::tracker : public refcounted { +public: + using algo_config = algo_config; + struct config { + updateable_value zstd_min_msg_size{0}; + updateable_value zstd_max_msg_size{std::numeric_limits::max()}; + updateable_value zstd_quota_fraction{0}; + updateable_value zstd_quota_refresh_ms{20}; + updateable_value zstd_longterm_quota_fraction{1000}; + updateable_value zstd_longterm_quota_refresh_ms{1000}; + updateable_value algo_config{{compression_algorithm::type::ZSTD, compression_algorithm::type::LZ4}}; + bool register_metrics = false; + updateable_value checksumming{true}; + }; +private: + friend advanced_rpc_compressor; + + config _cfg; + observer _algo_config_observer; + + std::array _stats; + metrics::metric_groups _metrics; + + // Compression contexts for non-streaming compression modes. + // They are shared by all compressors owned this tracker. + std::unique_ptr _global_zstd_cstream; + std::unique_ptr _global_zstd_dstream; + std::unique_ptr _global_lz4_cstream; + std::unique_ptr _global_lz4_dstream; + std::vector _compressors; + dict_ptr _most_recent_dict = nullptr; + + dict_sampler* _dict_sampler = nullptr; + + void register_metrics(); + void maybe_refresh_zstd_quota(uint64_t now) noexcept; + bool cpu_limit_exceeded() const noexcept; + uint64_t get_total_nanos_spent() const noexcept; + + zstd_dstream& get_global_zstd_dstream(); + zstd_cstream& get_global_zstd_cstream(); + lz4_dstream& get_global_lz4_dstream(); + lz4_cstream& get_global_lz4_cstream(); + + void ingest(const rpc::snd_buf& data); + void ingest(const rpc::rcv_buf& data); + + template + requires std::same_as || std::same_as + void ingest_generic(const T& data); + + size_t register_compressor(advanced_rpc_compressor*); + void unregister_compressor(size_t); +public: + tracker(config); + virtual ~tracker(); + + // Interface of rpc::compressor::factory. + // `tracker` itself doesn't inherit from `factory` (just because this inheritance would have no users), + // but a wrapper over `tracker` can use these to implement the interface. + const sstring& supported() const; + std::unique_ptr negotiate(sstring feature, bool is_server, std::function()> send_empty_frame); + std::span get_stats() const noexcept; + + void announce_dict(dict_ptr); + void attach_to_dict_sampler(dict_sampler*) noexcept; + void set_supported_algos(compression_algorithm_set algos) noexcept; +protected: + // These members are governed by `tracker_with_clock`. + // + // Why use nanos instead of Clock::duration? + // Because that would require templating `factory_base` and `advanced_rpc_compressor` on `Clock`. + // Forcing a common duration unit allows for encapsulation of clock-related details inside `tracker_with_clock`. + virtual uint64_t get_steady_nanos() const = 0; + + // There are two CPU limit accounting periods: short period and long period. + // Long period is multiple seconds and is meant to limit the throughput overhead. + // Short period is a few several milliseconds and is meant to limit the latency ovehead. + // Each period has a separate quota and we fall back to cheaper compression if any of + // them is exceeded. + // + // The long quota is periodically reset by a timer. + // The short quota is periodically reset manually by the tracker, because the period is very short. + // A timer with this period could generate unnecessary noise (e.g. keep waking up an otherwise-idle reactor). + constexpr static std::chrono::nanoseconds long_period = std::chrono::seconds(10); + uint64_t _short_period_start = 0; + uint64_t _long_period_start = 0; + uint64_t _nanos_used_before_this_short_period = 0; + uint64_t _nanos_used_before_this_long_period = 0; +}; + +// Implements clock-dependent functionality for `tracker`. +template +class advanced_rpc_compressor::tracker_with_clock : public advanced_rpc_compressor::tracker { + virtual uint64_t get_steady_nanos() const override { + return std::chrono::duration_cast(HighResClock::now().time_since_epoch()).count(); + } +public: + tracker_with_clock(config c) + : advanced_rpc_compressor::tracker(std::move(c)) + {} + // updateable_value must be created on the destination shard. + // Since tracker is sharded, we can't copy the tracker::config (which contains updateable_value) + // to all shards. But we can pass to all shards a function which will create the tracker::config. + tracker_with_clock(std::function f) + : tracker_with_clock(f()) + {} +}; + +class walltime_compressor_tracker final : public utils::advanced_rpc_compressor::tracker_with_clock { + using tracker_with_clock::tracker_with_clock; +}; + +// Helper for setting up the lw_shared_ptr>> tree +// used by the tracker to manage the lifetime of dicts. +future<> announce_dict_to_shards(seastar::sharded&, utils::shared_dict); + +} // namespace utils diff --git a/utils/advanced_rpc_compressor_protocol.hh b/utils/advanced_rpc_compressor_protocol.hh new file mode 100644 index 000000000000..f1d724bb3445 --- /dev/null +++ b/utils/advanced_rpc_compressor_protocol.hh @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "utils/shared_dict.hh" +#include "utils/advanced_rpc_compressor.hh" + +namespace utils { + +struct control_protocol_frame { + enum header_enum : uint8_t { + NONE, // This means that the field isn't filled. + UPDATE, + COMMIT, + }; + + struct one_side { + header_enum header = NONE; + uint64_t epoch = 0; + compression_algorithm_set algo = compression_algorithm_set::singleton(compression_algorithm::type::RAW); + shared_dict::dict_id dict; + constexpr static size_t serialized_size = 1+8+1+16+8+32; + void serialize(std::span); + static one_side deserialize(std::span); + }; + + // The negotiation algorithm is run for each of the two directions of the connection separately. + // The `receiver` field below is a message for the algorithm instance in which we are the receiver. + // `sender` is for the instance in which we are the sender. + // + // Even though usually only one of these will be filled with something meaningful (not `NONE`), + // we always send both just to keep the layout of the frame fixed. It simplifies the serialization. + one_side receiver; + one_side sender; + + constexpr static size_t serialized_size = 2 * one_side::serialized_size; + void serialize(std::span); + static control_protocol_frame deserialize(std::span); +}; + +} // namespace utils diff --git a/utils/refcounted.hh b/utils/refcounted.hh new file mode 100644 index 000000000000..b6fa23e4d5d8 --- /dev/null +++ b/utils/refcounted.hh @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include + +#pragma once + +namespace utils { + +// Aborts the program if destroyed with existing references. +class refcounted { + mutable uint64_t _count = 0; +public: + refcounted() = default; + refcounted(refcounted&&) = delete; + ~refcounted() noexcept { + assert(_count == 0); + } + template + class ref { + T& _target; + public: + ref(T& t) noexcept : _target(t) { + static_cast(_target)._count += 1; + } + ~ref() noexcept { + static_cast(_target)._count -= 1; + } + T* operator->() const noexcept { + return &_target; + } + T& operator*() const noexcept { + return _target; + } + }; +}; + +} diff --git a/utils/shared_dict.hh b/utils/shared_dict.hh new file mode 100644 index 000000000000..4948cb011623 --- /dev/null +++ b/utils/shared_dict.hh @@ -0,0 +1,58 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#define ZSTD_STATIC_LINKING_ONLY +#include +#define LZ4_STATIC_LINKING_ONLY +#include + +#include "utils/UUID.hh" + +#include + +namespace utils { + +using sha256_type = std::array; + +// For performance reasons (cache pressure), it is desirable to have only +// one instance of a particular dictionary on a node. +// +// `shared_dict` takes a raw dictionary buffer (which preferably contains +// a dictionary in zstd format, but any content is fine), and wraps around +// it with compressor-specific dictionary types. (Each compressor attached +// some algorithm-specific hash indices and entropy tables to it). +// +// This way different compressors and decompressors can share the same +// raw dictionary buffer. +// +// Dictionaries are always read-only, so it's fine (and strongly preferable) +// to share this object between shards. +struct shared_dict { + struct dict_id { + uint64_t timestamp = 0; + UUID origin_node{}; + sha256_type content_sha256{}; + bool operator==(const dict_id&) const = default; + }; + dict_id id{}; + std::vector data; + std::unique_ptr zstd_ddict{nullptr, ZSTD_freeDDict}; + std::unique_ptr zstd_cdict{nullptr, ZSTD_freeCDict}; + std::unique_ptr lz4_cdict{nullptr, LZ4_freeStream}; + std::span lz4_ddict; + // I got burned by an LZ4 bug (`<` used instead of `<=`) once when dealing with exactly 64 kiB, + // prefixes, so I'm using 64 kiB - 1 because of the trauma. + // But 64 kiB would probably work for this use case too. + constexpr static size_t max_lz4_dict_size = 64 * 1024 - 1; + shared_dict() = default; + shared_dict(std::span d, uint64_t timestamp, UUID origin_node, int zstd_compression_level = 1); +}; + +} // namespace utils From cc15ca329eaf8465e82dccd0a6ec9b21224e5ba4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Mon, 19 Aug 2024 16:35:47 +0200 Subject: [PATCH 076/397] db/system_keyspace: add system.dicts Adds a new system table which will act as the medium for distributing compression dictionaries over the cluster. This table will be managed by Raft (group 0). It will be hooked up to it in follow-up commits. --- db/system_keyspace.cc | 63 ++++++++++++++++++++++++++++++++++++++++++- db/system_keyspace.hh | 13 +++++++++ 2 files changed, 75 insertions(+), 1 deletion(-) diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 075c7a8f462a..93531455bc0a 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -46,6 +46,7 @@ #include "replica/query.hh" #include "types/types.hh" #include "service/raft/raft_group0_client.hh" +#include "utils/shared_dict.hh" #include "replica/database.hh" #include @@ -94,7 +95,8 @@ namespace { system_keyspace::ROLE_MEMBERS, system_keyspace::ROLE_ATTRIBUTES, system_keyspace::ROLE_PERMISSIONS, - system_keyspace::v3::CDC_LOCAL + system_keyspace::v3::CDC_LOCAL, + system_keyspace::DICTS }; if (ks_name == system_keyspace::NAME && tables.contains(cf_name)) { props.enable_schema_commitlog(); @@ -118,6 +120,7 @@ namespace { system_keyspace::ROLE_MEMBERS, system_keyspace::ROLE_ATTRIBUTES, system_keyspace::ROLE_PERMISSIONS, + system_keyspace::DICTS, }; if (ks_name == system_keyspace::NAME && tables.contains(cf_name)) { props.is_group0_table = true; @@ -1554,6 +1557,20 @@ schema_ptr system_keyspace::legacy::aggregates() { return schema; } +schema_ptr system_keyspace::dicts() { + static thread_local auto schema = [] { + auto id = generate_legacy_id(NAME, DICTS); + return schema_builder(NAME, DICTS, std::make_optional(id)) + .with_column("name", utf8_type, column_kind::partition_key) + .with_column("timestamp", timestamp_type) + .with_column("origin", uuid_type) + .with_column("data", bytes_type) + .with_hash_version() + .build(); + }(); + return schema; +} + future system_keyspace::load_local_info() { auto msg = co_await execute_cql(format("SELECT host_id, cluster_name FROM system.{} WHERE key=?", LOCAL), sstring(LOCAL)); @@ -2296,6 +2313,7 @@ std::vector system_keyspace::all_tables(const db::config& cfg) { v3::cdc_local(), raft(), raft_snapshots(), raft_snapshot_config(), group0_history(), discovery(), topology(), cdc_generations_v3(), topology_requests(), service_levels_v2(), view_build_status_v2(), + dicts(), }); if (cfg.check_experimental(db::experimental_features_t::feature::BROADCAST_TABLES)) { @@ -3439,6 +3457,49 @@ future system_keyspace::get_topology co_return m; } +future system_keyspace::get_insert_dict_mutation( + bytes data, + locator::host_id host_id, + db_clock::time_point dict_ts, + api::timestamp_type write_ts +) const { + const char* dict_name = "general"; + slogger.debug("Publishing new compression dictionary: {} {} {}", dict_name, dict_ts, host_id); + + static sstring insert_new = format("INSERT INTO {}.{} (name, timestamp, origin, data) VALUES (?, ?, ?, ?);", NAME, DICTS); + auto muts = co_await _qp.get_mutations_internal(insert_new, internal_system_query_state(), write_ts, { + data_value(dict_name), + data_value(dict_ts), + data_value(host_id.uuid()), + data_value(std::move(data)), + }); + if (muts.size() != 1) { + on_internal_error(slogger, "Expected to prepare a single mutation, but got multiple."); + } + co_return std::move(muts[0]); +} + +future system_keyspace::query_dict() const { + static sstring query = format("SELECT * FROM {}.{} WHERE name = ?;", NAME, DICTS); + auto result_set = co_await _qp.execute_internal( + query, db::consistency_level::ONE, internal_system_query_state(), {"general"}, cql3::query_processor::cache_internal::yes); + if (!result_set->empty()) { + auto &&row = result_set->one(); + auto content = row.get_as("data"); + auto timestamp = row.get_as("timestamp").time_since_epoch().count(); + auto origin = row.get_as("origin"); + const int zstd_compression_level = 1; + co_return utils::shared_dict( + std::as_bytes(std::span(content)), + timestamp, + origin, + zstd_compression_level + ); + } else { + co_return utils::shared_dict(); + } +} + sstring system_keyspace_name() { return system_keyspace::NAME; } diff --git a/db/system_keyspace.hh b/db/system_keyspace.hh index cb7e0a4bc656..79627efc7116 100644 --- a/db/system_keyspace.hh +++ b/db/system_keyspace.hh @@ -29,6 +29,10 @@ #include "types/types.hh" #include "auth_version.hh" +namespace utils { + class shared_dict; +}; + namespace sstables { struct entry_descriptor; class generation_type; @@ -184,6 +188,7 @@ public: static constexpr auto TABLETS = "tablets"; static constexpr auto SERVICE_LEVELS_V2 = "service_levels_v2"; static constexpr auto VIEW_BUILD_STATUS_V2 = "view_build_status_v2"; + static constexpr auto DICTS = "dicts"; // auth static constexpr auto ROLES = "roles"; @@ -278,6 +283,7 @@ public: static schema_ptr tablets(); static schema_ptr service_levels_v2(); static schema_ptr view_build_status_v2(); + static schema_ptr dicts(); // auth static schema_ptr roles(); @@ -651,6 +657,13 @@ public: future make_service_levels_version_mutation(int8_t version, const service::group0_guard& guard); future> get_service_levels_version_mutation(); + // Publishes a new compression dictionary to `dicts`, + // with the current timestamp. + future get_insert_dict_mutation( + bytes dict, locator::host_id self, db_clock::time_point dict_ts, api::timestamp_type write_ts) const; + // Queries `dicts` for the most recent compression dictionary. + future query_dict() const; + private: static std::optional decode_topology_features_state(::shared_ptr rs); From 6a982ee0dcc901117e930c34bf2ca3dd409ce866 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 4 Dec 2024 18:02:03 +0100 Subject: [PATCH 077/397] service: make Raft group 0 aware of system.dicts Adds glue which causes the contents of system.dicts to be sent in group 0 snapshots, and causes a callback to be called when system.dicts is updated locally. The callback is currently empty and will be hooked up to the RPC compressor tracker in one of the next commits. --- gms/feature_service.hh | 1 + service/raft/group0_state_machine.cc | 6 ++++++ service/raft/group0_state_machine.hh | 1 + service/storage_service.cc | 8 ++++++++ service/storage_service.hh | 9 +++++++++ 5 files changed, 25 insertions(+) diff --git a/gms/feature_service.hh b/gms/feature_service.hh index a0758f22c777..dd585cb7937b 100644 --- a/gms/feature_service.hh +++ b/gms/feature_service.hh @@ -156,6 +156,7 @@ public: gms::feature test_only_feature { *this, "TEST_ONLY_FEATURE"sv }; gms::feature address_nodes_by_host_ids { *this, "ADDRESS_NODES_BY_HOST_IDS"sv }; + gms::feature compression_dicts { *this, "COMPRESSION_DICTS"sv }; public: const std::unordered_map>& registered_features() const; diff --git a/service/raft/group0_state_machine.cc b/service/raft/group0_state_machine.cc index 3c472ed09cfc..edb6b09f8ab3 100644 --- a/service/raft/group0_state_machine.cc +++ b/service/raft/group0_state_machine.cc @@ -156,6 +156,9 @@ group0_state_machine::modules_to_reload group0_state_machine::get_modules_to_rel } else if (id == db::system_keyspace::role_members()->id() || id == db::system_keyspace::role_attributes()->id()) { modules.service_levels_effective_cache = true; } + if (mut.column_family_id() == db::system_keyspace::dicts()->id()) { + modules.compression_dictionary = true; + } } return modules; @@ -165,6 +168,9 @@ future<> group0_state_machine::reload_modules(modules_to_reload modules) { if (modules.service_levels_cache || modules.service_levels_effective_cache) { // this also updates SL effective cache co_await _ss.update_service_levels_cache(qos::update_both_cache_levels(modules.service_levels_cache), qos::query_context::group0); } + if (modules.compression_dictionary) { + co_await _ss.compression_dictionary_updated_callback(); + } } future<> group0_state_machine::merge_and_apply(group0_state_machine_merger& merger) { diff --git a/service/raft/group0_state_machine.hh b/service/raft/group0_state_machine.hh index 27220c0cd9da..976753af90f8 100644 --- a/service/raft/group0_state_machine.hh +++ b/service/raft/group0_state_machine.hh @@ -95,6 +95,7 @@ class group0_state_machine : public raft_state_machine { struct modules_to_reload { bool service_levels_cache = false; bool service_levels_effective_cache = false; + bool compression_dictionary = false; }; raft_group0_client& _client; diff --git a/service/storage_service.cc b/service/storage_service.cc index 167e0bdecc6c..62c5502cfdbb 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -918,6 +918,11 @@ future<> storage_service::update_service_levels_cache(qos::update_both_cache_lev co_await _sl_controller.local().update_cache(update_only_effective_cache, ctx); } +future<> storage_service::compression_dictionary_updated_callback() { + assert(this_shard_id() == 0); + return _compression_dictionary_updated_callback(); +} + // Moves the coroutine lambda onto the heap and extends its // lifetime until the resulting future is completed. // This allows to use captures in coroutine lambda after co_await-s. @@ -6990,6 +6995,9 @@ void storage_service::init_messaging_service() { if (ss._feature_service.view_build_status_on_group0) { additional_tables.push_back(db::system_keyspace::view_build_status_v2()->id()); } + if (ss._feature_service.compression_dicts) { + additional_tables.push_back(db::system_keyspace::dicts()->id()); + } } for (const auto& table : boost::join(params.tables, additional_tables)) { diff --git a/service/storage_service.hh b/service/storage_service.hh index 7d540ae38cbf..d21d9df70442 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -906,6 +906,13 @@ public: // update_both_cache_levels::no - update only effective service levels cache future<> update_service_levels_cache(qos::update_both_cache_levels update_only_effective_cache = qos::update_both_cache_levels::yes, qos::query_context ctx = qos::query_context::unspecified); + // Should be called whenever new compression dictionaries are published to system.dicts. + // This is an arbitrary callback passed through the constructor, + // but its intended usage is to set up the RPC connections to use the new dictionaries. + // + // Must be called on shard 0. + future<> compression_dictionary_updated_callback(); + future<> do_cluster_cleanup(); // Starts the upgrade procedure to topology on raft. @@ -991,6 +998,8 @@ private: // We need to be able to abort all group0 operation during shutdown, so we need special abort source for that abort_source _group0_as; + std::function()> _compression_dictionary_updated_callback; + friend class join_node_rpc_handshaker; friend class node_ops::node_ops_virtual_task; friend class node_ops::task_manager_module; From 5ce1e4410fd0e012eaca371df34c9b9baf7952d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Fri, 30 Aug 2024 14:13:40 +0200 Subject: [PATCH 078/397] message/dictionary_service: introduce dictionary_service This "service" is a bag for code responsible for dictionary training, created to unclutter main() from dictionary-specific logic. It starts the RPC dictionary training loop when the relevant cluster feature is enabled, pauses and unpauses it appropriately whenever relevant config or leadership status are updated, and publishes new dictionaries whenever the training fiber produces them. --- configure.py | 1 + message/CMakeLists.txt | 2 + message/dictionary_service.cc | 93 +++++++++++++++++++++++++++++++++++ message/dictionary_service.hh | 67 +++++++++++++++++++++++++ service/raft/raft_group0.cc | 9 ++++ service/raft/raft_group0.hh | 4 ++ 6 files changed, 176 insertions(+) create mode 100644 message/dictionary_service.cc create mode 100644 message/dictionary_service.hh diff --git a/configure.py b/configure.py index bee7f5111b07..473ed9127e9b 100755 --- a/configure.py +++ b/configure.py @@ -773,6 +773,7 @@ def find_ninja(): 'utils/limiting_data_source.cc', 'utils/updateable_value.cc', 'utils/dict_trainer.cc', + 'message/dictionary_service.cc', 'utils/directories.cc', 'gms/generation-number.cc', 'utils/rjson.cc', diff --git a/message/CMakeLists.txt b/message/CMakeLists.txt index 59dd3bc35b05..fe08ad5bbf2a 100644 --- a/message/CMakeLists.txt +++ b/message/CMakeLists.txt @@ -1,6 +1,8 @@ add_library(message STATIC) target_sources(message PRIVATE + dictionary_service.cc + dictionary_service.hh messaging_service.cc messaging_service.hh) target_include_directories(message diff --git a/message/dictionary_service.cc b/message/dictionary_service.cc new file mode 100644 index 000000000000..1ba6a0c88651 --- /dev/null +++ b/message/dictionary_service.cc @@ -0,0 +1,93 @@ +#include "dictionary_service.hh" +#include "service/raft/raft_group0.hh" +#include "gms/feature_service.hh" +#include "service/raft/raft_group0_client.hh" +#include +#include "db/system_keyspace.hh" + +dictionary_service::dictionary_service( + utils::dict_sampler& ds, + db::system_keyspace& sys_ks, + utils::alien_worker& alien_worker, + service::raft_group0_client& raft_group0_client, + service::raft_group0& raft_group0, + abort_source& as, + gms::feature_service& fs, + config<> cfg +) + : _sys_ks(sys_ks) + , _our_host_id(cfg.our_host_id) + , _rpc_dict_training_when(std::move(cfg.rpc_dict_training_when)) + , _raft_group0_client(raft_group0_client) + , _as(as) + , _training_fiber_future(make_ready_future<>()) + , _leadership_observer(raft_group0.observe_leadership([this] (bool leader) { + utils::dict_trainer_logger.debug("dictionary_service: _leadership_observer triggered"); + _is_leader = leader; + maybe_toggle_dict_training(); + })) + , _when_observer(_rpc_dict_training_when.observe([this] (const auto&) { + utils::dict_trainer_logger.debug("dictionary_service: _when_observer triggered"); + maybe_toggle_dict_training(); + })) + , _feature_observer(fs.compression_dicts.when_enabled([ + rpc_dict_training_min_time_seconds = std::move(cfg.rpc_dict_training_min_time_seconds), + rpc_dict_training_min_bytes = std::move(cfg.rpc_dict_training_min_bytes), + &alien_worker, + &ds, + this + ] { + utils::dict_trainer_logger.debug("dictionary_service: _feature_observer triggered"); + _training_fiber_future = _training_fiber.start( + ds, + [this] (utils::dict_sampler::dict_type d) { return publish_dict(std::move(d)); }, + std::move(rpc_dict_training_min_time_seconds), + std::move(rpc_dict_training_min_bytes), + alien_worker + ); + })) +{ + maybe_toggle_dict_training(); +} + + +void dictionary_service::maybe_toggle_dict_training() { + auto when = _rpc_dict_training_when(); + utils::dict_trainer_logger.debug("dictionary_service::maybe_toggle_dict_training(), called, _is_leader={}, when={}", _is_leader, when); + if (when == utils::dict_training_loop::when::type::NEVER) { + _training_fiber.pause(); + } else if (when == utils::dict_training_loop::when::type::ALWAYS) { + _training_fiber.unpause(); + } else if (when == utils::dict_training_loop::when::type::WHEN_LEADER) { + _is_leader ? _training_fiber.unpause() : _training_fiber.pause(); + } +}; + +future<> dictionary_service::stop() { + utils::dict_trainer_logger.debug("dictionary_service::stop(), called"); + // Don't let the feature observer start the training fiber after it's cancelled. + _feature_observer.reset(); + _training_fiber.cancel(); + co_await std::move(_training_fiber_future); +} + +future<> dictionary_service::publish_dict(utils::dict_sampler::dict_type d) { + while (true) { + utils::dict_trainer_logger.debug("dictionary_service::publish_dict(), called"); + try { + utils::dict_trainer_logger.debug("dictionary_service::publish_dict(), trying"); + auto batch = service::group0_batch(co_await _raft_group0_client.start_operation(_as)); + auto write_ts = batch.write_timestamp(); + auto new_dict_ts = db_clock::now(); + auto data = bytes(reinterpret_cast(d.data()), d.size()); + mutation publish_new_dict = co_await _sys_ks.get_insert_dict_mutation(std::move(data), _our_host_id, new_dict_ts, write_ts); + batch.add_mutation(std::move(publish_new_dict), "publish new compression dictionary"); + utils::dict_trainer_logger.debug("dictionary_service::publish_dict(), committing"); + co_await std::move(batch).commit(_raft_group0_client, _as, {}); + utils::dict_trainer_logger.debug("dictionary_service::publish_dict(), finished"); + break; + } catch (const service::group0_concurrent_modification&) { + utils::dict_trainer_logger.debug("group0_concurrent_modification in dictionary_service::publish_dict(), retrying"); + } + } +} diff --git a/message/dictionary_service.hh b/message/dictionary_service.hh new file mode 100644 index 000000000000..19435a81c6a1 --- /dev/null +++ b/message/dictionary_service.hh @@ -0,0 +1,67 @@ +#include "locator/host_id.hh" +#include "utils/updateable_value.hh" +#include "utils/dict_trainer.hh" +#include + +namespace db { + class system_keyspace; +} // namespace db + +namespace utils { + class alien_worker; +} // namespace utils + +namespace service { + class raft_group0_client; + class raft_group0; +} // namespace service + +namespace gms { + class feature_service; +} // namespace gms + +// A bag of code responsible for starting, stopping, pausing and unpausing RPC compression +// dictionary training, and for publishing its results to system.dicts (via Raft group 0). +// +// It starts the training when the relevant cluster feature is enabled, +// pauses and unpauses the training appropriately whenever relevant config or leadership status are updated, +// and publishes new dictionaries whenever the training fiber produces them. +class dictionary_service { + db::system_keyspace& _sys_ks; + locator::host_id _our_host_id; + utils::updateable_value> _rpc_dict_training_when; + service::raft_group0_client& _raft_group0_client; + abort_source& _as; + utils::dict_training_loop _training_fiber; + future<> _training_fiber_future; + + bool _is_leader = false; + utils::observer _leadership_observer; + utils::observer> _when_observer; + std::optional _feature_observer; + + void maybe_toggle_dict_training(); + future<> publish_dict(utils::dict_sampler::dict_type); +public: + // This template trick forces the user of `config` to initialize all fields explicitly. + template + struct config { + locator::host_id our_host_id = Uninitialized(); + utils::updateable_value rpc_dict_training_min_time_seconds = Uninitialized(); + utils::updateable_value rpc_dict_training_min_bytes = Uninitialized(); + utils::updateable_value> rpc_dict_training_when = Uninitialized(); + }; + // Note: the training fiber will start as soon as the relevant cluster feature is enabled. + dictionary_service( + utils::dict_sampler&, + db::system_keyspace&, + utils::alien_worker&, + service::raft_group0_client&, + service::raft_group0&, + abort_source& stop_signal, + gms::feature_service&, + config<> + ); + // For clean shutdown, this must be called and awaited before destruction. + future<> stop(); +}; diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index 089a0a7e2c53..f6dc84a72f6e 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -433,14 +433,23 @@ future<> raft_group0::leadership_monitor_fiber() { co_await group0_server().wait_for_state_change(&_leadership_monitor_as); } group0_log.info("gaining leadership"); + _leadership_observable.set(true); co_await group0_server().wait_for_state_change(&_leadership_monitor_as); group0_log.info("losing leadership"); + _leadership_observable.set(false); } } catch (...) { group0_log.debug("leadership_monitor_fiber aborted with {}", std::current_exception()); } } +utils::observer raft_group0::observe_leadership(std::function cb) { + if (_leadership_observable.get()) { + cb(true); + } + return _leadership_observable.observe(cb); +} + future<> raft_group0::join_group0(std::vector seeds, shared_ptr handshaker, service::storage_service& ss, cql3::query_processor& qp, service::migration_manager& mm, db::system_keyspace& sys_ks, bool topology_change_enabled) { SCYLLA_ASSERT(this_shard_id() == 0); diff --git a/service/raft/raft_group0.hh b/service/raft/raft_group0.hh index 5cd05a30607d..257ead64fe7e 100644 --- a/service/raft/raft_group0.hh +++ b/service/raft/raft_group0.hh @@ -11,6 +11,7 @@ #include "service/raft/discovery.hh" #include "service/raft/group0_fwd.hh" #include "gms/feature.hh" +#include "utils/updateable_value.hh" namespace cql3 { class query_processor; } @@ -126,6 +127,7 @@ class raft_group0 { future<> leadership_monitor_fiber(); future<> _leadership_monitor = make_ready_future<>(); abort_source _leadership_monitor_as; + utils::updateable_value_source _leadership_observable; public: // Passed to `setup_group0` when replacing a node. @@ -298,6 +300,8 @@ public: // Returns true after the group 0 server has been started. bool joined_group0() const; + utils::observer observe_leadership(std::function); + // Returns scheduling group group0 is configured to run with seastar::scheduling_group get_scheduling_group() { return _sg; From 75da99ce8b588fa8922160c5ff8406df9452ebcf Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Tue, 17 Dec 2024 11:47:43 +0200 Subject: [PATCH 079/397] test/perf: add perf_sort_by_proximity benchmark benchmark sort_by_proximity Baseline results on my desktop for sorting 3 nodes: single run iterations: 0 single run duration: 1.000s number of runs: 5 number of cores: 1 random seed: 20241224 test iterations median mad min max allocs tasks inst cycles sort_by_proximity_topology.perf_sort_by_proximity 12808773 77.368ns 0.062ns 77.300ns 77.873ns 0.000 0.000 1194.2 231.6 Signed-off-by: Benny Halevy --- configure.py | 1 + locator/token_metadata.cc | 9 +++ locator/token_metadata.hh | 8 +++ test/perf/CMakeLists.txt | 1 + test/perf/perf_sort_by_proximity.cc | 102 ++++++++++++++++++++++++++++ 5 files changed, 121 insertions(+) create mode 100644 test/perf/perf_sort_by_proximity.cc diff --git a/configure.py b/configure.py index 30df1cb02ba6..b24fdaa50ba6 100755 --- a/configure.py +++ b/configure.py @@ -584,6 +584,7 @@ def find_ninja(): 'test/perf/perf_idl', 'test/perf/perf_vint', 'test/perf/perf_big_decimal', + 'test/perf/perf_sort_by_proximity', ]) raft_tests = set([ diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index 37db2fde8c71..35c7efd7924b 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -1337,6 +1337,15 @@ future<> shared_token_metadata::mutate_token_metadata(seastar::noncopyable_funct set(make_token_metadata_ptr(std::move(tm))); } +void shared_token_metadata::mutate_token_metadata_for_test(seastar::noncopyable_function func) { + auto& tm = *_shared; + // bump the token_metadata ring_version + // to invalidate cached token/replication mappings + // when the modified token_metadata is committed. + tm.invalidate_cached_rings(); + func(tm); +} + future<> shared_token_metadata::mutate_on_all_shards(sharded& stm, seastar::noncopyable_function (token_metadata&)> func) { auto base_shard = this_shard_id(); SCYLLA_ASSERT(base_shard == 0); diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh index 038bb0e94452..0f604bdf8588 100644 --- a/locator/token_metadata.hh +++ b/locator/token_metadata.hh @@ -30,6 +30,8 @@ #include "locator/topology.hh" #include "locator/token_metadata_fwd.hh" +struct sort_by_proximity_topology; + // forward declaration since replica/database.hh includes this file namespace replica { class keyspace; @@ -482,6 +484,12 @@ public: // // Must be called on shard 0. static future<> mutate_on_all_shards(sharded& stm, seastar::noncopyable_function (token_metadata&)> func); + +private: + // for testing only, unsafe to be called without awaiting get_lock() first + void mutate_token_metadata_for_test(seastar::noncopyable_function func); + + friend struct ::sort_by_proximity_topology; }; } diff --git a/test/perf/CMakeLists.txt b/test/perf/CMakeLists.txt index 395b1cfed08c..d4b1d3dbe4d4 100644 --- a/test/perf/CMakeLists.txt +++ b/test/perf/CMakeLists.txt @@ -92,3 +92,4 @@ add_perf_test(perf_mutation_fragment) add_perf_test(perf_vint) add_perf_test(perf_row_cache_reads) add_perf_test(perf_s3_client) +add_perf_test(perf_sort_by_proximity) diff --git a/test/perf/perf_sort_by_proximity.cc b/test/perf/perf_sort_by_proximity.cc new file mode 100644 index 000000000000..6f86b54b06dd --- /dev/null +++ b/test/perf/perf_sort_by_proximity.cc @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2024-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include +#include +#include +#include +#include +#include "inet_address_vectors.hh" +#include "seastarx.hh" + +#include +#include + +#include "locator/token_metadata.hh" + +struct sort_by_proximity_topology { + static constexpr size_t DCS = 1; + static constexpr size_t RACKS_PER_DC = 3; + static constexpr size_t NODES_PER_RACK = 3; + static constexpr size_t NODES = DCS * RACKS_PER_DC * NODES_PER_RACK; + semaphore sem{1}; + std::optional stm; + std::unordered_map> nodes; + std::vector replica_sets; + + sort_by_proximity_topology() + { + locator::token_metadata::config tm_cfg; + gms::inet_address my_address("localhost"); + tm_cfg.topo_cfg.this_endpoint = my_address; + tm_cfg.topo_cfg.this_cql_address = my_address; + tm_cfg.topo_cfg.this_host_id = locator::host_id{utils::UUID(0, 1)}; + tm_cfg.topo_cfg.local_dc_rack = locator::endpoint_dc_rack::default_location; + + stm.emplace([this] () noexcept { return get_units(sem, 1); }, tm_cfg); + + unsigned i = 1; + stm->mutate_token_metadata_for_test([&] (locator::token_metadata& tm) { + auto& topology = tm.get_topology(); + for (size_t dc = 0; dc < DCS; ++dc) { + for (size_t rack = 0; rack < RACKS_PER_DC; ++rack) { + for (size_t node = 0; node < NODES_PER_RACK; ++node, ++i) { + auto id = locator::host_id{utils::UUID(0, i)}; + nodes[dc][rack].emplace_back(id); + topology.add_or_update_endpoint(id, + gms::inet_address((127u << 24) | i), + locator::endpoint_dc_rack{format("dc{}", dc), format("rack{}", rack)}, + locator::node::state::normal); + } + } + } + }); + auto num_replica_sets = std::pow(NODES_PER_RACK, RACKS_PER_DC); + replica_sets.reserve(num_replica_sets); + std::array node_in_dc; + std::ranges::fill(node_in_dc, 0); + std::array end_node_in_dc; + std::ranges::fill(end_node_in_dc, 0); + do { + host_id_vector_replica_set replicas; + replicas.reserve(DCS * RACKS_PER_DC); + for (size_t dc = 0; dc < DCS; ++dc) { + for (size_t rack = 0; rack < RACKS_PER_DC; ++rack) { + replicas.emplace_back(nodes[dc][rack][node_in_dc[rack]]); + } + } + replica_sets.emplace_back(std::move(replicas)); + for (size_t rack = 0; rack < RACKS_PER_DC; ++rack) { + if (++node_in_dc[rack] < NODES_PER_RACK) { + break; + } + node_in_dc[rack] = 0; + } + } while (node_in_dc != end_node_in_dc); + assert(replica_sets.size() == num_replica_sets); + } +}; + +PERF_TEST_F(sort_by_proximity_topology, perf_sort_by_proximity) +{ + const auto& topology = stm->get()->get_topology(); + size_t iterations = 0; + for (const auto& [dc, racks] : nodes) { + for (const auto& [rack, rack_nodes] : racks) { + for (auto node : rack_nodes) { + for (auto& replicas : replica_sets) { + topology.do_sort_by_proximity(node, replicas); + iterations++; + } + } + } + } + return iterations; +} From 3a3df43799db818c9a428891f30c88186a87eb6e Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Mon, 16 Dec 2024 13:56:26 +0200 Subject: [PATCH 080/397] storage_proxy: sort_endpoints_by_proximity: lookup my_id only if cannot sort by proximity topology::sort_by_proximity already sorts the local node address first, if present, so look it up only when using SimpleSnitch, where sort_by_proximity() is a no-op. Signed-off-by: Benny Halevy --- service/storage_proxy.cc | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 38e7991cbdfa..cac072226b4d 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -6610,11 +6610,15 @@ void storage_proxy::sort_endpoints_by_proximity(const locator::effective_replica return; } auto my_id = my_host_id(erm); - erm.get_topology().sort_by_proximity(my_id, ids); - // FIXME: before dynamic snitch is implement put local address (if present) at the beginning - auto it = std::ranges::find(ids, my_id); - if (it != ids.end() && it != ids.begin()) { - std::iter_swap(it, ids.begin()); + const auto& topology = erm.get_topology(); + if (topology.can_sort_by_proximity()) { + topology.do_sort_by_proximity(my_id, ids); + } else { + // FIXME: before dynamic snitch is implemented put local address (if present) at the beginning + auto it = std::ranges::find(ids, my_id); + if (it != ids.end() && it != ids.begin()) { + std::iter_swap(it, ids.begin()); + } } } From 4af522f61e30ff7ba819bab9e7846fd9d81342ef Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Fri, 20 Dec 2024 09:12:11 +0200 Subject: [PATCH 081/397] utils: small_vector: expose internal_capacity() So we can use it for defining other small_vector deriving their internal capacity from another small_vector type. Signed-off-by: Benny Halevy --- utils/small_vector.hh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/utils/small_vector.hh b/utils/small_vector.hh index 8ac37ffc5da3..86245781d940 100644 --- a/utils/small_vector.hh +++ b/utils/small_vector.hh @@ -251,6 +251,10 @@ public: } } + static constexpr size_t internal_capacity() noexcept { + return N; + } + size_t external_memory_usage() const { if (uses_internal_storage()) { return 0; From 0fe8bdd0db170f879da918b67bb38f2dc6ca5ba5 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Mon, 16 Dec 2024 14:39:02 +0200 Subject: [PATCH 082/397] locator/topology: sort_by_proximity: calculate distance only once And use a temporary vector to use the precalculated distances. A later patch will add some randomization to shuffle nodes at the same distance from the reference node. This improves the function performance by 50% for 3 replicas, from 77.4 ns to 39.2 ns, larger replica sets show greater improvement (over 4X for 15 nodes): Before: test iterations median mad min max allocs tasks inst cycles sort_by_proximity_topology.perf_sort_by_proximity 12808773 77.368ns 0.062ns 77.300ns 77.873ns 0.000 0.000 1194.2 231.6 After: sort_by_proximity_topology.perf_sort_by_proximity 25541973 39.225ns 0.114ns 38.966ns 39.339ns 0.000 0.000 588.5 116.6 Signed-off-by: Benny Halevy --- locator/topology.cc | 30 +++++++++----------- locator/topology.hh | 21 +++++++------- test/boost/network_topology_strategy_test.cc | 12 ++++++-- 3 files changed, 34 insertions(+), 29 deletions(-) diff --git a/locator/topology.cc b/locator/topology.cc index a46b843f31b8..afe0a81bb0ec 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -573,16 +573,20 @@ void topology::sort_by_proximity(locator::host_id address, host_id_vector_replic } void topology::do_sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const { - std::sort(addresses.begin(), addresses.end(), [this, &address](const locator::host_id& a1, const locator::host_id& a2) { - return compare_endpoints(address, a1, a2) < 0; - }); -} - -std::weak_ordering topology::compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const { const auto& loc = get_location(address); - const auto& loc1 = get_location(a1); - const auto& loc2 = get_location(a2); - + struct info { + locator::host_id id; + int distance; + }; + auto host_infos = addresses | std::views::transform([&] (locator::host_id id) { + const auto& loc1 = get_location(id); + return info{ id, distance(address, loc, id, loc1) }; + }) | std::ranges::to>(); + std::ranges::sort(host_infos, std::ranges::less{}, std::mem_fn(&info::distance)); + std::ranges::copy(host_infos | std::ranges::views::transform(std::mem_fn(&info::id)), addresses.begin()); +} + +int topology::distance(const locator::host_id& address, const endpoint_dc_rack& loc, const locator::host_id& a1, const endpoint_dc_rack& loc1) noexcept { // The farthest nodes from a given node are: // 1. Nodes in other DCs then the reference node // 2. Nodes in the other RACKs in the same DC as the reference node @@ -591,13 +595,7 @@ std::weak_ordering topology::compare_endpoints(const locator::host_id& address, int same_rack1 = same_dc1 & (loc1.rack == loc.rack); int same_node1 = a1 == address; int d1 = ((same_dc1 << 2) | (same_rack1 << 1) | same_node1) ^ 7; - - int same_dc2 = loc2.dc == loc.dc; - int same_rack2 = same_dc2 & (loc2.rack == loc.rack); - int same_node2 = a2 == address; - int d2 = ((same_dc2 << 2) | (same_rack2 << 1) | same_node2) ^ 7; - - return d1 <=> d2; + return d1; } void topology::for_each_node(std::function func) const { diff --git a/locator/topology.hh b/locator/topology.hh index 1521aae481c2..98f487314b30 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -369,6 +369,16 @@ public: */ void do_sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const; + /** + * Calculates topology-distance between two endpoints. + * + * The closest nodes to a given node are: + * 1. The node itself + * 2. Nodes in the same RACK + * 3. Nodes in the same DC + */ + static int distance(const locator::host_id& address, const endpoint_dc_rack& loc, const locator::host_id& address1, const endpoint_dc_rack& loc1) noexcept; + // Executes a function for each node in a state other than "none" and "left". void for_each_node(std::function func) const; @@ -413,17 +423,6 @@ private: return const_cast(nptr); } - /** - * compares two endpoints in relation to the target endpoint, returning as - * Comparator.compare would - * - * The closest nodes to a given node are: - * 1. The node itself - * 2. Nodes in the same RACK as the reference node - * 3. Nodes in the same DC as the reference node - */ - std::weak_ordering compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const; - unsigned _shard; config _cfg; const node* _this_node = nullptr; diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index 8d3df681d183..d80ddf27a87d 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -946,6 +946,14 @@ SEASTAR_TEST_CASE(test_invalid_dcs) { namespace locator { +std::weak_ordering compare_endpoints(const locator::topology& topo, const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) { + const auto& loc = topo.get_location(address); + const auto& loc1 = topo.get_location(a1); + const auto& loc2 = topo.get_location(a2); + + return topo.distance(address, loc, a1, loc1) <=> topo.distance(address, loc, a2, loc2); +} + void topology::test_compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const { std::optional expected; const auto& loc = get_location(address); @@ -976,7 +984,7 @@ void topology::test_compare_endpoints(const locator::host_id& address, const loc } } } - auto res = compare_endpoints(address, a1, a2); + auto res = compare_endpoints(*this, address, a1, a2); testlog.debug("compare_endpoint: address={} [{}/{}] a1={} [{}/{}] a2={} [{}/{}]: res={} expected={} expected_value={}", address, loc.dc, loc.rack, a1, loc1.dc, loc1.rack, @@ -1001,7 +1009,7 @@ void topology::test_sort_by_proximity(const locator::host_id& address, const hos } // Test sort monotonicity for (size_t i = 1; i < sorted_nodes.size(); ++i) { - BOOST_REQUIRE(compare_endpoints(address, sorted_nodes[i-1], sorted_nodes[i]) <= 0); + BOOST_REQUIRE(compare_endpoints(*this, address, sorted_nodes[i-1], sorted_nodes[i]) <= 0); } } From d1490bb7bfcdabe36ff50d88379e7edb7cbe2ddf Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Tue, 17 Dec 2024 18:27:43 +0200 Subject: [PATCH 083/397] locator/topology: do_sort_by_proximity: shuffle equal-distance replicas To improve balancing when reading in 1 < CL < ALL This implementation has a moderate impact on the function performance in contrast to full std::shuffle of the vector before stable_sort:ing it (especially with large number of nodes to sort). Before: test iterations median mad min max allocs tasks inst cycles sort_by_proximity_topology.perf_sort_by_proximity 25541973 39.225ns 0.114ns 38.966ns 39.339ns 0.000 0.000 588.5 116.6 After: sort_by_proximity_topology.perf_sort_by_proximity 19689561 50.195ns 0.119ns 50.076ns 51.145ns 0.000 0.000 622.5 150.6 Signed-off-by: Benny Halevy --- locator/topology.cc | 30 ++++++++++++++++++++++++++--- locator/topology.hh | 10 ++++++++++ test/perf/perf_sort_by_proximity.cc | 4 ++++ 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/locator/topology.cc b/locator/topology.cc index afe0a81bb0ec..7371c10df0a6 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -6,11 +6,14 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ +#include +#include +#include + #include #include #include #include -#include #include #include "utils/log.hh" @@ -108,6 +111,7 @@ topology::topology(config cfg) : _shard(this_shard_id()) , _cfg(cfg) , _sort_by_proximity(!cfg.disable_proximity_sorting) + , _random_engine(std::random_device{}()) { tlogger.trace("topology[{}]: constructing using config: endpoint={} id={} dc={} rack={}", fmt::ptr(this), cfg.this_endpoint, cfg.this_host_id, cfg.local_dc_rack.dc, cfg.local_dc_rack.rack); @@ -127,6 +131,7 @@ topology::topology(topology&& o) noexcept , _dc_racks(std::move(o._dc_racks)) , _sort_by_proximity(o._sort_by_proximity) , _datacenters(std::move(o._datacenters)) + , _random_engine(std::move(o._random_engine)) { SCYLLA_ASSERT(_shard == this_shard_id()); tlogger.trace("topology[{}]: move from [{}]", fmt::ptr(this), fmt::ptr(&o)); @@ -178,6 +183,7 @@ future topology::clone_gently() const { co_await coroutine::maybe_yield(); } ret._sort_by_proximity = _sort_by_proximity; + ret._random_engine = _random_engine; co_return ret; } @@ -580,10 +586,24 @@ void topology::do_sort_by_proximity(locator::host_id address, host_id_vector_rep }; auto host_infos = addresses | std::views::transform([&] (locator::host_id id) { const auto& loc1 = get_location(id); - return info{ id, distance(address, loc, id, loc1) }; + return info{id, distance(address, loc, id, loc1)}; }) | std::ranges::to>(); std::ranges::sort(host_infos, std::ranges::less{}, std::mem_fn(&info::distance)); - std::ranges::copy(host_infos | std::ranges::views::transform(std::mem_fn(&info::id)), addresses.begin()); + auto dst = addresses.begin(); + auto it = host_infos.begin(); + auto prev = it; + auto shuffler = _random_engine(); + for (++it; it < host_infos.end(); ++it, ++dst) { + if (prev->distance == it->distance) { + if (shuffler & 1) { + std::swap(prev->id, it->id); + } + shuffler = std::rotr(shuffler, 1); + } + *dst = prev->id; + prev = it; + } + *dst = prev->id; } int topology::distance(const locator::host_id& address, const endpoint_dc_rack& loc, const locator::host_id& a1, const endpoint_dc_rack& loc1) noexcept { @@ -635,6 +655,10 @@ topology::get_datacenter_host_ids() const { return ret; } +void topology::seed_random_engine(random_engine_type::result_type value) { + _random_engine.seed(value); +} + } // namespace locator namespace std { diff --git a/locator/topology.hh b/locator/topology.hh index 98f487314b30..db5921e761fe 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -27,6 +28,8 @@ using namespace seastar; +struct sort_by_proximity_topology; + namespace locator { class topology; } @@ -409,6 +412,8 @@ public: } private: + using random_engine_type = std::mt19937_64; + bool is_configured_this_node(const node&) const; const node& add_node(node_holder node); void remove_node(const node& node); @@ -423,6 +428,8 @@ private: return const_cast(nptr); } + void seed_random_engine(random_engine_type::result_type); + unsigned _shard; config _cfg; const node* _this_node = nullptr; @@ -455,7 +462,10 @@ private: return _nodes_by_endpoint; }; + mutable random_engine_type _random_engine; + friend class token_metadata_impl; + friend struct ::sort_by_proximity_topology; public: void test_compare_endpoints(const locator::host_id& address, const locator::host_id& a1, const locator::host_id& a2) const; void test_sort_by_proximity(const locator::host_id& address, const host_id_vector_replica_set& nodes) const; diff --git a/test/perf/perf_sort_by_proximity.cc b/test/perf/perf_sort_by_proximity.cc index 6f86b54b06dd..4d86bcea1df6 100644 --- a/test/perf/perf_sort_by_proximity.cc +++ b/test/perf/perf_sort_by_proximity.cc @@ -20,6 +20,8 @@ #include #include "locator/token_metadata.hh" +#include "test/lib/log.hh" +#include "test/lib/random_utils.hh" struct sort_by_proximity_topology { static constexpr size_t DCS = 1; @@ -57,6 +59,8 @@ struct sort_by_proximity_topology { } } } + auto seed = tests::random::get_int(); + topology.seed_random_engine(seed); }); auto num_replica_sets = std::pow(NODES_PER_RACK, RACKS_PER_DC); replica_sets.reserve(num_replica_sets); From 5eb3278d9e2ac036db4c270a1428c6461b462f57 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Mon, 25 Nov 2024 16:29:07 +0300 Subject: [PATCH 084/397] api: Use built_views table in get_built_indexes API Somehow system."IndexInfo" table and column_family/built_indexes REST API endpoint declare an index "built" at slightly different times: The former a virtual table which declares an index completely built when it appears on the system.built_views table. The latter uses different data -- it takes the list of indexes in the schema and eliminates indexes which are still listed in the system.scylla_views_builds_in_progress table. The mentioned system. tables are updated at different times, so API notices the change a bit later. It's worth improving the consistency of these two APIs by making the REST API endpoint piggy-back the load_built_views() instead of load_view_build_progress(). With that change the filtering of indexes should be negated. Fixes #21587 Signed-off-by: Pavel Emelyanov --- api/column_family.cc | 10 ++++++---- index/built_indexes_virtual_reader.hh | 2 ++ 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/api/column_family.cc b/api/column_family.cc index ba47c02e065f..5ffccd3f04d7 100644 --- a/api/column_family.cc +++ b/api/column_family.cc @@ -993,11 +993,13 @@ void set_column_family(http_context& ctx, routes& r, shardedget_path_param("name")); auto&& ks = std::get<0>(ks_cf); auto&& cf_name = std::get<1>(ks_cf); - return sys_ks.local().load_view_build_progress().then([ks, cf_name, &ctx](const std::vector& vb) mutable { + // Use of load_built_views() as filtering table should be in sync with + // built_indexes_virtual_reader filtering with BUILT_VIEWS table + return sys_ks.local().load_built_views().then([ks, cf_name, &ctx](const std::vector& vb) mutable { std::set vp; for (auto b : vb) { - if (b.view.first == ks) { - vp.insert(b.view.second); + if (b.first == ks) { + vp.insert(b.second); } } std::vector res; @@ -1005,7 +1007,7 @@ void set_column_family(http_context& ctx, routes& r, sharded( _db, s, From 644d36996daf8a539c1a8ed7a53cb36d660e14f3 Mon Sep 17 00:00:00 2001 From: Pavel Emelyanov Date: Tue, 12 Nov 2024 16:24:25 +0300 Subject: [PATCH 085/397] test: Add tests for MVs and indexes reporting by API endpoint(s) So far there's the /column_family/built_indexes one that reports the index names similar to how system.IndexInfo does, but it's not tested. This patch adds tests next to existing system. table ones. Signed-off-by: Pavel Emelyanov --- test/cqlpy/test_materialized_view.py | 15 +++++++++++++++ test/cqlpy/test_secondary_index.py | 10 ++++++++++ 2 files changed, 25 insertions(+) diff --git a/test/cqlpy/test_materialized_view.py b/test/cqlpy/test_materialized_view.py index d1b85bafbb17..866df33131f4 100644 --- a/test/cqlpy/test_materialized_view.py +++ b/test/cqlpy/test_materialized_view.py @@ -7,6 +7,7 @@ import time import re import pytest +from . import rest_api from .util import new_test_table, unique_name, new_materialized_view, ScyllaMetrics, new_secondary_index from cassandra.protocol import ConfigurationException, InvalidRequest, SyntaxException @@ -1672,3 +1673,17 @@ def test_view_update_with_ttl(cql, test_keyspace): cql.execute(f'update {table} using ttl 1 set x=5 where p=1') time.sleep(1.1) assert [] == list(cql.execute(f'select * from {mv}')) + +# Test view representation in REST API +def test_view_in_API(cql, test_keyspace): + with new_test_table(cql, test_keyspace, "p int PRIMARY KEY, v int") as base: + with new_materialized_view(cql, base, '*', 'v,p', 'v is not null and p is not null') as view: + view_name = view.split('.')[1] + res = rest_api.get_request(cql, f"storage_service/view_build_statuses/{test_keyspace}/{view_name}") + assert len(res) == 1 and 'value' in res[0] and res[0]['value'] in [ 'UNKNOWN', 'STARTED', 'SUCCESS' ] + # Indexes are implemented on top of materialized-views, but even then no MVs + # should appear in the output of built_indexes API. And since this API only + # reports views that are built, check that view is built first. + wait_for_view_built(cql, view) + res = rest_api.get_request(cql, f"column_family/built_indexes/{base.replace('.',':')}") + assert view_name not in res diff --git a/test/cqlpy/test_secondary_index.py b/test/cqlpy/test_secondary_index.py index 6cfc36416d93..1a9f9eebc758 100644 --- a/test/cqlpy/test_secondary_index.py +++ b/test/cqlpy/test_secondary_index.py @@ -10,6 +10,7 @@ import tempfile import pytest import os +from . import rest_api from cassandra.protocol import SyntaxException, AlreadyExists, InvalidRequest, ConfigurationException, ReadFailure, WriteFailure from cassandra.query import SimpleStatement from .cassandra_tests.porting import assert_rows, assert_row_count, assert_rows_ignoring_order, assert_empty @@ -1971,3 +1972,12 @@ def test_index_in_system_tables(cql, test_keyspace): assert f'{test_keyspace}::{index_name}' in res res = cql.execute(f'select * from system."IndexInfo" where table_name = \'{test_keyspace}\' AND index_name = \'{index_name}\'').one() assert (test_keyspace, index_name) == (res.table_name, res.index_name) + +# Test index representation in REST API +def test_index_in_API(cql, test_keyspace): + with new_test_table(cql, test_keyspace, "p int PRIMARY KEY, v int") as table: + index_name = unique_name() + cql.execute(f"CREATE INDEX {index_name} ON {table}(v)") + wait_for_index(cql, test_keyspace, index_name) + res = rest_api.get_request(cql, f"column_family/built_indexes/{table.replace('.',':')}") + assert index_name in res From f337ecbafa30ac2d3ee7217e9bd37efdcd08683c Mon Sep 17 00:00:00 2001 From: Evgeniy Naydanov Date: Tue, 24 Dec 2024 14:24:09 +0000 Subject: [PATCH 086/397] test.py: topology_random_failures: handle more node's hangs during 30s sleep The node is hanging and the coordinator just rollback a topology state. It's different from `stop_after_sending_join_node_request` and `stop_after_bootstrapping_initial_raft_configuration` because in these cases the coordinator just not able to start the topology change at all and a message in the coordinator's log is different. Error injections handled: - `stop_after_updating_cdc_generation` - `stop_before_streaming` And, actually, it can be any cluster event which lasts more than 30s. --- .../error_injections.py | 8 ++++++++ .../test_random_failures.py | 17 +++++++++++------ 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/test/topology_random_failures/error_injections.py b/test/topology_random_failures/error_injections.py index 55fafab3da99..817261bda1e8 100644 --- a/test/topology_random_failures/error_injections.py +++ b/test/topology_random_failures/error_injections.py @@ -27,3 +27,11 @@ "stop_after_streaming", "stop_after_bootstrapping_initial_raft_configuration", ) + +# Error injections which can cause a node's hang due to some timeouts. +ERROR_INJECTIONS_NODE_MAY_HANG = ( + "stop_after_sending_join_node_request", + "stop_after_updating_cdc_generation", + "stop_before_streaming", + "stop_after_bootstrapping_initial_raft_configuration", +) diff --git a/test/topology_random_failures/test_random_failures.py b/test/topology_random_failures/test_random_failures.py index c74c1fc507e8..1d7a8ada2ee8 100644 --- a/test/topology_random_failures/test_random_failures.py +++ b/test/topology_random_failures/test_random_failures.py @@ -23,7 +23,7 @@ from test.topology.conftest import skip_mode from test.pylib.internal_types import ServerUpState from test.topology_random_failures.cluster_events import CLUSTER_EVENTS, TOPOLOGY_TIMEOUT -from test.topology_random_failures.error_injections import ERROR_INJECTIONS +from test.topology_random_failures.error_injections import ERROR_INJECTIONS, ERROR_INJECTIONS_NODE_MAY_HANG if TYPE_CHECKING: from test.pylib.random_tables import RandomTables @@ -150,14 +150,19 @@ async def test_random_failures(manager: ManagerClient, server_log = await manager.server_open_log(server_id=s_info.server_id) - if cluster_event_duration + 1 >= WAIT_FOR_IP_TIMEOUT and error_injection in ( # give one more second for a tolerance - "stop_after_sending_join_node_request", - "stop_after_bootstrapping_initial_raft_configuration", - ): + if cluster_event_duration + 1 >= WAIT_FOR_IP_TIMEOUT and error_injection in ERROR_INJECTIONS_NODE_MAY_HANG: LOGGER.info("Expecting the added node can hang and we'll have a message in the coordinator's log. See #18638.") coordinator = await get_coordinator_host(manager=manager) coordinator_log = await manager.server_open_log(server_id=coordinator.server_id) - if matches := await coordinator_log.grep(r"The node may hang\. It's safe to shut it down manually now\."): + coordinator_log_pattern = r"The node may hang\. It's safe to shut it down manually now\." + if matches := await server_log.grep(r"init - Setting local host id to (?P[0-9a-f-]+)"): + line, match = matches[-1] + LOGGER.info("Found following message in the coordinator's log:\n\t%s", line) + coordinator_log_pattern += ( + rf"|updating topology state: rollback {match.group('hostid')} after bootstrapping failure, moving" + rf" transition state to left token ring and setting cleanup flag" + ) + if matches := await coordinator_log.grep(coordinator_log_pattern): LOGGER.info("Found following message in the coordinator's log:\n\t%s", matches[-1][0]) await manager.server_stop(server_id=s_info.server_id) From 5992e8b031526234e9b3f0d3951aec79c64d0c58 Mon Sep 17 00:00:00 2001 From: Evgeniy Naydanov Date: Tue, 24 Dec 2024 14:44:44 +0000 Subject: [PATCH 087/397] test.py: topology_random_failures: more deselects for #21534 More cases found which can cause the same 'local_is_initialized()' assertion during the node's bootstrap. --- .../cluster_events.py | 43 ++++++++++++++++++- 1 file changed, 42 insertions(+), 1 deletion(-) diff --git a/test/topology_random_failures/cluster_events.py b/test/topology_random_failures/cluster_events.py index b09a2f50d4e2..017f507c2ab9 100644 --- a/test/topology_random_failures/cluster_events.py +++ b/test/topology_random_failures/cluster_events.py @@ -67,6 +67,14 @@ def add_deselected_metadata(fn: Callable[P, T]) -> Callable[P, T]: # >>> await anext(cluster_event, None) +@deselect_for( + # TODO: remove this skip when #21534 will be resolved. + error_injections=[ + "stop_after_setting_mode_to_normal_raft_topology", + "stop_before_becoming_raft_voter", + ], + reason="See issue #21534 (assertion 'local_is_initialized()' failed during shutdown after a failed boot)", +) async def sleep_for_30_seconds(manager: ManagerClient, random_tables: RandomTables, error_injection: str) -> AsyncIterator[None]: @@ -525,6 +533,14 @@ async def remove_node(manager: ManagerClient, yield +@deselect_for( + # TODO: remove this skip when #21534 will be resolved. + error_injections=[ + "stop_after_setting_mode_to_normal_raft_topology", + "stop_before_becoming_raft_voter", + ], + reason="See issue #21534 (assertion 'local_is_initialized()' failed during shutdown after a failed boot)", +) async def restart_non_coordinator_node(manager: ManagerClient, random_tables: RandomTables, error_injection: str) -> AsyncIterator[None]: @@ -536,6 +552,14 @@ async def restart_non_coordinator_node(manager: ManagerClient, yield +@deselect_for( + # TODO: remove this skip when #21534 will be resolved. + error_injections=[ + "stop_after_setting_mode_to_normal_raft_topology", + "stop_before_becoming_raft_voter", + ], + reason="See issue #21534 (assertion 'local_is_initialized()' failed during shutdown after a failed boot)", +) async def restart_coordinator_node(manager: ManagerClient, random_tables: RandomTables, error_injection: str) -> AsyncIterator[None]: @@ -546,7 +570,14 @@ async def restart_coordinator_node(manager: ManagerClient, yield - +@deselect_for( + # TODO: remove this skip when #21534 will be resolved. + error_injections=[ + "stop_after_setting_mode_to_normal_raft_topology", + "stop_before_becoming_raft_voter", + ], + reason="See issue #21534 (assertion 'local_is_initialized()' failed during shutdown after a failed boot)", +) async def stop_non_coordinator_node_gracefully(manager: ManagerClient, random_tables: RandomTables, error_injection: str) -> AsyncIterator[None]: @@ -562,6 +593,7 @@ async def stop_non_coordinator_node_gracefully(manager: ManagerClient, # TODO: remove this skip when #21534 will be resolved. error_injections=[ "stop_after_setting_mode_to_normal_raft_topology", + "stop_before_becoming_raft_voter", ], reason="See issue #21534 (assertion 'local_is_initialized()' failed during shutdown after a failed boot)", ) @@ -576,6 +608,14 @@ async def stop_coordinator_node_gracefully(manager: ManagerClient, yield +@deselect_for( + # TODO: remove this skip when #21534 will be resolved. + error_injections=[ + "stop_after_setting_mode_to_normal_raft_topology", + "stop_before_becoming_raft_voter", + ], + reason="See issue #21534 (assertion 'local_is_initialized()' failed during shutdown after a failed boot)", +) async def kill_non_coordinator_node(manager: ManagerClient, random_tables: RandomTables, error_injection: str) -> AsyncIterator[None]: @@ -594,6 +634,7 @@ async def kill_non_coordinator_node(manager: ManagerClient, # TODO: remove this skip when #21534 will be resolved. error_injections=[ "stop_after_setting_mode_to_normal_raft_topology", + "stop_before_becoming_raft_voter", ], reason="See issue #21534 (assertion 'local_is_initialized()' failed during shutdown after a failed boot)", ) From 465449e4a1e66ddc44dae0071559400ce852e390 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Wed, 25 Dec 2024 13:53:54 +0200 Subject: [PATCH 088/397] test: combined_test: relicense Was inadvertantly released under the AGPL. --- test/boost/combined_tests.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/boost/combined_tests.cc b/test/boost/combined_tests.cc index fa4a06c216ed..f7b30f84df11 100644 --- a/test/boost/combined_tests.cc +++ b/test/boost/combined_tests.cc @@ -3,7 +3,7 @@ */ /* - * SPDX-License-Identifier: AGPL-3.0-or-later + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ #include "test/lib/scylla_test_case.hh" From 0fc7e786ddfd6efaeab38ce0a7997bf66de69769 Mon Sep 17 00:00:00 2001 From: Yaron Kaikov Date: Tue, 24 Dec 2024 08:16:36 +0200 Subject: [PATCH 089/397] .github/scripts/auto-backport.py: fix wrong username param In 2e6755ecca76d6d358dc6a8110d308cfd5f6db37 I have added a comment when PR has conflicts so the assignee can get a notification about it. There was a problem with the user mention param (a missing `.login`) Fixing it Closes scylladb/scylladb#22036 --- .github/scripts/auto-backport.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/scripts/auto-backport.py b/.github/scripts/auto-backport.py index 032bec1821f1..f992799746e9 100755 --- a/.github/scripts/auto-backport.py +++ b/.github/scripts/auto-backport.py @@ -49,7 +49,7 @@ def create_pull_request(repo, new_branch_name, base_branch_name, pr, backport_pr backport_pr.add_to_assignees(pr.user) if is_draft: backport_pr.add_to_labels("conflicts") - pr_comment = f"@{pr.user} - This PR was marked as draft because it has conflicts\n" + pr_comment = f"@{pr.user.login} - This PR was marked as draft because it has conflicts\n" pr_comment += "Please resolve them and mark this PR as ready for review" backport_pr.create_issue_comment(pr_comment) logging.info(f"Assigned PR to original author: {pr.user}") From d87e1eb7ef2e18059d6964b967d31125ea0da5f6 Mon Sep 17 00:00:00 2001 From: Konstantin Osipov Date: Fri, 20 Dec 2024 13:24:24 -0500 Subject: [PATCH 090/397] test: merge topology_experimental_raft into topology_custom This enables tablets in topology_custom, so explicitly disable them where tests don't support tablets. In scope of this rename patch a few imports. Importing dependencies from another test is a bad idea - please use shared libraries instead. Fixed #20193 Closes scylladb/scylladb#22014 --- test/topology_custom/suite.yaml | 11 ++++++++++ .../test_alternator.py | 0 .../test_blocked_bootstrap.py | 0 .../test_boot_after_ip_change.py | 3 ++- .../test_cdc_generation_clearing.py | 0 .../test_cdc_generation_data.py | 0 .../test_cdc_generation_publishing.py | 0 ...test_crash_coordinator_before_streaming.py | 0 .../test_different_group0_ids.py | 5 +++-- .../test_fencing.py | 0 test/topology_custom/test_gossip_boot.py | 3 ++- .../test_mv_admission_control.py | 2 +- test/topology_custom/test_mv_backlog.py | 2 +- .../test_mv_tablets.py | 0 .../test_mv_tablets_replace.py | 2 +- .../test_node_isolation.py | 0 .../test_not_enough_token_owners.py | 0 .../test_raft_cluster_features.py | 0 .../test_raft_ignore_nodes.py | 0 test/topology_custom/test_read_repair.py | 5 ++++- .../test_replace_ignore_nodes.py | 3 ++- .../test_restart_cluster.py | 0 test/topology_custom/test_table_drop.py | 3 ++- .../test_tablet_repair_scheduler.py | 0 .../test_tablets2.py} | 0 .../test_tablets_intranode.py | 0 .../test_tablets_merge.py | 0 .../test_tablets_removenode.py | 0 .../test_tombstone_gc.py | 0 .../test_topology_ops.py | 0 .../test_topology_recovery_basic.py | 0 .../test_topology_recovery_majority_loss.py | 0 .../test_topology_remove_garbage_group0.py | 3 ++- .../test_topology_upgrade.py | 0 ..._upgrade_not_stuck_after_recent_removal.py | 0 .../test_zero_token_nodes_multidc.py | 0 .../test_zero_token_nodes_no_replication.py | 0 .../test_zero_token_nodes_topology_ops.py | 4 +++- test/topology_experimental_raft/__init__.py | 0 test/topology_experimental_raft/conftest.py | 9 --------- test/topology_experimental_raft/suite.yaml | 20 ------------------- test/topology_tasks/test_tablet_tasks.py | 2 +- 42 files changed, 35 insertions(+), 42 deletions(-) rename test/{topology_experimental_raft => topology_custom}/test_alternator.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_blocked_bootstrap.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_cdc_generation_clearing.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_cdc_generation_data.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_cdc_generation_publishing.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_crash_coordinator_before_streaming.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_fencing.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_mv_tablets.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_mv_tablets_replace.py (98%) rename test/{topology_experimental_raft => topology_custom}/test_node_isolation.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_not_enough_token_owners.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_raft_cluster_features.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_raft_ignore_nodes.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_restart_cluster.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_tablet_repair_scheduler.py (100%) rename test/{topology_experimental_raft/test_tablets.py => topology_custom/test_tablets2.py} (100%) rename test/{topology_experimental_raft => topology_custom}/test_tablets_intranode.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_tablets_merge.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_tablets_removenode.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_tombstone_gc.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_topology_ops.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_topology_recovery_basic.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_topology_recovery_majority_loss.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_topology_upgrade.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_topology_upgrade_not_stuck_after_recent_removal.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_zero_token_nodes_multidc.py (100%) rename test/{topology_experimental_raft => topology_custom}/test_zero_token_nodes_no_replication.py (100%) delete mode 100644 test/topology_experimental_raft/__init__.py delete mode 100644 test/topology_experimental_raft/conftest.py delete mode 100644 test/topology_experimental_raft/suite.yaml diff --git a/test/topology_custom/suite.yaml b/test/topology_custom/suite.yaml index 76e57a8a5fca..fe5a5e3acd70 100644 --- a/test/topology_custom/suite.yaml +++ b/test/topology_custom/suite.yaml @@ -5,18 +5,29 @@ cluster: extra_scylla_config_options: authenticator: AllowAllAuthenticator authorizer: AllowAllAuthorizer + enable_user_defined_functions: False + enable_tablets: True run_first: - test_raft_recovery_stuck - test_raft_recovery_basic - test_group0_schema_versioning - test_tablets_migration - test_zero_token_nodes_topology_ops + - test_raft_cluster_features + - test_raft_ignore_nodes + - test_tablets + - test_tablets2 +skip_in_release: + - test_raft_cluster_features skip_in_debug: - test_shutdown_hang - test_replace - test_old_ip_notification_repro - test_node_shutdown_waits_for_pending_requests + - test_cdc_generation_clearing + - test_cdc_generation_publishing run_in_dev: + - test_raft_ignore_nodes - test_group0_schema_versioning - test_different_group0_ids - test_replace_ignore_nodes diff --git a/test/topology_experimental_raft/test_alternator.py b/test/topology_custom/test_alternator.py similarity index 100% rename from test/topology_experimental_raft/test_alternator.py rename to test/topology_custom/test_alternator.py diff --git a/test/topology_experimental_raft/test_blocked_bootstrap.py b/test/topology_custom/test_blocked_bootstrap.py similarity index 100% rename from test/topology_experimental_raft/test_blocked_bootstrap.py rename to test/topology_custom/test_blocked_bootstrap.py diff --git a/test/topology_custom/test_boot_after_ip_change.py b/test/topology_custom/test_boot_after_ip_change.py index 713d61fb883c..182450fe8c0a 100644 --- a/test/topology_custom/test_boot_after_ip_change.py +++ b/test/topology_custom/test_boot_after_ip_change.py @@ -22,7 +22,8 @@ async def test_boot_after_ip_change(manager: ManagerClient) -> None: Regression test for #14468. Does not apply to Raft-topology mode. """ cfg = {'enable_user_defined_functions': False, - 'force_gossip_topology_changes': True} + 'force_gossip_topology_changes': True, + 'enable_tablets': False} logger.info(f"Booting initial cluster") servers = [await manager.server_add(config=cfg) for _ in range(2)] await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) diff --git a/test/topology_experimental_raft/test_cdc_generation_clearing.py b/test/topology_custom/test_cdc_generation_clearing.py similarity index 100% rename from test/topology_experimental_raft/test_cdc_generation_clearing.py rename to test/topology_custom/test_cdc_generation_clearing.py diff --git a/test/topology_experimental_raft/test_cdc_generation_data.py b/test/topology_custom/test_cdc_generation_data.py similarity index 100% rename from test/topology_experimental_raft/test_cdc_generation_data.py rename to test/topology_custom/test_cdc_generation_data.py diff --git a/test/topology_experimental_raft/test_cdc_generation_publishing.py b/test/topology_custom/test_cdc_generation_publishing.py similarity index 100% rename from test/topology_experimental_raft/test_cdc_generation_publishing.py rename to test/topology_custom/test_cdc_generation_publishing.py diff --git a/test/topology_experimental_raft/test_crash_coordinator_before_streaming.py b/test/topology_custom/test_crash_coordinator_before_streaming.py similarity index 100% rename from test/topology_experimental_raft/test_crash_coordinator_before_streaming.py rename to test/topology_custom/test_crash_coordinator_before_streaming.py diff --git a/test/topology_custom/test_different_group0_ids.py b/test/topology_custom/test_different_group0_ids.py index 9ba514b65d44..bebfb75ee04f 100644 --- a/test/topology_custom/test_different_group0_ids.py +++ b/test/topology_custom/test_different_group0_ids.py @@ -28,8 +28,9 @@ async def test_different_group0_ids(manager: ManagerClient): """ # Consistent topology changes are disabled to use repair based node operations. - scylla_a = await manager.server_add(config={'force_gossip_topology_changes': True}) - scylla_b = await manager.server_add(start=False, config={'force_gossip_topology_changes': True}) + cfg = {'force_gossip_topology_changes': True, 'enable_tablets': False} + scylla_a = await manager.server_add(config = cfg) + scylla_b = await manager.server_add(start=False, config = cfg) await manager.server_start(scylla_b.server_id, seeds=[scylla_b.ip_addr]) await manager.server_stop(scylla_b.server_id) diff --git a/test/topology_experimental_raft/test_fencing.py b/test/topology_custom/test_fencing.py similarity index 100% rename from test/topology_experimental_raft/test_fencing.py rename to test/topology_custom/test_fencing.py diff --git a/test/topology_custom/test_gossip_boot.py b/test/topology_custom/test_gossip_boot.py index 954bbe5f05c3..47fb90612e10 100644 --- a/test/topology_custom/test_gossip_boot.py +++ b/test/topology_custom/test_gossip_boot.py @@ -12,7 +12,8 @@ async def test_gossip_boot(manager: ManagerClient): """ cfg = {'error_injections_at_startup': ['gossiper_replicate_sleep'], - 'force_gossip_topology_changes': True} + 'force_gossip_topology_changes': True, + 'enable_tablets': False} servers = [await manager.server_add(config=cfg, timeout=60) for _ in range(3)] logs = [await manager.server_open_log(s.server_id) for s in servers] diff --git a/test/topology_custom/test_mv_admission_control.py b/test/topology_custom/test_mv_admission_control.py index 32eecf9e98e6..af569fbf283d 100644 --- a/test/topology_custom/test_mv_admission_control.py +++ b/test/topology_custom/test_mv_admission_control.py @@ -12,7 +12,7 @@ from test.topology.conftest import skip_mode from test.pylib.util import wait_for_view -from test.topology_experimental_raft.test_mv_tablets import pin_the_only_tablet, get_tablet_replicas +from test.topology_custom.test_mv_tablets import pin_the_only_tablet, get_tablet_replicas from cassandra.cluster import ConsistencyLevel, EXEC_PROFILE_DEFAULT # type: ignore from cassandra.cqltypes import Int32Type # type: ignore diff --git a/test/topology_custom/test_mv_backlog.py b/test/topology_custom/test_mv_backlog.py index c3a5e2e0ce10..13339f1d76e2 100644 --- a/test/topology_custom/test_mv_backlog.py +++ b/test/topology_custom/test_mv_backlog.py @@ -11,7 +11,7 @@ import pytest from test.topology.conftest import skip_mode from test.pylib.util import wait_for_view, wait_for -from test.topology_experimental_raft.test_mv_tablets import pin_the_only_tablet +from test.topology_custom.test_mv_tablets import pin_the_only_tablet from test.pylib.tablets import get_tablet_replica logger = logging.getLogger(__name__) diff --git a/test/topology_experimental_raft/test_mv_tablets.py b/test/topology_custom/test_mv_tablets.py similarity index 100% rename from test/topology_experimental_raft/test_mv_tablets.py rename to test/topology_custom/test_mv_tablets.py diff --git a/test/topology_experimental_raft/test_mv_tablets_replace.py b/test/topology_custom/test_mv_tablets_replace.py similarity index 98% rename from test/topology_experimental_raft/test_mv_tablets_replace.py rename to test/topology_custom/test_mv_tablets_replace.py index c6bdd8128a44..a29e5e5657ff 100644 --- a/test/topology_experimental_raft/test_mv_tablets_replace.py +++ b/test/topology_custom/test_mv_tablets_replace.py @@ -18,7 +18,7 @@ from test.topology.conftest import skip_mode from test.topology.util import get_topology_coordinator, find_server_by_host_id -from test.topology_experimental_raft.test_mv_tablets import get_tablet_replicas +from test.topology_custom.test_mv_tablets import get_tablet_replicas logger = logging.getLogger(__name__) diff --git a/test/topology_experimental_raft/test_node_isolation.py b/test/topology_custom/test_node_isolation.py similarity index 100% rename from test/topology_experimental_raft/test_node_isolation.py rename to test/topology_custom/test_node_isolation.py diff --git a/test/topology_experimental_raft/test_not_enough_token_owners.py b/test/topology_custom/test_not_enough_token_owners.py similarity index 100% rename from test/topology_experimental_raft/test_not_enough_token_owners.py rename to test/topology_custom/test_not_enough_token_owners.py diff --git a/test/topology_experimental_raft/test_raft_cluster_features.py b/test/topology_custom/test_raft_cluster_features.py similarity index 100% rename from test/topology_experimental_raft/test_raft_cluster_features.py rename to test/topology_custom/test_raft_cluster_features.py diff --git a/test/topology_experimental_raft/test_raft_ignore_nodes.py b/test/topology_custom/test_raft_ignore_nodes.py similarity index 100% rename from test/topology_experimental_raft/test_raft_ignore_nodes.py rename to test/topology_custom/test_raft_ignore_nodes.py diff --git a/test/topology_custom/test_read_repair.py b/test/topology_custom/test_read_repair.py index fe2f5abddc9a..2211cd7f85e0 100644 --- a/test/topology_custom/test_read_repair.py +++ b/test/topology_custom/test_read_repair.py @@ -222,7 +222,10 @@ async def test_incremental_read_repair(data_class, workdir, manager): host1, host2 = await wait_for_cql_and_get_hosts(cql, [node1, node2], time.time() + 30) - cql.execute("CREATE KEYSPACE ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2}") + # The test generates and uploads sstables, assuming their specific + # contents. These assumptions are not held with tablets, which + # distribute data among sstables differently than vnodes. + cql.execute("CREATE KEYSPACE ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2} AND tablets = { 'enabled': false }") table_schema = f"CREATE TABLE ks.tbl ({data_class.column_spec}) WITH speculative_retry = 'NONE'" cql.execute(table_schema) diff --git a/test/topology_custom/test_replace_ignore_nodes.py b/test/topology_custom/test_replace_ignore_nodes.py index c336bdb29cd4..2f9b2a036fd9 100644 --- a/test/topology_custom/test_replace_ignore_nodes.py +++ b/test/topology_custom/test_replace_ignore_nodes.py @@ -26,7 +26,8 @@ async def test_replace_ignore_nodes(manager: ManagerClient) -> None: Preferably run it only in one mode e.g. dev. """ cfg = {'enable_user_defined_functions': False, - 'force_gossip_topology_changes': True} + 'force_gossip_topology_changes': True, + 'enable_tablets': False} logger.info(f"Booting initial cluster") servers = [await manager.server_add(config=cfg) for _ in range(7)] s2_id = await manager.get_host_id(servers[2].server_id) diff --git a/test/topology_experimental_raft/test_restart_cluster.py b/test/topology_custom/test_restart_cluster.py similarity index 100% rename from test/topology_experimental_raft/test_restart_cluster.py rename to test/topology_custom/test_restart_cluster.py diff --git a/test/topology_custom/test_table_drop.py b/test/topology_custom/test_table_drop.py index 31fcb880c97d..1869a29c449a 100644 --- a/test/topology_custom/test_table_drop.py +++ b/test/topology_custom/test_table_drop.py @@ -7,5 +7,6 @@ async def test_drop_table_during_streaming_receiver_side(manager: ManagerClient) 'error_injections_at_startup': ['stream_mutation_fragments_table_dropped'], 'enable_repair_based_node_ops': False, 'enable_user_defined_functions': False, - 'force_gossip_topology_changes': True + 'force_gossip_topology_changes': True, + 'enable_tablets': False }) for _ in range(2)] diff --git a/test/topology_experimental_raft/test_tablet_repair_scheduler.py b/test/topology_custom/test_tablet_repair_scheduler.py similarity index 100% rename from test/topology_experimental_raft/test_tablet_repair_scheduler.py rename to test/topology_custom/test_tablet_repair_scheduler.py diff --git a/test/topology_experimental_raft/test_tablets.py b/test/topology_custom/test_tablets2.py similarity index 100% rename from test/topology_experimental_raft/test_tablets.py rename to test/topology_custom/test_tablets2.py diff --git a/test/topology_experimental_raft/test_tablets_intranode.py b/test/topology_custom/test_tablets_intranode.py similarity index 100% rename from test/topology_experimental_raft/test_tablets_intranode.py rename to test/topology_custom/test_tablets_intranode.py diff --git a/test/topology_experimental_raft/test_tablets_merge.py b/test/topology_custom/test_tablets_merge.py similarity index 100% rename from test/topology_experimental_raft/test_tablets_merge.py rename to test/topology_custom/test_tablets_merge.py diff --git a/test/topology_experimental_raft/test_tablets_removenode.py b/test/topology_custom/test_tablets_removenode.py similarity index 100% rename from test/topology_experimental_raft/test_tablets_removenode.py rename to test/topology_custom/test_tablets_removenode.py diff --git a/test/topology_experimental_raft/test_tombstone_gc.py b/test/topology_custom/test_tombstone_gc.py similarity index 100% rename from test/topology_experimental_raft/test_tombstone_gc.py rename to test/topology_custom/test_tombstone_gc.py diff --git a/test/topology_experimental_raft/test_topology_ops.py b/test/topology_custom/test_topology_ops.py similarity index 100% rename from test/topology_experimental_raft/test_topology_ops.py rename to test/topology_custom/test_topology_ops.py diff --git a/test/topology_experimental_raft/test_topology_recovery_basic.py b/test/topology_custom/test_topology_recovery_basic.py similarity index 100% rename from test/topology_experimental_raft/test_topology_recovery_basic.py rename to test/topology_custom/test_topology_recovery_basic.py diff --git a/test/topology_experimental_raft/test_topology_recovery_majority_loss.py b/test/topology_custom/test_topology_recovery_majority_loss.py similarity index 100% rename from test/topology_experimental_raft/test_topology_recovery_majority_loss.py rename to test/topology_custom/test_topology_recovery_majority_loss.py diff --git a/test/topology_custom/test_topology_remove_garbage_group0.py b/test/topology_custom/test_topology_remove_garbage_group0.py index 8dac8d78e977..25b619853fb8 100644 --- a/test/topology_custom/test_topology_remove_garbage_group0.py +++ b/test/topology_custom/test_topology_remove_garbage_group0.py @@ -27,7 +27,8 @@ async def test_remove_garbage_group0_members(manager: ManagerClient): """ # 4 servers, one dead cfg = {'enable_user_defined_functions': False, - 'force_gossip_topology_changes': True} + 'force_gossip_topology_changes': True, + 'enable_tablets': False} servers = [await manager.server_add(config=cfg) for _ in range(4)] # Make sure that the driver has connected to all nodes, and they see each other as NORMAL diff --git a/test/topology_experimental_raft/test_topology_upgrade.py b/test/topology_custom/test_topology_upgrade.py similarity index 100% rename from test/topology_experimental_raft/test_topology_upgrade.py rename to test/topology_custom/test_topology_upgrade.py diff --git a/test/topology_experimental_raft/test_topology_upgrade_not_stuck_after_recent_removal.py b/test/topology_custom/test_topology_upgrade_not_stuck_after_recent_removal.py similarity index 100% rename from test/topology_experimental_raft/test_topology_upgrade_not_stuck_after_recent_removal.py rename to test/topology_custom/test_topology_upgrade_not_stuck_after_recent_removal.py diff --git a/test/topology_experimental_raft/test_zero_token_nodes_multidc.py b/test/topology_custom/test_zero_token_nodes_multidc.py similarity index 100% rename from test/topology_experimental_raft/test_zero_token_nodes_multidc.py rename to test/topology_custom/test_zero_token_nodes_multidc.py diff --git a/test/topology_experimental_raft/test_zero_token_nodes_no_replication.py b/test/topology_custom/test_zero_token_nodes_no_replication.py similarity index 100% rename from test/topology_experimental_raft/test_zero_token_nodes_no_replication.py rename to test/topology_custom/test_zero_token_nodes_no_replication.py diff --git a/test/topology_custom/test_zero_token_nodes_topology_ops.py b/test/topology_custom/test_zero_token_nodes_topology_ops.py index 68622de13665..2338a2d3152b 100644 --- a/test/topology_custom/test_zero_token_nodes_topology_ops.py +++ b/test/topology_custom/test_zero_token_nodes_topology_ops.py @@ -25,7 +25,9 @@ async def test_zero_token_nodes_topology_ops(manager: ManagerClient, tablets_ena - client requests to normal nodes in the presence of zero-token nodes (2 normal nodes, RF=2, CL=2) succeed """ logging.info('Trying to add a zero-token server in the gossip-based topology') - await manager.server_add(config={'join_ring': False, 'force_gossip_topology_changes': True}, + await manager.server_add(config={'join_ring': False, + 'force_gossip_topology_changes': True, + 'enable_tablets': False}, expected_error='the raft-based topology is disabled') normal_cfg = {'enable_tablets': tablets_enabled} diff --git a/test/topology_experimental_raft/__init__.py b/test/topology_experimental_raft/__init__.py deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/test/topology_experimental_raft/conftest.py b/test/topology_experimental_raft/conftest.py deleted file mode 100644 index 374b0244b2e7..000000000000 --- a/test/topology_experimental_raft/conftest.py +++ /dev/null @@ -1,9 +0,0 @@ -# -# Copyright (C) 2023-present ScyllaDB -# -# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -# -# This file configures pytest for all tests in this directory, and also -# defines common test fixtures for all of them to use - -from test.topology.conftest import * diff --git a/test/topology_experimental_raft/suite.yaml b/test/topology_experimental_raft/suite.yaml deleted file mode 100644 index d637c9ea3430..000000000000 --- a/test/topology_experimental_raft/suite.yaml +++ /dev/null @@ -1,20 +0,0 @@ -type: Topology -pool_size: 4 -cluster: - initial_size: 0 -extra_scylla_config_options: - authenticator: AllowAllAuthenticator - authorizer: AllowAllAuthorizer - enable_user_defined_functions: False - enable_tablets: True -run_first: - - test_raft_cluster_features - - test_raft_ignore_nodes - - test_tablets -skip_in_release: - - test_raft_cluster_features -skip_in_debug: - - test_cdc_generation_clearing - - test_cdc_generation_publishing -run_in_dev: - - test_raft_ignore_nodes diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index 6a4395cbaa7c..dfd88a495c23 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -12,7 +12,7 @@ from test.pylib.repair import create_table_insert_data_for_repair, get_tablet_task_id from test.pylib.tablets import get_all_tablet_replicas from test.topology.conftest import skip_mode -from test.topology_experimental_raft.test_tablets import inject_error_on +from test.topology_custom.test_tablets2 import inject_error_on from test.topology_tasks.task_manager_client import TaskManagerClient from test.topology_tasks.task_manager_types import TaskStatus, TaskStats From 311c52fbb187496fa58bea7420c780692bb21f8e Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Sat, 21 Dec 2024 11:07:06 +0200 Subject: [PATCH 091/397] utils: phased_barrier: advance_and_await: allocate new gate only when needed If there are no opearions in progress, there is no need to close the current gate and allocate a new one. The current gate can be reused for the new phase just as well. Signed-off-by: Benny Halevy --- utils/phased_barrier.hh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/utils/phased_barrier.hh b/utils/phased_barrier.hh index 06a9ad994500..4b1602967e6a 100644 --- a/utils/phased_barrier.hh +++ b/utils/phased_barrier.hh @@ -62,6 +62,10 @@ public: // It is fine to start multiple awaits in parallel. // Cannot fail. future<> advance_and_await() noexcept { + if (!operations_in_progress()) { + ++_phase; + return make_ready_future(); + } auto new_gate = [] { seastar::memory::scoped_critical_alloc_section _; return make_lw_shared(); From a25c3eaa1caa9826abba750dc004566bdda89eb6 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Sat, 21 Dec 2024 11:28:50 +0200 Subject: [PATCH 092/397] utils: phased_barrier: add close() method When services are stopped we generally want to call advance_and_await(), but we should also prevent starting new operations, so close() would do that be closing the phased_barrier active gate (which implicitly also awaits past operations similar to advance_and_await()). Add unit tests for that and use in existing services. Signed-off-by: Benny Halevy --- replica/table.cc | 6 +++- test/boost/serialized_action_test.cc | 54 ++++++++++++++++++++++++++++ utils/phased_barrier.hh | 8 +++++ 3 files changed, 67 insertions(+), 1 deletion(-) diff --git a/replica/table.cc b/replica/table.cc index 111fdace12b7..79f958b4e041 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -1684,7 +1684,11 @@ table::stop() { // Allow `compaction_group::stop` to stop ongoing compactions // while they may still hold the table _async_gate auto gate_closed_fut = _async_gate.close(); - co_await await_pending_ops(); + co_await when_all( + _pending_reads_phaser.close(), + _pending_writes_phaser.close(), + _pending_streams_phaser.close(), + _pending_flushes_phaser.close()); co_await _sg_manager->stop_storage_groups(); co_await _sstable_deletion_gate.close(); co_await std::move(gate_closed_fut); diff --git a/test/boost/serialized_action_test.cc b/test/boost/serialized_action_test.cc index 664ec9cefe5a..633541f5cdde 100644 --- a/test/boost/serialized_action_test.cc +++ b/test/boost/serialized_action_test.cc @@ -7,6 +7,7 @@ */ #include +#include #include #include #include "utils/serialized_action.hh" @@ -183,3 +184,56 @@ SEASTAR_THREAD_TEST_CASE(test_phased_barrier_reassignment) { bar2.advance_and_await().get(); completion_timer.cancel(); } + +SEASTAR_THREAD_TEST_CASE(test_phased_barrier_stop) { + utils::phased_barrier bar; + semaphore sem(0); + auto task = [&] () -> future<> { + auto op = bar.start(); + co_await get_units(sem, 1); + }; + auto task_fut = task(); + BOOST_REQUIRE(!task_fut.available()); + + auto close_fut = bar.close(); + BOOST_REQUIRE(!close_fut.available()); + BOOST_REQUIRE(bar.is_closed()); + + BOOST_REQUIRE_THROW(task().get(), seastar::gate_closed_exception); + + sem.signal(); + BOOST_REQUIRE_NO_THROW(task_fut.get()); + BOOST_REQUIRE_NO_THROW(close_fut.get()); + + BOOST_REQUIRE(bar.is_closed()); + BOOST_REQUIRE_THROW(task().get(), seastar::gate_closed_exception); +} + +SEASTAR_THREAD_TEST_CASE(test_phased_barrier_stop_while_awaiting) { + utils::phased_barrier bar; + semaphore sem(0); + auto task = [&] () -> future<> { + auto op = bar.start(); + co_await get_units(sem, 1); + }; + auto task_fut = task(); + BOOST_REQUIRE(!task_fut.available()); + + auto wait_fut = bar.advance_and_await(); + BOOST_REQUIRE(!wait_fut.available()); + BOOST_REQUIRE(!bar.is_closed()); + + auto close_fut = bar.close(); + BOOST_REQUIRE(!close_fut.available()); + BOOST_REQUIRE(bar.is_closed()); + + BOOST_REQUIRE_THROW(task().get(), seastar::gate_closed_exception); + + sem.signal(); + BOOST_REQUIRE_NO_THROW(task_fut.get()); + BOOST_REQUIRE_NO_THROW(wait_fut.get()); + BOOST_REQUIRE_NO_THROW(close_fut.get()); + + BOOST_REQUIRE(bar.is_closed()); + BOOST_REQUIRE_THROW(task().get(), seastar::gate_closed_exception); +} diff --git a/utils/phased_barrier.hh b/utils/phased_barrier.hh index 4b1602967e6a..32369770b319 100644 --- a/utils/phased_barrier.hh +++ b/utils/phased_barrier.hh @@ -58,6 +58,14 @@ public: return { _gate }; } + future<> close() noexcept { + return _gate->close(); + } + + bool is_closed() const noexcept { + return _gate->is_closed(); + } + // Starts a new phase and waits for all operations started in any of the earlier phases. // It is fine to start multiple awaits in parallel. // Cannot fail. From 6c031ad92fe44501fa0457a1991cf11aab309aa2 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Wed, 18 Dec 2024 16:59:03 +0800 Subject: [PATCH 093/397] test/topology: Percent-encode URL in pytest artifact links When embedding HTML documents in pytest reports with links to test artifacts, parameterized test names containing special characters like "[" and "]" can cause URL encoding issues. These characters, when used verbatim in URLs, can trigger HTTP 400 errors on web servers. This commit resolves the issue by percent-encoding the URLs for artifact links, ensuring compatibility with servers like Jenkins and preventing "HTTP ERROR 400 Illegal Path Character" errors. Changes: - Percent-encode test artifact URLs to handle special characters - Improve link robustness for parameterized test names Fixes scylladb/scylla-pkg#4599 Signed-off-by: Kefu Chai Closes scylladb/scylladb#21963 --- test/topology/conftest.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/test/topology/conftest.py b/test/topology/conftest.py index 65bf15aa48d0..5177a91b3e61 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -8,6 +8,7 @@ import pathlib import ssl import platform +import urllib.parse from functools import partial from typing import List, Optional, Dict from test.pylib.random_tables import RandomTables @@ -209,8 +210,9 @@ async def manager(request, manager_internal, record_property, build_mode): if request.config.getoption('artifacts_dir_url') is not None: # get the relative path to the tmpdir for the failed directory dir_path_relative = f"{failed_test_dir_path.as_posix()[failed_test_dir_path.as_posix().find('testlog'):]}" - full_url = f"failed_test_logs" - record_property("TEST_LOGS", full_url) + full_url = urllib.parse.urljoin(request.config.getoption('artifacts_dir_url') + '/', + urllib.parse.quote(dir_path_relative)) + record_property("TEST_LOGS", f"failed_test_logs") cluster_status = await manager_client.after_test(test_case_name, not failed) await manager_client.stop() # Stop client session and close driver after each test From 6acc5294a4e89c0b6e13f62e903be0d227eeeea4 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Wed, 11 Dec 2024 10:38:58 +0800 Subject: [PATCH 094/397] treewide: migrate from boost::copy_range to std::ranges::to now that we are allowed to use C++23. we now have the luxury of using `std::ranges::to`. in this change, we: - replace `boost::copy_range` to `std::ranges::to` - remove unused `#include` of boost headers Signed-off-by: Kefu Chai Closes scylladb/scylladb#21880 --- compaction/compaction.cc | 5 +---- compaction/compaction_manager.cc | 7 +++--- compaction/compaction_strategy.cc | 4 +--- compaction/time_window_compaction_strategy.cc | 3 +-- db/consistency_level.cc | 4 +--- gms/feature_service.cc | 5 ++--- gms/gossiper.cc | 8 ++----- locator/network_topology_strategy.cc | 4 ++-- locator/util.cc | 4 ++-- partition_range_compat.hh | 5 ++--- repair/repair.cc | 4 +--- replica/table.cc | 2 +- service/storage_service.cc | 12 +++++----- service/task_manager_module.cc | 2 +- sstables/sstable_set.cc | 2 +- sstables/sstables.cc | 2 +- test/boost/partitioner_test.cc | 6 ++--- test/boost/sstable_compaction_test.cc | 2 +- test/boost/sstable_datafile_test.cc | 2 +- test/lib/mutation_source_test.cc | 7 +----- test/perf/perf_fast_forward.cc | 4 +--- test/perf/perf_mutation_readers.cc | 22 +++++++++---------- 22 files changed, 46 insertions(+), 70 deletions(-) diff --git a/compaction/compaction.cc b/compaction/compaction.cc index de04659b4ad7..97de6272805a 100644 --- a/compaction/compaction.cc +++ b/compaction/compaction.cc @@ -16,10 +16,7 @@ #include #include -#include #include -#include -#include #include #include @@ -227,7 +224,7 @@ static api::timestamp_type get_max_purgeable_timestamp(const table_state& table_ static std::vector get_uncompacting_sstables(const table_state& table_s, std::vector sstables) { auto sstable_set = table_s.sstable_set_for_tombstone_gc(); - auto all_sstables = boost::copy_range>(*sstable_set->all()); + auto all_sstables = *sstable_set->all() | std::ranges::to(); auto& compacted_undeleted = table_s.compacted_undeleted_sstables(); all_sstables.insert(all_sstables.end(), compacted_undeleted.begin(), compacted_undeleted.end()); std::ranges::sort(all_sstables, std::ranges::less(), std::mem_fn(&sstable::generation)); diff --git a/compaction/compaction_manager.cc b/compaction/compaction_manager.cc index a0c05a0a219a..241cfadf17f9 100644 --- a/compaction/compaction_manager.cc +++ b/compaction/compaction_manager.cc @@ -28,7 +28,6 @@ #include "db/system_keyspace.hh" #include "tombstone_gc-internals.hh" #include -#include static logging::logger cmlog("compaction_manager"); using namespace std::chrono_literals; @@ -362,7 +361,7 @@ future compaction_manager::perform_tas future<> compaction_manager::on_compaction_completion(table_state& t, sstables::compaction_completion_desc desc, sstables::offstrategy offstrategy) { auto& cs = get_compaction_state(&t); - auto new_sstables = boost::copy_range>(desc.new_sstables); + auto new_sstables = desc.new_sstables | std::ranges::to(); for (const auto& sst : desc.old_sstables) { if (!new_sstables.contains(sst)) { cs.sstables_requiring_cleanup.erase(sst); @@ -1805,7 +1804,7 @@ class validate_sstables_compaction_task_executor : public sstables_task_executor } static std::vector get_all_sstables(table_state& t) { - auto s = boost::copy_range>(*t.main_sstable_set().all()); + auto s = *t.main_sstable_set().all() | std::ranges::to(); auto maintenance_set = t.maintenance_sstable_set().all(); s.insert(s.end(), maintenance_set->begin(), maintenance_set->end()); return s; @@ -1894,7 +1893,7 @@ class cleanup_sstables_compaction_task_executor : public compaction_task_executo : update_registration{registration} , _desc{desc} {} void on_removal(const std::vector& sstables) override { - auto exhausted = boost::copy_range>(sstables); + auto exhausted = sstables | std::ranges::to(); std::erase_if(_desc.sstables, [&] (const sstables::shared_sstable& sst) { return exhausted.contains(sst); }); diff --git a/compaction/compaction_strategy.cc b/compaction/compaction_strategy.cc index 315ac35f676a..5abe49dd9780 100644 --- a/compaction/compaction_strategy.cc +++ b/compaction/compaction_strategy.cc @@ -21,8 +21,6 @@ #include "compaction_strategy_state.hh" #include "cql3/statements/property_definitions.hh" #include "schema/schema.hh" -#include -#include #include "size_tiered_compaction_strategy.hh" #include "leveled_compaction_strategy.hh" #include "time_window_compaction_strategy.hh" @@ -303,7 +301,7 @@ void size_tiered_backlog_tracker::replace_sstables(const std::vector>(tmp_all), _stcs_options); + auto tmp_contrib = calculate_sstables_backlog_contribution(tmp_all | std::ranges::to(), _stcs_options); std::invoke([&] () noexcept { _all = std::move(tmp_all); diff --git a/compaction/time_window_compaction_strategy.cc b/compaction/time_window_compaction_strategy.cc index 3c6bd1713cd5..cc2dc3989e84 100644 --- a/compaction/time_window_compaction_strategy.cc +++ b/compaction/time_window_compaction_strategy.cc @@ -16,7 +16,6 @@ #include #include #include -#include #include @@ -525,7 +524,7 @@ int64_t time_window_compaction_strategy::estimated_pending_compactions(table_sta auto& state = get_state(table_s); auto min_threshold = table_s.min_compaction_threshold(); auto max_threshold = table_s.schema()->max_compaction_threshold(); - auto candidate_sstables = boost::copy_range>(*table_s.main_sstable_set().all()); + auto candidate_sstables = *table_s.main_sstable_set().all() | std::ranges::to(); auto [buckets, max_timestamp] = get_buckets(std::move(candidate_sstables), _options); int64_t n = 0; diff --git a/db/consistency_level.cc b/db/consistency_level.cc index f86783381154..1ecefa837eb2 100644 --- a/db/consistency_level.cc +++ b/db/consistency_level.cc @@ -11,8 +11,6 @@ #include "db/consistency_level.hh" #include "db/consistency_level_validations.hh" -#include -#include #include "exceptions/exceptions.hh" #include #include @@ -336,7 +334,7 @@ filter_for_query(consistency_level cl, if (!old_node && ht_max - ht_min > 0.01) { // if there is old node or hit rates are close skip calculations // local node is always first if present (see storage_proxy::get_endpoints_for_reading) unsigned local_idx = erm.get_topology().is_me(epi[0].first) ? 0 : epi.size() + 1; - auto weighted = boost::copy_range(miss_equalizing_combination(epi, local_idx, remaining_bf, bool(extra))); + auto weighted = miss_equalizing_combination(epi, local_idx, remaining_bf, bool(extra)) | std::ranges::to(); // Workaround for https://github.com/scylladb/scylladb/issues/9285 auto last = std::adjacent_find(weighted.begin(), weighted.end()); if (last == weighted.end()) { diff --git a/gms/feature_service.cc b/gms/feature_service.cc index ddb7b4e1f183..00f22866c164 100644 --- a/gms/feature_service.cc +++ b/gms/feature_service.cc @@ -14,7 +14,6 @@ #include "db/system_keyspace.hh" #include #include -#include #include "gms/gossiper.hh" #include "gms/i_endpoint_state_change_subscriber.hh" #include "utils/assert.hh" @@ -308,7 +307,7 @@ future<> feature_service::enable_features_on_startup(db::system_keyspace& sys_ks } co_await container().invoke_on_all([&features_to_enable] (auto& srv) -> future<> { - std::set feat = boost::copy_range>(features_to_enable); + auto feat = features_to_enable | std::ranges::to>(); co_await srv.enable(std::move(feat)); }); } @@ -380,7 +379,7 @@ future<> persistent_feature_enabler::enable_features() { co_await _sys_ks.save_local_enabled_features(std::move(feats_set), true); co_await _feat.container().invoke_on_all([&features] (feature_service& fs) -> future<> { - std::set features_v = boost::copy_range>(features); + auto features_v = features | std::ranges::to>(); co_await fs.enable(std::move(features_v)); }); } diff --git a/gms/gossiper.cc b/gms/gossiper.cc index 4862d06fd983..493be9a76ff7 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -35,11 +35,7 @@ #include #include "locator/host_id.hh" #include -#include #include -#include -#include -#include #include #include "gms/generation-number.hh" #include "locator/token_metadata.hh" @@ -998,7 +994,7 @@ future<> gossiper::failure_detector_loop() { co_await sleep_abortable(std::chrono::seconds(1), _abort_source); continue; } - auto nodes = boost::copy_range>(_live_endpoints); + auto nodes = _live_endpoints | std::ranges::to>(); auto live_endpoints_version = _live_endpoints_version; auto generation_number = my_endpoint_state().get_heart_beat_state().get_generation(); co_await coroutine::parallel_for_each(std::views::iota(0u, nodes.size()), [this, generation_number, live_endpoints_version, &nodes] (size_t idx) { @@ -1089,7 +1085,7 @@ void gossiper::run() { gossip_digest_syn message(get_cluster_name(), get_partitioner_name(), g_digests, get_group0_id()); if (_endpoints_to_talk_with.empty()) { - auto live_endpoints = boost::copy_range>(_live_endpoints); + auto live_endpoints = _live_endpoints | std::ranges::to>(); std::shuffle(live_endpoints.begin(), live_endpoints.end(), _random_engine); // This guarantees the local node will talk with all nodes // in live_endpoints at least once within nr_rounds gossip rounds. diff --git a/locator/network_topology_strategy.cc b/locator/network_topology_strategy.cc index 062c1254d0d0..80876603e378 100644 --- a/locator/network_topology_strategy.cc +++ b/locator/network_topology_strategy.cc @@ -399,8 +399,8 @@ future network_topology_strategy::add_tablets_in_dc(schema_p auto replicas = cur_replicas; // all_dc_racks is ordered lexicographically on purpose - auto all_dc_racks = boost::copy_range>>>( - tm->get_datacenter_racks_token_owners_nodes().at(dc)); + auto all_dc_racks = tm->get_datacenter_racks_token_owners_nodes().at(dc) + | std::ranges::to(); // Track all nodes with no replicas on them for this tablet, per rack. struct node_load { diff --git a/locator/util.cc b/locator/util.cc index ce006f84ad1d..fc1cba53942f 100644 --- a/locator/util.cc +++ b/locator/util.cc @@ -140,8 +140,8 @@ describe_ring(const replica::database& db, const gms::gossiper& gossiper, const if (left_inf != right_inf && left_inf != ranges.end() && right_inf != ranges.end() - && (boost::copy_range(left_inf->_endpoints) - == boost::copy_range(right_inf->_endpoints))) { + && ((left_inf->_endpoints | std::ranges::to()) == + (right_inf->_endpoints | std::ranges::to()))) { left_inf->_start_token = std::move(right_inf->_start_token); ranges.erase(right_inf); } diff --git a/partition_range_compat.hh b/partition_range_compat.hh index 40a260123a0e..a2351b1c3e2d 100644 --- a/partition_range_compat.hh +++ b/partition_range_compat.hh @@ -67,7 +67,7 @@ wrap(const std::vector>& v) { ret.emplace_back(v.back().start(), v.front().end()); return ret; } - return boost::copy_range>>(v); + return v | std::ranges::to>>(); } template @@ -81,8 +81,7 @@ wrap(std::vector>&& v) { ret.emplace_back(std::move(v.back()).start(), std::move(v.front()).end()); return ret; } - // want boost::adaptor::moved ... - return boost::copy_range>>(v); + return std::ranges::owning_view(std::move(v)) | std::ranges::to(); } inline diff --git a/repair/repair.cc b/repair/repair.cc index 12d9f46b3723..f31e777e1f61 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -28,9 +28,7 @@ #include #include #include -#include #include -#include #include #include @@ -1903,7 +1901,7 @@ future<> repair_service::do_decommission_removenode_with_repair(locator::token_m // Choose the decommission node n3 to run repair to // sync with one of the replica nodes, e.g., n1, in the // local DC. - neighbors_set = get_neighbors_set(boost::copy_range>(new_eps)); + neighbors_set = get_neighbors_set(new_eps | std::ranges::to>()); } } else { throw std::runtime_error(fmt::format("{}: keyspace={}, range={}, current_replica_endpoints={}, new_replica_endpoints={}, wrong number of new owner node={}", diff --git a/replica/table.cc b/replica/table.cc index 111fdace12b7..5344472cd527 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -2898,7 +2898,7 @@ future table::take_snapshot(sstring jsondir) { auto sstable_deletion_guard = co_await get_units(_sstable_deletion_sem, 1); - auto tables = boost::copy_range>(*_sstables->all()); + auto tables = *_sstables->all() | std::ranges::to>(); auto table_names = std::make_unique>(); co_await io_check([&jsondir] { return recursive_touch_directory(jsondir); }); diff --git a/service/storage_service.cc b/service/storage_service.cc index 167e0bdecc6c..992042219764 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -717,7 +717,7 @@ future<> storage_service::topology_state_load(state_change_hint hint) { } co_await _feature_service.container().invoke_on_all([&] (gms::feature_service& fs) { - return fs.enable(boost::copy_range>(_topology_state_machine._topology.enabled_features)); + return fs.enable(_topology_state_machine._topology.enabled_features | std::ranges::to>()); }); // Update the legacy `enabled_features` key in `system.scylla_local`. @@ -807,7 +807,7 @@ future<> storage_service::topology_state_load(state_change_hint hint) { if (!_gossiper.get_endpoint_state_ptr(ep)) { gms::loaded_endpoint_state st; st.endpoint = ep; - st.tokens = boost::copy_range>(tmptr->get_tokens(host_id)); + st.tokens = tmptr->get_tokens(host_id) | std::ranges::to>(); st.opt_dc_rack = node.get().dc_rack(); // Save tokens, not needed for raft topology management, but needed by legacy // Also ip -> id mapping is needed for address map recreation on reboot @@ -1237,7 +1237,7 @@ std::vector storage_service::build_mutation_from_join_params .set("shard_count", params.shard_count) .set("ignore_msb", params.ignore_msb) .set("cleanup_status", cleanup_status::clean) - .set("supported_features", boost::copy_range>(params.supported_features)); + .set("supported_features", params.supported_features | std::ranges::to>()); if (params.replaced_id) { node_builder @@ -1342,7 +1342,7 @@ future<> storage_service::raft_initialize_discovery_leader(const join_node_reque // We are the first node and we define the cluster. // Set the enabled_features field to our features. topology_mutation_builder builder(guard.write_timestamp()); - builder.add_enabled_features(boost::copy_range>(params.supported_features)) + builder.add_enabled_features(params.supported_features | std::ranges::to>()) .set_upgrade_state(topology::upgrade_state_type::done); // Skip upgrade, start right in the topology-on-raft mode auto enable_features_mutation = builder.build(); @@ -1375,7 +1375,7 @@ future<> storage_service::update_topology_with_local_metadata(raft::server& raft auto local_shard_count = smp::count; auto local_ignore_msb = _db.local().get_config().murmur3_partitioner_ignore_msb_bits(); auto local_release_version = version::release(); - auto local_supported_features = boost::copy_range>(_feature_service.supported_feature_set()); + auto local_supported_features = _feature_service.supported_feature_set() | std::ranges::to>(); auto synchronized = [&] () { auto it = _topology_state_machine._topology.find(raft_server.id()); @@ -1786,7 +1786,7 @@ future<> storage_service::join_topology(sharded .tokens_string = _db.local().get_config().join_ring() ? _db.local().get_config().initial_token() : sstring(), .shard_count = smp::count, .ignore_msb = _db.local().get_config().murmur3_partitioner_ignore_msb_bits(), - .supported_features = boost::copy_range>(_feature_service.supported_feature_set()), + .supported_features = _feature_service.supported_feature_set() | std::ranges::to>(), .request_id = utils::UUID_gen::get_time_UUID(), }; diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index eddd9bc167c0..36e30dc2639f 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -165,7 +165,7 @@ future> tablet_virtual_task::get_stats() { } std::vector tablet_virtual_task::get_table_ids() const { - return boost::copy_range>(_ss.get_token_metadata().tablets().all_tables() | boost::adaptors::transformed([] (const auto& table_to_tablets) { return table_to_tablets.first; })); + return _ss.get_token_metadata().tablets().all_tables() | std::views::transform([] (const auto& table_to_tablets) { return table_to_tablets.first; }) | std::ranges::to>(); } static void update_status(const locator::tablet_task_info& task_info, tasks::task_status& status, size_t& sched_nr) { diff --git a/sstables/sstable_set.cc b/sstables/sstable_set.cc index d6fe0e55cab3..67bb4011bc15 100644 --- a/sstables/sstable_set.cc +++ b/sstables/sstable_set.cc @@ -113,7 +113,7 @@ std::ostream& operator<<(std::ostream& os, const sstables::sstable_run& run) { os << format(" Identifier: {}\n", (*run.all().begin())->run_identifier()); } - auto frags = boost::copy_range>(run.all()); + auto frags = run.all() | std::ranges::to>(); std::ranges::sort(frags, std::ranges::less(), [] (const shared_sstable& x) { return x->get_first_decorated_key().token(); }); diff --git a/sstables/sstables.cc b/sstables/sstables.cc index 8e031b9d511c..e068ac79ce93 100644 --- a/sstables/sstables.cc +++ b/sstables/sstables.cc @@ -3020,7 +3020,7 @@ sstable::compute_shards_for_this_sstable(const dht::sharder& sharder_) const { shards.insert(rpras->shard); rpras = sharder.next(*_schema); } - return boost::copy_range>(shards); + return shards | std::ranges::to(); } future sstable::has_partition_key(const utils::hashed_key& hk, const dht::decorated_key& dk) { diff --git a/test/boost/partitioner_test.cc b/test/boost/partitioner_test.cc index 8ca1b0f8a0d8..f51ec0b0205d 100644 --- a/test/boost/partitioner_test.cc +++ b/test/boost/partitioner_test.cc @@ -191,9 +191,9 @@ SEASTAR_THREAD_TEST_CASE(test_ring_position_ordering) { testlog.info("Keys: {}", keys); - auto positions = boost::copy_range>(keys); - auto ext_positions = boost::copy_range>(keys); - auto views = boost::copy_range>(positions); + auto positions = keys | std::ranges::to>(); + auto ext_positions = keys | std::ranges::to>(); + auto views = positions | std::ranges::to>(); total_order_check(cmp) .next(dht::ring_position_view::min()) diff --git a/test/boost/sstable_compaction_test.cc b/test/boost/sstable_compaction_test.cc index e2d545716db7..5524dfdd37a0 100644 --- a/test/boost/sstable_compaction_test.cc +++ b/test/boost/sstable_compaction_test.cc @@ -123,7 +123,7 @@ class strategy_control_for_test : public strategy_control { } std::vector candidates(table_state& t) const override { - return _candidates_opt.value_or(boost::copy_range>(*t.main_sstable_set().all())); + return _candidates_opt.value_or(*t.main_sstable_set().all() | std::ranges::to()); } std::vector candidates_as_runs(table_state& t) const override { diff --git a/test/boost/sstable_datafile_test.cc b/test/boost/sstable_datafile_test.cc index ac4478cbc8e5..75a983edb09f 100644 --- a/test/boost/sstable_datafile_test.cc +++ b/test/boost/sstable_datafile_test.cc @@ -2055,7 +2055,7 @@ SEASTAR_TEST_CASE(sstable_owner_shards) { auto assert_sstable_owners = [&] (std::unordered_set expected_owners, unsigned ignore_msb, unsigned smp_count) { SCYLLA_ASSERT(expected_owners.size() <= smp_count); auto sst = make_shared_sstable(expected_owners, ignore_msb, smp_count); - auto owners = boost::copy_range>(sst->get_shards_for_this_sstable()); + auto owners = sst->get_shards_for_this_sstable() | std::ranges::to>(); BOOST_REQUIRE(std::ranges::all_of(expected_owners, [&] (unsigned expected_owner) { return owners.contains(expected_owner); })); diff --git a/test/lib/mutation_source_test.cc b/test/lib/mutation_source_test.cc index 5523c95d6328..4a29661e6785 100644 --- a/test/lib/mutation_source_test.cc +++ b/test/lib/mutation_source_test.cc @@ -7,9 +7,6 @@ */ #include -#include -#include -#include #include #include #include "partition_slice_builder.hh" @@ -323,9 +320,7 @@ static void test_slicing_and_fast_forwarding(tests::reader_concurrency_semaphore for (auto prange_size = 1u; prange_size < mutations.size(); prange_size += 2) { for (auto pstart = 0u; pstart + prange_size <= mutations.size(); pstart++) { - auto ms = boost::copy_range>( - mutations | boost::adaptors::sliced(pstart, pstart + prange_size) - ); + auto ms = mutations | std::views::drop(pstart) | std::views::take(prange_size) | std::ranges::to(); if (prange_size == 1) { test_ckey({dht::partition_range::make_singular(mutations[pstart].decorated_key())}, ms, mutation_reader::forwarding::yes); test_ckey({dht::partition_range::make_singular(mutations[pstart].decorated_key())}, ms, mutation_reader::forwarding::no); diff --git a/test/perf/perf_fast_forward.cc b/test/perf/perf_fast_forward.cc index cb5510ce324a..f99bba2fbc00 100644 --- a/test/perf/perf_fast_forward.cc +++ b/test/perf/perf_fast_forward.cc @@ -2106,9 +2106,7 @@ int scylla_fast_forward_main(int argc, char** argv) { return make_ready_future(); }); - auto requested_test_groups = boost::copy_range>( - app.configuration()["run-tests"].as>() - ); + auto requested_test_groups = app.configuration()["run-tests"].as>() | std::ranges::to(); auto enabled_test_groups = test_groups | std::views::filter([&] (auto&& tc) { return requested_test_groups.contains(tc.name); }); diff --git a/test/perf/perf_mutation_readers.cc b/test/perf/perf_mutation_readers.cc index cdd5b0eb1372..b8bfa3bffe67 100644 --- a/test/perf/perf_mutation_readers.cc +++ b/test/perf/perf_mutation_readers.cc @@ -11,10 +11,6 @@ #include #include -#include -#include -#include - #include "test/lib/simple_schema.hh" #include "test/lib/simple_position_reader_queue.hh" #include "test/perf/perf.hh" @@ -98,11 +94,13 @@ std::vector> combined::create_disjoint_interleaved_streams auto base = create_single_stream(s, permit); std::vector> mss; for (auto i = 0; i < 4; i++) { - mss.emplace_back(boost::copy_range>( + mss.emplace_back( base - | boost::adaptors::sliced(i, base.size()) - | boost::adaptors::strided(4) - )); + | std::views::drop(i) + | std::views::take(base.size() - i) + | std::views::stride(4) + | std::ranges::to() + ); } return mss; } @@ -113,10 +111,12 @@ std::vector> combined::create_disjoint_ranges_streams(simp std::vector> mss; auto slice = base.size() / 4; for (auto i = 0; i < 4; i++) { - mss.emplace_back(boost::copy_range>( + mss.emplace_back( base - | boost::adaptors::sliced(i * slice, std::min((i + 1) * slice, base.size())) - )); + | std::views::drop(i * slice) + | std::views::take(std::min((i + 1) * slice, base.size()) - i * slice) + | std::ranges::to() + ); } return mss; } From bc487c94566cbc2f87c5e2e57850cf84884211dc Mon Sep 17 00:00:00 2001 From: Yaron Kaikov Date: Mon, 23 Dec 2024 13:50:23 +0200 Subject: [PATCH 095/397] .github: cherry-pick each commit instead of merge commit when available Until today, when we had a PR with multiple commits we cherry-pick the merge commit only, which created a PR with only one commit (the merge commit) with all relevant changes This was causing an issue when there was a need to backport part of the commits like in https://github.com/scylladb/scylladb/pull/21990 (reported by @gleb-cloudius) Changing the logic to cherry-pick each commit Closes scylladb/scylladb#22027 --- .github/scripts/auto-backport.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/scripts/auto-backport.py b/.github/scripts/auto-backport.py index f992799746e9..7719b28800a3 100755 --- a/.github/scripts/auto-backport.py +++ b/.github/scripts/auto-backport.py @@ -66,7 +66,8 @@ def get_pr_commits(repo, pr, stable_branch, start_commit=None): if pr.merged: merge_commit = repo.get_commit(pr.merge_commit_sha) if len(merge_commit.parents) > 1: # Check if this merge commit includes multiple commits - commits.append(pr.merge_commit_sha) + for commit in pr.get_commits(): + commits.append(commit.sha) else: if start_commit: promoted_commits = repo.compare(start_commit, stable_branch).commits @@ -114,7 +115,7 @@ def backport(repo, pr, version, commits, backport_base_branch): is_draft = False for commit in commits: try: - repo_local.git.cherry_pick(commit, '-m1', '-x') + repo_local.git.cherry_pick(commit, '-x') except GitCommandError as e: logging.warning(f'Cherry-pick conflict on commit {commit}: {e}') is_draft = True From 3e22998dc13156b373e24688371216403938abc2 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Mon, 2 Dec 2024 14:12:43 +0200 Subject: [PATCH 096/397] sstables: parse(summary): reserve positions vector We know the number of positions in advance so reserve the chunked_vector capacity for that. Note: reservation replaces the existing reset of the positions member. This is safe since we parse the summary only once as sstable::read_summary() returns early if the summary component is already populated. Signed-off-by: Benny Halevy Closes scylladb/scylladb#21767 --- sstables/sstables.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sstables/sstables.cc b/sstables/sstables.cc index e068ac79ce93..c5e9f434031a 100644 --- a/sstables/sstables.cc +++ b/sstables/sstables.cc @@ -504,7 +504,7 @@ future<> parse(const schema& schema, sstable_version_types v, random_access_read // Positions are encoded in little-endian. auto b = buf.get(); - s.positions = utils::chunked_vector(); + s.positions.reserve(s.header.size + 1); while (s.positions.size() != s.header.size) { s.positions.push_back(seastar::read_le(b)); b += sizeof(pos_type); From 1224200d7ad9d80da0b328783f85d6c5fd35ed8b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Tue, 17 Jan 2023 03:04:39 +0100 Subject: [PATCH 097/397] configure.py: unify build rules for cxxbridge .cc files and regular .cc files This is going to prevent some code duplication in following patches. --- configure.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/configure.py b/configure.py index b24fdaa50ba6..5df0cb7efaf2 100755 --- a/configure.py +++ b/configure.py @@ -2201,6 +2201,13 @@ def write_build_file(f, gen_headers.append('$builddir/{}/gen/rust/cxx.h'.format(mode)) gen_headers_dep = ' '.join(gen_headers) + for hh in rust_headers: + src = rust_headers[hh] + f.write('build {}: rust_header {}\n'.format(hh, src)) + cc = hh.replace('.hh', '.cc') + f.write('build {}: rust_source {}\n'.format(cc, src)) + obj = cc.replace('.cc', '.o') + compiles[obj] = cc for obj in compiles: src = compiles[obj] f.write('build {}: cxx.{} {} || {} {}\n'.format(obj, mode, src, seastar_dep, gen_headers_dep)) @@ -2219,13 +2226,6 @@ def write_build_file(f, for hh in ragels: src = ragels[hh] f.write('build {}: ragel {}\n'.format(hh, src)) - for hh in rust_headers: - src = rust_headers[hh] - f.write('build {}: rust_header {}\n'.format(hh, src)) - cc = hh.replace('.hh', '.cc') - f.write('build {}: rust_source {}\n'.format(cc, src)) - obj = cc.replace('.cc', '.o') - f.write('build {}: cxx.{} {} || {}\n'.format(obj, mode, cc, gen_headers_dep)) f.write('build {}: cxxbridge_header\n'.format('$builddir/{}/gen/rust/cxx.h'.format(mode))) librust = '$builddir/{}/rust-{}/librust_combined'.format(mode, mode) f.write('build {}.a: rust_lib.{} rust/Cargo.lock\n depfile={}.d\n'.format(librust, mode, librust)) From 192cb6de4b4d0ebfcabeb6595eac0d7a04a4ca81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 25 Jan 2023 14:44:56 +0100 Subject: [PATCH 098/397] configure.py: add a `default` to `add_tristate`. It will be used in the next patch. --- configure.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/configure.py b/configure.py index 5df0cb7efaf2..222afeada151 100755 --- a/configure.py +++ b/configure.py @@ -76,8 +76,8 @@ def get_flags(): return re.sub(r'^flags\s+: ', '', line).split() -def add_tristate(arg_parser, name, dest, help): - arg_parser.add_argument('--enable-' + name, dest=dest, action='store_true', default=None, +def add_tristate(arg_parser, name, dest, help, default=None): + arg_parser.add_argument('--enable-' + name, dest=dest, action='store_true', default=default, help='Enable ' + help) arg_parser.add_argument('--disable-' + name, dest=dest, action='store_false', default=None, help='Disable ' + help) From 4b03b91fbd63b90f9f92a49cdf688f4e0a5b4fab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Tue, 31 May 2022 14:53:45 +0200 Subject: [PATCH 099/397] configure.py: introduce link-time optimization This patch introduces link-time optimization (LTO) to the build. The performance gains from LTO alone are modest (~7%), but it's vital ingredient of effective profile-guided optimization, which will be introduced later. In general, use of LTO is quite simple and transparent to build systems. It is sufficient to add the -flto flag to compile and link steps, and use a LTO-aware linker. At compile time, -ffat-lto-objects will cause the compiler to emit .o files both LTO-ready LLVM IR for main executable optimization and machine code for fast test linking. At link time, those pieces of IR will be compiled together, allowing cross-object optimization of the main executable and the fast linking of test executables. Due to it's high compile time cost, the optimization can be toggled with a configure.py option. As of this patch, it's disabled by default. --- configure.py | 84 +++++++++++++++++++++++++++++++++++++++++++--------- 1 file changed, 70 insertions(+), 14 deletions(-) diff --git a/configure.py b/configure.py index 222afeada151..e8a7a71ea9df 100755 --- a/configure.py +++ b/configure.py @@ -378,6 +378,7 @@ def find_ninja(): 'build_seastar_shared_libs': True, 'default': True, 'description': 'a mode with no optimizations, with sanitizers, and with additional debug checks enabled, used for testing', + 'advanced_optimizations': False, }, 'release': { 'cxxflags': '-ffunction-sections -fdata-sections ', @@ -390,6 +391,7 @@ def find_ninja(): 'build_seastar_shared_libs': False, 'default': True, 'description': 'a mode with optimizations and no debug checks, used for production builds', + 'advanced_optimizations': True, }, 'dev': { 'cxxflags': '-DDEVEL -DSEASTAR_ENABLE_ALLOC_FAILURE_INJECTION -DSCYLLA_ENABLE_ERROR_INJECTION -DSCYLLA_ENABLE_PREEMPTION_SOURCE', @@ -402,6 +404,7 @@ def find_ninja(): 'build_seastar_shared_libs': True, 'default': True, 'description': 'a mode with no optimizations and no debug checks, optimized for fast build times, used for development', + 'advanced_optimizations': False, }, 'sanitize': { 'cxxflags': '-DDEBUG -DSANITIZE -DDEBUG_LSA_SANITIZER -DSCYLLA_ENABLE_ERROR_INJECTION', @@ -414,6 +417,7 @@ def find_ninja(): 'build_seastar_shared_libs': False, 'default': False, 'description': 'a mode with optimizations and sanitizers enabled, used for finding memory errors', + 'advanced_optimizations': False, }, 'coverage': { 'cxxflags': '-fprofile-instr-generate -fcoverage-mapping -g -gz', @@ -426,6 +430,7 @@ def find_ninja(): 'build_seastar_shared_libs': False, 'default': False, 'description': 'a mode exclusively used for generating test coverage reports', + 'advanced_optimizations': False, }, } @@ -617,6 +622,10 @@ def find_ninja(): 'scylla', ]) +lto_binaries = set([ + 'scylla' +]) + tests = scylla_tests | perf_tests | raft_tests other = set([ @@ -693,6 +702,8 @@ def find_ninja(): help='List all available build artifacts, that can be passed to --with') arg_parser.add_argument('--date-stamp', dest='date_stamp', type=str, help='Set datestamp for SCYLLA-VERSION-GEN') +add_tristate(arg_parser, name='lto', dest='lto', default=False, + help='link-time optimization.') arg_parser.add_argument('--use-cmake', action=argparse.BooleanOptionalAction, default=False, help='Whether to use CMake as the build system') arg_parser.add_argument('--coverage', action = 'store_true', help = 'Compile scylla with coverage instrumentation') arg_parser.add_argument('--build-dir', action='store', default='build', @@ -1680,6 +1691,18 @@ def dynamic_linker_option(): modes[mode]['lib_cflags'] = user_cflags modes[mode]['lib_ldflags'] = user_ldflags + linker_flags +def prepare_advanced_optimizations(*, modes, build_modes, args): + for mode in modes: + modes[mode]['has_lto'] = False + + for mode in modes: + if not modes[mode]['advanced_optimizations']: + continue + + if args.lto: + modes[mode]['has_lto'] = True + modes[mode]['lib_cflags'] += ' -flto=thin -ffat-lto-objects' + # cmake likes to separate things with semicolons def semicolon_separated(*flags): # original flags may be space separated, so convert to string still @@ -1691,8 +1714,6 @@ def real_relpath(path, start): return os.path.relpath(os.path.realpath(path), os.path.realpath(start)) def configure_seastar(build_dir, mode, mode_config): - seastar_build_dir = os.path.join(build_dir, mode, 'seastar') - seastar_cxx_ld_flags = mode_config['cxx_ld_flags'] # We want to "undo" coverage for seastar if we have it enabled. if args.coverage: @@ -1734,7 +1755,9 @@ def configure_seastar(build_dir, mode, mode_config): if mode_config['build_seastar_shared_libs']: seastar_cmake_args += ['-DBUILD_SHARED_LIBS=ON'] - seastar_cmd = ['cmake', '-G', 'Ninja', real_relpath(args.seastar_path, seastar_build_dir)] + seastar_cmake_args + cmake_args = seastar_cmake_args[:] + seastar_build_dir = os.path.join(build_dir, mode, 'seastar') + seastar_cmd = ['cmake', '-G', 'Ninja', real_relpath(args.seastar_path, seastar_build_dir)] + cmake_args cmake_dir = seastar_build_dir if dpdk: # need to cook first @@ -1774,9 +1797,12 @@ def configure_abseil(build_dir, mode, mode_config): '-DABSL_PROPAGATE_CXX_STD=ON', ] + cmake_args = abseil_cmake_args[:] abseil_build_dir = os.path.join(build_dir, mode, 'abseil') - abseil_cmd = ['cmake', '-G', 'Ninja', real_relpath('abseil', abseil_build_dir)] + abseil_cmake_args + abseil_cmd = ['cmake', '-G', 'Ninja', real_relpath('abseil', abseil_build_dir)] + cmake_args + if args.verbose: + print(' \\\n '.join(abseil_cmd)) os.makedirs(abseil_build_dir, exist_ok=True) subprocess.check_call(abseil_cmd, shell=False, cwd=abseil_build_dir) @@ -2079,10 +2105,16 @@ def write_build_file(f, ragels = {} antlr3_grammars = set() rust_headers = {} + + # We want LTO, but with the regular LTO, clang generates special LLVM IR files instead of + # regular ELF objects after the compile phase, and these special LLVM bitcode can only be + # used for LTO builds. The cost of compiling all tests with LTO is prohibitively high, so + # we can't use these IR files for tests -- we need to compile regular ELF objects as well. + # Therefore, we build FatLTO objects, which contain LTO compatible IR and the regular + # object code. And we enable LTO when linking the main Scylla executable, while disable + # it when linking anything else. + seastar_lib_ext = 'so' if modeval['build_seastar_shared_libs'] else 'a' - seastar_dep = f'$builddir/{mode}/seastar/libseastar.{seastar_lib_ext}' - seastar_testing_dep = f'$builddir/{mode}/seastar/libseastar_testing.{seastar_lib_ext}' - abseil_dep = ' '.join(f'$builddir/{mode}/abseil/{lib}' for lib in abseil_libs) for binary in sorted(build_artifacts): if binary in other or binary in wasms: continue @@ -2104,13 +2136,25 @@ def write_build_file(f, objs.append(f'$builddir/{mode}/gen/rust/{obj}') if has_rust: objs.append(f'$builddir/{mode}/rust-{mode}/librust_combined.a') + + do_lto = modes[mode]['has_lto'] and binary in lto_binaries + seastar_dep = f'$builddir/{mode}/seastar/libseastar.{seastar_lib_ext}' + seastar_testing_dep = f'$builddir/{mode}/seastar/libseastar_testing.{seastar_lib_ext}' + abseil_dep = ' '.join(f'$builddir/{mode}/abseil/{lib}' for lib in abseil_libs) + seastar_testing_libs = f'$seastar_testing_libs_{mode}' + local_libs = f'$seastar_libs_{mode} $libs' objs.extend([f'$builddir/{mode}/abseil/{lib}' for lib in abseil_libs]) + + if do_lto: + local_libs += ' -flto=thin -ffat-lto-objects' + else: + local_libs += ' -fno-lto' if binary in tests: if binary in pure_boost_tests: local_libs += ' ' + maybe_static(args.staticboost, '-lboost_unit_test_framework') if binary not in tests_not_using_seastar_test_framework: - local_libs += ' ' + f"$seastar_testing_libs_{mode}" + local_libs += f' {seastar_testing_libs}' else: local_libs += ' ' + '-lgnutls' + ' ' + '-lboost_unit_test_framework' # Our code's debugging information is huge, and multiplied @@ -2125,7 +2169,7 @@ def write_build_file(f, f.write(' libs = {}\n'.format(local_libs)) else: if binary == 'scylla': - local_libs += ' ' + "$seastar_testing_libs_{}".format(mode) + local_libs += f' {seastar_testing_libs}' f.write('build $builddir/{}/{}: {}.{} {} | {} {} {}\n'.format(mode, binary, regular_link_rule, mode, str.join(' ', objs), seastar_dep, seastar_testing_dep, abseil_dep)) f.write(' libs = {}\n'.format(local_libs)) f.write(f'build $builddir/{mode}/{binary}.stripped: strip $builddir/{mode}/{binary}\n') @@ -2210,7 +2254,9 @@ def write_build_file(f, compiles[obj] = cc for obj in compiles: src = compiles[obj] - f.write('build {}: cxx.{} {} || {} {}\n'.format(obj, mode, src, seastar_dep, gen_headers_dep)) + seastar_dep = f'$builddir/{mode}/seastar/libseastar.{seastar_lib_ext}' + abseil_dep = ' '.join(f'$builddir/{mode}/abseil/{lib}' for lib in abseil_libs) + f.write(f'build {obj}: cxx.{mode} {src} || {seastar_dep} {abseil_dep} {gen_headers_dep}\n') if src in modeval['per_src_extra_cxxflags']: f.write(' cxxflags = {seastar_cflags} $cxxflags $cxxflags_{mode} {extra_cxxflags}\n'.format(mode=mode, extra_cxxflags=modeval["per_src_extra_cxxflags"][src], **modeval)) for swagger in swaggers: @@ -2219,7 +2265,7 @@ def write_build_file(f, obj = swagger.objects(gen_dir)[0] src = swagger.source f.write('build {} | {} : swagger {} | {}/scripts/seastar-json2code.py\n'.format(hh, cc, src, args.seastar_path)) - f.write('build {}: cxx.{} {}\n'.format(obj, mode, cc)) + f.write(f'build {obj}: cxx.{mode} {cc}\n') for hh in serializers: src = serializers[hh] f.write('build {}: serializer {} | idl-compiler.py\n'.format(hh, src)) @@ -2235,7 +2281,7 @@ def write_build_file(f, grammar.source.rsplit('.', 1)[0])) for cc in grammar.sources('$builddir/{}/gen'.format(mode)): obj = cc.replace('.cpp', '.o') - f.write('build {}: cxx.{} {} || {}\n'.format(obj, mode, cc, ' '.join(serializers))) + f.write(f'build {obj}: cxx.{mode} {cc} || {" ".join(serializers)}\n') flags = '-Wno-parentheses-equality' if cc.endswith('Parser.cpp'): # Unoptimized parsers end up using huge amounts of stack space and overflowing their stack @@ -2243,12 +2289,14 @@ def write_build_file(f, if '-DSANITIZE' in modeval['cxxflags'] and has_sanitize_address_use_after_scope: flags += ' -fno-sanitize-address-use-after-scope' - f.write(f' obj_cxxflags = {flags}\n') + f.write(' obj_cxxflags = %s\n' % flags) f.write(f'build $builddir/{mode}/gen/empty.cc: gen\n') for hh in headers: f.write('build $builddir/{mode}/{hh}.o: checkhh.{mode} {hh} | $builddir/{mode}/gen/empty.cc || {gen_headers_dep}\n'.format( mode=mode, hh=hh, gen_headers_dep=gen_headers_dep)) + seastar_dep = f'$builddir/{mode}/seastar/libseastar.{seastar_lib_ext}' + seastar_testing_dep = f'$builddir/{mode}/seastar/libseastar_testing.{seastar_lib_ext}' f.write('build {seastar_dep}: ninja $builddir/{mode}/seastar/build.ninja | always\n' .format(**locals())) f.write(' pool = submodule_pool\n') @@ -2406,6 +2454,12 @@ def write_build_file(f, mode = {mode} ''')) + + build_ninja_files=[] + for mode in build_modes: + build_ninja_files += [f'{outdir}/{mode}/seastar/build.ninja'] + build_ninja_files += [f'{outdir}/{mode}/abseil/build.ninja'] + f.write(textwrap.dedent('''\ rule configure command = ./configure.py --out={buildfile_final_name}.new --out-final-name={buildfile_final_name} $configure_args && mv {buildfile_final_name}.new {buildfile_final_name} @@ -2425,7 +2479,7 @@ def write_build_file(f, description = List configured modes build mode_list: mode_list default {modes_list} - ''').format(modes_list=' '.join(default_modes), build_ninja_list=' '.join([f'{outdir}/{mode}/{dir}/build.ninja' for mode in build_modes for dir in ['seastar', 'abseil']]), **globals())) + ''').format(modes_list=' '.join(default_modes), build_ninja_list=" ".join(build_ninja_files), **globals())) unit_test_list = set(test for test in build_artifacts if test in set(tests)) f.write(textwrap.dedent('''\ rule unit_test_list @@ -2468,6 +2522,8 @@ def create_build_system(args): mode_config['per_src_extra_cxxflags']['release.cc'] = ' '.join(get_release_cxxflags(scylla_product, scylla_version, scylla_release)) + prepare_advanced_optimizations(modes=modes, build_modes=build_modes, args=args) + if not args.dist_only: global user_cflags, libs # args.buildfile builds seastar with the rules of From dd1a847d61689c2383d14c802c5ecb894b48546a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 25 Jan 2023 14:17:43 +0100 Subject: [PATCH 100/397] configure.py: enable LTO in release builds by default --- configure.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/configure.py b/configure.py index e8a7a71ea9df..3d84035aae89 100755 --- a/configure.py +++ b/configure.py @@ -702,7 +702,7 @@ def find_ninja(): help='List all available build artifacts, that can be passed to --with') arg_parser.add_argument('--date-stamp', dest='date_stamp', type=str, help='Set datestamp for SCYLLA-VERSION-GEN') -add_tristate(arg_parser, name='lto', dest='lto', default=False, +add_tristate(arg_parser, name='lto', dest='lto', default=True, help='link-time optimization.') arg_parser.add_argument('--use-cmake', action=argparse.BooleanOptionalAction, default=False, help='Whether to use CMake as the build system') arg_parser.add_argument('--coverage', action = 'store_true', help = 'Compile scylla with coverage instrumentation') From 6f01ceae3da29801c9427d4fae759211c38db758 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Tue, 17 Jan 2023 03:32:57 +0100 Subject: [PATCH 101/397] configure.py: don't include non-default modes in dist-server-* rules dist-server-tar only includes default modes. Let dist-server-deb and dist-server-rpm behave consistently with it. --- configure.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/configure.py b/configure.py index 3d84035aae89..d908794408dc 100755 --- a/configure.py +++ b/configure.py @@ -2380,8 +2380,8 @@ def write_build_file(f, build dist-unified-tar: phony {' '.join([f'$builddir/{mode}/dist/tar/{scylla_product}-unified-{scylla_version}-{scylla_release}.{arch}.tar.gz' for mode in default_modes])} build dist-unified: phony dist-unified-tar - build dist-server-deb: phony {' '.join(['$builddir/dist/{mode}/debian'.format(mode=mode) for mode in build_modes])} - build dist-server-rpm: phony {' '.join(['$builddir/dist/{mode}/redhat'.format(mode=mode) for mode in build_modes])} + build dist-server-deb: phony {' '.join(['$builddir/dist/{mode}/debian'.format(mode=mode) for mode in default_modes])} + build dist-server-rpm: phony {' '.join(['$builddir/dist/{mode}/redhat'.format(mode=mode) for mode in default_modes])} build dist-server-tar: phony {' '.join(['$builddir/{mode}/dist/tar/{scylla_product}-{scylla_version}-{scylla_release}.{arch}.tar.gz'.format(mode=mode, scylla_product=scylla_product, arch=arch, scylla_version=scylla_version, scylla_release=scylla_release) for mode in default_modes])} build dist-server-debuginfo: phony {' '.join(['$builddir/{mode}/dist/tar/{scylla_product}-debuginfo-{scylla_version}-{scylla_release}.{arch}.tar.gz'.format(mode=mode, scylla_product=scylla_product, arch=arch, scylla_version=scylla_version, scylla_release=scylla_release) for mode in default_modes])} build dist-server: phony dist-server-tar dist-server-debuginfo dist-server-rpm dist-server-deb From f73b122de31925978b8e066f4acae328fcaa09fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Tue, 31 May 2022 14:05:23 +0200 Subject: [PATCH 102/397] pgo: introduce a PGO training script Profile-guided optimization consists of the following steps: 1. Build the program as usual, but with with special options (instrumentation or just some supplementary info tables, depending on the exact flavor of PGO in use). 2. Collect an execution profile from the special binary by running a training workload on it. 3. Rebuild the program again, using the collected profile. This commit introduces a script automating step 2: running PGO training workloads on Scylla. The contents of training workloads will be added in future commits. The changes in configure.py responsible for steps 1. and 3. will also appear in future commits. As input, the script takes a path to the instrumented binary, a path to a the output file, and a directory with (optionally) prepopulated datasets for use in training. The output profile file can be then passed to the compiler to perform a PGO build. The script current supports two kinds of PGO instrumentation: LLVM instrumentation (binary instrumented with -fprofile-generate and -fcs-profile-generate passed to clang during compilation) and BOLT instrumentation (binary instrumented with `llvm-bolt -instrument`, with logs from this operation saved to $binary_path.boltlog) The actual training workloads for generating the profile will be added in later commits. --- pgo/conf/cassandra-rackdc.properties | 2 + pgo/conf/scylla.yaml | 1 + pgo/pgo.py | 735 +++++++++++++++++++++++++++ 3 files changed, 738 insertions(+) create mode 100644 pgo/conf/cassandra-rackdc.properties create mode 120000 pgo/conf/scylla.yaml create mode 100644 pgo/pgo.py diff --git a/pgo/conf/cassandra-rackdc.properties b/pgo/conf/cassandra-rackdc.properties new file mode 100644 index 000000000000..e39f2d2c5018 --- /dev/null +++ b/pgo/conf/cassandra-rackdc.properties @@ -0,0 +1,2 @@ +dc=dc1 +rack=rack1 diff --git a/pgo/conf/scylla.yaml b/pgo/conf/scylla.yaml new file mode 120000 index 000000000000..6128a1ecea3c --- /dev/null +++ b/pgo/conf/scylla.yaml @@ -0,0 +1 @@ +../../conf/scylla.yaml \ No newline at end of file diff --git a/pgo/pgo.py b/pgo/pgo.py new file mode 100644 index 000000000000..f40bdcf8c439 --- /dev/null +++ b/pgo/pgo.py @@ -0,0 +1,735 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright (C) 2023-present ScyllaDB +# + +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# + +from collections.abc import Coroutine, AsyncIterator, Callable +from contextvars import ContextVar +from dataclasses import dataclass +from typing import Any, Optional +import asyncio +import contextlib +import glob +import json +import logging +import os +import pathlib +import random +import re +import shlex +import shutil +import signal +import subprocess +import sys +import tempfile +import typing +import uuid + +################################################################################ +# Common aliases. + +PathLike = str | pathlib.Path +T = typing.TypeVar('T') + +@dataclass +class Process: + p: asyncio.subprocess.Process + argv: list[str] + logfile: Optional[PathLike] + def __str__(self) -> str: + status = self.p.returncode if self.p.returncode is not None else 'running' + return f"PID={self.p.pid} argv={self.argv}, status={status}" + +@dataclass +class ProcessError(Exception): + proc: Process + +# process outcome = (process metadata, stdout contents if captured, stderr contents if captured) +ProcessOutcome = tuple[Process, Optional[bytes], Optional[bytes]] + +def q(s: PathLike): + return shlex.quote(str(s)) + +################################################################################ +# Loggers + +module_logger = logging.getLogger("pgo") +config_logger = module_logger.getChild("config") +process_logger = module_logger.getChild("processes") +training_logger = module_logger.getChild("training") + +# If set, stdout and stderr of child processes will be output on the stdout +# and stderr of the script, and less urgent log levels will be enabled. +SUBPROCESS_OUTPUT: ContextVar[bool] = ContextVar('SUBPROCESS_OUTPUT', default=False) + +# If set, less urgent log levels will be printed to stdout. +VERBOSE: ContextVar[bool] = ContextVar('VERBOSE', default=False) + +# If set, every child process $pid will have its stdout and stderr logged to +# {LOGDIR.get()}/$pid.log +LOGDIR: ContextVar[Optional[str]] = ContextVar('LOGDIR') + +# If set, all output of the script will be written to this path. Both all logs +# and all subprocess outputs. +GLOBAL_LOGFILE: ContextVar[Optional[str]] = ContextVar('GLOBAL_LOGFILE', default=None) + +################################################################################ +# Environment + +NODE_CPUSETS: ContextVar[Optional[list[str]]] = ContextVar('NODE_CPUSETS') +CS_CPUSET: ContextVar[Optional[str]] = ContextVar('CS_CPUSET') + +def configure_cpusets(): + """ + Let's try to schedule Scylla nodes on separate cpusets, and the load generators on yet + different cpuset, to speed up the training and/or allow for reasonable tests. + """ + num_cpus = os.cpu_count() + if num_cpus >= 12: + config_logger.info(f"{num_cpus} (>=12) CPUs available") + NODE_CPUSETS.set([f"0,{num_cpus//2}", f"1,{1+num_cpus//2}", f"2,{2+num_cpus//2}"]) + CS_CPUSET.set(f"3-{num_cpus//2-1},{3+num_cpus//2}-{num_cpus-1}") + else: + config_logger.info(f"{num_cpus} (<12) CPUs available") + config_logger.warning(f"Due to a small number of available CPUs, the training will be run in overprovisioned mode: load generators and Scylla nodes will share cores. This will make the training much slower. This slowness could also result in an inaccurate (unrealistic) profile.") + NODE_CPUSETS.set(None) + CS_CPUSET.set(None) + config_logger.info(f"Choosing cpusets for nodes: {NODE_CPUSETS.get()}") + config_logger.info(f"Choosing cpuset for load generators: {CS_CPUSET.get()}") + +JAVA_HOME: ContextVar[Optional[str]] = ContextVar('JAVA_HOME') + +async def configure_java() -> None: + """ + cassandra-stress can only deal with Java 8 or Java 11 + """ + version_output = (await bash("java -version", stderr=asyncio.subprocess.PIPE))[2] + assert type(version_output) == bytes + version_first_line = version_output.decode().split(sep='\n')[0] + config_logger.info(f"First line of java -version: {version_first_line}") + if re.search(r'version.*1\.8\.0', version_first_line): + config_logger.info(f"Default Java version recognized as Java 8. Proceeding with the default.") + JAVA_HOME.set(None) + elif re.search(r'version.*11\.[0-9]+\.[0-9]+', version_first_line): + config_logger.info(f"Default Java version recognized as Java 11. Proceeding with the default.") + JAVA_HOME.set(None) + else: + config_logger.info(f"Default Java version recognized as neither Java 8 nor Java 11.") + if os.path.exists(java_8_path := '/usr/lib/jvm/java-1.8.0'): + config_logger.warning(f"{java_8_path} found. Choosing it as JAVA_HOME.") + JAVA_HOME.set(java_8_path) + elif os.path.exists(java_11_path := '/usr/lib/jvm/java-11'): + config_logger.warning(f"{java_11_path} found. Choosing it as JAVA_HOME.") + JAVA_HOME.set(java_11_path) + else: + error = "Failed to find a suitable Java version. Java 8 or Java 11 is required." + config_logger.error(error) + raise RuntimeError(error) + +################################################################################ +# Child process utilities + +@contextlib.asynccontextmanager +async def with_context(ctx: ContextVar[T], val: T) -> AsyncIterator[None]: + x = ctx.set(val) + try: + yield None + finally: + ctx.reset(x) + +async def clean_gather(*coros: Coroutine) -> list: + """Differs from asyncio.gather() in that it cancels all other tasks when one + fails, and waits for the cancellations to complete. + """ + async with asyncio.TaskGroup() as tg: + return await asyncio.gather(*[tg.create_task(c) for c in coros]) + +def shielded(coro: Coroutine) -> Coroutine: + """Launches a task wrapping `coro` and returns a coroutine which reliably + awaits the task. + + The returned coroutine (and the task it awaits) cannot be cancelled from + the outside. `await` called on the returned coroutine, when cancelled, will + still wait until the `coro` task exits. The task won't see the cancellation + request. + + This is mainly useful for preventing cleanup tasks ("destructors") from + being interrupted. + """ + return _shielded(asyncio.create_task(coro)) + +async def _shielded(task: asyncio.Task): + try: + await asyncio.shield(task) + finally: + await task + +def cancel_process(proc: Process, timeout: Optional[float] = None, sig: int = signal.SIGINT) -> Coroutine: + """Cancels a child process spawned with run(). + + If the child process has already exited, does nothing. Otherwise sends a + SIGINT to its process group and returns a coroutine which waits (with an + optional timeout) for the child to exit. This coroutine is shielded from + cancellation -- if cancelled, it will defer the cancel until the child + exits or the wait times out. + + Note that the proper way to cancel the child is to send a SIGINT to its + PGID, not just its PID. This is how most programs (e.g. shells) expect to + be cancelled, because sending SIGINT to the entire process group is what + terminals do when they receive Ctrl+C. So e.g. if our child process is a + shell which has spawned its own child, sending SIGINT just to the shell + would kill only the shell, but its own child would not see the SIGINT, + staying alive even after its parent shell and its grandparent script exit. + + Assumes the PGID of `proc` is equal to its PID, as is set up by default by + run(). + """ + # This purpose of this optimistic early return is just to avoid log spam. + # (If the process has exited, "Cancelling" in logs would be misleading). + if proc.p.returncode is not None: + # This proc.p.wait() should be a no-op. + return proc.p.wait() + return shielded(_cancel_process(proc=proc, timeout=timeout, sig=sig)) + +async def _cancel_process(proc: Process, timeout: Optional[float], sig: int) -> None: + process_logger.info(f"Cancelling (using signal {sig}) PIDG of {proc}") + try: + os.killpg(proc.p.pid, sig) + except ProcessLookupError: + # This either means that the child's entire process group already quit, + # or that the child's PGID is different from its PID. + # The latter means that the programmer is breaking the assumptions of + # cancel_process(), so let's check against that prophylactically. + try: + assert os.getpgid(proc.p.pid) == proc.p.pid + except ProcessLookupError: + # Seems that the process really quit. That's OK. + pass + try: + async with asyncio.timeout(timeout): + await proc.p.wait() + except asyncio.TimeoutError as e: + process_logger.error(f"Error waiting for {proc}: wait timeout ({timeout}s) exceeded. Moving on without further wait. The process might be still alive after this script exits.") + raise + except BaseException as e: + process_logger.error(f"Error waiting for {proc}: {e}. The process might be still alive after this script exits.") + raise + +async def run(command: list[str], cpuset: Optional[str] = None, **kwargs) -> Process: + """Spawns a child process. + + kwargs are passed through to Popen(). + + The child is ran in its own process group (with PGID equal to its PID), + (unless process_group is explicitly set in kwargs). This is so that a + Ctrl+C sent to the script doesn't automatically send SIGINT to children. + This is important because an uninvited interruption of a child process + could easily mess up some cleanup procedures, and we don't want cleanups + to be interrupted, lest something is leaked when the script exits. + """ + cmd = list(command) + if cpuset: + cmd = ["taskset", "-c", cpuset] + cmd + + orig_cmd = cmd[:] # Copy the command here for the purpose of logging, before it's uglified up by the wrappers below. + + kwargs.setdefault("process_group", 0) + + # Optionally log standard streams to the per-process log and to the global log. + logdir = LOGDIR.get() + global_logfile = GLOBAL_LOGFILE.get() + n_handlers = bool(logdir) + bool(global_logfile) + if n_handlers > 0: + cmd = ["bash", "-c", 'exec 1> >(tee -a /dev/fd/3 >&1); exec 2> >(tee -a /dev/fd/3 >&2); exec "$@"', "run()"] + cmd + if global_logfile: + if n_handlers > 1: + cmd = ["bash", "-c", 'exec 3> >(tee >(ts "%Y-%m-%d %H:%M:%.S" >>"$0") >&3); exec "$@"', global_logfile] + cmd + else: + cmd = ["bash", "-c", 'exec 3> >(ts "%Y-%m-%d %H:%M:%.S" >>"$0"); exec "$@"', global_logfile] + cmd + n_handlers -= 1 + if logdir: + if n_handlers > 1: + cmd = ["bash", "-c", 'exec 3> >(tee -a "$0"/$$.log >&3); exec "$@"', logdir] + cmd + else: + cmd = ["bash", "-c", 'exec 3>>"$0"/$$.log; exec "$@"', logdir] + cmd + n_handlers -= 1 + + if not SUBPROCESS_OUTPUT.get(): + kwargs.setdefault("stdout", asyncio.subprocess.DEVNULL) + kwargs.setdefault("stderr", asyncio.subprocess.DEVNULL) + + process_logger.debug(f"Running a process: {orig_cmd}") + p = await asyncio.create_subprocess_exec(*cmd, **kwargs) + + logfile = f"{logdir}/{p.pid}.log" if logdir else None + proc = Process(p, orig_cmd, logfile) + process_logger.debug(f"Started {proc}") + return proc + +async def wait(proc: Process) -> ProcessOutcome: + """Waits for a process spawned with run() to exit. + + If the wait is cancelled, the child process is cancelled, + and the wait doesn't return until the child exits. + """ + process_logger.debug(f"Waiting for {proc}") + try: + o, e = await proc.p.communicate() + process_logger.debug(f"Reaped {proc}") + return (proc, o, e) + finally: + await cancel_process(proc) + +async def run_checked(command: list[str], **kwargs) -> ProcessOutcome: + """Runs a process to completion. + + Checks that it exited with code 0, otherwise raises an exception. + Convenience wrapper over run() and wait(). + """ + proc, stdout, stderr = await wait(await run(command, **kwargs)) + assert proc.p.returncode is not None + if proc.p.returncode != 0: + raise ProcessError(proc) + return proc, stdout, stderr + +async def query(command: list[str], **kwargs) -> bytes: + """Runs a process and returns its stdout's contents. Convenience wrapper over run_checked.""" + proc, stdout, stderr = await run_checked(command, stdout=asyncio.subprocess.PIPE, **kwargs) + assert type(stdout) == bytes + return stdout + +async def bash(command: str, **kwargs) -> ProcessOutcome: + """Runs a bash command. Convenience wrapper over run_checked.""" + return await run_checked(["bash", "-c", command], **kwargs) + +################################################################################ +# Scylla cluster utilities + +async def wait_for_node(proc: Process, addr: str, timeout: Optional[float] = None): + """Waits for the Scylla node to start serving traffic (by opening its CQL port). + Raises a timeout exception if the optional `timeout` elapses or if the node process + exits before opening the port. + """ + cql_port = 9042 + async with asyncio.TaskGroup() as tg: + tasks: list[asyncio.Task] = [ + died := tg.create_task(proc.p.wait()), + started := tg.create_task(wait_for_port(addr, cql_port)), + ] + done, not_done = await asyncio.wait(tasks, return_when=asyncio.FIRST_COMPLETED, timeout=timeout) + for t in tasks: + t.cancel() + if started in done: + return + elif died in done: + training_logger.error(f"Node {addr} died before opening the CQL port ({cql_port}).") + raise asyncio.TimeoutError + else: + training_logger.error(f"Node {addr} did not open the CQL port ({cql_port}) within the arbitrary timeout ({timeout} seconds).") + raise asyncio.TimeoutError + +# We attempt to randomize cluster names in order to prevent accidental spooky +# inter-cluster talk when the script is run concurrently with +# other instances of Scylla (possibly from other invocation of the script) +# on the same machine. +def cluster_metadata(workdir: PathLike) -> dict: + """Reads the cluster metadata file for a given cluster workdir. + If there is no metadata file, creates it and initializes its parameters with arbitrarily chosen arguments. + """ + fname = f"{workdir}/cluster_metadata" + if os.path.exists(fname): + with open(fname, "r") as f: + res = json.load(f) + assert res["name"] and res["subnet"] + return res + else: + subnet = f"127.{random.randrange(0,256)}.{random.randrange(0,256)}" + name = fr"PGO:{workdir}:{uuid.uuid1()}" + res = {"subnet": subnet, "name": name} + with open(fname, "w") as f: + json.dump(res, f) + return res + +class AddressAlreadyInUseException(Exception): + def __init__(self, addresses, diagnostics): + super().__init__(f"Attempted to start a cluster with addresses {','.join(addresses)}, but some of them appear to be already used:\n{diagnostics}") + +def concat_lists_with_separator(lists: list[list[T]], sep: T) -> list: + """concat_lists_with_separator([["src", "x"], ["src", "y"], ["src", "z"]], "|") == ["src", "x", "|", "src", "y", "|", "src", "z"] + """ + res: list = [] + for x in lists: + res.extend(x) + res.append(sep) + return res[:-1] + +async def validate_addrs_unused(addresses: list[str]) -> None: + """Checks that there are no TCP sockets listening on any of the addresses. + If such sockets exist, raises an exception. + """ + # ss_filter looks like this: src 127.0.0.1 | src 127.0.0.2 | src 127.0.0.3 + ss_filter = concat_lists_with_separator([["src", x] for x in addresses], "|") + + ss_command = ["ss", "--tcp", "--numeric", "--listening", "--no-header", "--processes"] + ss_filter + ss_output = (await query(ss_command)).strip() + if ss_output: + diagnostics = f"Command: {shlex.join(ss_command)}\nOutput (expected empty):\n{ss_output.decode()}" + raise AddressAlreadyInUseException(addresses, diagnostics) + +async def start_node(executable: PathLike, cluster_workdir: PathLike, addr: str, seed: str, cluster_name: str, extra_opts: list[str]) -> Process: + """Starts a Scylla node. + Its --workdir will be $cluster_workdir/$addr/, its log file will be $cluster_workdir/$addr.log, + and its PWD will be $cluster_workdir. + + """ + # These paths are relative to cluster_workdir. + # The directory change to it happens via the cwd=cluster_workdir in run() + llvm_profile_file = f"{addr}-%m.profraw" + scylla_workdir = f"{addr}" + logfile = f"{addr}.log" + command = [ + "env", + f"LLVM_PROFILE_FILE={llvm_profile_file}", + f"SCYLLA_HOME={os.path.realpath(os.getcwd())}", # We assume that the script has Scylla's `conf/` as its filesystem neighbour. + os.path.realpath(executable), + f"--workdir={scylla_workdir}", + "--ring-delay-ms=0", + "--developer-mode=yes", + "--memory=1G", + "--unsafe-bypass-fsync=1", + "--kernel-page-cache=1", + f"--seed-provider-parameters=seeds={seed}", + f"--listen-address={addr}", + f"--api-address={addr}", + f"--rpc-address={addr}", + f"--prometheus-address={addr}", + f"--cluster-name={cluster_name}", + f"--endpoint-snitch=GossipingPropertyFileSnitch", + f"--read-request-timeout-in-ms=60000", + f"--write-request-timeout-in-ms=60000", + f"--cas-contention-timeout-in-ms=60000", + ] + list(extra_opts) + return await run(['bash', '-c', fr"""exec {shlex.join(command)} >{q(logfile)} 2>&1"""], cwd=cluster_workdir) + +async def start_cluster(executable: PathLike, addrs: list[str], cpusets: Optional[list[str]], workdir: PathLike, cluster_name: str, extra_opts: list[str]) -> list[Process]: + """Starts a Scylla cluster, with a directory structure like this: + my_workdir/ # {workdir} + ├── 127.73.130.1.log + ├── 127.73.130.1 # {addrs[0]} + │   └── commitlog, data, hints, view_hints + ├── 127.73.130.2.log + ├── 127.73.130.2 # {addrs[1]} + │   └── commitlog, data, hints, view_hints + ├── 127.73.130.3.log + ├── 127.73.130.3 # {addrs[2]} + │   └── commitlog, data, hints, view_hints + └── cluster_metadata + + The (Linux-wise) working directory of all nodes is the top level directory ({workdir}). + That's where their profile files will be output. + + If start_cluster() is cancelled, it will cancel already spawned nodes + and wait for them to exit. + """ + assert addrs + + # Cannot prevent address clashes (because TOCTOU), but better than no validation. + await validate_addrs_unused(addrs) + + if cpusets: + assert len(cpusets) >= len(addrs) + cpuset_args = [[f"--cpuset={cpusets[i]}"] for i in range(len(addrs))] + else: + cpuset_args = [["--smp=2", "--overprovisioned"] for i in range(len(addrs))] + + timeout = 300 + procs = [] + seed = addrs[0] + try: + for i in range(0, len(addrs)): + proc = await start_node(executable, addr=addrs[i], seed=seed, cluster_workdir=workdir, cluster_name=cluster_name, extra_opts=extra_opts+cpuset_args[i]) + procs.append(proc) + await wait_for_node(proc, addrs[i], timeout) + except: + await stop_cluster(procs, addrs) + raise + return procs + +async def stop_cluster(procs: list[Process], addrs: list[str]) -> None: + """Stops a Scylla cluster started with start_cluster(). + Doesn't return until all nodes exit, even if stop_cluster() is cancelled. + + """ + await clean_gather(*[cancel_process(p, timeout=60) for p in procs]) + +async def wait_for_port(addr: str, port: int) -> None: + await bash(fr'until printf "" >>/dev/tcp/{addr}/{port}; do sleep 0.1; done 2>/dev/null') + +async def merge_profraw(directory: PathLike) -> None: + "Merges LLVM *.profraw files in the directory into a prof.profdata file therein." + if glob.glob(f"{directory}/*.profraw"): + await bash(fr"llvm-profdata merge {q(directory)}/*.profraw -output {q(directory)}/prof.profdata") + +async def get_bolt_opts(executable: PathLike) -> list[str]: + """Returns the extra opts which have to be passed to a BOLT-instrumented Scylla + to trigger a generation of a BOLT profile file. + + The relevant info (address of BOLT's dump function) is extracted from $executable.boltlog -- + it is assumed that BOLT's logs printed during the instrumentation of this executable are written there. + If there is no such file, returns an empty list. + """ + file = f"{executable}.boltlog" + if os.path.exists(file): + opt = await query(["sed", "-n", r'/entry point is/s/^.*\(0x[[:xdigit:]]*\).*$/\1/p', file]) + addr = opt.decode("ascii").strip() + + return [f"--bolt-instrumentation-dump-address={addr}"] + else: + return [] + +async def quiesce_cluster(addrs: list[str]) -> None: + """Waits until all given nodes are done with compactions.""" + training_logger.info("Waiting for compactions to end.") + grep = shlex.quote("compaction_manager_compactions{") + awk = shlex.quote("{sum += $2} END {print sum}") + while True: + _, out, _ = await bash(fr"""for x in {" ".join(addrs)}; do curl --silent $x:9180/metrics; done | grep {grep} | awk {awk}""", + stdout=asyncio.subprocess.PIPE) + assert type(out) == bytes + if float(out.decode()) == 0: + break + await asyncio.sleep(10) + +@contextlib.asynccontextmanager +async def with_cluster(executable: PathLike, workdir: PathLike, cpusets: Optional[list[str]] = None) -> AsyncIterator[tuple[list[str], list[Process]]]: + """Provides a Scylla cluster. + Doesn't monitor the state of the cluster in any way, just starts the cluster as + the context manager enters, waits for each CQL port to open, yields the cluster's + control info and stops the cluster as the context manager exits. + """ + meta = cluster_metadata(workdir) + cluster_name = meta["name"] + subnet = meta["subnet"] + addrs = [f"{subnet}.{i}" for i in range(1,255)][:3] + cpusets = cpusets or NODE_CPUSETS.get() + extra_opts = await get_bolt_opts(executable) + training_logger.debug(f"BOLT opts for {executable} are {extra_opts}") + training_logger.info(f"Starting a cluster of {executable} in {workdir}") + procs = await start_cluster(addrs=addrs, executable=executable, workdir=workdir, cpusets=cpusets, cluster_name=cluster_name, extra_opts=extra_opts) + training_logger.info(f"Started the cluster in {workdir}") + try: + yield addrs, procs + finally: + training_logger.info(f"Stopping the cluster in {workdir}") + await stop_cluster(procs, addrs) + training_logger.info(f"Stopped the cluster in {workdir}") + +################################################################################ +# cassandra-stress utilities + +def cs_command(cmd: list[str], n: int, node: str, cl: str, pop: Optional[str] = None, warmup: bool = False, rate: str = "threads=200", schema: Optional[str] = None) -> list[str]: + """Strings together a cassandra-stress command from given options.""" + return (["env", f"JAVA_HOME={JAVA_HOME.get()}"] if JAVA_HOME.get() else []) + [ + "../tools/java/tools/bin/cassandra-stress", + *cmd, + f"n={n}", + f"cl={cl}", + ] + (["no-warmup"] if not warmup else []) + [ + ] + (["-pop", pop] if pop else []) + [ + "-mode", "native", "cql3", "protocolVersion=4", + "-node", node, + "-rate", rate, + ] + (["-schema", schema] if schema else []) + [ + ] + +async def cs(run_kwargs: dict[str, Any] = {}, **params: Any) -> Process: + """Runs a cassandra-stress process. + Raises an exception if it reports a workload failure. + """ + run_kwargs.setdefault('cpuset', CS_CPUSET.get()) + cmd = cs_command(**params) + training_logger.info(f"Running cassandra-stress: {cmd}") + proc, *_ = await run_checked(cs_command(**params), **run_kwargs) + training_logger.info(f"cassandra-stress finished successfully") + return proc + +RF3_SCHEMA = "replication(strategy=NetworkTopologyStrategy,dc1=3)" +RF1_SCHEMA = "replication(strategy=NetworkTopologyStrategy,dc1=1)" + +def kw(**kwargs): + """Python syntax hack. Personal preference. + kw(a=0, b=1, c=2) == {"a": 0, "b": 1, "c": 2} + """ + return kwargs + +################################################################################ +# Training workload definitions + +# The reason why we separate training and population phases is that the same training happens +# multiple times (for PGO, CSPGO and BOLT separately), so it saves some time to only populate once. +# +# The reason why there is a level of indirection between trainers and populators (trainers refer to their +# dataset by dataset name, not by function name) is to facilitate sharing of datasets between trainers. +# But this facility isn't currently used. + +@contextlib.asynccontextmanager +async def with_cs_populate(executable: PathLike, workdir: PathLike) -> AsyncIterator[str]: + """Provides a Scylla cluster and waits for compactions to end before stopping it.""" + async with with_cluster(executable=executable, workdir=workdir) as (addrs, procs): + yield addrs[0] + async with asyncio.timeout(3600): + # Should it also flush memtables? + await quiesce_cluster(addrs=addrs) + +@contextlib.asynccontextmanager +async def with_cs_train(executable: PathLike, workdir: PathLike) -> AsyncIterator[str]: + """Provides a Scylla cluster and merges generated profile files after it's stopped.""" + async with with_cluster(executable=executable, workdir=workdir) as (addrs, procs): + yield addrs[0] + await merge_profraw(workdir) + +class Trainer(typing.Protocol): + async def __call__(self, executable: PathLike, workdir: PathLike) -> None: ... +class Populator(typing.Protocol): + async def __call__(self, executable: PathLike, workdir: PathLike) -> None: ... + +# Maps training workload names to their dataset names and functions responsible for running the workload. +trainers: dict[str, tuple[str, Trainer]] = {} +# Maps dataset names to the functions responsible for populating them. +populators: dict[str, Populator] = {} + +################################################################################ +# Training procedures + +def executable_exists(executable: PathLike) -> bool: + """Checks if the file exists and is executable""" + return bool(shutil.which(executable)) + +async def populate_datasets(executable: PathLike, dataset_names: list[str], dataset_dir: PathLike) -> None: + """Populates the given datasets if they don't already exist. + + A "dataset" is simply a copy of the entire cluster workdir -- it consists + of multiple Scylla workdirs and a bit of metadata. + After this function, there will be a $dataset_dir/$x cluster workdir for each dataset x. + These cluster workdirs can be copied somewhere for training and restored with start_cluster(). + """ + for t in dataset_names: + t_dir = fr"{dataset_dir}/{t}" + if not os.path.exists(t_dir): + training_logger.info(f"Dataset {t_dir} doesn't exist. Populating.") + tmpdir = fr"{dataset_dir}/tmp-{t}" + await bash(fr"rm -rf {q(tmpdir)} && mkdir -p {q(tmpdir)}") + await populators[t](executable, tmpdir) + # If we have been using a profile-instrumented Scylla binary for populating, + # remove its leftover profile files in the cluster directory. + await bash(fr"find {q(tmpdir)} '(' -name '*.profraw' -o -name '*.fdata' ')' -delete") + await bash(fr"mv {q(tmpdir)}/ {q(dataset_dir)}/{t}") + training_logger.info(f"Dataset {t_dir} populated.") + else: + training_logger.info(f"Dataset {t_dir} already exists. Not populating.") + +async def train_full(executable: PathLike, output_profile_file: PathLike, dataset_dir: PathLike) -> None: + """Runs all known training workloads on the given executable, using + prepopulated datasets in `dataset_dir`, or populating them first if they + don't exist. Subsequent trainings will be able to reuse the datasets. + + The output of the training is a single profile file (prof.profdata) merged + from all profile files generated by all nodes in all workloads. + + The training clusters will be set up in the directory "workdirs", located in + the same directory as the output profile file. + """ + # We create ancestor directories of the output file, if they don't exist. + topdir = os.path.dirname(output_profile_file) + os.makedirs(topdir) + + # Set up logging to {output_profile_file}.log/. + os.makedirs(f"{output_profile_file}.log") + LOGDIR.set(os.path.realpath(f"{output_profile_file}.log")) + GLOBAL_LOGFILE.set(os.path.realpath(f"{output_profile_file}.log/global.log")) + for file in [f"{LOGDIR.get()}/script.log", GLOBAL_LOGFILE.get()]: + logfile = logging.FileHandler(str(file)) + logfile.setLevel(logging.DEBUG) + formatter = logging.Formatter(fmt='%(asctime)s %(levelname)-8s %(name)-14s %(message)s') + logfile.setFormatter(formatter) + logging.getLogger().addHandler(logfile) + + training_logger.info(f"Starting training of executable {executable}. Exhaustive logs can be found in {LOGDIR.get()}/") + + configure_cpusets() + await configure_java() + + assert executable_exists(executable) + + workdirs = fr"{topdir}/workdirs" + os.makedirs(workdirs) + + workloads = list(trainers) + await populate_datasets(executable=executable, dataset_names=[trainers[w][0] for w in workloads], dataset_dir=dataset_dir) + n = len(workloads) + for i, w in enumerate(workloads): + training_logger.info(f"Preparing training workload: {w} [{i+1} out of {n}]") + workdir = fr"{workdirs}/{w}" + w_dataset = fr"{dataset_dir}/{trainers[w][0]}" + training_logger.info(f"Copying dataset {w_dataset} to {workdir}") + await bash(fr"cp -r {q(w_dataset)} {q(workdir)}") + training_logger.info(f"Running training workload: {w} [{i+1} out of {n}]") + await trainers[w][1](executable=executable, workdir=workdir) + training_logger.info(f"Finished training workload: {w} [{i+1} out of {n}]") + + training_logger.info(f"Merging profile files") + + if profdatas := glob.glob(fr"{workdirs}/**/*.profdata", recursive=True): + await bash(fr"llvm-profdata merge {shlex.join(profdatas)} -output {q(output_profile_file)}") + + if fdatas := glob.glob(fr"{workdirs}/**/*.fdata", recursive=True): + await bash(fr"merge-fdata {shlex.join(fdatas)} > {q(output_profile_file)}") + + training_logger.info(f"Finished training of executable {executable}. Output file is {output_profile_file}") + +async def dump_log_tail(e: ProcessError): + msg = f"{e.proc} failed." + if e.proc.logfile: + msg += f" Dumping last 50 lines of its log.\nDump of {e.proc.logfile}:\n" + msg += (await query(["tail", "-n", "50", str(e.proc.logfile)])).decode() + process_logger.critical(msg) + +async def main() -> None: + try: + match sys.argv[1]: + case "train_full": + await train_full(executable=sys.argv[2], output_profile_file=sys.argv[3], dataset_dir=sys.argv[4]) + case _: + print(f"Unknown command {sys.argv[1]}") + except ProcessError as e: + await dump_log_tail(e) + raise + except ExceptionGroup as eg: + for ex in eg.exceptions: + if isinstance(ex, ProcessError): + await dump_log_tail(ex) + raise + +if __name__ == "__main__": + # We keep the working directory of the script in the directory where the script is placed, + # that is: ${scylla_repository}/pgo. + os.chdir(os.path.dirname(os.path.realpath(__file__))) + + logging.getLogger().setLevel(logging.NOTSET) + + # Set up logging to stdout. + console = logging.StreamHandler() + formatter = logging.Formatter(fmt='%(levelname)-8s %(name)-14s %(message)s') + console.setFormatter(formatter) + logging.getLogger().addHandler(console) + + if not VERBOSE.get(): + console.setLevel(logging.INFO) + console.addFilter(lambda r: r.levelno >= logging.WARNING if r.name == "pgo.processes" else True) + + asyncio.run(main()) From c1297dbcd2c4856de4d7fd2073905b9e8e463da6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Feb 2023 03:37:10 +0100 Subject: [PATCH 103/397] pgo: add a basic workload This commit adds the default cassandra-stress workload to the PGO training suite. --- pgo/pgo.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/pgo/pgo.py b/pgo/pgo.py index f40bdcf8c439..1ee6b170d1cf 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -603,6 +603,22 @@ async def __call__(self, executable: PathLike, workdir: PathLike) -> None: ... # Maps dataset names to the functions responsible for populating them. populators: dict[str, Populator] = {} +# BASIC ================================================== + +async def populate_basic(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + await cs(cmd=["write"], n=2000000, cl="local_quorum", schema=RF3_SCHEMA, node=server) + +async def train_basic(executable: PathLike, workdir: PathLike) -> None: + N = 2500000 # Preferably keep big enough to cause compactions. + async with with_cs_train(executable=executable, workdir=workdir) as server: + await cs(cmd=["mixed", "ratio(read=1,write=1)"], n=N, pop=f"dist=UNIFORM(1..{2000000})", cl="local_quorum", node=server) + +# disable as it's very similar to CLUSTERING workload +# and exactly the same as we use for our performance tests +#trainers["basic"] = ("basic_dataset", train_basic) +populators["basic_dataset"] = populate_basic + ################################################################################ # Training procedures From 65abecaedeeb5790607de6e835d9081f3ca54e36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Feb 2023 03:37:17 +0100 Subject: [PATCH 104/397] pgo: add a clustering workload In contrast to the basic workload, this workload uses clustering keys, CK range queries, RF=1, logged batches, and more CQL types. Tests seem to show that this workload is mostly aligned with the existing basic workload (where "aligned" means that training on workload A improves workload B about as much as training on workload B). The config YAML is based on the example YAML attached to cassandra-stress sources. --- pgo/conf/clustering.yaml | 92 ++++++++++++++++++++++++++++++++++++++++ pgo/pgo.py | 14 ++++++ 2 files changed, 106 insertions(+) create mode 100644 pgo/conf/clustering.yaml diff --git a/pgo/conf/clustering.yaml b/pgo/conf/clustering.yaml new file mode 100644 index 000000000000..035b8450a246 --- /dev/null +++ b/pgo/conf/clustering.yaml @@ -0,0 +1,92 @@ +# +# This is an example YAML profile for cassandra-stress +# +# insert data +# cassandra-stress user profile=/home/jake/stress1.yaml ops(insert=1) +# +# read, using query simple1: +# cassandra-stress profile=/home/jake/stress1.yaml ops(simple1=1) +# +# mixed workload (90/10) +# cassandra-stress user profile=/home/jake/stress1.yaml ops(insert=1,simple1=9) + + +# +# Keyspace info +# +keyspace: keyspace1 + +# +# The CQL for creating a keyspace (optional if it already exists) +# +keyspace_definition: | + CREATE KEYSPACE keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}; + +# +# Table info +# +table: standard1 + +# +# The CQL for creating a table you wish to stress (optional if it already exists) +# +table_definition: | + CREATE TABLE standard1 ( + name text, + choice boolean, + date timestamp, + address inet, + dbl double, + lval bigint, + ival int, + uid timeuuid, + value blob, + PRIMARY KEY((name,choice), date, address, dbl, lval, ival, uid) + ); + +# +# Optional meta information on the generated columns in the above table +# The min and max only apply to text and blob types +# The distribution field represents the total unique population +# distribution of that column across rows. Supported types are +# +# EXP(min..max) An exponential distribution over the range [min..max] +# EXTREME(min..max,shape) An extreme value (Weibull) distribution over the range [min..max] +# GAUSSIAN(min..max,stdvrng) A gaussian/normal distribution, where mean=(min+max)/2, and stdev is (mean-min)/stdvrng +# GAUSSIAN(min..max,mean,stdev) A gaussian/normal distribution, with explicitly defined mean and stdev +# UNIFORM(min..max) A uniform distribution over the range [min, max] +# FIXED(val) A fixed distribution, always returning the same value +# Aliases: extr, gauss, normal, norm, weibull +# +# If preceded by ~, the distribution is inverted +# +# Defaults for all columns are size: uniform(4..8), population: uniform(1..100B), cluster: fixed(1) +# +columnspec: + - name: name + size: uniform(10..20) + population: seq(1..3M) # the range of unique values to select for the field (default is 100Billion) + - name: date + cluster: uniform(20..40) + - name: lval + population: gaussian(1..1000) + cluster: uniform(1..4) + +insert: + partitions: uniform(1..50) # number of unique partitions to update in a single operation + # if batchcount > 1, multiple batches will be used but all partitions will + # occur in all batches (unless they finish early); only the row counts will vary + batchtype: LOGGED # type of batch to use + select: uniform(1..10)/10 # uniform chance any single generated CQL row will be visited in a partition; + # generated for each partition independently, each time we visit it + +# +# A list of queries you wish to run against the schema +# +queries: + simple1: + cql: select * from standard1 where name = ? and choice = ? LIMIT 100 + fields: samerow # samerow or multirow (select arguments from the same row, or randomly from all rows in the partition) + range1: + cql: select * from standard1 where name = ? and choice = ? and date >= ? LIMIT 100 + fields: multirow # samerow or multirow (select arguments from the same row, or randomly from all rows in the partition) diff --git a/pgo/pgo.py b/pgo/pgo.py index 1ee6b170d1cf..bd521c5988a5 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -619,6 +619,20 @@ async def train_basic(executable: PathLike, workdir: PathLike) -> None: #trainers["basic"] = ("basic_dataset", train_basic) populators["basic_dataset"] = populate_basic +# CLUSTERING ================================================== + +async def populate_clustering(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + await cs(cmd=["user", "profile=./conf/clustering.yaml", "ops(insert=1)"], n=5000000, cl="local_quorum", node=server) + +async def train_clustering(executable: PathLike, workdir: PathLike) -> None: + N = 2500000 # Preferably keep big enough to cause compactions. + async with with_cs_train(executable=executable, workdir=workdir) as server: + await cs(cmd=["user", "profile=./conf/clustering.yaml", "ops(insert=5,simple1=1,range1=1)"], n=N, cl="local_quorum", node=server) + +trainers["clustering"] = ("clustering_dataset", train_clustering) +populators["clustering_dataset"] = populate_clustering + ################################################################################ # Training procedures From e217c124a6321d9c92c18fca93ba887fccd76c42 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Feb 2023 03:37:24 +0100 Subject: [PATCH 105/397] pgo: add a decommission workload This workload is added to teach PGO about streaming. Tests show that this workload is mostly orthogonal to CQL workloads (where "orthogonal" means that training on workload A doesn't improve workload B much, while training on workload A doesn't improve workload B much), so adding it to the training is quite important. --- pgo/pgo.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/pgo/pgo.py b/pgo/pgo.py index bd521c5988a5..681779bade71 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -633,6 +633,21 @@ async def train_clustering(executable: PathLike, workdir: PathLike) -> None: trainers["clustering"] = ("clustering_dataset", train_clustering) populators["clustering_dataset"] = populate_clustering +# DECOMMISSION ================================================== + +async def populate_decommission(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + await cs(cmd=["write"], n=10000000, pop=f"dist=UNIFORM(1..8000000)", schema=RF1_SCHEMA, node=server, cl="one") + +async def train_decommission(executable: PathLike, workdir: PathLike) -> None: + async with with_cluster(executable=executable, workdir=workdir) as (addrs, procs): + await asyncio.sleep(5) # FIXME: artificial gossip sleep, get rid of it. + await bash(fr"curl --silent -X POST http://{addrs[2]}:10000/storage_service/decommission") + await merge_profraw(workdir) + +trainers["decommission"] = ("decommission_dataset", train_decommission) +populators["decommission_dataset"] = populate_decommission + ################################################################################ # Training procedures From e67f4a5c51809d18d6f966c06015f6ed5f5f21bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Feb 2023 03:37:31 +0100 Subject: [PATCH 106/397] pgo: add a LWT workload This workload is added to teach PGO about LWT codepaths. Tests seem to show that it's mostly aligned with existing CQL workloads. The config YAML was copied from one of scylla-cluster-tests test cases. --- pgo/conf/lwt.yaml | 108 ++++++++++++++++++++++++++++++++++++++++++++++ pgo/pgo.py | 14 ++++++ 2 files changed, 122 insertions(+) create mode 100644 pgo/conf/lwt.yaml diff --git a/pgo/conf/lwt.yaml b/pgo/conf/lwt.yaml new file mode 100644 index 000000000000..aa29558489ec --- /dev/null +++ b/pgo/conf/lwt.yaml @@ -0,0 +1,108 @@ +# Keyspace Name +keyspace: ks + +# The CQL for creating a keyspace (optional if it already exists) +keyspace_definition: | + CREATE KEYSPACE ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3}; + +# Table name +table: targettable + +# The CQL for creating a table you wish to stress (optional if it already exists) +table_definition: | + CREATE TABLE targettable ( + pr1 timeuuid, + cl1 text, + aux1 timeuuid, + aux2 int, + aux3 smallint, + aux4 bigint, + aux5 double, + aux6 float, + aux7 tinyint, + aux8 decimal, + aux9 text, + PRIMARY KEY(pr1, cl1) + ); + +### Column Distribution Specifications ### + +columnspec: + - name: pr1 + population: uniform(1..1M) + - name: cl1 + cluster: uniform(1..1k) + population: uniform(1..20M) + - name: aux1 + size: uniform(1..1k) + population: uniform(1..1k) + - name: aux2 + size: uniform(1..1k) + population: uniform(1..1k) + - name: aux3 + size: uniform(1..1k) + population: uniform(1..1k) + - name: aux4 + size: uniform(1..1k) + population: uniform(1..1k) + - name: aux5 + size: uniform(1..1k) + population: uniform(1..1k) + - name: aux6 + size: uniform(1..1k) + population: uniform(1..1k) + - name: aux7 + size: uniform(1..254) + population: uniform(1..254) + - name: aux8 + size: uniform(1..1k) + population: uniform(1..1k) + - name: aux9 + size: uniform(1..1k) + population: uniform(1..1k) + +### Batch Ratio Distribution Specifications ### + +# Insert is used only to propagate data into the dataset. This is only query that supports batches, +# it is actually works only thru batches. +# In order to call it add it to the ops statement: 'ops(insert=1,...)' +# It overrides custom query 'insert' if it is defined +# batch size maximum is hardcoded to 65535, it can be limiting row number in partition for +# big row insertation, by setting 'partitions' key to fixed(X) + +insert: # SchemaInsert.java, does batched prepared statements insert + partitions: fixed(2) + select: fixed(2)/10240M + batchtype: UNLOGGED + +# After running "ops(insert=1) n=1000000" you will get ~1m records with ~1.3k avg row size +# across ~895k partitions, table size will be ~1.3g + +queries: + stmt-select: # SchemaQuery.java: not batched execution + cql: select pr1, cl1, aux1, aux2, aux3, aux4, aux5, aux6, aux7, aux8, aux9 from targettable where pr1 = ? AND cl1 = ? + fields: samerow + stmt-update: # SchemaQuery.java: not batched execution + cql: update targettable set aux1 = ?, aux2 = ?, aux3 = ?, aux4 = ?, aux5 = ?, aux6 = ?, aux7 = ?, aux8 = ?, aux9 = ? where pr1 = ? AND cl1 = ? + fields: samerow + stmt-insert: # SchemaQuery.java: not batched execution + cql: insert into targettable(pr1, cl1, aux1, aux2, aux3, aux4, aux5, aux6, aux7, aux8, aux9) values (?,?,?,?,?,?,?,?,?,?,?) + fields: samerow + stmt-delete: # SchemaQuery.java: not batched execution + cql: delete from targettable where pr1 = ? AND cl1 = ? + fields: samerow + stmt-update-if-cond: # CASQuery.java: not batched execution, do select first then update + cql: update targettable set aux1 = ?, aux2 = ?, aux3 = ?, aux4 = ?, aux5 = ?, aux6 = ?, aux7 = ?, aux8 = ?, aux9 = ? where pr1 = ? AND cl1 = ? if aux1 = ? AND aux2 = ? AND aux3 = ? AND aux4 = ? AND aux5 = ? AND aux6 = ? AND aux7 = ? AND aux8 = ? AND aux9 = ? + fields: samerow + stmt-update-if-exists: # CASQuery.java: not batched execution, do select first then update + cql: update targettable set aux1 = ?, aux2 = ?, aux3 = ?, aux4 = ?, aux5 = ?, aux6 = ?, aux7 = ?, aux8 = ?, aux9 = ? where pr1 = ? AND cl1 = ? if exists + fields: samerow + stmt-insert-if-not-exists: # CASQuery.java: not batched execution, do select first then insert + cql: insert into targettable(pr1, cl1, aux1, aux2, aux3, aux4, aux5, aux6, aux7, aux8, aux9) values (?,?,?,?,?,?,?,?,?,?,?) if not exists + fields: samerow + stmt-delete-if-cond: # CASQuery.java: not batched execution, do select first then delete + cql: delete from targettable where pr1 = ? AND cl1 = ? IF aux1 = ? AND aux2 = ? AND aux3 = ? AND aux4 = ? AND aux5 = ? AND aux6 = ? AND aux7 = ? AND aux8 = ? AND aux9 = ? + fields: samerow + stmt-delete-if-exists: # CASQuery.java: not batched execution, do select first then delete + cql: delete from targettable where pr1 = ? AND cl1 = ? if exists + fields: samerow diff --git a/pgo/pgo.py b/pgo/pgo.py index 681779bade71..a6c435e8b76a 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -648,6 +648,20 @@ async def train_decommission(executable: PathLike, workdir: PathLike) -> None: trainers["decommission"] = ("decommission_dataset", train_decommission) populators["decommission_dataset"] = populate_decommission +# LWT ================================================== + +async def populate_lwt(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + await cs(cmd=["user", "profile=./conf/lwt.yaml", "ops(insert=1)"], n=1000000, cl="local_quorum", node=server) + +async def train_lwt(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_train(executable=executable, workdir=workdir) as server: + ops = "ops(stmt-insert=1,stmt-select=1,stmt-update=1,stmt-delete=1,stmt-insert-if-not-exists=1,stmt-update-if-cond=1,stmt-update-if-exists=1,stmt-delete-if-cond=1,stmt-delete-if-exists=1)" + await cs(cmd=["user", "profile=./conf/lwt.yaml", ops], n=100000, cl="local_quorum", node=server) + +trainers["lwt"] = ("lwt_dataset", train_lwt) +populators["lwt_dataset"] = populate_lwt + ################################################################################ # Training procedures From 47dc0399cb321f834283e61f0c0e99aa1a9d09c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Feb 2023 03:37:40 +0100 Subject: [PATCH 107/397] pgo: add a secondary index workload This workload is added to teach PGO about secondary indexes. Tests seem to show that it's mostly aligned with existing CQL workloads. The config YAML was copied from one of scylla-cluster-test test cases. --- pgo/conf/si.yaml | 74 ++++++++++++++++++++++++++++++++++++++++++++++++ pgo/pgo.py | 13 +++++++++ 2 files changed, 87 insertions(+) create mode 100644 pgo/conf/si.yaml diff --git a/pgo/conf/si.yaml b/pgo/conf/si.yaml new file mode 100644 index 000000000000..529ba66185aa --- /dev/null +++ b/pgo/conf/si.yaml @@ -0,0 +1,74 @@ +keyspace: sec_index + +keyspace_definition: | + + CREATE KEYSPACE IF NOT EXISTS sec_index WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3}; + +table: users + +table_definition: | + + CREATE TABLE IF NOT EXISTS users ( + userid bigint, + initials int, + first_name text, + last_name text, + password text, + email text, + address text, + userdata blob, + last_access timeuuid, + PRIMARY KEY(userid) + ); + +extra_definitions: + - CREATE INDEX IF NOT EXISTS users_last_name_ind ON sec_index.users (last_name) + - CREATE INDEX IF NOT EXISTS users_first_name_ind ON sec_index.users (first_name) + - CREATE INDEX IF NOT EXISTS users_initials_ind ON sec_index.users (initials) + +columnspec: + - name: userid + population: exp(1..10000000) + + - name: initials + size: fixed(2) + population: gaussian(1..20000) + + - name: first_name + size: fixed(5) + + - name: last_name + size: fixed(5) + + - name: password + size: fixed(80) # sha-512 + + - name: email + size: uniform(16..50) + + - name: address + size: uniform(16..50) + + - name: userdata + size: fixed(10240) + +insert: + partitions: fixed(1) + batchtype: UNLOGGED + +queries: + si_read1: + cql: select * from sec_index.users where userid = ? + fields: samerow + si_read2: + cql: select * from sec_index.users where first_name = ? + fields: samerow + si_read3: + cql: select * from sec_index.users where last_name = ? + fields: samerow + si_read4: + cql: select * from sec_index.users where first_name = ? and last_name = ? ALLOW FILTERING + fields: samerow + si_read5: + cql: select userid, initials, first_name, last_name from sec_index.users where initials = ? + fields: samerow diff --git a/pgo/pgo.py b/pgo/pgo.py index a6c435e8b76a..33de819f10d5 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -662,6 +662,19 @@ async def train_lwt(executable: PathLike, workdir: PathLike) -> None: trainers["lwt"] = ("lwt_dataset", train_lwt) populators["lwt_dataset"] = populate_lwt +# SI ================================================== + +async def populate_si(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + await cs(cmd=["user", "profile=./conf/si.yaml", "ops(insert=1)"], n=100000, cl="local_quorum", node=server) + +async def train_si(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_train(executable=executable, workdir=workdir) as server: + await cs(cmd=["user", "profile=./conf/si.yaml", "ops(insert=25,si_read1=1,si_read2=1,si_read3=1,si_read4=1,si_read5=10)"], n=100000, cl="local_quorum", node=server) + +trainers["si"] = ("si_dataset", train_si) +populators["si_dataset"] = populate_si + ################################################################################ # Training procedures From 1c9ce0a9ee609230e0552d6d3ae7fd8c8574eff7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Feb 2023 03:37:49 +0100 Subject: [PATCH 108/397] pgo: add a counters workload This workload is added to teach PGO about counters. Tests seem to show it's mostly aligned with existing CQL workloads. The config YAML is based on the default cassandra-stress schema. --- pgo/conf/counters.yaml | 13 +++++++++++++ pgo/pgo.py | 22 ++++++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 pgo/conf/counters.yaml diff --git a/pgo/conf/counters.yaml b/pgo/conf/counters.yaml new file mode 100644 index 000000000000..04995426f20d --- /dev/null +++ b/pgo/conf/counters.yaml @@ -0,0 +1,13 @@ +DROP KEYSPACE IF EXISTS counters; + +CREATE KEYSPACE IF NOT EXISTS counters +WITH replication = {'class': 'NetworkTopologyStrategy', 'dc1': '3'}; + +CREATE TABLE IF NOT EXISTS counters.counter1 ( + key blob PRIMARY KEY, + "C0" counter, + "C1" counter, + "C2" counter, + "C3" counter, + "C4" counter + ); diff --git a/pgo/pgo.py b/pgo/pgo.py index 33de819f10d5..c8905710378d 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -675,6 +675,28 @@ async def train_si(executable: PathLike, workdir: PathLike) -> None: trainers["si"] = ("si_dataset", train_si) populators["si_dataset"] = populate_si +# COUNTERS ================================================== + +async def populate_counters(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + await bash(fr"../tools/java/bin/cqlsh -f conf/counters.yaml {server}") + # Sleeps added in reaction to schema disagreement errors. + # FIXME: get rid of this sleep and find a sane way to wait for schema + # agreement. + await asyncio.sleep(3) + await cs(cmd=["counter_write"], n=1000000, cl="local_quorum", node=server, schema="keyspace=counters") + +async def train_counters(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_train(executable=executable, workdir=workdir) as server: + await cs(cmd=["counter_write"], n=50000, pop=f"dist=UNIFORM(1..1000000)", cl="local_quorum", node=server, schema="keyspace=counters") + await cs(cmd=["counter_read"], n=50000, pop=f"dist=UNIFORM(1..1000000)", cl="local_quorum", node=server, schema="keyspace=counters") + +# This workload depends on cqlsh, so it's commented out until we merge +# python3 support in cqlsh (which, at the moment of writing, is supposed +# to be imminent). +#trainers["counters"] = ("counters_dataset", train_counters) +populators["counters_dataset"] = populate_counters + ################################################################################ # Training procedures From 95c8d88b963eced5994b6dc89f091563e42eccdf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Feb 2023 03:37:54 +0100 Subject: [PATCH 109/397] pgo: add a repair workload This workload is added to teach PGO about repair. Tests are inconclusive about its alignment with existing workloads, because repair doesn't seem utilize 100% of the reactor. --- pgo/conf/repair.yaml | 92 ++++++++++++++++++++++++++++++++++++++++++++ pgo/pgo.py | 23 +++++++++++ 2 files changed, 115 insertions(+) create mode 100644 pgo/conf/repair.yaml diff --git a/pgo/conf/repair.yaml b/pgo/conf/repair.yaml new file mode 100644 index 000000000000..2933498114f3 --- /dev/null +++ b/pgo/conf/repair.yaml @@ -0,0 +1,92 @@ +# +# This is an example YAML profile for cassandra-stress +# +# insert data +# cassandra-stress user profile=/home/jake/stress1.yaml ops(insert=1) +# +# read, using query simple1: +# cassandra-stress profile=/home/jake/stress1.yaml ops(simple1=1) +# +# mixed workload (90/10) +# cassandra-stress user profile=/home/jake/stress1.yaml ops(insert=1,simple1=9) + + +# +# Keyspace info +# +keyspace: ks + +# +# The CQL for creating a keyspace (optional if it already exists) +# +keyspace_definition: | + CREATE KEYSPACE ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3}; + +# +# Table info +# +table: standard1 + +# +# The CQL for creating a table you wish to stress (optional if it already exists) +# +table_definition: | + CREATE TABLE standard1 ( + name text, + choice boolean, + date timestamp, + address inet, + dbl double, + lval bigint, + ival int, + uid timeuuid, + value blob, + PRIMARY KEY((name,choice), date, address, dbl, lval, ival, uid) + ); + +# +# Optional meta information on the generated columns in the above table +# The min and max only apply to text and blob types +# The distribution field represents the total unique population +# distribution of that column across rows. Supported types are +# +# EXP(min..max) An exponential distribution over the range [min..max] +# EXTREME(min..max,shape) An extreme value (Weibull) distribution over the range [min..max] +# GAUSSIAN(min..max,stdvrng) A gaussian/normal distribution, where mean=(min+max)/2, and stdev is (mean-min)/stdvrng +# GAUSSIAN(min..max,mean,stdev) A gaussian/normal distribution, with explicitly defined mean and stdev +# UNIFORM(min..max) A uniform distribution over the range [min, max] +# FIXED(val) A fixed distribution, always returning the same value +# Aliases: extr, gauss, normal, norm, weibull +# +# If preceded by ~, the distribution is inverted +# +# Defaults for all columns are size: uniform(4..8), population: uniform(1..100B), cluster: fixed(1) +# +columnspec: + - name: name + size: uniform(1..10) + population: uniform(1..4M) # the range of unique values to select for the field (default is 100Billion) + - name: date + cluster: uniform(20..40) + - name: lval + population: gaussian(1..1000) + cluster: uniform(1..4) + +insert: + partitions: uniform(1..50) # number of unique partitions to update in a single operation + # if batchcount > 1, multiple batches will be used but all partitions will + # occur in all batches (unless they finish early); only the row counts will vary + batchtype: UNLOGGED # type of batch to use + select: uniform(1..10)/10 # uniform chance any single generated CQL row will be visited in a partition; + # generated for each partition independently, each time we visit it + +# +# A list of queries you wish to run against the schema +# +queries: + simple1: + cql: select * from standard1 where name = ? and choice = ? LIMIT 100 + fields: samerow # samerow or multirow (select arguments from the same row, or randomly from all rows in the partition) + range1: + cql: select * from standard1 where name = ? and choice = ? and date >= ? LIMIT 100 + fields: multirow # samerow or multirow (select arguments from the same row, or randomly from all rows in the partition) diff --git a/pgo/pgo.py b/pgo/pgo.py index c8905710378d..8921bd104246 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -697,6 +697,29 @@ async def train_counters(executable: PathLike, workdir: PathLike) -> None: #trainers["counters"] = ("counters_dataset", train_counters) populators["counters_dataset"] = populate_counters +# REPAIR ================================================== + +async def populate_repair(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + await cs(cmd=["user", "profile=./conf/repair.yaml", "ops(insert=1)"], n=5000000, cl="local_quorum", node=server) + await cs(cmd=["write"], n=1000000, cl="local_quorum", schema=RF3_SCHEMA, node=server) + +async def train_repair(executable: PathLike, workdir: PathLike) -> None: + # The idea is to remove some user data sstables from the node (in an offline cluster), + # start the cluster, and run repair on the affected node. + # I don't know if it's a good PGO workload. + # Does this cover repair codepaths reasonably? + addr = cluster_metadata(workdir)["subnet"] + ".2" + await bash(fr"rm -rf {workdir}/{addr}/data/ks/*") + async with with_cluster(executable=executable, workdir=workdir) as (addrs, procs): + await asyncio.sleep(5) # FIXME: artificial gossip sleep, get rid of it. + repair_id = (await query(["curl", "--silent", "-X", "POST", fr"http://{addr}:10000/storage_service/repair_async/ks"])).decode() + await query(["curl", "--silent", fr"http://{addr}:10000/storage_service/repair_status/?id={repair_id}"]) + await merge_profraw(workdir) + +trainers["repair"] = ("repair_dataset", train_repair) +populators["repair_dataset"] = populate_repair + ################################################################################ # Training procedures From 80989556ac91c0aa82ee9745ce1e8ae0ca467c78 Mon Sep 17 00:00:00 2001 From: Marcin Maliszkiewicz Date: Tue, 7 May 2024 11:51:26 +0200 Subject: [PATCH 110/397] pgo: add alternator workloads training MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch adds a set of alternator workloads to pgo training script. To confirm that added workloads are indeed affecting profile we can compare: ⤖ llvm-profdata show ./build/release-pgo/profiles/workdirs/clustering/prof.profdata Instrumentation level: IR entry_first = 0 Total functions: 105075 Maximum function count: 1079870885 Maximum internal block count: 2197851358 and ⤖ llvm-profdata show ./build/release-pgo/profiles/workdirs/alternator/prof.profdata Instrumentation level: IR entry_first = 0 Total functions: 105075 Maximum function count: 5240506052 Maximum internal block count: 9112894084 to see that function counters are on similar levels, they are around 5x higher for alternator but that's because it combines 5 specific sub-workloads. To confirm that final profile contains alterantor functions we can inspect: ⤖ llvm-profdata show --counts --function=alternator --value-cutoff 100000 ./build/release-pgo/profiles/merged.profdata (...) Instrumentation level: IR entry_first = 0 Functions shown: 356 Total functions: 105075 Number of functions with maximum count (< 100000): 97275 Number of functions with maximum count (>= 100000): 7800 Maximum function count: 7248370728 Maximum internal block count: 13722347326 we can see that 356 functions which symbol name contains word alternator were identified as 'hot' (with max count grater than 100'000). Running: ⤖ llvm-profdata show --counts --function=alternator --value-cutoff 1 ./build/release-pgo/profiles/merged.profdata (...) Instrumentation level: IR entry_first = 0 Functions shown: 806 Total functions: 105075 Number of functions with maximum count (< 1): 67036 Number of functions with maximum count (>= 1): 38039 Maximum function count: 7248370728 Maximum internal block count: 13722347326 we can see that 806 alternator functions were executed at least once during training. And finally to confirm that alternator specific PGO brings any speedups we run: for workload in read scan write write_gsi write_rmw do ./build/release/scylla perf-alternator-workloads --smp 4 --cpuset "10,12,14,16" --workload $workload --duration 1 --remote-host 127.0.0.1 2> /dev/null | grep median done results BEFORE: median 258137.51910849303 median absolute deviation: 786.06 median 547.2578202937141 median absolute deviation: 6.33 median 145718.19856685458 median absolute deviation: 5689.79 median 89024.67095807113 median absolute deviation: 1302.56 median 43708.101729598646 median absolute deviation: 294.47 results AFTER: median 303968.55333940056 median absolute deviation: 1152.19 median 622.4757636209254 median absolute deviation: 8.42 median 198566.0403745328 median absolute deviation: 1689.96 median 91696.44912842038 median absolute deviation: 1891.84 median 51445.356525664996 median absolute deviation: 1780.15 We can see that single node cluster tps increase is typically 13% - 17% with notable exceptions, improvement for write_gsi is 3% and for write workload whopping 36%. The increase is on top of CQL PGO. Write workload is executed more often because it's involved also as data preparation for read and scan. Some further improvement could be to separate preparation from training as it's done for CQL but it would be a bit odd if ~3x higher counters for one flow have so big impact. Additional disclaimers: - tests are performing exactly the same workloads as in training so there might be some bias - tests are running single node cluster, more realistic setup will likely show lower improvement Fixes https://github.com/scylladb/scylla-enterprise/issues/4066 --- pgo/pgo.py | 43 ++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 42 insertions(+), 1 deletion(-) diff --git a/pgo/pgo.py b/pgo/pgo.py index 8921bd104246..51425bfc0aa1 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -263,7 +263,7 @@ async def run(command: list[str], cpuset: Optional[str] = None, **kwargs) -> Pro kwargs.setdefault("stdout", asyncio.subprocess.DEVNULL) kwargs.setdefault("stderr", asyncio.subprocess.DEVNULL) - process_logger.debug(f"Running a process: {orig_cmd}") + process_logger.info(f"Running a process: {orig_cmd}") p = await asyncio.create_subprocess_exec(*cmd, **kwargs) logfile = f"{logdir}/{p.pid}.log" if logdir else None @@ -408,11 +408,14 @@ async def start_node(executable: PathLike, cluster_workdir: PathLike, addr: str, f"--api-address={addr}", f"--rpc-address={addr}", f"--prometheus-address={addr}", + f"--alternator-address={addr}", f"--cluster-name={cluster_name}", f"--endpoint-snitch=GossipingPropertyFileSnitch", f"--read-request-timeout-in-ms=60000", f"--write-request-timeout-in-ms=60000", f"--cas-contention-timeout-in-ms=60000", + f"--alternator-port=8000", + f"--alternator-write-isolation=only_rmw_uses_lwt", ] + list(extra_opts) return await run(['bash', '-c', fr"""exec {shlex.join(command)} >{q(logfile)} 2>&1"""], cwd=cluster_workdir) @@ -619,6 +622,44 @@ async def train_basic(executable: PathLike, workdir: PathLike) -> None: #trainers["basic"] = ("basic_dataset", train_basic) populators["basic_dataset"] = populate_basic +# ALTERNATOR ================================================== + +async def train_alternator(executable: PathLike, workdir: PathLike) -> None: + os.makedirs(workdir, exist_ok=True) + async with with_cluster(executable=executable, workdir=workdir) as (addrs, procs): + await asyncio.sleep(5) # FIXME: artificial gossip sleep, get rid of it. + + workloads = [ + ["write", 250_000], + ["read", 250_000], + ["scan", 1_000], + ["write_gsi", 250_000], + ["write_rmw", 250_000], + ] + for workload in workloads: + # the tool doesn't yet support load balancing so we + # run one process per node + await clean_gather(*[run_checked([executable, + "perf-alternator", + "--smp", "1", + "--workload", f"{workload[0]}", + "--partitions", "100000", + # we reuse cluster data so don't need to pre-populate + "--prepopulate-partitions", "0", + "--operations-per-shard", f"{workload[1]}", + "--cpuset", f'{CS_CPUSET.get()}', + "--remote-host", addr, + ]) for addr in addrs]) + + await merge_profraw(workdir) + +async def populate_alternator(executable: PathLike, workdir: PathLike) -> None: + async with with_cs_populate(executable=executable, workdir=workdir) as server: + pass # this avoids profiling cluster bootstrap code + +trainers["alternator"] = ("alternator_dataset", train_alternator) +populators["alternator_dataset"] = populate_alternator + # CLUSTERING ================================================== async def populate_clustering(executable: PathLike, workdir: PathLike) -> None: From a868b44ad8cea09a7dead89cc4388fea257c0587 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Tue, 14 Jun 2022 16:37:50 +0200 Subject: [PATCH 111/397] configure.py: introduce profile-guided optimization This commit enables profile-guided optimizations (PGO) in the Scylla build. A full LLVM PGO requires 3 builds: 1. With -fprofile-generate to generate context-free (pre-inlining) profile. This profile influences inlining, indirect-call promotion and call graph simplifications. 2. With -fprofile-use=results_of_build_1 -fcs-profile-generate to generate context-sensitive (post-inlining) profile. This profile influences post-inline and codegen optimizations. 3. With -fprofile-use=merged_results_of_builds_1_2 to build the final binary with both profiles. We do all three in one ninja call by adding release-pgo and release-cs-pgo "stages" to release. They are a copy of regular release mode, just with the flags described above added. With the full course, release objects depend on the profile file produced by build/release-cs-pgo/scylla, while release-cs-pgo depends on the profile file generated by build/release-pgo/scylla. The stages are orthogonal and enabled with separate options. It's recommended to run them both for full performance, but unfortunately each one adds a full build of scylla to the compile time, so maybe we can drop one of them in the future if it turns out e.g. that regular PGO doesn't have a big effect. It's strongly recommended to combine PGO with LTO. The latter enables the entire class of binary layout optimizations, which for us is probably the most important part of the entire thing. --- configure.py | 136 ++++++++++++++++++++++++++++++++++++++++++++------- pgo/train | 2 + 2 files changed, 121 insertions(+), 17 deletions(-) create mode 100755 pgo/train diff --git a/configure.py b/configure.py index d908794408dc..5c3eef9e326b 100755 --- a/configure.py +++ b/configure.py @@ -9,6 +9,7 @@ # import argparse +import copy import os import platform import re @@ -704,6 +705,14 @@ def find_ninja(): help='Set datestamp for SCYLLA-VERSION-GEN') add_tristate(arg_parser, name='lto', dest='lto', default=True, help='link-time optimization.') +arg_parser.add_argument('--use-profile', dest='use_profile', action='store', + help='Path to the (optional) profile file to be used in the build. Meant to be used with the profile file (build/release/profiles/merged.profdata) generated during a previous build of build/release/scylla with --pgo (--cspgo).') +arg_parser.add_argument('--pgo', dest='pgo', action='store_true', default=False, + help='Generate and use fresh PGO profiles when building Scylla. Only supported with clang for now.') +arg_parser.add_argument('--cspgo', dest='cspgo', action='store_true', default=False, + help='Generate and use fresh CSPGO profiles when building Scylla. A clang-specific optional addition to --pgo.') +arg_parser.add_argument('--experimental-pgo', dest='experimental_pgo', action='store_true', default=False, + help='When building with PGO, enable nonconservative (potentially pessimizing) optimizations. Only supported with clang for now. Not recommended.') arg_parser.add_argument('--use-cmake', action=argparse.BooleanOptionalAction, default=False, help='Whether to use CMake as the build system') arg_parser.add_argument('--coverage', action = 'store_true', help = 'Compile scylla with coverage instrumentation') arg_parser.add_argument('--build-dir', action='store', default='build', @@ -1691,18 +1700,90 @@ def dynamic_linker_option(): modes[mode]['lib_cflags'] = user_cflags modes[mode]['lib_ldflags'] = user_ldflags + linker_flags + def prepare_advanced_optimizations(*, modes, build_modes, args): for mode in modes: modes[mode]['has_lto'] = False + modes[mode]['is_profile'] = False + + profile_modes = {} for mode in modes: if not modes[mode]['advanced_optimizations']: continue + # When building with PGO, -Wbackend-plugin generates a warning for every + # function which changed its control flow graph since the profile was + # taken. + # We allow stale profiles, so these warnings are just noise to us. + # Let's silence them. + modes[mode]['lib_cflags'] += ' -Wno-backend-plugin' + if args.lto: modes[mode]['has_lto'] = True modes[mode]['lib_cflags'] += ' -flto=thin -ffat-lto-objects' + profile_path = None # Absolute path to the profile, for use in compiler flags. Can't use $builddir because it's also passed to seastar. + profile_target = None # Path with $builddir in front, for consistency with all other ninja targets. + + if args.use_profile: + profile_path = os.path.abspath(args.use_profile) + profile_target = args.use_profile + + # pgso (profile-guided size-optimization) adds optsize hints (-Os) to cold code. + # We don't want to optimize anything for size, because that's a potential source + # of performance regressions, and the benefits are dubious. Let's disable pgso + # by default. (Currently is enabled in Clang by default.) + # + # Value profiling allows the compiler to track not only the outcomes of branches + # but also the values of variables at interesting decision points. + # Currently Clang uses value profiling for two things: specializing for the most + # common sizes of memory ops (e.g. memcpy, memcmp) and specializing for the most + # common targets of indirect branches. + # It's valuable in general, but our training suite is not realistic and exhaustive + # enough to be confident about value profiling. Let's also keep it disabled by + # default, conservatively. (Currently it is enabled in Clang by default.) + conservative_opts = "" if args.experimental_pgo else "-mllvm -pgso=false -mllvm -enable-value-profiling=false" + + llvm_instr_types = [] + if args.pgo: + llvm_instr_types += [""] + if args.cspgo: + llvm_instr_types += ["cs-"] + for it in llvm_instr_types: + submode = copy.deepcopy(modes[mode]) + submode_name = f'{mode}-{it}pgo' + submode['parent_mode'] = mode + if profile_path is not None: + submode['lib_cflags'] += f" -fprofile-use={profile_path}" + submode['cxx_ld_flags'] += f" -fprofile-use={profile_path}" + submode['profile_target'] = profile_target + submode['lib_cflags'] += f" -f{it}profile-generate={os.path.realpath(outdir)}/{submode_name} {conservative_opts}" + submode['cxx_ld_flags'] += f" -f{it}profile-generate={os.path.realpath(outdir)}/{submode_name} {conservative_opts}" + # Profile collection depends on java tools because we use cassandra-stress as the load. + submode['profile_recipe'] = textwrap.dedent(f"""\ + build $builddir/{submode_name}/profiles/prof.profdata: train $builddir/{submode_name}/scylla | dist-tools-tar + build $builddir/{submode_name}/profiles/merged.profdata: merge_profdata $builddir/{submode_name}/profiles/prof.profdata {profile_target or str()} + """) + submode['is_profile'] = True + profile_path = f"{os.path.realpath(outdir)}/{submode_name}/profiles/merged.profdata" + profile_target = f"$builddir/{submode_name}/profiles/merged.profdata" + + profile_modes[submode_name] = submode + + if profile_path is not None: + modes[mode]['lib_cflags'] += f" -fprofile-use={profile_path} {conservative_opts}" + modes[mode]['cxx_ld_flags'] += f" -fprofile-use={profile_path} {conservative_opts}" + modes[mode]['profile_target'] = profile_target + modes[mode].setdefault('profile_recipe', "") + modes[mode]['profile_recipe'] += textwrap.dedent(f"""\ + build $builddir/{mode}/profiles/merged.profdata: copy {profile_target or profile_path or str()} + """) + + modes.update(profile_modes) + build_modes.update(profile_modes) + + # cmake likes to separate things with semicolons def semicolon_separated(*flags): # original flags may be space separated, so convert to string still @@ -2004,6 +2085,13 @@ def write_build_file(f, rule wasm2wat command = wasm2wat $in > $out description = WASM2WAT $out + rule run_profile + command = rm -r `dirname $out` && pgo/run_all $in `dirname $out` $type + rule train + command = rm -r `dirname $out` && pgo/train `realpath $in` `realpath -m $out` `realpath -m $builddir/pgo_datasets` + pool = console + rule merge_profdata + command = llvm-profdata merge $in -output=$out ''').format(configure_args=configure_args, outdir=outdir, cxx=args.cxx, @@ -2084,6 +2172,7 @@ def write_build_file(f, command = ./test.py --mode={mode} --repeat={test_repeat} --timeout={test_timeout} pool = console description = TEST {mode} + # This rule is unused for PGO stages. They use the rust lib from the parent mode. rule rust_lib.{mode} command = CARGO_BUILD_DEP_INFO_BASEDIR='.' cargo build --locked --manifest-path=rust/Cargo.toml --target-dir=$builddir/{mode} --profile=rust-{mode} $ && touch $out @@ -2096,6 +2185,8 @@ def write_build_file(f, wasms = str.join(' ', ['$builddir/' + x for x in sorted(build_artifacts & wasms)]), ) ) + if profile_recipe := modes[mode].get('profile_recipe'): + f.write(profile_recipe) include_cxx_target = f'{mode}-build' if not args.dist_only else '' include_dist_target = f'dist-{mode}' if args.enable_dist is None or args.enable_dist else '' f.write(f'build {mode}: phony {include_cxx_target} {include_dist_target}\n') @@ -2116,6 +2207,11 @@ def write_build_file(f, seastar_lib_ext = 'so' if modeval['build_seastar_shared_libs'] else 'a' for binary in sorted(build_artifacts): + if modeval['is_profile'] and binary != "scylla": + # Just to avoid clutter in build.ninja + continue + profile_dep = modes[mode].get('profile_target', "") + if binary in other or binary in wasms: continue srcs = deps[binary] @@ -2135,7 +2231,8 @@ def write_build_file(f, obj = dep[:idx].replace('rust/','') + '.o' objs.append(f'$builddir/{mode}/gen/rust/{obj}') if has_rust: - objs.append(f'$builddir/{mode}/rust-{mode}/librust_combined.a') + parent_mode = modes[mode].get('parent_mode', mode) + objs.append(f'$builddir/{parent_mode}/rust-{parent_mode}/librust_combined.a') do_lto = modes[mode]['has_lto'] and binary in lto_binaries seastar_dep = f'$builddir/{mode}/seastar/libseastar.{seastar_lib_ext}' @@ -2256,7 +2353,7 @@ def write_build_file(f, src = compiles[obj] seastar_dep = f'$builddir/{mode}/seastar/libseastar.{seastar_lib_ext}' abseil_dep = ' '.join(f'$builddir/{mode}/abseil/{lib}' for lib in abseil_libs) - f.write(f'build {obj}: cxx.{mode} {src} || {seastar_dep} {abseil_dep} {gen_headers_dep}\n') + f.write(f'build {obj}: cxx.{mode} {src} | {profile_dep} || {seastar_dep} {abseil_dep} {gen_headers_dep}\n') if src in modeval['per_src_extra_cxxflags']: f.write(' cxxflags = {seastar_cflags} $cxxflags $cxxflags_{mode} {extra_cxxflags}\n'.format(mode=mode, extra_cxxflags=modeval["per_src_extra_cxxflags"][src], **modeval)) for swagger in swaggers: @@ -2265,7 +2362,7 @@ def write_build_file(f, obj = swagger.objects(gen_dir)[0] src = swagger.source f.write('build {} | {} : swagger {} | {}/scripts/seastar-json2code.py\n'.format(hh, cc, src, args.seastar_path)) - f.write(f'build {obj}: cxx.{mode} {cc}\n') + f.write(f'build {obj}: cxx.{mode} {cc} | {profile_dep}\n') for hh in serializers: src = serializers[hh] f.write('build {}: serializer {} | idl-compiler.py\n'.format(hh, src)) @@ -2273,15 +2370,16 @@ def write_build_file(f, src = ragels[hh] f.write('build {}: ragel {}\n'.format(hh, src)) f.write('build {}: cxxbridge_header\n'.format('$builddir/{}/gen/rust/cxx.h'.format(mode))) - librust = '$builddir/{}/rust-{}/librust_combined'.format(mode, mode) - f.write('build {}.a: rust_lib.{} rust/Cargo.lock\n depfile={}.d\n'.format(librust, mode, librust)) + if 'parent_mode' not in modes[mode]: + librust = '$builddir/{}/rust-{}/librust_combined'.format(mode, mode) + f.write('build {}.a: rust_lib.{} rust/Cargo.lock\n depfile={}.d\n'.format(librust, mode, librust)) for grammar in antlr3_grammars: outs = ' '.join(grammar.generated('$builddir/{}/gen'.format(mode))) f.write('build {}: antlr3.{} {}\n stem = {}\n'.format(outs, mode, grammar.source, grammar.source.rsplit('.', 1)[0])) for cc in grammar.sources('$builddir/{}/gen'.format(mode)): obj = cc.replace('.cpp', '.o') - f.write(f'build {obj}: cxx.{mode} {cc} || {" ".join(serializers)}\n') + f.write(f'build {obj}: cxx.{mode} {cc} | {profile_dep} || {" ".join(serializers)}\n') flags = '-Wno-parentheses-equality' if cc.endswith('Parser.cpp'): # Unoptimized parsers end up using huge amounts of stack space and overflowing their stack @@ -2292,26 +2390,36 @@ def write_build_file(f, f.write(' obj_cxxflags = %s\n' % flags) f.write(f'build $builddir/{mode}/gen/empty.cc: gen\n') for hh in headers: - f.write('build $builddir/{mode}/{hh}.o: checkhh.{mode} {hh} | $builddir/{mode}/gen/empty.cc || {gen_headers_dep}\n'.format( - mode=mode, hh=hh, gen_headers_dep=gen_headers_dep)) + f.write('build $builddir/{mode}/{hh}.o: checkhh.{mode} {hh} | $builddir/{mode}/gen/empty.cc {profile_dep} || {gen_headers_dep}\n'.format( + mode=mode, hh=hh, gen_headers_dep=gen_headers_dep, profile_dep=profile_dep)) seastar_dep = f'$builddir/{mode}/seastar/libseastar.{seastar_lib_ext}' seastar_testing_dep = f'$builddir/{mode}/seastar/libseastar_testing.{seastar_lib_ext}' - f.write('build {seastar_dep}: ninja $builddir/{mode}/seastar/build.ninja | always\n' + f.write('build {seastar_dep}: ninja $builddir/{mode}/seastar/build.ninja | always {profile_dep}\n' .format(**locals())) f.write(' pool = submodule_pool\n') f.write(' subdir = $builddir/{mode}/seastar\n'.format(**locals())) f.write(' target = seastar\n'.format(**locals())) - f.write('build {seastar_testing_dep}: ninja $builddir/{mode}/seastar/build.ninja | always\n' + f.write('build {seastar_testing_dep}: ninja $builddir/{mode}/seastar/build.ninja | always {profile_dep}\n' .format(**locals())) f.write(' pool = submodule_pool\n') f.write(' subdir = $builddir/{mode}/seastar\n'.format(**locals())) f.write(' target = seastar_testing\n'.format(**locals())) - f.write('build $builddir/{mode}/seastar/apps/iotune/iotune: ninja $builddir/{mode}/seastar/build.ninja\n' + f.write(' profile_dep = {profile_dep}\n'.format(**locals())) + + for lib in abseil_libs: + f.write('build $builddir/{mode}/abseil/{lib}: ninja $builddir/{mode}/abseil/build.ninja | always {profile_dep}\n'.format(**locals())) + f.write(' pool = submodule_pool\n') + f.write(' subdir = $builddir/{mode}/abseil\n'.format(**locals())) + f.write(' target = {lib}\n'.format(**locals())) + f.write(' profile_dep = {profile_dep}\n'.format(**locals())) + + f.write('build $builddir/{mode}/seastar/apps/iotune/iotune: ninja $builddir/{mode}/seastar/build.ninja | $builddir/{mode}/seastar/libseastar.{seastar_lib_ext}\n' .format(**locals())) f.write(' pool = submodule_pool\n') f.write(' subdir = $builddir/{mode}/seastar\n'.format(**locals())) f.write(' target = iotune\n'.format(**locals())) + f.write(' profile_dep = {profile_dep}\n'.format(**locals())) f.write(textwrap.dedent('''\ build $builddir/{mode}/iotune: copy $builddir/{mode}/seastar/apps/iotune/iotune build $builddir/{mode}/iotune.stripped: strip $builddir/{mode}/iotune @@ -2351,12 +2459,6 @@ def write_build_file(f, f.write(f'build $builddir/{mode}/dist/tar/{scylla_product}-unified-package-{scylla_version}-{scylla_release}.tar.gz: copy $builddir/{mode}/dist/tar/{scylla_product}-unified-{scylla_version}-{scylla_release}.{arch}.tar.gz\n') f.write(f'build $builddir/{mode}/dist/tar/{scylla_product}-unified-{arch}-package-{scylla_version}-{scylla_release}.tar.gz: copy $builddir/{mode}/dist/tar/{scylla_product}-unified-{scylla_version}-{scylla_release}.{arch}.tar.gz\n') - for lib in abseil_libs: - f.write('build $builddir/{mode}/abseil/{lib}: ninja $builddir/{mode}/abseil/build.ninja\n'.format(**locals())) - f.write(' pool = submodule_pool\n') - f.write(' subdir = $builddir/{mode}/abseil\n'.format(**locals())) - f.write(' target = {lib}\n'.format(**locals())) - checkheaders_mode = 'dev' if 'dev' in modes else modes.keys()[0] f.write('build checkheaders: phony || {}\n'.format(' '.join(['$builddir/{}/{}.o'.format(checkheaders_mode, hh) for hh in headers]))) diff --git a/pgo/train b/pgo/train new file mode 100755 index 000000000000..7ca892d861da --- /dev/null +++ b/pgo/train @@ -0,0 +1,2 @@ +#!/bin/bash -xue +exec python3 -u "$(dirname "$(realpath "$0")")"/pgo.py train_full "$@" From 131b1d6f81f36786ce9e11c1e573ca559dc90e66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Fri, 7 Apr 2023 23:56:42 +0200 Subject: [PATCH 112/397] configure.py: prepare the build for a default PGO profile in version control This patch adds the following logic to the release build: pgo/profiles/profile.profdata.xz is the default profile file, compressed. This file is stored in version control using git LFS. A ninja rule is added which creates build/profile.profdata by decompressing it. If no profile file is explicitly specified, ./configure.py checks whether the compressed default profile file exists and is compressed. (If it exists, but isn't compressed, the user most likely has git lfs disabled or not installed. In this case, the file visible in the working tree will be the LFS placeholder text file describing the LFS metadata.) If the compressed file exists, build/profile.profdata is chosen as the used profile file. If it doesn't exist, a warning is printed and configure.py falls back to a profileless build. The default profile file can be explicitly disabled by passing the empty --use-profile="" to configure.py A script is added which re-generates the profile. After the script is run, the re-generated compressed profile can be staged, committed, pushed and merged to update the default profile. --- .gitattributes | 1 + configure.py | 53 +++++++++++++++++++++++++++++++-- scripts/refresh-pgo-profiles.sh | 31 +++++++++++++++++++ 3 files changed, 82 insertions(+), 3 deletions(-) create mode 100755 scripts/refresh-pgo-profiles.sh diff --git a/.gitattributes b/.gitattributes index 3bef27601f0e..969d75ef537d 100644 --- a/.gitattributes +++ b/.gitattributes @@ -2,3 +2,4 @@ *.hh diff=cpp *.svg binary docs/_static/api/js/* binary +pgo/profiles/** filter=lfs diff=lfs merge=lfs -text diff --git a/configure.py b/configure.py index 5c3eef9e326b..7a17cd98bc69 100755 --- a/configure.py +++ b/configure.py @@ -11,6 +11,7 @@ import argparse import copy import os +import pathlib import platform import re import shlex @@ -1723,12 +1724,58 @@ def prepare_advanced_optimizations(*, modes, build_modes, args): modes[mode]['has_lto'] = True modes[mode]['lib_cflags'] += ' -flto=thin -ffat-lto-objects' - profile_path = None # Absolute path to the profile, for use in compiler flags. Can't use $builddir because it's also passed to seastar. - profile_target = None # Path with $builddir in front, for consistency with all other ninja targets. + # Absolute path (in case of the initial profile) or path + # beginning with $builddir (in case of generated profiles), + # for use in ninja dependency rules. + # Using absoulte paths only would work too, but we use + # $builddir for consistency with all other ninja targets. + profile_target = None + # Absolute path to the profile, for use in compiler flags. + # Can't use $builddir here because the flags are also passed + # to seastar, which doesn't understand ninja variables. + profile_path = None if args.use_profile: profile_path = os.path.abspath(args.use_profile) - profile_target = args.use_profile + profile_target = profile_path + elif args.use_profile is None: + # Use the default profile. There is a rule in later part of configure.py + # which extracts the default profile from an archive in pgo/profiles, + # (stored in git LFS) to build/ + + default_profile_archive_path = f"pgo/profiles/{platform.machine()}/profile.profdata.xz" + default_profile_filename = pathlib.Path(default_profile_archive_path).stem + + # We are checking whether the profile archive is compressed, + # instead of just checking for its existence, because of how git LFS works. + # + # When a file is stored in LFS, the underlying git repository only receives a text file stub + # containing some metadata of the actual file. On checkout, LFS filters download the actual + # file based on that metadata and substitute it for the stub. + # If LFS is disabled or not installed, git will simply check out the stub, + # which will be a regular text file. + # + # By ignoring existing but uncompressed profile files we are accommodating users who don't + # have LFS installed yet, or don't want to be forced to use it. + # + validate_archive = subprocess.run(["file", default_profile_archive_path], capture_output=True) + if "compressed data" in validate_archive.stdout.decode(): + default_profile_filename = pathlib.Path(default_profile_archive_path).stem + profile_path = os.path.abspath("build/" + default_profile_filename) + profile_target = "$builddir/" + default_profile_filename + modes[mode].setdefault('profile_recipe', '') + modes[mode]['profile_recipe'] += textwrap.dedent(f"""\ + rule xz_uncompress + command = xz --uncompress --stdout $in > $out + description = XZ_UNCOMPRESS $in to $out + build {profile_target}: xz_uncompress {default_profile_archive_path} + """) + else: + # Avoid breaking existing pipelines without git-lfs installed. + print(f"WARNING: {default_profile_archive_path} is not an archive. Building without a profile.", file=sys.stderr) + else: + # Passing --use-profile="" explicitly disables the default profile. + pass # pgso (profile-guided size-optimization) adds optsize hints (-Os) to cold code. # We don't want to optimize anything for size, because that's a potential source diff --git a/scripts/refresh-pgo-profiles.sh b/scripts/refresh-pgo-profiles.sh new file mode 100755 index 000000000000..58d94d8c16d7 --- /dev/null +++ b/scripts/refresh-pgo-profiles.sh @@ -0,0 +1,31 @@ +#!/bin/bash +# +# Copyright (C) 2023-present ScyllaDB +# +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# + +set -eu + +SCRIPT_PATH="$(realpath "$0")" +PROJECT_BASE="$(realpath "$(dirname "$0")"/..)" +WORKING_DIR="$(realpath "$PWD")" +if [ "$PROJECT_BASE" != "$WORKING_DIR" ]; then + echo "Error: $SCRIPT_PATH should be ran with $PROJECT_BASE instead of $WORKING_DIR as the working directory" >&2 + exit 1 +fi + +BUILD_PATH=build/release-cs-pgo/profiles/merged.profdata +TARGET_PATH=pgo/profiles/$(uname -m)/profile.profdata.xz +./configure.py --mode=release --pgo --cspgo --use-profile= + +# ninja "$BUILD_PATH" would avoid a build step, but let's do it voluntarily +# to check that the profile doesn't cause any compilation problems. +ninja build/release/scylla + +# Profiles are stored in version control, so we want very strong compression. +mkdir -p "$(dirname "$TARGET_PATH")" +xz --compress -9 --stdout "$BUILD_PATH" >"$TARGET_PATH" + +echo "Profile $TARGET_PATH regenerated. You can now stage, commit, and push it." From 610f1b7a0ae0a38c9c12a627e7527c64425882b9 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 26 Nov 2024 09:12:25 +0800 Subject: [PATCH 113/397] build: Unify Abseil CXX flags configuration - Set ABSL_GCC_FLAGS and ABSL_LLVM_FLAGS with a more generic absl_cxx_flags - Enables more flexible configuration of compiler flags for Abseil libraries - Provides a centralized approach to setting compilation flags Previously, sanitizer-specific flags were directly applied to Abseil library builds. This change allows for more extensible compiling flag management across different build configurations. Signed-off-by: Kefu Chai --- CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 543762b2af78..f1eda0b8d5c1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -103,12 +103,12 @@ endif() set(ABSL_PROPAGATE_CXX_STD ON CACHE BOOL "" FORCE) find_package(Sanitizers QUIET) -set(sanitizer_cxx_flags +list(APPEND absl_cxx_flags $<$:$;$>) if(CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set(ABSL_GCC_FLAGS ${sanitizer_cxx_flags}) + list(APPEND ABSL_GCC_FLAGS ${absl_cxx_flags}) elseif(CMAKE_CXX_COMPILER_ID STREQUAL "Clang") - set(ABSL_LLVM_FLAGS ${sanitizer_cxx_flags}) + list(APPEND ABSL_LLVM_FLAGS ${absl_cxx_flags}) endif() set(ABSL_DEFAULT_LINKOPTS $<$:$;$>) From ffe8c5dcdbcd67811d0e7c0497444970b464d50d Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 19 Nov 2024 18:17:05 +0800 Subject: [PATCH 114/397] build: collect scylla libraries with `scylla_libs` variable with which, we can set the properties of these targets in a single place. Signed-off-by: Kefu Chai --- CMakeLists.txt | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f1eda0b8d5c1..22d957fb2bd3 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -285,7 +285,7 @@ add_version_library(scylla_version add_executable(scylla main.cc) -target_link_libraries(scylla PRIVATE +set(scylla_libs scylla-main api auth @@ -320,6 +320,8 @@ target_link_libraries(scylla PRIVATE transport types utils) +target_link_libraries(scylla PRIVATE + ${scylla_libs}) target_link_libraries(scylla PRIVATE Seastar::seastar From 2647369d46bd9d54cade1d4335528455f371f8fc Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 28 Oct 2024 10:02:42 +0800 Subject: [PATCH 115/397] build: set LTO and PGO flags for Seastar in cmake build This change extends scylla commit 7cb74df to scylla-enterprise-commit 4ece7e1. we recently started building Seastar as an external project, so we need to prepare its compilation flags separately. in enterprise scylla, we prepare the LTO and PGO related cflags in `prepare_advanced_optimizations()`. this function is called when preparing the build rules directly from `configure.py`, and despite we have equivalant settings in CMake, they cannot be applied to Seastar due to the reason above. in this change, we set up the the LTO and PGO compilation flags when generating the buiding system for Seastar when building using CMake. Signed-off-by: Kefu Chai --- configure.py | 1 + 1 file changed, 1 insertion(+) diff --git a/configure.py b/configure.py index 7a17cd98bc69..e694f8d3dcde 100755 --- a/configure.py +++ b/configure.py @@ -2787,6 +2787,7 @@ def configure_using_cmake(args): if __name__ == '__main__': if args.use_cmake: + prepare_advanced_optimizations(modes=modes, build_modes=build_modes, args=args) configure_using_cmake(args) else: create_build_system(args) From 41547896702a0c0e1e458d9a859e371f9a40444a Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Sun, 14 May 2023 00:04:24 +0800 Subject: [PATCH 116/397] build: cmake: add "Scylla_ENABLE_LTO" option add an option named "Scylla_ENABLE_LTO", which is off by default. if it is on, build the whole tree with ThinLTO enabled. Signed-off-by: Kefu Chai --- CMakeLists.txt | 14 ++++++++++++++ cmake/enable_lto.cmake | 30 ++++++++++++++++++++++++++++++ cmake/mode.common.cmake | 12 ++++++++++++ configure.py | 1 + test/CMakeLists.txt | 6 ++++++ 5 files changed, 63 insertions(+) create mode 100644 cmake/enable_lto.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 22d957fb2bd3..16713425a422 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -22,6 +22,8 @@ if(DEFINED CMAKE_BUILD_TYPE) endif() endif(DEFINED CMAKE_BUILD_TYPE) +option(Scylla_ENABLE_LTO "Turn on link-time optimization for the 'release' mode." ON) + include(mode.common) get_property(is_multi_config GLOBAL PROPERTY GENERATOR_IS_MULTI_CONFIG) if(is_multi_config) @@ -42,6 +44,7 @@ else() endif() include(limit_jobs) + # Configure Seastar compile options to align with Scylla set(CMAKE_CXX_STANDARD "23" CACHE INTERNAL "") set(CMAKE_CXX_EXTENSIONS ON CACHE INTERNAL "") @@ -102,6 +105,10 @@ endif() set(ABSL_PROPAGATE_CXX_STD ON CACHE BOOL "" FORCE) +if(Scylla_ENABLE_LTO) + list(APPEND absl_cxx_flags $<$:${CMAKE_CXX_COMPILE_OPTIONS_IPO};-ffat-lto-objects>) +endif() + find_package(Sanitizers QUIET) list(APPEND absl_cxx_flags $<$:$;$>) @@ -323,6 +330,13 @@ set(scylla_libs target_link_libraries(scylla PRIVATE ${scylla_libs}) +if(Scylla_ENABLE_LTO) + include(enable_lto) + foreach(target scylla ${scylla_libs}) + enable_lto(${target}) + endforeach() +endif() + target_link_libraries(scylla PRIVATE Seastar::seastar absl::headers diff --git a/cmake/enable_lto.cmake b/cmake/enable_lto.cmake new file mode 100644 index 000000000000..c3a3786d8cc3 --- /dev/null +++ b/cmake/enable_lto.cmake @@ -0,0 +1,30 @@ +# +# Copyright 2024-present ScyllaDB +# + +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# +function(enable_lto name) + get_target_property(type ${name} TYPE) + if(type MATCHES "OBJECT_LIBRARY|STATIC_LIBRARY|SHARED_LIBRARY|EXECUTABLE") + target_compile_options(${name} PRIVATE + $<$:-ffat-lto-objects>) + set_property(TARGET ${name} PROPERTY + INTERPROCEDURAL_OPTIMIZATION_RELWITHDEBINFO ON) + if(type MATCHES "SHARED_LIBRARY|EXECUTABLE") + target_link_options(${name} + PRIVATE $<$:-ffat-lto-objects>) + endif() + elseif(type STREQUAL "INTERFACE_LIBRARY") + if (name MATCHES "^scylla_(.*)$") + # Special handling for scylla_* libraries with whole archive linking + set(library "${CMAKE_MATCH_1}") + enable_lto(${library}) + # For non-scylla_* INTERFACE libraries, we don't compile them, + # hence no need to set the LTO compile options or property + endif() + else() + message(FATAL_ERROR "Unsupported TYPE: ${name}:${type}") + endif() +endfunction() diff --git a/cmake/mode.common.cmake b/cmake/mode.common.cmake index 35e58415e9c0..929a7cbc15aa 100644 --- a/cmake/mode.common.cmake +++ b/cmake/mode.common.cmake @@ -156,3 +156,15 @@ else() get_padded_dynamic_linker_option(dynamic_linker_option 511) endif() add_link_options("${dynamic_linker_option}") + +if(Scylla_ENABLE_LTO) + include(CheckIPOSupported) + block() + set(CMAKE_EXE_LINKER_FLAGS "${linker_flag}") + set(CMAKE_TRY_COMPILE_PLATFORM_VARIABLES CMAKE_EXE_LINKER_FLAGS) + check_ipo_supported(RESULT ipo_supported OUTPUT error) + if(NOT ipo_supported) + message(FATAL_ERROR "LTO is not supported: ${error}") + endif() + endblock() +endif() diff --git a/configure.py b/configure.py index e694f8d3dcde..0b52eeb816c3 100755 --- a/configure.py +++ b/configure.py @@ -2758,6 +2758,7 @@ def configure_using_cmake(args): 'Scylla_DIST': 'ON' if args.enable_dist in (None, True) else 'OFF', 'Scylla_TEST_TIMEOUT': args.test_timeout, 'Scylla_TEST_REPEAT': args.test_repeat, + 'Scylla_ENABLE_LTO': 'ON' if args.lto else 'OFF', } if args.date_stamp: settings['Scylla_DATE_STAMP'] = args.date_stamp diff --git a/test/CMakeLists.txt b/test/CMakeLists.txt index 77d90e2087c5..10ffdec4b3b5 100644 --- a/test/CMakeLists.txt +++ b/test/CMakeLists.txt @@ -50,6 +50,12 @@ function(add_scylla_test name) list(APPEND scylla_tests "${dirname}/${name}") set(scylla_tests "${scylla_tests}" PARENT_SCOPE) + if(Scylla_ENABLE_LTO) + # The runtime benefits of LTO don't outweight the compile time costs for tests. + target_link_options(${name} PRIVATE + $<$:-fno-lto>) + endif() + target_include_directories(${name} PRIVATE ${CMAKE_SOURCE_DIR}) From 6adf70ec03447e26065371d993472a70386c38f4 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Thu, 29 Jun 2023 20:56:41 +0800 Subject: [PATCH 117/397] build: cmake: add CMake options for PGO support - "Scylla_BUILD_INSTRUMENTED" option Scylla_BUILD_INSTRUMENTED allows us to instrument the code at different level, namely, IR, and CSIR. this option mirrors "--pgo" and "--cspgo" options in `configure.py` . please note, the instrumentation at the frontend is not supported, as the IR based instrumentation is better when it comes to the use case of optimization for performance. see https://lists.llvm.org/pipermail/llvm-dev/2015-August/089044.html for the rationales. - "Scylla_PROFDATA_FILE" option this option allows us to specify the profile data previous generated with the "Scylla_BUILD_INSTRUMENTED" option. this option mirrors the `--use-profile` option in `configure.py`, but it does not take the empty option as a special case and consider it as a file fetched from Git LFS. that will be handled by another option in a follow-up change. please note, one cannot use -DScylla_BUILD_INSTRUMENTED=PGO and -DScylla_PROFDATA_FILE=... at the same time. clang just does not allow this. but CSPGO is fine. - "Scylla_PROFDATA_COMPRESSED_FILE" option this option allows us to specify the compressed profile data previouly generated with the "Scylla_BUILD_INSTRUMENTED" option. along with "Scylla_PROFDATA_FILE", this option mirros the functionality of `--use-profile` in `configure.py`. the goal is to ensure user always gets the result with the specified options. if anything goes wrong, we just error out. Signed-off-by: Kefu Chai --- CMakeLists.txt | 4 ++ cmake/mode.common.cmake | 94 +++++++++++++++++++++++++++++++++++++++++ configure.py | 16 +++++++ pgo/CMakeLists.txt | 33 +++++++++++++++ 4 files changed, 147 insertions(+) create mode 100644 pgo/CMakeLists.txt diff --git a/CMakeLists.txt b/CMakeLists.txt index 16713425a422..31cf2ce768e0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -355,3 +355,7 @@ add_dependencies(compiler-training if(Scylla_DIST) add_subdirectory(dist) endif() + +if(Scylla_BUILD_INSTRUMENTED) + add_subdirectory(pgo) +endif() diff --git a/cmake/mode.common.cmake b/cmake/mode.common.cmake index 929a7cbc15aa..0099fd43c937 100644 --- a/cmake/mode.common.cmake +++ b/cmake/mode.common.cmake @@ -118,6 +118,100 @@ macro(update_cxx_flags flags) endif() endmacro() +set(pgo_opts "") +# Clang supports three instrumenttation methods for profile generation: +# - -fprofile-instr-generate: this happens in the frontend (AST phase) +# - -fprofile-generate: this happens in the middle end. this instruments the LLVM IR +# generated by the front-end. and is called the regular PGO. +# - -fcs-profile-generate: "cs" is short for Context Sensitive. this instrumentation +# method is called CSPGO in comparison with the regular PGO above. +# We use IR and CSIR to represent the last two instrumentation methods in the option +# of Scylla_BUILD_INSTRUMENTED. the frontend instrumentation is not supported, because +# the IR-based instrumentation is superier than the frontend-based instrumentation when +# profiling executable for optimization purposes. +set(Scylla_BUILD_INSTRUMENTED OFF CACHE STRING + "Build ScyllaDB with PGO instrumentation. May be specified as IR, CSIR") +if(Scylla_BUILD_INSTRUMENTED) + file(TO_NATIVE_PATH "${CMAKE_BINARY_DIR}/${Scylla_BUILD_INSTRUMENTED}" Scylla_PROFILE_DATA_DIR) + if(Scylla_BUILD_INSTRUMENTED STREQUAL "IR") + # instrument code at IR level, also known as the regular PGO + string(APPEND pgo_opts " -fprofile-generate=\"${Scylla_PROFILE_DATA_DIR}\"") + elseif(Scylla_BUILD_INSTRUMENTED STREQUAL "CSIR") + # instrument code with Context Sensitive IR, also known as CSPGO. + string(APPEND pgo_opts " -fcs-profile-generate=\"${Scylla_PROFILE_DATA_DIR}\"") + else() + message(FATAL_ERROR "Unknown Scylla_BUILD_INSTRUMENTED: ${}") + endif() +endif() + +set(Scylla_PROFDATA_FILE "" CACHE FILEPATH + "Path to the profiling data file to use when compiling.") +set(Scylla_PROFDATA_COMPRESSED_FILE "" CACHE FILEPATH + "Path to the compressed profiling data file to use when compiling") +if(Scylla_PROFDATA_FILE AND Scylla_PROFDATA_COMPRESSED_FILE) + message(FATAL_ERROR + "Both Scylla_PROFDATA_FILE and Scylla_PROFDATA_COMPRESSED_FILE are specified!") +endif() + +function(extract_compressed_file) + find_program(XZCAT xzcat + REQUIRED) + + cmake_parse_arguments(parsed_args "" "INPUT;OUTPUT" "" ${ARGN}) + set(input ${parsed_args_INPUT}) + + get_filename_component(ext "${input}" LAST_EXT) + get_filename_component(stem "${input}" NAME_WLE) + set(output "${CMAKE_BINARY_DIR}/${stem}") + if(ext STREQUAL ".xz") + execute_process( + COMMAND ${XZCAT} "${input}" + OUTPUT_FILE "${output}" + WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}") + else() + message(FATAL_ERROR "Unknown compression format: ${ext}") + endif() + set(${parsed_args_OUTPUT} ${output} PARENT_SCOPE) +endfunction(extract_compressed_file) + +if(Scylla_PROFDATA_FILE) + if(NOT EXISTS "${Scylla_PROFDATA_FILE}") + message(FATAL_ERROR + "Specified Scylla_PROFDATA_FILE (${Scylla_PROFDATA_FILE}) does not exist") + endif() + set(profdata_file "${Scylla_PROFDATA_FILE}") +elseif(Scylla_PROFDATA_COMPRESSED_FILE) + # read the header to see if the file is fetched by LFS upon checkout + file(READ "${Scylla_PROFDATA_COMPRESSED_FILE}" file_header LIMIT 7) + if(file_header MATCHES "version") + message(FATAL_ERROR "Please install git-lfs for using profdata stored in Git LFS") + endif() + extract_compressed_file( + INPUT "${Scylla_PROFDATA_COMPRESSED_FILE}" + OUTPUT "profdata_file") +endif() + +if(profdata_file) + if(Scylla_BUILD_INSTRUMENTED STREQUAL "IR") + # -fprofile-use is not allowed with -fprofile-generate + message(WARNING "Only CSIR supports using and generating profdata at the same time.") + unset(pgo_opts) + endif() + # When building with PGO, -Wbackend-plugin generates a warning for every + # function which changed its control flow graph since the profile was + # taken. + # We allow stale profiles, so these warnings are just noise to us. + # Let's silence them. + string(APPEND CMAKE_CXX_FLAGS " -Wno-backend-plugin") + string(APPEND CMAKE_CXX_FLAGS " -fprofile-use=\"${profdata_file}\"") +endif() + +if(pgo_opts) + string(APPEND CMAKE_CXX_FLAGS "${pgo_opts}") + string(APPEND CMAKE_EXE_LINKER_FLAGS "${pgo_opts}") + string(APPEND CMAKE_SHARED_LINKER_FLAGS "${pgo_opts}") +endif() + # Force SHA1 build-id generation add_link_options("LINKER:--build-id=sha1") include(CheckLinkerFlag) diff --git a/configure.py b/configure.py index 0b52eeb816c3..a06cba0e176f 100755 --- a/configure.py +++ b/configure.py @@ -2766,6 +2766,22 @@ def configure_using_cmake(args): settings['Boost_USE_STATIC_LIBS'] = 'ON' if args.clang_inline_threshold != -1: settings['Scylla_CLANG_INLINE_THRESHOLD'] = args.clang_inline_threshold + if args.cspgo: + settings['Scylla_BUILD_INSTRUMENTED'] = "CSIR" + elif args.pgo: + settings['Scylla_BUILD_INSTRUMENTED'] = "IR" + if args.use_profile: + settings['Scylla_PROFDATA_FILE'] = args.use_profile + elif args.use_profile is None: + profile_archive_path = f"pgo/profiles/{platform.machine()}/profile.profdata.xz" + if "compressed data" in subprocess.check_output(["file", profile_archive_path], text=True): + settings['Scylla_PROFDATA_COMPRESSED_FILE'] = profile_archive_path + else: + # Avoid breaking existing pipelines without git-lfs installed. + print(f"WARNING: {profile_archive_path} is not an archive. Building without a profile.", file=sys.stderr) + # scripts/refresh-pgo-profiles.sh does not specify the path to the profile + # so we don't define Scylla_PROFDATA_COMPRESSED_FILE, and use the default + # value source_dir = os.path.realpath(os.path.dirname(__file__)) if os.path.isabs(args.build_dir): diff --git a/pgo/CMakeLists.txt b/pgo/CMakeLists.txt new file mode 100644 index 000000000000..9d4279580f3a --- /dev/null +++ b/pgo/CMakeLists.txt @@ -0,0 +1,33 @@ +# ts is used by pgo/pgo.py +find_program(TIMESTAMP ts + REQUIRED) + +set(profdata_dir "${Scylla_PROFILE_DATA_DIR}/profiles") +set(sampled_profdata "${profdata_dir}/prof.profdata") +# Profile collection depends on java tools because we use cassandra-stress as the load. +add_custom_command( + OUTPUT "${sampled_profdata}" + DEPENDS + scylla + dist-tools-tar + COMMAND ${CMAKE_COMMAND} -E rm -r "${profdata_dir}" + COMMAND ${CMAKE_SOURCE_DIR}/pgo/train "$" + ${sampled_profdata} + ${CMAKE_BINARY_DIR}/pgo_datasets) +add_custom_target(sampled_profdata + DEPENDS "${sampled_profdata}") + +set(all_profdata_files ${sampled_profdata}) +if(profdata_file) + list(APPEND all_profdata_files "${profdata_file}") +endif() +set(merged_profdata "${profdata_dir}/merged.profdata") + +find_program(LLVM_PROFDATA llvm-profdata + REQUIRED) +add_custom_command( + OUTPUT "${merged_profdata}" + DEPENDS "${all_profdata_files}" + COMMAND ${LLVM_PROFDATA} merge ${all_profdata_files} -output "${merged_profdata}") +add_custom_target(merged_profdata + DEPENDS "${merged_profdata}") From 71eccf01c79e647d2f0f1e091ef56903cbf286a3 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 16 Dec 2024 11:26:26 +0800 Subject: [PATCH 118/397] test/pylib: use "foo not in bar" instead of "not foo in bar" for better readability Signed-off-by: Kefu Chai --- test/pylib/scylla_cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 457d228690ae..61b9286ccd5b 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -301,7 +301,7 @@ def __init__(self, mode: str, exe: str, vardir: str, self.resources_certificate_file = self.resourcesdir / "scylla.crt" self.resources_keyfile_file = self.resourcesdir / "scylla.key" - if property_file and not "endpoint_snitch" in config_options: + if property_file and "endpoint_snitch" not in config_options: config_options["endpoint_snitch"] = "GossipingPropertyFileSnitch" # Sum of basic server configuration and the user-provided config options. From cf35562e8910ddf3bce2be819466ea580016b3e2 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 16 Dec 2024 11:28:52 +0800 Subject: [PATCH 119/397] test/pylib: use `foo` instead of `'{}'.format(foo)` for better readability Signed-off-by: Kefu Chai --- test/pylib/scylla_cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 61b9286ccd5b..3e4ac9878f2f 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -88,7 +88,7 @@ def make_scylla_conf(mode: str, workdir: pathlib.Path, host_addr: str, seed_addr 'seed_provider': [{ 'class_name': 'org.apache.cassandra.locator.SimpleSeedProvider', 'parameters': [{ - 'seeds': '{}'.format(','.join(seed_addrs)) + 'seeds': ','.join(seed_addrs) }] }], @@ -335,7 +335,7 @@ def change_seeds(self, seeds: List[str]): if self.is_running: raise RuntimeError(f"Can't change seeds of a running server {self.ip_addr}.") self.seeds = seeds - self.config['seed_provider'][0]['parameters'][0]['seeds'] = '{}'.format(','.join(seeds)) + self.config['seed_provider'][0]['parameters'][0]['seeds'] = ','.join(seeds) self._write_config_file() @property From fdb2d2209c3fc6b5c1a4ef3dcf44717d11324a63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 4 Dec 2024 18:01:54 +0100 Subject: [PATCH 120/397] messaging_service: use advanced_rpc_compression::tracker for compression This patch sets up an `alien_worker`, `advanced_rpc_compression::tracker`, `dict_sampler` and `dictionary_service` in `main()`, and wires them to each other and to `messaging_service`. `messaging_service` compresses its network traffic with compressors managed by the `advanced_rpc_compression::tracker`. All this traffic is passed as a single merged "stream" through `dict_sampler`. `dictionary_service` has access to `dict_sampler`. On chosen nodes (by default: the Raft leader), it uses the sampler to maintain a random multi-megabyte sample of the sampler's stream. Every several minutes, it copies the sample, trains a compression dictionary on it (by calling zstd's training library via the `alien_worker` thread) and publishes the new dictionary to `system.dicts` via Raft. This update triggers a callback into `advanced_rpc_compression::tracker` on all nodes, which updates the dictionary used by the compressors it manages. --- conf/scylla.yaml | 41 +++ db/config.cc | 81 +++++- db/config.hh | 14 + docs/dev/advanced_rpc_compression.md | 267 ++++++++++++++++++ .../advanced-internode-compression.rst | 74 +++++ .../procedures/config-change/index.rst | 3 + main.cc | 66 ++++- message/messaging_service.cc | 59 +++- message/messaging_service.hh | 10 +- service/storage_service.cc | 5 +- service/storage_service.hh | 3 +- test/lib/cql_test_env.cc | 19 +- test/manual/gossip.cc | 6 +- test/manual/message.cc | 6 +- test/topology_custom/test_rpc_compression.py | 258 +++++++++++++++++ 15 files changed, 888 insertions(+), 24 deletions(-) create mode 100644 docs/dev/advanced_rpc_compression.md create mode 100644 docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst create mode 100644 test/topology_custom/test_rpc_compression.py diff --git a/conf/scylla.yaml b/conf/scylla.yaml index 3ef9b89a267b..46becd020390 100644 --- a/conf/scylla.yaml +++ b/conf/scylla.yaml @@ -477,6 +477,47 @@ commitlog_total_space_in_mb: -1 # none - nothing is compressed. # internode_compression: none +# Enables inter-node traffic compression metrics (`scylla_rpc_compression_...`) +# and enables a new implementation of inter-node traffic compressors, +# capable of using zstd (in addition to the default lz4) +# and shared dictionaries. +# (Those features must still be enabled by other settings). +# Has minor CPU cost. +# +# internode_compression_enable_advanced: false + +# Enables training of shared compression dictionaries on inter-node traffic. +# New dictionaries are distributed throughout the cluster via Raft, +# and used to improve the effectiveness of inter-node traffic compression +# when `internode_compression_enable_advanced` is enabled. +# +# WARNING: this may leak unencrypted data to disk. The trained dictionaries +# contain randomly-selected pieces of data written to the cluster. +# When the Raft log is unencrypted, those pieces of data will be +# written to disk unencrypted. At the moment of writing, there is no +# way to encrypt the Raft log. +# This problem is tracked by https://github.com/scylladb/scylla-enterprise/issues/4717. +# +# Can be: never - Dictionaries aren't trained by this node. +# when_leader - New dictionaries are trained by this node only if +# it's the current Raft leader. +# always - Dictionaries are trained by this node unconditionally. +# +# For efficiency reasons, training shouldn't be enabled on more than one node. +# To enable it on a single node, one can let the cluster pick the trainer +# by setting `when_leader` on all nodes, or specify one manually by setting `always` +# on one node and `never` on others. +# +# rpc_dict_training_when: never + +# A number in range [0.0, 1.0] specifying the share of CPU which can be spent +# by this node on compressing inter-node traffic with zstd. +# +# Depending on the workload, enabling zstd might have a drastic negative +# effect on performance, so it shouldn't be done lightly. +# +# internode_compression_zstd_max_cpu_fraction: 0.0 + # Enable or disable tcp_nodelay for inter-dc communication. # Disabling it will result in larger (but fewer) network packets being sent, # reducing overhead from the TCP protocol itself, at the cost of increasing diff --git a/db/config.cc b/db/config.cc index 7c21860e306b..1bc964a0f240 100644 --- a/db/config.cc +++ b/db/config.cc @@ -179,6 +179,13 @@ const config_type config_type_for = con template <> const config_type config_type_for> = config_type("error injection list", error_injection_list_to_json); +template <> +const config_type config_type_for> = config_type( + "dictionary training conditions", printable_to_json>); + +template <> +const config_type config_type_for = config_type( + "advanced rpc compressor config", printable_vector_to_json>); } namespace YAML { @@ -312,6 +319,41 @@ struct convert { } }; + +template <> +class convert> { +public: + static bool decode(const Node& node, enum_option& rhs) { + std::string name; + if (!convert::decode(node, name)) { + return false; + } + try { + std::istringstream(name) >> rhs; + } catch (boost::program_options::invalid_option_value&) { + return false; + } + return true; + } +}; + +template <> +class convert> { +public: + static bool decode(const Node& node, enum_option& rhs) { + std::string name; + if (!convert::decode(node, name)) { + return false; + } + try { + std::istringstream(name) >> rhs; + } catch (boost::program_options::invalid_option_value&) { + return false; + } + return true; + } +}; + } #if defined(DEBUG) @@ -776,9 +818,42 @@ db::config::config(std::shared_ptr exts) "Sets the receiving socket buffer size in bytes for inter-node calls.") , internode_compression(this, "internode_compression", value_status::Used, "none", "Controls whether traffic between nodes is compressed. The valid values are:\n" - "* all: All traffic is compressed.\n" - "* dc: Traffic between data centers is compressed.\n" - "* none: No compression.") + "\tall: All traffic is compressed.\n" + "\tdc : Traffic between data centers is compressed.\n" + "\tnone : No compression.") + , internode_compression_zstd_max_cpu_fraction(this, "internode_compression_zstd_max_cpu_fraction", liveness::LiveUpdate, value_status::Used, 0.000, + "ZSTD compression of RPC will consume at most this fraction of each internode_compression_zstd_quota_refresh_period_ms time slice.\n" + "If you wish to try out zstd for RPC compression, 0.05 is a reasonable starting point.") + , internode_compression_zstd_cpu_quota_refresh_period_ms(this, "internode_compression_zstd_cpu_quota_refresh_period_ms", liveness::LiveUpdate, value_status::Used, 20, + "Advanced. ZSTD compression of RPC will consume at most internode_compression_zstd_max_cpu_fraction (plus one message) of in each time slice of this length.") + , internode_compression_zstd_max_longterm_cpu_fraction(this, "internode_compression_zstd_max_longterm_cpu_fraction", liveness::LiveUpdate, value_status::Used, 1.000, + "ZSTD compression of RPC will consume at most this fraction of each internode_compression_zstd_longterm_cpu_quota_refresh_period_ms time slice.") + , internode_compression_zstd_longterm_cpu_quota_refresh_period_ms(this, "internode_compression_zstd_longterm_cpu_quota_refresh_period_ms", liveness::LiveUpdate, value_status::Used, 10000, + "Advanced. ZSTD compression of RPC will consume at most internode_compression_zstd_max_longterm_cpu_fraction (plus one message) of in each time slice of this length.") + , internode_compression_zstd_min_message_size(this, "internode_compression_zstd_min_message_size", liveness::LiveUpdate, value_status::Used, 0, + "Minimum RPC message size which can be compressed with ZSTD. Messages smaller than this threshold will always be compressed with LZ4. " + "ZSTD has high per-message overhead, and might be a bad choice for small messages. This knob allows for some experimentation with that. ") + , internode_compression_zstd_max_message_size(this, "internode_compression_zstd_max_message_size", liveness::LiveUpdate, value_status::Used, std::numeric_limits::max(), + "Maximum RPC message size which can be compressed with ZSTD. RPC messages might be large, but they are always compressed at once. This might cause reactor stalls. " + "If this happens, this option can be used to make the stalls less severe.") + , internode_compression_checksumming(this, "internode_compression_checksumming", liveness::LiveUpdate, value_status::Used, true, + "Computes and checks checksums for compressed RPC frames. This is a paranoid precaution against corruption bugs in the compression protocol.") + , internode_compression_algorithms(this, "internode_compression_algorithms", liveness::LiveUpdate, value_status::Used, + { utils::compression_algorithm::type::ZSTD, utils::compression_algorithm::type::LZ4, }, + "Specifies RPC compression algorithms supported by this node. ") + , internode_compression_enable_advanced(this, "internode_compression_enable_advanced", liveness::MustRestart, value_status::Used, false, + "Enables the new implementation of RPC compression. If disabled, Scylla will fall back to the old implementation.") + , rpc_dict_training_when(this, "rpc_dict_training_when", liveness::LiveUpdate, value_status::Used, utils::dict_training_loop::when::type::NEVER, + "Specifies when RPC compression dictionary training is performed by this node.\n" + "* `never` disables it unconditionally.\n" + "* `when_leader` enables it only whenever the node is the Raft leader.\n" + "* `always` (not recommended) enables it unconditionally.\n" + "\n" + "Training shouldn't be enabled on more than one node at a time, because overly-frequent dictionary announcements might indefinitely delay nodes from agreeing on a new dictionary.") + , rpc_dict_training_min_time_seconds(this, "rpc_dict_training_min_time_seconds", liveness::LiveUpdate, value_status::Used, 3600, + "Specifies the minimum duration of RPC compression dictionary training.") + , rpc_dict_training_min_bytes(this, "rpc_dict_training_min_bytes", liveness::LiveUpdate, value_status::Used, 1'000'000'000, + "Specifies the minimum volume of RPC compression dictionary training.") , inter_dc_tcp_nodelay(this, "inter_dc_tcp_nodelay", value_status::Used, false, "Enable or disable tcp_nodelay for inter-data center communication. When disabled larger, but fewer, network packets are sent. This reduces overhead from the TCP protocol itself. However, if cross data-center responses are blocked, it will increase latency.") , streaming_socket_timeout_in_ms(this, "streaming_socket_timeout_in_ms", value_status::Unused, 0, diff --git a/db/config.hh b/db/config.hh index a1641cebfabf..64f9ffce3a43 100644 --- a/db/config.hh +++ b/db/config.hh @@ -25,6 +25,8 @@ #include "utils/updateable_value.hh" #include "utils/s3/creds.hh" #include "utils/error_injection.hh" +#include "utils/dict_trainer.hh" +#include "utils/advanced_rpc_compressor.hh" namespace seastar { class file; @@ -278,6 +280,18 @@ public: named_value internode_send_buff_size_in_bytes; named_value internode_recv_buff_size_in_bytes; named_value internode_compression; + named_value internode_compression_zstd_max_cpu_fraction; + named_value internode_compression_zstd_cpu_quota_refresh_period_ms; + named_value internode_compression_zstd_max_longterm_cpu_fraction; + named_value internode_compression_zstd_longterm_cpu_quota_refresh_period_ms; + named_value internode_compression_zstd_min_message_size; + named_value internode_compression_zstd_max_message_size; + named_value internode_compression_checksumming; + named_value internode_compression_algorithms; + named_value internode_compression_enable_advanced; + named_value> rpc_dict_training_when; + named_value rpc_dict_training_min_time_seconds; + named_value rpc_dict_training_min_bytes; named_value inter_dc_tcp_nodelay; named_value streaming_socket_timeout_in_ms; named_value start_native_transport; diff --git a/docs/dev/advanced_rpc_compression.md b/docs/dev/advanced_rpc_compression.md new file mode 100644 index 000000000000..c77f2e046e7e --- /dev/null +++ b/docs/dev/advanced_rpc_compression.md @@ -0,0 +1,267 @@ +# Advanced RPC compression modes + +## Introduction + +For many years, the only mode of compression available for inter-node (RPC) traffic +had been per-message LZ4 compression. This is suboptimal, because it fails to take +advantage of the substantial redundancy across messages. + +To exploit cross-message redundancy, a few common techniques are used: +- Streaming compression, in which both sides of the connection keep some messages + from the recent past, and compress new messages against that context. +- Buffering, in which multiple messages are compressed together. +- Fixed-dictionary compression, in which messages are compressed against some + representative context, pre-agreed by both sides. + +Streaming compression, when used indiscriminately, has some fundamental scaling problems. +Each stream-compressed connection requires its own history buffer, which puts memory +pressure, both on RAM (hundreds of streams might require a significant fraction of a machine's +memory) and on CPU caches (history buffers are randomly accessed for each compressed +byte, and if there are many of them, they are likely to be cold, potentially slowing down +the compression significantly). + +Technically, the drawbacks of streaming compression can be worked around (e.g. by limiting +stream compression only to the few most important connections, and/or by using a sparse +network topology), but this requires uncomfortably complex solutions. + +Buffering doesn't put as much pressure on memory as streaming, but adds latency, +and has its own fundamental scaling issue -- the more nodes, the lower the utilization +of each connection, which means a smaller number of messages buffered in the same unit +of time. With a sufficiently big cluster, only one message can be gathered during +a reasonably short delay, rendering the mechanism useless. + +A fixed dictionary requires more assumptions about the data to work well +(i.e. that connections are similar and their content is relatively uniform over time), +but it has a constant overhead regardless of the number of connections, so it scales perfectly. +And given that most of the time Scylla clusters are symmetric, a dictionary trained on one +node should work reasonably well on all nodes. + +Thus, we chose to employ dictionary compression for Scylla's RPC compressions. + +## Feature overview + +The general idea is: +1. Some node in the cluster periodically trains a compression dictionary + on its own RPC traffic, and distributes it. +2. Whenever a new dictionary becomes available on both sides of an RPC connection, + the connection switches to using the new dictionary. + +## Details + +### Trainer selection + +Given the assumption that shards in the cluster are symmetric, there's no need to involve +more than one shard in the training of any particular dictionary. And since involving only +one shard is simpler than involving many, we only involve one. + +In addition, it doesn't make sense to train multiple dictionaries concurrently, +since only the last one will be eventually used, and the earlier ones will only cause +noise. + +Given the above, we designate a single shard in the cluster (at a time) to perform the traffic +sampling and dictionary training. As of this writing, we designate shard 0 of the raft leader +for that, just because it's a single shard selection mechanism that's already implemented. + +The rest of the design doesn't rely on the single-trainer property for correctness, +and it's fine if multiple shards think they are the trainer (leader) for some time. + +However, as is explained later in this doc, it is currently needed for progress +(towards switching connections to new dictionary versions) that the most recent dictionary +version in the cluster remains stable for some time, so a split-brain situation +could theoretically delay progress. But that's acceptable. + +### Sampling + +Sampling is implemented by merging all outgoing RPC traffic on shard 0 into a single stream, +splitting this stream into pages (of size ~8 kiB), and taking a uniform +random sample (total size ~16 MiB) of all pages seen during the sampling +period, via reservoir sampling. The sample is kept in RAM. + +The page size should be big enough for the strings it contains to be worth compressing +(so, at least several bytes), but the number of pages in the sample (inversely proportional +to page size) should be big enough to minimize the chance of all pages falling onto +non-representative parts of the stream. The current page size was chosen based on intuition. + +The sample size is based on the recommendations from zstd's documentation -- it says that +~100 kiB is a reasonable dictionary size, and that the sample should be on the order of +100 times larger than the dictionary. + +The duration of sampling is configurable by the minimum amount of data that has to be ingested +(to avoid training unrepresentative dictionaries during periods of downtime) and by the minimum +amount of time that has to pass (to get some predictable upper limit on the rate of dictionary +updates). + +### Training + +When the sampling finishes, a dictionary (~100 kiB) is trained on the (~16 MiB) sample. + +For this, we use Zstd's dictionary builder. Since it requires a few seconds of CPU time, +it can't be used directly under the reactor, so we run it in a separate, low-priority OS thread. + +Note: LZ4 limits the max size of the dictionary to 64 kiB. +But currently we only train a single ~100 kiB dictionary and +use the first 64 kiB of that as the dict for LZ4. +Perhaps we are getting a suboptimal LZ4 dictionary this way. +Maybe we should do a separate 64 kiB training. + +### Distribution + +We distribute new dictionaries via a Raft-based Scylla table. +(Local to each node, but with coordinated mutation via Raft). +The trainer inserts new dictionaries to the table via a Raft command. +Other nodes pick it up when the Raft command is applied to them. + +The table is: +``` +CREATE TABLE system.dicts ( + name text, + timestamp timestamp, + origin uuid, + data blob, + PRIMARY KEY (name) +); +``` + +As of now, only a single `name` is used (`"general"`) — because there is only a single +dictionary in the cluster, shared by all RPC connections. In the future, we might want to +have separate dictionaries for different DCs, or for different service levels, etc. +If that comes to be, they will be stored in their own partitions. + +The `timestamp` and `origin` fields describe the time and node on which the dictionary +was created. They don't really have to exist, but we have them for observation/debugging purposes. +(And they are used as a part of the dictionary's ID in the negotiation protocol for the same purpose). + +The `data` blob contains the actual dictionary contents. Ideally it's a zstd-formatted +dictionary, but any blob should work. + +During dictionary negotiation, nodes identify dictionaries by a `` triple. + +If two different dictionaries with a different content but the same SHA256 are ever +published with the same ``, the negotiation might result in a corrupted +connection. No countermeasures against this have been implemented as of now. + +### Negotiation + +Dictionary compression requires that the sender and the receiver both use the same dictionary. +In particular, the sender can only start using a new dictionary after the receiver acquires +it, and the receiver can't forget a dictionary as long as the sender uses it. + +Since this is the only part of the entire design which requires some distributed coordination, +this is probably the most complicated part. + +There are many possible approaches. For example, the easiest one is to just let the sender +send the new dictionary over the connection whenever it wants to switch. This requires no +inter-node coordination. However, it's relatively expensive -- the number of connections is quadratic +in the size of the cluster, and this method requires each connection to transfer the entire +dictionary. Ideally, each node should only have to download each new dictionary only once. + +#### Description + +The negotiation algorithm we currently use works like this (it is run for each connection direction independently): + +There are two peers: sender (compresses) and receiver (decompresses). They communicate via an ordered +and reliable stream of messages. +There's one peculiarity here: it is assumed that messages can be canceled by newer messages. +i.e. when multiple messages are requested to be sent, only the last one is guaranteed +to eventually be sent, while older ones can be cancelled. +Because the algorithm respects this assumption, it can be run in constant memory -- +there never has to be a queue of messages longer than 1. + +Each peer references three dicts: +- `recent` -- the target dict. We want to switch to it as soon as our peer also acquires it. +Updates to this reference come asynchronously from outside; it is an input to the algorithm. +It is assumed that both sides of the connection will eventually have the same `recent`, +and this will hold for a long time. +- `committed` -- the dict we last proposed to our peer. This reference is set to `recent` at +the moment we make the suggestion. +The receiver needs this reference so that it can immediately switch to it when the sender accepts +the suggestion. +- `current` -- the one actually used for compression at the moment. It is set to `committed` at the moment +the proposal is accepted. + +The algorithm works in epochs. Each message has an epoch number attached. +Sender monotonically increments the epoch whenever it makes a new proposal. +Receiver always echoes back the last observed epoch. + +The flow of each epoch is as follows: +0. Receiver notifies sender about any change in `recent` by sending ``. +1. When sender gets ``, or when sender's `recent` changes, + sender increments the epoch, and requests `` to be sent. + Atomically with the send (i.e. if and only if it isn't cancelled by another message), + it sets `committed := recent`. +2. When receiver gets ` | x == recent`, it sets `committed := recent`, + and requests `` to be sent. +3. When sender gets ` | x == committed` from the current epoch, + it requests `` to be sent. Atomically with the send, it sets `current := committed`. + (Else when sender gets ` | x != committed` from the current epoch, + it requests `` to be sent. This could be omitted.) +4. When sender receives ` | x == committed`, it sets `current := committed`. + (When it receives ` | x != committed`, it doesn't do anything). + +#### Proof sketch + +Property 1 (safety): both peers use the same dictionary for every message: + +Both sides modify `current` (by setting `current := committed`) only at the point +when they process sender's `COMMIT`. So to prove safety, it suffices to prove +that both sides have the same `committed` at this point. + +Sender sends its COMMIT in response to receiver's COMMIT. +But receiver sent the COMMIT only if receiver's `committed` was equal to sender's `committed` +at the time of sender's UPDATE. + +But since the epoch hasn't changed, sender couldn't have changed its `committed` since that UPDATE, +and it also hasn't sent anything which could have modified the receiver's `committed`. + +So both sender and the receiver have the same `committed` +when they process the sender's COMMIT. + +Property 2 (liveness): if both peers have `recent` set to the same dict `x` for sufficiently long, +the algorithm will eventually switch `current` to it: + +At the moment receiver acquires `x`, it sends a notification to sender, which will +start a new epoch. At the moment sender acquires `x`, it will also start a new epoch. +Whichever of these two epochs comes later, will eventually succeed. +(Easy to prove). + +#### Notes + +There are some small differences between the description above here and the implementation, +but they shouldn't alter the algorithm. (E.g. instead of requesting ``, +sender sets `committed := current` and requests ``. This should be +equivalent.) + +The algorithm also negotiates parameters other than the dict. +In particular, it negotiates the available compression libraries (zstd vs lz4). +But these params don't require coordination -- the receiver can decompress messages +regardless of the values chosen by the sender -- so this is easy. +The receiver just communicates its preferences in its COMMIT, and the sender +unilaterally sets the parameters based on these preferences. + +There is a TLA specification of the algorithm in +[advanced_rpc_compression_negotiation.tla](advanced_rpc_compression_negotiation.tla). + +### Usage + +Dictionaries are read-only, so -- to minimize memory pressure -- we take care to have only +one copy of each dictionary, on shard 0, kept alive by foreign shared pointers. + +Likewise, we keep only a single compressor and a single decompressor for each algorithm, +and whenever a connection needs to use it, it plugs the correct dictionary in. +Switching dictionaries should be cheaper than keeping mulitple copies of the compressors. + +### Wire protocol details + +This section describes the layout of a compressed frame produced by `advanced_rpc_compressor::compress()`. + +The compression algorithm is selected on per-message basis. +(Rationale: this allows the sender to weaken the compression unilaterally if it doens't have the resources for the "normal" algorithm.) +The 7 least significant bits of byte 0 of each compressed message +contain an enum value describing the compression algorithm used for this message. + +The most significant bit of byte 0 tells if there is a control (negotiation) +header present. If this bit is set, then bytes 1-133 contain a packed `control_protocol_frame` struct, +carrying the negotiation protocol described earlier. See `control_protocol_frame::serialize()` for details. + +After byte 0 and the optional negotiation header, the rest of the message contains the actual contents +of the RPC frame, compressed with the selected algorithm. The details of this part are different for each algorithm. diff --git a/docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst b/docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst new file mode 100644 index 000000000000..101917bcfa01 --- /dev/null +++ b/docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst @@ -0,0 +1,74 @@ +========================================== +Advanced Internode (RPC) Compression +========================================== + +:label-tip:`Available with the Premium plan` + +Internode (RPC) compression controls whether traffic between nodes is +compressed. If enabled, it reduces network bandwidth usage. + +To further reduce network traffic, you can configure ScyllaDB Enterprise to use +ZSTD-based compression and shared dictionary compression. You can enable one or +both of these features to limit network throughput and reduce network transfer costs. + +Enabling ZSTD-based Compression +---------------------------------- + +ZSTD-based compression utilizes the ZSTD algorithm for RPC compression. + +To enable ZSTD-based compression: + +#. Set ``internode_compression_enable_advanced`` to ``true``. +#. Set ``internode_compression_zstd_max_cpu_fraction`` to + a non-zero value in the range [0, 1], where: + + * 0 - The node will never use ZSTD for RPC compression. + * 1 - The node will always use ZSTD for RPC compression (for all messages, + regardless of CPU consumption). + +A good rule of thumb is to start from ``0.05``. As a result, the node will spend +not more than 5% of its total CPU on RPC ZSTD compression. + +Note that enabling ZSTD increases CPU usage. We recommend monitoring metrics +to ensure additional CPU consumption does not disrupt the workload. If ZSTD +impacts performance, try setting ``internode_compression_zstd_max_cpu_fraction`` +to a lower value. + + +Enabling Shared Dictionary Compression +------------------------------------------- + +Shared dictionary compression allows you to limit network throughput and reduce +network transfer costs. + +If enabled, the message-by-message compression is replaced with a more +efficient compression based on a dictionary trained on one node and shared +with other nodes. The dictionary is periodically re-trained on the node’s +RPC traffic and then distributed across the cluster. + +To enable shared dictionaries: + +#. Set ``internode_compression_enable_advanced`` to ``true``. +#. Configure the ``rpc_dict_training_when`` parameter. + In typical scenarios, you should set it to ``when_leader``. + As a result, the :doc:`Raft leader ` node will + train and publish new dictionaries. + +.. code:: + + internode_compression_enable_advanced: true + rpc_dict_training_when: when_leader + +.. warning:: Enabling shared dictionary training might leak unencrypted data to disk. + + Trained dictionaries contain randomly chosen samples of data transferred between + nodes. The data samples are persisted in the Raft log, which is not encrypted. + As a result, some data from otherwise encrypted tables might be stored on disk + unencrypted. + + +Reference +------------ + +See the *Inter-node settings* section in :doc:`Configuration Parameters ` +for a complete list of internode compression options. diff --git a/docs/operating-scylla/procedures/config-change/index.rst b/docs/operating-scylla/procedures/config-change/index.rst index 26755e873cde..a39e5efae703 100644 --- a/docs/operating-scylla/procedures/config-change/index.rst +++ b/docs/operating-scylla/procedures/config-change/index.rst @@ -8,6 +8,7 @@ ScyllaDB Configuration Procedures How to Switch Snitches How to Change Compaction Strategy How to do a Rolling Restart + Advanced Internode (RPC) Compression Procedures to change ScyllaDB Configuration settings. @@ -17,3 +18,5 @@ Procedures to change ScyllaDB Configuration settings. * :doc:`How to do a Rolling Restart ` +* :doc:`Advanced Internode (RPC) Compression ` + diff --git a/main.cc b/main.cc index d2013cbba4d9..a0c9e4dbd5e1 100644 --- a/main.cc +++ b/main.cc @@ -113,6 +113,10 @@ #include "service/raft/raft_group0_client.hh" #include "service/raft/raft_group0.hh" #include "gms/gossip_address_map.hh" +#include "utils/alien_worker.hh" +#include "utils/advanced_rpc_compressor.hh" +#include "utils/shared_dict.hh" +#include "message/dictionary_service.hh" #include @@ -775,6 +779,16 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl sharded gossiper; sharded snitch; + // This worker wasn't designed to be used from multiple threads. + // If you are attempting to do that, make sure you know what you are doing. + // (It uses a std::mutex-based queue under the hood, so it may cause + // performance problems under contention). + // + // Note: we are creating this thread before app.run so that it doesn't + // inherit Seastar's CPU affinity masks. We want this thread to be free + // to migrate between CPUs; we think that's what makes the most sense. + auto rpc_dict_training_worker = utils::alien_worker(startlog, 19); + return app.run(ac, av, [&] () -> future { auto&& opts = app.configuration(); @@ -803,7 +817,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl return seastar::async([&app, cfg, ext, &cm, &sstm, &db, &qp, &bm, &proxy, &mapreduce_service, &mm, &mm_notifier, &ctx, &opts, &dirs, &prometheus_server, &cf_cache_hitrate_calculator, &load_meter, &feature_service, &gossiper, &snitch, &token_metadata, &erm_factory, &snapshot_ctl, &messaging, &sst_dir_semaphore, &raft_gr, &service_memory_limiter, - &repair, &sst_loader, &ss, &lifecycle_notifier, &stream_manager, &task_manager] { + &repair, &sst_loader, &ss, &lifecycle_notifier, &stream_manager, &task_manager, &rpc_dict_training_worker] { try { if (opts.contains("relabel-config-file") && !opts["relabel-config-file"].as().empty()) { // calling update_relabel_config_from_file can cause an exception that would stop startup @@ -1432,6 +1446,25 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl return make_ready_future<>(); }).get(); + utils::dict_sampler dict_sampler; + auto arct_cfg = [&] { + return utils::advanced_rpc_compressor::tracker::config{ + .zstd_min_msg_size = cfg->internode_compression_zstd_min_message_size, + .zstd_max_msg_size = cfg->internode_compression_zstd_max_message_size, + .zstd_quota_fraction = cfg->internode_compression_zstd_max_cpu_fraction, + .zstd_quota_refresh_ms = cfg->internode_compression_zstd_cpu_quota_refresh_period_ms, + .zstd_longterm_quota_fraction = cfg->internode_compression_zstd_max_longterm_cpu_fraction, + .zstd_longterm_quota_refresh_ms = cfg->internode_compression_zstd_longterm_cpu_quota_refresh_period_ms, + .algo_config = cfg->internode_compression_algorithms, + .register_metrics = cfg->internode_compression_enable_advanced(), + .checksumming = cfg->internode_compression_checksumming, + }; + }; + static sharded compressor_tracker; + compressor_tracker.start(arct_cfg).get(); + auto stop_compressor_tracker = defer_verbose_shutdown("compressor_tracker", [] { compressor_tracker.stop().get(); }); + compressor_tracker.local().attach_to_dict_sampler(&dict_sampler); + netw::messaging_service::config mscfg; mscfg.id = host_id; @@ -1464,6 +1497,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl } else if (compress_what == "dc") { mscfg.compress = netw::messaging_service::compress_what::dc; } + mscfg.enable_advanced_rpc_compression = cfg->internode_compression_enable_advanced(); if (encrypt == "all") { mscfg.encrypt = netw::messaging_service::encrypt_what::all; @@ -1500,7 +1534,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl } // Delay listening messaging_service until gossip message handlers are registered - messaging.start(mscfg, scfg, creds, std::ref(feature_service), std::ref(gossip_address_map)).get(); + messaging.start(mscfg, scfg, creds, std::ref(feature_service), std::ref(gossip_address_map), std::ref(compressor_tracker)).get(); auto stop_ms = defer_verbose_shutdown("messaging service", [&messaging] { messaging.invoke_on_all(&netw::messaging_service::stop).get(); }); @@ -1640,6 +1674,11 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl tsm.stop().get(); }); + auto compression_dict_updated_callback = [] () -> future<> { + auto dict = co_await sys_ks.local().query_dict(); + co_await utils::announce_dict_to_shards(compressor_tracker, std::move(dict)); + }; + supervisor::notify("initializing storage service"); debug::the_storage_service = &ss; ss.start(std::ref(stop_signal.as_sharded_abort_source()), @@ -1648,7 +1687,9 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl std::ref(messaging), std::ref(repair), std::ref(stream_manager), std::ref(lifecycle_notifier), std::ref(bm), std::ref(snitch), std::ref(tablet_allocator), std::ref(cdc_generation_service), std::ref(view_builder), std::ref(qp), std::ref(sl_controller), - std::ref(tsm), std::ref(task_manager), std::ref(gossip_address_map)).get(); + std::ref(tsm), std::ref(task_manager), std::ref(gossip_address_map), + compression_dict_updated_callback + ).get(); auto stop_storage_service = defer_verbose_shutdown("storage_service", [&] { ss.stop().get(); @@ -2059,6 +2100,25 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl return ss.local().join_cluster(sys_dist_ks, proxy, service::start_hint_manager::yes, generation_number); }).get(); + dictionary_service dict_service( + dict_sampler, + sys_ks.local(), + rpc_dict_training_worker, + group0_client, + group0_service, + stop_signal.as_local_abort_source(), + feature_service.local(), + dictionary_service::config{ + .our_host_id = host_id, + .rpc_dict_training_min_time_seconds = cfg->rpc_dict_training_min_time_seconds, + .rpc_dict_training_min_bytes = cfg->rpc_dict_training_min_bytes, + .rpc_dict_training_when = cfg->rpc_dict_training_when, + } + ); + auto stop_dict_service = defer_verbose_shutdown("dictionary training", [&] { + dict_service.stop().get(); + }); + supervisor::notify("starting tracing"); tracing.invoke_on_all(&tracing::tracing::start, std::ref(qp), std::ref(mm)).get(); auto stop_tracing = defer_verbose_shutdown("tracing", [&tracing] { diff --git a/message/messaging_service.cc b/message/messaging_service.cc index 35825e985183..158a2edbc6d3 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -141,11 +141,42 @@ using gossip_digest_ack = gms::gossip_digest_ack; using gossip_digest_ack2 = gms::gossip_digest_ack2; using namespace std::chrono_literals; -static rpc::lz4_fragmented_compressor::factory lz4_fragmented_compressor_factory; -static rpc::lz4_compressor::factory lz4_compressor_factory; -static rpc::multi_algo_compressor_factory compressor_factory { - &lz4_fragmented_compressor_factory, - &lz4_compressor_factory, +class messaging_service::compressor_factory_wrapper { + struct advanced_rpc_compressor_factory : rpc::compressor::factory { + utils::walltime_compressor_tracker& _tracker; + advanced_rpc_compressor_factory(utils::walltime_compressor_tracker& tracker) + : _tracker(tracker) + {} + const sstring& supported() const override { + return _tracker.supported(); + } + std::unique_ptr negotiate(sstring feature, bool is_server, std::function()> send_empty_frame) const override { + return _tracker.negotiate(std::move(feature), is_server, std::move(send_empty_frame)); + } + std::unique_ptr negotiate(sstring feature, bool is_server) const override { + assert(false && "negotiate() without send_empty_frame shouldn't happen"); + return nullptr; + } + }; + rpc::lz4_fragmented_compressor::factory _lz4_fragmented_compressor_factory; + rpc::lz4_compressor::factory _lz4_compressor_factory; + advanced_rpc_compressor_factory _arcf; + rpc::multi_algo_compressor_factory _multi_factory; +public: + compressor_factory_wrapper(decltype(advanced_rpc_compressor_factory::_tracker) t, bool enable_advanced) + : _arcf(t) + , _multi_factory(enable_advanced ? rpc::multi_algo_compressor_factory{ + &_arcf, + &_lz4_fragmented_compressor_factory, + &_lz4_compressor_factory, + } : rpc::multi_algo_compressor_factory{ + &_lz4_fragmented_compressor_factory, + &_lz4_compressor_factory, + }) + {} + seastar::rpc::compressor::factory& get_factory() { + return _multi_factory; + } }; struct messaging_service::rpc_protocol_server_wrapper : public rpc_protocol::server { using rpc_protocol::server::server; }; @@ -238,9 +269,16 @@ future<> messaging_service::unregister_handler(messaging_verb verb) { return _rpc->unregister_handler(verb); } -messaging_service::messaging_service(locator::host_id id, gms::inet_address ip, uint16_t port, gms::feature_service& feature_service, gms::gossip_address_map& address_map) +messaging_service::messaging_service( + locator::host_id id, + gms::inet_address ip, + uint16_t port, + gms::feature_service& feature_service, + gms::gossip_address_map& address_map, + utils::walltime_compressor_tracker& wct) : messaging_service(config{std::move(id), ip, ip, port}, - scheduling_config{{{{}, "$default"}}, {}, {}}, nullptr, feature_service, address_map) + scheduling_config{{{{}, "$default"}}, {}, {}}, + nullptr, feature_service, address_map, wct) {} static @@ -339,7 +377,7 @@ void messaging_service::do_start_listen() { bool listen_to_bc = _cfg.listen_on_broadcast_address && _cfg.ip != broadcast_address; rpc::server_options so; if (_cfg.compress != compress_what::none) { - so.compressor_factory = &compressor_factory; + so.compressor_factory = &_compressor_factory_wrapper->get_factory(); } so.load_balancing_algorithm = server_socket::load_balancing_algorithm::port; @@ -429,7 +467,7 @@ void messaging_service::do_start_listen() { } messaging_service::messaging_service(config cfg, scheduling_config scfg, std::shared_ptr credentials, gms::feature_service& feature_service, - gms::gossip_address_map& address_map) + gms::gossip_address_map& address_map, utils::walltime_compressor_tracker& arct) : _cfg(std::move(cfg)) , _rpc(new rpc_protocol_wrapper(serializer { })) , _credentials_builder(credentials ? std::make_unique(*credentials) : nullptr) @@ -438,6 +476,7 @@ messaging_service::messaging_service(config cfg, scheduling_config scfg, std::sh , _scheduling_config(scfg) , _scheduling_info_for_connection_index(initial_scheduling_info()) , _feature_service(feature_service) + , _compressor_factory_wrapper(std::make_unique(arct, _cfg.enable_advanced_rpc_compression)) , _address_map(address_map) { _rpc->set_logger(&rpc_logger); @@ -962,7 +1001,7 @@ shared_ptr messaging_service::ge // send keepalive messages each minute if connection is idle, drop connection after 10 failures opts.keepalive = std::optional({60s, 60s, 10}); if (must_compress) { - opts.compressor_factory = &compressor_factory; + opts.compressor_factory = &_compressor_factory_wrapper->get_factory(); } opts.tcp_nodelay = must_tcp_nodelay; opts.reuseaddr = true; diff --git a/message/messaging_service.hh b/message/messaging_service.hh index ce81a2025e82..a982ce912b31 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -24,6 +24,7 @@ #include "service/maintenance_mode.hh" #include "gms/gossip_address_map.hh" #include "tasks/types.hh" +#include "utils/advanced_rpc_compressor.hh" #include #include @@ -241,6 +242,7 @@ public: struct rpc_protocol_client_wrapper; struct rpc_protocol_server_wrapper; struct shard_info; + struct compressor_factory_wrapper; using msg_addr = netw::msg_addr; using inet_address = gms::inet_address; @@ -297,6 +299,7 @@ public: uint16_t ssl_port = 0; encrypt_what encrypt = encrypt_what::none; compress_what compress = compress_what::none; + bool enable_advanced_rpc_compression = false; tcp_nodelay_what tcp_nodelay = tcp_nodelay_what::all; bool listen_on_broadcast_address = false; size_t rpc_memory_limit = 1'000'000; @@ -350,6 +353,7 @@ private: std::vector _scheduling_info_for_connection_index; std::vector _connection_index_for_tenant; gms::feature_service& _feature_service; + std::unique_ptr _compressor_factory_wrapper; struct connection_ref; std::unordered_multimap _host_connections; @@ -365,8 +369,10 @@ private: public: using clock_type = lowres_clock; - messaging_service(locator::host_id id, gms::inet_address ip, uint16_t port, gms::feature_service& feature_service, gms::gossip_address_map& address_map); - messaging_service(config cfg, scheduling_config scfg, std::shared_ptr, gms::feature_service& feature_service, gms::gossip_address_map& address_map); + messaging_service(locator::host_id id, gms::inet_address ip, uint16_t port, + gms::feature_service&, gms::gossip_address_map&, utils::walltime_compressor_tracker&); + messaging_service(config cfg, scheduling_config scfg, std::shared_ptr, + gms::feature_service&, gms::gossip_address_map&, utils::walltime_compressor_tracker&); ~messaging_service(); future<> start(); diff --git a/service/storage_service.cc b/service/storage_service.cc index 62c5502cfdbb..f9bf6329174f 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -175,7 +175,9 @@ storage_service::storage_service(abort_source& abort_source, sharded& sl_controller, topology_state_machine& topology_state_machine, tasks::task_manager& tm, - gms::gossip_address_map& address_map) + gms::gossip_address_map& address_map, + std::function()> compression_dictionary_updated_callback + ) : _abort_source(abort_source) , _feature_service(feature_service) , _db(db) @@ -207,6 +209,7 @@ storage_service::storage_service(abort_source& abort_source, , _cdc_gens(cdc_gens) , _view_builder(view_builder) , _topology_state_machine(topology_state_machine) + , _compression_dictionary_updated_callback(std::move(compression_dictionary_updated_callback)) { tm.register_module(_node_ops_module->get_name(), _node_ops_module); tm.register_module(_tablets_module->get_name(), _tablets_module); diff --git a/service/storage_service.hh b/service/storage_service.hh index d21d9df70442..2b6579852857 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -232,7 +232,8 @@ public: sharded& sl_controller, topology_state_machine& topology_state_machine, tasks::task_manager& tm, - gms::gossip_address_map& address_map); + gms::gossip_address_map& address_map, + std::function()> compression_dictionary_updated_callback); ~storage_service(); node_ops::task_manager_module& get_node_ops_module() noexcept; diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index 140a283c39e6..cef4094bad2e 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -138,6 +138,7 @@ class single_node_cql_env : public cql_test_env { sharded _mnotifier; sharded _sl_controller; sharded _topology_state_machine; + sharded _compressor_tracker; sharded _mm; sharded _batchlog_manager; sharded _gossiper; @@ -699,6 +700,15 @@ class single_node_cql_env : public cql_test_env { _gossip_address_map.stop().get(); }); + auto arct_cfg = [&] { + return utils::advanced_rpc_compressor::tracker::config{ + .zstd_quota_fraction{1.0}, + .register_metrics = true, + }; + }; + _compressor_tracker.start(arct_cfg).get(); + auto stop_compressor_tracker = defer([this] { _compressor_tracker.stop().get(); }); + uint16_t port = 7000; seastar::server_socket tmp; @@ -719,7 +729,8 @@ class single_node_cql_env : public cql_test_env { port = tmp.local_address().port(); } // Don't start listening so tests can be run in parallel if cfg_in.ms_listen is not set to true explicitly. - _ms.start(host_id, listen, std::move(port), std::ref(_feature_service), std::ref(_gossip_address_map)).get(); + _ms.start(host_id, listen, std::move(port), std::ref(_feature_service), + std::ref(_gossip_address_map), std::ref(_compressor_tracker)).get(); stop_ms = defer(stop_type(stop_ms_func)); if (cfg_in.ms_listen) { @@ -824,6 +835,8 @@ class single_node_cql_env : public cql_test_env { abort_sources.local(), _group0_registry.local(), _ms, _gossiper.local(), _feature_service.local(), _sys_ks.local(), group0_client, scheduling_groups.gossip_scheduling_group}; + auto compression_dict_updated_callback = [] { return make_ready_future<>(); }; + _ss.start(std::ref(abort_sources), std::ref(_db), std::ref(_gossiper), std::ref(_sys_ks), @@ -842,7 +855,9 @@ class single_node_cql_env : public cql_test_env { std::ref(_sl_controller), std::ref(_topology_state_machine), std::ref(_task_manager), - std::ref(_gossip_address_map)).get(); + std::ref(_gossip_address_map), + compression_dict_updated_callback + ).get(); auto stop_storage_service = defer([this] { _ss.stop().get(); }); _mnotifier.local().register_listener(&_ss.local()); diff --git a/test/manual/gossip.cc b/test/manual/gossip.cc index a124aec8c1db..a0ee2fe31521 100644 --- a/test/manual/gossip.cc +++ b/test/manual/gossip.cc @@ -58,6 +58,7 @@ int main(int ac, char ** av) { sharded abort_sources; sharded token_metadata; + sharded compressor_tracker; sharded feature_service; sharded gossip_address_map; sharded messaging; @@ -71,13 +72,16 @@ int main(int ac, char ** av) { tm_cfg.topo_cfg.this_cql_address = my_address; token_metadata.start([] () noexcept { return db::schema_tables::hold_merge_lock(); }, tm_cfg).get(); auto stop_token_mgr = defer([&] { token_metadata.stop().get(); }); + compressor_tracker.start([] { return utils::walltime_compressor_tracker::config{}; }).get(); + auto stop_compressor_tracker = deferred_stop(compressor_tracker); auto cfg = gms::feature_config_from_db_config(db::config(), {}); feature_service.start(cfg).get(); gossip_address_map.start().get(); - messaging.start(locator::host_id{}, listen, 7000, std::ref(feature_service), std::ref(gossip_address_map)).get(); + messaging.start(locator::host_id{}, listen, 7000, std::ref(feature_service), + std::ref(gossip_address_map), std::ref(compressor_tracker)).get(); auto stop_messaging = deferred_stop(messaging); gms::gossip_config gcfg; diff --git a/test/manual/message.cc b/test/manual/message.cc index 66560e646c0d..73226d0b8f0b 100644 --- a/test/manual/message.cc +++ b/test/manual/message.cc @@ -193,13 +193,17 @@ int main(int ac, char ** av) { sharded token_metadata; token_metadata.start([] () noexcept { return db::schema_tables::hold_merge_lock(); }, tm_cfg).get(); auto stop_tm = deferred_stop(token_metadata); + seastar::sharded compressor_tracker; + compressor_tracker.start([] { return utils::walltime_compressor_tracker::config{}; }).get(); + auto stop_compressor_tracker = deferred_stop(compressor_tracker); seastar::sharded feature_service; auto cfg = gms::feature_config_from_db_config(db::config(), {}); feature_service.start(cfg).get(); seastar::sharded gossip_address_map; gossip_address_map.start().get(); seastar::sharded messaging; - messaging.start(locator::host_id{}, listen, 7000, std::ref(feature_service), std::ref(gossip_address_map)).get(); + messaging.start(locator::host_id{}, listen, 7000, std::ref(feature_service), + std::ref(gossip_address_map), std::ref(compressor_tracker)).get(); auto stop_messaging = deferred_stop(messaging); seastar::sharded testers; testers.start(std::ref(messaging)).get(); diff --git a/test/topology_custom/test_rpc_compression.py b/test/topology_custom/test_rpc_compression.py new file mode 100644 index 000000000000..a56a9dd2dde5 --- /dev/null +++ b/test/topology_custom/test_rpc_compression.py @@ -0,0 +1,258 @@ +# +# Copyright (C) 2023-present ScyllaDB +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# +""" +Test RPC compression +""" +from test.pylib.internal_types import ServerInfo +from test.pylib.rest_client import ScyllaMetrics +from test.pylib.util import wait_for_cql_and_get_hosts, unique_name +from test.pylib.manager_client import ManagerClient +import pytest +import asyncio +import time +import logging +import random +from cassandra.cluster import ConsistencyLevel +from cassandra.query import SimpleStatement +import contextlib +import typing +import functools + +logger = logging.getLogger(__name__) + +async def live_update_config(manager: ManagerClient, servers: list[ServerInfo], key: str, value: str): + cql = manager.get_cql() + hosts = await wait_for_cql_and_get_hosts(cql, servers, deadline = time.time() + 60) + await asyncio.gather(*[cql.run_async("UPDATE system.config SET value=%s WHERE name=%s", [value, key], host=host) for host in hosts]) + +def uncompressed_sent(metrics: list[ScyllaMetrics], algo: str) -> float: + return sum([m.get("scylla_rpc_compression_bytes_sent", {"algorithm": algo}) for m in metrics]) +def compressed_sent(metrics: list[ScyllaMetrics], algo: str) -> float: + return sum([m.get("scylla_rpc_compression_compressed_bytes_sent", {"algorithm": algo}) for m in metrics]) +def approximately_equal(a: float, b: float, factor: float) -> bool: + assert factor < 1.0 + return factor < a / b < (1/factor) +async def get_metrics(manager: ManagerClient, servers: list[ServerInfo]) -> list[ScyllaMetrics]: + return await asyncio.gather(*[manager.metrics.query(s.ip_addr) for s in servers]) + +async def with_retries(test_once: typing.Callable[[], typing.Awaitable], timeout: float): + async with asyncio.timeout(timeout): + while True: + try: + await test_once() + except Exception as e: + logger.info(f"test attempt failed with {e}, retrying") + await asyncio.sleep(1) + else: + break + +@pytest.mark.asyncio +async def test_basic(manager: ManagerClient) -> None: + """Tests basic functionality of internode compression. + Also, tests that changing internode_compression_zstd_max_cpu_fraction from 0.0 to 1.0 enables zstd as expected. + """ + cfg = { + 'internode_compression_enable_advanced': True, + 'internode_compression': "all", + 'internode_compression_zstd_max_cpu_fraction': 0.0} + logger.info(f"Booting initial cluster") + servers = await manager.servers_add(servers_num=2, config=cfg) + + cql = manager.get_cql() + + replication_factor = 2 + ks = unique_name() + await cql.run_async(f"create keyspace {ks} with replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {replication_factor}}}") + await cql.run_async(f"create table {ks}.cf (pk int, v blob, primary key (pk))") + write_stmt = cql.prepare(f"update {ks}.cf set v = ? where pk = ?") + write_stmt.consistency_level = ConsistencyLevel.ALL + + # 128 kiB message, should give compression ratio of ~0.5 for lz4 and ~0.25 for zstd. + message = b''.join(bytes(random.choices(range(16), k=1024)) * 2 for _ in range(64)) + + async def test_algo(algo: str, expected_ratio: float): + n_messages = 100 + metrics_before = await get_metrics(manager, servers) + await asyncio.gather(*[cql.run_async(write_stmt, parameters=[message, pk]) for pk in range(n_messages)]) + metrics_after = await get_metrics(manager, servers) + + volume = n_messages * len(message) * (replication_factor - 1) + uncompressed = uncompressed_sent(metrics_after, algo) - uncompressed_sent(metrics_before, algo) + compressed = compressed_sent(metrics_after, algo) - compressed_sent(metrics_before, algo) + assert approximately_equal(uncompressed, volume, 0.9) + assert approximately_equal(compressed, expected_ratio * volume, 0.9) + + await with_retries(functools.partial(test_algo, "lz4", 0.5), timeout=600) + + await live_update_config(manager, servers, "internode_compression_zstd_max_cpu_fraction", "1.0") + + await with_retries(functools.partial(test_algo, "zstd", 0.25), timeout=600) + +@pytest.mark.asyncio +async def test_dict_training(manager: ManagerClient) -> None: + """Tests population of system.dicts with dicts trained on RPC traffic.""" + training_min_bytes = 128*1024 + cfg = { + 'internode_compression_enable_advanced': True, + 'internode_compression': "all", + 'internode_compression_zstd_max_cpu_fraction': 0.0, + 'rpc_dict_training_when': 'never', + 'rpc_dict_training_min_bytes': training_min_bytes, + 'rpc_dict_training_min_time_seconds': 0, + } + cmdline = [ + '--logger-log-level=dict_training=trace' + ] + logger.info(f"Booting initial cluster") + servers = await manager.servers_add(servers_num=2, config=cfg, cmdline=cmdline) + + cql = manager.get_cql() + + replication_factor = 2 + ks = unique_name() + await cql.run_async(f"create keyspace {ks} with replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {replication_factor}}}") + await cql.run_async(f"create table {ks}.cf (pk int, v blob, primary key (pk))") + write_stmt = cql.prepare(f"update {ks}.cf set v = ? where pk = ?") + dict_stmt = cql.prepare(f"select data from system.dicts") + write_stmt.consistency_level = ConsistencyLevel.ALL + + msg_size = 16*1024 + msg_train = random.randbytes(msg_size) + msg_notrain = random.randbytes(msg_size) + + async def write_messages(m: bytes): + n_messages = 100 + assert n_messages * len(m) > training_min_bytes + await asyncio.gather(*[cql.run_async(write_stmt, parameters=[m, pk]) for pk in range(n_messages)]) + + async def set_dict_training_when(x: str): + await live_update_config(manager, servers, "rpc_dict_training_when", x) + + await write_messages(msg_notrain) + await set_dict_training_when("when_leader") + await write_messages(msg_train) + await set_dict_training_when("never") + await write_messages(msg_notrain) + await set_dict_training_when("when_leader") + await write_messages(msg_train) + + ngram_size = 8 + def make_ngrams(x: bytes) -> list[bytes]: + return [x[i:i+ngram_size] for i in range(len(x) - ngram_size)] + msg_train_ngrams = set(make_ngrams(msg_train)) + msg_notrain_ngrams = set(make_ngrams(msg_notrain)) + + async def test_once() -> None: + results = await cql.run_async(dict_stmt) + dicts = [bytes(x[0]) for x in results] + dict_ngrams = set(make_ngrams(bytes().join(dicts))) + assert len(msg_train_ngrams & dict_ngrams) > 0.5 * len(msg_train_ngrams) + assert len(msg_notrain_ngrams & dict_ngrams) < 0.5 * len(msg_notrain_ngrams) + + await with_retries(test_once, timeout=600) + +@pytest.mark.asyncio +async def test_external_dicts(manager: ManagerClient) -> None: + """Tests internode compression with external dictionaries""" + cfg = { + 'internode_compression_enable_advanced': True, + 'internode_compression': "all", + 'internode_compression_zstd_max_cpu_fraction': 0.0, + 'rpc_dict_training_when': 'when_leader', + 'rpc_dict_training_min_bytes': 10000000, + 'rpc_dict_training_min_time_seconds': 0, + } + cmdline = [ + '--logger-log-level=dict_training=trace', + '--logger-log-level=advanced_rpc_compressor=debug' + ] + logger.info(f"Booting initial cluster") + servers = await manager.servers_add(servers_num=2, config=cfg, cmdline=cmdline) + + cql = manager.get_cql() + + replication_factor = 2 + ks = unique_name() + await cql.run_async(f"create keyspace {ks} with replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {replication_factor}}}") + await cql.run_async(f"create table {ks}.cf (pk int, v blob, primary key (pk))") + write_stmt = cql.prepare(f"update {ks}.cf set v = ? where pk = ?") + write_stmt.consistency_level = ConsistencyLevel.ALL + + msg_size = 32*1024 + ngram_size = 64 + common_ngrams = [random.randbytes(ngram_size) for _ in range(msg_size//2//ngram_size)] + + # 128 kiB messages, should give compression ratio of ~0.5 for lz4 and ~0.25 for zstd + # when compressed with a common dictionary. + def make_message() -> bytes: + common_part = b''.join(random.sample(common_ngrams, k=msg_size//2//ngram_size)) + assert len(common_part) == msg_size // 2 + unique_part = bytes(random.choices(range(16), k=msg_size//2)) + assert len(unique_part) == msg_size // 2 + return common_part + unique_part + + async def test_once(algo: str, expected_ratio: float): + n_messages = 1000 + metrics_before = await get_metrics(manager, servers) + messages = [make_message() for _ in range(n_messages)] + await asyncio.gather(*[cql.run_async(write_stmt, parameters=[m, pk]) for pk, m in enumerate(messages)]) + metrics_after = await get_metrics(manager, servers) + + volume = sum(len(m) for m in messages) * (replication_factor - 1) + uncompressed = uncompressed_sent(metrics_after, algo) - uncompressed_sent(metrics_before, algo) + compressed = compressed_sent(metrics_after, algo) - compressed_sent(metrics_before, algo) + assert approximately_equal(uncompressed, volume, 0.8) + assert approximately_equal(compressed, expected_ratio * volume, 0.8) + + await with_retries(functools.partial(test_once, "lz4", 0.5), timeout=600) + + await live_update_config(manager, servers, "internode_compression_zstd_max_cpu_fraction", "1.0"), + + await with_retries(functools.partial(test_once, "zstd", 0.25), timeout=600) + +# Similar to test_external_dicts, but simpler. +@pytest.mark.asyncio +async def test_external_dicts_sanity(manager: ManagerClient) -> None: + """Tests internode compression with external dictionaries, by spamming the same UPDATE statement.""" + cfg = { + 'internode_compression_enable_advanced': True, + 'internode_compression': "all", + 'internode_compression_zstd_max_cpu_fraction': 0.0, + 'rpc_dict_training_when': 'when_leader', + 'rpc_dict_training_min_bytes': 10000000, + 'rpc_dict_training_min_time_seconds': 0, + } + cmdline = [ + '--logger-log-level=dict_training=trace', + '--logger-log-level=advanced_rpc_compressor=debug', + ] + logger.info(f"Booting initial cluster") + servers = await manager.servers_add(servers_num=2, config=cfg, cmdline=cmdline) + + cql = manager.get_cql() + + replication_factor = 2 + ks = unique_name() + await cql.run_async(f"create keyspace {ks} with replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {replication_factor}}}") + await cql.run_async(f"create table {ks}.cf (pk int, v blob, primary key (pk))") + write_stmt = cql.prepare(f"update {ks}.cf set v = ? where pk = ?") + write_stmt.consistency_level = ConsistencyLevel.ALL + + msg = random.randbytes(8192) + + async def test_algo(algo: str, expected_ratio): + n_messages = 1000 + metrics_before = await get_metrics(manager, servers) + await asyncio.gather(*[cql.run_async(write_stmt, parameters=[msg, pk]) for pk in range(n_messages)]) + metrics_after = await get_metrics(manager, servers) + + volume = len(msg) * n_messages * (replication_factor - 1) + uncompressed = uncompressed_sent(metrics_after, algo) - uncompressed_sent(metrics_before, algo) + compressed = compressed_sent(metrics_after, algo) - compressed_sent(metrics_before, algo) + assert approximately_equal(uncompressed, volume, 0.8) + assert compressed < expected_ratio * uncompressed + + await with_retries(functools.partial(test_algo, "lz4", 0.04), timeout=600) From 2b1ba9c3fddd2604f1a2a0db88d2d4dbba5055b7 Mon Sep 17 00:00:00 2001 From: Konstantin Osipov Date: Thu, 26 Dec 2024 09:04:49 -0500 Subject: [PATCH 121/397] test.py: rethrow CancelledError when executing a test Commit 870f3b00fcfe05c733d59f67d0d4af834aebc9bb, "Add option to fail after number of failures" adds tracking on the number of cancelled tests. For the purpose, it intercepts CancelledError and sets test's is_cancelled flag. This introduced a regression reported in gh-21636: Ctrl-C no longer works, since CancelledError is muted. There was no intent to mute the exception, re-throw it after accounting the test as cancelled. --- test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/test.py b/test.py index a1e9269544ec..91dd4ce05314 100755 --- a/test.py +++ b/test.py @@ -246,6 +246,7 @@ async def run(self, test: 'Test', options: argparse.Namespace): break except asyncio.CancelledError: test.is_cancelled = True + raise finally: self.pending_test_count -= 1 self.n_failed += int(test.failed) From 8b7a5ca88de3fe2bfe30e156c3c4771b48133d13 Mon Sep 17 00:00:00 2001 From: Konstantin Osipov Date: Fri, 27 Dec 2024 14:54:13 -0500 Subject: [PATCH 122/397] test.py: only access combined_tests if it exists When the scylla source tree is only partially built, we still may want to run the tests. test.py builds a case cache at boot, and executes --list-cases for that, for all built tests. After amalgamating boost unit tests into a single file, it started running it unconditionally, which broke partial builds. Hence, only use combined_tests executable if it exists. Fixes #22038 --- test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test.py b/test.py index 91dd4ce05314..e9a4868e71bc 100755 --- a/test.py +++ b/test.py @@ -373,7 +373,8 @@ class BoostTestSuite(UnitTestSuite): def _generate_cache(self) -> None: # Apply combined test only for test/boost - if self.name != 'boost': + exe_path = pathlib.Path(self.mode, "test", self.name, 'combined_tests') + if self.name != 'boost' or not exe_path.exists(): return exe = path_to(self.mode, "test", self.name, 'combined_tests') res = subprocess.run( From 2701b5d50d57e6ff04daab063fb53b615006da68 Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Thu, 26 Dec 2024 16:17:49 +0200 Subject: [PATCH 123/397] cql3: allow set subscript This allows to use subscript on a set column, in addition to map/list which was possible until now. The behavior is compatible with Cassandra - a subscript with a specific value returns the value if it's found in the set, and null otherwise. --- cql3/expr/expression.cc | 11 ++++++++++- cql3/expr/prepare_expr.cc | 12 ++++++++++-- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/cql3/expr/expression.cc b/cql3/expr/expression.cc index f79ce6ddfff5..c4f0d9d42f85 100644 --- a/cql3/expr/expression.cc +++ b/cql3/expr/expression.cc @@ -141,7 +141,7 @@ get_value(const subscript& s, const evaluation_inputs& inputs) { auto col_type = static_pointer_cast(type_of(s.val)); const auto deserialized = type_of(s.val)->deserialize(managed_bytes_view(*serialized)); const auto key = evaluate(s.sub, inputs); - auto&& key_type = col_type->is_map() ? col_type->name_comparator() : int32_type; + auto&& key_type = col_type->is_list() ? int32_type : col_type->name_comparator(); if (key.is_null()) { // For m[null] return null. // This is different from Cassandra - which treats m[null] @@ -162,6 +162,15 @@ get_value(const subscript& s, const evaluation_inputs& inputs) { }); }); return found == data_map.cend() ? std::nullopt : managed_bytes_opt(found->second.serialize_nonnull()); + } else if (col_type->is_set()) { + const auto& data_set = value_cast(deserialized); + const auto found = key.view().with_linearized([&] (bytes_view key_bv) { + using entry = data_value; + return std::find_if(data_set.cbegin(), data_set.cend(), [&] (const entry& element) { + return key_type->compare(element.serialize_nonnull(), key_bv) == 0; + }); + }); + return found == data_set.cend() ? std::nullopt : managed_bytes_opt(found->serialize_nonnull()); } else if (col_type->is_list()) { const auto& data_list = value_cast(deserialized); auto key_deserialized = key.view().with_linearized([&] (bytes_view key_bv) { diff --git a/cql3/expr/prepare_expr.cc b/cql3/expr/prepare_expr.cc index 7e3f52484a0b..f9fc20b7fe5c 100644 --- a/cql3/expr/prepare_expr.cc +++ b/cql3/expr/prepare_expr.cc @@ -1191,18 +1191,24 @@ try_prepare_expression(const expression& expr, data_dictionary::database db, con auto col_spec = column_specification_of(sub_col); lw_shared_ptr subscript_column_spec; + data_type value_cmp; if (sub_col_type.is_map()) { subscript_column_spec = map_key_spec_of(*col_spec); + value_cmp = static_cast(sub_col_type).value_comparator(); + } else if (sub_col_type.is_set()) { + subscript_column_spec = set_value_spec_of(*col_spec); + value_cmp = static_cast(sub_col_type).name_comparator(); } else if (sub_col_type.is_list()) { subscript_column_spec = list_key_spec_of(*col_spec); + value_cmp = static_cast(sub_col_type).value_comparator(); } else { - throw exceptions::invalid_request_exception(format("Column {} is not a map/list, cannot be subscripted", col_spec->name->text())); + throw exceptions::invalid_request_exception(format("Column {} is not a map/set/list, cannot be subscripted", col_spec->name->text())); } return subscript { .val = sub_col, .sub = prepare_expression(sub.sub, db, schema.ks_name(), &schema, std::move(subscript_column_spec)), - .type = static_cast(sub_col_type).value_comparator(), + .type = value_cmp, }; }, [&] (const unresolved_identifier& unin) -> std::optional { @@ -1388,6 +1394,8 @@ static lw_shared_ptr get_lhs_receiver(const expression& pr const column_value& sub_col = get_subscripted_column(col_val); if (sub_col.col->type->is_map()) { return map_value_spec_of(*sub_col.col->column_specification); + } else if (sub_col.col->type->is_set()) { + return set_value_spec_of(*sub_col.col->column_specification); } else { return list_value_spec_of(*sub_col.col->column_specification); } From 6281fb825f8b47db57d440482ccc39a6de80c7c9 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Fri, 27 Dec 2024 17:32:36 +0800 Subject: [PATCH 124/397] test/pytest.ini: ignore warning on deprecated record_property fixture `record_property` generates XML which is not compatible with xunit2, so pytest decided to deprecated when the generating xunit reports. and pytest generates following warning when a test failure is reported using this fixture: ``` object_store/test_backup.py:337: PytestWarning: record_property is incompatible with junit_family 'xunit2' (use 'legacy' or 'xunit1') ``` this warning is not related to the test, but more about how we report a failure using pytrest. it is distracting, so let's silence it. See also https://github.com/pytest-dev/pytest/issues/5202 Signed-off-by: Kefu Chai Closes scylladb/scylladb#22067 --- test/pytest.ini | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/test/pytest.ini b/test/pytest.ini index f2f5a4580262..4f1c9fca712e 100644 --- a/test/pytest.ini +++ b/test/pytest.ini @@ -14,8 +14,18 @@ norecursedirs = manual perf lib # Ignore warnings about HTTPS requests without certificate verification # (see issue #15287). Pytest breaks urllib3.disable_warnings() in conftest.py, # so we need to do this here. +# +# Ignore warning of +# PytestWarning: record_property is incompatible with junit_family 'xunit2' (use 'legacy' or 'xunit1') +# Because `record_property` adds inside , which is not allowed +# as per the latest xunit2 schema. see +# https://github.com/windyroad/JUnit-Schema/blob/master/JUnit.xsdtestcase, +# an alternative is `record_testsuite_property`, but we want to attach test +# log on a per-test basis. so let's continue using this feature before +# switching to xunit1 or legacy. filterwarnings = ignore::urllib3.exceptions.InsecureRequestWarning + ignore:record_property is incompatible with junit_family:pytest.PytestWarning tmp_path_retention_count = 1 tmp_path_retention_policy = failed From 2352063f20965830f19cc1dd31319a79dd5d5ceb Mon Sep 17 00:00:00 2001 From: Piotr Smaron Date: Wed, 4 Dec 2024 10:26:13 +0100 Subject: [PATCH 125/397] server: set `connection_stage` to READY when authenticated MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If authentication is enabled, but STARTUP isn't followed by REGISTER (which is optional, and in practice only happens on only one of a driver's connections — because there's no point listening for the same events on multiple connections), connections are wrongly displayed in the system.clients as AUTHENTICATING instead of READY, even when they are ready. This commit fixes this problem. Fixes: scylladb/scylladb#12640 Closes scylladb/scylladb#21774 --- test/cqlpy/test_permissions.py | 21 ++++++++++++++++++++- transport/server.cc | 2 ++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/test/cqlpy/test_permissions.py b/test/cqlpy/test_permissions.py index 8e0e98a849f1..febbad558fdf 100644 --- a/test/cqlpy/test_permissions.py +++ b/test/cqlpy/test_permissions.py @@ -81,6 +81,25 @@ def check_enforced(cql, username, permission, resource, function): revoke(cql, permission, resource, username) eventually_unauthorized(function) +def test_user_displays_as_authenticated(cql): + with new_session(cql, "cassandra") as user_session_1: + with new_session(cql, "cassandra") as user_session_2: + with new_session(cql, "cassandra") as user_session_3: + # Every opened session should create around 5 connections, + # so we'll have 3x5=15 opened connections, + # and we expect that their status will be READY, + # but, on rare occasions, it may happen that + # the connection's status is still stuck at ESTABLISHED or AUTHENTICATING, + # and we must handle this case as well, + # so we simply sleep and retry later. + for retry in range(0, 5): + res = user_session_1.execute("SELECT connection_stage FROM system.clients") + if all([r[0] == "READY" for r in res]): + return + else: + time.sleep(2) + assert False + # Test that data permissions can be granted and revoked, and that they're effective def test_grant_revoke_data_permissions(cql, test_keyspace): with new_user(cql) as username: @@ -540,7 +559,7 @@ def ensure_unauthorized(fun): # every user with SELECT permission for a table should be able to use # the native functions (non UDF/UDA functions) # ref: https://github.com/scylladb/scylladb/issues/16526 -def test_native_functions_always_exeutable(cql): +def test_native_functions_always_executable(cql): schema = "a int primary key" with new_test_keyspace(cql,"WITH REPLICATION = { 'class': 'NetworkTopologyStrategy', 'replication_factor': 1 }") as keyspace: with new_test_table(cql,keyspace,schema) as table: diff --git a/transport/server.cc b/transport/server.cc index e4e863b6c077..64690029c238 100644 --- a/transport/server.cc +++ b/transport/server.cc @@ -946,6 +946,8 @@ future> cql_server::connection::process_au return client_state.maybe_update_per_service_level_params(); }); return f.then([this, stream, challenge = std::move(challenge), trace_state]() mutable { + _authenticating = false; + _ready = true; return make_ready_future>(make_auth_success(stream, std::move(challenge), trace_state)); }); }); From 6f11edbf3f481787cf6245935d513c9b2c828c44 Mon Sep 17 00:00:00 2001 From: Wojciech Mitros Date: Tue, 10 Dec 2024 00:30:57 +0100 Subject: [PATCH 126/397] db: set base info before adding schema to registry In the following patches, we'll assure that view schemas returned by the schema registry always have base info set. To prepare for that, make sure that the base info is always set before inserting it into schema registry, --- replica/database.cc | 23 ++++++++++++++++++----- replica/table.cc | 2 -- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/replica/database.cc b/replica/database.cc index 87da7731d3c4..1e4af6ce92bf 100644 --- a/replica/database.cc +++ b/replica/database.cc @@ -914,6 +914,14 @@ db::commitlog* database::commitlog_for(const schema_ptr& schema) { } future<> database::add_column_family(keyspace& ks, schema_ptr schema, column_family::config cfg, is_new_cf is_new) { + if (schema->is_view()) { + try { + auto base_schema = find_schema(schema->view_info()->base_id()); + schema->view_info()->set_base_info(schema->view_info()->make_base_dependent_view_info(*base_schema)); + } catch (no_such_column_family&) { + throw std::invalid_argument("The base table " + schema->view_info()->base_name() + " was already dropped"); + } + } schema = local_schema_registry().learn(schema); schema->registry_entry()->mark_synced(); auto&& rs = ks.get_replication_strategy(); @@ -958,6 +966,14 @@ future<> database::add_column_family_and_make_directory(schema_ptr schema, is_ne } bool database::update_column_family(schema_ptr new_schema) { + if (new_schema->is_view()) { + try { + auto base_schema = find_schema(new_schema->view_info()->base_id()); + new_schema->view_info()->set_base_info(new_schema->view_info()->make_base_dependent_view_info(*base_schema)); + } catch (no_such_column_family&) { + throw std::invalid_argument("The base table " + new_schema->view_info()->base_name() + " was already dropped"); + } + } column_family& cfm = find_column_family(new_schema->id()); bool columns_changed = !cfm.schema()->equal_columns(*new_schema); auto s = local_schema_registry().learn(new_schema); @@ -965,11 +981,8 @@ bool database::update_column_family(schema_ptr new_schema) { cfm.set_schema(s); find_keyspace(s->ks_name()).metadata()->add_or_update_column_family(s); if (s->is_view()) { - try { - find_column_family(s->view_info()->base_id()).add_or_update_view(view_ptr(s)); - } catch (no_such_column_family&) { - // Update view mutations received after base table drop. - } + // We already tested that the base table exists + find_column_family(s->view_info()->base_id()).add_or_update_view(view_ptr(s)); } cfm.get_index_manager().reload(); return columns_changed; diff --git a/replica/table.cc b/replica/table.cc index 5344472cd527..e71638fd0801 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -3223,8 +3223,6 @@ static std::vector::iterator find_view(std::vector& views, c } void table::add_or_update_view(view_ptr v) { - v->view_info()->set_base_info( - v->view_info()->make_base_dependent_view_info(*_schema)); auto existing = find_view(_views, v); if (existing != _views.end()) { *existing = std::move(v); From dfe3810f64ca0e4669239edfadcefbb1f3acf535 Mon Sep 17 00:00:00 2001 From: Wojciech Mitros Date: Tue, 10 Dec 2024 02:19:40 +0100 Subject: [PATCH 127/397] schema_registry: cache base schemas for views Currently, when we load a frozen schema into the registry, we lose the base info if the schema was of a view. Because of that, in various places we need to set the base info again, and in some codepaths we may miss it completely, which may make us unable to process some requests (for example, when executing reverse queries on views). Even after setting the base info, we may still lose it if the schema entry gets deactivated. To fix this, this patch adds the base schema to the registry, alongside the view schema. With the base schema, we can now set the base info when returning the schema from the registry. As a result, we can now assume that all view schemas returned by the registry have base_info set. To store the base schema, the loader methods now have to return the base schema alongside the view schema. At the same time, when loading into the registry, we need to check whether we're loading a view schema, and if so, we need to also provide the base schema. When inserting a regular table schema, the base schema should be a disengaged optional. --- schema/schema.cc | 12 ++++++-- schema/schema_registry.cc | 45 +++++++++++++++++++----------- schema/schema_registry.hh | 17 +++++++++-- service/migration_manager.cc | 11 ++++++-- test/boost/schema_registry_test.cc | 26 ++++++++--------- 5 files changed, 73 insertions(+), 38 deletions(-) diff --git a/schema/schema.cc b/schema/schema.cc index 204c5654b701..a305f01546f3 100644 --- a/schema/schema.cc +++ b/schema/schema.cc @@ -1940,8 +1940,16 @@ schema_ptr schema::make_reversed() const { } schema_ptr schema::get_reversed() const { - return local_schema_registry().get_or_load(reversed(_raw._version), [this] (table_schema_version) { - return frozen_schema(make_reversed()); + return local_schema_registry().get_or_load(reversed(_raw._version), [this] (table_schema_version) -> base_and_view_schemas { + auto s = make_reversed(); + + if (s->is_view()) { + if (!s->view_info()->base_info()) { + on_internal_error(dblog, format("Tried to make a reverse schema for view {}.{} with an uninitialized base info", s->ks_name(), s->cf_name())); + } + return {frozen_schema(s), s->view_info()->base_info()->base_schema()}; + } + return {frozen_schema(s)}; }); } diff --git a/schema/schema_registry.cc b/schema/schema_registry.cc index 0bbd412a89d2..0920e4c94f32 100644 --- a/schema/schema_registry.cc +++ b/schema/schema_registry.cc @@ -170,8 +170,11 @@ void schema_registry::clear() { _entries.clear(); } -schema_ptr schema_registry_entry::load(frozen_schema fs) { - _frozen_schema = std::move(fs); +schema_ptr schema_registry_entry::load(base_and_view_schemas fs) { + _frozen_schema = std::move(fs.schema); + if (fs.base_schema) { + _base_schema = std::move(fs.base_schema); + } auto s = get_schema(); if (_state == state::LOADING) { _schema_promise.set_value(s); @@ -184,6 +187,9 @@ schema_ptr schema_registry_entry::load(frozen_schema fs) { schema_ptr schema_registry_entry::load(schema_ptr s) { _frozen_schema = frozen_schema(s); + if (s->is_view()) { + _base_schema = s->view_info()->base_info()->base_schema(); + } _schema = &*s; _schema->_registry_entry = this; _erase_timer.cancel(); @@ -203,7 +209,7 @@ future schema_registry_entry::start_loading(async_schema_loader load _state = state::LOADING; slogger.trace("Loading {}", _version); // Move to background. - (void)f.then_wrapped([self = shared_from_this(), this] (future&& f) { + (void)f.then_wrapped([self = shared_from_this(), this] (future&& f) { _loader = {}; if (_state != state::LOADING) { slogger.trace("Loading of {} aborted", _version); @@ -232,6 +238,10 @@ schema_ptr schema_registry_entry::get_schema() { if (s->version() != _version) { throw std::runtime_error(format("Unfrozen schema version doesn't match entry version ({}): {}", _version, *s)); } + if (s->is_view()) { + // We may encounter a no_such_column_family here, which means that the base table was deleted and we should fail the request + s->view_info()->set_base_info(s->view_info()->make_base_dependent_view_info(**_base_schema)); + } _erase_timer.cancel(); s->_registry_entry = this; _schema = &*s; @@ -325,17 +335,17 @@ schema_ptr global_schema_ptr::get() const { if (this_shard_id() == _cpu_of_origin) { return _ptr; } else { - auto registered_schema = [](const schema_registry_entry& e) { + auto registered_schema = [](const schema_registry_entry& e, std::optional base_schema = std::nullopt) -> schema_ptr { schema_ptr ret = local_schema_registry().get_or_null(e.version()); if (!ret) { - ret = local_schema_registry().get_or_load(e.version(), [&e](table_schema_version) { - return e.frozen(); + ret = local_schema_registry().get_or_load(e.version(), [&e, &base_schema](table_schema_version) -> base_and_view_schemas { + return {e.frozen(), base_schema}; }); } return ret; }; - schema_ptr registered_bs; + std::optional registered_bs; // the following code contains registry entry dereference of a foreign shard // however, it is guaranteed to succeed since we made sure in the constructor // that _bs_schema and _ptr will have a registry on the foreign shard where this @@ -344,16 +354,10 @@ schema_ptr global_schema_ptr::get() const { if (_base_schema) { registered_bs = registered_schema(*_base_schema->registry_entry()); if (_base_schema->registry_entry()->is_synced()) { - registered_bs->registry_entry()->mark_synced(); - } - } - schema_ptr s = registered_schema(*_ptr->registry_entry()); - if (s->is_view()) { - if (!s->view_info()->base_info()) { - // we know that registered_bs is valid here because we make sure of it in the constructors. - s->view_info()->set_base_info(s->view_info()->make_base_dependent_view_info(*registered_bs)); + registered_bs.value()->registry_entry()->mark_synced(); } } + schema_ptr s = registered_schema(*_ptr->registry_entry(), registered_bs); if (_ptr->registry_entry()->is_synced()) { s->registry_entry()->mark_synced(); } @@ -370,8 +374,15 @@ global_schema_ptr::global_schema_ptr(const schema_ptr& ptr) if (e) { return s; } else { - return local_schema_registry().get_or_load(s->version(), [&s] (table_schema_version) { - return frozen_schema(s); + return local_schema_registry().get_or_load(s->version(), [&s] (table_schema_version) -> base_and_view_schemas { + if (s->is_view()) { + if (!s->view_info()->base_info()) { + on_internal_error(slogger, format("Tried to build a global schema for view {}.{} with an uninitialized base info", s->ks_name(), s->cf_name())); + } + return {frozen_schema(s), s->view_info()->base_info()->base_schema()}; + } else { + return {frozen_schema(s)}; + } }); } }; diff --git a/schema/schema_registry.hh b/schema/schema_registry.hh index bf925e95d3db..db65ddb9bb31 100644 --- a/schema/schema_registry.hh +++ b/schema/schema_registry.hh @@ -22,8 +22,13 @@ class schema_ctxt; class schema_registry; -using async_schema_loader = std::function(table_schema_version)>; -using schema_loader = std::function; +struct base_and_view_schemas { + frozen_schema schema; + std::optional base_schema; +}; + +using async_schema_loader = std::function(table_schema_version)>; +using schema_loader = std::function; class schema_version_not_found : public std::runtime_error { public: @@ -61,6 +66,8 @@ class schema_registry_entry : public enable_lw_shared_from_this _schema_promise; // valid when state == LOADING std::optional _frozen_schema; // engaged when state == LOADED + std::optional _base_schema;// engaged when state == LOADED for view schemas + // valid when state == LOADED // This is != nullptr when there is an alive schema_ptr associated with this entry. const ::schema* _schema = nullptr; @@ -77,7 +84,7 @@ public: schema_registry_entry(schema_registry_entry&&) = delete; schema_registry_entry(const schema_registry_entry&) = delete; ~schema_registry_entry(); - schema_ptr load(frozen_schema); + schema_ptr load(base_and_view_schemas); schema_ptr load(schema_ptr); future start_loading(async_schema_loader); schema_ptr get_schema(); // call only when state >= LOADED @@ -108,6 +115,7 @@ public: // alive the registry will keep its entry. To ensure remote nodes can query current node // for schema version, make sure that schema_ptr for the request is alive around the call. // +// Schemas of views returned by this registry always have base_info set. class schema_registry { std::unordered_map> _entries; std::unique_ptr _ctxt; @@ -125,6 +133,7 @@ public: void init(const db::schema_ctxt&); // Looks up schema by version or loads it using supplied loader. + // If the schema refers to a view, the loader must return both view and base schemas. schema_ptr get_or_load(table_schema_version, const schema_loader&); // Looks up schema by version or returns an empty pointer if not available. @@ -134,6 +143,7 @@ public: // deferring. The loader is copied must be alive only until this method // returns. If the loader fails, the future resolves with // schema_version_loading_failed. + // If the schema refers to a view, the loader must return both view and base schemas. future get_or_load(table_schema_version, const async_schema_loader&); // Looks up schema version. Throws schema_version_not_found when not found @@ -149,6 +159,7 @@ public: // the schema which was passed as argument. // The schema instance pointed to by the argument will be attached to the registry // entry and will keep it alive. + // If the schema refers to a view, it must have base_info set. schema_ptr learn(const schema_ptr&); // Removes all entries from the registry. This in turn removes all dependencies diff --git a/service/migration_manager.cc b/service/migration_manager.cc index 21d0e730c5b0..4326e246095e 100644 --- a/service/migration_manager.cc +++ b/service/migration_manager.cc @@ -1058,13 +1058,18 @@ static future get_schema_definition(table_schema_version v, locator: // with TTL (refresh TTL in case column mapping already existed prior to that). auto us = s.unfreeze(db::schema_ctxt(proxy)); // if this is a view - sanity check that its schema doesn't need fixing. + schema_ptr base_schema; if (us->is_view()) { auto& db = proxy.local().local_db(); - schema_ptr base_schema = db.find_schema(us->view_info()->base_id()); + base_schema = db.find_schema(us->view_info()->base_id()); db::schema_tables::check_no_legacy_secondary_index_mv_schema(db, view_ptr(us), base_schema); } - return db::schema_tables::store_column_mapping(proxy, us, true).then([us] { - return frozen_schema{us}; + return db::schema_tables::store_column_mapping(proxy, us, true).then([us, base_schema] -> base_and_view_schemas { + if (us->is_view()) { + return {frozen_schema(us), base_schema}; + } else { + return {frozen_schema(us)}; + } }); }); }).then([&storage_proxy] (schema_ptr s) { diff --git a/test/boost/schema_registry_test.cc b/test/boost/schema_registry_test.cc index 49f2726dced0..9957726c29bd 100644 --- a/test/boost/schema_registry_test.cc +++ b/test/boost/schema_registry_test.cc @@ -58,7 +58,7 @@ SEASTAR_THREAD_TEST_CASE(test_load_with_non_nantive_type) { .build(); local_schema_registry().get_or_load(s->version(), [s] (table_schema_version) { - return make_ready_future(frozen_schema(s)); + return make_ready_future(frozen_schema(s)); }).get(); } @@ -69,7 +69,7 @@ SEASTAR_TEST_CASE(test_async_loading) { auto s2 = random_schema(); auto s1_loaded = local_schema_registry().get_or_load(s1->version(), [s1] (table_schema_version) { - return make_ready_future(frozen_schema(s1)); + return make_ready_future(frozen_schema(s1)); }).get(); BOOST_REQUIRE(s1_loaded); @@ -78,7 +78,7 @@ SEASTAR_TEST_CASE(test_async_loading) { BOOST_REQUIRE(s1_later); auto s2_loaded = local_schema_registry().get_or_load(s2->version(), [s2] (table_schema_version) { - return yield().then([s2] { return frozen_schema(s2); }); + return yield().then([s2] -> base_and_view_schemas { return {frozen_schema(s2)}; }); }).get(); BOOST_REQUIRE(s2_loaded); @@ -92,7 +92,7 @@ SEASTAR_TEST_CASE(test_schema_is_synced_when_syncer_doesnt_defer) { return seastar::async([] { dummy_init dummy; auto s = random_schema(); - s = local_schema_registry().get_or_load(s->version(), [s] (table_schema_version) { return frozen_schema(s); }); + s = local_schema_registry().get_or_load(s->version(), [s] (table_schema_version) -> base_and_view_schemas { return {frozen_schema(s)}; }); BOOST_REQUIRE(!s->is_synced()); s->registry_entry()->maybe_sync([] { return make_ready_future<>(); }).get(); BOOST_REQUIRE(s->is_synced()); @@ -103,7 +103,7 @@ SEASTAR_TEST_CASE(test_schema_is_synced_when_syncer_defers) { return seastar::async([] { dummy_init dummy; auto s = random_schema(); - s = local_schema_registry().get_or_load(s->version(), [s] (table_schema_version) { return frozen_schema(s); }); + s = local_schema_registry().get_or_load(s->version(), [s] (table_schema_version) -> base_and_view_schemas { return {frozen_schema(s)}; }); BOOST_REQUIRE(!s->is_synced()); s->registry_entry()->maybe_sync([] { return yield(); }).get(); BOOST_REQUIRE(s->is_synced()); @@ -114,7 +114,7 @@ SEASTAR_TEST_CASE(test_failed_sync_can_be_retried) { return seastar::async([] { dummy_init dummy; auto s = random_schema(); - s = local_schema_registry().get_or_load(s->version(), [s] (table_schema_version) { return frozen_schema(s); }); + s = local_schema_registry().get_or_load(s->version(), [s] (table_schema_version) -> base_and_view_schemas { return {frozen_schema(s)}; }); BOOST_REQUIRE(!s->is_synced()); promise<> fail_sync; @@ -202,8 +202,8 @@ SEASTAR_THREAD_TEST_CASE(test_table_is_attached) { .with_column(random_column_name(), bytes_type) .build(); - auto learned_s2 = local_schema_registry().get_or_load(s2->version(), [&] (table_schema_version) { - return frozen_schema(s2); + auto learned_s2 = local_schema_registry().get_or_load(s2->version(), [&] (table_schema_version) -> base_and_view_schemas { + return {frozen_schema(s2)}; }); BOOST_REQUIRE(learned_s2->maybe_table() == s0->maybe_table()); @@ -221,9 +221,9 @@ SEASTAR_THREAD_TEST_CASE(test_table_is_attached) { .build(); utils::throttle s3_thr; auto s3_entered = s3_thr.block(); - auto learned_s3 = local_schema_registry().get_or_load(s3->version(), [&, fs = frozen_schema(s3)] (table_schema_version) -> future { + auto learned_s3 = local_schema_registry().get_or_load(s3->version(), [&, fs = frozen_schema(s3)] (table_schema_version) -> future { co_await s3_thr.enter(); - co_return fs; + co_return base_and_view_schemas{fs}; }); s3_entered.get(); local_schema_registry().learn(s3); @@ -238,12 +238,12 @@ SEASTAR_THREAD_TEST_CASE(test_table_is_attached) { .build(); utils::throttle s4_thr; auto s4_entered = s4_thr.block(); - auto learned_s4 = local_schema_registry().get_or_load(s4->version(), [&, fs = frozen_schema(s4)] (table_schema_version) -> future { + auto learned_s4 = local_schema_registry().get_or_load(s4->version(), [&, fs = frozen_schema(s4)] (table_schema_version) -> future { co_await s4_thr.enter(); - co_return fs; + co_return base_and_view_schemas(fs); }); s4_entered.get(); - s4 = local_schema_registry().get_or_load(s4->version(), [&, fs = frozen_schema(s4)] (table_schema_version) { return fs; }); + s4 = local_schema_registry().get_or_load(s4->version(), [&, fs = frozen_schema(s4)] (table_schema_version) -> base_and_view_schemas { return {fs}; }); s4_thr.unblock(); auto s4_s = learned_s4.get(); BOOST_REQUIRE(s4_s->maybe_table() == s0->maybe_table()); From 82f2e1b44cad579ba11e73250ddc4502186822ad Mon Sep 17 00:00:00 2001 From: Wojciech Mitros Date: Tue, 10 Dec 2024 02:20:33 +0100 Subject: [PATCH 128/397] schema_registry: update cached base schemas when updating a view The schema registry now holds base schemas for view schemas. The base schema may change without changing the view schema, so to preserve the change in the schema registry, we also update the base schema in the registry when updating the base info in the view schema. --- replica/table.cc | 11 +++++++++-- schema/schema_registry.cc | 4 ++++ schema/schema_registry.hh | 3 +++ 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/replica/table.cc b/replica/table.cc index e71638fd0801..c1e8ce4fe8d0 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -3204,8 +3204,15 @@ void table::set_schema(schema_ptr s) { _schema = std::move(s); for (auto&& v : _views) { - v->view_info()->set_base_info( - v->view_info()->make_base_dependent_view_info(*_schema)); + auto base_info = v->view_info()->make_base_dependent_view_info(*_schema); + v->view_info()->set_base_info(base_info); + if (v->registry_entry()) { + v->registry_entry()->update_base_schema(_schema); + } + if (auto reverse_schema = local_schema_registry().get_or_null(reversed(v->version()))) { + reverse_schema->view_info()->set_base_info(base_info); + reverse_schema->registry_entry()->update_base_schema(_schema); + } } set_compaction_strategy(_schema->compaction_strategy()); diff --git a/schema/schema_registry.cc b/schema/schema_registry.cc index 0920e4c94f32..509fa93e39b7 100644 --- a/schema/schema_registry.cc +++ b/schema/schema_registry.cc @@ -262,6 +262,10 @@ frozen_schema schema_registry_entry::frozen() const { return *_frozen_schema; } +void schema_registry_entry::update_base_schema(schema_ptr s) { + _base_schema = s; +} + future<> schema_registry_entry::maybe_sync(std::function()> syncer) { switch (_sync_state) { case schema_registry_entry::sync_state::SYNCED: diff --git a/schema/schema_registry.hh b/schema/schema_registry.hh index db65ddb9bb31..36f6acf08228 100644 --- a/schema/schema_registry.hh +++ b/schema/schema_registry.hh @@ -94,6 +94,9 @@ public: future<> maybe_sync(std::function()> sync); // Marks this schema version as synced. Syncing cannot be in progress. void mark_synced(); + // Updates the frozen base schema for a view, should be called when updating the base info + // Is not needed when we set the base info for the first time - that means this schema is not in the registry + void update_base_schema(schema_ptr); // Can be called from other shards frozen_schema frozen() const; // Can be called from other shards From 3094ff7cbe014e9d4c087836df8dd65cb550e259 Mon Sep 17 00:00:00 2001 From: Wojciech Mitros Date: Tue, 10 Dec 2024 02:34:26 +0100 Subject: [PATCH 129/397] schema_registry: avoid setting base info when getting the schema from registry After the previous patches, the view schemas returned by schema registry always have their base info set. As such, we no longer need to set it after getting the view schema from the registry. This patch removes these unnecessary updates. --- schema/schema_registry.cc | 12 +----------- service/migration_manager.cc | 33 ++------------------------------- 2 files changed, 3 insertions(+), 42 deletions(-) diff --git a/schema/schema_registry.cc b/schema/schema_registry.cc index 509fa93e39b7..548aea48dece 100644 --- a/schema/schema_registry.cc +++ b/schema/schema_registry.cc @@ -393,17 +393,7 @@ global_schema_ptr::global_schema_ptr(const schema_ptr& ptr) schema_ptr s = ensure_registry_entry(ptr); if (s->is_view()) { - if (s->view_info()->base_info()) { - _base_schema = ensure_registry_entry(s->view_info()->base_info()->base_schema()); - } else if (ptr->view_info()->base_info()) { - _base_schema = ensure_registry_entry(ptr->view_info()->base_info()->base_schema()); - } else { - on_internal_error(slogger, format("Tried to build a global schema for view {}.{} with an uninitialized base info", s->ks_name(), s->cf_name())); - } - - if (!s->view_info()->base_info() || !s->view_info()->base_info()->base_schema()->registry_entry()) { - s->view_info()->set_base_info(s->view_info()->make_base_dependent_view_info(*_base_schema)); - } + _base_schema = ensure_registry_entry(s->view_info()->base_info()->base_schema()); } _ptr = s; } diff --git a/service/migration_manager.cc b/service/migration_manager.cc index 4326e246095e..8f8a1cd73e4c 100644 --- a/service/migration_manager.cc +++ b/service/migration_manager.cc @@ -1072,20 +1072,6 @@ static future get_schema_definition(table_schema_version v, locator: } }); }); - }).then([&storage_proxy] (schema_ptr s) { - // If this is a view so this schema also needs a reference to the base - // table. - if (s->is_view()) { - if (!s->view_info()->base_info()) { - auto& db = storage_proxy.local_db(); - // This line might throw a no_such_column_family - // It should be fine since if we tried to register a view for which - // we don't know the base table, our registry is broken. - schema_ptr base_schema = db.find_schema(s->view_info()->base_id()); - s->view_info()->set_base_info(s->view_info()->make_base_dependent_view_info(*base_schema)); - } - } - return s; }); } @@ -1109,6 +1095,8 @@ future migration_manager::get_schema_for_write(table_schema_version } if (!s) { + // The schema returned by get_schema_definition comes (eventually) from the schema registry, + // so if it is a view, it already has base info and we don't need to set it later s = co_await get_schema_definition(v, dst, shard, ms, _storage_proxy); } @@ -1121,23 +1109,6 @@ future migration_manager::get_schema_for_write(table_schema_version co_await maybe_sync(s, dst); } } - // here s is guaranteed to be valid and synced - if (s->is_view() && !s->view_info()->base_info()) { - // The way to get here is if the view schema was deactivated - // and reactivated again, or if we loaded it from the schema - // history. - auto& db = _storage_proxy.local_db(); - // This line might throw a no_such_column_family but - // that is fine, if the schema is synced, it means that if - // we failed to get the base table, we learned about the base - // table not existing (which means that the view also doesn't exist - // any more), which means that this schema is actually useless for either - // read or write so we better throw than return an incomplete useless - // schema - schema_ptr base_schema = db.find_schema(s->view_info()->base_id()); - s->view_info()->set_base_info(s->view_info()->make_base_dependent_view_info(*base_schema)); - } - co_return s; } From 74cbc77f502214206582f5bf166035f962e65fc6 Mon Sep 17 00:00:00 2001 From: Wojciech Mitros Date: Mon, 16 Dec 2024 01:58:55 +0100 Subject: [PATCH 130/397] test: add test for schema registry maintaining base info for views In this patch we test the behavior of schema registry in a few scenarios where it was identified it could misbehave. The first one is reverse schemas for views. Previously, SELECT queries with reverse order on views could fail because we didn't have base info in the registry for such schemas. The second one is schemas that temporarily died in the registry. This can happen when, while processing a query for a given schema version, all related schema_ptrs were destroyed, but this schema was requested before schema_registry::grace_period() has passed. In this scenario, the base info would not be recovered, causing errors. --- test/boost/schema_registry_test.cc | 44 ++++++++++++++++++++++++++-- test/cqlpy/test_materialized_view.py | 11 +++++++ 2 files changed, 53 insertions(+), 2 deletions(-) diff --git a/test/boost/schema_registry_test.cc b/test/boost/schema_registry_test.cc index 9957726c29bd..c5b53d1d3022 100644 --- a/test/boost/schema_registry_test.cc +++ b/test/boost/schema_registry_test.cc @@ -12,6 +12,8 @@ #undef SEASTAR_TESTING_MAIN #include #include +#include +#include #include "data_dictionary/user_types_metadata.hh" #include "schema/schema_registry.hh" #include "schema/schema_builder.hh" @@ -23,6 +25,7 @@ #include "utils/throttle.hh" #include "test/lib/cql_test_env.hh" #include "gms/feature_service.hh" +#include "view_info.hh" BOOST_AUTO_TEST_SUITE(schema_registry_test) @@ -40,10 +43,11 @@ static schema_ptr random_schema() { struct dummy_init { std::unique_ptr config; gms::feature_service fs; - + seastar::lowres_clock::duration grace_period; dummy_init() : config(std::make_unique()) - , fs(gms::feature_config_from_db_config(*config)) { + , fs(gms::feature_config_from_db_config(*config)) + , grace_period(std::chrono::seconds(config->schema_registry_grace_period())) { local_schema_registry().init(db::schema_ctxt(*config, std::make_shared(), fs)); } }; @@ -258,4 +262,40 @@ SEASTAR_THREAD_TEST_CASE(test_table_is_attached) { }).get(); } +SEASTAR_THREAD_TEST_CASE(test_schema_is_recovered_after_dying) { + dummy_init dummy; + auto base_schema = schema_builder("ks", "cf") + .with_column("pk", int32_type, column_kind::partition_key) + .with_column("v", int32_type) + .build(); + auto base_registry_schema = local_schema_registry().get_or_load(base_schema->version(), + [base_schema] (table_schema_version) -> base_and_view_schemas { return {frozen_schema(base_schema)}; }); + base_registry_schema = nullptr; + auto recovered_registry_schema = local_schema_registry().get_or_null(base_schema->version()); + BOOST_REQUIRE(recovered_registry_schema); + recovered_registry_schema = nullptr; + seastar::sleep(dummy.grace_period).get(); + BOOST_REQUIRE(!local_schema_registry().get_or_null(base_schema->version())); +} + +SEASTAR_THREAD_TEST_CASE(test_view_info_is_recovered_after_dying) { + dummy_init dummy; + auto base_schema = schema_builder("ks", "cf") + .with_column("pk", int32_type, column_kind::partition_key) + .with_column("v", int32_type) + .build(); + schema_builder view_builder("ks", "cf_view"); + auto view_schema = schema_builder("ks", "cf_view") + .with_column("v", int32_type, column_kind::partition_key) + .with_column("pk", int32_type) + .with_view_info(*base_schema, false, "pk IS NOT NULL AND v IS NOT NULL") + .build(); + view_schema->view_info()->set_base_info(view_schema->view_info()->make_base_dependent_view_info(*base_schema)); + local_schema_registry().get_or_load(view_schema->version(), + [view_schema, base_schema] (table_schema_version) -> base_and_view_schemas { return {frozen_schema(view_schema), base_schema}; }); + auto view_registry_schema = local_schema_registry().get_or_null(view_schema->version()); + BOOST_REQUIRE(view_registry_schema); + BOOST_REQUIRE(view_registry_schema->view_info()->base_info()); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/test/cqlpy/test_materialized_view.py b/test/cqlpy/test_materialized_view.py index 866df33131f4..a6b57a8ffa38 100644 --- a/test/cqlpy/test_materialized_view.py +++ b/test/cqlpy/test_materialized_view.py @@ -1687,3 +1687,14 @@ def test_view_in_API(cql, test_keyspace): wait_for_view_built(cql, view) res = rest_api.get_request(cql, f"column_family/built_indexes/{base.replace('.',':')}") assert view_name not in res + +# Test that we can perform reads from the view in reverse order without crashing. +# Reproduces issue https://github.com/scylladb/scylladb/issues/21354 +def test_reverse_read_from_view(cql, test_keyspace): + with new_test_table(cql, test_keyspace, 'a int PRIMARY KEY, b int') as table: + with new_materialized_view(cql, table, '*', 'b, a', 'a is not null and b is not null') as mv: + cql.execute(f'insert into {table} (a, b) values (1, 1)') + cql.execute(f'insert into {table} (a, b) values (2, 1)') + assert {(1,),(2,)} == set(cql.execute(f'select a from {mv} where b=1')) + assert [(1,),(2,)] == list(cql.execute(f'select a from {mv} where b=1 order by a asc')) + assert [(2,),(1,)] == list(cql.execute(f'select a from {mv} where b=1 order by a desc')) From 5ef7afb968baeeec7929ee41dc38ad2629e4f564 Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Mon, 23 Dec 2024 17:27:16 +0200 Subject: [PATCH 131/397] cql3: allow SELECT of specific collection key This adds to the grammar the option to SELECT a specific key in a collection column using subscript syntax. For example: SELECT map['key'] FROM table SELECT map['key1']['key2'] FROM table The key can also be parameterized in a prepared query. For this we need to pass the query options to result_set_builder where we process the selectors. Fixes scylladb/scylladb#7751 --- cql3/Cql.g | 4 +- cql3/expr/prepare_expr.cc | 3 +- cql3/selection/selectable.cc | 2 +- cql3/selection/selection.cc | 4 +- cql3/selection/selection.hh | 2 + cql3/statements/select_statement.cc | 18 +- docs/cql/dml/select.rst | 4 +- docs/cql/types.rst | 12 ++ service/mapreduce_service.cc | 1 + service/pager/query_pagers.cc | 2 +- .../validation/entities/collections_test.py | 46 ++--- .../operations/select_order_by_test.py | 3 +- test/cqlpy/test_select_collection_element.py | 158 ++++++++++++++++++ 13 files changed, 221 insertions(+), 38 deletions(-) create mode 100644 test/cqlpy/test_select_collection_element.py diff --git a/cql3/Cql.g b/cql3/Cql.g index e09d155ffa2a..f62298d14ccc 100644 --- a/cql3/Cql.g +++ b/cql3/Cql.g @@ -460,7 +460,9 @@ unaliasedSelector returns [uexpression s] | f=functionName args=selectionFunctionArgs { tmp = function_call{std::move(f), std::move(args)}; } | K_CAST '(' arg=unaliasedSelector K_AS t=native_type ')' { tmp = cast{.style = cast::cast_style::sql, .arg = std::move(arg), .type = std::move(t)}; } ) - ( '.' fi=cident { tmp = field_selection{std::move(tmp), std::move(fi)}; } )* + ( '.' fi=cident { tmp = field_selection{std::move(tmp), std::move(fi)}; } + | '[' sub=term ']' { tmp = subscript{std::move(tmp), std::move(sub)}; } + )* { $s = tmp; } ; diff --git a/cql3/expr/prepare_expr.cc b/cql3/expr/prepare_expr.cc index f9fc20b7fe5c..e771273f40e2 100644 --- a/cql3/expr/prepare_expr.cc +++ b/cql3/expr/prepare_expr.cc @@ -1286,7 +1286,8 @@ test_assignment(const expression& expr, data_dictionary::database db, const sstr return expression_test_assignment(col_val.col->type, receiver); }, [&] (const subscript&) -> test_result { - on_internal_error(expr_logger, "subscripts are not yet reachable via test_assignment()"); + // not implemented. issue #22075 + return assignment_testable::test_result::NOT_ASSIGNABLE; }, [&] (const unresolved_identifier& ui) -> test_result { return unresolved_identifier_test_assignment(ui, db, keyspace, schema_opt, receiver); diff --git a/cql3/selection/selectable.cc b/cql3/selection/selectable.cc index 5c9fb8544f2d..f1a3a4eac0f7 100644 --- a/cql3/selection/selectable.cc +++ b/cql3/selection/selectable.cc @@ -41,7 +41,7 @@ selectable_processes_selection(const expr::expression& selectable) { on_internal_error(slogger, "no way to express 'SELECT a binop b' in the grammar yet"); }, [] (const expr::subscript&) -> bool { - on_internal_error(slogger, "no way to express 'SELECT a[b]' in the grammar yet"); + return true; }, [&] (const expr::column_value& column) -> bool { return false; diff --git a/cql3/selection/selection.cc b/cql3/selection/selection.cc index 777deeab359b..1a1ee5dd53bf 100644 --- a/cql3/selection/selection.cc +++ b/cql3/selection/selection.cc @@ -382,7 +382,7 @@ class selection_with_processing : public selection { .clustering_key = rs.current_clustering_key, .static_and_regular_columns = rs.current, .selection = &_sel, - .options = nullptr, + .options = rs._options, .static_and_regular_timestamps = rs._timestamps, .static_and_regular_ttls = rs._ttls, .temporaries = {}, @@ -516,6 +516,7 @@ selection::collect_metadata(const schema& schema, const std::vector group_by_cell_indices, uint64_t limit, uint64_t per_partition_limit) : _result_set(std::make_unique(::make_shared(*(s.get_result_metadata())))) @@ -527,6 +528,7 @@ result_set_builder::result_set_builder(const selection& s, gc_clock::time_point , _per_partition_remaining_previous_partition(per_partition_limit) , _last_group(_group_by_cell_indices.size()) , _group_began(false) + , _options(options) , _now(now) { if (s._collect_timestamps) { diff --git a/cql3/selection/selection.hh b/cql3/selection/selection.hh index f75dc270a6ed..f3bb0ce5e91e 100644 --- a/cql3/selection/selection.hh +++ b/cql3/selection/selection.hh @@ -189,6 +189,7 @@ public: std::vector current_clustering_key; std::vector _timestamps; std::vector _ttls; + const query_options* _options; private: const gc_clock::time_point _now; public: @@ -244,6 +245,7 @@ public: }; result_set_builder(const selection& s, gc_clock::time_point now, + const query_options* options = nullptr, std::vector group_by_cell_indices = {}, uint64_t limit = std::numeric_limits::max(), uint64_t per_partition_limit = std::numeric_limits::max()); diff --git a/cql3/statements/select_statement.cc b/cql3/statements/select_statement.cc index fd47f218a04f..25d40eaf53f4 100644 --- a/cql3/statements/select_statement.cc +++ b/cql3/statements/select_statement.cc @@ -499,7 +499,7 @@ select_statement::execute_without_checking_exception_message_aggregate_or_paged( auto per_partition_limit = get_limit(options, _per_partition_limit, true); if (aggregate || nonpaged_filtering) { - auto builder = cql3::selection::result_set_builder(*_selection, now, *_group_by_cell_indices, limit, per_partition_limit.value()); + auto builder = cql3::selection::result_set_builder(*_selection, now, &options, *_group_by_cell_indices, limit, per_partition_limit.value()); coordinator_result result_void = co_await utils::result_do_until( [&p, &builder, limit] { return p->is_exhausted() || (limit < builder.result_set_size()); @@ -916,7 +916,7 @@ select_statement::process_results_complex(foreign_ptr cmd, const query_options& options, gc_clock::time_point now) const { - cql3::selection::result_set_builder builder(*_selection, now); + cql3::selection::result_set_builder builder(*_selection, now, &options); co_return co_await builder.with_thread_if_needed([&] { if (_restrictions_need_filtering) { results->ensure_counts(); @@ -1192,7 +1192,7 @@ indexed_table_select_statement::do_execute(query_processor& qp, // the paging state between requesting data from replicas. const bool aggregate = _selection->is_aggregate() || has_group_by(); if (aggregate) { - cql3::selection::result_set_builder builder(*_selection, now, *_group_by_cell_indices); + cql3::selection::result_set_builder builder(*_selection, now, &options, *_group_by_cell_indices); std::unique_ptr internal_options = std::make_unique(cql3::query_options(options)); stop_iteration stop; // page size is set to the internal count page size, regardless of the user-provided value @@ -1371,9 +1371,9 @@ indexed_table_select_statement::read_posting_list(query_processor& qp, int32_t page_size = options.get_page_size(); if (page_size <= 0 || !service::pager::query_pagers::may_need_paging(*_view_schema, page_size, *cmd, partition_ranges)) { return qp.proxy().query_result(_view_schema, cmd, std::move(partition_ranges), options.get_consistency(), {timeout, state.get_permit(), state.get_client_state(), state.get_trace_state()}) - .then(utils::result_wrap([this, now, selection = std::move(selection), partition_slice = std::move(partition_slice)] (service::storage_proxy::coordinator_query_result qr) + .then(utils::result_wrap([this, now, &options, selection = std::move(selection), partition_slice = std::move(partition_slice)] (service::storage_proxy::coordinator_query_result qr) -> coordinator_result<::shared_ptr> { - cql3::selection::result_set_builder builder(*selection, now); + cql3::selection::result_set_builder builder(*selection, now, &options); query::result_view::consume(*qr.query_result, std::move(partition_slice), cql3::selection::result_set_builder::visitor(builder, *_view_schema, *selection)); @@ -1799,8 +1799,8 @@ mutation_fragments_select_statement::do_execute(query_processor& qp, service::qu *command, key_ranges))) { return do_query(erm_keepalive, {}, qp.proxy(), _schema, command, std::move(key_ranges), cl, {timeout, state.get_permit(), state.get_client_state(), state.get_trace_state(), {}, {}}) - .then(wrap_result_to_error_message([this, erm_keepalive, now, slice = command->slice] (service::storage_proxy_coordinator_query_result&& qr) mutable { - cql3::selection::result_set_builder builder(*_selection, now); + .then(wrap_result_to_error_message([this, erm_keepalive, now, &options, slice = command->slice] (service::storage_proxy_coordinator_query_result&& qr) mutable { + cql3::selection::result_set_builder builder(*_selection, now, &options); query::result_view::consume(*qr.query_result, std::move(slice), cql3::selection::result_set_builder::visitor(builder, *_schema, *_selection)); auto msg = ::make_shared(result(builder.build())); @@ -1982,6 +1982,10 @@ std::unique_ptr select_statement::prepare(data_dictionary::d } } + for (auto& ps : prepared_selectors) { + expr::fill_prepare_context(ps.expr, ctx); + } + for (auto& ps : prepared_selectors) { aggregation_depth = std::max(aggregation_depth, expr::aggregation_depth(ps.expr)); } diff --git a/docs/cql/dml/select.rst b/docs/cql/dml/select.rst index 4f6d08800fb3..4a3ff7944756 100644 --- a/docs/cql/dml/select.rst +++ b/docs/cql/dml/select.rst @@ -20,10 +20,12 @@ Querying data from data is done using a ``SELECT`` statement: : [ BYPASS CACHE ] : [ USING TIMEOUT `timeout` ] select_clause: `selector` [ AS `identifier` ] ( ',' `selector` [ AS `identifier` ] )* - selector: `column_name` + selector: ( `column_name` : | CAST '(' `selector` AS `cql_type` ')' : | `function_name` '(' [ `selector` ( ',' `selector` )* ] ')' : | COUNT '(' '*' ')' + : ) + : ( '.' `field_name` | '[' `term` ']' )* where_clause: `relation` ( AND `relation` )* group_by_clause: `column_name` (',' `column_name` )* relation: `column_name` `operator` `term` diff --git a/docs/cql/types.rst b/docs/cql/types.rst index a2869df6014f..f7d8983f5061 100644 --- a/docs/cql/types.rst +++ b/docs/cql/types.rst @@ -334,6 +334,10 @@ Further, maps support: Note that for removing multiple elements in a ``map``, you remove from it a ``set`` of keys. +- Selecting one element:: + + SELECT favs['fruit'] FROM users WHERE id = 'jsmith'; + Lastly, TTLs are allowed for both ``INSERT`` and ``UPDATE``, but in both cases, the TTL set only applies to the newly inserted/updated elements. In other words:: @@ -376,6 +380,10 @@ Further, sets support: UPDATE images SET tags = tags - { 'cat' } WHERE name = 'cat.jpg'; +- Selecting an element (if the element doesn't exist, returns null):: + + SELECT tags['gray'] FROM images; + Lastly, as for :ref:`maps `, TTLs, if used, only apply to the newly inserted values. .. _lists: @@ -432,6 +440,10 @@ Further, lists support: UPDATE plays SET scores = scores - [ 12, 21 ] WHERE id = '123-afde'; +- Selecting an element by its position in the list:: + + SELECT scores[1] FROM plays; + .. warning:: The append and prepend operations are not idempotent by nature. So, in particular, if one of these operation timeouts, then retrying the operation is not safe, and it may (or may not) lead to appending/prepending the value twice. diff --git a/service/mapreduce_service.cc b/service/mapreduce_service.cc index 275ed040e825..2a434b196f35 100644 --- a/service/mapreduce_service.cc +++ b/service/mapreduce_service.cc @@ -443,6 +443,7 @@ future mapreduce_service::execute_on_this_shard( auto rs_builder = cql3::selection::result_set_builder( *selection, now, + nullptr, std::vector() // Represents empty GROUP BY indices. ); diff --git a/service/pager/query_pagers.cc b/service/pager/query_pagers.cc index cfe6316e845c..75de0bc59838 100644 --- a/service/pager/query_pagers.cc +++ b/service/pager/query_pagers.cc @@ -222,7 +222,7 @@ future> query_pager::fetch_page(uint32_t page_ future>> query_pager::fetch_page_result(uint32_t page_size, gc_clock::time_point now, db::timeout_clock::time_point timeout) { return do_with( - cql3::selection::result_set_builder(*_selection, now), + cql3::selection::result_set_builder(*_selection, now, &_options), [this, page_size, now, timeout](auto& builder) { return this->fetch_page_result(builder, page_size, now, timeout).then(utils::result_wrap([&builder] { return builder.with_thread_if_needed([&builder] () -> result> { diff --git a/test/cqlpy/cassandra_tests/validation/entities/collections_test.py b/test/cqlpy/cassandra_tests/validation/entities/collections_test.py index b0e0f66efd4f..1053f26d1a68 100644 --- a/test/cqlpy/cassandra_tests/validation/entities/collections_test.py +++ b/test/cqlpy/cassandra_tests/validation/entities/collections_test.py @@ -876,7 +876,7 @@ def testMultipleOperationOnSetWithinTheSameQuery(cql, test_keyspace): execute(cql, table, "UPDATE %s SET s = s - ? , s = s + ? WHERE pk = ?", {3}, {3, 4}, 1) assert_rows(execute(cql, table, "SELECT * FROM %s WHERE pk = 1") , [1, {0, 1, 2, 4}]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice and subscript assignment not yet supported. Issue #22075") def testMapOperation(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int, c int, l text, m map, fm frozen>, sm map STATIC, fsm frozen> STATIC, o int, PRIMARY KEY (k, c))") as table: execute(cql, table, "INSERT INTO %s(k, c, l, m, fm, sm, fsm, o) VALUES (0, 0, 'foobar', ?, ?, ?, ?, 42)", @@ -1028,7 +1028,7 @@ def testMapOperation(cql, test_keyspace): [0, "foobar", {"1": "value1", "22": "value22"}, 42], [0, "foobar", {"1": "value1_2"}, 42]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice and subscript assignment not yet supported. Issue #22075") def testMapOperationWithIntKey(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int, c int, l text, m map, fm frozen>, sm map STATIC, fsm frozen> STATIC, o int, PRIMARY KEY (k, c))") as table: # used type "int" as map key intentionally since CQL parsing relies on "BigInteger" @@ -1133,7 +1133,7 @@ def testMapOperationWithIntKey(cql, test_keyspace): [0, "foobar", {1: "value1", 22: "value22"}, 42], [0, "foobar", {1: "value1_2"}, 42]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testMapOperationOnPartKey(cql, test_keyspace): with create_table(cql, test_keyspace, "(k frozen> PRIMARY KEY, l text, o int)") as table: execute(cql, table, "INSERT INTO %s(k, l, o) VALUES (?, 'foobar', 42)", {"1": "value1", "22": "value22", "333": "value333"}) @@ -1168,7 +1168,7 @@ def testMapOperationOnPartKey(cql, test_keyspace): assert_rows(execute(cql, table, "SELECT l, k, o FROM %s WHERE k = ?", {"1": "value1", "22": "value22", "333": "value333"}), ["foobar", {"1": "value1", "22": "value22", "333": "value333"}, 42]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testMapOperationOnClustKey(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int, c frozen>, l text, o int, PRIMARY KEY (k,c))") as table: execute(cql, table, "INSERT INTO %s(k, c, l, o) VALUES (0, ?, 'foobar', 42)", {"1": "value1", "22": "value22", "333": "value333"}) @@ -1203,7 +1203,7 @@ def testMapOperationOnClustKey(cql, test_keyspace): assert_rows(execute(cql, table, "SELECT k, l, c, o FROM %s WHERE k = 0 AND c = ?", {"1": "value1", "22": "value22", "333": "value333"}), [0, "foobar", {"1": "value1", "22": "value22", "333": "value333"}, 42]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a set slice not yet supported. Issue #22075") def testSetOperation(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int, c int, l text, s set, fs frozen>, ss set STATIC, fss frozen> STATIC, o int, PRIMARY KEY (k, c))") as table: execute(cql, table, "INSERT INTO %s(k, c, l, s, fs, ss, fss, o) VALUES (0, 0, 'foobar', ?, ?, ?, ?, 42)", @@ -1273,7 +1273,7 @@ def testSetOperation(cql, test_keyspace): [0, "foobar", {"1", "22", "22_2", "333"}, 42], [0, "foobar", {"22", "333"}, 42]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testCollectionSliceOnMV(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int, c int, l text, m map, o int, PRIMARY KEY (k, c))") as table: assert_invalid_message(cql, table, "Can only select columns by name when defining a materialized view (got m['abc'])", @@ -1281,18 +1281,18 @@ def testCollectionSliceOnMV(cql, test_keyspace): assert_invalid_message(cql, table, "Can only select columns by name when defining a materialized view (got m['abc'..'def'])", "CREATE MATERIALIZED VIEW " + test_keyspace + ".view1 AS SELECT m['abc'..'def'] FROM %s WHERE k IS NOT NULL AND c IS NOT NULL AND m IS NOT NULL PRIMARY KEY (c, k)"); -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") -def testElementAccessOnList(cql, test_keyspace): - with create_table(cql, test_keyspace, "(pk int PRIMARY KEY, l list)") as table: - execute(cql, table, "INSERT INTO %s (pk, l) VALUES (1, [1, 2, 3])"); - - assert_invalid_message(cql, table, "Element selection is only allowed on sets and maps, but l is a list", - "SELECT pk, l[0] FROM %s"); - - assert_invalid_message(cql, table, "Slice selection is only allowed on sets and maps, but l is a list", - "SELECT pk, l[1..3] FROM %s"); +# Scylla supports element access on list. +# def testElementAccessOnList(cql, test_keyspace): +# with create_table(cql, test_keyspace, "(pk int PRIMARY KEY, l list)") as table: +# execute(cql, table, "INSERT INTO %s (pk, l) VALUES (1, [1, 2, 3])"); +# +# assert_invalid_message(cql, table, "Element selection is only allowed on sets and maps, but l is a list", +# "SELECT pk, l[0] FROM %s"); +# +# assert_invalid_message(cql, table, "Slice selection is only allowed on sets and maps, but l is a list", +# "SELECT pk, l[1..3] FROM %s"); -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testCollectionOperationResultSetMetadata(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int PRIMARY KEY, m map, fm frozen>, s set, fs frozen>)") as table: execute(cql, table, "INSERT INTO %s (k, m, fm, s, fs) VALUES (?, ?, ?, ?, ?)", @@ -1350,7 +1350,7 @@ def testCollectionOperationResultSetMetadata(cql, test_keyspace): } """ -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testFrozenCollectionNestedAccess(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int PRIMARY KEY, m map>>>)") as table: execute(cql, table, "INSERT INTO %s(k, m) VALUES (0, ?)", {"1": {"a": {1, 2, 4}, "b": {3}}, "2": {"a": {2, 4}}}) @@ -1368,7 +1368,7 @@ def testFrozenCollectionNestedAccess(cql, test_keyspace): assert_rows(execute(cql, table, "SELECT m[?][?][?] FROM %s WHERE k = 0", "1", "a", 2), [2]) assert_rows(execute(cql, table, "SELECT m[?][?][?..?] FROM %s WHERE k = 0", "1", "a", 2, 3), [{2}]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testUDTAndCollectionNestedAccess(cql, test_keyspace): sm_tuple = collections.namedtuple('sm_tuple', ['s', 'm']) with create_type(cql, test_keyspace, "(s set, m map)") as type_name: @@ -1406,7 +1406,7 @@ def testUDTAndCollectionNestedAccess(cql, test_keyspace): assert_rows(execute(cql, table, "SELECT v.m[..?] FROM %s WHERE k = 0", "b"), [{"a": "v1"}]) assert_rows(execute(cql, table, "SELECT v.m[?] FROM %s WHERE k = 0", "d"), ["v2"]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testMapOverlappingSlices(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int PRIMARY KEY, m map)") as table: execute(cql, table, "INSERT INTO %s(k, m) VALUES (?, ?)", 0, {0: 0, 1: 1, 2: 2, 3: 3, 4: 4, 5: 5}) @@ -1431,7 +1431,7 @@ def testMapOverlappingSlices(cql, test_keyspace): assert_rows(execute(cql, table, "SELECT m[1..3], m[2] FROM %s WHERE k=?", 0), [{1: 1, 2: 2, 3: 3}, 2]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testMapOverlappingSlicesWithDoubles(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int PRIMARY KEY, m map)") as table: execute(cql, table, "INSERT INTO %s(k, m) VALUES (?, ?)", 0, {0.0: 0.0, 1.1: 1.1, 2.2: 2.2, 3.0: 3.0, 4.4: 4.4, 5.5: 5.5}) @@ -1453,7 +1453,7 @@ def testMapOverlappingSlicesWithDoubles(cql, test_keyspace): assert_rows(execute(cql, table, "SELECT m[1.1..3.0], m[2.2] FROM %s WHERE k=?", 0), [{1.1: 1.1, 2.2: 2.2, 3.0: 3.0}, 2.2]) -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testNestedAccessWithNestedMap(cql, test_keyspace): with create_table(cql, test_keyspace, "(id text PRIMARY KEY, m map>>)") as table: execute(cql, table, "INSERT INTO %s (id,m) VALUES ('1', {1: {2: 'one-two'}})") @@ -1507,7 +1507,7 @@ def testInsertingCollectionsWithInvalidElements(cql, test_keyspace): assert_invalid_message(cql, table, "Invalid map literal for m: value (1, '1', 1.0, 1) is not of type frozen>", "INSERT INTO %s (k, m) VALUES (0, {1 : (1, '1', 1.0, 1)})") -@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting part of map or list not yet supported. Issue #7751") +@pytest.mark.xfail(reason="Cassandra 4.0 feature of selecting a collection slice not yet supported. Issue #22075") def testSelectionOfEmptyCollections(cql, test_keyspace): with create_table(cql, test_keyspace, "(k int PRIMARY KEY, m frozen>, s frozen>)") as table: execute(cql, table, "INSERT INTO %s(k) VALUES (0)") diff --git a/test/cqlpy/cassandra_tests/validation/operations/select_order_by_test.py b/test/cqlpy/cassandra_tests/validation/operations/select_order_by_test.py index 7d1ffded7e6f..787fe2862a6d 100644 --- a/test/cqlpy/cassandra_tests/validation/operations/select_order_by_test.py +++ b/test/cqlpy/cassandra_tests/validation/operations/select_order_by_test.py @@ -317,8 +317,7 @@ def testOrderByForInClause(cql, test_keyspace): assert_invalid_message(cql, table, "LIMIT must be strictly positive", "SELECT v as c2 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1 DESC , c2 DESC LIMIT 0; ", 1, 1, 2) -# Reproduces #7751 -@pytest.mark.xfail(reason="Issue #7751") +@pytest.mark.skip(reason="Issue #22061") def testOrderByForInClauseWithCollectionElementSelection(cql, test_keyspace): with create_table(cql, test_keyspace, "(pk int, c frozen>, v int, PRIMARY KEY (pk, c))") as table: execute(cql, table, "INSERT INTO %s (pk, c, v) VALUES (0, {1, 2}, 0)") diff --git a/test/cqlpy/test_select_collection_element.py b/test/cqlpy/test_select_collection_element.py new file mode 100644 index 000000000000..a2840cd78634 --- /dev/null +++ b/test/cqlpy/test_select_collection_element.py @@ -0,0 +1,158 @@ +# Copyright 2024-present ScyllaDB +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + +############################################################################# +# Tests for SELECT of a specific key in a collection column +############################################################################# + +import pytest +import re +import time +from cassandra.protocol import InvalidRequest +from .util import unique_name, unique_key_int, unique_key_string, new_test_table, new_type, new_function + + +@pytest.fixture(scope="module") +def table1(cql, test_keyspace): + table = test_keyspace + "." + unique_name() + cql.execute(f"CREATE TABLE {table} (p int PRIMARY KEY, m map)") + yield table + cql.execute("DROP TABLE " + table) + +def test_basic_int_key_selection(cql, table1): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1}(p,m) VALUES ({p}, " + "{1:10,2:20})") + assert list(cql.execute(f"SELECT m[1] FROM {table1} WHERE p={p}")) == [(10,)] + assert list(cql.execute(f"SELECT m[2] FROM {table1} WHERE p={p}")) == [(20,)] + assert list(cql.execute(f"SELECT m[3] FROM {table1} WHERE p={p}")) == [(None,)] + +def test_basic_string_key_selection(cql, test_keyspace): + schema = 'p int PRIMARY KEY, m map' + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p,m) VALUES ({p}, " + "{'aa':10,'ab':20})") + assert list(cql.execute(f"SELECT m['aa'] FROM {table} WHERE p={p}")) == [(10,)] + assert list(cql.execute(f"SELECT m['ab'] FROM {table} WHERE p={p}")) == [(20,)] + assert list(cql.execute(f"SELECT m['ac'] FROM {table} WHERE p={p}")) == [(None,)] + +def test_subscript_type_mismatch(cql, table1): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1}(p,m) VALUES ({p}, " + "{1:10,2:20})") + with pytest.raises(InvalidRequest): + cql.execute(f"SELECT m['x'] FROM {table1} WHERE p={p}") + +def test_subscript_with_alias(cql, table1): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1}(p,m) VALUES ({p}, " + "{1:10,2:20})") + assert [(r.m1, r.m2) for r in cql.execute(f"SELECT m[1] as m1, m[2] as m2 FROM {table1} WHERE p={p}")] == [(10, 20)] + +def test_frozen_map_subscript(cql, test_keyspace): + schema = 'p int PRIMARY KEY, m frozen>' + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p,m) VALUES ({p}, " + "{1:10,2:20})") + assert list(cql.execute(f"SELECT m[1] FROM {table} WHERE p={p}")) == [(10,)] + assert list(cql.execute(f"SELECT m[2] FROM {table} WHERE p={p}")) == [(20,)] + assert list(cql.execute(f"SELECT m[3] FROM {table} WHERE p={p}")) == [(None,)] + +def test_nested_key_selection(cql, test_keyspace): + schema = 'p int PRIMARY KEY, m map>>' + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p, m) VALUES ({p}, " + "{'1': {'a': 10, 'b': 11}, '2': {'a': 12}})") + assert list(cql.execute(f"SELECT m['1']['a'] FROM {table} WHERE p={p}")) == [(10,)] + assert list(cql.execute(f"SELECT m['1']['b'] FROM {table} WHERE p={p}")) == [(11,)] + assert list(cql.execute(f"SELECT m['2']['a'] FROM {table} WHERE p={p}")) == [(12,)] + assert list(cql.execute(f"SELECT m['2']['b'] FROM {table} WHERE p={p}")) == [(None,)] + +def test_prepare_key(cql, table1): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1} (p,m) VALUES ({p}, " + "{1:10,2:20})") + + lookup1 = cql.prepare(f"SELECT m[?] FROM {table1} WHERE p = ?") + assert list(cql.execute(lookup1, [1, p])) == [(10,)] + assert list(cql.execute(lookup1, [2, p])) == [(20,)] + assert list(cql.execute(lookup1, [3, p])) == [(None,)] + + lookup2 = cql.prepare(f"SELECT m[:x1], m[:x2] FROM {table1} WHERE p = :key") + assert list(cql.execute(lookup2, {'x1':2, 'x2':1, 'key':p})) == [(20,10)] + +def test_null_map(cql, table1): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1}(p) VALUES ({p})") + assert list(cql.execute(f"SELECT m[1] FROM {table1} WHERE p={p}")) == [(None,)] + +# scylla only because scylla returns null while cassandra returns error +def test_null_subscript(scylla_only, cql, table1): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1}(p,m) VALUES ({p}, " + "{1:10,2:20})") + assert list(cql.execute(f"SELECT m[null] FROM {table1} WHERE p={p}")) == [(None,)] + +def test_subscript_and_field(cql, test_keyspace): + with new_type(cql, test_keyspace, '(a int)') as typ: + schema = f"p int PRIMARY KEY, m map>" + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p,m) VALUES ({p}, " + "{1:{a:10}})") + assert list(cql.execute(f"SELECT m[1].a FROM {table} WHERE p={p}")) == [(10,)] + +def test_field_and_subscript(cql, test_keyspace): + with new_type(cql, test_keyspace, '(a frozen>)') as typ: + schema = f"p int PRIMARY KEY, t {typ}" + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p,t) VALUES ({p}, " + "{a:{1:10}})") + assert list(cql.execute(f"SELECT t.a[1] FROM {table} WHERE p={p}")) == [(10,)] + +def test_field_and_subscript_and_field(cql, test_keyspace): + with new_type(cql, test_keyspace, '(b int)') as typ1, \ + new_type(cql, test_keyspace, f"(a frozen>)") as typ2: + schema = f"p int PRIMARY KEY, t {typ2}" + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p,t) VALUES ({p}, " + "{a:{1:{b:10}}})") + assert list(cql.execute(f"SELECT t.a[1].b FROM {table} WHERE p={p}")) == [(10,)] + +def test_other_types_cannot_be_subscripted(cql, table1): + with pytest.raises(InvalidRequest, match='not a'): + cql.execute(f"SELECT p[2] FROM {table1}") + with pytest.raises(InvalidRequest, match='not a'): + cql.execute(f"SELECT token(p)[2] FROM {table1}") + +def test_udf_subscript(scylla_only, cql, test_keyspace, table1): + fn = "(k int) CALLED ON NULL INPUT RETURNS int LANGUAGE Lua AS 'return k+1'" + with new_function(cql, test_keyspace, fn, 'add_one'): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1}(p,m) VALUES ({p}, " + "{1:10,2:20})") + assert list(cql.execute(f"SELECT m[add_one(1)] FROM {table1} WHERE p={p}")) == [(20,)] + +# cassandra doesn't support subscript on a list +def test_list_subscript(scylla_only, cql, test_keyspace): + schema = 'p int PRIMARY KEY, l list' + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p,l) VALUES ({p}, " + "[10,20])") + assert list(cql.execute(f"SELECT l[0] FROM {table} WHERE p={p}")) == [(10,)] + assert list(cql.execute(f"SELECT l[1] FROM {table} WHERE p={p}")) == [(20,)] + assert list(cql.execute(f"SELECT l[2] FROM {table} WHERE p={p}")) == [(None,)] + assert list(cql.execute(f"SELECT l[10] FROM {table} WHERE p={p}")) == [(None,)] + +def test_set_subscript(cql, test_keyspace): + schema = 'p int PRIMARY KEY, s set' + with new_test_table(cql, test_keyspace, schema) as table: + p = unique_key_int() + cql.execute(f"INSERT INTO {table}(p,s) VALUES ({p}, " + "{10,20})") + assert list(cql.execute(f"SELECT s[0] FROM {table} WHERE p={p}")) == [(None,)] + assert list(cql.execute(f"SELECT s[10] FROM {table} WHERE p={p}")) == [(10,)] + assert list(cql.execute(f"SELECT s[11] FROM {table} WHERE p={p}")) == [(None,)] + assert list(cql.execute(f"SELECT s[20] FROM {table} WHERE p={p}")) == [(20,)] + +# scylla only because cassandra doesn't support lua language +@pytest.mark.xfail(reason="#22075") +def test_subscript_function_arg(scylla_only, cql, test_keyspace, table1): + fn = "(k int) CALLED ON NULL INPUT RETURNS int LANGUAGE Lua AS 'return k+1'" + with new_function(cql, test_keyspace, fn, 'add_one'): + p = unique_key_int() + cql.execute(f"INSERT INTO {table1}(p,m) VALUES ({p}, " + "{1:10,2:20})") + assert list(cql.execute(f"SELECT add_one(m[1]) FROM {table1} WHERE p={p}")) == [(11,)] From bf3d0b3543d2e7d670c518b22b1f21052a95cecf Mon Sep 17 00:00:00 2001 From: Tomasz Grabiec Date: Sun, 27 Oct 2024 21:27:19 +0100 Subject: [PATCH 132/397] reader_concurrency_semaphore: Optimize resource_units destruction by postponing wait list processing MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Observed 3% throughput improvement in sstable-heavy workload bounded by CPU. SStable parsing involves lots of buffer operations which obtain and destroy resource_units. Before the patch, reosurce_unit destruction invoked maybe_admit_waiters(), which performs some computations on waiting permits. We don't really need to admit on each change of resources, since the CPU is used by other things anyway. We can batch the computation. There is already a fiber which does this for processing the _ready_list. We can reuse it for processing _wait_list as well. The changes violate an assumption made by tests that releasing resources immediately triggers an admission check. Therefore, some of the BOOST_REQUIRE_EQUAL needs to be replaced with REQUIRE_EVENTUALLY_EQUAL as the admision check is now done in the fiber processing the _ready_list. `perf-simple-query` --tablets --smp 1 -m 1G results obtained for fixed 400MHz frequency: Before: ``` enable-cache=1 Running test with config: {partitions=10000, concurrency=100, mode=read, frontend=cql, query_single_key=no, counters=no} Disabling auto compaction Creating 10000 partitions... 112590.60 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 41353 insns/op, 17992 cycles/op, 0 errors) 122620.68 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 41310 insns/op, 17713 cycles/op, 0 errors) 118169.48 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 41353 insns/op, 17857 cycles/op, 0 errors) 120634.65 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 41328 insns/op, 17733 cycles/op, 0 errors) 117317.18 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 41347 insns/op, 17822 cycles/op, 0 errors) throughput: mean=118266.52 standard-deviation=3797.81 median=118169.48 median-absolute-deviation=2368.13 maximum=122620.68 minimum=112590.60 instructions_per_op: mean=41337.86 standard-deviation=18.73 median=41346.89 median-absolute-deviation=14.64 maximum=41352.53 minimum=41309.83 cpu_cycles_per_op: mean=17823.50 standard-deviation=111.75 median=17821.97 median-absolute-deviation=90.45 maximum=17992.04 minimum=17713.00 ``` After ``` enable-cache=1 Running test with config: {partitions=10000, concurrency=100, mode=read, frontend=cql, query_single_key=no, counters=no} Disabling auto compaction Creating 10000 partitions... 123689.63 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 40997 insns/op, 17384 cycles/op, 0 errors) 129643.24 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 40997 insns/op, 17325 cycles/op, 0 errors) 128907.27 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 41009 insns/op, 17325 cycles/op, 0 errors) 130342.56 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 40993 insns/op, 17286 cycles/op, 0 errors) 130294.09 tps ( 63.1 allocs/op, 0.0 logallocs/op, 14.1 tasks/op, 40972 insns/op, 17336 cycles/op, 0 errors) throughput: mean=128575.36 standard-deviation=2792.75 median=129643.24 median-absolute-deviation=1718.73 maximum=130342.56 minimum=123689.63 instructions_per_op: mean=40993.51 standard-deviation=13.23 median=40996.73 median-absolute-deviation=3.30 maximum=41008.86 minimum=40972.48 cpu_cycles_per_op: mean=17331.16 standard-deviation=35.02 median=17324.84 median-absolute-deviation=6.49 maximum=17383.97 minimum=17286.33 ``` Closes scylladb/scylladb#21918 [avi: patch was co-authored by Łukasz Paszkowski ] --- reader_concurrency_semaphore.cc | 22 +++++++++++++------ reader_concurrency_semaphore.hh | 2 ++ .../reader_concurrency_semaphore_test.cc | 14 ++++++------ 3 files changed, 24 insertions(+), 14 deletions(-) diff --git a/reader_concurrency_semaphore.cc b/reader_concurrency_semaphore.cc index bd27b892b517..2dfc438265ef 100644 --- a/reader_concurrency_semaphore.cc +++ b/reader_concurrency_semaphore.cc @@ -967,6 +967,8 @@ future<> reader_concurrency_semaphore::execution_loop() noexcept { co_return; } + maybe_admit_waiters(); + while (!_ready_list.empty()) { auto& permit = _ready_list.front(); dequeue_permit(permit); @@ -1023,7 +1025,7 @@ void reader_concurrency_semaphore::consume(reader_permit::impl& permit, resource void reader_concurrency_semaphore::signal(const resources& r) noexcept { _resources += r; - maybe_admit_waiters(); + maybe_wake_execution_loop(); } namespace sm = seastar::metrics; @@ -1139,7 +1141,7 @@ reader_concurrency_semaphore::inactive_read_handle reader_concurrency_semaphore: permit->on_register_as_inactive(); if (_blessed_permit == &*permit) { _blessed_permit = nullptr; - maybe_admit_waiters(); + maybe_wake_execution_loop(); } if (!should_evict_inactive_read()) { try { @@ -1463,7 +1465,7 @@ future<> reader_concurrency_semaphore::do_wait_admission(reader_permit::impl& pe // So at any point in time, there should either be no waiters, or it // shouldn't be able to admit new reads. Otherwise something went wrong. maybe_dump_reader_permit_diagnostics(*this, "semaphore could admit new reads yet there are waiters", nullptr); - maybe_admit_waiters(); + maybe_wake_execution_loop(); } else if (admit == can_admit::maybe) { tracing::trace(permit.trace_state(), "[reader concurrency semaphore {}] evicting inactive reads in the background to free up resources", _name); ++_stats.reads_queued_with_eviction; @@ -1512,6 +1514,12 @@ void reader_concurrency_semaphore::maybe_admit_waiters() noexcept { } } +void reader_concurrency_semaphore::maybe_wake_execution_loop() noexcept { + if (!_wait_list.empty()) { + _ready_list_cv.signal(); + } +} + future<> reader_concurrency_semaphore::request_memory(reader_permit::impl& permit, size_t memory) { // Already blocked on memory? if (permit.get_state() == reader_permit::state::waiting_for_memory) { @@ -1568,7 +1576,7 @@ void reader_concurrency_semaphore::on_permit_destroyed(reader_permit::impl& perm --_stats.current_permits; if (_blessed_permit == &permit) { _blessed_permit = nullptr; - maybe_admit_waiters(); + maybe_wake_execution_loop(); } } @@ -1580,13 +1588,13 @@ void reader_concurrency_semaphore::on_permit_not_need_cpu() noexcept { SCYLLA_ASSERT(_stats.need_cpu_permits); --_stats.need_cpu_permits; SCYLLA_ASSERT(_stats.need_cpu_permits >= _stats.awaits_permits); - maybe_admit_waiters(); + maybe_wake_execution_loop(); } void reader_concurrency_semaphore::on_permit_awaits() noexcept { ++_stats.awaits_permits; SCYLLA_ASSERT(_stats.need_cpu_permits >= _stats.awaits_permits); - maybe_admit_waiters(); + maybe_wake_execution_loop(); } void reader_concurrency_semaphore::on_permit_not_awaits() noexcept { @@ -1652,7 +1660,7 @@ void reader_concurrency_semaphore::set_resources(resources r) { auto delta = r - _initial_resources; _initial_resources = r; _resources += delta; - maybe_admit_waiters(); + maybe_wake_execution_loop(); } void reader_concurrency_semaphore::broken(std::exception_ptr ex) { diff --git a/reader_concurrency_semaphore.hh b/reader_concurrency_semaphore.hh index 0daad1d25a4c..44aba5437b94 100644 --- a/reader_concurrency_semaphore.hh +++ b/reader_concurrency_semaphore.hh @@ -231,6 +231,8 @@ private: void maybe_admit_waiters() noexcept; + void maybe_wake_execution_loop() noexcept; + // Request more memory for the permit. // Request is instantly granted while memory consumption of all reads is // below _kill_limit_multiplier. diff --git a/test/boost/reader_concurrency_semaphore_test.cc b/test/boost/reader_concurrency_semaphore_test.cc index 613e8431b852..cb39f0990f22 100644 --- a/test/boost/reader_concurrency_semaphore_test.cc +++ b/test/boost/reader_concurrency_semaphore_test.cc @@ -851,6 +851,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_admission) { auto permit2 = semaphore.obtain_permit(schema, get_name(), 1024, db::timeout_clock::now(), {}).get(); auto irh2 = semaphore.register_inactive_read(make_empty_flat_reader_v2(s.schema(), permit2)); + BOOST_REQUIRE(eventually_true([&] { return !irh1 || !irh2; })); require_can_admit(true, "evictable reads"); } BOOST_REQUIRE_EQUAL(semaphore.available_resources(), initial_resources); @@ -870,7 +871,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_admission) { BOOST_REQUIRE_EQUAL(stats_after.reads_enqueued_for_admission, stats_before.reads_enqueued_for_admission + 1); BOOST_REQUIRE_EQUAL(semaphore.get_stats().waiters, 1); - std::ignore = post_enqueue_hook(cookie1); + [[maybe_unused]] auto guard = post_enqueue_hook(cookie1); if (!eventually_true([&] { return permit2_fut.available(); })) { semaphore.broken(); @@ -1163,9 +1164,9 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_evict_inactive_reads_ BOOST_REQUIRE_EQUAL(semaphore.available_resources().count, 0); BOOST_REQUIRE(irh1); - // Marking p2 as awaits should now allow p3 to be admitted by evicting p1 + // Marking p2 as awaits should eventually allow p3 to be admitted by evicting p1 rd2.mark_as_awaits(); - BOOST_REQUIRE_EQUAL(semaphore.get_stats().waiters, 0); + REQUIRE_EVENTUALLY_EQUAL(semaphore.get_stats().waiters, 0); BOOST_REQUIRE_EQUAL(semaphore.get_stats().need_cpu_permits, 1); BOOST_REQUIRE_EQUAL(semaphore.get_stats().awaits_permits, 1); BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 0); @@ -1210,7 +1211,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_set_resources) { BOOST_REQUIRE_EQUAL(semaphore.get_stats().waiters, 1); semaphore.set_resources({4, 4 * 1024}); - BOOST_REQUIRE_EQUAL(semaphore.get_stats().waiters, 0); + REQUIRE_EVENTUALLY_EQUAL(semaphore.get_stats().waiters, 0); BOOST_REQUIRE_EQUAL(semaphore.available_resources(), reader_resources(1, 1024)); BOOST_REQUIRE_EQUAL(semaphore.initial_resources(), reader_resources(4, 4 * 1024)); permit3_fut.get(); @@ -1992,7 +1993,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_necessary_evicting) { BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 1); ncpu_guard.reset(); - BOOST_REQUIRE(!handle); + REQUIRE_EVENTUALLY_EQUAL(bool(handle), false); BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 0); BOOST_REQUIRE_EQUAL(semaphore.get_stats().permit_based_evictions, ++evicted_reads); @@ -2021,8 +2022,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_necessary_evicting) { BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 1); ncpu_guard.reset(); - thread::yield(); // allow debug builds to schedule the fiber evicting the reads again - BOOST_REQUIRE(!handle); + REQUIRE_EVENTUALLY_EQUAL(bool(handle), false); BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 0); BOOST_REQUIRE_EQUAL(semaphore.get_stats().permit_based_evictions, ++evicted_reads); From 4260f3f55a00e7ff1d3d7f387ebc62921abb38b7 Mon Sep 17 00:00:00 2001 From: Evgeniy Naydanov Date: Thu, 26 Dec 2024 12:31:43 +0000 Subject: [PATCH 133/397] test.py: topology_random_failures: log randomization parameters in test Logging randomization parameters in the pytest_generate_tests hook doesn't play well for us. To make these parameters more visible move the logging to the test level. Closes scylladb/scylladb#22055 --- .../test_random_failures.py | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/test/topology_random_failures/test_random_failures.py b/test/topology_random_failures/test_random_failures.py index 1d7a8ada2ee8..c9f97f6218f8 100644 --- a/test/topology_random_failures/test_random_failures.py +++ b/test/topology_random_failures/test_random_failures.py @@ -35,9 +35,9 @@ # Following parameters can be adjusted to run same sequence of tests from a previous run. Look at logs for the values. # Also see `pytest_generate_tests()` below for details. -TESTS_SHUFFLE_SEED = None # seed for the tests order randomization -ERROR_INJECTIONS_COUNT = None # limit number of error injections -CLUSTER_EVENTS_COUNT = None # limit number of cluster events +TESTS_SHUFFLE_SEED = random.randrange(sys.maxsize) # seed for the tests order randomization +ERROR_INJECTIONS_COUNT = len(ERROR_INJECTIONS) # change it to limit number of error injections +CLUSTER_EVENTS_COUNT = len(CLUSTER_EVENTS) # change it to limit number of cluster events WAIT_FOR_IP_TIMEOUT = 30 # seconds @@ -49,8 +49,7 @@ def pytest_generate_tests(metafunc: pytest.Metafunc) -> None: cluster_events = CLUSTER_EVENTS[:CLUSTER_EVENTS_COUNT] tests = list(itertools.product(error_injections, cluster_events)) - seed = random.randrange(sys.maxsize) if TESTS_SHUFFLE_SEED is None else TESTS_SHUFFLE_SEED - random.Random(seed).shuffle(tests) + random.Random(TESTS_SHUFFLE_SEED).shuffle(tests) # Deselect unsupported combinations. Do it after the shuffle to have the stable order. tests = [ @@ -59,12 +58,6 @@ def pytest_generate_tests(metafunc: pytest.Metafunc) -> None: metafunc.parametrize(["error_injection", "cluster_event"], tests[:TESTS_COUNT]) - LOGGER.info( - "To repeat this run set TESTS_COUNT to %s, TESTS_SHUFFLE_SEED to %s, ERROR_INJECTIONS_COUNT to %s," - " and CLUSTER_EVENTS_COUNT to %s", - TESTS_COUNT, seed, len(error_injections), len(cluster_events), - ) - @pytest.fixture async def four_nodes_cluster(manager: ManagerClient) -> None: @@ -90,6 +83,12 @@ async def test_random_failures(manager: ManagerClient, random_tables: RandomTables, error_injection: str, cluster_event: ClusterEventType) -> None: + LOGGER.info( + "To repeat this run set TESTS_COUNT to %s, TESTS_SHUFFLE_SEED to %s, ERROR_INJECTIONS_COUNT to %s," + " and CLUSTER_EVENTS_COUNT to %s", + TESTS_COUNT, TESTS_SHUFFLE_SEED, ERROR_INJECTIONS_COUNT, CLUSTER_EVENTS_COUNT, + ) + table = await random_tables.add_table(ncolumns=5) await table.insert_seq() From 745b6d7d0dfba524e38335d5a8e768a6e3c86a3c Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 26 Dec 2024 11:38:13 +0200 Subject: [PATCH 134/397] gossiper: ignore gossiper entries with local host id in gossiper mode as well We already ignore a gossiper entries with host id equal to local host id in raft mode since those entries are just outdated entries since before ip change. The same logic applies to gossiper mode as well though, so do the same in both modes. Fixes: scylladb/scylladb#21930 Message-ID: --- gms/gossiper.cc | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/gms/gossiper.cc b/gms/gossiper.cc index 493be9a76ff7..f9ab541fae7c 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -647,16 +647,16 @@ future<> gossiper::apply_state_locally(std::map ma if (ep == get_broadcast_address()) { return make_ready_future<>(); } + locator::host_id hid = map[ep].get_host_id(); + if (hid == locator::host_id::create_null_id()) { + // If there is no host id in the new state there should be one locally + hid = get_host_id(ep); + } + if (hid == my_host_id()) { + logger.trace("Ignoring gossip for {} because it maps to local id, but is not local address", ep); + return make_ready_future<>(); + } if (_topo_sm) { - locator::host_id hid = map[ep].get_host_id(); - if (hid == locator::host_id::create_null_id()) { - // If there is no host id in the new state there should be one locally - hid = get_host_id(ep); - } - if (hid == my_host_id()) { - logger.trace("Ignoring gossip for {} because it maps to local id, but is not local address", ep); - return make_ready_future<>(); - } if (_topo_sm->_topology.left_nodes.contains(raft::server_id(hid.uuid()))) { logger.trace("Ignoring gossip for {} because it left", ep); return make_ready_future<>(); From 85bd799308a8cadd3a4adbea8a93efcbcb3a9a6e Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Sun, 15 Dec 2024 11:59:18 +0200 Subject: [PATCH 135/397] storage_service: replicate_to_all_cores: prevent stalls when preparing per-table erms Although the `network_topology_stratergy::make_replication_map` -> `tablet_aware_replication_strategy::do_make_replication_map` is not cpu intensive it still allocates and constructs a shared `tablet_effective_replication_map`, and that might stall with thousands of tablet-based tables. Therefore coroutinize the preparation loop to allow yielding. Signed-off-by: Benny Halevy --- service/storage_service.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 1e7fd9c67b43..f4fcc6d91e73 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -3119,10 +3119,10 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt } }); // Prepare per-table erms. - co_await container().invoke_on_all([&] (storage_service& ss) { + co_await container().invoke_on_all([&] (storage_service& ss) -> future<> { auto& db = ss._db.local(); auto tmptr = pending_token_metadata_ptr[this_shard_id()]; - db.get_tables_metadata().for_each_table([&] (table_id id, lw_shared_ptr table) { + co_await db.get_tables_metadata().for_each_table_gently([&] (table_id id, lw_shared_ptr table) { auto rs = db.find_keyspace(table->schema()->ks_name()).get_replication_strategy_ptr(); locator::effective_replication_map_ptr erm; if (auto pt_rs = rs->maybe_as_per_table()) { @@ -3135,6 +3135,7 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt } else { pending_table_erms[this_shard_id()].emplace(id, std::move(erm)); } + return make_ready_future(); }); }); } catch (...) { From c4db90799ab9c82b79dd0e519922e0c16333c1a8 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Tue, 31 Dec 2024 13:45:54 +0200 Subject: [PATCH 136/397] test: address_map: check generation handling during entry addition Check that adding an entry with smaller generation does not overwrite existing entry. --- test/boost/address_map_test.cc | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/test/boost/address_map_test.cc b/test/boost/address_map_test.cc index f8b0fb6da4b1..7594a2cd088e 100644 --- a/test/boost/address_map_test.cc +++ b/test/boost/address_map_test.cc @@ -246,6 +246,23 @@ SEASTAR_THREAD_TEST_CASE(test_address_map_operations) { scoped_no_abort_on_internal_error abort_guard; BOOST_CHECK_THROW(m.find_by_addr(gms::inet_address{}), std::runtime_error); } + { + // Check that an update with smaller generation will not overwrite update with larger one + // but other way around works + sharded> m_svc; + m_svc.start().get(); + auto stop_map = defer([&m_svc] { m_svc.stop().get(); }); + auto& m = m_svc.local(); + + m.add_or_update_entry(id1, addr1, gms::generation_type{2}); + m.add_or_update_entry(id1, addr2, gms::generation_type{1}); + + BOOST_CHECK(m.find(id1).value() == addr1); + + m.add_or_update_entry(id1, addr2, gms::generation_type{3}); + + BOOST_CHECK(m.find(id1).value() == addr2); + } } SEASTAR_THREAD_TEST_CASE(test_address_map_replication) { From c4b26ba8dc4d215f3b8704cbc3264105fc859cc0 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Tue, 31 Dec 2024 13:48:20 +0200 Subject: [PATCH 137/397] test: drop test_old_ip_notification_repro.py The test no longer test anything since the address map is updated much earlier now by the gossiper itself, not by the notifiers. The functionality is tested by a unit test now. --- gms/gossiper.cc | 18 ------- .../test_old_ip_notification_repro.py | 53 ------------------- 2 files changed, 71 deletions(-) delete mode 100644 test/topology_custom/test_old_ip_notification_repro.py diff --git a/gms/gossiper.cc b/gms/gossiper.cc index f9ab541fae7c..c19b2db0cda4 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -1730,24 +1730,6 @@ void gossiper::mark_alive(inet_address addr) { } future<> gossiper::real_mark_alive(inet_address addr) { - co_await utils::get_local_injector().inject("gossiper::real_mark_alive", [this, endpoint = addr] (auto& handler) -> future<> { - auto app_state_ptr = get_application_state_ptr(endpoint, application_state::HOST_ID); - if (!app_state_ptr) { - co_return; - } - - locator::host_id id(utils::UUID(app_state_ptr->value())); - auto second_node_ip = handler.get("second_node_ip"); - SCYLLA_ASSERT(second_node_ip); - - logger.info("real_mark_alive {}/{} second_node_ip={}", id, endpoint, *second_node_ip); - if (endpoint == gms::inet_address(sstring{*second_node_ip})) { - logger.info("Sleeping before real_mark_alive for {}/{}", id, endpoint); - co_await handler.wait_for_message(std::chrono::steady_clock::now() + std::chrono::minutes{1}); - logger.info("Finished sleeping before real_mark_alive for {}/{}", id, endpoint); - } - }); - auto permit = co_await lock_endpoint(addr, null_permit_id); // After sending echo message, the Node might not be in the diff --git a/test/topology_custom/test_old_ip_notification_repro.py b/test/topology_custom/test_old_ip_notification_repro.py deleted file mode 100644 index 3a1084d67346..000000000000 --- a/test/topology_custom/test_old_ip_notification_repro.py +++ /dev/null @@ -1,53 +0,0 @@ -# -# Copyright (C) 2023-present ScyllaDB -# -# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -# - -import logging -import time -import pytest - -from test.pylib.manager_client import ManagerClient -from test.pylib.rest_client import inject_error, read_barrier -from test.pylib.util import wait_for_cql_and_get_hosts -from test.topology.conftest import skip_mode - - -logger = logging.getLogger(__name__) - - -@pytest.mark.asyncio -@skip_mode('release', 'error injections are not supported in release mode') -async def test_old_ip_notification_repro(manager: ManagerClient) -> None: - """ - Regression test for #14257. - It starts two nodes. It introduces a sleep in gossiper::real_mark_alive - when receiving a gossip notification about - HOST_ID update from the second node. Then it restarts the second node with - a different IP. Due to the sleep, the old notification from the old IP arrives - after the second node has restarted. If the bug is present, this notification - overrides the address map entry and the second read barrier times out, since - the first node cannot reach the second node with the old IP. - """ - s1 = await manager.server_add() - s2 = await manager.server_add(start=False) - async with inject_error(manager.api, s1.ip_addr, 'gossiper::real_mark_alive', - parameters={ "second_node_ip": s2.ip_addr }) as handler: - # This injection delays the gossip notification from the initial IP of s2. - logger.info(f"Starting {s2}") - await manager.server_start(s2.server_id) - logger.info(f"Stopping {s2}") - await manager.server_stop_gracefully(s2.server_id) - await manager.server_change_ip(s2.server_id) - logger.info(f"Starting {s2}") - await manager.server_start(s2.server_id) - logger.info(f"Wait for cql") - await manager.get_ready_cql([s1]) - logger.info(f"Read barrier") - await read_barrier(manager.api, s1.ip_addr) # Wait for s1 to be aware of s2 with the new IP. - await handler.message() # s1 receives the gossip notification from the initial IP of s2. - logger.info(f"Read barrier") - # If IP of s2 is overridden by its initial IP, the read barrier should time out. - await read_barrier(manager.api, s1.ip_addr) - logger.info(f"Done") From 48e7ffc3005a6887e5ff3b7a345926544f3977f1 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Thu, 12 Dec 2024 09:03:31 +0100 Subject: [PATCH 138/397] qos: return correct error code when SL does not exist The `nonexistant_service_level_exception` can be thrown by service levels code and propagated up to the CQL server layer, where it is converted into a CQL protocol error. The aforementioned exception inherits from `service_level_argument_exception`, which in turn inherits from `std::invalid_argument` - which doesn't mean much to the CQL layer and is converted to a generic SERVER_ERROR. We can do better and return a more meaningful error code for this exception. Change the base class of service_level_argument_exception to exceptions::invalid_request_exception which gets converted to an INVALID error. The INVALID error code was already being used by the enterprise version, so this commit just synchronizes error handling with enterprise. --- service/qos/qos_common.hh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/service/qos/qos_common.hh b/service/qos/qos_common.hh index 16efa26037c9..5d366e9e9380 100644 --- a/service/qos/qos_common.hh +++ b/service/qos/qos_common.hh @@ -17,6 +17,7 @@ #include #include #include +#include "exceptions/exceptions.hh" namespace cql3 { class query_processor; @@ -90,9 +91,9 @@ using service_levels_info = std::map; /// /// A logical argument error for a service_level statement operation. /// -class service_level_argument_exception : public std::invalid_argument { +class service_level_argument_exception : public exceptions::invalid_request_exception { public: - using std::invalid_argument::invalid_argument; + using exceptions::invalid_request_exception::invalid_request_exception; }; /// From 75d2d0d949595095496ff7c216b6b8c8cead45ae Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 18:19:06 +0100 Subject: [PATCH 139/397] build: increase the max number of scheduling groups Workload prioritization assigns scheduling groups to service levels, and the number of scheduling groups that can exist at the same time is limited with a compile-time parameter in seastar. The documentation for workload prioritization says that we currently support 7 user-managed service levels and 1 created by default. Increase the current compile-time limit in order to align with the documentation. --- CMakeLists.txt | 2 +- configure.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d0f29c1fe070..9c2b90bab251 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -95,7 +95,7 @@ else() set(Seastar_EXCLUDE_APPS_FROM_ALL ON CACHE BOOL "" FORCE) set(Seastar_EXCLUDE_TESTS_FROM_ALL ON CACHE BOOL "" FORCE) set(Seastar_IO_URING ON CACHE BOOL "" FORCE) - set(Seastar_SCHEDULING_GROUPS_COUNT 16 CACHE STRING "" FORCE) + set(Seastar_SCHEDULING_GROUPS_COUNT 19 CACHE STRING "" FORCE) set(Seastar_UNUSED_RESULT_ERROR ON CACHE BOOL "" FORCE) add_subdirectory(seastar) target_compile_definitions (seastar diff --git a/configure.py b/configure.py index d93835ff29dd..f8e8fd407a56 100755 --- a/configure.py +++ b/configure.py @@ -1871,7 +1871,7 @@ def configure_seastar(build_dir, mode, mode_config): '-DSeastar_DEPRECATED_OSTREAM_FORMATTERS=OFF', '-DSeastar_UNUSED_RESULT_ERROR=ON', '-DCMAKE_EXPORT_COMPILE_COMMANDS=ON', - '-DSeastar_SCHEDULING_GROUPS_COUNT=16', + '-DSeastar_SCHEDULING_GROUPS_COUNT=19', '-DSeastar_IO_URING=ON', ] From ecbf8721de3fdf9f89f4d6cc5fb7f788d6387748 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 16:52:02 +0100 Subject: [PATCH 140/397] gms: introduce WORKLOAD_PRIORITIZATION cluster feature Information about the number of shares per service level will be stored in an additional column in the service levels table, which is managed through group0. We will need the feature to make sure that all nodes in the cluster know about the new column before any node starts applying group0 commands the would touch the new column. This feature also serves a role for the legacy service levels implementation that uses system_distributed for storage: after all nodes are upgraded to support workload prioritization, one of the nodes will perform a schema change operation and will add the new column. --- gms/feature_service.hh | 1 + 1 file changed, 1 insertion(+) diff --git a/gms/feature_service.hh b/gms/feature_service.hh index dd585cb7937b..d52056d5df83 100644 --- a/gms/feature_service.hh +++ b/gms/feature_service.hh @@ -156,6 +156,7 @@ public: gms::feature test_only_feature { *this, "TEST_ONLY_FEATURE"sv }; gms::feature address_nodes_by_host_ids { *this, "ADDRESS_NODES_BY_HOST_IDS"sv }; + gms::feature workload_prioritization { *this, "WORKLOAD_PRIORITIZATION"sv }; gms::feature compression_dicts { *this, "COMPRESSION_DICTS"sv }; public: From 346fc84c3ecf2f9808defc57fd4f53a8d67f7af8 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 17:57:48 +0100 Subject: [PATCH 141/397] db/system_keyspace: adjust SL schema for workload prioritization Add a "shares" column which hold the number of shares allocated to given service level. It is not used by the code at all right now, subsequent commits will make good use of it. --- db/system_keyspace.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 93531455bc0a..1304601472d8 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -1157,6 +1157,7 @@ schema_ptr system_keyspace::service_levels_v2() { .with_column("service_level", utf8_type, column_kind::partition_key) .with_column("timeout", duration_type) .with_column("workload_type", utf8_type) + .with_column("shares", int32_type) .with_hash_version() .build(); }(); From ea25b296844cf9b84fb111d0767c5c89f8a3076d Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 10:38:41 +0100 Subject: [PATCH 142/397] db/system_distributed_keyspace: add shares column and upgrade code Add the "shares" column to the system_distributed_keyspace.service_levels table, which is used by legacy code. Because this table is in a distributed and not local keyspace, adding the column to an existing cluster during rolling upgrade requires a bit of care. A callback is added to the workload prioritization cluster feature which runs when the feature becomes enabled and adds the column for all nodes in the cluster. --- db/system_distributed_keyspace.cc | 59 +++++++++++++++++++++++++------ db/system_distributed_keyspace.hh | 5 +++ service/storage_service.cc | 31 ++++++++++++++++ service/storage_service.hh | 2 ++ 4 files changed, 86 insertions(+), 11 deletions(-) diff --git a/db/system_distributed_keyspace.cc b/db/system_distributed_keyspace.cc index adae10f3b5e9..121e9790a59f 100644 --- a/db/system_distributed_keyspace.cc +++ b/db/system_distributed_keyspace.cc @@ -13,6 +13,7 @@ #include "replica/database.hh" #include "db/consistency_level_type.hh" #include "db/system_keyspace.hh" +#include "db/config.hh" #include "schema/schema_builder.hh" #include "timeout_config.hh" #include "types/types.hh" @@ -21,6 +22,8 @@ #include "cdc/generation.hh" #include "cql3/query_processor.hh" #include "service/storage_proxy.hh" +#include "gms/feature_service.hh" + #include "service/migration_manager.hh" #include "locator/host_id.hh" @@ -152,8 +155,14 @@ static const sstring CDC_TIMESTAMPS_KEY = "timestamps"; schema_ptr service_levels() { static thread_local auto schema = [] { auto id = generate_legacy_id(system_distributed_keyspace::NAME, system_distributed_keyspace::SERVICE_LEVELS); - return schema_builder(system_distributed_keyspace::NAME, system_distributed_keyspace::SERVICE_LEVELS, std::make_optional(id)) + auto builder = schema_builder(system_distributed_keyspace::NAME, system_distributed_keyspace::SERVICE_LEVELS, std::make_optional(id)) .with_column("service_level", utf8_type, column_kind::partition_key) + .with_column("shares", int32_type); + if (utils::get_local_injector().is_enabled("service_levels_v1_table_without_shares")) { + builder.remove_column("shares"); + } + + return builder .with_hash_version() .build(); }(); @@ -207,9 +216,12 @@ system_distributed_keyspace::system_distributed_keyspace(cql3::query_processor& , _sp(sp) { } -static thread_local std::pair new_columns[] { - {"timeout", duration_type}, - {"workload_type", utf8_type} +static std::vector> new_service_levels_columns(bool workload_prioritization_enabled) { + std::vector> new_columns {{"timeout", duration_type}, {"workload_type", utf8_type}}; + if (workload_prioritization_enabled) { + new_columns.push_back({"shares", int32_type}); + } + return new_columns; }; static schema_ptr get_current_service_levels(data_dictionary::database db) { @@ -218,11 +230,11 @@ static schema_ptr get_current_service_levels(data_dictionary::database db) { : service_levels(); } -static schema_ptr get_updated_service_levels(data_dictionary::database db) { +static schema_ptr get_updated_service_levels(data_dictionary::database db, bool workload_prioritization_enabled) { SCYLLA_ASSERT(this_shard_id() == 0); auto schema = get_current_service_levels(db); schema_builder b(schema); - for (const auto& col : new_columns) { + for (const auto& col : new_service_levels_columns(workload_prioritization_enabled)) { auto& [col_name, col_type] = col; bytes options_name = to_bytes(col_name.data()); if (schema->get_column_definition(options_name)) { @@ -234,20 +246,20 @@ static schema_ptr get_updated_service_levels(data_dictionary::database db) { return b.build(); } -future<> system_distributed_keyspace::start() { +future<> system_distributed_keyspace::create_tables(std::vector tables) { if (this_shard_id() != 0) { _started = true; co_return; } auto db = _sp.data_dictionary(); - auto tables = ensured_tables(); while (true) { // Check if there is any work to do before taking the group 0 guard. + bool workload_prioritization_enabled = _sp.features().workload_prioritization; bool keyspaces_setup = db.has_keyspace(NAME) && db.has_keyspace(NAME_EVERYWHERE); bool tables_setup = std::all_of(tables.begin(), tables.end(), [db] (schema_ptr t) { return db.has_schema(t->ks_name(), t->cf_name()); } ); - bool service_levels_up_to_date = get_current_service_levels(db)->equal_columns(*get_updated_service_levels(db)); + bool service_levels_up_to_date = get_current_service_levels(db)->equal_columns(*get_updated_service_levels(db, workload_prioritization_enabled)); if (keyspaces_setup && tables_setup && service_levels_up_to_date) { dlogger.info("system_distributed(_everywhere) keyspaces and tables are up-to-date. Not creating"); _started = true; @@ -287,12 +299,12 @@ future<> system_distributed_keyspace::start() { // Get mutations for creating and updating tables. auto num_keyspace_mutations = mutations.size(); co_await coroutine::parallel_for_each(ensured_tables(), - [this, &mutations, db, ts, sd_ksm, sde_ksm] (auto&& table) -> future<> { + [this, &mutations, db, ts, sd_ksm, sde_ksm, workload_prioritization_enabled] (auto&& table) -> future<> { auto ksm = table->ks_name() == NAME ? sd_ksm : sde_ksm; // Ensure that the service_levels table contains new columns. if (table->cf_name() == SERVICE_LEVELS) { - table = get_updated_service_levels(db); + table = get_updated_service_levels(db, workload_prioritization_enabled); } if (!db.has_schema(table->ks_name(), table->cf_name())) { @@ -325,6 +337,24 @@ future<> system_distributed_keyspace::start() { } } + future<> system_distributed_keyspace::start_workload_prioritization() { + if (this_shard_id() != 0) { + co_return; + } + if (_qp.db().features().workload_prioritization) { + co_await create_tables({get_updated_service_levels(_qp.db(), true)}); + } +} + +future<> system_distributed_keyspace::start() { + if (this_shard_id() != 0) { + _started = true; + co_return; + } + + co_await create_tables(ensured_tables()); +} + future<> system_distributed_keyspace::stop() { return make_ready_future<>(); } @@ -740,6 +770,13 @@ system_distributed_keyspace::get_cdc_desc_v1_timestamps(context ctx) { co_return res; } +bool system_distributed_keyspace::workload_prioritization_tables_exists() { + auto wp_table = get_updated_service_levels(_qp.db(), true); + auto table = _qp.db().try_find_table(NAME, wp_table->cf_name()); + + return table && table->schema()->equal_columns(*wp_table); +} + future system_distributed_keyspace::get_service_levels(qos::query_context ctx) const { return qos::get_service_levels(_qp, NAME, SERVICE_LEVELS, db::consistency_level::ONE, ctx); } diff --git a/db/system_distributed_keyspace.hh b/db/system_distributed_keyspace.hh index 6a3a9ab76674..20a8418f4932 100644 --- a/db/system_distributed_keyspace.hh +++ b/db/system_distributed_keyspace.hh @@ -82,6 +82,7 @@ public: system_distributed_keyspace(cql3::query_processor&, service::migration_manager&, service::storage_proxy&); future<> start(); + future<> start_workload_prioritization(); future<> stop(); bool started() const { return _started; } @@ -116,6 +117,10 @@ public: future get_service_level(sstring service_level_name) const; future<> set_service_level(sstring service_level_name, qos::service_level_options slo) const; future<> drop_service_level(sstring service_level_name) const; + bool workload_prioritization_tables_exists(); + +private: + future<> create_tables(std::vector tables); }; } diff --git a/service/storage_service.cc b/service/storage_service.cc index f4fcc6d91e73..fed207321ae3 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -108,6 +108,8 @@ #include "service/topology_mutation.hh" #include "service/topology_coordinator.hh" #include "cql3/query_processor.hh" +#include "service/qos/service_level_controller.hh" +#include "service/qos/standard_service_level_distributed_data_accessor.hh" #include #include @@ -2043,6 +2045,35 @@ future<> storage_service::join_topology(sharded co_await _sys_ks.local().cdc_set_rewritten(std::nullopt); } + // now, that the system distributed keyspace is initialized and started, + // pass an accessor to the service level controller so it can interact with it + // but only if the conditions are right (the cluster supports or have supported + // workload prioritization before): + if (!sys_dist_ks.local().workload_prioritization_tables_exists()) { + // if we got here, it means that the workload priotization didn't exist before and + // also that the cluster currently doesn't support workload prioritization. + // we delay the creation of the tables and accessing them until it does. + // + // the callback might be run immediately and it uses async methods, so the thread is needed + co_await seastar::async([&] { + _workload_prioritization_registration = _feature_service.workload_prioritization.when_enabled([&sys_dist_ks] () { + // since we are creating tables here and we wouldn't want to have a race condition + // we will first wait for a random period of time and only then start the routine + // the race condition can happen because the feature flag will "light up" in about + // the same time on all nodes. The more nodes there are, the higher the chance for + // a race. + std::random_device seed_gen; + std::default_random_engine rnd_engine(seed_gen()); + std::uniform_int_distribution<> delay_generator(0,5000000); + sleep(std::chrono::microseconds(delay_generator(rnd_engine))).get(); + sys_dist_ks.invoke_on_all(&db::system_distributed_keyspace::start_workload_prioritization).get(); + slogger.info("Workload prioritization v1 started."); + }); + }); + } else { + slogger.info("Workload prioritization v1 is already started."); + } + if (!cdc_gen_id) { // If we didn't observe any CDC generation at this point, then either // 1. we're replacing a node, diff --git a/service/storage_service.hh b/service/storage_service.hh index 2b6579852857..0d44ed864c0c 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -28,6 +28,7 @@ #include "dht/token_range_endpoints.hh" #include #include "gms/application_state.hh" +#include "gms/feature.hh" #include #include #include "replica/database_fwd.hh" @@ -174,6 +175,7 @@ private: using client_shutdown_hook = noncopyable_function; std::vector _protocol_servers; std::vector _listeners; + gms::feature::listener_registration _workload_prioritization_registration; gate _async_gate; condition_variable _tablet_split_monitor_event; From 2eb35f37d0c07468d73a4c1de195f740f84a0741 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Thu, 12 Dec 2024 08:47:51 +0100 Subject: [PATCH 143/397] qos: explicitly specify columns when querying service level tables The service levels table is queried with a `SELECT * ...` query, by using the `execute_internal` method which prepares and caches the query in an special cache for internal queries, separate from the user query cache. During rolling upgrade from a version which does not support service level shares to the one that does, the `shares` column is added. The aforementioned internal query cache is _not_ invalidated on schema change, so the cache might still contain the prepared query from the time before the column was added, and that prepared query will fetch the old set of column without the new `shares` column. In order to solve this, explicitly specify the columns in the query string, using the full set of column names from the time when the query is executed. Note that this is a problem only for the legacy, non-raft service levels. Raft-based service levels use a local table for which the schema is determined on startup. Also note that this code only fetches values from the `shares` column but does not make any use of it otherwise. It will be handled by later commits in this series. --- service/qos/qos_common.cc | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index ad782ded1c59..f132d95cc8ae 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -11,6 +11,8 @@ #include "cql3/query_processor.hh" #include "cql3/result_set.hh" #include "cql3/untyped_result_set.hh" +#include +#include #include namespace qos { @@ -144,8 +146,15 @@ static service_level_options::timeout_type get_duration(const cql3::untyped_resu return std::chrono::duration_cast(std::chrono::nanoseconds(dur_opt->nanoseconds)); }; +static sstring get_columns(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name) { + auto schema = qp.db().find_schema(ks_name, cf_name); + return boost::algorithm::join(schema->all_columns() | boost::adaptors::transformed([] (const auto& col) { + return col.name_as_cql_string(); + }), " ,"); +} + future get_service_levels(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name, db::consistency_level cl, qos::query_context ctx) { - sstring prepared_query = seastar::format("SELECT * FROM {}.{};", ks_name, cf_name); + sstring prepared_query = seastar::format("SELECT {} FROM {}.{};", get_columns(qp, ks_name, cf_name), ks_name, cf_name); auto result_set = co_await qp.execute_internal(prepared_query, cl, qos_query_state(ctx), cql3::query_processor::cache_internal::yes); qos::service_levels_info service_levels; @@ -167,7 +176,7 @@ future get_service_levels(cql3::query_processor& qp, s } future get_service_level(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name, sstring service_level_name, db::consistency_level cl) { - sstring prepared_query = seastar::format("SELECT * FROM {}.{} WHERE service_level = ?;", ks_name, cf_name); + sstring prepared_query = seastar::format("SELECT {} FROM {}.{} WHERE service_level = ?;", get_columns(qp, ks_name, cf_name), ks_name, cf_name); auto result_set = co_await qp.execute_internal(prepared_query, cl, qos_query_state(), {service_level_name}, cql3::query_processor::cache_internal::yes); qos::service_levels_info service_levels; From a6f681029f1709c84af5baf2dc1d1247256b03d5 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Sat, 21 Dec 2024 16:41:14 +0100 Subject: [PATCH 144/397] qos: add shares to service_level_options Add service level shares related fields to service_level_options and slo_effective_names structs, and adjust the existing methods of the former (merge_with, init_effective_names) to account for them. --- service/qos/qos_common.cc | 43 ++++++++++++++++++++++++++++++++++++++- service/qos/qos_common.hh | 7 +++++++ 2 files changed, 49 insertions(+), 1 deletion(-) diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index f132d95cc8ae..29eec8f8a951 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -45,6 +45,19 @@ service_level_options service_level_options::replace_defaults(const service_leve // no-op break; } + std::visit(overloaded_functor { + [&] (const unset_marker& um) { + // reset the value to the default one + ret.shares = default_values.shares; + }, + [&] (const delete_marker& dm) { + // remove the value + ret.shares = unset_marker{}; + }, + [&] (const int32_t&) { + // leave the value as is + }, + }, ret.shares); return ret; } @@ -54,6 +67,11 @@ service_level_options service_level_options::merge_with(const service_level_opti slo.effective_names->timeout = other.effective_names->timeout; } }; + auto maybe_update_shares_name = [] (service_level_options& slo, const service_level_options& other) { + if (slo.effective_names && other.effective_names) { + slo.effective_names->shares = other.effective_names->shares; + } + }; auto maybe_update_workload_name = [] (service_level_options& slo, const service_level_options& other) { if (slo.effective_names && other.effective_names) { slo.effective_names->workload = other.effective_names->workload; @@ -93,6 +111,28 @@ service_level_options service_level_options::merge_with(const service_level_opti maybe_update_workload_name(ret, other); } + std::visit(overloaded_functor { + [&] (const unset_marker& um) { + ret.shares = other.shares; + maybe_update_shares_name(ret, other); + }, + [&] (const delete_marker& dm) { + ret.shares = other.shares; + maybe_update_shares_name(ret, other); + }, + [&] (const int32_t& s) { + if (auto* other_shares = std::get_if(&other.shares)) { + auto prev_shares = ret.shares; + ret.shares = std::min(s, *other_shares); + + if (prev_shares != ret.shares) { + ret.shares_name = other.shares_name; + maybe_update_shares_name(ret, other); + } + } + }, + }, ret.shares); + return ret; } @@ -124,7 +164,8 @@ std::optional service_level_options::parse void service_level_options::init_effective_names(std::string_view service_level_name) { effective_names = service_level_options::slo_effective_names { .timeout = sstring(service_level_name), - .workload = sstring(service_level_name) + .workload = sstring(service_level_name), + .shares = sstring(service_level_name), }; } diff --git a/service/qos/qos_common.hh b/service/qos/qos_common.hh index 5d366e9e9380..af7f223febaa 100644 --- a/service/qos/qos_common.hh +++ b/service/qos/qos_common.hh @@ -17,6 +17,7 @@ #include #include #include +#include #include "exceptions/exceptions.hh" namespace cql3 { @@ -62,6 +63,11 @@ struct service_level_options { timeout_type timeout = unset_marker{}; workload_type workload = workload_type::unspecified; + using shares_type = std::variant; + shares_type shares = unset_marker{}; + + std::optional shares_name; // service level name, if shares is set + service_level_options replace_defaults(const service_level_options& other) const; // Merges the values of two service level options. The semantics depends // on the type of the parameter - e.g. for timeouts, a min value is preferred. @@ -75,6 +81,7 @@ struct service_level_options { struct slo_effective_names { sstring timeout; sstring workload; + sstring shares; bool operator==(const slo_effective_names& other) const = default; bool operator!=(const slo_effective_names& other) const = default; From ff51551a94662679658728ec91c5dd27635acef2 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Sat, 21 Dec 2024 16:48:44 +0100 Subject: [PATCH 145/397] qos: use the shares field in service level reads/writes Now, the newly introduced `shares` field is used when service levels are either read from or written into system tables. --- db/system_distributed_keyspace.cc | 18 ++++++++++++++++++ main.cc | 1 + service/qos/qos_common.cc | 11 +++++++++++ ..._service_level_distributed_data_accessor.cc | 15 +++++++++++++++ service/qos/service_level_controller.hh | 2 ++ test/lib/cql_test_env.cc | 2 +- 6 files changed, 48 insertions(+), 1 deletion(-) diff --git a/db/system_distributed_keyspace.cc b/db/system_distributed_keyspace.cc index 121e9790a59f..0703fee77701 100644 --- a/db/system_distributed_keyspace.cc +++ b/db/system_distributed_keyspace.cc @@ -803,6 +803,19 @@ future<> system_distributed_keyspace::set_service_level(sstring service_level_na }, }, tv); }; + auto to_data_value_g = [&] (const std::variant& v) { + return std::visit(overloaded_functor { + [&] (const qos::service_level_options::unset_marker&) { + return data_value::make_null(data_type_for()); + }, + [&] (const qos::service_level_options::delete_marker&) { + return data_value::make_null(data_type_for()); + }, + [&] (const T& v) { + return data_value(v); + }, + }, v); + }; data_value workload = slo.workload == qos::service_level_options::workload_type::unspecified ? data_value::make_null(utf8_type) : data_value(qos::service_level_options::to_string(slo.workload)); @@ -813,6 +826,11 @@ future<> system_distributed_keyspace::set_service_level(sstring service_level_na workload, service_level_name}, cql3::query_processor::cache_internal::no); + co_await _qp.execute_internal(format("UPDATE {}.{} SET shares = ? WHERE service_level = ?;", NAME, SERVICE_LEVELS), + db::consistency_level::ONE, + internal_distributed_query_state(), + {to_data_value_g(slo.shares), service_level_name}, + cql3::query_processor::cache_internal::no); } future<> system_distributed_keyspace::drop_service_level(sstring service_level_name) const { diff --git a/main.cc b/main.cc index a0c9e4dbd5e1..c802fa978f64 100644 --- a/main.cc +++ b/main.cc @@ -1346,6 +1346,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl //starting service level controller qos::service_level_options default_service_level_configuration; + default_service_level_configuration.shares = 1000; sl_controller.start(std::ref(auth_service), std::ref(token_metadata), std::ref(stop_signal.as_sharded_abort_source()), default_service_level_configuration).get(); sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); auto stop_sl_controller = defer_verbose_shutdown("service level controller", [] { diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index 29eec8f8a951..59e20e41889c 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -7,6 +7,7 @@ */ #include "qos_common.hh" +#include "service/qos/service_level_controller.hh" #include "utils/overloaded_functor.hh" #include "cql3/query_processor.hh" #include "cql3/result_set.hh" @@ -187,6 +188,14 @@ static service_level_options::timeout_type get_duration(const cql3::untyped_resu return std::chrono::duration_cast(std::chrono::nanoseconds(dur_opt->nanoseconds)); }; +static qos::service_level_options::shares_type get_shares(const cql3::untyped_result_set_row& row, std::string_view col_name) { + auto shares_opt = row.get_opt(col_name); + if (!shares_opt) { + return qos::service_level_controller::default_shares; + } + return *shares_opt; +} + static sstring get_columns(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name) { auto schema = qp.db().find_schema(ks_name, cf_name); return boost::algorithm::join(schema->all_columns() | boost::adaptors::transformed([] (const auto& col) { @@ -206,6 +215,7 @@ future get_service_levels(cql3::query_processor& qp, s qos::service_level_options slo{ .timeout = get_duration(row, "timeout"), .workload = workload.value_or(qos::service_level_options::workload_type::unspecified), + .shares = get_shares(row, "shares"), }; service_levels.emplace(service_level_name, slo); } catch (...) { @@ -228,6 +238,7 @@ future get_service_level(cql3::query_processor& qp, std::st qos::service_level_options slo{ .timeout = get_duration(row, "timeout"), .workload = workload.value_or(qos::service_level_options::workload_type::unspecified), + .shares = get_shares(row, "shares"), }; service_levels.emplace(service_level_name, slo); } catch (...) { diff --git a/service/qos/raft_service_level_distributed_data_accessor.cc b/service/qos/raft_service_level_distributed_data_accessor.cc index 588081f37988..3bcf124cd124 100644 --- a/service/qos/raft_service_level_distributed_data_accessor.cc +++ b/service/qos/raft_service_level_distributed_data_accessor.cc @@ -60,11 +60,26 @@ future<> raft_service_level_distributed_data_accessor::set_service_level(sstring validate_state(_group0_client); static sstring insert_query = format("INSERT INTO {}.{} (service_level, timeout, workload_type) VALUES (?, ?, ?);", db::system_keyspace::NAME, db::system_keyspace::SERVICE_LEVELS_V2); + static sstring update_shares_query = format("UPDATE {}.{} SET shares = ? WHERE service_level = ?", db::system_keyspace::NAME, db::system_keyspace::SERVICE_LEVELS_V2); data_value workload = slo.workload == qos::service_level_options::workload_type::unspecified ? data_value::make_null(utf8_type) : data_value(qos::service_level_options::to_string(slo.workload)); auto muts = co_await _qp.get_mutations_internal(insert_query, qos_query_state(), mc.write_timestamp(), {service_level_name, timeout_to_data_value(slo.timeout), workload}); + auto muts_shares = co_await std::visit(overloaded_functor { + [&] (const service_level_options::unset_marker& um) -> future> { + co_return std::vector(); + }, + [&] (const service_level_options::delete_marker& dm) -> future> { + co_return co_await _qp.get_mutations_internal(update_shares_query, qos_query_state(), mc.write_timestamp(), {data_value::make_null(int32_type), data_value(service_level_name)}); + }, + [&] (const int32_t& s) -> future> { + co_return co_await _qp.get_mutations_internal(update_shares_query, qos_query_state(), mc.write_timestamp(), {data_value(s), data_value(service_level_name)}); + } + }, slo.shares); + + muts.insert(muts.end(), muts_shares.begin(), muts_shares.end()); + mc.add_mutations(std::move(muts), format("service levels internal statement: {}", insert_query)); } diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 46b208279153..453932b104e9 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -99,6 +99,8 @@ using update_both_cache_levels = bool_class; */ class service_level_controller : public peering_sharded_service, public service::endpoint_lifecycle_subscriber { public: + static inline const int32_t default_shares = 1000; + class service_level_distributed_data_accessor { public: virtual future get_service_levels(qos::query_context ctx = qos::query_context::unspecified) const = 0; diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index cef4094bad2e..5946864edff1 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -659,7 +659,7 @@ class single_node_cql_env : public cql_test_env { set_abort_on_internal_error(true); const gms::inet_address listen("127.0.0.1"); - _sl_controller.start(std::ref(_auth_service), std::ref(_token_metadata), std::ref(abort_sources), qos::service_level_options{}).get(); + _sl_controller.start(std::ref(_auth_service), std::ref(_token_metadata), std::ref(abort_sources), qos::service_level_options{.shares = 1000}).get(); auto stop_sl_controller = defer([this] { _sl_controller.stop().get(); }); _sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); From 4cfd26efaff39662eb8f799350332441376a769a Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 08:14:16 +0100 Subject: [PATCH 146/397] qos: manage and assign scheduling groups to service levels Introduce the core logic of workload prioritization, responsible for assigning scheduling groups to service levels. The service level controller maintains a pool of scheduling groups for the currently present service levels, as well as a pool of unused scheduling groups which were previously used by some service level that was deleted during node's lifetime. When a new service level is created, the SL controller either assigns a scheduling group from the unused SG pool, or creates a new one if the pool is empty. The scheduling group is renamed to "sl:". When updating shares of a service level (and also when creating a new service level), the shares of the corresponding scheduling group are synchronized with those of the service level. When a service level is deleted, its group is released to the aforementioned pool of unused scheduling groups and the prefix of its name is changed from "sl:" to "sl_deleted:". For now, these scheduling groups are not used by any user operations. This will be changed in subsequent commits. --- main.cc | 2 +- service/qos/qos_common.cc | 16 ++ service/qos/qos_common.hh | 23 ++ .../qos_configuration_change_subscriber.hh | 1 + service/qos/service_level_controller.cc | 245 +++++++++++++++--- service/qos/service_level_controller.hh | 31 ++- test/boost/service_level_controller_test.cc | 5 +- test/lib/cql_test_env.cc | 2 +- test/topology_custom/test_alternator.py | 3 +- 9 files changed, 285 insertions(+), 43 deletions(-) diff --git a/main.cc b/main.cc index c802fa978f64..b0fe95778ef0 100644 --- a/main.cc +++ b/main.cc @@ -1347,7 +1347,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl //starting service level controller qos::service_level_options default_service_level_configuration; default_service_level_configuration.shares = 1000; - sl_controller.start(std::ref(auth_service), std::ref(token_metadata), std::ref(stop_signal.as_sharded_abort_source()), default_service_level_configuration).get(); + sl_controller.start(std::ref(auth_service), std::ref(token_metadata), std::ref(stop_signal.as_sharded_abort_source()), default_service_level_configuration, dbcfg.statement_scheduling_group).get(); sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); auto stop_sl_controller = defer_verbose_shutdown("service level controller", [] { sl_controller.stop().get(); diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index 59e20e41889c..e95733aa285c 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -137,6 +137,14 @@ service_level_options service_level_options::merge_with(const service_level_opti return ret; } +sstring service_level_options::to_string(timeout_type tt) { + return std::visit(make_visitor( + [] (unset_marker) -> sstring { return "null"; }, + [] (delete_marker) -> sstring { return ""; }, + [] (lowres_clock::duration value) { return seastar::format("{}", value); } + ), tt); +} + std::string_view service_level_options::to_string(const workload_type& wt) { switch (wt) { case workload_type::unspecified: return "unspecified"; @@ -162,6 +170,14 @@ std::optional service_level_options::parse return std::nullopt; } +sstring service_level_options::to_string(shares_type st) { + return std::visit(make_visitor( + [] (unset_marker) -> sstring { return "default"; }, + [] (delete_marker) -> sstring { return ""; }, + [] (int32_t value) { return seastar::format("{}", value); } + ), st); +} + void service_level_options::init_effective_names(std::string_view service_level_name) { effective_names = service_level_options::slo_effective_names { .timeout = sstring(service_level_name), diff --git a/service/qos/qos_common.hh b/service/qos/qos_common.hh index af7f223febaa..0991bd217094 100644 --- a/service/qos/qos_common.hh +++ b/service/qos/qos_common.hh @@ -75,9 +75,13 @@ struct service_level_options { bool operator==(const service_level_options& other) const = default; + static sstring to_string(timeout_type); + static std::string_view to_string(const workload_type& wt); static std::optional parse_workload_type(std::string_view sv); + static sstring to_string(shares_type); + struct slo_effective_names { sstring timeout; sstring workload; @@ -118,10 +122,29 @@ service::query_state& qos_query_state(qos::query_context ctx = qos::query_contex future get_service_levels(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name, db::consistency_level cl, qos::query_context ctx); future get_service_level(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name, sstring service_level_name, db::consistency_level cl); +class service_level_scheduling_groups_exhausted : public std::runtime_error { +public: + static constexpr const char* msg = "Can't create scheduling group for {}, consider removing this service level or some other service level"; + service_level_scheduling_groups_exhausted(sstring name) : std::runtime_error(format(msg, name)) { + } +}; + } +template <> struct fmt::formatter : fmt::formatter { + auto format(qos::service_level_options::timeout_type tt, fmt::format_context& ctx) const { + return formatter::format(qos::service_level_options::to_string(tt), ctx); + } +}; + template <> struct fmt::formatter : fmt::formatter { auto format(qos::service_level_options::workload_type wt, fmt::format_context& ctx) const { return formatter::format(qos::service_level_options::to_string(wt), ctx); } }; + +template <> struct fmt::formatter : fmt::formatter { + auto format(qos::service_level_options::shares_type st, fmt::format_context& ctx) const { + return formatter::format(qos::service_level_options::to_string(st), ctx); + } +}; diff --git a/service/qos/qos_configuration_change_subscriber.hh b/service/qos/qos_configuration_change_subscriber.hh index 0804e4609d37..b26fc30489e4 100644 --- a/service/qos/qos_configuration_change_subscriber.hh +++ b/service/qos/qos_configuration_change_subscriber.hh @@ -15,6 +15,7 @@ namespace qos { struct service_level_info { sstring name; + seastar::scheduling_group sg; }; class qos_configuration_change_subscriber { public: diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index 3b32843422e6..198e5305a00d 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -15,6 +15,7 @@ #include #include #include +#include #include "cql3/untyped_result_set.hh" #include "db/config.hh" #include "db/consistency_level_type.hh" @@ -33,26 +34,34 @@ #include "service/storage_service.hh" #include "service/topology_state_machine.hh" #include "utils/sorting.hh" +#include namespace qos { static logging::logger sl_logger("service_level_controller"); sstring service_level_controller::default_service_level_name = "default"; - - - -service_level_controller::service_level_controller(sharded& auth_service, locator::shared_token_metadata& tm, abort_source& as, service_level_options default_service_level_config): - _sl_data_accessor(nullptr), - _auth_service(auth_service), - _token_metadata(tm), - _last_successful_config_update(seastar::lowres_clock::now()), - _logged_intervals(0), - _early_abort_subscription(as.subscribe([this] () noexcept { do_abort(); })) - +constexpr const char* scheduling_group_name_pattern = "sl:{}"; +constexpr const char* deleted_scheduling_group_name_pattern = "sl_deleted:{}"; +constexpr const char* temp_scheduling_group_name_pattern = "sl_temp:{}"; + +service_level_controller::service_level_controller(sharded& auth_service, locator::shared_token_metadata& tm, abort_source& as, service_level_options default_service_level_config, scheduling_group default_scheduling_group, bool destroy_default_sg_on_drain) + : _sl_data_accessor(nullptr) + , _auth_service(auth_service) + , _token_metadata(tm) + , _last_successful_config_update(seastar::lowres_clock::now()) + , _logged_intervals(0) + , _early_abort_subscription(as.subscribe([this] () noexcept { do_abort(); })) { + // We can't rename the system default scheduling group so we have to reject it. + assert(default_scheduling_group != get_default_scheduling_group()); if (this_shard_id() == global_controller) { _global_controller_db = std::make_unique(); _global_controller_db->default_service_level_config = default_service_level_config; + _global_controller_db->default_sg = default_scheduling_group; + _global_controller_db->destroy_default_sg = destroy_default_sg_on_drain; + // since the first thing that is being done is adding the default service level, we only + // need to throw the given group to the pool of scheduling groups for reuse. + _global_controller_db->deleted_scheduling_groups.emplace_back(default_scheduling_group); } } @@ -132,12 +141,47 @@ future<> service_level_controller::stop() { _global_controller_db->notifications_serializer.broken(); try { - co_await std::exchange(_global_controller_db->distributed_data_update, make_ready_future<>()); + auto f = co_await coroutine::as_future(std::exchange(_global_controller_db->distributed_data_update, make_ready_future<>())); + // delete all sg's in _service_levels_db, leaving it empty. + for (auto it = _service_levels_db.begin(); it != _service_levels_db.end(); ) { + _global_controller_db->deleted_scheduling_groups.emplace_back(it->second.sg); + it = _service_levels_db.erase(it); + } + f.get(); } catch (const broken_semaphore& ignored) { } catch (const sleep_aborted& ignored) { } catch (const exceptions::unavailable_exception& ignored) { } catch (const exceptions::read_timeout_exception& ignored) { } + + // exclude scheduling groups we shouldn't destroy + std::erase_if(_global_controller_db->deleted_scheduling_groups, [this] (scheduling_group& sg) { + if (sg == default_scheduling_group()) { + return true; + } else if (!_global_controller_db->destroy_default_sg && _global_controller_db->default_sg == sg) { + return true; + } else { + return false; + } + }); + + // destroy all sg's in _global_controller_db->deleted_scheduling_groups, leaving it empty + // if any destroy_scheduling_group call fails, return one of the exceptions + std::deque deleted_scheduling_groups = std::move(_global_controller_db->deleted_scheduling_groups); + std::exception_ptr ex; + + while (!deleted_scheduling_groups.empty()) { + auto f = co_await coroutine::as_future(destroy_scheduling_group(deleted_scheduling_groups.front())); + if (f.failed()) { + auto e = f.get_exception(); + sl_logger.error("Destroying scheduling group \"{}\" on stop failed: {}. Ignored.", deleted_scheduling_groups.front().name(), e); + ex = std::move(e); + } + deleted_scheduling_groups.pop_front(); + } + if (ex) { + std::rethrow_exception(std::move(ex)); + } } void service_level_controller::abort_group0_operations() { @@ -163,7 +207,8 @@ future<> service_level_controller::update_service_levels_cache(qos::query_contex // detects it the scan query done inside this call is failing. service_levels = _sl_data_accessor->get_service_levels(ctx).get(); - service_levels_info service_levels_for_add_or_update; + service_levels_info service_levels_for_update; + service_levels_info service_levels_for_add; service_levels_info service_levels_for_delete; auto current_it = _service_levels_db.begin(); @@ -187,7 +232,7 @@ future<> service_level_controller::update_service_levels_cache(qos::query_contex if (current_it->second.slo != new_state_it->second) { // The service level configuration is different // in the new state and the old state, meaning it needs to be updated. - service_levels_for_add_or_update.insert(*new_state_it); + service_levels_for_update.insert(*new_state_it); } current_it++; new_state_it++; @@ -196,36 +241,63 @@ future<> service_level_controller::update_service_levels_cache(qos::query_contex //removed, but only if it is not static since static configurations dont //come from the distributed keyspace but from code. if (!current_it->second.is_static) { - sl_logger.info("service level \"{}\" was deleted.", current_it->first.c_str()); service_levels_for_delete.emplace(current_it->first, current_it->second.slo); } current_it++; } else { /*new_it->first < current_it->first */ // The service level exits in the new state but not in the old state // so it needs to be added. - sl_logger.info("service level \"{}\" was added.", new_state_it->first.c_str()); - service_levels_for_add_or_update.insert(*new_state_it); + service_levels_for_add.insert(*new_state_it); new_state_it++; } } for (; current_it != _service_levels_db.end(); current_it++) { if (!current_it->second.is_static) { - sl_logger.info("service level \"{}\" was deleted.", current_it->first.c_str()); service_levels_for_delete.emplace(current_it->first, current_it->second.slo); } } for (; new_state_it != service_levels.end(); new_state_it++) { - sl_logger.info("service level \"{}\" was added.", new_state_it->first.c_str()); - service_levels_for_add_or_update.emplace(new_state_it->first, new_state_it->second); + service_levels_for_add.emplace(new_state_it->first, new_state_it->second); } for (auto&& sl : service_levels_for_delete) { do_remove_service_level(sl.first, false).get(); + sl_logger.info("service level \"{}\" was deleted.", sl.first.c_str()); } - for (auto&& sl : service_levels_for_add_or_update) { + for (auto&& sl : service_levels_for_update) { do_add_service_level(sl.first, sl.second).get(); + sl_logger.info("service level \"{}\" was updated. New values: (timeout: {}, workload_type: {}, shares: {})", + sl.first, sl.second.timeout, sl.second.workload, sl.second.shares); } + _effective_service_levels_db.clear(); + for (auto&& sl : service_levels_for_add) { + bool make_room = false; + std::map::reverse_iterator it; + try { + do_add_service_level(sl.first, sl.second).get(); + sl_logger.info("service level \"{}\" was added.", sl.first.c_str()); + } catch (service_level_scheduling_groups_exhausted &ex) { + it = _service_levels_db.rbegin(); + if (it->first == default_service_level_name) { + it++; + } + if (it->first.compare(sl.first) > 0) { + make_room = true; + } else { + _effectively_dropped_sls.insert(sl.first); + sl_logger.warn("{}", ex.what()); + } + } + if (make_room) { + sl_logger.warn("service level \"{}\" will be effectively dropped to make scheduling group available to \"{}\", please consider removing a service level." + , it->first, sl.first ); + do_remove_service_level(it->first, false).get(); + _effectively_dropped_sls.insert(it->first); + do_add_service_level(sl.first, sl.second).get(); + } + } + }); }); } @@ -258,9 +330,16 @@ future<> service_level_controller::update_effective_service_levels_cache() { std::optional sl_options; if (auto sl_name_it = attributes.find(role); sl_name_it != attributes.end()) { - auto sl = _service_levels_db.at(sl_name_it->second); - sl_options = sl.slo; - sl_options->init_effective_names(sl_name_it->second); + if (auto sl_it = _service_levels_db.find(sl_name_it->second); sl_it != _service_levels_db.end()) { + sl_options = sl_it->second.slo; + sl_options->init_effective_names(sl_name_it->second); + sl_options->shares_name = sl_name_it->second; + } else if (_effectively_dropped_sls.contains(sl_name_it->second)) { + // service level might be effective dropped, then it's not present in `_service_levels_db` + sl_logger.warn("Service level {} is effectively dropped and its values are ignored.", sl_name_it->second); + } else { + sl_logger.error("Couldn't find service level {} in first level cache", sl_name_it->second); + } } auto [it, it_end] = hierarchy.equal_range(role); @@ -330,7 +409,9 @@ future> service_level_controller::find_effe } sl_it->second.slo.init_effective_names(*sl_name); - return sl_it->second.slo; + auto slo = sl_it->second.slo; + slo.shares_name = sl_name; + return slo; } catch (...) { // when we fail, we act as if the attribute does not exist so the node // will not be brought down. return std::nullopt; @@ -361,9 +442,13 @@ std::optional service_level_controller::find_cached_effec future<> service_level_controller::notify_service_level_added(sstring name, service_level sl_data) { return seastar::async( [this, name, sl_data] { - _subscribers.thread_for_each([name, sl_data] (qos_configuration_change_subscriber* subscriber) { + service_level_info sl_info = { + .name = name, + .sg = sl_data.sg, + }; + _subscribers.thread_for_each([name, sl_data, sl_info] (qos_configuration_change_subscriber* subscriber) { try { - subscriber->on_before_service_level_add(sl_data.slo, {name}).get(); + subscriber->on_before_service_level_add(sl_data.slo, sl_info).get(); } catch (...) { sl_logger.error("notify_service_level_added: exception occurred in one of the observers callbacks {}", std::current_exception()); } @@ -379,13 +464,26 @@ future<> service_level_controller::notify_service_level_updated(sstring name, se if (sl_it != _service_levels_db.end()) { service_level_options slo_before = sl_it->second.slo; return seastar::async( [this,sl_it, name, slo_before, slo] { - _subscribers.thread_for_each([name, slo_before, slo] (qos_configuration_change_subscriber* subscriber) { + future<> f = make_ready_future(); + service_level_info sl_info = { + .name = name, + .sg = sl_it->second.sg, + }; + _subscribers.thread_for_each([name, slo_before, slo, sl_info] (qos_configuration_change_subscriber* subscriber) { try { - subscriber->on_before_service_level_change(slo_before, slo, {name}).get(); + subscriber->on_before_service_level_change(slo_before, slo, sl_info).get(); } catch (...) { sl_logger.error("notify_service_level_updated: exception occurred in one of the observers callbacks {}", std::current_exception()); } }); + if (sl_it->second.slo.shares != slo.shares) { + int32_t new_shares = default_shares; + if (auto new_shares_p = std::get_if(&slo.shares)) { + new_shares = *new_shares_p; + } + sl_it->second.sg.set_shares(new_shares); + } + sl_it->second.slo = slo; }); } @@ -395,11 +493,19 @@ future<> service_level_controller::notify_service_level_updated(sstring name, se future<> service_level_controller::notify_service_level_removed(sstring name) { auto sl_it = _service_levels_db.find(name); if (sl_it != _service_levels_db.end()) { + if (this_shard_id() == global_controller) { + _global_controller_db->deleted_scheduling_groups.emplace_back(sl_it->second.sg); + co_await rename_scheduling_group(sl_it->second.sg, seastar::format(deleted_scheduling_group_name_pattern, sl_it->first)); + } + service_level_info sl_info = { + .name = name, + .sg = sl_it->second.sg, + }; _service_levels_db.erase(sl_it); - co_return co_await seastar::async( [this, name] { - _subscribers.thread_for_each([name] (qos_configuration_change_subscriber* subscriber) { + co_return co_await seastar::async( [this, name, sl_info] { + _subscribers.thread_for_each([name, sl_info] (qos_configuration_change_subscriber* subscriber) { try { - subscriber->on_after_service_level_remove({name}).get(); + subscriber->on_after_service_level_remove(sl_info).get(); } catch (...) { sl_logger.error("notify_service_level_removed: exception occurred in one of the observers callbacks {}", std::current_exception()); } @@ -409,6 +515,10 @@ future<> service_level_controller::notify_service_level_removed(sstring name) { co_return; } +scheduling_group service_level_controller::get_default_scheduling_group() { + return _default_service_level.sg; +} + future<> service_level_controller::notify_effective_service_levels_cache_reloaded() { co_await _subscribers.for_each([] (qos_configuration_change_subscriber* subscriber) -> future<> { return subscriber->on_effective_service_levels_cache_reloaded(); @@ -518,6 +628,25 @@ future service_level_controller::get_distributed_service_le return _sl_data_accessor ? _sl_data_accessor->get_service_level(service_level_name) : make_ready_future(); } +future service_level_controller::validate_before_service_level_add() { + assert(this_shard_id() == global_controller); + if (_global_controller_db->deleted_scheduling_groups.size() > 0) { + return make_ready_future(true); + } else if (_global_controller_db->scheduling_groups_exhausted) { + return make_ready_future(false); + } else { + return create_scheduling_group(seastar::format(temp_scheduling_group_name_pattern, _global_controller_db->unique_group_counter++), 1).then_wrapped([this] (future new_sg_f) { + if (new_sg_f.failed()) { + new_sg_f.ignore_ready_future(); + _global_controller_db->scheduling_groups_exhausted = true; + return make_ready_future(false); + } + _global_controller_db->deleted_scheduling_groups.emplace_back(new_sg_f.get()); + return make_ready_future(true); + }); + } +} + future<> service_level_controller::set_distributed_service_level(sstring name, service_level_options slo, set_service_level_op_type op_type, service::group0_batch& mc) { auto sl_info = co_await _sl_data_accessor->get_service_levels(); auto it = sl_info.find(name); @@ -533,6 +662,13 @@ future<> service_level_controller::set_distributed_service_level(sstring name, s co_return; } } + + if (op_type != set_service_level_op_type::alter) { + bool validation_result = co_await container().invoke_on(global_controller, &service_level_controller::validate_before_service_level_add); + if (!validation_result&& !utils::get_local_injector().enter("allow_service_level_over_limit")) { + throw exceptions::invalid_request_exception("Can't create service level - no more scheduling groups exist"); + } + } co_return co_await _sl_data_accessor->set_service_level(name, slo, mc); } @@ -554,8 +690,49 @@ future<> service_level_controller::do_add_service_level(sstring name, service_le return make_ready_future(); } } else { - return do_with(service_level(slo, is_static), std::move(name), [this] (service_level& sl, sstring& name) { - return container().invoke_on_all(&service_level_controller::notify_service_level_added, name, sl); + return do_with(service_level(slo, is_static, default_scheduling_group()), + std::move(name), [this] (service_level& sl, sstring& name) { + return make_ready_future().then([this, &sl, &name] () mutable { + int32_t share_count = default_shares; + if (auto* maybe_shares = std::get_if(&sl.slo.shares)) { + share_count = *maybe_shares; + } + + if (!_global_controller_db->deleted_scheduling_groups.empty()) { + auto&& it = std::find_if(_global_controller_db->deleted_scheduling_groups.begin() + , _global_controller_db->deleted_scheduling_groups.end() + , [sg_name_to_find = seastar::format(deleted_scheduling_group_name_pattern, name)] (const scheduling_group& sg) { + return (sg.name() == sg_name_to_find); + }); + if (it != _global_controller_db->deleted_scheduling_groups.end()) { + sl.sg = *it; + _global_controller_db->deleted_scheduling_groups.erase(it); + } else { + sl.sg = _global_controller_db->deleted_scheduling_groups.front(); + _global_controller_db->deleted_scheduling_groups.pop_front(); + } + return container().invoke_on_all([&sl, share_count] (service_level_controller& service) { + scheduling_group non_const_sg = sl.sg; + return non_const_sg.set_shares((float)share_count); + }).then([&sl, &name] { + return rename_scheduling_group(sl.sg, seastar::format(scheduling_group_name_pattern, name)); + }); + } else if (_global_controller_db->scheduling_groups_exhausted) { + return make_exception_future<>(service_level_scheduling_groups_exhausted(name)); + } else { + return create_scheduling_group(seastar::format(scheduling_group_name_pattern, name), share_count).then_wrapped([this, name, &sl] (future sg_fut) { + if (sg_fut.failed()) { + sg_fut.ignore_ready_future(); + _global_controller_db->scheduling_groups_exhausted = true; + return make_exception_future<>(service_level_scheduling_groups_exhausted(name)); + } + sl.sg = sg_fut.get(); + return make_ready_future<>(); + }); + } + }).then([this, &sl, &name] () { + return container().invoke_on_all(&service_level_controller::notify_service_level_added, name, sl); + }); }); } return make_ready_future(); diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 453932b104e9..3534b62aaaae 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -50,12 +50,14 @@ namespace qos { struct service_level { service_level_options slo; bool is_static = false; + scheduling_group sg; service_level() = default; - service_level(service_level_options slo, bool is_static) + service_level(service_level_options slo, bool is_static, scheduling_group sg) : slo(std::move(slo)) , is_static(is_static) + , sg(sg) {} }; @@ -68,7 +70,8 @@ using update_both_cache_levels = bool_class; * 1. Global controller which is responsible for all of the data and plumbing * manipulation. * 2. Local controllers that act upon the data and facilitates execution in - * the service level context + * the service level context: i.e functions in their service level's + * scheduling group and io operations with their correct io priority. * * Definitions: * service level - User creates service level with some parameters (timeout/workload type). @@ -118,8 +121,7 @@ public: private: struct global_controller_data { service_levels_info static_configurations{}; - int schedg_group_cnt = 0; - int io_priority_cnt = 0; + std::deque deleted_scheduling_groups{}; service_level_options default_service_level_config; // The below future is used to serialize work so no reordering can occur. // This is needed so for example: delete(x), add(x) will not reverse yielding @@ -129,6 +131,13 @@ private: future<> distributed_data_update = make_ready_future(); abort_source dist_data_update_aborter; abort_source group0_aborter; + scheduling_group default_sg; + bool destroy_default_sg; + // a counter for making unique temp scheduling groups names + int unique_group_counter; + // A flag that indicates that we exhausted all of our scheduling groups + // and we can't create new ones. + bool scheduling_groups_exhausted = false; }; std::unique_ptr _global_controller_db; @@ -139,6 +148,8 @@ private: std::map _service_levels_db; // role name -> effective service_level_options std::map _effective_service_levels_db; + // Keeps names of effectively dropped service levels. Those service levels exits in the table but are not present in _service_levels_db cache + std::set _effectively_dropped_sls; service_level _default_service_level; service_level_distributed_data_accessor_ptr _sl_data_accessor; sharded& _auth_service; @@ -149,7 +160,8 @@ private: optimized_optional _early_abort_subscription; void do_abort() noexcept; public: - service_level_controller(sharded& auth_service, locator::shared_token_metadata& tm, abort_source& as, service_level_options default_service_level_config); + service_level_controller(sharded& auth_service, locator::shared_token_metadata& tm, abort_source& as, service_level_options default_service_level_config, + scheduling_group default_scheduling_group, bool destroy_default_sg_on_drain = false); /** * this function must be called *once* from any shard before any other functions are called. @@ -193,6 +205,11 @@ public: void abort_group0_operations(); + /** + * @return the default service level scheduling group (see service_level_controller::initialize). + */ + scheduling_group get_default_scheduling_group(); + /** * Start legacy update loop if RAFT_SERVICE_LEVELS_CHANGE feature is not enabled yet * or the cluster is in recovery mode @@ -334,6 +351,10 @@ private: alter }; + /** Validate that we can handle an addition of another service level + * Must be called from on the global controller + */ + future validate_before_service_level_add(); future<> set_distributed_service_level(sstring name, service_level_options slo, set_service_level_op_type op_type, service::group0_batch& mc); future> describe_created_service_levels() const; diff --git a/test/boost/service_level_controller_test.cc b/test/boost/service_level_controller_test.cc index 453470d2a465..f090fd7adf20 100644 --- a/test/boost/service_level_controller_test.cc +++ b/test/boost/service_level_controller_test.cc @@ -102,11 +102,14 @@ template <> struct fmt::formatter : fmt::formatter sl_controller; sharded auth_service; + service_level_options sl_options; + sl_options.shares.emplace(1000); + scheduling_group default_scheduling_group = create_scheduling_group("sl_default_sg", 1.0).get(); locator::shared_token_metadata tm({}, {locator::topology::config{ .local_dc_rack = locator::endpoint_dc_rack::default_location }}); sharded as; as.start().get(); auto stop_as = defer([&as] { as.stop().get(); }); - sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), service_level_options{}).get(); + sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), sl_options, default_scheduling_group).get(); qos_configuration_change_suscriber_simple ccss; sl_controller.local().register_subscriber(&ccss); sl_controller.local().add_service_level("sl1", service_level_options{}).get(); diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index 5946864edff1..47f2002c0a72 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -659,7 +659,7 @@ class single_node_cql_env : public cql_test_env { set_abort_on_internal_error(true); const gms::inet_address listen("127.0.0.1"); - _sl_controller.start(std::ref(_auth_service), std::ref(_token_metadata), std::ref(abort_sources), qos::service_level_options{.shares = 1000}).get(); + _sl_controller.start(std::ref(_auth_service), std::ref(_token_metadata), std::ref(abort_sources), qos::service_level_options{.shares = 1000}, scheduling_groups.statement_scheduling_group).get(); auto stop_sl_controller = defer([this] { _sl_controller.stop().get(); }); _sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); diff --git a/test/topology_custom/test_alternator.py b/test/topology_custom/test_alternator.py index 3f5287df6b3d..05080477ed45 100644 --- a/test/topology_custom/test_alternator.py +++ b/test/topology_custom/test_alternator.py @@ -140,7 +140,8 @@ async def get_cpu_metrics(): for ip in ips: metrics = await manager.metrics.query(ip) ms_streaming += metrics.get('scylla_scheduler_runtime_ms', {'group': 'streaming'}) - ms_statement += metrics.get('scylla_scheduler_runtime_ms', {'group': 'statement'}) + # in enterprise, default execution is in sl:default, not statement + ms_statement += metrics.get('scylla_scheduler_runtime_ms', {'group': 'sl:default'}) return (ms_streaming, ms_statement) ms_streaming_before, ms_statement_before = await get_cpu_metrics() From 7383013f43643282d318cf379285f82d45b960e2 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 19:51:03 +0100 Subject: [PATCH 147/397] replica/database: add reader concurrency semaphore groups Replace the reader concurrency semaphores for user reads and view updates with the newly introduced reader concurrency semaphore group, which assigns a semaphore for each service level. Each group is statically assigned to some pool of memory on startup and dynamically distribute this memory between the semaphores, relative to the number of shares of the corresponding scheduling group. The intent of having a separate reader concurrency semaphore for each scheduling group is to prevent priority inversion issues due to reads with different priorities waiting on the same semaphore, as well as make memory allocation more fair between service levels due to the adjusted number of shares. --- CMakeLists.txt | 1 + configure.py | 1 + main.cc | 30 +-- reader_concurrency_semaphore.hh | 2 + reader_concurrency_semaphore_group.cc | 112 ++++++++++ reader_concurrency_semaphore_group.hh | 90 ++++++++ replica/database.cc | 192 +++++++++++++++--- replica/database.hh | 33 ++- scylla-gdb.py | 23 ++- service/qos/service_level_controller.hh | 17 ++ test/boost/database_test.cc | 99 ++++++++- .../reader_concurrency_semaphore_test.cc | 108 ++++++++++ test/lib/cql_test_env.cc | 13 +- test/lib/cql_test_env.hh | 2 + 14 files changed, 662 insertions(+), 61 deletions(-) create mode 100644 reader_concurrency_semaphore_group.cc create mode 100644 reader_concurrency_semaphore_group.hh diff --git a/CMakeLists.txt b/CMakeLists.txt index 9c2b90bab251..1a19edda5e5e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -199,6 +199,7 @@ target_sources(scylla-main tombstone_gc_options.cc tombstone_gc.cc reader_concurrency_semaphore.cc + reader_concurrency_semaphore_group.cc row_cache.cc schema_mutations.cc serializer.cc diff --git a/configure.py b/configure.py index f8e8fd407a56..5960768e4ff3 100755 --- a/configure.py +++ b/configure.py @@ -1160,6 +1160,7 @@ def find_ninja(): 'service/topology_coordinator.cc', 'node_ops/node_ops_ctl.cc', 'node_ops/task_manager_module.cc', + 'reader_concurrency_semaphore_group.cc', ] + [Antlr3Grammar('cql3/Cql.g')] \ + scylla_raft_core ) diff --git a/main.cc b/main.cc index b0fe95778ef0..0b982a60e618 100644 --- a/main.cc +++ b/main.cc @@ -1211,6 +1211,20 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl sstm.stop().get(); }); + static sharded auth_service; + static sharded maintenance_auth_service; + static sharded sl_controller; + debug::the_sl_controller = &sl_controller; + + //starting service level controller + qos::service_level_options default_service_level_configuration; + default_service_level_configuration.shares = 1000; + sl_controller.start(std::ref(auth_service), std::ref(token_metadata), std::ref(stop_signal.as_sharded_abort_source()), default_service_level_configuration, dbcfg.statement_scheduling_group).get(); + sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); + auto stop_sl_controller = defer_verbose_shutdown("service level controller", [] { + sl_controller.stop().get(); + }); + lang::manager::config lang_config; lang_config.lua.max_bytes = cfg->user_defined_function_allocation_limit_bytes(); lang_config.lua.max_contiguous = cfg->user_defined_function_contiguous_allocation_limit_bytes(); @@ -1247,7 +1261,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl // because it obtains the list of pre-existing segments for replay, which must // not include reserve segments created by active commitlogs. db.local().init_commitlog().get(); - db.invoke_on_all(&replica::database::start).get(); + db.invoke_on_all(&replica::database::start, std::ref(sl_controller)).get(); ::sigquit_handler sigquit_handler(db); @@ -1339,20 +1353,6 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl api::unset_server_config(ctx).get(); }); - static sharded auth_service; - static sharded maintenance_auth_service; - static sharded sl_controller; - debug::the_sl_controller = &sl_controller; - - //starting service level controller - qos::service_level_options default_service_level_configuration; - default_service_level_configuration.shares = 1000; - sl_controller.start(std::ref(auth_service), std::ref(token_metadata), std::ref(stop_signal.as_sharded_abort_source()), default_service_level_configuration, dbcfg.statement_scheduling_group).get(); - sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); - auto stop_sl_controller = defer_verbose_shutdown("service level controller", [] { - sl_controller.stop().get(); - }); - static sharded sys_dist_ks; static sharded sys_ks; static sharded view_update_generator; diff --git a/reader_concurrency_semaphore.hh b/reader_concurrency_semaphore.hh index 44aba5437b94..b52afefc9ae5 100644 --- a/reader_concurrency_semaphore.hh +++ b/reader_concurrency_semaphore.hh @@ -124,6 +124,8 @@ public: uint64_t sstables_read = 0; // Permits waiting on something: admission, memory or execution uint64_t waiters = 0; + + friend auto operator<=>(const stats&, const stats&) = default; }; using permit_list_type = bi::list< diff --git a/reader_concurrency_semaphore_group.cc b/reader_concurrency_semaphore_group.cc new file mode 100644 index 000000000000..fcc5ea78948f --- /dev/null +++ b/reader_concurrency_semaphore_group.cc @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2021-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "reader_concurrency_semaphore_group.hh" + +// Calling adjust is serialized since 2 adjustments can't happen simultaneosly, +// if they did the behaviour would be undefined. +future<> reader_concurrency_semaphore_group::adjust() { + return with_semaphore(_operations_serializer, 1, [this] () { + ssize_t distributed_memory = 0; + for (auto& [sg, wsem] : _semaphores) { + const ssize_t memory_share = std::floor((double(wsem.weight) / double(_total_weight)) * _total_memory); + wsem.sem.set_resources({_max_concurrent_reads, memory_share}); + distributed_memory += memory_share; + } + // Slap the remainder on one of the semaphores. + // This will be a few bytes, doesn't matter where we add it. + auto& sem = _semaphores.begin()->second.sem; + sem.set_resources(sem.initial_resources() + reader_resources{0, _total_memory - distributed_memory}); + }); +} + +// The call to change_weight is serialized as a consequence of the call to adjust. +future<> reader_concurrency_semaphore_group::change_weight(weighted_reader_concurrency_semaphore& sem, size_t new_weight) { + auto diff = new_weight - sem.weight; + if (diff) { + sem.weight += diff; + _total_weight += diff; + return adjust(); + } + return make_ready_future<>(); +} + +future<> reader_concurrency_semaphore_group::wait_adjust_complete() { + return with_semaphore(_operations_serializer, 1, [] { + return make_ready_future<>(); + }); +} + +future<> reader_concurrency_semaphore_group::stop() noexcept { + return parallel_for_each(_semaphores, [] (auto&& item) { + return item.second.sem.stop(); + }).then([this] { + _semaphores.clear(); + }); +} + +reader_concurrency_semaphore& reader_concurrency_semaphore_group::get(scheduling_group sg) { + return _semaphores.at(sg).sem; +} +reader_concurrency_semaphore* reader_concurrency_semaphore_group::get_or_null(scheduling_group sg) { + auto it = _semaphores.find(sg); + if (it == _semaphores.end()) { + return nullptr; + } else { + return &(it->second.sem); + } +} +reader_concurrency_semaphore& reader_concurrency_semaphore_group::add_or_update(scheduling_group sg, size_t shares) { + auto result = _semaphores.try_emplace( + sg, + 0, + _max_concurrent_reads, + _name_prefix ? format("{}_{}", *_name_prefix, sg.name()) : sg.name(), + _max_queue_length, + _serialize_limit_multiplier, + _kill_limit_multiplier, + _cpu_concurrency + ); + auto&& it = result.first; + // since we serialize all group changes this change wait will be queues and no further operations + // will be executed until this adjustment ends. + (void)change_weight(it->second, shares); + return it->second.sem; +} + +future<> reader_concurrency_semaphore_group::remove(scheduling_group sg) { + auto node_handle = _semaphores.extract(sg); + if (!node_handle.empty()) { + weighted_reader_concurrency_semaphore& sem = node_handle.mapped(); + return sem.sem.stop().then([this, &sem] { + return change_weight(sem, 0); + }).finally([node_handle = std::move(node_handle)] () { + // this holds on to the node handle until we destroy it only after the semaphore + // is stopped properly. + }); + } + return make_ready_future(); +} + +size_t reader_concurrency_semaphore_group::size() { + return _semaphores.size(); +} + +void reader_concurrency_semaphore_group::foreach_semaphore(std::function func) { + for (auto& [sg, wsem] : _semaphores) { + func(sg, wsem.sem); + } +} + +future<> +reader_concurrency_semaphore_group::foreach_semaphore_async(std::function (scheduling_group, reader_concurrency_semaphore&)> func) { + auto units = co_await get_units(_operations_serializer, 1); + for (auto& [sg, wsem] : _semaphores) { + co_await func(sg, wsem.sem); + } +} diff --git a/reader_concurrency_semaphore_group.hh b/reader_concurrency_semaphore_group.hh new file mode 100644 index 000000000000..6d073ef47228 --- /dev/null +++ b/reader_concurrency_semaphore_group.hh @@ -0,0 +1,90 @@ +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +/* + * Copyright (C) 2021-present ScyllaDB + */ + +#pragma once + +#include +#include +#include "reader_concurrency_semaphore.hh" +#include +#include + +// The reader_concurrency_semaphore_group is a group of semaphores that shares a common pool of memory, +// the memory is dynamically divided between them according to a relative slice of shares each semaphore +// is given. +// All of the mutating operations on the group are asynchronic and serialized. The semaphores are created +// and managed by the group. + +class reader_concurrency_semaphore_group { + size_t _total_memory; + size_t _total_weight; + size_t _max_concurrent_reads; + size_t _max_queue_length; + utils::updateable_value _serialize_limit_multiplier; + utils::updateable_value _kill_limit_multiplier; + utils::updateable_value _cpu_concurrency; + + friend class database_test_wrapper; + + struct weighted_reader_concurrency_semaphore { + size_t weight; + ssize_t memory_share; + reader_concurrency_semaphore sem; + weighted_reader_concurrency_semaphore(size_t shares, int count, sstring name, size_t max_queue_length, + utils::updateable_value serialize_limit_multiplier, + utils::updateable_value kill_limit_multiplier, + utils::updateable_value cpu_concurrency) + : weight(shares) + , memory_share(0) + , sem(utils::updateable_value(count), 0, name, max_queue_length, std::move(serialize_limit_multiplier), std::move(kill_limit_multiplier), + std::move(cpu_concurrency), reader_concurrency_semaphore::register_metrics::yes) {} + }; + + std::unordered_map _semaphores; + seastar::semaphore _operations_serializer; + std::optional _name_prefix; + + future<> change_weight(weighted_reader_concurrency_semaphore& sem, size_t new_weight); + +public: + reader_concurrency_semaphore_group(size_t memory, size_t max_concurrent_reads, size_t max_queue_length, + utils::updateable_value serialize_limit_multiplier, + utils::updateable_value kill_limit_multiplier, + utils::updateable_value cpu_concurrency, + std::optional name_prefix = std::nullopt) + : _total_memory(memory) + , _total_weight(0) + , _max_concurrent_reads(max_concurrent_reads) + , _max_queue_length(max_queue_length) + , _serialize_limit_multiplier(std::move(serialize_limit_multiplier)) + , _kill_limit_multiplier(std::move(kill_limit_multiplier)) + , _cpu_concurrency(std::move(cpu_concurrency)) + , _operations_serializer(1) + , _name_prefix(std::move(name_prefix)) { } + + ~reader_concurrency_semaphore_group() { + assert(_semaphores.empty()); + } + future<> adjust(); + future<> wait_adjust_complete(); + + future<> stop() noexcept; + reader_concurrency_semaphore& get(scheduling_group sg); + reader_concurrency_semaphore* get_or_null(scheduling_group sg); + reader_concurrency_semaphore& add_or_update(scheduling_group sg, size_t shares); + future<> remove(scheduling_group sg); + size_t size(); + void foreach_semaphore(std::function func); + + future<> foreach_semaphore_async(std::function (scheduling_group, reader_concurrency_semaphore&)> func); + + auto sum_read_concurrency_sem_var(std::invocable auto member) { + using ret_type = std::invoke_result_t; + return boost::accumulate(_semaphores | boost::adaptors::map_values | boost::adaptors::transformed([=] (weighted_reader_concurrency_semaphore& wrcs) { return std::invoke(member, wrcs.sem); }), ret_type(0)); + } +}; diff --git a/replica/database.cc b/replica/database.cc index 87da7731d3c4..234f2991bef2 100644 --- a/replica/database.cc +++ b/replica/database.cc @@ -67,6 +67,7 @@ #include "locator/abstract_replication_strategy.hh" #include "timeout_config.hh" #include "tombstone_gc.hh" +#include "service/qos/service_level_controller.hh" #include "replica/data_dictionary_impl.hh" #include "replica/global_table_ptr.hh" @@ -220,14 +221,8 @@ void database::setup_scylla_memory_diagnostics_producer() { writeln("Replica:\n"); writeln(" Read Concurrency Semaphores:\n"); - const std::pair semaphores[] = { - {"user", _read_concurrency_sem}, - {"streaming", _streaming_concurrency_sem}, - {"system", _system_read_concurrency_sem}, - {"compaction", _compaction_concurrency_sem}, - {"view update", _view_update_read_concurrency_sem}, - }; - for (const auto& [name, sem] : semaphores) { + + static auto semaphore_dump = [&writeln] (const sstring& name, const reader_concurrency_semaphore& sem) { const auto initial_res = sem.initial_resources(); const auto available_res = sem.available_resources(); if (sem.is_unlimited()) { @@ -245,7 +240,17 @@ void database::setup_scylla_memory_diagnostics_producer() { utils::to_hr_size(initial_res.memory), sem.get_stats().waiters); } - } + }; + + semaphore_dump("streaming", _streaming_concurrency_sem); + semaphore_dump("system", _system_read_concurrency_sem); + semaphore_dump("compaction", _compaction_concurrency_sem); + _reader_concurrency_semaphores_group.foreach_semaphore([] (scheduling_group sg, reader_concurrency_semaphore& sem) { + semaphore_dump(sg.name(), sem); + }); + _view_update_read_concurrency_semaphores_group.foreach_semaphore([] (scheduling_group sg, reader_concurrency_semaphore& sem) { + semaphore_dump(sg.name(), sem); + }); writeln(" Execution Stages:\n"); const std::pair execution_stage_summaries[] = { @@ -311,6 +316,42 @@ class db_user_types_storage : public data_dictionary::dummy_user_types_storage { } }; +reader_concurrency_semaphore& +database::read_concurrency_sem() { + reader_concurrency_semaphore* sem = _reader_concurrency_semaphores_group.get_or_null(current_scheduling_group()); + if (!sem) { + // this line is commented out, however we shouldn't get here because it means that a user query or even worse, + // some random query was triggered from an unanticipated scheduling groups and this violates the isolation we are trying to achieve. + // It is commented out for two reasons: + // 1. So we will be able to ease into this new system, first testing functionality and effect and only then mix in exceptions and asserts. + // 2. So the series containing those changes will be backportable without causing too harsh regressions (aborts) on one hand and without forcing + // extensive changes on the other hand. + // Follow Up: uncomment this line and run extensive testing. Handle every case of abort. + // seastar::on_internal_error(dblog, format("Tried to run a user query in a wrong scheduling group (scheduling group: '{}')", current_scheduling_group().name())); + sem = _reader_concurrency_semaphores_group.get_or_null(_default_read_concurrency_group); + if (!sem) { + // If we got here - the initialization went very wrong and we can't do anything about it. + // This can only happen if someone touched the initialization code which is assumed to initialize at least + // this default semaphore. + seastar::on_internal_error(dblog, "Default read concurrency semaphore wasn't found, something probably went wrong during database::start"); + } + } + return *sem; +} + +// With same concerns as read_concurrency_sem(). +reader_concurrency_semaphore& +database::view_update_read_concurrency_sem() { + reader_concurrency_semaphore* sem = _view_update_read_concurrency_semaphores_group.get_or_null(current_scheduling_group()); + if (!sem) { + sem = _view_update_read_concurrency_semaphores_group.get_or_null(_default_read_concurrency_group); + if (!sem) { + seastar::on_internal_error(dblog, "Default view update read concurrency semaphore wasn't found, something probably went wrong during database::start"); + } + } + return *sem; +} + database::database(const db::config& cfg, database_config dbcfg, service::migration_notifier& mn, gms::feature_service& feat, const locator::shared_token_metadata& stm, compaction_manager& cm, sstables::storage_manager& sstm, lang::manager& langm, sstables::directory_semaphore& sst_dir_sem, const abort_source& abort, utils::cross_shard_barrier barrier) : _stats(make_lw_shared()) @@ -329,15 +370,6 @@ database::database(const db::config& cfg, database_config dbcfg, service::migrat } return backlog; })) - , _read_concurrency_sem( - utils::updateable_value(max_count_concurrent_reads), - max_memory_concurrent_reads(), - "user", - max_inactive_queue_length(), - _cfg.reader_concurrency_semaphore_serialize_limit_multiplier, - _cfg.reader_concurrency_semaphore_kill_limit_multiplier, - _cfg.reader_concurrency_semaphore_cpu_concurrency, - reader_concurrency_semaphore::register_metrics::yes) // No timeouts or queue length limits - a failure here can kill an entire repair. // Trust the caller to limit concurrency. , _streaming_concurrency_sem( @@ -360,15 +392,14 @@ database::database(const db::config& cfg, database_config dbcfg, service::migrat utils::updateable_value(std::numeric_limits::max()), utils::updateable_value(std::numeric_limits::max()), reader_concurrency_semaphore::register_metrics::yes) - , _view_update_read_concurrency_sem( - utils::updateable_value(max_count_concurrent_view_update_reads), + , _view_update_read_concurrency_semaphores_group( max_memory_concurrent_view_update_reads(), - "view_update", + utils::updateable_value(max_count_concurrent_view_update_reads), max_inactive_view_update_queue_length(), _cfg.view_update_reader_concurrency_semaphore_serialize_limit_multiplier, _cfg.view_update_reader_concurrency_semaphore_kill_limit_multiplier, _cfg.view_update_reader_concurrency_semaphore_cpu_concurrency, - reader_concurrency_semaphore::register_metrics::yes) + "view_update") , _row_cache_tracker(_cfg.index_cache_fraction.operator utils::updateable_value(), cache_tracker::register_metrics::yes) , _apply_stage("db_apply", &database::do_apply) , _version(empty_version) @@ -392,6 +423,10 @@ database::database(const db::config& cfg, database_config dbcfg, service::migrat , _feat(feat) , _shared_token_metadata(stm) , _lang_manager(langm) + , _reader_concurrency_semaphores_group(max_memory_concurrent_reads(), max_count_concurrent_reads, max_inactive_queue_length(), + _cfg.reader_concurrency_semaphore_serialize_limit_multiplier, + _cfg.reader_concurrency_semaphore_kill_limit_multiplier, + _cfg.reader_concurrency_semaphore_cpu_concurrency) , _stop_barrier(std::move(barrier)) , _update_memtable_flush_static_shares_action([this, &cfg] { return _memtable_controller.update_static_shares(cfg.memtable_flush_static_shares()); }) , _memtable_flush_static_shares_observer(cfg.memtable_flush_static_shares.observe(_update_memtable_flush_static_shares_action.make_observer())) @@ -485,6 +520,12 @@ namespace replica { static const metrics::label class_label("class"); + +auto +database::sum_read_concurrency_sem_stat(std::invocable auto stats_member) { + return _reader_concurrency_semaphores_group.sum_read_concurrency_sem_var([&] (reader_concurrency_semaphore& rcs) { return std::invoke(stats_member, rcs.get_stats()); }); +} + void database::setup_metrics() { _dirty_memory_manager.setup_collectd("regular"); @@ -1605,7 +1646,7 @@ query::max_result_size database::get_query_max_result_size() const { reader_concurrency_semaphore& database::get_reader_concurrency_semaphore() { switch (classify_request(_dbcfg)) { - case request_class::user: return _read_concurrency_sem; + case request_class::user: return read_concurrency_sem(); case request_class::system: return _system_read_concurrency_sem; case request_class::maintenance: return _streaming_concurrency_sem; } @@ -1634,9 +1675,15 @@ future<> database::clear_inactive_reads_for_tablet(table_id table, dht::token_ra } future<> database::foreach_reader_concurrency_semaphore(std::function(reader_concurrency_semaphore&)> func) { - for (auto* sem : {&_read_concurrency_sem, &_streaming_concurrency_sem, &_compaction_concurrency_sem, &_system_read_concurrency_sem, &_view_update_read_concurrency_sem}) { + for (auto* sem : {&_streaming_concurrency_sem, &_compaction_concurrency_sem, &_system_read_concurrency_sem}) { co_await func(*sem); } + co_await _reader_concurrency_semaphores_group.foreach_semaphore_async([&] (scheduling_group sg, reader_concurrency_semaphore& sem) -> future<> { + co_await func(sem); + }); + co_await _view_update_read_concurrency_semaphores_group.foreach_semaphore_async([&] (scheduling_group sg, reader_concurrency_semaphore& sem) -> future<> { + co_await func(sem); + }); } std::ostream& operator<<(std::ostream& out, const column_family& cf) { @@ -1935,7 +1982,7 @@ future<> database::do_apply(schema_ptr s, const frozen_mutation& m, tracing::tra co_await coroutine::return_exception(std::runtime_error("view update generator not plugged to push updates")); } - auto lock_f = co_await coroutine::as_future(cf.push_view_replica_updates(_view_update_generator, s, m, timeout, std::move(tr_state), _view_update_read_concurrency_sem)); + auto lock_f = co_await coroutine::as_future(cf.push_view_replica_updates(_view_update_generator, s, m, timeout, std::move(tr_state), view_update_read_concurrency_sem())); if (lock_f.failed()) { auto ex = lock_f.get_exception(); if (is_timeout_exception(ex)) { @@ -2182,7 +2229,54 @@ void database::revert_initial_system_read_concurrency_boost() { dblog.debug("Reverted system read concurrency from initial {} to normal {}", database::max_count_concurrent_reads, database::max_count_system_concurrent_reads); } -future<> database::start() { +future<> database::start(sharded& sl_controller) { + sl_controller.local().register_subscriber(this); + _unsubscribe_qos_configuration_change = [this, &sl_controller] () { + return sl_controller.local().unregister_subscriber(this); + }; + qos::service_level default_service_level = sl_controller.local().get_service_level(qos::service_level_controller::default_service_level_name); + int32_t default_shares = 1000; + if (int32_t* default_shares_p = std::get_if(&(default_service_level.slo.shares))) { + default_shares = *default_shares_p; + } else { + on_internal_error(dblog, "The default service_level should always contain shares value"); + } + + // The former _dbcfg.statement_scheduling_group and the later can be the same group, so we want + // the later to be the accurate one. + _default_read_concurrency_group = default_service_level.sg; + _reader_concurrency_semaphores_group.add_or_update(default_service_level.sg, default_shares); + _view_update_read_concurrency_semaphores_group.add_or_update(default_service_level.sg, default_shares); + + // lets insert the statement scheduling group only if we haven't reused it in sl_controller, + // but it shouldn't happen + if (!_reader_concurrency_semaphores_group.get_or_null(_dbcfg.statement_scheduling_group)) { + // This is super ugly, we need to either force the database to use system scheduling group for non-user queries + // or, if we have user queries running on this scheduling group make it's definition more robust (what runs in it). + // Another ugly thing here is that we have to have a pre-existing knowladge about the shares ammount this group was + // built with. I think we should have a followup that makes this more robust. + _reader_concurrency_semaphores_group.add_or_update(_dbcfg.statement_scheduling_group, 1000); + _view_update_read_concurrency_semaphores_group.add_or_update(_dbcfg.statement_scheduling_group, 1000); + } + + // This will wait for the semaphores to be given some memory. + // We need this since the below statements (get_distributed_service_levels in particular) will need + // to run queries and for this they will need to admit some memory. + co_await _reader_concurrency_semaphores_group.wait_adjust_complete(); + co_await _view_update_read_concurrency_semaphores_group.wait_adjust_complete(); + + auto service_levels = co_await sl_controller.local().get_distributed_service_levels(qos::query_context::group0); + for (auto&& service_level_record : service_levels) { + auto service_level = sl_controller.local().get_service_level(service_level_record.first); + if (service_level.slo.shares_name && *service_level.slo.shares_name != qos::service_level_controller::default_service_level_name) { + // We know slo.shares is valid becuse we know that slo.shares_name is valid + _reader_concurrency_semaphores_group.add_or_update(service_level.sg, std::get(service_level.slo.shares)); + _view_update_read_concurrency_semaphores_group.add_or_update(service_level.sg, std::get(service_level.slo.shares)); + } + } + + co_await _reader_concurrency_semaphores_group.adjust(); + co_await _view_update_read_concurrency_semaphores_group.adjust(); _large_data_handler->start(); // We need the compaction manager ready early so we can reshard. _compaction_manager.enable(); @@ -2215,10 +2309,12 @@ future<> database::shutdown() { } future<> database::stop() { + if (_unsubscribe_qos_configuration_change) { + co_await std::exchange(_unsubscribe_qos_configuration_change, {})(); + } if (!_shutdown) { co_await shutdown(); } - // try to ensure that CL has done disk flushing if (_commitlog) { dblog.info("Shutting down commitlog"); @@ -2250,11 +2346,11 @@ future<> database::stop() { dblog.info("Stopping querier cache"); co_await _querier_cache.stop(); dblog.info("Stopping concurrency semaphores"); - co_await _read_concurrency_sem.stop(); + co_await _reader_concurrency_semaphores_group.stop(); + co_await _view_update_read_concurrency_semaphores_group.stop(); co_await _streaming_concurrency_sem.stop(); co_await _compaction_concurrency_sem.stop(); co_await _system_read_concurrency_sem.stop(); - co_await _view_update_read_concurrency_sem.stop(); dblog.info("Joining memtable update action"); co_await _update_memtable_flush_static_shares_action.join(); } @@ -3029,3 +3125,41 @@ future>> query_data( } } // namespace replica + +namespace replica { + +/** This callback is going to be called just before the service level is available **/ +future<> database::on_before_service_level_add(qos::service_level_options slo, qos::service_level_info sl_info) { + if (auto shares_p = std::get_if(&slo.shares)) { + _reader_concurrency_semaphores_group.add_or_update(sl_info.sg, *shares_p); + _view_update_read_concurrency_semaphores_group.add_or_update(sl_info.sg, *shares_p); + // the call to add_or_update_read_concurrency_sem will take the semaphore until the adjustment + // is completed, we need to wait for the operation to complete. + co_await _reader_concurrency_semaphores_group.wait_adjust_complete(); + co_await _view_update_read_concurrency_semaphores_group.wait_adjust_complete(); + } +} +/** This callback is going to be called just after the service level is removed **/ +future<> database::on_after_service_level_remove(qos::service_level_info sl_info) { + co_await _reader_concurrency_semaphores_group.remove(sl_info.sg); + co_await _view_update_read_concurrency_semaphores_group.remove(sl_info.sg); +} +/** This callback is going to be called just before the service level is changed **/ +future<> database::on_before_service_level_change(qos::service_level_options slo_before, qos::service_level_options slo_after, + qos::service_level_info sl_info) { + if (auto shares_p = std::get_if(&slo_after.shares)) { + _reader_concurrency_semaphores_group.add_or_update(sl_info.sg, *shares_p); + _view_update_read_concurrency_semaphores_group.add_or_update(sl_info.sg, *shares_p); + // the call to add_or_update_read_concurrency_sem will take the semaphore until the adjustment + // is completed, we need to wait for the operation to complete. + co_await _reader_concurrency_semaphores_group.wait_adjust_complete(); + co_await _view_update_read_concurrency_semaphores_group.wait_adjust_complete(); + } +} + +future<> +database::on_effective_service_levels_cache_reloaded() { + co_return; +} + +} diff --git a/replica/database.hh b/replica/database.hh index d0e25343846f..94fc42e372e5 100644 --- a/replica/database.hh +++ b/replica/database.hh @@ -47,7 +47,7 @@ #include "utils/phased_barrier.hh" #include "backlog_controller.hh" #include "dirty_memory_manager.hh" -#include "reader_concurrency_semaphore.hh" +#include "reader_concurrency_semaphore_group.hh" #include "db/timeout_clock.hh" #include "querier.hh" #include "cache_temperature.hh" @@ -67,6 +67,7 @@ #include "utils/serialized_action.hh" #include "compaction/compaction_fwd.hh" #include "compaction_group.hh" +#include "service/qos/qos_configuration_change_subscriber.hh" class cell_locker; class cell_locker_stats; @@ -137,6 +138,10 @@ class view_update_generator; } +namespace qos { + class service_level_controller; +} + class mutation_reordered_with_truncate_exception : public std::exception {}; class column_family_test; @@ -1383,7 +1388,7 @@ class db_user_types_storage; // local metadata reads // use table::shard_for_reads()/table::shard_for_writes() for data -class database : public peering_sharded_service { +class database : public peering_sharded_service, qos::qos_configuration_change_subscriber { friend class ::database_test_wrapper; public: enum class table_kind { @@ -1487,13 +1492,13 @@ private: flush_controller _memtable_controller; drain_progress _drain_progress {}; - reader_concurrency_semaphore _read_concurrency_sem; + reader_concurrency_semaphore _streaming_concurrency_sem; reader_concurrency_semaphore _compaction_concurrency_sem; reader_concurrency_semaphore _system_read_concurrency_sem; - // The view update read concurrency semaphore used for view updates coming from user writes. - reader_concurrency_semaphore _view_update_read_concurrency_sem; + // The view update read concurrency semaphores used for view updates coming from user writes. + reader_concurrency_semaphore_group _view_update_read_concurrency_semaphores_group; db::timeout_semaphore _view_update_concurrency_sem{max_memory_pending_view_updates()}; cache_tracker _row_cache_tracker; @@ -1540,6 +1545,10 @@ private: const locator::shared_token_metadata& _shared_token_metadata; lang::manager& _lang_manager; + reader_concurrency_semaphore_group _reader_concurrency_semaphores_group; + scheduling_group _default_read_concurrency_group; + noncopyable_function()> _unsubscribe_qos_configuration_change; + utils::cross_shard_barrier _stop_barrier; db::rate_limiter _rate_limiter; @@ -1579,6 +1588,10 @@ private: future<> create_in_memory_keyspace(const lw_shared_ptr& ksm, locator::effective_replication_map_factory& erm_factory, system_keyspace system); void setup_metrics(); void setup_scylla_memory_diagnostics_producer(); + reader_concurrency_semaphore& read_concurrency_sem(); + reader_concurrency_semaphore& view_update_read_concurrency_sem(); + auto sum_read_concurrency_sem_var(std::invocable auto member); + auto sum_read_concurrency_sem_stat(std::invocable auto stats_member); future<> do_apply(schema_ptr, const frozen_mutation&, tracing::trace_state_ptr tr_state, db::timeout_clock::time_point timeout, db::commitlog_force_sync sync, db::per_partition_rate_limit::info rate_limit_info); future<> do_apply_many(const std::vector&, db::timeout_clock::time_point timeout); @@ -1714,7 +1727,7 @@ public: /// reads, to speed up startup. After startup this should be reverted to /// the normal concurrency. void revert_initial_system_read_concurrency_boost(); - future<> start(); + future<> start(sharded&); future<> shutdown(); future<> stop(); future<> close_tables(table_kind kind_to_close); @@ -1906,6 +1919,14 @@ public: } future<> clear_inactive_reads_for_tablet(table_id table, dht::token_range tablet_range); + + /** This callback is going to be called just before the service level is available **/ + virtual future<> on_before_service_level_add(qos::service_level_options slo, qos::service_level_info sl_info) override; + /** This callback is going to be called just after the service level is removed **/ + virtual future<> on_after_service_level_remove(qos::service_level_info sl_info) override; + /** This callback is going to be called just before the service level is changed **/ + virtual future<> on_before_service_level_change(qos::service_level_options slo_before, qos::service_level_options slo_after, qos::service_level_info sl_info) override; + virtual future<> on_effective_service_levels_cache_reloaded() override; }; // A helper function to parse the directory name back diff --git a/scylla-gdb.py b/scylla-gdb.py index 92082943a992..d098c5be6e11 100755 --- a/scylla-gdb.py +++ b/scylla-gdb.py @@ -2311,13 +2311,21 @@ def print_replica_stats(): if not db: return + per_service_level_sem = [] + for sg, sem in unordered_map(db["_reader_concurrency_semaphores_group"]["_semaphores"]): + per_service_level_sem.append(scylla_memory.format_semaphore_stats(sem["sem"])) + + per_service_level_vu_sem = [] + for sg, sem in unordered_map(db["_view_update_read_concurrency_semaphores_group"]["_semaphores"]): + per_service_level_vu_sem.append(scylla_memory.format_semaphore_stats(sem["sem"])) + database_typename = lookup_type(['replica::database', 'database'])[1].name gdb.write('Replica:\n') gdb.write(' Read Concurrency Semaphores:\n {}\n {}\n {}\n {}\n'.format( - scylla_memory.format_semaphore_stats(db['_read_concurrency_sem']), + '\n '.join(per_service_level_sem), scylla_memory.format_semaphore_stats(db['_streaming_concurrency_sem']), scylla_memory.format_semaphore_stats(db['_system_read_concurrency_sem']), - scylla_memory.format_semaphore_stats(db['_view_update_read_concurrency_sem']))) + '\n '.join(per_service_level_vu_sem))) gdb.write(' Execution Stages:\n') for es_path in [('_apply_stage',)]: @@ -5809,12 +5817,17 @@ def invoke(self, args, from_tty): semaphores = [gdb.parse_and_eval(arg) for arg in args.split(' ')] else: db = find_db() - semaphores = [db["_read_concurrency_sem"], db["_streaming_concurrency_sem"], db["_system_read_concurrency_sem"]] + semaphores = [db["_streaming_concurrency_sem"], db["_system_read_concurrency_sem"]] semaphores.append(db["_compaction_concurrency_sem"]) try: - semaphores.append(db["_view_update_read_concurrency_sem"]) + semaphores += [weighted_sem["sem"] for (_, weighted_sem) in unordered_map(db["_reader_concurrency_semaphores_group"]["_semaphores"])] + except gdb.error: + # compatibility with code before per-scheduling-group semaphore + pass + try: + semaphores += [weighted_sem["sem"] for (_, weighted_sem) in unordered_map(db["_view_update_read_concurrency_semaphores_group"]["_semaphores"])] except gdb.error: - # 6.2 compatibility + # 2024.2 compatibility pass for semaphore in semaphores: diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 3534b62aaaae..402113b2d58d 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -205,6 +205,23 @@ public: void abort_group0_operations(); + /** + * this is an executor of a function with arguments under a specific + * service level. + * @param service_level_name + * @param func - the function to be executed + * @param args - the arguments to pass to the function. + * @return a future that is resolved when the function's operation is resolved + * (if it returns a future). or a ready future containing the returned value + * from the function/ + */ + template > + requires std::invocable + futurize_t with_service_level(sstring service_level_name, Func&& func) { + service_level& sl = get_service_level(service_level_name); + return with_scheduling_group(sl.sg, std::move(func)); + } + /** * @return the default service level scheduling group (see service_level_controller::initialize). */ diff --git a/test/boost/database_test.cc b/test/boost/database_test.cc index 10a2381d678c..647c44da5c2d 100644 --- a/test/boost/database_test.cc +++ b/test/boost/database_test.cc @@ -61,7 +61,7 @@ class database_test_wrapper { explicit database_test_wrapper(replica::database& db) : _db(db) { } reader_concurrency_semaphore& get_user_read_concurrency_semaphore() { - return _db._read_concurrency_sem; + return _db.read_concurrency_sem(); } reader_concurrency_semaphore& get_streaming_read_concurrency_semaphore() { return _db._streaming_concurrency_sem; @@ -69,6 +69,14 @@ class database_test_wrapper { reader_concurrency_semaphore& get_system_read_concurrency_semaphore() { return _db._system_read_concurrency_sem; } + + size_t get_total_user_reader_concurrency_semaphore_memory() { + return _db._reader_concurrency_semaphores_group._total_memory; + } + + size_t get_total_user_reader_concurrency_semaphore_weight() { + return _db._reader_concurrency_semaphores_group._total_weight; + } }; static future<> apply_mutation(sharded& sharded_db, table_id uuid, const mutation& m, bool do_flush = false, @@ -1151,7 +1159,8 @@ SEASTAR_THREAD_TEST_CASE(reader_concurrency_semaphore_selection_test) { auto& db = e.local_db(); database_test_wrapper tdb(db); for (const auto& [sched_group, expected_sem_getter] : scheduling_group_and_expected_semaphore) { - with_scheduling_group(sched_group, [&db, sched_group = sched_group, expected_sem_ptr = &expected_sem_getter(tdb)] { + with_scheduling_group(sched_group, [&db, sched_group = sched_group, &tdb, &expected_sem_getter = expected_sem_getter] { + auto expected_sem_ptr = &expected_sem_getter(tdb); auto& sem = db.get_reader_concurrency_semaphore(); if (&sem != expected_sem_ptr) { BOOST_FAIL(fmt::format("Unexpected semaphore for scheduling group {}, expected {}, got {}", sched_group.name(), expected_sem_ptr->name(), sem.name())); @@ -1296,6 +1305,92 @@ SEASTAR_TEST_CASE(upgrade_sstables) { }); } +SEASTAR_THREAD_TEST_CASE(per_service_level_reader_concurrency_semaphore_test) { + cql_test_config cfg; + do_with_cql_env_thread([] (cql_test_env& e) { + const size_t num_service_levels = 3; + const size_t num_keys_to_insert = 10; + const size_t num_individual_reads_to_test = 50; + auto& db = e.local_db(); + database_test_wrapper dbt(db); + size_t total_memory = dbt.get_total_user_reader_concurrency_semaphore_memory(); + sharded& sl_controller = e.service_level_controller_service(); + std::array sl_names; + qos::service_level_options slo; + size_t expected_total_weight = 0; + auto index_to_weight = [] (size_t i) -> size_t { + return (i + 1)*100; + }; + + // make the default service level take as little memory as possible + slo.shares.emplace(1); + expected_total_weight += 1; + sl_controller.local().add_service_level(qos::service_level_controller::default_service_level_name, slo).get(); + + // Just to make the code more readable. + auto get_reader_concurrency_semaphore_for_sl = [&] (sstring sl_name) -> reader_concurrency_semaphore& { + return *sl_controller.local().with_service_level(sl_name, noncopyable_function([&] { + return &db.get_reader_concurrency_semaphore(); + })).get(); + }; + + for (unsigned i = 0; i < num_service_levels; i++) { + sstring sl_name = format("sl{}", i); + slo.shares.emplace(index_to_weight(i)); + sl_controller.local().add_service_level(sl_name, slo).get(); + expected_total_weight += index_to_weight(i); + // Make sure that the total weight is tracked correctly in the semaphore group + BOOST_REQUIRE_EQUAL(expected_total_weight, dbt.get_total_user_reader_concurrency_semaphore_weight()); + sl_names[i] = sl_name; + size_t total_distributed_memory = 0; + for (unsigned j = 0 ; j <= i ; j++) { + reader_concurrency_semaphore& sem = get_reader_concurrency_semaphore_for_sl(sl_names[j]); + // Make sure that all semaphores that has been created until now - have the right amount of available memory + // after the operation has ended. + // We allow for a small delta of up to num_service_levels. This allows an off-by-one for each semaphore, + // the remainder being added to one of the semaphores. + // We make sure this didn't leak/create memory by checking the total below. + const auto delta = std::abs(ssize_t((index_to_weight(j) * total_memory) / expected_total_weight) - sem.available_resources().memory); + BOOST_REQUIRE_LE(delta, num_service_levels); + total_distributed_memory += sem.available_resources().memory; + } + total_distributed_memory += get_reader_concurrency_semaphore_for_sl(qos::service_level_controller::default_service_level_name).available_resources().memory; + BOOST_REQUIRE_EQUAL(total_distributed_memory, total_memory); + } + + auto get_semaphores_stats_snapshot = [&] () { + std::unordered_map snapshot; + for (auto&& sl_name : sl_names) { + snapshot[sl_name] = get_reader_concurrency_semaphore_for_sl(sl_name).get_stats(); + } + return snapshot; + }; + e.execute_cql("CREATE TABLE tbl (a int, b int, PRIMARY KEY (a));").get(); + + for (unsigned i = 0; i < num_keys_to_insert; i++) { + for (unsigned j = 0; j < num_keys_to_insert; j++) { + e.execute_cql(format("INSERT INTO tbl(a, b) VALUES ({}, {});", i, j)).get(); + } + } + + for (unsigned i = 0; i < num_individual_reads_to_test; i++) { + int random_service_level = tests::random::get_int(num_service_levels - 1); + auto snapshot_before = get_semaphores_stats_snapshot(); + + sl_controller.local().with_service_level(sl_names[random_service_level], noncopyable_function()> ([&] { + return e.execute_cql("SELECT * FROM tbl;").discard_result(); + })).get(); + auto snapshot_after = get_semaphores_stats_snapshot(); + for (auto& [sl_name, stats] : snapshot_before) { + // Make sure that the only semaphore that experienced any activity (at least measured activity) is + // the semaphore that belongs to the current service level. + BOOST_REQUIRE((stats == snapshot_after[sl_name] && sl_name != sl_names[random_service_level]) || + (stats != snapshot_after[sl_name] && sl_name == sl_names[random_service_level])); + } + } + }, std::move(cfg)).get(); +} + SEASTAR_TEST_CASE(populate_from_quarantine_works) { auto tmpdir_for_data = make_lw_shared(); auto db_cfg_ptr = make_shared(); diff --git a/test/boost/reader_concurrency_semaphore_test.cc b/test/boost/reader_concurrency_semaphore_test.cc index cb39f0990f22..be0283916dae 100644 --- a/test/boost/reader_concurrency_semaphore_test.cc +++ b/test/boost/reader_concurrency_semaphore_test.cc @@ -11,6 +11,7 @@ #include #include "reader_concurrency_semaphore.hh" #include "sstables/sstables_manager.hh" +#include "reader_concurrency_semaphore_group.hh" #include "test/lib/log.hh" #include "test/lib/simple_schema.hh" #include "test/lib/cql_assertions.hh" @@ -1219,6 +1220,113 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_set_resources) { } // namespace reader_concurrency_semaphore_test +SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_group) { + const auto initial_resources = reader_resources{100, 100 * 1024}; + auto serialize_multiplier = utils::updateable_value_source(2); + auto kill_multiplier = utils::updateable_value_source(3); + auto cpu_concurrency = utils::updateable_value_source(1); + + reader_concurrency_semaphore_group sem_group(initial_resources.memory, initial_resources.count, 1000, + utils::updateable_value(serialize_multiplier), + utils::updateable_value(kill_multiplier), + utils::updateable_value(cpu_concurrency)); + auto stop_sem = deferred_stop(sem_group); + + circular_buffer recycle_bin; + + const auto initial_shares = 1000; + struct scheduling_group_with_shares { + scheduling_group sg; + size_t shares; + + scheduling_group_with_shares(scheduling_group sg, size_t shares) : sg(sg), shares(shares) { } + }; + std::vector scheduling_groups; + const auto max_sched_groups = 8; + + auto check_sem_group = [&] { + const auto total_shares = boost::accumulate(scheduling_groups + | boost::adaptors::transformed([] (const scheduling_group_with_shares& sgs) { return sgs.shares; }), size_t(0)); + ssize_t total_memory = 0; + sem_group.foreach_semaphore([&] (scheduling_group sg, reader_concurrency_semaphore& sem) { + const auto res = sem.available_resources(); + BOOST_CHECK_EQUAL(res.count, initial_resources.count); // currently count is not partitioned among the semaphores + auto it = std::find_if(scheduling_groups.begin(), scheduling_groups.end(), [sg] (const scheduling_group_with_shares& sgs) { return sgs.sg == sg; }); + BOOST_REQUIRE(it != scheduling_groups.end()); + const auto shares = it->shares; + const ssize_t expected_memory = std::floor((double(shares) / double(total_shares)) * initial_resources.memory); + const auto memory_diff = std::abs(res.memory - expected_memory); + testlog.trace("{}: {}/{} (shares) -> {}/{} (memory) | res.memory: {}", sg.name(), shares, total_shares, expected_memory, initial_resources.memory, res.memory); + BOOST_CHECK_LE(memory_diff, scheduling_groups.size()); // due to integer division, we allow for ceil/floor (off-by-one), the remainder being added to any semaphore + total_memory += res.memory; + }); + BOOST_CHECK_EQUAL(total_memory, initial_resources.memory); // no off-by-one allowed on the total + }; + + auto add_sg = [&, sgi = 0] () mutable { + if (scheduling_groups.size() >= max_sched_groups) { + return false; + } + testlog.debug("create sg{}", sgi); + scheduling_group sg; + const auto sg_name = format("sg{}", sgi++); + if (recycle_bin.empty()) { + sg = create_scheduling_group(sg_name, initial_shares).get(); + } else { + sg = recycle_bin.front(); + recycle_bin.pop_front(); + rename_scheduling_group(sg, sg_name).get(); + } + scheduling_groups.emplace_back(sg, initial_shares); + sem_group.add_or_update(sg, initial_shares); + sem_group.wait_adjust_complete().get(); + return true; + }; + + while (add_sg()) { + check_sem_group(); + } + + for (size_t i = 0; i < 32; ++i) { + testlog.debug("iteration {}", i); + std::shuffle(scheduling_groups.begin(), scheduling_groups.end(), tests::random::gen()); + switch (tests::random::get_int(0, 3)) { + case 0: // add + { + testlog.debug("maybe add sg"); + if (add_sg()) { + break; + } + [[fallthrough]]; + } + case 1: //remove + { + const auto& sgs = scheduling_groups.back(); + testlog.debug("maybe remove {}", sgs.sg.name()); + if (scheduling_groups.size() > 1) { + testlog.debug("remove {}", sgs.sg.name()); + sem_group.remove(sgs.sg).get(); + recycle_bin.push_back(sgs.sg); + scheduling_groups.pop_back(); + break; + } + [[fallthrough]]; + } + default: //update + { + auto& sgs = scheduling_groups.back(); + const auto new_shares = tests::random::get_int(100, 1000); + sgs.shares = new_shares; + testlog.debug("update {}: {}->{}", sgs.sg.name(), sgs.shares, new_shares); + sem_group.add_or_update(sgs.sg, new_shares); + sem_group.wait_adjust_complete().get(); + break; + } + } + check_sem_group(); + } +} + namespace { class allocating_reader { diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index 47f2002c0a72..511608357686 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -593,6 +593,10 @@ class single_node_cql_env : public cql_test_env { _sstm.start(std::ref(*cfg), sstables::storage_manager::config{}).get(); auto stop_sstm = deferred_stop(_sstm); + _sl_controller.start(std::ref(_auth_service), std::ref(_token_metadata), std::ref(abort_sources), qos::service_level_options{.shares = 1000}, scheduling_groups.statement_scheduling_group).get(); + auto stop_sl_controller = defer([this] { _sl_controller.stop().get(); }); + _sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); + lang::manager::config lang_config; lang_config.lua.max_bytes = cfg->user_defined_function_allocation_limit_bytes(); lang_config.lua.max_contiguous = cfg->user_defined_function_contiguous_allocation_limit_bytes(); @@ -618,7 +622,7 @@ class single_node_cql_env : public cql_test_env { _db.stop().get(); }); - _db.invoke_on_all(&replica::database::start).get(); + _db.invoke_on_all(&replica::database::start, std::ref(_sl_controller)).get(); smp::invoke_on_all([blocked_reactor_notify_ms] { engine().update_blocked_reactor_notify_ms(blocked_reactor_notify_ms); @@ -659,9 +663,6 @@ class single_node_cql_env : public cql_test_env { set_abort_on_internal_error(true); const gms::inet_address listen("127.0.0.1"); - _sl_controller.start(std::ref(_auth_service), std::ref(_token_metadata), std::ref(abort_sources), qos::service_level_options{.shares = 1000}, scheduling_groups.statement_scheduling_group).get(); - auto stop_sl_controller = defer([this] { _sl_controller.stop().get(); }); - _sl_controller.invoke_on_all(&qos::service_level_controller::start).get(); _sys_ks.start(std::ref(_qp), std::ref(_db)).get(); auto stop_sys_kd = defer([this] { @@ -1105,6 +1106,10 @@ class single_node_cql_env : public cql_test_env { return cql_transport::messages::propagate_exception_as_future(std::move(msg)); }); } + + virtual sharded& service_level_controller_service() override { + return _sl_controller; + } }; std::atomic single_node_cql_env::active = { false }; diff --git a/test/lib/cql_test_env.hh b/test/lib/cql_test_env.hh index 040684ca82dd..0779da7b2601 100644 --- a/test/lib/cql_test_env.hh +++ b/test/lib/cql_test_env.hh @@ -184,6 +184,8 @@ public: virtual sharded& get_task_manager() = 0; data_dictionary::database data_dictionary(); + + virtual sharded& service_level_controller_service() = 0; }; future<> do_with_cql_env(std::function(cql_test_env&)> func, cql_test_config = {}, std::optional = {}); From f1b9737e0710bba85142d3615f5e09eb537725f3 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 18:47:59 +0100 Subject: [PATCH 148/397] messaging_service: use separate set of connections per service levels In order to make sure that the scheduling group carries over RPC, and also to prevent priority inversion issues between different service levels, modify the messaging service to use separate RPC connections for each service level in order to serve user traffic. The above is achieved by reusing the existing concept of "tenants" in messaging service: when a new service level (or, more accurately, service-level specific scheduling group) is first used in an RPC, a new tenant is created. In addition, extend the service level controller to be able to quickly look up the service level name of the currently active scheduling group in order to speed up the logic for choosing the tenant. --- main.cc | 3 +- message/messaging_service.cc | 181 +++++++++++++++++++++--- message/messaging_service.hh | 15 +- service/qos/service_level_controller.cc | 28 +++- service/qos/service_level_controller.hh | 14 ++ test/lib/cql_test_env.cc | 3 +- test/manual/gossip.cc | 13 +- test/manual/message.cc | 16 ++- 8 files changed, 241 insertions(+), 32 deletions(-) diff --git a/main.cc b/main.cc index 0b982a60e618..bc8980cbe875 100644 --- a/main.cc +++ b/main.cc @@ -1516,7 +1516,6 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl netw::messaging_service::scheduling_config scfg; scfg.statement_tenants = { - {dbcfg.statement_scheduling_group, "$user"}, {default_scheduling_group(), "$system"}, {dbcfg.streaming_scheduling_group, "$maintenance", false} }; @@ -1535,7 +1534,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl } // Delay listening messaging_service until gossip message handlers are registered - messaging.start(mscfg, scfg, creds, std::ref(feature_service), std::ref(gossip_address_map), std::ref(compressor_tracker)).get(); + messaging.start(mscfg, scfg, creds, std::ref(feature_service), std::ref(gossip_address_map), std::ref(compressor_tracker), std::ref(sl_controller)).get(); auto stop_ms = defer_verbose_shutdown("messaging service", [&messaging] { messaging.invoke_on_all(&netw::messaging_service::stop).get(); }); diff --git a/message/messaging_service.cc b/message/messaging_service.cc index 158a2edbc6d3..31e8692aa57b 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -18,6 +18,8 @@ #include "message/messaging_service.hh" #include #include "gms/gossiper.hh" +#include "service/storage_service.hh" +#include "service/qos/service_level_controller.hh" #include "streaming/prepare_message.hh" #include "gms/gossip_digest_syn.hh" #include "gms/gossip_digest_ack.hh" @@ -275,10 +277,11 @@ messaging_service::messaging_service( uint16_t port, gms::feature_service& feature_service, gms::gossip_address_map& address_map, - utils::walltime_compressor_tracker& wct) + utils::walltime_compressor_tracker& wct, + qos::service_level_controller& sl_controller) : messaging_service(config{std::move(id), ip, ip, port}, scheduling_config{{{{}, "$default"}}, {}, {}}, - nullptr, feature_service, address_map, wct) + nullptr, feature_service, address_map, wct, sl_controller) {} static @@ -386,9 +389,10 @@ void messaging_service::do_start_listen() { // the first by wrapping its server_socket, but not the second. auto limits = rpc_resource_limits(_cfg.rpc_memory_limit); limits.isolate_connection = [this] (sstring isolation_cookie) { - rpc::isolation_config cfg; - cfg.sched_group = scheduling_group_for_isolation_cookie(isolation_cookie); - return cfg; + + return scheduling_group_for_isolation_cookie(isolation_cookie).then([] (scheduling_group sg) { + return rpc::isolation_config{.sched_group = sg}; + }); }; if (!_server[0] && _cfg.encrypt != encrypt_what::all && _cfg.port) { auto listen = [&] (const gms::inet_address& a, rpc::streaming_domain_type sdomain) { @@ -467,7 +471,7 @@ void messaging_service::do_start_listen() { } messaging_service::messaging_service(config cfg, scheduling_config scfg, std::shared_ptr credentials, gms::feature_service& feature_service, - gms::gossip_address_map& address_map, utils::walltime_compressor_tracker& arct) + gms::gossip_address_map& address_map, utils::walltime_compressor_tracker& arct, qos::service_level_controller& sl_controller) : _cfg(std::move(cfg)) , _rpc(new rpc_protocol_wrapper(serializer { })) , _credentials_builder(credentials ? std::make_unique(*credentials) : nullptr) @@ -476,6 +480,7 @@ messaging_service::messaging_service(config cfg, scheduling_config scfg, std::sh , _scheduling_config(scfg) , _scheduling_info_for_connection_index(initial_scheduling_info()) , _feature_service(feature_service) + , _sl_controller(sl_controller) , _compressor_factory_wrapper(std::make_unique(arct, _cfg.enable_advanced_rpc_compression)) , _address_map(address_map) { @@ -743,13 +748,21 @@ msg_addr messaging_service::addr_for_host_id(locator::host_id hid) { } unsigned -messaging_service::get_rpc_client_idx(messaging_verb verb) const { +messaging_service::get_rpc_client_idx(messaging_verb verb) { auto idx = s_rpc_client_idx_table[static_cast(verb)]; if (idx < PER_SHARD_CONNECTION_COUNT) { return idx; } + // this is just a workaround for a wrong initialization order in messaging_service's + // constructor that causes _connection_index_for_tenant to be queried before it is + // initialized. This WA makes the behaviour match OSS in this case and it should be + // removed once it is fixed in OSS. If it isn't removed the behaviour will still be + // correct but we will lose cycles on an unnecesairy check. + if (_connection_index_for_tenant.size() == 0) { + return idx; + } const auto curr_sched_group = current_scheduling_group(); for (unsigned i = 0; i < _connection_index_for_tenant.size(); ++i) { if (_connection_index_for_tenant[i].sched_group == curr_sched_group) { @@ -757,16 +770,42 @@ messaging_service::get_rpc_client_idx(messaging_verb verb) const { // i == 0: the default tenant maps to the default client indexes belonging to the interval // [PER_SHARD_CONNECTION_COUNT, PER_SHARD_CONNECTION_COUNT + PER_TENANT_CONNECTION_COUNT). idx += i * PER_TENANT_CONNECTION_COUNT; - break; + return idx; } else { // If the tenant is disable, immediately return current index to // use $system tenant. return idx; } } + } - return idx; + // if we got here - it means that two conditions are met: + // 1. We are trying to get a client for a statement/statement_ack verb. + // 2. We are running in a scheduling group that is not assigned to one of the + // static tenants (e.g $system) + // If this scheduling group is of one of the system's static statement tenants we + // whould have caught it in the loop above. + // The other posibility is that we are running in a scheduling group belongs to + // a service level, maybe a deleted one, this is why it is possible that we will + // not find the service level name. + + std::optional service_level = _sl_controller.get_active_service_level(); + scheduling_group sg_for_tenant = curr_sched_group; + if (!service_level) { + service_level = qos::service_level_controller::default_service_level_name; + sg_for_tenant = _sl_controller.get_default_scheduling_group(); + } + auto it = _dynamic_tenants_to_client_idx.find(*service_level); + // the second part of this condition checks that the service level didn't "suddenly" + // changed scheduling group. If it did, it means probably that it was dropped and + // added again, if it happens we will update it's connection indexes since it is + // basically a new tenant with the same name. + if (it == _dynamic_tenants_to_client_idx.end() || + _scheduling_info_for_connection_index[it->second].sched_group != sg_for_tenant) { + return add_statement_tenant(*service_level,sg_for_tenant) + (idx - PER_SHARD_CONNECTION_COUNT); + } + return it->second; } std::vector @@ -802,25 +841,96 @@ messaging_service::scheduling_group_for_verb(messaging_verb verb) const { return _scheduling_info_for_connection_index[idx].sched_group; } -scheduling_group +future messaging_service::scheduling_group_for_isolation_cookie(const sstring& isolation_cookie) const { // Once per connection, so a loop is fine. for (auto&& info : _scheduling_info_for_connection_index) { if (info.isolation_cookie == isolation_cookie) { - return info.sched_group; + return make_ready_future(info.sched_group); } } - // Check for the case of the client using a connection class we don't - // recognize, but we know its a tenant, not a system connection. - // Fall-back to the default tenant in this case. - for (auto&& connection_prefix : _connection_types_prefix) { - if (isolation_cookie.find(connection_prefix.data()) == 0) { - return _scheduling_config.statement_tenants.front().sched_group; + + // We first check if this is a statement isolation cookie - if it is, we will search for the + // appropriate service level in the service_level_controller since in can be that + // _scheduling_info_for_connection_index is not yet updated (drop readd case for example) + // in the future we will only fall back here for new service levels that havn't been referenced + // before. + // It is safe to assume that an unknown connection type can be rejected since a connection + // with an unknown purpose on the inbound side is useless. + // However, until we get rid of the backward compatibility code below, we can't reject the + // connection since there is a slight chance that this connection comes from an old node that + // still doesn't use the "connection type prefix" convention. + auto tenant_connection = [] (const sstring& isolation_cookie) -> bool { + for (auto&& connection_prefix : _connection_types_prefix) { + if(isolation_cookie.find(connection_prefix.data()) == 0) { + return true; + } } + return false; + }; + + std::string service_level_name = ""; + if (tenant_connection(isolation_cookie)) { + // Extract the service level name from the connection isolation cookie. + service_level_name = isolation_cookie.substr(std::string(isolation_cookie).find_first_of(':') + 1); + } else if (_sl_controller.has_service_level(isolation_cookie)) { + // Backward Compatibility Code - This entire "else if" block should be removed + // in the major version that follows the one that contains this code. + // When upgrading from an older enterprise version the isolation cookie is not + // prefixed with "statement:" or any other connection type prefix, so an isolation cookie + // that comes from an older node will simply contain the service level name. + // we do an extra step to be also future proof and make sure it is indeed a service + // level's name, since if this is the older version and we upgrade to a new one + // we could have more connection classes (eg: streaming,gossip etc...) and we wouldn't + // want it to overload the default statement's scheduling group. + // it is not bulet proof in the sense that if a new tenant class happens to have the exact + // name as one of the service levels it will be diverted to the default statement scheduling + // group but it has a small chance of happening. + service_level_name = isolation_cookie; + mlogger.info("Trying to allow an rpc connection from an older node for service level {}", service_level_name); + } else { + // Client is using a new connection class that the server doesn't recognize yet. + // Assume it's important, after server upgrade we'll recognize it. + service_level_name = isolation_cookie; + mlogger.warn("Assuming an unknown cookie is from an older node and represent some not yet discovered service level {} - Trying to allow it.", service_level_name); + } + + if (_sl_controller.has_service_level(service_level_name)) { + return make_ready_future(_sl_controller.get_scheduling_group(service_level_name)); + } else if (service_level_name.starts_with('$')) { + // Tenant names starting with '$' are reserved for internal ones. If the tenant is not recognized + // to this point, it means we are in the middle of cluster upgrade and we don't know this tenant yet. + // Hardwire it to the default service level to keep things simple. + // This also includes `$user` tenant which is used in OSS and may appear in mixed OSS/Enterprise cluster. + return make_ready_future(_sl_controller.get_default_scheduling_group()); + } else { + mlogger.info("Service level {} is still unknown, will try to create it now and allow the RPC connection.", service_level_name); + // If the service level don't exist there are two possibilities, it is either created but still not known by this + // node. Or it has been deleted and the initiating node hasn't caught up yet, in both cases it is safe to __try__ and + // create a new service level (internally), it will naturally catch up eventually and by creating it here we prevent + // an rpc connection for a valid service level to permanently get stuck in the default service level scheduling group. + // If we can't create the service level (we already have too many service levels), we will reject the connection by returning + // an exeptional future. + qos::service_level_options slo; + // We put here the minimal ammount of shares for this service level to be functional. When the node catches up it will + // be either deleted or the number of shares and other configuration options will be updated. + slo.shares.emplace(1000); + slo.shares_name.emplace(service_level_name); + return _sl_controller.add_service_level(service_level_name, slo).then([this, service_level_name] () { + if (_sl_controller.has_service_level(service_level_name)) { + return make_ready_future(_sl_controller.get_scheduling_group(service_level_name)); + } else { + // The code until here is best effort, to provide fast catchup in case the configuration changes very quickly and being used + // before this node caught up, or alternatively during startup while the configuration table hasn't been consulted yet. + // If for some reason we couldn't add the service level, it is better to wait for the configuration to settle, + // this occasion should be rare enough, even if it happen, two paths are possible, either the initiating node will + // catch up, figure out the service level has been deleted and will not reattempt this rpc connection, or that this node will + // eventually catch up with the correct configuration (mainly some service levels that have been deleted and "made room" for this service level) and + // will eventually allow the connection. + return make_exception_future(std::runtime_error(fmt::format("Rejecting RPC connection for service level: {}, probably only a transitional effect", service_level_name))); + } + }); } - // Client is using a new connection class that the server doesn't recognize yet. - // Assume it's important, after server upgrade we'll recognize it. - return default_scheduling_group(); } @@ -1248,6 +1358,37 @@ future<> messaging_service::unregister_repair_get_full_row_hashes_with_rpc_strea // Wrappers for verbs +unsigned messaging_service::add_statement_tenant(sstring tenant_name, scheduling_group sg) { + auto idx = _clients.size(); + auto scheduling_info_for_connection_index_size = _scheduling_info_for_connection_index.size(); + auto undo = defer([&] { + _clients.resize(idx); + _clients_with_host_id.resize(idx); + _scheduling_info_for_connection_index.resize(scheduling_info_for_connection_index_size); + }); + _clients.resize(_clients.size() + PER_TENANT_CONNECTION_COUNT); + _clients_with_host_id.resize(_clients_with_host_id.size() + PER_TENANT_CONNECTION_COUNT); + // this functions as a way to delete an obsolete tenant with the same name but keeping _clients + // indexing and _scheduling_info_for_connection_index indexing in sync. + sstring first_cookie = sstring(_connection_types_prefix[0]) + tenant_name; + for (unsigned i = 0; i < _scheduling_info_for_connection_index.size(); i++) { + if (_scheduling_info_for_connection_index[i].isolation_cookie == first_cookie) { + // remove all connections associated with this tenant, since we are reinserting it. + for (size_t j = 0; j < _connection_types_prefix.size() ; j++) { + _scheduling_info_for_connection_index[i + j].isolation_cookie = ""; + } + break; + } + } + for (auto&& connection_prefix : _connection_types_prefix) { + sstring isolation_cookie = sstring(connection_prefix) + tenant_name; + _scheduling_info_for_connection_index.emplace_back(scheduling_info_for_connection_index{sg, isolation_cookie}); + } + _dynamic_tenants_to_client_idx.insert_or_assign(tenant_name, idx); + undo.cancel(); + return idx; +} + // Wrapper for TASKS_CHILDREN_REQUEST void messaging_service::register_tasks_get_children(std::function (const rpc::client_info& cinfo, tasks::get_children_request)>&& func) { register_handler(this, messaging_verb::TASKS_GET_CHILDREN, std::move(func)); diff --git a/message/messaging_service.hh b/message/messaging_service.hh index a982ce912b31..eb5c52613ca6 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -119,6 +119,10 @@ using get_children_request = task_id; using get_children_response = std::vector; } +namespace qos { + class service_level_controller; +} + namespace netw { /* All verb handler identifiers */ @@ -353,6 +357,8 @@ private: std::vector _scheduling_info_for_connection_index; std::vector _connection_index_for_tenant; gms::feature_service& _feature_service; + std::unordered_map _dynamic_tenants_to_client_idx; + qos::service_level_controller& _sl_controller; std::unique_ptr _compressor_factory_wrapper; struct connection_ref; @@ -370,9 +376,9 @@ public: using clock_type = lowres_clock; messaging_service(locator::host_id id, gms::inet_address ip, uint16_t port, - gms::feature_service&, gms::gossip_address_map&, utils::walltime_compressor_tracker&); + gms::feature_service&, gms::gossip_address_map&, utils::walltime_compressor_tracker&, qos::service_level_controller&); messaging_service(config cfg, scheduling_config scfg, std::shared_ptr, - gms::feature_service&, gms::gossip_address_map&, utils::walltime_compressor_tracker&); + gms::feature_service&, gms::gossip_address_map&, utils::walltime_compressor_tracker&, qos::service_level_controller&); ~messaging_service(); future<> start(); @@ -468,10 +474,11 @@ public: std::unique_ptr& rpc(); static msg_addr get_source(const rpc::client_info& client); scheduling_group scheduling_group_for_verb(messaging_verb verb) const; - scheduling_group scheduling_group_for_isolation_cookie(const sstring& isolation_cookie) const; + future scheduling_group_for_isolation_cookie(const sstring& isolation_cookie) const; std::vector initial_scheduling_info() const; - unsigned get_rpc_client_idx(messaging_verb verb) const; + unsigned get_rpc_client_idx(messaging_verb verb); static constexpr std::array _connection_types_prefix = {"statement:", "statement-ack:", "forward:"}; // "forward" is the old name for "mapreduce" + unsigned add_statement_tenant(sstring tenant_name, scheduling_group sg); void init_feature_listeners(); private: diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index 198e5305a00d..ea62020fdf58 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -453,7 +453,12 @@ future<> service_level_controller::notify_service_level_added(sstring name, ser sl_logger.error("notify_service_level_added: exception occurred in one of the observers callbacks {}", std::current_exception()); } }); - _service_levels_db.emplace(name, sl_data); + auto result= _service_levels_db.emplace(name, sl_data); + if (result.second) { + unsigned sl_idx = internal::scheduling_group_index(sl_data.sg); + _sl_lookup[sl_idx].first = &(result.first->first); + _sl_lookup[sl_idx].second = &(result.first->second); + } }); } @@ -493,6 +498,9 @@ future<> service_level_controller::notify_service_level_updated(sstring name, se future<> service_level_controller::notify_service_level_removed(sstring name) { auto sl_it = _service_levels_db.find(name); if (sl_it != _service_levels_db.end()) { + unsigned sl_idx = internal::scheduling_group_index(sl_it->second.sg); + _sl_lookup[sl_idx].first = nullptr; + _sl_lookup[sl_idx].second = nullptr; if (this_shard_id() == global_controller) { _global_controller_db->deleted_scheduling_groups.emplace_back(sl_it->second.sg); co_await rename_scheduling_group(sl_it->second.sg, seastar::format(deleted_scheduling_group_name_pattern, sl_it->first)); @@ -519,6 +527,24 @@ scheduling_group service_level_controller::get_default_scheduling_group() { return _default_service_level.sg; } +scheduling_group service_level_controller::get_scheduling_group(sstring service_level_name) { + auto service_level_it = _service_levels_db.find(service_level_name); + if (service_level_it != _service_levels_db.end()) { + return service_level_it->second.sg; + } else { + return get_default_scheduling_group(); + } +} + +std::optional service_level_controller::get_active_service_level() { + unsigned sched_idx = internal::scheduling_group_index(current_scheduling_group()); + if (_sl_lookup[sched_idx].first) { + return sstring(*_sl_lookup[sched_idx].first); + } else { + return std::nullopt; + } +} + future<> service_level_controller::notify_effective_service_levels_cache_reloaded() { co_await _subscribers.for_each([] (qos_configuration_change_subscriber* subscriber) -> future<> { return subscriber->on_effective_service_levels_cache_reloaded(); diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 402113b2d58d..5b85b1fd5dcf 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -150,6 +150,7 @@ private: std::map _effective_service_levels_db; // Keeps names of effectively dropped service levels. Those service levels exits in the table but are not present in _service_levels_db cache std::set _effectively_dropped_sls; + std::pair _sl_lookup[max_scheduling_groups()]; service_level _default_service_level; service_level_distributed_data_accessor_ptr _sl_data_accessor; sharded& _auth_service; @@ -226,6 +227,19 @@ public: * @return the default service level scheduling group (see service_level_controller::initialize). */ scheduling_group get_default_scheduling_group(); + /** + * Get the scheduling group for a specific service level. + * @param service_level_name - the service level which it's scheduling group + * should be returned. + * @return if the service level exists the service level's scheduling group. else + * get_scheduling_group("default") + */ + scheduling_group get_scheduling_group(sstring service_level_name); + /** + * @return the name of the currently active service level if such exists or an empty + * optional if no active service level. + */ + std::optional get_active_service_level(); /** * Start legacy update loop if RAFT_SERVICE_LEVELS_CHANGE feature is not enabled yet diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index 511608357686..c1c906859ec4 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -731,7 +731,8 @@ class single_node_cql_env : public cql_test_env { } // Don't start listening so tests can be run in parallel if cfg_in.ms_listen is not set to true explicitly. _ms.start(host_id, listen, std::move(port), std::ref(_feature_service), - std::ref(_gossip_address_map), std::ref(_compressor_tracker)).get(); + std::ref(_gossip_address_map), std::ref(_compressor_tracker), + std::ref(_sl_controller)).get(); stop_ms = defer(stop_type(stop_ms_func)); if (cfg_in.ms_listen) { diff --git a/test/manual/gossip.cc b/test/manual/gossip.cc index a0ee2fe31521..806e29008451 100644 --- a/test/manual/gossip.cc +++ b/test/manual/gossip.cc @@ -18,11 +18,13 @@ #include "message/messaging_service.hh" #include "gms/gossiper.hh" #include "gms/application_state.hh" +#include "service/qos/service_level_controller.hh" #include "utils/log.hh" #include #include #include "db/schema_tables.hh" + namespace bpo = boost::program_options; // === How to run the test @@ -62,6 +64,7 @@ int main(int ac, char ** av) { sharded feature_service; sharded gossip_address_map; sharded messaging; + sharded auth_service; abort_sources.start().get(); auto stop_abort_source = defer([&] { abort_sources.stop().get(); }); @@ -72,6 +75,13 @@ int main(int ac, char ** av) { tm_cfg.topo_cfg.this_cql_address = my_address; token_metadata.start([] () noexcept { return db::schema_tables::hold_merge_lock(); }, tm_cfg).get(); auto stop_token_mgr = defer([&] { token_metadata.stop().get(); }); + locator::shared_token_metadata tm({}, {}); + sharded sl_controller; + scheduling_group default_scheduling_group = create_scheduling_group("sl_default_sg", 1.0).get(); + sharded as; + as.start().get(); + auto stop_as = defer([&as] { as.stop().get(); }); + sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), qos::service_level_options{.shares = 1000}, default_scheduling_group).get(); compressor_tracker.start([] { return utils::walltime_compressor_tracker::config{}; }).get(); auto stop_compressor_tracker = deferred_stop(compressor_tracker); @@ -81,7 +91,8 @@ int main(int ac, char ** av) { gossip_address_map.start().get(); messaging.start(locator::host_id{}, listen, 7000, std::ref(feature_service), - std::ref(gossip_address_map), std::ref(compressor_tracker)).get(); + std::ref(gossip_address_map), std::ref(compressor_tracker), + std::ref(sl_controller)).get(); auto stop_messaging = deferred_stop(messaging); gms::gossip_config gcfg; diff --git a/test/manual/message.cc b/test/manual/message.cc index 73226d0b8f0b..7e153edd3e09 100644 --- a/test/manual/message.cc +++ b/test/manual/message.cc @@ -27,6 +27,7 @@ #include "locator/token_metadata.hh" #include "db/schema_tables.hh" #include "idl/gossip.dist.hh" +#include "service/qos/service_level_controller.hh" using namespace std::chrono_literals; using namespace netw; @@ -181,9 +182,12 @@ int main(int ac, char ** av) { ("cpuid", bpo::value()->default_value(0), "Server cpuid"); distributed db; + sharded auth_service; + locator::shared_token_metadata tm({}, {}); + distributed sl_controller; - return app.run_deprecated(ac, av, [&app] { - return seastar::async([&app] { + return app.run_deprecated(ac, av, [&app, &auth_service, &tm, &sl_controller] { + return seastar::async([&app, &auth_service, &tm, &sl_controller] { auto config = app.configuration(); bool stay_alive = config["stay-alive"].as(); const gms::inet_address listen = gms::inet_address(config["listen-address"].as()); @@ -193,6 +197,11 @@ int main(int ac, char ** av) { sharded token_metadata; token_metadata.start([] () noexcept { return db::schema_tables::hold_merge_lock(); }, tm_cfg).get(); auto stop_tm = deferred_stop(token_metadata); + auto default_scheduling_group = create_scheduling_group("sl_default_sg", 1.0).get(); + sharded as; + as.start().get(); + auto stop_as = defer([&as] { as.stop().get(); }); + sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), qos::service_level_options{.shares = 1000}, default_scheduling_group).get(); seastar::sharded compressor_tracker; compressor_tracker.start([] { return utils::walltime_compressor_tracker::config{}; }).get(); auto stop_compressor_tracker = deferred_stop(compressor_tracker); @@ -203,7 +212,8 @@ int main(int ac, char ** av) { gossip_address_map.start().get(); seastar::sharded messaging; messaging.start(locator::host_id{}, listen, 7000, std::ref(feature_service), - std::ref(gossip_address_map), std::ref(compressor_tracker)).get(); + std::ref(gossip_address_map), std::ref(compressor_tracker), + std::ref(sl_controller)).get(); auto stop_messaging = deferred_stop(messaging); seastar::sharded testers; testers.start(std::ref(messaging)).get(); From 6d90a933cdcba0be603f5a59bd550019eb65ef14 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Wed, 18 Dec 2024 08:39:13 +0100 Subject: [PATCH 149/397] transport/server: use scheduling group assigned to current user Now, when the user logs in and the connection becomes authenticated, the processing loop of the connection is switched to the scheduling group that corresponds to the service level assigned to the logged in user. The scheduling group is also updated when the service level assigned to this user changes. Starting from this commit, the scheduling groups managed by the service level controller are actually being used by user workload. --- client_data.hh | 1 + generic_server.cc | 28 +++++++++++++++++++- generic_server.hh | 12 +++++++++ service/qos/service_level_controller.cc | 11 ++++++++ service/qos/service_level_controller.hh | 28 ++++++++++++++++++++ transport/controller.cc | 10 +++++++ transport/controller.hh | 1 + transport/server.cc | 35 +++++++++++++++++++++++-- transport/server.hh | 7 +++++ 9 files changed, 130 insertions(+), 3 deletions(-) diff --git a/client_data.hh b/client_data.hh index 91a77c13b746..f2f6a2370fe7 100644 --- a/client_data.hh +++ b/client_data.hh @@ -45,6 +45,7 @@ struct client_data { std::optional ssl_enabled; std::optional ssl_protocol; std::optional username; + std::optional scheduling_group_name; sstring stage_str() const { return to_string(connection_stage); } sstring client_type_str() const { return to_string(ct); } diff --git a/generic_server.cc b/generic_server.cc index 96da666fe944..9076034bb8f1 100644 --- a/generic_server.cc +++ b/generic_server.cc @@ -38,6 +38,19 @@ connection::~connection() _server._connections_list.erase(iter); } +connection::execute_under_tenant_type +connection::no_tenant() { + // return a function that runs the process loop with no scheduling group games + return [] (connection_process_loop loop) { + return loop(); + }; +} + +void connection::switch_tenant(execute_under_tenant_type exec) { + _execute_under_current_tenant = std::move(exec); + _tenant_switch = true; +} + future<> server::for_each_gently(noncopyable_function fn) { _gentle_iterators.emplace_front(*this); std::list::iterator gi = _gentle_iterators.begin(); @@ -63,13 +76,26 @@ static bool is_broken_pipe_or_connection_reset(std::exception_ptr ep) { return false; } +future<> connection::process_until_tenant_switch() { + _tenant_switch = false; + { + return do_until([this] { + return _read_buf.eof() || _tenant_switch; + }, [this] { + return process_request(); + }); + } +} + future<> connection::process() { return with_gate(_pending_requests_gate, [this] { return do_until([this] { return _read_buf.eof(); }, [this] { - return process_request(); + return _execute_under_current_tenant([this] { + return process_until_tenant_switch(); + }); }).then_wrapped([this] (future<> f) { handle_error(std::move(f)); }); diff --git a/generic_server.hh b/generic_server.hh index 9a1db7ab9289..5d856f9b8792 100644 --- a/generic_server.hh +++ b/generic_server.hh @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -35,6 +36,11 @@ class server; // member function to perform request processing. This base class provides a // `_read_buf` and a `_write_buf` for reading requests and writing responses. class connection : public boost::intrusive::list_base_hook<> { +public: + using connection_process_loop = noncopyable_function ()>; + using execute_under_tenant_type = noncopyable_function (connection_process_loop)>; + bool _tenant_switch = false; + execute_under_tenant_type _execute_under_current_tenant = no_tenant(); protected: server& _server; connected_socket _fd; @@ -44,6 +50,8 @@ protected: seastar::gate _pending_requests_gate; seastar::gate::holder _hold_server; +private: + future<> process_until_tenant_switch(); public: connection(server& server, connected_socket&& fd); virtual ~connection(); @@ -57,6 +65,10 @@ public: virtual void on_connection_close(); virtual future<> shutdown(); + + void switch_tenant(execute_under_tenant_type execute); + + static execute_under_tenant_type no_tenant(); }; // A generic TCP socket server. diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index ea62020fdf58..6e56ef5dfb11 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -536,6 +536,17 @@ scheduling_group service_level_controller::get_scheduling_group(sstring service_ } } +future service_level_controller::get_user_scheduling_group(const std::optional& usr) { + if (usr && usr->name) { + auto sl_opt = co_await find_effective_service_level(*usr->name); + auto& sl_name = (sl_opt && sl_opt->shares_name) ? *sl_opt->shares_name : default_service_level_name; + co_return get_scheduling_group(sl_name); + } + else { + co_return get_default_scheduling_group(); + } +} + std::optional service_level_controller::get_active_service_level() { unsigned sched_idx = internal::scheduling_group_index(current_scheduling_group()); if (_sl_lookup[sched_idx].first) { diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 5b85b1fd5dcf..9004a1625cd9 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -206,6 +206,28 @@ public: void abort_group0_operations(); + /** + * this is an executor of a function with arguments under a service level + * that corresponds to a given user. + * @param usr - the user for determining the service level + * @param func - the function to be executed + * @return a future that is resolved when the function's operation is resolved + * (if it returns a future). or a ready future containing the returned value + * from the function/ + */ + template > + requires std::invocable + futurize_t with_user_service_level(const std::optional& usr, Func&& func) { + if (usr && usr->name) { + return find_effective_service_level(*usr->name).then([this, func = std::move(func)] (std::optional opts) mutable { + auto& service_level_name = (opts && opts->shares_name) ? *opts->shares_name : default_service_level_name; + return with_service_level(service_level_name, std::move(func)); + }); + } else { + return with_service_level(default_service_level_name, std::move(func)); + } + } + /** * this is an executor of a function with arguments under a specific * service level. @@ -235,6 +257,12 @@ public: * get_scheduling_group("default") */ scheduling_group get_scheduling_group(sstring service_level_name); + /** + * Get the scheduling group of a specific user + * @param user - the user for determining the service level + * @return if the user is authenticated the user's scheduling group. otherwise get_scheduling_group("default") + */ + future get_user_scheduling_group(const std::optional& usr); /** * @return the name of the currently active service level if such exists or an empty * optional if no active service level. diff --git a/transport/controller.cc b/transport/controller.cc index e3b3fd435a34..458ba7beeb63 100644 --- a/transport/controller.cc +++ b/transport/controller.cc @@ -351,6 +351,16 @@ future> controller::get_client_data() { return _server ? _server->local().get_client_data() : protocol_server::get_client_data(); } +future<> controller::update_connections_scheduling_group() { + if (!_server) { + co_return; + } + + co_await _server->invoke_on_all([] (auto& server) { + return server.update_connections_scheduling_group(); + }); +} + future> controller::get_connections_service_level_params() { if (!_server) { co_return std::vector(); diff --git a/transport/controller.hh b/transport/controller.hh index 30d02077b466..35ab8fdbb7a2 100644 --- a/transport/controller.hh +++ b/transport/controller.hh @@ -79,6 +79,7 @@ public: virtual future<> stop_server() override; virtual future<> request_stop_server() override; virtual future> get_client_data() override; + future<> update_connections_scheduling_group(); future> get_connections_service_level_params(); }; diff --git a/transport/server.cc b/transport/server.cc index 64690029c238..39a9e65858b9 100644 --- a/transport/server.cc +++ b/transport/server.cc @@ -16,14 +16,17 @@ #include "cql3/statements/batch_statement.hh" #include "cql3/statements/modification_statement.hh" +#include "seastar/core/scheduling.hh" #include "types/collection.hh" #include "types/list.hh" #include "types/set.hh" #include "types/map.hh" #include "dht/token-sharding.hh" #include "service/migration_manager.hh" +#include "service/storage_service.hh" #include "service/memory_limiter.hh" #include "service/storage_proxy.hh" +#include "service/qos/service_level_controller.hh" #include "db/consistency_level_type.hh" #include "db/write_type.hh" #include @@ -199,12 +202,15 @@ cql_sg_stats::cql_sg_stats(maintenance_socket_enabled used_by_maintenance_socket if (std::find(vector_ref.begin(), vector_ref.end(), current_scheduling_group().name()) != vector_ref.end()) { return; } + + _use_metrics = true; register_metrics(); } void cql_sg_stats::register_metrics() { namespace sm = seastar::metrics; + auto new_metrics = sm::metric_groups(); std::vector transport_metrics; auto cur_sg_name = current_scheduling_group().name(); @@ -230,7 +236,14 @@ void cql_sg_stats::register_metrics() ); } - _metrics.add_group("transport", std::move(transport_metrics)); + new_metrics.add_group("transport", std::move(transport_metrics)); + _metrics = std::exchange(new_metrics, {}); +} + +void cql_sg_stats::rename_metrics() { + if (_use_metrics) { + register_metrics(); + } } cql_server::cql_server(distributed& qp, auth::service& auth_service, @@ -605,6 +618,7 @@ cql_server::connection::connection(cql_server& server, socket_address server_add , _server(server) , _server_addr(server_addr) , _client_state(service::client_state::external_tag{}, server._auth_service, &server._sl_controller, server.timeout_config(), addr) + , _current_scheduling_group(default_scheduling_group()) { _shedding_timer.set_callback([this] { clogger.debug("Shedding all incoming requests due to overload"); @@ -640,6 +654,7 @@ client_data cql_server::connection::make_client_data() const { } else if (_authenticating) { cd.connection_stage = client_connection_stage::authenticating; } + cd.scheduling_group_name = _current_scheduling_group.name(); return cd; } @@ -933,6 +948,14 @@ future> cql_server::connection::process_st co_return res; } +void cql_server::connection::update_scheduling_group() { + switch_tenant([this] (noncopyable_function ()> process_loop) -> future<> { + auto shg = co_await _server._sl_controller.get_user_scheduling_group(_client_state.user()); + _current_scheduling_group = shg; + co_return co_await _server._sl_controller.with_user_service_level(_client_state.user(), std::move(process_loop)); + }); +} + future> cql_server::connection::process_auth_response(uint16_t stream, request_reader in, service::client_state& client_state, tracing::trace_state_ptr trace_state) { auto sasl_challenge = client_state.get_auth_service()->underlying_authenticator().new_sasl_challenge(); @@ -941,6 +964,7 @@ future> cql_server::connection::process_au if (sasl_challenge->is_complete()) { return sasl_challenge->get_authenticated_user().then([this, sasl_challenge, stream, &client_state, challenge = std::move(challenge), trace_state](auth::authenticated_user user) mutable { client_state.set_login(std::move(user)); + update_scheduling_group(); auto f = client_state.check_user_can_login(); f = f.then([&client_state] { return client_state.maybe_update_per_service_level_params(); @@ -1230,7 +1254,6 @@ process_batch_internal(service::client_state& client_state, distributed(ps->statement.get()) == nullptr) { throw exceptions::invalid_request_exception("Invalid statement in batch: only UPDATE, INSERT and DELETE statements are allowed."); } - ::shared_ptr modif_statement_ptr = static_pointer_cast(ps->statement); if (init_trace) { tracing::add_table_name(trace_state, modif_statement_ptr->keyspace(), modif_statement_ptr->column_family()); @@ -2053,6 +2076,13 @@ future> cql_server::get_client_data() { co_return ret; } +future<> cql_server::update_connections_scheduling_group() { + return for_each_gently([] (generic_server::connection& conn) { + connection& cql_conn = dynamic_cast(conn); + cql_conn.update_scheduling_group(); + }); +} + future<> cql_server::update_connections_service_level_params() { if (!_sl_controller.is_v2()) { // Auto update of connections' service level params requires @@ -2071,6 +2101,7 @@ future<> cql_server::update_connections_service_level_params() { cs.update_per_service_level_params(*slo); } } + cql_conn.update_scheduling_group(); }); } diff --git a/transport/server.hh b/transport/server.hh index 527ea508e744..9e4308a6dfe5 100644 --- a/transport/server.hh +++ b/transport/server.hh @@ -10,6 +10,7 @@ #include "auth/service.hh" #include +#include "seastar/core/scheduling.hh" #include "service/endpoint_lifecycle_subscriber.hh" #include "service/migration_listener.hh" #include "auth/authenticator.hh" @@ -130,7 +131,9 @@ struct cql_sg_stats { cql_sg_stats(maintenance_socket_enabled); request_kind_stats& get_cql_opcode_stats(cql_binary_opcode op) { return _cql_requests_stats[static_cast(op)]; } void register_metrics(); + void rename_metrics(); private: + bool _use_metrics = false; seastar::metrics::metric_groups _metrics; std::vector _cql_requests_stats; }; @@ -198,6 +201,7 @@ public: } future> get_client_data(); + future<> update_connections_scheduling_group(); future<> update_connections_service_level_params(); future> get_connections_service_level_params(); private: @@ -214,10 +218,12 @@ private: cql_compression _compression = cql_compression::none; service::client_state _client_state; timer _shedding_timer; + scheduling_group _current_scheduling_group; bool _shed_incoming_requests = false; unsigned _request_cpu = 0; bool _ready = false; bool _authenticating = false; + bool _tenant_switch = false; enum class tracing_request_type : uint8_t { not_requested, @@ -244,6 +250,7 @@ private: static std::pair make_client_key(const service::client_state& cli_state); client_data make_client_data() const; const service::client_state& get_client_state() const { return _client_state; } + void update_scheduling_group(); service::client_state& get_client_state() { return _client_state; } private: friend class process_request_executor; From 0f62eb45d136dbc35c56d7bf2af3c1f8618618b4 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 20:41:57 +0100 Subject: [PATCH 150/397] cql3/statements: update SL statements for workload prioritization Introduce the "SHARES" keyword which can be used in conjunction with existing CQL statements related to the service levels. Adjust the CQL statements for service levels: - CREATE/ALTER now allow to set shares (only if the cluster is fully upgraded) - LIST EFFECTIVE SERVICE LEVEL now return the number of shares in a new column - LIST SERVICE LEVEL(S) also return the number of shares, and has the additional column "percentage of all service level shares" --- cql3/Cql.g | 2 + .../alter_service_level_statement.cc | 1 + .../create_service_level_statement.cc | 11 +++- .../list_effective_service_level_statement.cc | 19 +++++++ .../list_service_level_statement.cc | 50 +++++++++++++++++-- cql3/statements/service_level_statement.cc | 8 +++ cql3/statements/service_level_statement.hh | 3 ++ cql3/statements/sl_prop_defs.cc | 12 ++++- cql3/statements/sl_prop_defs.hh | 7 ++- test/boost/cql_query_test.cc | 22 ++++++-- 10 files changed, 122 insertions(+), 13 deletions(-) diff --git a/cql3/Cql.g b/cql3/Cql.g index f62298d14ccc..628c823737f1 100644 --- a/cql3/Cql.g +++ b/cql3/Cql.g @@ -2127,6 +2127,7 @@ basic_unreserved_keyword returns [sstring str] | K_SERVICE_LEVELS | K_ATTACHED | K_FOR + | K_SHARES | K_GROUP | K_TIMEOUT | K_SERVICE @@ -2335,6 +2336,7 @@ K_SERVICE: S E R V I C E; K_LEVEL: L E V E L; K_LEVELS: L E V E L S; K_EFFECTIVE: E F F E C T I V E; +K_SHARES: S H A R E S; K_SCYLLA_TIMEUUID_LIST_INDEX: S C Y L L A '_' T I M E U U I D '_' L I S T '_' I N D E X; K_SCYLLA_COUNTER_SHARD_LIST: S C Y L L A '_' C O U N T E R '_' S H A R D '_' L I S T; diff --git a/cql3/statements/alter_service_level_statement.cc b/cql3/statements/alter_service_level_statement.cc index c1a9a15b614e..b47b4460f1d0 100644 --- a/cql3/statements/alter_service_level_statement.cc +++ b/cql3/statements/alter_service_level_statement.cc @@ -38,6 +38,7 @@ alter_service_level_statement::execute(query_processor& qp, service::query_state &state, const query_options &, std::optional guard) const { service::group0_batch mc{std::move(guard)}; + validate_shares_option(qp, _slo); qos::service_level& sl = state.get_service_level_controller().get_service_level(_service_level); qos::service_level_options slo = _slo.replace_defaults(sl.slo); auto& slc = state.get_service_level_controller(); diff --git a/cql3/statements/create_service_level_statement.cc b/cql3/statements/create_service_level_statement.cc index 394741685e91..5f67b0bcec5b 100644 --- a/cql3/statements/create_service_level_statement.cc +++ b/cql3/statements/create_service_level_statement.cc @@ -10,9 +10,11 @@ #include "exceptions/exceptions.hh" #include "seastarx.hh" #include "cql3/statements/create_service_level_statement.hh" +#include "service/qos/qos_common.hh" #include "service/qos/service_level_controller.hh" #include "service/client_state.hh" #include "service/query_state.hh" +#include "utils/error_injection.hh" namespace cql3 { @@ -44,7 +46,14 @@ create_service_level_statement::execute(query_processor& qp, } service::group0_batch mc{std::move(guard)}; - qos::service_level_options slo = _slo.replace_defaults(qos::service_level_options{}); + validate_shares_option(qp, _slo); + + auto default_slo = qos::service_level_options{.shares = qos::service_level_controller::default_shares}; + if (utils::get_local_injector().is_enabled("create_service_levels_without_default_shares")) { + default_slo.shares = qos::service_level_options::unset_marker{}; + } + qos::service_level_options slo = _slo.replace_defaults(default_slo); + auto& sl = state.get_service_level_controller(); co_await sl.add_distributed_service_level(_service_level, slo, _if_not_exists, mc); co_await sl.commit_mutations(std::move(mc)); diff --git a/cql3/statements/list_effective_service_level_statement.cc b/cql3/statements/list_effective_service_level_statement.cc index 918c6dbf1080..9059662d3d51 100644 --- a/cql3/statements/list_effective_service_level_statement.cc +++ b/cql3/statements/list_effective_service_level_statement.cc @@ -53,6 +53,20 @@ static bytes_opt decompose_timeout (const qos::service_level_options::timeout_ty }, duration); }; +static bytes_opt decompose_shares(const qos::service_level_options::shares_type& shares) { + return std::visit(overloaded_functor{ + [&] (const qos::service_level_options::unset_marker&) { + return bytes_opt(); + }, + [&] (const qos::service_level_options::delete_marker&) { + return bytes_opt(); + }, + [&] (const int32_t& s) -> bytes_opt { + return utf8_type->decompose(fmt::format("{}", s)); + }, + }, shares); +}; + future<::shared_ptr> list_effective_service_level_statement::execute(query_processor& qp, service::query_state& state, const query_options&, std::optional) const { static thread_local const std::vector> metadata({ @@ -84,6 +98,11 @@ list_effective_service_level_statement::execute(query_processor& qp, service::qu utf8_type->decompose(slo->effective_names->timeout), decompose_timeout(slo->timeout) }); + rs->add_row({ + utf8_type->decompose("shares"), + utf8_type->decompose(slo->effective_names->shares), + decompose_shares(slo->shares) + }); auto rows = ::make_shared(result(std::move(std::move(rs)))); co_return ::static_pointer_cast(rows); diff --git a/cql3/statements/list_service_level_statement.cc b/cql3/statements/list_service_level_statement.cc index a9b861678df0..2a7fd9428895 100644 --- a/cql3/statements/list_service_level_statement.cc +++ b/cql3/statements/list_service_level_statement.cc @@ -47,10 +47,14 @@ list_service_level_statement::execute(query_processor& qp, type); }; - static thread_local const std::vector> metadata({make_column("service_level", utf8_type), + std::vector> metadata({make_column("service_level", utf8_type), make_column("timeout", duration_type), - make_column("workload_type", utf8_type) + make_column("workload_type", utf8_type), + make_column("shares", int32_type), }); + if (_describe_all) { + metadata.push_back(make_column("percentage of all service level shares", utf8_type)); + } return make_ready_future().then([this, &state] () { if (_describe_all) { @@ -59,7 +63,7 @@ list_service_level_statement::execute(query_processor& qp, return state.get_service_level_controller().get_distributed_service_level(_service_level); } }) - .then([] (qos::service_levels_info sl_info) { + .then([this, metadata = std::move(metadata)] (qos::service_levels_info sl_info) { auto d = [] (const qos::service_level_options::timeout_type& duration) -> bytes_opt { return std::visit(overloaded_functor{ [&] (const qos::service_level_options::unset_marker&) { @@ -74,15 +78,51 @@ list_service_level_statement::execute(query_processor& qp, }, }, duration); }; + auto dd = [] (const std::variant& v) -> bytes_opt { + return std::visit(overloaded_functor{ + [&] (const qos::service_level_options::unset_marker&) { + return bytes_opt(); + }, + [&] (const qos::service_level_options::delete_marker&) { + return bytes_opt(); + }, + [&] (const T& v) -> bytes_opt { + return data_type_for()->decompose(v); + }, + }, v); + }; + auto get_shares_value = [] (const std::variant& shares) { + if (std::holds_alternative(shares)) { + return std::get(shares); + } else { + return qos::service_level_controller::default_shares; + } + }; + + int32_t sum_of_shares = 0; + if (_describe_all) { + for (auto &&[_, slo]: sl_info) { + sum_of_shares += get_shares_value(slo.shares); + } + } + auto rs = std::make_unique(metadata); for (auto &&[sl_name, slo] : sl_info) { bytes_opt workload = slo.workload == qos::service_level_options::workload_type::unspecified ? bytes_opt() : utf8_type->decompose(qos::service_level_options::to_string(slo.workload)); - rs->add_row(std::vector{ + + auto row = std::vector{ utf8_type->decompose(sl_name), d(slo.timeout), - workload}); + workload, + dd(slo.shares)}; + if (_describe_all) { + row.push_back(utf8_type->decompose( + fmt::format("{:.2f}%", 100.0f * get_shares_value(slo.shares) / sum_of_shares) + )); + } + rs->add_row(std::move(row)); } auto rows = ::make_shared(result(std::move(std::move(rs)))); diff --git a/cql3/statements/service_level_statement.cc b/cql3/statements/service_level_statement.cc index d0e48b64542a..a91acbeb76d9 100644 --- a/cql3/statements/service_level_statement.cc +++ b/cql3/statements/service_level_statement.cc @@ -7,6 +7,8 @@ */ #include "service_level_statement.hh" +#include "service/storage_proxy.hh" +#include "gms/feature_service.hh" namespace cql3 { @@ -28,5 +30,11 @@ bool service_level_statement::needs_guard(query_processor&, service::query_state return state.get_service_level_controller().is_v2(); } +void service_level_statement::validate_shares_option(const query_processor& qp, const qos::service_level_options& slo) const { + if (!std::holds_alternative(slo.shares) && !qp.proxy().features().workload_prioritization) { + throw exceptions::invalid_request_exception("`shares` option can only be used when the cluster is fully upgraded to enterprise"); + } +} + } } diff --git a/cql3/statements/service_level_statement.hh b/cql3/statements/service_level_statement.hh index 354fb2c4ee60..7aad668114c0 100644 --- a/cql3/statements/service_level_statement.hh +++ b/cql3/statements/service_level_statement.hh @@ -11,6 +11,7 @@ #include "cql3/cql_statement.hh" #include "cql3/query_processor.hh" #include "raw/parsed_statement.hh" +#include "service/qos/qos_common.hh" #include "service/query_state.hh" namespace cql3 { @@ -49,6 +50,8 @@ public: bool depends_on(std::string_view ks_name, std::optional cf_name) const override; future<> check_access(query_processor& qp, const service::client_state& state) const override; +protected: + void validate_shares_option(const query_processor& qp, const qos::service_level_options& slo) const; }; } diff --git a/cql3/statements/sl_prop_defs.cc b/cql3/statements/sl_prop_defs.cc index 5e9230feae88..a35840ffed9f 100644 --- a/cql3/statements/sl_prop_defs.cc +++ b/cql3/statements/sl_prop_defs.cc @@ -17,7 +17,7 @@ namespace statements { void sl_prop_defs::validate() { static std::set timeout_props { - "timeout", "workload_type" + "timeout", "workload_type", sstring(KW_SHARES), }; auto get_duration = [&] (const std::optional& repr) -> qos::service_level_options::timeout_type { if (!repr) { @@ -42,6 +42,7 @@ void sl_prop_defs::validate() { property_definitions::validate(timeout_props); _slo.timeout = get_duration(get_simple("timeout")); + auto workload_string_opt = get_simple("workload_type"); if (workload_string_opt) { auto workload = qos::service_level_options::parse_workload_type(*workload_string_opt); @@ -55,6 +56,15 @@ void sl_prop_defs::validate() { _slo.workload = qos::service_level_options::workload_type::delete_marker; } } + + if (has_property(KW_SHARES)) { + auto shares = get_int(KW_SHARES, SHARES_DEFAULT_VAL); + if ((shares < SHARES_MIN_VAL) || (shares > SHARES_MAX_VAL )) { + throw exceptions::syntax_exception(format("'SHARES' can only take values of {}-{} (given {})", + SHARES_MIN_VAL, SHARES_MAX_VAL, shares)); + } + _slo.shares = shares; + } } qos::service_level_options sl_prop_defs::get_service_level_options() const { diff --git a/cql3/statements/sl_prop_defs.hh b/cql3/statements/sl_prop_defs.hh index 072eb29e86cb..27669603e0fb 100644 --- a/cql3/statements/sl_prop_defs.hh +++ b/cql3/statements/sl_prop_defs.hh @@ -18,8 +18,13 @@ namespace statements { class sl_prop_defs : public property_definitions { qos::service_level_options _slo; public: - void validate(); + + static constexpr auto KW_SHARES = "shares"; + static constexpr int SHARES_DEFAULT_VAL = 1000; + static constexpr int SHARES_MIN_VAL = 1; + static constexpr int SHARES_MAX_VAL = 1000; + qos::service_level_options get_service_level_options() const; }; diff --git a/test/boost/cql_query_test.cc b/test/boost/cql_query_test.cc index 008b07525086..95a7308081c3 100644 --- a/test/boost/cql_query_test.cc +++ b/test/boost/cql_query_test.cc @@ -40,6 +40,7 @@ #include "db/extensions.hh" #include "cql3/cql_config.hh" #include "test/lib/exception_utils.hh" +#include "service/qos/qos_common.hh" #include "utils/rjson.hh" #include "schema/schema_builder.hh" #include "service/migration_manager.hh" @@ -5061,14 +5062,21 @@ SEASTAR_TEST_CASE(test_user_based_sla_queries) { e.execute_cql("CREATE SERVICE_LEVEL sl_1;").get(); auto msg = e.execute_cql("LIST SERVICE_LEVEL sl_1;").get(); assert_that(msg).is_rows().with_rows({ - {utf8_type->decompose("sl_1"), {}, {}}, + {utf8_type->decompose("sl_1"), {}, {}, int32_type->decompose(1000)}, + }); + //create and alter service levels + e.execute_cql("CREATE SERVICE_LEVEL sl_2 WITH SHARES = 200;").get(); + e.execute_cql("ALTER SERVICE_LEVEL sl_1 WITH SHARES = 111;").get(); + msg = e.execute_cql("LIST ALL SERVICE_LEVELS;").get(); + assert_that(msg).is_rows().with_rows({ + {utf8_type->decompose("sl_1"), {}, {}, int32_type->decompose(111), utf8_type->decompose("35.69%")}, + {utf8_type->decompose("sl_2"), {}, {}, int32_type->decompose(200), utf8_type->decompose("64.31%")}, }); - e.execute_cql("CREATE SERVICE_LEVEL sl_2;").get(); //drop service levels e.execute_cql("DROP SERVICE_LEVEL sl_1;").get(); msg = e.execute_cql("LIST ALL SERVICE_LEVELS;").get(); assert_that(msg).is_rows().with_rows({ - {utf8_type->decompose("sl_2"), {}, {}}, + {utf8_type->decompose("sl_2"), {}, {}, int32_type->decompose(200), utf8_type->decompose("100.00%")}, }); // validate exceptions (illegal requests) @@ -5076,9 +5084,12 @@ SEASTAR_TEST_CASE(test_user_based_sla_queries) { e.execute_cql("DROP SERVICE_LEVEL IF EXISTS sl_1;").get(); BOOST_REQUIRE_THROW(e.execute_cql("CREATE SERVICE_LEVEL sl_2;").get(), exceptions::invalid_request_exception); - BOOST_REQUIRE_THROW(e.execute_cql("CREATE SERVICE_LEVEL sl_2;").get(), exceptions::invalid_request_exception); + BOOST_REQUIRE_THROW(e.execute_cql("CREATE SERVICE_LEVEL sl_2 WITH SHARES = 999;").get(), exceptions::invalid_request_exception); e.execute_cql("CREATE SERVICE_LEVEL IF NOT EXISTS sl_2;").get(); + BOOST_REQUIRE_THROW(e.execute_cql("CREATE SERVICE_LEVEL sl_1 WITH SHARES = 0;").get(), exceptions::syntax_exception); + BOOST_REQUIRE_THROW(e.execute_cql("CREATE SERVICE_LEVEL sl_1 WITH SHARES = 1001;").get(), exceptions::syntax_exception); + // test attach role e.execute_cql("ATTACH SERVICE_LEVEL sl_2 TO tester").get(); msg = e.execute_cql("LIST ATTACHED SERVICE_LEVEL OF tester;").get(); @@ -5096,7 +5107,7 @@ SEASTAR_TEST_CASE(test_user_based_sla_queries) { BOOST_CHECK(true); // tests detaching service levels e.execute_cql("CREATE ROLE tester2;").get(); - e.execute_cql("CREATE SERVICE_LEVEL sl_1;").get(); + e.execute_cql("CREATE SERVICE_LEVEL sl_1 WITH SHARES = 998;").get(); e.execute_cql("ATTACH SERVICE_LEVEL sl_1 TO tester2;").get(); e.execute_cql("DETACH SERVICE_LEVEL FROM tester;").get(); msg = e.execute_cql("LIST ATTACHED SERVICE_LEVEL OF tester2;").get(); @@ -5130,6 +5141,7 @@ SEASTAR_TEST_CASE(test_user_based_sla_queries) { msg = e.execute_cql("LIST ALL ATTACHED SERVICE_LEVELS;").get(); assert_that(msg).is_rows().with_rows({ }); + BOOST_REQUIRE_THROW(e.execute_cql("ALTER SERVICE_LEVEL i_do_not_exist WITH shares = 1;").get(), exceptions::invalid_request_exception); }); } From ce4032dfc0df06389ba0c8d4fad6ff7471c92edf Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 21:40:42 +0100 Subject: [PATCH 151/397] qos: include number of shares in DESCRIBE Now, the CREATE statements generated for each service level by the DESCRIBE SCHEMA WITH INTERNALS statement will account for the service level's shares. --- service/qos/service_level_controller.cc | 6 ++- test/cqlpy/test_describe.py | 58 +++++++++++++++++++++---- 2 files changed, 54 insertions(+), 10 deletions(-) diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index 6e56ef5dfb11..3399c8ab4afa 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -917,7 +917,7 @@ future<> service_level_controller::unregister_subscriber(qos_configuration_chang static sstring describe_service_level(std::string_view sl_name, const service_level_options& sl_opts) { using slo = service_level_options; - utils::small_vector opts{}; + utils::small_vector opts{}; const sstring sl_name_formatted = cql3::util::maybe_quote(sl_name); @@ -943,6 +943,10 @@ static sstring describe_service_level(std::string_view sl_name, const service_le on_internal_error(sl_logger, "Unexpected workload type"); } + if (auto* maybe_shares = std::get_if(&sl_opts.shares)) { + opts.push_back(seastar::format("SHARES = {}", *maybe_shares)); + } + if (opts.size() == 0) { return seastar::format("CREATE SERVICE LEVEL {};", sl_name_formatted); } diff --git a/test/cqlpy/test_describe.py b/test/cqlpy/test_describe.py index 618c90a0aac2..f79d50d3e5ea 100644 --- a/test/cqlpy/test_describe.py +++ b/test/cqlpy/test_describe.py @@ -1488,19 +1488,28 @@ def __exit__(self, exc_type, exc_val, exc_tb) -> None: self.cql.execute(f"DROP SERVICE LEVEL {make_identifier(sl, quotation_mark='"')}") class ServiceLevel: - def __init__(self, name: str, timeout: int|None = None, wl_type: str|None = None): + default_shares_value = 1000 + + def __init__(self, name: str, timeout: int|None = None, wl_type: str|None = None, shares: int|None = None): self.name = name self.timeout = timeout self.wl_type = wl_type + self.shares = shares - def get_create_stmt(self) -> str: + # replace_default_shares - Scylla automatically assigns default value of shares it they are not + # specified. Set this argument to True to include the default shares in create statement + # to match describe result. + def get_create_stmt(self, replace_default_shares = False) -> str: # Note: `CREATE SERVICE LEVEL` statements returned by `DESC SCHEMA WITH INTERNALS` always uses # `std::chrono::milliseconds` as its resolution. For that reason, we use milliseconds in # create statements too so that they're easy to compare with Scylla's output. timeout = None if not self.timeout else f"TIMEOUT = {self.timeout}ms" wl_type = None if not self.wl_type else f"WORKLOAD_TYPE = '{self.wl_type}'" + shares = None if not self.shares else f"SHARES = {self.shares}" + if shares is None and replace_default_shares: + shares = f"SHARES = {self.default_shares_value}" - opts = [opt for opt in [timeout, wl_type] if opt is not None] + opts = [opt for opt in [timeout, wl_type, shares] if opt is not None] if opts: return f"CREATE SERVICE LEVEL {self.name} WITH {" AND ".join(opts)};" @@ -2358,7 +2367,7 @@ def test_desc_service_levels_format(cql): assert result.keyspace_name == None assert result.type == "service_level" assert result.name == sl.name - assert result.create_statement == sl.get_create_stmt() + assert result.create_statement == sl.get_create_stmt(replace_default_shares=True) def test_desc_service_levels_quotation_marks(cql): """ @@ -2387,8 +2396,8 @@ def test_desc_service_levels_quotation_marks(cql): desc_iter = extract_create_statements(desc_elements) expected_result = { - sl1_double_quote.get_create_stmt(), - sl2_double_quote.get_create_stmt() + sl1_double_quote.get_create_stmt(replace_default_shares=True), + sl2_double_quote.get_create_stmt(replace_default_shares=True) } assert set(desc_iter) == expected_result @@ -2412,7 +2421,7 @@ def test_desc_service_levels_uppercase(cql): assert list(sl_iter) == [sl.name] desc_iter = extract_create_statements(desc_elements) - assert list(desc_iter) == [sl.get_create_stmt()] + assert list(desc_iter) == [sl.get_create_stmt(replace_default_shares=True)] def test_desc_service_levels_unicode(cql): """ @@ -2433,7 +2442,7 @@ def test_desc_service_levels_unicode(cql): assert list(sl_iter) == [sl.name] desc_iter = extract_create_statements(desc_elements) - assert list(desc_iter) == [sl.get_create_stmt()] + assert list(desc_iter) == [sl.get_create_stmt(replace_default_shares=True)] def test_desc_auth_service_levels(cql): """ @@ -2452,8 +2461,12 @@ def test_desc_auth_service_levels(cql): # Timeout and workload parameter. ServiceLevel("sl7", timeout=25000, wl_type="interactive") } + service_levels |= { ServiceLevel(sl.name + 's', wl_type=sl.wl_type, timeout=sl.timeout, shares=400) for sl in service_levels } + + sl_create_stmts = set(map(lambda sl: sl.get_create_stmt(replace_default_shares=True), service_levels)) - sl_create_stmts = set(map(lambda sl: sl.get_create_stmt(), service_levels)) + # Enterprise is limited in the number of service levels it supports + sl_create_stmts = set(random.sample(list(sl_create_stmts), k=5)) for stmt in sl_create_stmts: cql.execute(stmt) @@ -2464,6 +2477,33 @@ def test_desc_auth_service_levels(cql): assert sl_create_stmts == set(desc_iter) +def test_desc_service_levels_default_shares(cql): + """ + Verify that DESCRIBE handles the default value of shares correctly: + (a) when a service level is created without specifying the number of shares, + we should get a create statement with the default number of shares, + (b) when a service level is created with the default number of shares but specified explicitly, + we should get a create statement with that number of shares too. + """ + + with AuthSLContext(cql): + default_share_count = 1000 + + stmts = [ + "CREATE SERVICE LEVEL sl_default;", + f"CREATE SERVICE LEVEL sl_set WITH SHARES = {default_share_count};", + ] + + for stmt in stmts: + cql.execute(stmt) + + desc_iter = cql.execute("DESC SCHEMA WITH INTERNALS") + desc_iter = filter_service_levels(desc_iter) + desc_iter = extract_create_statements(desc_iter) + + stmts[0] = f"CREATE SERVICE LEVEL sl_default WITH SHARES = {default_share_count};" + assert stmts == list(desc_iter) + def test_desc_attach_service_level_format(cql): """ Verify that the format of the output of `DESC SCHEMA WITH INTERNALS` corresponding to From bbc655ff32340c4e04f3e1ddfc323c4d0d2c772c Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 08:46:11 +0100 Subject: [PATCH 152/397] test/boost: update service_level_controller_test for workload prio Adjust some of the existing tests in service_level_controller_test.cc and add some more in order to test the workload prioritization features, i.e. the service level shares. --- test/boost/service_level_controller_test.cc | 191 +++++++++++++++++++- test/lib/cql_test_env.hh | 1 + 2 files changed, 187 insertions(+), 5 deletions(-) diff --git a/test/boost/service_level_controller_test.cc b/test/boost/service_level_controller_test.cc index f090fd7adf20..5375cedaaf17 100644 --- a/test/boost/service_level_controller_test.cc +++ b/test/boost/service_level_controller_test.cc @@ -15,6 +15,7 @@ #include #include "seastarx.hh" +#include "service/qos/qos_common.hh" #include "test/lib/scylla_test_case.hh" #include "test/lib/test_utils.hh" #include @@ -112,17 +113,20 @@ SEASTAR_THREAD_TEST_CASE(subscriber_simple) { sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), sl_options, default_scheduling_group).get(); qos_configuration_change_suscriber_simple ccss; sl_controller.local().register_subscriber(&ccss); - sl_controller.local().add_service_level("sl1", service_level_options{}).get(); - sl_controller.local().add_service_level("sl2", service_level_options{}).get(); + sl_controller.local().add_service_level("sl1", sl_options).get(); + sl_controller.local().add_service_level("sl2", sl_options).get(); + sl_controller.local().add_service_level("sl3", service_level_options{}).get(); service_level_options slo; + slo.shares.emplace(500); slo.workload = service_level_options::workload_type::interactive; sl_controller.local().add_service_level("sl1", slo).get(); sl_controller.local().remove_service_level("sl2", false).get(); std::vector expected_result = { - add_op{"sl1", service_level_options{}}, - add_op{"sl2", service_level_options{}}, - change_op{"sl1", service_level_options{}, slo}, + add_op{"sl1", sl_options}, + add_op{"sl2", sl_options}, + add_op{"sl3", service_level_options{}}, + change_op{"sl1", sl_options, slo}, remove_op{"sl2"}, }; @@ -131,3 +135,180 @@ SEASTAR_THREAD_TEST_CASE(subscriber_simple) { as.invoke_on_all([] (auto& as) { as.request_abort(); }).get(); sl_controller.stop().get(); } + +SEASTAR_THREAD_TEST_CASE(too_many_service_levels) { + class data_accessor : public service_level_controller::service_level_distributed_data_accessor { + public: + mutable service_levels_info configuration; + future get_service_levels(qos::query_context) const override { + return make_ready_future(configuration); + } + future get_service_level(sstring service_level_name) const override { + service_levels_info ret; + if (configuration.contains(service_level_name)) { + ret[service_level_name] = configuration[service_level_name]; + } + return make_ready_future(ret); + } + future<> set_service_level(sstring service_level_name, qos::service_level_options slo, service::group0_batch&) const override { + configuration[service_level_name] = slo; + return make_ready_future<>(); + } + future<> drop_service_level(sstring service_level_name, service::group0_batch&) const override { + if (configuration.contains(service_level_name)) { + configuration.erase(service_level_name); + } + return make_ready_future<>(); + } + virtual bool is_v2() const override { + return true; + } + virtual ::shared_ptr upgrade_to_v2(cql3::query_processor& qp, service::raft_group0_client& group0_client) const override { + return make_shared(); + } + virtual future<> commit_mutations(service::group0_batch&& mc, abort_source& as) const override { + return make_ready_future<>(); + } + + }; + + shared_ptr test_accessor = make_shared(); + sharded sl_controller; + sharded auth_service; + service_level_options sl_options; + sl_options.shares.emplace(1000); + sl_options.workload = service_level_options::workload_type::interactive; + scheduling_group default_scheduling_group = create_scheduling_group("sl_default_sg1", 1.0).get(); + locator::shared_token_metadata tm({}, {locator::topology::config{ .local_dc_rack = locator::endpoint_dc_rack::default_location }}); + sharded as; + as.start().get(); + auto stop_as = defer([&as] { as.stop().get(); }); + sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), sl_options, default_scheduling_group, true).get(); + sl_controller.local().set_distributed_data_accessor(test_accessor); + int service_level_id = 0; + unsigned service_level_count = 0; + std::vector expected_service_levels; + while (service_level_count <= max_scheduling_groups()) { + try { + sstring sl_name = format("sl{:020}",service_level_id); + sl_controller.local().add_service_level(sl_name, sl_options).get(); + test_accessor->configuration[sl_name] = sl_options; + expected_service_levels.emplace_back(sl_name); + // create the service levels with gaps, this will allow to later "push" another service + // level between two others if odd id numbers are used. + service_level_id+=2; + service_level_count++; + } catch (std::runtime_error) { + break; + } + } + // If we have failed to create at least 2 service levels the test can pass but it will + // not really test anything. We know that there are a lot more available scheduling groups + // than only two. + BOOST_REQUIRE(service_level_count >= 2); + // make sure the service levels we believe to be active really have been created. + sl_controller.local().update_service_levels_cache().get(); + for (auto&& sl : expected_service_levels) { + BOOST_REQUIRE(sl_controller.local().has_service_level(sl)); + } + // Squize a service level betwin id 0 and id 2 - only to the configuration since + // we know that a creation of another service level will fail. + test_accessor->configuration[format("sl{:020}",1)] = sl_options; + + // do a config poll round + // we expect a failure to apply the configuration since it contains more service levels + // than available scheduling groups. + try { + sl_controller.local().update_service_levels_cache().get(); + } catch (std::runtime_error) { + } + expected_service_levels.clear(); + // Record the state of service levels after a configuration round (with a bad configuration). + for (auto&& sl : test_accessor->configuration) { + const auto& [sl_name, slo] = sl; + if (sl_controller.local().has_service_level(sl_name)) { + expected_service_levels.emplace_back(sl_name); + } + } + sl_controller.stop().get(); + // Simulate a rebooted node which haven't "witnesed" the configuration change and only knows + // the current configuration. + sharded new_sl_controller; + default_scheduling_group = create_scheduling_group("sl_default_sg2", 1.0).get(); + new_sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), sl_options, default_scheduling_group, true).get(); + new_sl_controller.local().set_distributed_data_accessor(test_accessor); + try { + new_sl_controller.local().update_service_levels_cache().get(); + } catch (std::runtime_error) { + } + // Finally, make sure that this rebooted node have the same service levels as the node + // that did "witness" the configuration change. + for (auto&& sl : expected_service_levels) { + BOOST_REQUIRE(new_sl_controller.local().has_service_level(sl)); + } + new_sl_controller.stop().get(); +} + +SEASTAR_THREAD_TEST_CASE(add_remove_bad_sequence) { + sharded sl_controller; + sharded auth_service; + service_level_options sl_options; + sl_options.shares.emplace(1000); + scheduling_group default_scheduling_group = create_scheduling_group("sl_default_sg3", 1.0).get(); + locator::shared_token_metadata tm({}, {locator::topology::config{ .local_dc_rack = locator::endpoint_dc_rack::default_location }}); + sharded as; + as.start().get(); + auto stop_as = defer([&as] { as.stop().get(); }); + sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), sl_options, default_scheduling_group, true).get(); + service_level_options slo; + slo.shares.emplace(500); + slo.workload = service_level_options::workload_type::interactive; + sl_controller.local().add_service_level("a", slo).get(); + sl_controller.local().add_service_level("b", slo).get(); + sl_controller.local().remove_service_level("b", false).get(); + sl_controller.local().remove_service_level("a", false).get(); + sl_controller.local().add_service_level("a", slo).get(); + sl_controller.local().remove_service_level("a", false).get(); + sl_controller.stop().get(); +} + +SEASTAR_THREAD_TEST_CASE(verify_unset_shares_in_cache_when_service_level_created_without_shares) { + using std::literals::chrono_literals::operator""ms; + + sharded sl_controller; + sharded auth_service; + + service_level_options sl_options; + sl_options.shares.emplace(1000); + scheduling_group default_scheduling_group = create_scheduling_group("sl_default_sg", 1.0).get(); + locator::shared_token_metadata tm({}, {locator::topology::config{ .local_dc_rack = locator::endpoint_dc_rack::default_location }}); + sharded as; + + as.start().get(); + auto stop_as = defer([&as] { as.stop().get(); }); + sl_controller.start(std::ref(auth_service), std::ref(tm), std::ref(as), sl_options, default_scheduling_group).get(); + + using timeout_duration = typename seastar::lowres_clock::duration; + using workload_type = typename service_level_options::workload_type; + + std::pair configs[] = { + {"sl_all_default", service_level_options{}}, + {"sl_timeout_set", service_level_options{.timeout = timeout_duration(10ms)}}, + {"sl_workload_set", service_level_options{.workload = workload_type::batch}}, + {"sl_shares_set", service_level_options {.shares = 100}}, + {"sl_timeout_and_workload_set", service_level_options{.timeout = timeout_duration(100ms), .workload = workload_type::interactive}}, + {"sl_timeout_and_shares_set", service_level_options{.timeout = timeout_duration(200ms), .shares = 50}}, + {"sl_workload_and_shares_set", service_level_options{.workload = workload_type::interactive, .shares = 250}}, + {"sl_everything_set", service_level_options{.timeout = timeout_duration(50ms), .workload = workload_type::interactive, .shares = 700}} + }; + + for (const auto& [name, opts] : configs) { + sl_controller.local().add_service_level(name, opts).get(); + const auto& sl = sl_controller.local().get_service_level(name); + BOOST_REQUIRE_MESSAGE(opts == sl.slo, seastar::format("Comparing options of {}", name)); + sl_controller.local().remove_service_level(name, false).get(); + } + + as.invoke_on_all([] (auto& as) { as.request_abort(); }).get(); + sl_controller.stop().get(); +} diff --git a/test/lib/cql_test_env.hh b/test/lib/cql_test_env.hh index 0779da7b2601..22e75273536c 100644 --- a/test/lib/cql_test_env.hh +++ b/test/lib/cql_test_env.hh @@ -16,6 +16,7 @@ #include #include +#include "service/qos/service_level_controller.hh" #include "replica/database.hh" #include "transport/messages/result_message_base.hh" #include "cql3/query_options_fwd.hh" From 9319d65971e424b6b8bfa8cf884e0a614586a0ca Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 17 Dec 2024 08:35:57 +0100 Subject: [PATCH 153/397] db/virtual_tables: add scheduling group column to system.clients Add the "scheduling_group" column to the system.clients table which names the scheduling group that currently serves the connection/client. --- db/virtual_tables.cc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc index 03f9bbc1e5c6..b42be729489e 100644 --- a/db/virtual_tables.cc +++ b/db/virtual_tables.cc @@ -736,6 +736,7 @@ class clients_table : public streaming_virtual_table { .with_column("ssl_enabled", boolean_type) .with_column("ssl_protocol", utf8_type) .with_column("username", utf8_type) + .with_column("scheduling_group", utf8_type) .with_hash_version() .build(); } @@ -842,6 +843,9 @@ class clients_table : public streaming_virtual_table { set_cell(cr.cells(), "ssl_protocol", *cd.ssl_protocol); } set_cell(cr.cells(), "username", cd.username ? *cd.username : sstring("anonymous")); + if (cd.scheduling_group_name) { + set_cell(cr.cells(), "scheduling_group", *cd.scheduling_group_name); + } co_await result.emit_row(std::move(cr)); } co_await result.emit_partition_end(); From a65c0c3735a405bee418c9a2ef48c295db699ffc Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 09:58:46 +0100 Subject: [PATCH 154/397] api/cql_server_test: add information about scheduling group Now, information about connections' scheduling group is included in the HTTP API for querying information about connections' parameters. --- api/cql_server_test.cc | 10 +++++++--- transport/server.cc | 2 +- transport/server.hh | 2 ++ 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/api/cql_server_test.cc b/api/cql_server_test.cc index 77e0301d1b7d..d50394ca858d 100644 --- a/api/cql_server_test.cc +++ b/api/cql_server_test.cc @@ -26,21 +26,24 @@ struct connection_sl_params : public json::json_base { json::json_element _role_name; json::json_element _workload_type; json::json_element _timeout; + json::json_element _scheduling_group; - connection_sl_params(const sstring& role_name, const sstring& workload_type, const sstring& timeout) { + connection_sl_params(const sstring& role_name, const sstring& workload_type, const sstring& timeout, const sstring& scheduling_group) { _role_name = role_name; _workload_type = workload_type; _timeout = timeout; + _scheduling_group = scheduling_group; register_params(); } connection_sl_params(const connection_sl_params& params) - : connection_sl_params(params._role_name(), params._workload_type(), params._timeout()) {} + : connection_sl_params(params._role_name(), params._workload_type(), params._timeout(), params._scheduling_group()) {} void register_params() { add(&_role_name, "role_name"); add(&_workload_type, "workload_type"); add(&_timeout, "timeout"); + add(&_scheduling_group, "scheduling_group"); } }; @@ -54,7 +57,8 @@ void set_cql_server_test(http_context& ctx, seastar::httpd::routes& r, cql_trans return connection_sl_params( std::move(params.role_name), sstring(qos::service_level_options::to_string(params.workload_type)), - to_string(cql_duration(months_counter{0}, days_counter{0}, nanoseconds_counter{nanos}))); + to_string(cql_duration(months_counter{0}, days_counter{0}, nanoseconds_counter{nanos})), + std::move(params.scheduling_group_name)); }); co_return result; }); diff --git a/transport/server.cc b/transport/server.cc index 39a9e65858b9..90c08f1787a5 100644 --- a/transport/server.cc +++ b/transport/server.cc @@ -2115,7 +2115,7 @@ future> cql_server::get_connections ? (user->name ? *(user->name) : "ANONYMOUS") : "UNAUTHENTICATED"; - sl_params.emplace_back(std::move(role_name), client_state.get_timeout_config(), client_state.get_workload_type()); + sl_params.emplace_back(std::move(role_name), client_state.get_timeout_config(), client_state.get_workload_type(), cql_conn.get_scheduling_group().name()); }); co_return sl_params; } diff --git a/transport/server.hh b/transport/server.hh index 9e4308a6dfe5..546f03ebd8c1 100644 --- a/transport/server.hh +++ b/transport/server.hh @@ -142,6 +142,7 @@ struct connection_service_level_params { sstring role_name; timeout_config timeout_config; qos::service_level_options::workload_type workload_type; + sstring scheduling_group_name; }; class cql_server : public seastar::peering_sharded_service, public generic_server::server { @@ -252,6 +253,7 @@ private: const service::client_state& get_client_state() const { return _client_state; } void update_scheduling_group(); service::client_state& get_client_state() { return _client_state; } + scheduling_group get_scheduling_group() const { return _current_scheduling_group; } private: friend class process_request_executor; future>> process_request_one(fragmented_temporary_buffer::istream buf, uint8_t op, uint16_t stream, service::client_state& client_state, tracing_request_type tracing_request, service_permit permit); From 49f5fc0e70310118a8687ead409c317d2e4ce73f Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 09:31:35 +0100 Subject: [PATCH 155/397] api: introduce service levels specific API Introduces two endpoints with operations specific to service levels: - switch_tenants: updates the scheduling group of all connections to be aligned with the service level specific to the logged in user. This is mostly legacy API, as with service levels on raft this is done automatically. - count_connections: for each user and for each scheduling group, counts how many connections are assigned to that user and scheduling group. This API is used in tests. --- api/CMakeLists.txt | 2 + api/api-doc/service_levels.json | 56 ++++++++++ api/api.cc | 7 ++ api/api_init.hh | 5 + api/service_levels.cc | 63 ++++++++++++ api/service_levels.hh | 17 +++ configure.py | 2 + main.cc | 3 + test/cqlpy/test_service_level_api.py | 148 +++++++++++++++++++++++++++ 9 files changed, 303 insertions(+) create mode 100644 api/api-doc/service_levels.json create mode 100644 api/service_levels.cc create mode 100644 api/service_levels.hh create mode 100644 test/cqlpy/test_service_level_api.py diff --git a/api/CMakeLists.txt b/api/CMakeLists.txt index e1e1a1c9ba99..2a6a3fab6f23 100644 --- a/api/CMakeLists.txt +++ b/api/CMakeLists.txt @@ -42,6 +42,7 @@ set(swagger_files api-doc/messaging_service.json api-doc/metrics.json api-doc/raft.json + api-doc/service_levels.json api-doc/storage_proxy.json api-doc/storage_service.json api-doc/stream_manager.json @@ -82,6 +83,7 @@ target_sources(api lsa.cc messaging_service.cc raft.cc + service_levels.cc storage_proxy.cc storage_service.cc stream_manager.cc diff --git a/api/api-doc/service_levels.json b/api/api-doc/service_levels.json new file mode 100644 index 000000000000..58c1cf96bdbf --- /dev/null +++ b/api/api-doc/service_levels.json @@ -0,0 +1,56 @@ +{ + "apiVersion":"0.0.1", + "swaggerVersion":"1.2", + "basePath":"{{Protocol}}://{{Host}}", + "resourcePath":"/service_levels", + "produces":[ + "application/json" + ], + "apis":[ + { + "path":"/service_levels/switch_tenants", + "operations":[ + { + "method":"POST", + "summary":"Switch tenants on all opened connections if needed", + "type":"void", + "nickname":"do_switch_tenants", + "produces":[ + "application/json" + ], + "parameters":[] + } + ] + }, + { + "path":"/service_levels/count_connections", + "operations":[ + { + "method":"GET", + "summary":"Count opened CQL connections per scheduling group per user", + "type":"connections_count_map", + "nickname":"count_connections", + "produces":[ + "application/json" + ], + "parameters":[] + } + ] + } + ], + "models":{}, + "components": { + "schemas": { + "connections_count_map": { + "type": "object", + "additionalProperties": { + "type": "object", + "additionalProperties": { + "type": "integer" + } + } + } + } + } + +} \ No newline at end of file diff --git a/api/api.cc b/api/api.cc index f3ba941f0722..ff5c06432d60 100644 --- a/api/api.cc +++ b/api/api.cc @@ -36,6 +36,7 @@ #include "tasks.hh" #include "raft.hh" #include "gms/gossip_address_map.hh" +#include "service_levels.hh" logging::logger apilog("api"); @@ -358,6 +359,12 @@ future<> unset_server_cql_server_test(http_context& ctx) { #endif +future<> set_server_service_levels(http_context &ctx, cql_transport::controller& ctl, sharded& qp) { + return register_api(ctx, "service_levels", "The service levels API", [&ctl, &qp] (http_context& ctx, routes& r) { + set_service_levels(ctx, r, ctl, qp); + }); +} + future<> set_server_tasks_compaction_module(http_context& ctx, sharded& ss, sharded& snap_ctl) { auto rb = std::make_shared < api_registry_builder > (ctx.api_doc); diff --git a/api/api_init.hh b/api/api_init.hh index 236600c79729..435f637c0cab 100644 --- a/api/api_init.hh +++ b/api/api_init.hh @@ -73,6 +73,10 @@ namespace tasks { class task_manager; } +namespace cql3 { +class query_processor; +} + namespace api { struct http_context { @@ -141,6 +145,7 @@ future<> set_format_selector(http_context& ctx, db::sstables_format_selector& se future<> unset_format_selector(http_context& ctx); future<> set_server_cql_server_test(http_context& ctx, cql_transport::controller& ctl); future<> unset_server_cql_server_test(http_context& ctx); +future<> set_server_service_levels(http_context& ctx, cql_transport::controller& ctl, sharded& qp); future<> set_server_commitlog(http_context& ctx, sharded&); future<> unset_server_commitlog(http_context& ctx); diff --git a/api/service_levels.cc b/api/service_levels.cc new file mode 100644 index 000000000000..753a6e319807 --- /dev/null +++ b/api/service_levels.cc @@ -0,0 +1,63 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "service_levels.hh" +#include "api/api-doc/service_levels.json.hh" +#include "cql3/query_processor.hh" +#include "cql3/untyped_result_set.hh" +#include "db/consistency_level_type.hh" +#include "seastar/json/json_elements.hh" +#include "transport/controller.hh" +#include + + +namespace api { + +namespace sl = httpd::service_levels_json; +using namespace json; +using namespace seastar::httpd; + + +void set_service_levels(http_context& ctx, routes& r, cql_transport::controller& ctl, sharded& qp) { + sl::do_switch_tenants.set(r, [&ctl] (std::unique_ptr req) -> future { + co_await ctl.update_connections_scheduling_group(); + co_return json_void(); + }); + + sl::count_connections.set(r, [&qp] (std::unique_ptr req) -> future { + auto connections = co_await qp.local().execute_internal( + "SELECT username, scheduling_group FROM system.clients WHERE client_type='cql' ALLOW FILTERING", + db::consistency_level::LOCAL_ONE, + cql3::query_processor::cache_internal::no + ); + + using connections_per_user = std::unordered_map; + using connections_per_scheduling_group = std::unordered_map; + connections_per_scheduling_group result; + + for (auto it = connections->begin(); it != connections->end(); it++) { + auto user = it->get_as("username"); + auto shg = it->get_as("scheduling_group"); + + if (result.contains(shg)) { + result[shg][user]++; + } + else { + result[shg] = {{user, 1}}; + } + } + + co_return result; + }); + +} + + + + +} \ No newline at end of file diff --git a/api/service_levels.hh b/api/service_levels.hh new file mode 100644 index 000000000000..e2e3993774dd --- /dev/null +++ b/api/service_levels.hh @@ -0,0 +1,17 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "api.hh" + +namespace api { + +void set_service_levels(http_context& ctx, httpd::routes& r, cql_transport::controller& ctl, sharded& qp); + +} \ No newline at end of file diff --git a/configure.py b/configure.py index 5960768e4ff3..68c05bad791b 100755 --- a/configure.py +++ b/configure.py @@ -1215,6 +1215,8 @@ def find_ninja(): Json2Code('api/api-doc/raft.json'), Json2Code('api/api-doc/cql_server_test.json'), 'api/cql_server_test.cc', + 'api/service_levels.cc', + Json2Code('api/api-doc/service_levels.json'), ] alternator = [ diff --git a/main.cc b/main.cc index bc8980cbe875..cd4da43c5214 100644 --- a/main.cc +++ b/main.cc @@ -2286,6 +2286,9 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl // Register controllers after drain_on_shutdown() below, so that even on start // failure drain is called and stops controllers cql_transport::controller cql_server_ctl(auth_service, mm_notifier, gossiper, qp, service_memory_limiter, sl_controller, lifecycle_notifier, *cfg, cql_sg_stats_key, maintenance_socket_enabled::no, dbcfg.statement_scheduling_group); + + api::set_server_service_levels(ctx, cql_server_ctl, qp).get(); + alternator::controller alternator_ctl(gossiper, proxy, mm, sys_dist_ks, cdc_generation_service, service_memory_limiter, auth_service, sl_controller, *cfg, dbcfg.statement_scheduling_group); redis::controller redis_ctl(proxy, auth_service, mm, *cfg, gossiper, dbcfg.statement_scheduling_group); diff --git a/test/cqlpy/test_service_level_api.py b/test/cqlpy/test_service_level_api.py new file mode 100644 index 000000000000..6a6ae25d9c73 --- /dev/null +++ b/test/cqlpy/test_service_level_api.py @@ -0,0 +1,148 @@ +# -*- coding: utf-8 -*- +# Copyright 2024-present ScyllaDB +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + +######################################## +# Tests for the service levels HTTP API. +######################################## + +import pytest +from .rest_api import get_request, post_request +from .util import new_session, unique_name +import time + +def count_opened_connections(cql, retry_unauthenticated=True): + response = get_request(cql, "service_levels/count_connections") + return response + +def switch_tenants(cql): + return post_request(cql, "service_levels/switch_tenants") + +def count_opened_connections_from_table(cql): + connections = cql.execute("SELECT username, scheduling_group FROM system.clients WHERE client_type='cql' ALLOW FILTERING") + result = {} + for row in connections: + user = row[0] + shg = row[1] + + if shg in result: + if user in result[shg]: + result[shg][user] += 1 + else: + result[shg][user] = 1 + else: + result[shg] = {user: 1} + + return result + +def wait_until_all_connections_authenticated(cql, wait_s = 1, timeout_s = 30): + start_time = time.time() + while time.time() - start_time < timeout_s: + result = cql.execute("SELECT COUNT(*) FROM system.clients WHERE username='anonymous' ALLOW FILTERING") + if result.one()[0] == 0: + return + else: + time.sleep(wait_s) + + raise RuntimeError(f"Awaiting for connections authentication timed out.") + +def wait_for_scheduling_group_assignment(cql, user, scheduling_group, wait_s = 2, timeout_s = 60): + start_time = time.time() + while time.time() - start_time < timeout_s: + connections = cql.execute(f"SELECT username, scheduling_group FROM system.clients WHERE client_type='cql' AND username='{user}' ALLOW FILTERING") + + require_wait = False + for row in connections: + if row[1] != f"sl:{scheduling_group}": + require_wait = True + break + if require_wait: + time.sleep(wait_s) + continue + return + + raise RuntimeError(f"Awaiting for user '{user}' to switch tenant to scheduling group '{scheduling_group}' timed out.") + +# Test if `/service_levels/count_connections` prints counted CQL connections +# per scheduling group per user. +def test_count_opened_cql_connections(cql): + user = f"test_user_{unique_name()}" + sl = f"sl_{unique_name()}" + + cql.execute(f"CREATE ROLE {user} WITH login = true AND password='{user}'") + cql.execute(f"CREATE SERVICE LEVEL {sl} WITH shares = 100") + cql.execute(f"ATTACH SERVICE LEVEL {sl} TO {user}") + + # Service level controller updates in 10 seconds interval, so wait + # for sl1 to be assgined to test_user + time.sleep(10) + try: + with new_session(cql, user): # new sessions is created only to create user's connection to Scylla + wait_until_all_connections_authenticated(cql) + wait_for_scheduling_group_assignment(cql, user, sl) + + api_response = count_opened_connections(cql) + assert f"sl:{sl}" in api_response + assert user in api_response[f"sl:{sl}"] + + table_response = count_opened_connections_from_table(cql) + assert api_response == table_response + finally: + cql.execute(f"DETACH SERVICE LEVEL FROM {user}") + cql.execute(f"DROP ROLE {user}") + cql.execute(f"DROP SERVICE LEVEL {sl}") + +# Test if `/service_levels/switch_tenants` updates scheduling group +# of CQL connections without restarting them. +# +# This test creates a `test_user` and 2 service levels `sl1` and `sl2`. +# Firstly the user is assigned to `sl1` and his connections is created. +# Then the test changes user's service level to `sl2` and +# `/service_levels/switch_tenants` endpoint is called. +def test_switch_tenants(cql): + user = f"test_user_{unique_name()}" + sl1 = f"sl1_{unique_name()}" + sl2 = f"sl2_{unique_name()}" + + + cql.execute(f"CREATE ROLE {user} WITH login = true AND password='{user}'") + cql.execute(f"CREATE SERVICE LEVEL {sl1} WITH shares = 100") + cql.execute(f"CREATE SERVICE LEVEL {sl2} WITH shares = 200") + cql.execute(f"ATTACH SERVICE LEVEL {sl1} TO {user}") + + # Service level controller updates in 10 seconds interval, so wait + # for sl1 to be assgined to test_user + time.sleep(10) + try: + with new_session(cql, user): # new sessions is created only to create user's connection to Scylla + wait_until_all_connections_authenticated(cql) + wait_for_scheduling_group_assignment(cql, user, sl1) + + user_connections_sl1 = cql.execute(f"SELECT scheduling_group FROM system.clients WHERE username='{user}' ALLOW FILTERING") + for conn in user_connections_sl1: + assert conn[0] == f"sl:{sl1}" + + cql.execute(f"DETACH SERVICE LEVEL FROM {user}") + cql.execute(f"ATTACH SERVICE LEVEL {sl2} TO {user}") + # Again wait for service level controller to notice the change + time.sleep(10) + + switch_tenants(cql) + wait_for_scheduling_group_assignment(cql, user, sl2) + + user_connections_sl2 = cql.execute(f"SELECT scheduling_group FROM system.clients WHERE username='{user}' ALLOW FILTERING") + print(count_opened_connections(cql)) + for conn in user_connections_sl2: + assert conn[0] == f"sl:{sl2}" + finally: + cql.execute(f"DETACH SERVICE LEVEL FROM {user}") + cql.execute(f"DROP ROLE {user}") + cql.execute(f"DROP SERVICE LEVEL {sl1}") + cql.execute(f"DROP SERVICE LEVEL {sl2}") + + + + + + From 29b153c9e753e74f26277f0750f07350802181f9 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 21:01:49 +0100 Subject: [PATCH 156/397] cqlpy/test_service_levels: add workload prioritization tests Adjust existing cqlpy tests and add more in order to test the workload prioritization feature: - The DESCRIBE test is updated to check that generated statements contain information about shares - Two tests for shares in the LIST EFFECTIVE SERVICE LEVEL statement - Regression test which checks that we can create as many service levels as promised in the documentation (currently 7), but no more - Test which checks that NULL shares in the service levels table are treated as the default 1000 shares --- test/cqlpy/test_service_levels.py | 94 ++++++++++++++++++++++++++++--- 1 file changed, 86 insertions(+), 8 deletions(-) diff --git a/test/cqlpy/test_service_levels.py b/test/cqlpy/test_service_levels.py index f4307ec2c3a1..efb5ae560514 100644 --- a/test/cqlpy/test_service_levels.py +++ b/test/cqlpy/test_service_levels.py @@ -8,8 +8,9 @@ # to roles in order to apply various role-specific parameters, like timeouts. ############################################################################# -from contextlib import contextmanager +from contextlib import contextmanager, ExitStack from .util import unique_name, new_test_table, new_user +from .rest_api import scylla_inject_error from cassandra.protocol import InvalidRequest, ReadTimeout from cassandra.util import Duration @@ -18,14 +19,30 @@ import time @contextmanager -def new_service_level(cql, timeout=None, workload_type=None, role=None): +def new_service_level(cql, timeout=None, workload_type=None, shares=None, role=None): params = "" - if timeout and workload_type: - params = f"WITH timeout = {timeout} AND workload_type = '{workload_type}'" - elif timeout: - params = f"WITH timeout = {timeout}" - elif workload_type: - params = f"WITH workload_type = '{workload_type}'" + if timeout or workload_type or shares: + params = "WITH " + first = True + + if timeout: + if first: + first = False + else: + params += "AND " + params += f"timeout = {timeout} " + if workload_type: + if first: + first = False + else: + params += "AND " + params += f"workload_type = '{workload_type}' " + if shares: + if first: + first = False + else: + params += "AND " + params += f"shares = {shares} " attach_to = role if role else cql.cluster.auth_provider.username @@ -96,3 +113,64 @@ def test_list_effective_service_level(scylla_only, cql): if row.service_level_option == "workload_type": assert row.effective_service_level == sl2 assert row.value == "batch" + +def test_list_effective_service_level_shares(scylla_only, cql): + sl1 = "sl1" + sl2 = "sl2" + shares1 = 500 + shares2 = 200 + + with new_user(cql, "r1") as r1: + with new_user(cql, "r2") as r2: + with new_service_level(cql, shares=shares1, role=r1) as sl1: + with new_service_level(cql, shares=shares2, role=r2) as sl2: + cql.execute(f"GRANT {r2} TO {r1}") + + list_r1 = cql.execute(f"LIST EFFECTIVE SERVICE LEVEL OF {r1}") + for row in list_r1: + if row.service_level_option == "shares": + assert row.effective_service_level == sl2 + assert row.value == f"{shares2}" + list_r2 = cql.execute(f"LIST EFFECTIVE SERVICE LEVEL OF {r2}") + for row in list_r2: + if row.service_level_option == "shares": + assert row.effective_service_level == sl2 + assert row.value == f"{shares2}" + +def test_list_effective_service_level_without_attached(scylla_only, cql): + with new_user(cql) as role: + with pytest.raises(InvalidRequest, match=f"Role {role} doesn't have assigned any service level"): + cql.execute(f"LIST EFFECTIVE SERVICE LEVEL OF {role}") + +# Scylla Enterprise limits the number of service levels to a small number (8 including 1 default service level). +# This test verifies that attempting to create more service levels than that results in an InvalidRequest error +# and doesn't silently succeed. +# The test also has a regression check if a user can create exactly 7 service levels. +# In case you are adding a new internal scheduling group and this test failed, you should increase `SCHEDULING_GROUPS_COUNT` +# +# Reproduces enterprise issue #4481. +# Reproduces enterprise issue #5014. +def test_scheduling_groups_limit(scylla_only, cql): + sl_count = 100 + created_count = 0 + + with pytest.raises(InvalidRequest, match="Can't create service level - no more scheduling groups exist"): + with ExitStack() as stack: + for i in range(sl_count): + stack.enter_context(new_service_level(cql)) + created_count = created_count + 1 + + assert created_count > 0 + assert created_count == 7 # regression check + +def test_default_shares_in_listings(scylla_only, cql): + with scylla_inject_error(cql, "create_service_levels_without_default_shares", one_shot=False), \ + new_user(cql) as role: + with new_service_level(cql, role=role) as sl: + list_effective = cql.execute(f"LIST EFFECTIVE SERVICE LEVEL OF {role}") + shares_info = [row for row in list_effective if row.service_level_option == "shares"][0] + assert shares_info.value == "1000" + assert shares_info.effective_service_level == sl + + list_sl = cql.execute(f"LIST SERVICE LEVEL {sl}").one() + assert list_sl.shares == 1000 From 473bb44722534ca96d6cddd3020b6c85b64348c8 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 2 Dec 2024 21:41:12 +0100 Subject: [PATCH 157/397] test/auth_cluster: test workload prioritization in service level tests Update `test_connections_parameters_auto_update` to also check that the scheduling group of given connections is appropriately changed when a different service level is assigned to the user that the connection uses for authentication. Apart from that, more tests are added: - Check for the logic that forbids setting shares for a service level until all nodes in the cluster are upgraded - Test for handling the case when there are more scheduling groups than it is allowed (it might happen after upgrade from a non-workload-prio version) - Regression test for a bug where less scheduling groups could have been created than allowed due to some metrics not being renamed on scheduling group name change. --- test/auth_cluster/test_raft_service_levels.py | 148 +++++++++++++++++- 1 file changed, 144 insertions(+), 4 deletions(-) diff --git a/test/auth_cluster/test_raft_service_levels.py b/test/auth_cluster/test_raft_service_levels.py index 2e715f1bc0ac..b2ebb6bfed3c 100644 --- a/test/auth_cluster/test_raft_service_levels.py +++ b/test/auth_cluster/test_raft_service_levels.py @@ -11,7 +11,7 @@ from test.pylib.util import unique_name, wait_for_cql_and_get_hosts from test.pylib.manager_client import ManagerClient from test.topology.util import trigger_snapshot, wait_until_topology_upgrade_finishes, enter_recovery_state, reconnect_driver, \ - delete_raft_topology_state, delete_raft_data_and_upgrade_state, wait_until_upgrade_finishes + delete_raft_topology_state, delete_raft_data_and_upgrade_state, wait_until_upgrade_finishes, wait_for_token_ring_and_group0_consistency from test.topology.conftest import skip_mode from cassandra import ConsistencyLevel from cassandra.query import SimpleStatement @@ -189,9 +189,9 @@ def create_roles_stmts(): def create_service_levels_stmts(): return [ - "CREATE SERVICE LEVEL sl1 WITH timeout=30m AND workload_type='interactive'", - "CREATE SERVICE LEVEL sl2 WITH timeout=1h AND workload_type='batch'", - "CREATE SERVICE LEVEL sl3 WITH timeout=30s", + "CREATE SERVICE LEVEL sl1 WITH timeout=30m AND workload_type='interactive' AND shares=1000", + "CREATE SERVICE LEVEL sl2 WITH timeout=1h AND workload_type='batch' AND shares=500", + "CREATE SERVICE LEVEL sl3 WITH timeout=30s AND shares=800", ] def attach_service_levels_stms(): @@ -230,6 +230,7 @@ async def assert_connections_params(manager: ManagerClient, hosts, expect): continue assert param["workload_type"] == expect[role]["workload_type"] assert param["timeout"] == expect[role]["timeout"] + assert param["scheduling_group"] @pytest.mark.asyncio @skip_mode('release', 'cql server testing REST API is not supported in release mode') @@ -248,14 +249,17 @@ async def test_connections_parameters_auto_update(manager: ManagerClient, build_ "r1": { "workload_type": "unspecified", "timeout": default_timeout(build_mode), + "scheduling_group": "sl:default", }, "r2": { "workload_type": "unspecified", "timeout": default_timeout(build_mode), + "scheduling_group": "sl:default", }, "r3": { "workload_type": "unspecified", "timeout": default_timeout(build_mode), + "scheduling_group": "sl:default", }, }) @@ -271,14 +275,17 @@ async def test_connections_parameters_auto_update(manager: ManagerClient, build_ "r1": { "workload_type": "interactive", "timeout": "30m", + "scheduling_group": "sl:sl1", }, "r2": { "workload_type": "batch", "timeout": "1h", + "scheduling_group": "sl:sl2", }, "r3": { "workload_type": "unspecified", "timeout": "30s", + "scheduling_group": "sl:sl3", }, }) @@ -292,14 +299,17 @@ async def test_connections_parameters_auto_update(manager: ManagerClient, build_ "r1": { "workload_type": "batch", "timeout": "30s", + "scheduling_group": "sl:sl2", }, "r2": { "workload_type": "batch", "timeout": "30s", + "scheduling_group": "sl:sl2", }, "r3": { "workload_type": "unspecified", "timeout": "30s", + "scheduling_group": "sl:sl3", }, }) @@ -332,3 +342,133 @@ async def test_service_level_cache_after_restart(manager: ManagerClient): result = await cql.run_async("SELECT workload_type FROM system.service_levels_v2") assert len(result) == 1 and result[0].workload_type == 'batch' + +@pytest.mark.asyncio +@skip_mode('release', 'error injection is disabled in release mode') +async def test_shares_check(manager: ManagerClient): + srv = await manager.server_add(config={ + "error_injections_at_startup": [ + { "name": "suppress_features", "value": "WORKLOAD_PRIORITIZATION"} + ] + }) + await manager.server_start(srv.server_id) + + sl1 = f"sl_{unique_name()}" + sl2 = f"sl_{unique_name()}" + cql = manager.get_cql() + + await cql.run_async(f"CREATE SERVICE LEVEL {sl1}") + with pytest.raises(InvalidRequest, match="`shares` option can only be used when the cluster is fully upgraded to enterprise"): + await cql.run_async(f"CREATE SERVICE LEVEL {sl2} WITH shares=500") + with pytest.raises(InvalidRequest, match="`shares` option can only be used when the cluster is fully upgraded to enterprise"): + await cql.run_async(f"ALTER SERVICE LEVEL {sl1} WITH shares=100") + + await manager.server_stop_gracefully(srv.server_id) + await manager.server_update_config(srv.server_id, "error_injections_at_startup", []) + await manager.server_start(srv.server_id) + await wait_for_cql_and_get_hosts(manager.get_cql(), [srv], time.time() + 60) + + cql = manager.get_cql() + await cql.run_async(f"CREATE SERVICE LEVEL {sl2} WITH shares=500") + await cql.run_async(f"ALTER SERVICE LEVEL {sl1} WITH shares=100") + +@pytest.mark.asyncio +@skip_mode('release', 'error injection is not supported in release mode') +async def test_workload_prioritization_upgrade(manager: ManagerClient): + # This test simulates OSS->enterprise upgrade in v1 service levels. + # Using error injection, the test disables WORKLOAD_PRIORITIZATION feature + # and removes `shares` column from system_distributed.service_levels table. + config = { + 'authenticator': 'AllowAllAuthenticator', + 'authorizer': 'AllowAllAuthorizer', + 'force_gossip_topology_changes': True, + 'error_injections_at_startup': [ + { + 'name': 'suppress_features', + 'value': 'WORKLOAD_PRIORITIZATION' + }, + { + 'name': 'service_levels_v1_table_without_shares' + } + ] + } + servers = [await manager.server_add(config=config) for _ in range(3)] + cql = manager.get_cql() + hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60) + + # Validate that service levels' table has no `shares` column + sl_schema = await cql.run_async("DESC TABLE system_distributed.service_levels") + assert "shares int" not in sl_schema[0].create_statement + with pytest.raises(InvalidRequest): + await cql.run_async("CREATE SERVICE LEVEL sl1 WITH shares = 100") + + # Do rolling restart of the cluster and remove error injections + for server in servers: + await manager.server_update_config(server.server_id, 'error_injections_at_startup', []) + await manager.rolling_restart(servers) + + # Validate that `shares` column was added + logs = [await manager.server_open_log(server.server_id) for server in servers] + await logs[0].wait_for("Workload prioritization v1 started|Workload prioritization v1 is already started", timeout=10) + sl_schema_upgraded = await cql.run_async("DESC TABLE system_distributed.service_levels") + assert "shares int" in sl_schema_upgraded[0].create_statement + await cql.run_async("CREATE SERVICE LEVEL sl2 WITH shares = 100") + +@pytest.mark.asyncio +@skip_mode('release', 'error injection is disabled in release mode') +async def test_service_levels_over_limit(manager: ManagerClient): + srv = await manager.server_add(config={ + "error_injections_at_startup": ['allow_service_level_over_limit'] + }) + await manager.server_start(srv.server_id) + cql = manager.get_cql() + hosts = await wait_for_cql_and_get_hosts(cql, [srv], time.time() + 60) + + SL_LIMIT = 7 + sls = [] + for i in range(SL_LIMIT + 1): + sl = f"sl_{i}_{unique_name()}" + sls.append(sl) + await cql.run_async(f"CREATE SERVICE LEVEL {sl}") + + log = await manager.server_open_log(srv.server_id) + mark = await log.mark() + await cql.run_async(f"ATTACH SERVICE LEVEL {sls[-1]} TO CASSANDRA") + await log.wait_for(f"Service level {sls[-1]} is effectively dropped and its values are ignored.", timeout=10, from_mark=mark) + + mark = await log.mark() + # When service levels exceed the limit, last service levels in alphabetical order are effectively dropped + sl_name = f"aaa_sl_{unique_name()}" + await cql.run_async(f"CREATE SERVICE LEVEL {sl_name}") + await log.wait_for(f"service level \"{sls[-2]}\" will be effectively dropped to make scheduling group available to \"{sl_name}\", please consider removing a service level.", timeout=10, from_mark=mark) + +# Reproduces issue scylla-enterprise#4912 +@pytest.mark.asyncio +async def test_service_level_metric_name_change(manager: ManagerClient) -> None: + s = await manager.server_add() + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + cql = manager.get_cql() + + sl1 = unique_name() + sl2 = unique_name() + + # creates scheduling group `sl:sl1` + await cql.run_async(f"CREATE SERVICE LEVEL {sl1}") + # renames scheduling group `sl:sl1` to `sl_deleted:sl1` + await cql.run_async(f"DROP SERVICE LEVEL {sl1}") + # renames scheduling group `sl_deleted:sl1` to `sl:sl2` + await cql.run_async(f"CREATE SERVICE LEVEL {sl2}") + # creates scheduling group `sl:sl1` + await cql.run_async(f"CREATE SERVICE LEVEL {sl1}") + # In issue #4912, service_level_controller thought there was no room + # for `sl:sl1` scheduling group because create_scheduling_group() failed due to + # `seastar::metrics::double_registration (registering metrics twice for metrics: transport_cql_requests_count)` + # but the scheduling group was actually created. + # When sl2 is dropped, service_level_controller tries to rename its + # scheduling group to `sl:sl1`, triggering + # `seastar::metrics::double_registration (registering metrics twice for metrics: scheduler_runtime_ms)` + await cql.run_async(f"DROP SERVICE LEVEL {sl2}") + + # Check if group0 is healthy + s2 = await manager.server_add() + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) From 241e710c1907c5c738065fa33acfd05188daee67 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 10:54:38 +0100 Subject: [PATCH 158/397] docs/dev: describe workload prioritization features in service_levels The concept of shares, and some helper HTTP APIs, are now described in the developer documentation for service levels. --- docs/dev/service_levels.md | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/docs/dev/service_levels.md b/docs/dev/service_levels.md index cf1fcff63fc9..dbfe35d31d41 100644 --- a/docs/dev/service_levels.md +++ b/docs/dev/service_levels.md @@ -33,7 +33,8 @@ SELECT * FROM system.role_attributes WHERE role='r' and attribute_name='service CREATE TABLE system_distributed.service_levels ( service_level text PRIMARY KEY, timeout duration, - workload_type text) + workload_type text, + shares int); ``` The table is used to store and distribute the service levels configuration. @@ -41,6 +42,7 @@ The table column names meanings are: *service_level* - the name of the service level. *timeout* - timeout for operations performed by users under this service level *workload_type* - type of workload declared for this service level (NULL, interactive or batch) +*shares* - a number that represents this service level priority in relation to other service levels. ``` select * from system_distributed.service_levels ; @@ -136,6 +138,35 @@ the conflicts are resolved as follows: - `X` vs `NULL` -> `X` - `batch` vs `interactive` -> `batch` - under the assumption that `batch` is safer, because it would not trigger load shedding as eagerly as `interactive` + So for example to create a service level that is twice more important than the default service + level (which has shares of 1000) one can run: + + ``` + INSERT INTO system_distributed.service_level (service_level, shares) VALUES ('double_importance',2000); + ``` + +## Service levels REST API + +In a current state, Service Levels/Workload Prioritization has its own flaws, one of which is a requirement to restart connections to apply changes of users' service levels change. + +Until we improve service levels controller to make the changes automatically, here is a REST API to ease to work of maintaining and managing service levels and connections. + +A `tenant` (used below) is equal to scheduling group under which a connection is working. + +### Switch tenants + +`/service_levels/switch_tenants` endpoint triggers a tenant switch on all opened CQL connections on a single node without any interruption or their restart. +The response is returned immediately but the actual work might take up to tens of seconds. + +### Inspecting current scheduling group of connections + +`/service_levels/count_connections` endpoing is a tool to inspect status of all opened CQL connections. It returns a map with connections count per scheduling group, per user: +``` +{'sl:default': {'cassandra': 3}, 'sl:sl1': {'test_user': 3}} +``` + +In fact, this endpoint is a wrapper which executes simple query on `system.clients` table and aggregates the result. The table has added `scheduling_group` column, so to inspect a particular connection, it can be directly looked up in `system.clients` table. + ### Effective service level Actual values of service level's options may come from different service levels, not only from the one user is assigned with. This can be achieved by assigning one role to another. From 07b162fb5b484dc8883b58706d5d8398e3bec2d1 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 11:15:32 +0100 Subject: [PATCH 159/397] docs: add documentation for workload prioritization The doc pages were slightly adjusted during migration not to mention Scylla Enterprise and to fix some whitespace issues. --- docs/features/index.rst | 1 + docs/features/workload-attributes.rst | 6 +- docs/features/workload-prioritization.rst | 448 ++++++++++++++++++ .../security/rbac-usecase.rst | 4 +- 4 files changed, 454 insertions(+), 5 deletions(-) create mode 100644 docs/features/workload-prioritization.rst diff --git a/docs/features/index.rst b/docs/features/index.rst index 34f1e520bee8..2d561f380c6f 100644 --- a/docs/features/index.rst +++ b/docs/features/index.rst @@ -14,6 +14,7 @@ This document highlights ScyllaDB's key data modeling features. Counters Change Data Capture Workload Attributes + Workload Prioritization .. panel-box:: :title: ScyllaDB Features diff --git a/docs/features/workload-attributes.rst b/docs/features/workload-attributes.rst index b2eb24a495b1..9f1a904b1231 100644 --- a/docs/features/workload-attributes.rst +++ b/docs/features/workload-attributes.rst @@ -13,7 +13,7 @@ You can define a workload's attribute using the *service level* concept. The ser attributes to users and roles. When a user logs into the system, all of the attributes attached to that user and to the roles granted to that user are combined and become a set of workload attributes. -See `Service Level Management `_ for more information about service levels. +See :ref:`Service Level Management ` for more information about service levels. Prerequisites --------------- @@ -126,7 +126,7 @@ Available Workload Types * - ``unspecified`` - A generic workload without any specific characteristics (default). * - ``interactive`` - - A workload sensitive to latency, expected to have high/unbounded concurrency, with dynamic characteristics. For example, a workload assigned to users clicking on a website and generating events with their clicks. + - A workload sensitive to latency, expected to have high/unbounded concurrency, with dynamic characteristics, :doc:`OLTP `. For example, a workload assigned to users clicking on a website and generating events with their clicks. * - ``batch`` - - A workload for processing large amounts of data, not sensitive to latency, expected to have fixed concurrency. For example, a workload assigned to processing billions of historical sales records to generate statistics. + - A workload for processing large amounts of data, not sensitive to latency, expected to have fixed concurrency, :doc:`OLAP `. For example, a workload assigned to processing billions of historical sales records to generate statistics. diff --git a/docs/features/workload-prioritization.rst b/docs/features/workload-prioritization.rst new file mode 100644 index 000000000000..b2b17cf40b19 --- /dev/null +++ b/docs/features/workload-prioritization.rst @@ -0,0 +1,448 @@ +======================== +Workload Prioritization +======================== + +:label-tip:`ScyllaDB Enterprise` + +In a typical database there are numerous workloads running at the same time. +Each workload type dictates a different acceptable level of latency and throughput. +For example, consider the following two workloads: + +* OLTP ( Online Transaction Processing) - backend database for your application + + - High volume of requests + - Fast processing + - In essence - Latency sensitive + +* OLAP (Online Analytical Processing ) - performs data analytics in the background + + - High volume of data + - Slow queries + - In essence - Latency agnostic + +Using Service Level CQL commands, database administrators (working on Scylla Enterprise) can set different workload prioritization levels (levels of service) for each workload without sacrificing latency or throughput. +By assigning each service level to the different roles within your organization, DBAs ensure that each role_ receives the level of service the role requires. + +.. _`role` : /operating-scylla/security/rbac_usecase/ + +Prerequisites +============= +To create a level of service and assign it to a role, you need: + +* An :doc:`authenticated ` and :doc:`authorized ` user +* At least one :ref:`role created `. + +Work by Example +--------------- + +To follow the examples in this document, create the roles `spark` and `web`. You can assign permissions to these roles later, if needed. + +**Procedure** + +Run the following: + +.. code-block:: cql + + CREATE ROLE Spark; + CREATE ROLE Web; + +Workload Prioritization Workflow +================================ + +1. `Create a Service Level`_ +2. `Assign a Service Level to a Role`_ + +.. _workload-priorization-service-level-management: + +Service Level Management +======================== + +These commands set, list, and edit the level of service. + +Create a Service Level +---------------------- + +When you create a service level, you allocate a percentage of resources to the service level. Remember to use the correct naming convention to name your service level. If you decide to use :doc:`Reserved Keywords `, enclose them in either single or double quotes (for example ``'primary'``). + +**Syntax** + +.. code-block:: none + + CREATE SERVICE_LEVEL [IF NOT EXISTS] [WITH SHARES = ]; + +Where: + +* ``service_level_name`` - Specifies the name of the service you're creating. This can be any string without spaces. The name should be meaningful, such as class names (silver, gold, diamond, platinum), or categories (OLAP or OLTP), etc. +* ``shares_number`` - The number of shares of the resources you're granting to the service level name. You can use any number within the range from 1 to 1000. **Default : 1000** + +Example +....... + +There are 3 service levels (OLAP, OLTP, Default) where: (the percentage of resources = (Assigned Shares / Total Shares) x 100). Total Shares in this case is the total of all allocated shares + the Default SLA (1000). The percentage of resources would be: + +.. list-table:: + :widths: 30 30 30 + :header-rows: 1 + + * - Service Level Name + - Shares + - Percentage of Resources + * - OLAP + - 100 + - 4% + * - OLTP + - 1000 + - 48% + * - Default + - 1000 + - 48% + * - Total + - 2100 + - 100% + +**Procedure** + +1. To create these service levels, run the following CQL commands: + +.. code-block:: cql + + CREATE SERVICE_LEVEL IF NOT EXISTS OLAP WITH SHARES = 100; + CREATE SERVICE_LEVEL IF NOT EXISTS OLTP WITH SHARES = 1000; + +2. Confirm the service level change reflects the new service level allocations: + +.. code-block:: cql + + LIST ALL SERVICE_LEVELS; + + service_level | shares + --------------+------- + olap | 100 + --------------+------- + oltp | 1000 + (2 rows) + +Change Resource Allocation for a Service Level +----------------------------------------------- + +You can change resource allocation for a given service level. If you don't specify the number the shares, the default setting (1000) is used. + +**Syntax** + +.. code-block:: none + + ALTER SERVICE_LEVEL + WITH SHARES = ; + + +Where: + +* ``service_level_name`` - Specifies the name of the service level you created. See `Create a Service Level`_. +* ``shares_number`` - The number of shares in the CPU that you're granting to the service level name. You can use any number within the range from 1 to 1000. **Default : 1000** + + +.. warning:: + + Altering the SERVICE LEVEL does not affect active sessions (see `#12923 `_). + + To apply a new timeout to existing clients, execute a :doc:`rolling restart ` after the ALTER command. + + +Example +........ + +Analysts are complaining that they don't have enough resources. To increase the resources, you change the service level attributes for the OLAP service level. + +**Procedure** + +1. Run the following: + +.. code-block:: cql + + ALTER SERVICE_LEVEL OLAP WITH SHARES = 500; + +2. Confirm the service level change reflects the new service level allocation: + +.. code-block:: cql + + LIST SERVICE_LEVEL OLAP; + + service_level | shares + --------------+------- + olap | 500 + (1 rows) + +3. To change it back to the original setting (or to remain consistent for the examples that follow) change the shares amount back to the original. + +.. code-block:: cql + + ALTER SERVICE_LEVEL OLAP WITH SHARES = 100; + +Display Specified Service Level Parameters +------------------------------------------ + +Lists the specified service level with its class parameters. If the service level is attached to a role it does not appear in this list. + +**Syntax** + +.. code-block:: none + + LIST SERVICE_LEVEL ; + +Where: + +* ``service_level_name`` - Specifies the name of the service level you created. See `Create a Service Level`_. + +Example +....... + +In this example you list the service level parameters for OLTP. + +**Procedure** + +Run the following: + +.. code-block:: cql + + LIST SERVICE_LEVEL OLTP; + + service_level | shares + --------------+------- + oltp | 1000 + (1 rows) + +Display All Service Levels and Parameters +----------------------------------------- + +Lists all service levels with their class parameters. This list contains all service levels including those which are assigned to roles. + +**Syntax** + +.. code-block:: none + + LIST ALL SERVICE_LEVELS; + +Example +....... + +In this example, you list all service levels and their parameters. + +**Procedure** + +Run the following: + +.. code-block:: cql + + LIST ALL SERVICE_LEVELS; + + service_level | shares + ---------------+-------- + olap | 100 + oltp | 1000 + (2 rows) + + +Delete a Service Level +---------------------- + +Permanently removes the service level. Any role attached to this service level is automatically assigned to the Default SLA if there is no other service level attached to the role. + +**Syntax** + +.. code-block:: none + + DROP SERVICE_LEVEL IF EXISTS ; + +Where: + +* ``service_level_name`` - Specifies the name of the service level you created. See `Create a Service Level`_. +* ``IF EXISTS`` - If the service level does not exist and IF EXISTS is not used an error is returned. + + +Example +....... + +In this example you drop the OLTP service level. + +**Procedure** + +Run the following: + +.. code-block:: cql + + DROP SERVICE_LEVEL IF EXISTS OLTP; + +Manage Roles with Service Levels +================================ + +Once you have created roles and service levels you can attach and remove the service levels from the roles and list which roles are attached to which service levels. + +Assign a Service Level to a Role +-------------------------------- + +If you have created a role and a service level, you can attach the service level to the role. + +.. note:: A role can only be assigned **one** service level. However, the same service level can be attached to many roles. If a role inherits a service level from another role, the highest level of service from all the roles wins. + +**Syntax** + +.. code-block:: none + + ATTACH SERVICE_LEVEL TO ; + +Where: + +* ``service_level_name`` - Specifies the name of the service level you created. See `Create a Service Level`_. +* ``role_name`` - Specifies the role that you want to use the service level on. This is the role you created with :ref:`create role `. + +.. note:: Any role which does not have an SLA attached to it, receives the default SLA. + +Example +....... + +Continuing from the example in `Create a Service Level`_, you can attach the service levels that you created to different roles in your organization as follows: + +.. list-table:: + :widths: 50 50 + :header-rows: 1 + + * - Service Level Name + - Role Name + * - OLAP + - Spark + * - OLTP + - Web + + +**Procedure** + +To assign these service levels to the roles, run the following CQL commands: + +.. code-block:: cql + + ATTACH SERVICE_LEVEL OLAP TO Spark; + ATTACH SERVICE_LEVEL OLTP TO Web; + +List All Attached Service Levels for All Roles +---------------------------------------------- + +Lists all directly attached service levels for all roles. This does not include any service level which the role inherits from other roles. + +**Syntax** + +.. code-block:: none + + LIST ALL ATTACHED SERVICE_LEVELS; + +Example +....... + +In this example you list all service levels attached to any role. + +**Procedure** + +Run the following: + +.. code-block:: cql + + LIST ALL ATTACHED SERVICE_LEVELS; + + role | service_level + -------+--------------- + spark | olap + -------+--------------- + web | oltp + + (2 rows) + +List the Roles Assigned to a Specific Service Level +---------------------------------------------------- + +Lists all roles directly attached to a service level. This does not include any service level which the role inherits from other roles. + +**Syntax** + +.. code-block:: none + + LIST ATTACHED SERVICE_LEVEL OF ; + +Where: + +* ``role_name`` - Specifies the role that you want to use the service level on. This is the role you created with :ref:`create role `. + +Example +....... + +In this example, you list all of Roles which are assigned to the OLAP Service Level. + +**Procedure** + +Run the following: + +.. code-block:: cql + + LIST ATTACHED SERVICE_LEVEL OF Spark; + + role | service_level + -------+--------------- + spark | olap + + (1 rows) + +Remove a Service Level from a Role +---------------------------------- + +Removes a service level from a specified role. Once the service level is removed from a role, if there are other service levels attached to roles which that role inherits, the service level in the hierarchy with the most amount of shares wins. + +**Syntax** + +.. code-block:: none + + DETACH SERVICE_LEVEL FROM ; + +Where: + +* ``role_name`` - Specifies the role that you want to use the service level on. This is the role you created with :ref:`create role `. + +Example +....... + +In this example, you re-assign the Spark to a different level of service by detaching it from one level of service and attaching it to another. + +**Procedure** + +Run the following: + +.. code-block:: cql + + DETACH SERVICE_LEVEL FROM Spark; + +At this point, the Spark role receives the Default SLA, until it is assigned another service level. You assign a new service level to this role using `Assign a Service Level to a Role`_. + +Using Workload Prioritization with your Application +=================================================== + +In order for workload prioritization to take effect, application users need to be assigned to a relevant role. In addition, each role you create needs to be assigned to a specific Service Level. Any user that signs into the application without a role is automatically assigned the `Default` service level. This is always be the case with users who sign in anonymously. + + +Limits +====== +Scylla Enterprise is limited to 8 service levels, including the default one; this means you can create up to 7 service levels. + + +Additional References +===================== + +`OLAP or OLTP? Why Not Both? `_ Session by Glauber Costa from Scylla Summit 2018 + +`Scylla University: Workload Prioritization lesson `_ - The lesson covers: + +* The evolving requirements for operational (OLTP) and analytics (OLAP) workloads in the modern datacenter +* How Scylla provides built-in control over workload priority and makes it easy for administrators to configure workload priorities +* The impact of minimizing integrations and maintenance tasks, while also shrinking the datacenter footprint and maximizing utilization +* Test results of how it performs in real-world settings + + + + + diff --git a/docs/operating-scylla/security/rbac-usecase.rst b/docs/operating-scylla/security/rbac-usecase.rst index 79bfd78b9820..059da7f561d1 100644 --- a/docs/operating-scylla/security/rbac-usecase.rst +++ b/docs/operating-scylla/security/rbac-usecase.rst @@ -22,7 +22,7 @@ In the same manner, should someone leave the organization, all you would have to Should someone change positions at the company, just assign the new employee to the new role and revoke roles no longer required for the new position. To build an RBAC environment, you need to create the roles and their associated permissions and then assign or grant the roles to the individual users. Roles inherit the permissions of any other roles that they are granted. The hierarchy of roles can be either simple or extremely complex. This gives great flexibility to database administrators, where they can create specific permission conditions without incurring a huge administrative burden. -In addition to standard roles, `ScyllaDB Enterprise `_ users can implement `Workload Prioritization `_, which allows you to attach roles to Service Levels, thus granting resources to roles as the role demands. +In addition to standard roles, ScyllaDB Enterprise users can implement :doc:`Workload Prioritization `, which allows you to attach roles to Service Levels, thus granting resources to roles as the role demands. .. _rbac-usecase-grant-roles-and-permissions: @@ -213,4 +213,4 @@ Additional References * :doc:`Authorization` * :doc:`CQLSh the CQL shell` -* `Workload Prioritization `_ - to attach a service level to a role. Only available in `ScyllaDB Enterprise `_. +* :doc:`Workload Prioritization ` - to attach a service level to a role From 67b11e846aa0a130b0f1b1e2d15296289e3ff84d Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 17 Dec 2024 07:50:26 +0100 Subject: [PATCH 160/397] test.py: support multiple commands in prepare_cql in suite.yml This will be needed for alternator tests introduced in the next commit, which will have to execute multiple CQL operations during preparation. --- test.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/test.py b/test.py index e9a4868e71bc..4052150c082a 100755 --- a/test.py +++ b/test.py @@ -1135,8 +1135,13 @@ async def run(self, options: argparse.Namespace) -> Test: try: cluster.before_test(self.uname) prepare_cql = self.suite.cfg.get("prepare_cql", None) - if prepare_cql: - next(iter(cluster.running.values())).control_connection.execute(prepare_cql) + if prepare_cql and not hasattr(cluster, 'prepare_cql_executed'): + cc = next(iter(cluster.running.values())).control_connection + if not isinstance(prepare_cql, collections.abc.Iterable): + prepare_cql = [prepare_cql] + for stmt in prepare_cql: + cc.execute(stmt) + cluster.prepare_cql_executed = True logger.info("Leasing Scylla cluster %s for test %s", cluster, self.uname) self.args.insert(0, "--host={}".format(cluster.endpoint())) self.is_before_test_ok = True From b23bc3a5d518d6658c9b7795f34a308c353f5c9e Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Sat, 21 Dec 2024 18:27:37 +0100 Subject: [PATCH 161/397] alternator: execute under scheduling group for service level Now, the Alternator API requests are executed under the correct scheduling group of the service level assigned to the currently logged in user. --- alternator/server.cc | 13 +++- docs/alternator/compatibility.md | 31 +++++++++ test/alternator/conftest.py | 7 +- test/alternator/run | 13 ++++ test/alternator/suite.yaml | 5 ++ test/alternator/test_metrics.py | 6 +- test/alternator/test_service_levels.py | 95 ++++++++++++++++++++++++++ 7 files changed, 160 insertions(+), 10 deletions(-) create mode 100644 test/alternator/test_service_levels.py diff --git a/alternator/server.cc b/alternator/server.cc index 1f3fd4b7ea36..f20f52786c63 100644 --- a/alternator/server.cc +++ b/alternator/server.cc @@ -456,9 +456,16 @@ future server::handle_api_request(std::unique_ptr tracing::trace_state_ptr trace_state = maybe_trace_query(client_state, username, op, content); tracing::trace(trace_state, "{}", op); - rjson::value json_request = co_await _json_parser.parse(std::move(content)); - co_return co_await callback_it->second(_executor, client_state, trace_state, - make_service_permit(std::move(units)), std::move(json_request), std::move(req)); + + auto user = client_state.user(); + auto f = [this, content = std::move(content), &callback = callback_it->second, + client_state = std::move(client_state), trace_state = std::move(trace_state), + units = std::move(units), req = std::move(req)] () mutable -> future { + rjson::value json_request = co_await _json_parser.parse(std::move(content)); + co_return co_await callback(_executor, client_state, trace_state, + make_service_permit(std::move(units)), std::move(json_request), std::move(req)); + }; + co_return co_await _sl_controller.with_user_service_level(user, std::ref(f)); } void server::set_routes(routes& r) { diff --git a/docs/alternator/compatibility.md b/docs/alternator/compatibility.md index 956bdf07a7a6..0ea3f9520727 100644 --- a/docs/alternator/compatibility.md +++ b/docs/alternator/compatibility.md @@ -159,6 +159,37 @@ If you don't know the name of the table, you can try a forbidden operation and the AccessDeniedException error will contain the name of the table that was lacking permissions. +## Workload Isolation + +In DynamoDB read/write capacity of each table can be defined either to a fixed +value (provisioned mode) or to be adaptive (on-demand). On top of that requests +are also subject to per table and per account quotas. + +Due to the nature of Alternator deployment the whole cluster is available to serve +user requests and underlying hardware can be utilized to its full capacity. When +there is a need to allow more resources to given workload on the expense of some competing +one we offer feature called **Workload Prioritization**. + +To use this feature define service level with a fixed amount of shares +(higher value means proportionally more capacity) and attach it to a role +which then will be used to authorize requests. This can be currently done +only via CQL API, here is an example on how to do that: +```cql +CREATE ROLE alice WITH PASSWORD = 'abcd' AND LOGIN = true; +CREATE ROLE bob WITH PASSWORD = 'abcd' AND LOGIN = true; + +CREATE SERVICE_LEVEL IF NOT EXISTS olap WITH SHARES = 100; +CREATE SERVICE_LEVEL IF NOT EXISTS oltp WITH SHARES = 1000; + +ATTACH SERVICE_LEVEL olap TO alice; +ATTACH SERVICE_LEVEL oltp TO bob; +``` +Note that `alternator_enforce_authorization` has to be enabled in Scylla configuration. + +See [Authorization](##Authorization) section to learn more about roles and authorization. +See +to read about **Workload Prioritization** in detail. + ## Metrics Scylla has an advanced and extensive monitoring framework for inspecting diff --git a/test/alternator/conftest.py b/test/alternator/conftest.py index 7ea204015151..0bf69410ae93 100644 --- a/test/alternator/conftest.py +++ b/test/alternator/conftest.py @@ -68,7 +68,7 @@ def pytest_collection_modifyitems(config, items): # from the appropriate system table, but can't do it with Alternator (because # we don't know yet the secret key!), so we need to do it with CQL. @cache -def get_valid_alternator_role(url): +def get_valid_alternator_role(url, role='cassandra'): from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider auth_provider = PlainTextAuthProvider( @@ -85,7 +85,6 @@ def get_valid_alternator_role(url): # We could have looked for any role/salted_hash pair, but we # already know a role "cassandra" exists (we just used it to # connect to CQL!), so let's just use that role. - role = 'cassandra' salted_hash = list(session.execute(f"SELECT salted_hash FROM {ks}.roles WHERE role = '{role}'"))[0].salted_hash if salted_hash is None: break @@ -129,7 +128,7 @@ def dynamodb(request): region_name='us-east-1', aws_access_key_id=user, aws_secret_access_key=secret, config=boto_config.merge(botocore.client.Config(retries={"max_attempts": 0}, read_timeout=300))) -def new_dynamodb_session(request, dynamodb): +def new_dynamodb_session(request, dynamodb, user='cassandra', password='secret_pass'): ses = boto3.Session() host = urlparse(dynamodb.meta.client._endpoint.host) conf = botocore.client.Config(parameter_validation=False) @@ -137,7 +136,7 @@ def new_dynamodb_session(request, dynamodb): return boto3.resource('dynamodb', config=conf) if host.hostname == 'localhost': conf = conf.merge(botocore.client.Config(retries={"max_attempts": 0}, read_timeout=300)) - user, secret = get_valid_alternator_role(dynamodb.meta.client._endpoint.host) + user, secret = get_valid_alternator_role(dynamodb.meta.client._endpoint.host, role=user) return ses.resource('dynamodb', endpoint_url=dynamodb.meta.client._endpoint.host, verify=host.scheme != 'http', region_name='us-east-1', aws_access_key_id=user, aws_secret_access_key=secret, config=conf) diff --git a/test/alternator/run b/test/alternator/run index 64131dad6de7..797edf142c17 100755 --- a/test/alternator/run +++ b/test/alternator/run @@ -115,6 +115,19 @@ run.wait_for_services(pid, [ lambda: check_alternator(alternator_url), ]) +# Set up the the proper authentication credentials needed by the Alternator +# test. Currently this can only be done through CQL, which is why above we +# needed to make sure CQL is available. +cluster = run.get_cql_cluster(ip) +cql = cluster.connect() + +# Additional role and service level are created to test the feature properly (alternator doesn't have it's own API to set it up so we need to use CQL). +cql.execute("INSERT INTO system_auth_v2.roles (role, salted_hash) VALUES ('alternator_custom_sl', 'secret_pass')") +cql.execute("CREATE SERVICE LEVEL sl_alternator") +cql.execute("ATTACH SERVICE LEVEL sl_alternator TO alternator_custom_sl") + +cluster.shutdown() + # Finally run pytest: success = run.run_pytest(sys.path[0], ['--url', alternator_url] + sys.argv[1:]) diff --git a/test/alternator/suite.yaml b/test/alternator/suite.yaml index ce11ad8094ea..9264015b1e77 100644 --- a/test/alternator/suite.yaml +++ b/test/alternator/suite.yaml @@ -1,5 +1,10 @@ type: Python pool_size: 6 +prepare_cql: + - INSERT INTO system.roles (role, can_login, salted_hash) VALUES ('alternator_custom_sl', true, 'secret_pass') + - CREATE SERVICE LEVEL sl_alternator + - ATTACH SERVICE LEVEL sl_alternator TO alternator_custom_sl + run_first: - test_streams - test_scan diff --git a/test/alternator/test_metrics.py b/test/alternator/test_metrics.py index 9ba9008b0334..c2e18c9c1833 100644 --- a/test/alternator/test_metrics.py +++ b/test/alternator/test_metrics.py @@ -95,13 +95,13 @@ def get_metric(metrics, name, requested_labels=None, the_metrics=None): # of the specified metrics. Helps reduce the amount of code duplication # below. @contextmanager -def check_increases_metric(metrics, metric_names): +def check_increases_metric(metrics, metric_names, requested_labels=None): the_metrics = get_metrics(metrics) - saved_metrics = { x: get_metric(metrics, x, None, the_metrics) for x in metric_names } + saved_metrics = { x: get_metric(metrics, x, requested_labels, the_metrics) for x in metric_names } yield the_metrics = get_metrics(metrics) for n in metric_names: - assert saved_metrics[n] < get_metric(metrics, n, None, the_metrics), f'metric {n} did not increase' + assert saved_metrics[n] < get_metric(metrics, n, requested_labels, the_metrics), f'metric {n} did not increase' @contextmanager def check_increases_metric_exact(metrics, metric_name, increase_value): diff --git a/test/alternator/test_service_levels.py b/test/alternator/test_service_levels.py new file mode 100644 index 000000000000..9689d57b068e --- /dev/null +++ b/test/alternator/test_service_levels.py @@ -0,0 +1,95 @@ +# Copyright 2023-present ScyllaDB +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + +import pytest +from test.alternator.util import random_string, is_aws +from test.alternator.conftest import new_dynamodb_session +from test.alternator.test_metrics import metrics, get_metrics, check_increases_metric +from contextlib import contextmanager +from cassandra.auth import PlainTextAuthProvider +from cassandra.cluster import Cluster, ExecutionProfile, EXEC_PROFILE_DEFAULT, ConsistencyLevel +from cassandra.policies import RoundRobinPolicy +import time +import re + +# Quote an identifier if it needs to be double-quoted in CQL. Quoting is +# *not* needed if the identifier matches [a-z][a-z0-9_]*, otherwise it does. +# double-quotes ('"') in the string are doubled. +def maybe_quote(identifier): + if re.match('^[a-z][a-z0-9_]*$', identifier): + return identifier + return '"' + identifier.replace('"', '""') + '"' + +# Convenience context manager for temporarily GRANTing some permission and +# then revoking it. +@contextmanager +def temporary_grant(cql, permission, resource, role): + role = maybe_quote(role) + cql.execute(f"GRANT {permission} ON {resource} TO {role}") + try: + yield + finally: + cql.execute(f"REVOKE {permission} ON {resource} FROM {role}") + +# Convenience function for getting the full CQL table name (ksname.cfname) +# for the given Alternator table. This uses our insider knowledge that +# table named "x" is stored in keyspace called "alternator_x", and if we +# ever change this we'll need to change this function too. +def cql_table_name(tab): + return maybe_quote('alternator_' + tab.name) + '.' + maybe_quote(tab.name) + +# This file is all about testing RBAC as configured via CQL, so we need to +# connect to CQL to set these tests up. The "cql" fixture below enables that. +# If we're not testing Scylla, or the CQL port is not available on the same +# IP address as the Alternator IP address, a test using this fixture will +# be skipped with a message about the CQL API not being available. +@pytest.fixture(scope="module") +def cql(dynamodb): + if is_aws(dynamodb): + pytest.skip('Scylla-only CQL API not supported by AWS') + url = dynamodb.meta.client._endpoint.host + host, = re.search(r'.*://([^:]*):', url).groups() + profile = ExecutionProfile( + load_balancing_policy=RoundRobinPolicy(), + consistency_level=ConsistencyLevel.LOCAL_QUORUM, + serial_consistency_level=ConsistencyLevel.LOCAL_SERIAL) + cluster = Cluster(execution_profiles={EXEC_PROFILE_DEFAULT: profile}, + contact_points=[host], + port=9042, + protocol_version=4, + auth_provider=PlainTextAuthProvider(username='cassandra', password='cassandra'), + ) + try: + ret = cluster.connect() + # "BEGIN BATCH APPLY BATCH" is the closest to do-nothing I could find + ret.execute("BEGIN BATCH APPLY BATCH") + except NoHostAvailable: + pytest.skip('Could not connect to Scylla-only CQL API') + yield ret + cluster.shutdown() + +def test_service_level_metrics(test_table, request, dynamodb, cql, metrics): + print("Please make sure authorization is enforced in your Scylla installation: alternator_enforce_authorization: true") + p = random_string() + c = random_string() + _ = get_metrics(metrics) + # Use additional user created by test/alternator/run to execute write under sl_alternator service level. + ses = new_dynamodb_session(request, dynamodb, user='alternator_custom_sl') + # service_level_controler acts asynchronously in a loop so we can fail metric check + # if it hasn't processed service level update yet. It can take as long as 10 seconds. + started = time.time() + timeout = 30 + while True: + try: + with temporary_grant(cql, 'MODIFY', cql_table_name(test_table), 'alternator_custom_sl'): + with check_increases_metric(metrics, + ['scylla_storage_proxy_coordinator_write_latency_count'], + {'scheduling_group_name': 'sl:sl_alternator'}): + ses.meta.client.put_item(TableName=test_table.name, Item={'p': p, 'c': c}) + break # no exception, test passed + except: + if time.time() - started > timeout: + raise + else: + time.sleep(0.5) # retry From 07fdf9d21fffd247222a7a994d6a7e3351ad3e8b Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 13 Dec 2024 08:43:13 +0100 Subject: [PATCH 162/397] qos: un-shared-from-this standard_service_level_distributed_data_accessor Apparently, it is not needed for standard_service_level_distributed_data_accessor to derive from enable_shared_from_this. --- .../qos/standard_service_level_distributed_data_accessor.hh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/service/qos/standard_service_level_distributed_data_accessor.hh b/service/qos/standard_service_level_distributed_data_accessor.hh index a308ffd63b2e..c37faeabcf7e 100644 --- a/service/qos/standard_service_level_distributed_data_accessor.hh +++ b/service/qos/standard_service_level_distributed_data_accessor.hh @@ -19,8 +19,7 @@ namespace db { class system_distributed_keyspace; } namespace qos { -class standard_service_level_distributed_data_accessor : public service_level_controller::service_level_distributed_data_accessor, - public ::enable_shared_from_this { +class standard_service_level_distributed_data_accessor : public service_level_controller::service_level_distributed_data_accessor { private: db::system_distributed_keyspace& _sys_dist_ks; public: From 051c310f0290d14bed28a1c162dca5cdfcc44240 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Mon, 16 Jan 2023 19:05:22 +0200 Subject: [PATCH 163/397] tracing: record scheduling group in trace event record We have a "thread" field (unfortunately not yet displayed in cqlsh, but visible in the table) that records the shard on which a particular event was recorded. Record the scheduling group as well, as this can be useful to understand where the query came from. (cherry picked from commit 3c03b5f66376dca230868e54148ad1c6a1ad0ee2) --- tracing/trace_keyspace_helper.cc | 2 +- tracing/tracing.hh | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tracing/trace_keyspace_helper.cc b/tracing/trace_keyspace_helper.cc index 286e5459b5f9..c6067e253f98 100644 --- a/tracing/trace_keyspace_helper.cc +++ b/tracing/trace_keyspace_helper.cc @@ -380,7 +380,7 @@ std::vector trace_keyspace_helper::make_event_mutation_data(gms cql3::raw_value::make_value(utf8_type->decompose(record.message)), cql3::raw_value::make_value(inet_addr_type->decompose(my_address.addr())), cql3::raw_value::make_value(int32_type->decompose(elapsed_to_micros(record.elapsed))), - cql3::raw_value::make_value(utf8_type->decompose(_local_tracing.get_thread_name())), + cql3::raw_value::make_value(utf8_type->decompose(fmt::format("{}/{}", _local_tracing.get_thread_name(), record.scheduling_group_name))), cql3::raw_value::make_value(long_type->decompose(int64_t(session_records.parent_id.get_id()))), cql3::raw_value::make_value(long_type->decompose(int64_t(session_records.my_span_id.get_id()))), cql3::raw_value::make_value(int32_type->decompose((int32_t)(session_records.ttl.count()))) diff --git a/tracing/tracing.hh b/tracing/tracing.hh index 66267bc05130..1209c384bf97 100644 --- a/tracing/tracing.hh +++ b/tracing/tracing.hh @@ -175,6 +175,7 @@ struct event_record { std::string message; elapsed_clock::duration elapsed; i_tracing_backend_helper::wall_clock::time_point event_time_point; + sstring scheduling_group_name = current_scheduling_group().name(); event_record(sstring message_, elapsed_clock::duration elapsed_, i_tracing_backend_helper::wall_clock::time_point event_time_point_) : message(std::move(message_)) From 1ce373d80b17a5b5543cb988a7be1ced4195d9e0 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Sat, 14 Dec 2024 22:33:23 +0200 Subject: [PATCH 164/397] schema: deinline some speculative_retry methods This string conversion functions are not in any fast path. Deinlining them moves a include out of a common header file. Some files accessed on boost::iterator_range via lexical_cast.hpp, so they gain a new dependency. Closes scylladb/scylladb#21950 --- clustering_ranges_walker.hh | 2 ++ mutation_query.cc | 2 ++ partition_range_compat.hh | 1 + schema/schema.cc | 48 +++++++++++++++++++++++++++++++++++++ schema/schema.hh | 47 ++---------------------------------- 5 files changed, 55 insertions(+), 45 deletions(-) diff --git a/clustering_ranges_walker.hh b/clustering_ranges_walker.hh index 24d90bc26bff..4555b2e0eb6b 100644 --- a/clustering_ranges_walker.hh +++ b/clustering_ranges_walker.hh @@ -16,6 +16,8 @@ #include "mutation/mutation_fragment.hh" #include "mutation/mutation_fragment_v2.hh" +#include + // Utility for in-order checking of overlap with position ranges. class clustering_ranges_walker { const schema& _schema; diff --git a/mutation_query.cc b/mutation_query.cc index d831d3f19f82..7ae06eafb370 100644 --- a/mutation_query.cc +++ b/mutation_query.cc @@ -11,6 +11,8 @@ #include "mutation_query.hh" #include "schema/schema_registry.hh" +#include + reconcilable_result::~reconcilable_result() {} reconcilable_result::reconcilable_result() diff --git a/partition_range_compat.hh b/partition_range_compat.hh index a2351b1c3e2d..1f6ec0cb9cf8 100644 --- a/partition_range_compat.hh +++ b/partition_range_compat.hh @@ -12,6 +12,7 @@ #include #include "interval.hh" #include "dht/ring_position.hh" +#include namespace compat { diff --git a/schema/schema.cc b/schema/schema.cc index 204c5654b701..fe5ad80c4b0f 100644 --- a/schema/schema.cc +++ b/schema/schema.cc @@ -40,6 +40,54 @@ constexpr int32_t schema::NAME_LENGTH; extern logging::logger dblog; +sstring +speculative_retry::to_sstring() const { + if (_t == type::NONE) { + return "NONE"; + } else if (_t == type::ALWAYS) { + return "ALWAYS"; + } else if (_t == type::CUSTOM) { + return format("{:.2f}ms", _v); + } else if (_t == type::PERCENTILE) { + return format("{:.1f}PERCENTILE", 100 * _v); + } else { + throw std::invalid_argument(format("unknown type: {:d}\n", uint8_t(_t))); + } +} + +speculative_retry +speculative_retry::from_sstring(sstring str) { + std::transform(str.begin(), str.end(), str.begin(), ::toupper); + + sstring ms("MS"); + sstring percentile("PERCENTILE"); + + auto convert = [&str] (sstring& t) { + try { + return boost::lexical_cast(str.substr(0, str.size() - t.size())); + } catch (boost::bad_lexical_cast& e) { + throw std::invalid_argument(format("cannot convert {} to speculative_retry\n", str)); + } + }; + + type t; + double v = 0; + if (str == "NONE") { + t = type::NONE; + } else if (str == "ALWAYS") { + t = type::ALWAYS; + } else if (str.compare(str.size() - ms.size(), ms.size(), ms) == 0) { + t = type::CUSTOM; + v = convert(ms); + } else if (str.compare(str.size() - percentile.size(), percentile.size(), percentile) == 0) { + t = type::PERCENTILE; + v = convert(percentile) / 100; + } else { + throw std::invalid_argument(format("cannot convert {} to speculative_retry\n", str)); + } + return speculative_retry(t, v); +} + sstring to_sstring(column_kind k) { switch (k) { case column_kind::partition_key: return "PARTITION_KEY"; diff --git a/schema/schema.hh b/schema/schema.hh index 469140b08a81..05da1d6616c2 100644 --- a/schema/schema.hh +++ b/schema/schema.hh @@ -14,7 +14,6 @@ #include #include #include -#include #include #include "cql3/column_specification.hh" @@ -163,50 +162,8 @@ private: public: speculative_retry(type t, double v) : _t(t), _v(v) {} - sstring to_sstring() const { - if (_t == type::NONE) { - return "NONE"; - } else if (_t == type::ALWAYS) { - return "ALWAYS"; - } else if (_t == type::CUSTOM) { - return format("{:.2f}ms", _v); - } else if (_t == type::PERCENTILE) { - return format("{:.1f}PERCENTILE", 100 * _v); - } else { - throw std::invalid_argument(format("unknown type: {:d}\n", uint8_t(_t))); - } - } - static speculative_retry from_sstring(sstring str) { - std::transform(str.begin(), str.end(), str.begin(), ::toupper); - - sstring ms("MS"); - sstring percentile("PERCENTILE"); - - auto convert = [&str] (sstring& t) { - try { - return boost::lexical_cast(str.substr(0, str.size() - t.size())); - } catch (boost::bad_lexical_cast& e) { - throw std::invalid_argument(format("cannot convert {} to speculative_retry\n", str)); - } - }; - - type t; - double v = 0; - if (str == "NONE") { - t = type::NONE; - } else if (str == "ALWAYS") { - t = type::ALWAYS; - } else if (str.compare(str.size() - ms.size(), ms.size(), ms) == 0) { - t = type::CUSTOM; - v = convert(ms); - } else if (str.compare(str.size() - percentile.size(), percentile.size(), percentile) == 0) { - t = type::PERCENTILE; - v = convert(percentile) / 100; - } else { - throw std::invalid_argument(format("cannot convert {} to speculative_retry\n", str)); - } - return speculative_retry(t, v); - } + sstring to_sstring() const; + static speculative_retry from_sstring(sstring str); type get_type() const { return _t; } From 233e3969c458f82c8b9f597565149452143ba928 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Thu, 2 Jan 2025 14:35:23 +0800 Subject: [PATCH 165/397] utils: correct misspellings these misspellings were identified by codespell. let's fix them. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22143 --- configure.py | 2 +- docs/dev/advanced_rpc_compression.md | 4 ++-- utils/dict_trainer.hh | 2 +- utils/stream_compressor.cc | 4 ++-- utils/stream_compressor.hh | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/configure.py b/configure.py index d93835ff29dd..be09e24aa7fd 100755 --- a/configure.py +++ b/configure.py @@ -1737,7 +1737,7 @@ def prepare_advanced_optimizations(*, modes, build_modes, args): # Absolute path (in case of the initial profile) or path # beginning with $builddir (in case of generated profiles), # for use in ninja dependency rules. - # Using absoulte paths only would work too, but we use + # Using absolute paths only would work too, but we use # $builddir for consistency with all other ninja targets. profile_target = None # Absolute path to the profile, for use in compiler flags. diff --git a/docs/dev/advanced_rpc_compression.md b/docs/dev/advanced_rpc_compression.md index c77f2e046e7e..927faa007bb3 100644 --- a/docs/dev/advanced_rpc_compression.md +++ b/docs/dev/advanced_rpc_compression.md @@ -248,14 +248,14 @@ one copy of each dictionary, on shard 0, kept alive by foreign shared pointers. Likewise, we keep only a single compressor and a single decompressor for each algorithm, and whenever a connection needs to use it, it plugs the correct dictionary in. -Switching dictionaries should be cheaper than keeping mulitple copies of the compressors. +Switching dictionaries should be cheaper than keeping multiple copies of the compressors. ### Wire protocol details This section describes the layout of a compressed frame produced by `advanced_rpc_compressor::compress()`. The compression algorithm is selected on per-message basis. -(Rationale: this allows the sender to weaken the compression unilaterally if it doens't have the resources for the "normal" algorithm.) +(Rationale: this allows the sender to weaken the compression unilaterally if it doesn't have the resources for the "normal" algorithm.) The 7 least significant bits of byte 0 of each compressed message contain an enum value describing the compression algorithm used for this message. diff --git a/utils/dict_trainer.hh b/utils/dict_trainer.hh index 6e6e15adec71..d2175b9ea01f 100644 --- a/utils/dict_trainer.hh +++ b/utils/dict_trainer.hh @@ -60,7 +60,7 @@ public: // // If the abort source is triggered before the min_sampling_bytes threshold is met, // the sampling will be canceled and the returned future will resolve to the aborting exception. - // In reasonable use cases, min_sampling_duration should be abortable withe the same abort source. + // In reasonable use cases, min_sampling_duration should be abortable with the same abort source. seastar::future> sample(request, seastar::abort_source&); // When in the sampling phase, this will feed the data to the sampler. diff --git a/utils/stream_compressor.cc b/utils/stream_compressor.cc index 3f7c371cef3b..5c19a9edd4a5 100644 --- a/utils/stream_compressor.cc +++ b/utils/stream_compressor.cc @@ -167,7 +167,7 @@ void lz4_cstream::resetFast() noexcept { } // When new data arrives in `in`, we copy an arbitrary amount of it to `_buf`, -// (the amount is arbirary, but it has to fit contiguously in `_buf`), +// (the amount is arbitrary, but it has to fit contiguously in `_buf`), // compress the new block from `_buf` to `_lz4_scratch`, // then we copy everything from `_lz4_scratch` to `out`. // Repeat until `in` is empty. @@ -182,7 +182,7 @@ size_t lz4_cstream::compress(ZSTD_outBuffer* out, ZSTD_inBuffer* in, ZSTD_EndDir _buf_pos = 0; } // We will compress the biggest prefix of `in` that fits contiguously inside `buf`. - // In principle, this is sligthly suboptimal -- ideally, if `in` is smaller than the contiguous space in `buf`, + // In principle, this is slightly suboptimal -- ideally, if `in` is smaller than the contiguous space in `buf`, // we should only copy `in` to `buf` and wait with the compressor call until future `in`s // fill the contiguous space entirely, or `end` is `ZSTD_e_flush` or `ZSTD_e_end`. // But for streaming LZ4 it doesn't really make a difference. diff --git a/utils/stream_compressor.hh b/utils/stream_compressor.hh index f309b7b53592..4516a6e59b5f 100644 --- a/utils/stream_compressor.hh +++ b/utils/stream_compressor.hh @@ -132,7 +132,7 @@ class lz4_cstream final : public stream_compressor { // has a matching decompression call with decompressor's _buf as target, // with the same length and offset in _buf. std::vector _buf; - // The current position in the ringbuffer _buf. New input will be appened at this position. + // The current position in the ringbuffer _buf. New input will be appended at this position. size_t _buf_pos = 0; // This pair describes the compressed data in `_lz4_scratch`, which is pending output. // We have to copy it out before we can compress new data to the scratch buffer. @@ -147,7 +147,7 @@ public: void reset() noexcept override; void resetFast() noexcept; // When new data arrives in `in`, we copy an arbitrary amount of it to `_buf`, - // (the amount is arbirary, but it has to fit contiguously in `_buf`), + // (the amount is arbitrary, but it has to fit contiguously in `_buf`), // compress the new block from `_buf` to `_lz4_scratch`, // then we copy everything from `_lz4_scratch` to `out`. // Repeat until `in` is empty. From 174199610bf5538469a43fb4d6531a5665509a89 Mon Sep 17 00:00:00 2001 From: Artsiom Mishuta Date: Sun, 29 Dec 2024 13:52:53 +0100 Subject: [PATCH 166/397] test.py: add more log info if the server is broken attribute server_broken_reason into the server was introduced, to store the raw information regarding why the server was broken additional information was added in the error messages in case of "server broken" fixes: #21630 Closes scylladb/scylladb#22074 --- test/pylib/scylla_cluster.py | 9 +++++---- test/topology/conftest.py | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 457d228690ae..de755ea9f824 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -1285,6 +1285,7 @@ def __init__(self, test_uname: str, clusters: Pool[ScyllaCluster], base_dir: str self.runner = aiohttp.web.AppRunner(app) self.tasks_history = dict() self.server_broken_event = asyncio.Event() + self.server_broken_reason = "" def repr_tasks_history(self): out = "Cluster_history" @@ -1370,7 +1371,7 @@ def outer_wrapper(handler: Callable)-> Callable: @wraps(handler) async def inner_wrapper(request): if blockable and self.server_broken_event.is_set(): - raise Exception("ScyllaClusterManager BROKEN, Previous test broke ScyllaClusterManager server") + raise Exception(f"ScyllaClusterManager BROKEN, Previous test broke ScyllaClusterManager server, server_broken_reason: {self.server_broken_reason}") self.logger.info("[ScyllaClusterManager][%s] %s", asyncio.current_task().get_name(), request.url) self.tasks_history[asyncio.current_task()] = request return await handler(request) @@ -1490,11 +1491,12 @@ async def _after_test(self, _request) -> dict[str, bool]: self.is_after_test_ok = True cluster_str = str(self.cluster) - return {"cluster_str":cluster_str, "server_broken":self.server_broken_event.is_set()} + return {"cluster_str":cluster_str, "server_broken":self.server_broken_event.is_set(), "message": self.server_broken_reason } def break_manager(self, reason, test): # make ScyllaClusterManager not operatable from client side - self.logger.error(" %s, test case {test} BROKE ScyllaClusterManager", reason) + self.server_broken_reason = f"{reason}, test case {test} BROKE ScyllaClusterManager" + self.logger.error(self.server_broken_reason) self.server_broken_event.set() async def _mark_dirty(self, _request) -> None: @@ -1541,7 +1543,6 @@ async def _cluster_server_unpause(self, request) -> None: async def _cluster_server_add(self, request) -> dict[str, object]: """Add a new server.""" - assert self.cluster data = await request.json() diff --git a/test/topology/conftest.py b/test/topology/conftest.py index 5177a91b3e61..b81a2e1fd0f8 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -217,7 +217,7 @@ async def manager(request, manager_internal, record_property, build_mode): cluster_status = await manager_client.after_test(test_case_name, not failed) await manager_client.stop() # Stop client session and close driver after each test if cluster_status["server_broken"]: - pytest.fail(f"test case {test_case_name} leave unfinished tasks on Scylla server. Server marked as broken") + pytest.fail(f"test case {test_case_name} leave unfinished tasks on Scylla server. Server marked as broken, server_broken_reason: {cluster_status["message"]}") # "cql" fixture: set up client object for communicating with the CQL API. # Since connection is managed by manager just return that object From de42dce4c4d9f66f0419a77abc3a13f3c961b588 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Thu, 2 Jan 2025 14:05:18 +0800 Subject: [PATCH 167/397] pgo: use java-11 when running cassandra-stress we updated tools/java/build.xml recently to only build for java-11. so if - the `java` executable in `$PATH` points to a java which is neither java-8 nor java-11. - java-8 is installed java-8 is used to execute the cassandra-stress tool. and we would have following failure: ``` Error: A JNI error has occurred, please check your installation and try again Exception in thread "main" java.lang.UnsupportedClassVersionError: org/apache/cassandra/stress/Stress has been compiled by a more recent version of the Java Runtime (class file version 55.0), this version of the Java Runtime only recogniz es class file versions up to 52.0 at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:756) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:473) at java.net.URLClassLoader.access$100(URLClassLoader.java:74) at java.net.URLClassLoader$1.run(URLClassLoader.java:369) at java.net.URLClassLoader$1.run(URLClassLoader.java:363) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:362) at java.lang.ClassLoader.loadClass(ClassLoader.java:418) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352) at java.lang.ClassLoader.loadClass(ClassLoader.java:351) at sun.launcher.LauncherHelper.checkAndLoadMain(LauncherHelper.java:621) ``` in order to be compatible with the bytecode targeting java-11, let's run cassandra-stress with java-11. we do not need to support java-8, because the new tools/java is now building cassandra-stress targeting java-11 jre. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22142 --- pgo/pgo.py | 35 ++++++++++++++++------------------- 1 file changed, 16 insertions(+), 19 deletions(-) diff --git a/pgo/pgo.py b/pgo/pgo.py index 51425bfc0aa1..bd4f0cba13f2 100644 --- a/pgo/pgo.py +++ b/pgo/pgo.py @@ -106,30 +106,27 @@ def configure_cpusets(): async def configure_java() -> None: """ - cassandra-stress can only deal with Java 8 or Java 11 + cassandra-stress can only deal with Java 11 """ version_output = (await bash("java -version", stderr=asyncio.subprocess.PIPE))[2] - assert type(version_output) == bytes + assert isinstance(version_output, bytes) version_first_line = version_output.decode().split(sep='\n')[0] config_logger.info(f"First line of java -version: {version_first_line}") - if re.search(r'version.*1\.8\.0', version_first_line): - config_logger.info(f"Default Java version recognized as Java 8. Proceeding with the default.") + version = 11 + if re.search(rf'version.*{version}\.[0-9]+\.[0-9]+', version_first_line): + config_logger.info(f"Default Java version recognized as Java {version}. Proceeding with the default.") JAVA_HOME.set(None) - elif re.search(r'version.*11\.[0-9]+\.[0-9]+', version_first_line): - config_logger.info(f"Default Java version recognized as Java 11. Proceeding with the default.") - JAVA_HOME.set(None) - else: - config_logger.info(f"Default Java version recognized as neither Java 8 nor Java 11.") - if os.path.exists(java_8_path := '/usr/lib/jvm/java-1.8.0'): - config_logger.warning(f"{java_8_path} found. Choosing it as JAVA_HOME.") - JAVA_HOME.set(java_8_path) - elif os.path.exists(java_11_path := '/usr/lib/jvm/java-11'): - config_logger.warning(f"{java_11_path} found. Choosing it as JAVA_HOME.") - JAVA_HOME.set(java_11_path) - else: - error = "Failed to find a suitable Java version. Java 8 or Java 11 is required." - config_logger.error(error) - raise RuntimeError(error) + return + + config_logger.info(f"Default Java version is not recognized as Java {version}.") + if os.path.exists(java_path := f'/usr/lib/jvm/java-{version}'): + config_logger.warning(f"{java_path} found. Choosing it as JAVA_HOME.") + JAVA_HOME.set(java_path) + return + + error = f"Failed to find a suitable Java version. Java {version} is required." + config_logger.error(error) + raise RuntimeError(error) ################################################################################ # Child process utilities From 7d42b80228b76447101018f617727c8585c36c71 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Thu, 19 Dec 2024 15:37:03 +0200 Subject: [PATCH 168/397] service/storage_proxy: data_read_resolver::resolve(): remove unneded maybe_yield() We already have a yield in the loop via apply_gently(), the maybe_yield is superfluous so remove it. Follow-up to https://github.com/scylladb/scylladb/pull/21884 Closes scylladb/scylladb#21984 --- service/storage_proxy.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index cac072226b4d..6265191f121f 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -5035,7 +5035,6 @@ class data_read_resolver : public abstract_read_resolver { if (ver.par) { mutation_application_stats app_stats; co_await apply_gently(m.partition(), *schema, ver.par->mut().partition(), *schema, app_stats); - co_await coroutine::maybe_yield(); } } auto live_row_count = m.live_row_count(); From 491b7232de821b3bbe837e0f0ca050638e66b7a7 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 2 Jan 2025 18:40:45 +0200 Subject: [PATCH 169/397] locator: drop inet_address usage to figure out per dc/rack replication It allows to correctly calculate replication map even without knowing IPs of the nodes. --- locator/network_topology_strategy.cc | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/locator/network_topology_strategy.cc b/locator/network_topology_strategy.cc index 80876603e378..e59581a32ee2 100644 --- a/locator/network_topology_strategy.cc +++ b/locator/network_topology_strategy.cc @@ -176,13 +176,13 @@ class natural_endpoints_tracker { // all token owners in each DC, so we can check when we have exhausted all // the token-owning members of a DC // - std::unordered_map> _token_owners; + std::unordered_map> _token_owners; // // all racks (with non-token owners filtered out) in a DC so we can check // when we have exhausted all racks in a DC // - std::unordered_map>> _racks; + std::unordered_map>> _racks; std::unordered_map _dcs; @@ -193,8 +193,8 @@ class natural_endpoints_tracker { : _tm(tm) , _tp(_tm.get_topology()) , _dc_rep_factor(dc_rep_factor) - , _token_owners(_tm.get_datacenter_token_owners_ips()) - , _racks(_tm.get_datacenter_racks_token_owners_ips()) + , _token_owners(_tm.get_datacenter_token_owners()) + , _racks(_tm.get_datacenter_racks_token_owners()) { // not aware of any cluster members SCYLLA_ASSERT(!_token_owners.empty() && !_racks.empty()); From acbc667d3e1ceebeaf6548bcfe82da855b693155 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Mon, 30 Dec 2024 13:50:45 +0200 Subject: [PATCH 170/397] storage_service: set raft topology change mode before using it in join_cluster ss::join_cluster calls raft_topology_change_enabled() before the mode is initialized below in the same function. Fix it by changing the order. --- service/storage_service.cc | 77 +++++++++++++++++++------------------- 1 file changed, 39 insertions(+), 38 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index f4fcc6d91e73..7468d91b0487 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2938,38 +2938,7 @@ future<> storage_service::join_cluster(sharded& set_mode(mode::STARTING); - std::unordered_map loaded_endpoints; - if (_db.local().get_config().load_ring_state() && !raft_topology_change_enabled()) { - slogger.info("Loading persisted ring state"); - loaded_endpoints = co_await _sys_ks.local().load_endpoint_state(); - - auto tmlock = co_await get_token_metadata_lock(); - auto tmptr = co_await get_mutable_token_metadata_ptr(); - for (auto& [host_id, st] : loaded_endpoints) { - if (st.endpoint == get_broadcast_address()) { - // entry has been mistakenly added, delete it - slogger.warn("Loaded saved endpoint={}/{} has my broadcast address. Deleting it", host_id, st.endpoint); - co_await _sys_ks.local().remove_endpoint(st.endpoint); - } else { - if (host_id == my_host_id()) { - on_internal_error(slogger, format("Loaded saved endpoint {} with my host_id={}", st.endpoint, host_id)); - } - if (!st.opt_dc_rack) { - st.opt_dc_rack = locator::endpoint_dc_rack::default_location; - slogger.warn("Loaded no dc/rack for saved endpoint={}/{}. Set to default={}/{}", host_id, st.endpoint, st.opt_dc_rack->dc, st.opt_dc_rack->rack); - } - const auto& dc_rack = *st.opt_dc_rack; - slogger.debug("Loaded tokens: endpoint={}/{} dc={} rack={} tokens={}", host_id, st.endpoint, dc_rack.dc, dc_rack.rack, st.tokens); - tmptr->update_topology(host_id, dc_rack, locator::node::state::normal); - co_await tmptr->update_normal_tokens(st.tokens, host_id); - tmptr->update_host_id(host_id, st.endpoint); - // gossiping hasn't started yet - // so no need to lock the endpoint - co_await _gossiper.add_saved_endpoint(host_id, st, gms::null_permit_id); - } - } - co_await replicate_to_all_cores(std::move(tmptr)); - } + std::unordered_map loaded_endpoints = co_await _sys_ks.local().load_endpoint_state(); // Seeds are now only used as the initial contact point nodes. If the // loaded_endpoints are empty which means this node is a completely new @@ -2981,12 +2950,6 @@ future<> storage_service::join_cluster(sharded& loaded_endpoints | std::views::transform([] (const auto& x) { return x.second.endpoint; }) | std::ranges::to>(); - auto loaded_peer_features = co_await _sys_ks.local().load_peer_features(); - slogger.info("initial_contact_nodes={}, loaded_endpoints={}, loaded_peer_features={}", - initial_contact_nodes, loaded_endpoints | std::views::keys, loaded_peer_features.size()); - for (auto& x : loaded_peer_features) { - slogger.info("peer={}, supported_features={}", x.first, x.second); - } if (_group0->client().in_recovery()) { slogger.info("Raft recovery - starting in legacy topology operations mode"); @@ -3044,6 +3007,44 @@ future<> storage_service::join_cluster(sharded& // We must allow restarts of zero-token nodes in the gossip-based topology due to the recovery mode. } + if (_db.local().get_config().load_ring_state() && !raft_topology_change_enabled()) { + slogger.info("Loading persisted ring state"); + + auto tmlock = co_await get_token_metadata_lock(); + auto tmptr = co_await get_mutable_token_metadata_ptr(); + for (auto& [host_id, st] : loaded_endpoints) { + if (st.endpoint == get_broadcast_address()) { + // entry has been mistakenly added, delete it + slogger.warn("Loaded saved endpoint={}/{} has my broadcast address. Deleting it", host_id, st.endpoint); + co_await _sys_ks.local().remove_endpoint(st.endpoint); + } else { + if (host_id == my_host_id()) { + on_internal_error(slogger, format("Loaded saved endpoint {} with my host_id={}", st.endpoint, host_id)); + } + if (!st.opt_dc_rack) { + st.opt_dc_rack = locator::endpoint_dc_rack::default_location; + slogger.warn("Loaded no dc/rack for saved endpoint={}/{}. Set to default={}/{}", host_id, st.endpoint, st.opt_dc_rack->dc, st.opt_dc_rack->rack); + } + const auto& dc_rack = *st.opt_dc_rack; + slogger.debug("Loaded tokens: endpoint={}/{} dc={} rack={} tokens={}", host_id, st.endpoint, dc_rack.dc, dc_rack.rack, st.tokens); + tmptr->update_topology(host_id, dc_rack, locator::node::state::normal); + co_await tmptr->update_normal_tokens(st.tokens, host_id); + tmptr->update_host_id(host_id, st.endpoint); + // gossiping hasn't started yet + // so no need to lock the endpoint + co_await _gossiper.add_saved_endpoint(host_id, st, gms::null_permit_id); + } + } + co_await replicate_to_all_cores(std::move(tmptr)); + } + + auto loaded_peer_features = co_await _sys_ks.local().load_peer_features(); + slogger.info("initial_contact_nodes={}, loaded_endpoints={}, loaded_peer_features={}", + initial_contact_nodes, loaded_endpoints | std::views::keys, loaded_peer_features.size()); + for (auto& x : loaded_peer_features) { + slogger.info("peer={}, supported_features={}", x.first, x.second); + } + co_return co_await join_topology(sys_dist_ks, proxy, std::move(initial_contact_nodes), std::move(loaded_endpoints), std::move(loaded_peer_features), get_ring_delay(), start_hm, new_generation); } From 4fee8e0e09a5288fb60fbff24f899498e0c1a23a Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Mon, 30 Dec 2024 15:35:43 +0200 Subject: [PATCH 171/397] storage_service: load peers into gossiper on boot in raft topology mode Gossiper manages address map now, so load peers table into the gossiper on reboot to be able to map ids to ips as early as possible. --- service/storage_service.cc | 92 ++++++++++++++++++++++---------------- service/storage_service.hh | 2 +- 2 files changed, 54 insertions(+), 40 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 7468d91b0487..6725d21997e0 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -511,18 +511,20 @@ future storage_service::sync_raft_t // add one. const auto& host_id_to_ip_map = *(co_await get_host_id_to_ip_map()); - // Some state that is used to fill in 'peeers' table is still propagated over gossiper. + // Some state that is used to fill in 'peers' table is still propagated over gossiper. // Populate the table with the state from the gossiper here since storage_service::on_change() // (which is called each time gossiper state changes) may have skipped it because the tokens // for the node were not in the 'normal' state yet auto info = get_peer_info_for_update(*ip); - // And then amend with the info from raft - info.tokens = rs.ring.value().tokens; - info.data_center = rs.datacenter; - info.rack = rs.rack; - info.release_version = rs.release_version; - info.supported_features = fmt::to_string(fmt::join(rs.supported_features, ",")); - sys_ks_futures.push_back(_sys_ks.local().update_peer_info(*ip, host_id, info)); + if (info) { + // And then amend with the info from raft + info->tokens = rs.ring.value().tokens; + info->data_center = rs.datacenter; + info->rack = rs.rack; + info->release_version = rs.release_version; + info->supported_features = fmt::to_string(fmt::join(rs.supported_features, ",")); + sys_ks_futures.push_back(_sys_ks.local().update_peer_info(*ip, host_id, *info)); + } if (!prev_normal.contains(id)) { nodes_to_notify.joined.push_back(*ip); } @@ -2550,7 +2552,7 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit slogger.debug("handle_state_normal: endpoint={} is_normal_token_owner={} endpoint_to_remove={} owned_tokens={}", endpoint, is_normal_token_owner, endpoints_to_remove.contains(endpoint), owned_tokens); if (!is_me(endpoint) && !owned_tokens.empty() && !endpoints_to_remove.count(endpoint)) { try { - auto info = get_peer_info_for_update(endpoint); + auto info = get_peer_info_for_update(endpoint).value(); info.tokens = std::move(owned_tokens); co_await _sys_ks.local().update_peer_info(endpoint, host_id, info); } catch (...) { @@ -2761,17 +2763,16 @@ future<> storage_service::on_restart(gms::inet_address endpoint, gms::endpoint_s return make_ready_future(); } -db::system_keyspace::peer_info storage_service::get_peer_info_for_update(inet_address endpoint) { +std::optional storage_service::get_peer_info_for_update(inet_address endpoint) { auto ep_state = _gossiper.get_endpoint_state_ptr(endpoint); if (!ep_state) { return db::system_keyspace::peer_info{}; } auto info = get_peer_info_for_update(endpoint, ep_state->get_application_state_map()); - if (!info) { + if (!info && !raft_topology_change_enabled()) { on_internal_error_noexcept(slogger, seastar::format("get_peer_info_for_update({}): application state has no peer info: {}", endpoint, ep_state->get_application_state_map())); - return db::system_keyspace::peer_info{}; } - return *info; + return info; } std::optional storage_service::get_peer_info_for_update(inet_address endpoint, const gms::application_state_map& app_state_map) { @@ -3007,35 +3008,48 @@ future<> storage_service::join_cluster(sharded& // We must allow restarts of zero-token nodes in the gossip-based topology due to the recovery mode. } - if (_db.local().get_config().load_ring_state() && !raft_topology_change_enabled()) { - slogger.info("Loading persisted ring state"); - - auto tmlock = co_await get_token_metadata_lock(); - auto tmptr = co_await get_mutable_token_metadata_ptr(); - for (auto& [host_id, st] : loaded_endpoints) { - if (st.endpoint == get_broadcast_address()) { - // entry has been mistakenly added, delete it - slogger.warn("Loaded saved endpoint={}/{} has my broadcast address. Deleting it", host_id, st.endpoint); - co_await _sys_ks.local().remove_endpoint(st.endpoint); - } else { - if (host_id == my_host_id()) { - on_internal_error(slogger, format("Loaded saved endpoint {} with my host_id={}", st.endpoint, host_id)); - } - if (!st.opt_dc_rack) { - st.opt_dc_rack = locator::endpoint_dc_rack::default_location; - slogger.warn("Loaded no dc/rack for saved endpoint={}/{}. Set to default={}/{}", host_id, st.endpoint, st.opt_dc_rack->dc, st.opt_dc_rack->rack); + if (!raft_topology_change_enabled()) { + if (_db.local().get_config().load_ring_state()) { + slogger.info("Loading persisted ring state"); + + auto tmlock = co_await get_token_metadata_lock(); + auto tmptr = co_await get_mutable_token_metadata_ptr(); + for (auto& [host_id, st] : loaded_endpoints) { + if (st.endpoint == get_broadcast_address()) { + // entry has been mistakenly added, delete it + slogger.warn("Loaded saved endpoint={}/{} has my broadcast address. Deleting it", host_id, st.endpoint); + co_await _sys_ks.local().remove_endpoint(st.endpoint); + } else { + if (host_id == my_host_id()) { + on_internal_error(slogger, format("Loaded saved endpoint {} with my host_id={}", st.endpoint, host_id)); + } + if (!st.opt_dc_rack) { + st.opt_dc_rack = locator::endpoint_dc_rack::default_location; + slogger.warn("Loaded no dc/rack for saved endpoint={}/{}. Set to default={}/{}", host_id, st.endpoint, st.opt_dc_rack->dc, st.opt_dc_rack->rack); + } + const auto& dc_rack = *st.opt_dc_rack; + slogger.debug("Loaded tokens: endpoint={}/{} dc={} rack={} tokens={}", host_id, st.endpoint, dc_rack.dc, dc_rack.rack, st.tokens); + tmptr->update_topology(host_id, dc_rack, locator::node::state::normal); + co_await tmptr->update_normal_tokens(st.tokens, host_id); + tmptr->update_host_id(host_id, st.endpoint); + // gossiping hasn't started yet + // so no need to lock the endpoint + co_await _gossiper.add_saved_endpoint(host_id, st, gms::null_permit_id); } - const auto& dc_rack = *st.opt_dc_rack; - slogger.debug("Loaded tokens: endpoint={}/{} dc={} rack={} tokens={}", host_id, st.endpoint, dc_rack.dc, dc_rack.rack, st.tokens); - tmptr->update_topology(host_id, dc_rack, locator::node::state::normal); - co_await tmptr->update_normal_tokens(st.tokens, host_id); - tmptr->update_host_id(host_id, st.endpoint); - // gossiping hasn't started yet - // so no need to lock the endpoint - co_await _gossiper.add_saved_endpoint(host_id, st, gms::null_permit_id); } + co_await replicate_to_all_cores(std::move(tmptr)); } - co_await replicate_to_all_cores(std::move(tmptr)); + } else { + slogger.info("Loading persisted peers into the gossiper"); + // If topology coordinator is enabled only load peers into the gossiper (since it is were ID to IP maopping is managed) + // No need to update topology. + co_await coroutine::parallel_for_each(loaded_endpoints, [&] (auto& e) -> future<> { + auto& [host_id, st] = e; + if (host_id == my_host_id()) { + on_internal_error(slogger, format("Loaded saved endpoint {} with my host_id={}", st.endpoint, host_id)); + } + co_await _gossiper.add_saved_endpoint(host_id, st, gms::null_permit_id); + }); } auto loaded_peer_features = co_await _sys_ks.local().load_peer_features(); diff --git a/service/storage_service.hh b/service/storage_service.hh index 2b6579852857..ed5d102ff582 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -532,7 +532,7 @@ public: virtual void on_drop_aggregate(const sstring& ks_name, const sstring& aggregate_name) override {} virtual void on_drop_view(const sstring& ks_name, const sstring& view_name) override {} private: - db::system_keyspace::peer_info get_peer_info_for_update(inet_address endpoint); + std::optional get_peer_info_for_update(inet_address endpoint); // return an engaged value iff app_state_map has changes to the peer info std::optional get_peer_info_for_update(inet_address endpoint, const gms::application_state_map& app_state_map); From 2736a3e1526ee96c14204f119c0318bd5a4aeabd Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Mon, 30 Dec 2024 16:37:16 +0200 Subject: [PATCH 172/397] storage_service: do not add endpoint to the gossiper during topology loading. As removed comment says it was done because storage_service::join_cluster did not load gossiper endpoint but now it does. --- service/storage_service.cc | 36 +----------------------------------- 1 file changed, 1 insertion(+), 35 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 6725d21997e0..78de47f1c82e 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -789,45 +789,11 @@ future<> storage_service::topology_state_load(state_change_hint hint) { co_await update_fence_version(_topology_state_machine._topology.fence_version); - // We don't load gossiper endpoint states in storage_service::join_cluster - // if raft_topology_change_enabled(). On the other hand gossiper is still needed - // even in case of raft_topology_change_enabled() mode, since it still contains part - // of the cluster state. To work correctly, the gossiper needs to know the current - // endpoints. We cannot rely on seeds alone, since it is not guaranteed that seeds - // will be up to date and reachable at the time of restart. - const auto tmptr = get_token_metadata_ptr(); - for (const auto& node : tmptr->get_topology().get_nodes()) { - const auto& host_id = node.get().host_id(); - const auto& ep = node.get().endpoint(); - if (is_me(host_id)) { - continue; - } - if (ep == inet_address{}) { - continue; - } - auto permit = co_await _gossiper.lock_endpoint(ep, gms::null_permit_id); - // Add the endpoint if it doesn't exist yet in gossip - // since it is not loaded in join_cluster in the - // raft_topology_change_enabled() case. - if (!_gossiper.get_endpoint_state_ptr(ep)) { - gms::loaded_endpoint_state st; - st.endpoint = ep; - st.tokens = tmptr->get_tokens(host_id) | std::ranges::to>(); - st.opt_dc_rack = node.get().dc_rack(); - // Save tokens, not needed for raft topology management, but needed by legacy - // Also ip -> id mapping is needed for address map recreation on reboot - if (node.get().is_this_node() && !st.tokens.empty()) { - st.opt_status = gms::versioned_value::normal(st.tokens); - } - co_await _gossiper.add_saved_endpoint(host_id, std::move(st), permit.id()); - } - } - // As soon as a node joins token_metadata.topology we // need to drop all its rpc connections with ignored_topology flag. { std::vector> futures; - tmptr->get_topology().for_each_node([&](const locator::node& n) { + get_token_metadata_ptr()->get_topology().for_each_node([&](const locator::node& n) { const auto ep = n.endpoint(); if (ep != inet_address{} && !saved_tmpr->get_topology().has_endpoint(ep)) { futures.push_back(remove_rpc_client_with_ignored_topology(ep, n.host_id())); From 7e3a19673494534e04217380338fb8d001ec988d Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 1 Jan 2025 12:56:21 +0200 Subject: [PATCH 173/397] gossiper: fix the logic of shadow_round parameter Currently the logic is mirrored shadow_round is true in on shadow round. Fix it but flipping all the logic. --- gms/gossiper.cc | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/gms/gossiper.cc b/gms/gossiper.cc index c19b2db0cda4..19747be504f7 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -604,7 +604,7 @@ future<> gossiper::do_apply_state_locally(gms::inet_address node, endpoint_state logger.warn("received an invalid gossip generation for peer {}; local generation = {}, received generation = {}", node, local_generation, remote_generation); } else if (remote_generation > local_generation) { - logger.trace("Updating heartbeat state generation to {} from {} for {} (notify={})", remote_generation, local_generation, node, shadow_round); + logger.trace("Updating heartbeat state generation to {} from {} for {} (notify={})", remote_generation, local_generation, node, !shadow_round); // major state change will handle the update by inserting the remote state directly co_await handle_major_state_change(node, std::move(remote_state), permit.id(), shadow_round); } else if (remote_generation == local_generation) { @@ -617,21 +617,21 @@ future<> gossiper::do_apply_state_locally(gms::inet_address node, endpoint_state } else { logger.debug("Ignoring remote version {} <= {} for {}", remote_max_version, local_max_version, node); } - if (!is_alive(node) && !is_dead_state(get_endpoint_state(node)) && shadow_round) { // unless of course, it was dead + if (!is_alive(node) && !is_dead_state(get_endpoint_state(node)) && !shadow_round) { // unless of course, it was dead mark_alive(node); } } else { logger.debug("Ignoring remote generation {} < {}", remote_generation, local_generation); } } else { - logger.debug("Applying remote_state for node {} ({} node)", node, shadow_round ? "old" : "new"); + logger.debug("Applying remote_state for node {} ({} node)", node, !shadow_round ? "old" : "new"); co_await handle_major_state_change(node, std::move(remote_state), permit.id(), shadow_round); } } future<> gossiper::apply_state_locally_in_shadow_round(std::unordered_map map) { for (auto& [node, remote_state] : map) { - co_await do_apply_state_locally(node, std::move(remote_state), false); + co_await do_apply_state_locally(node, std::move(remote_state), true); } } @@ -668,7 +668,7 @@ future<> gossiper::apply_state_locally(std::map ma } } return seastar::with_semaphore(_apply_state_locally_semaphore, 1, [this, &ep, &map] () mutable { - return do_apply_state_locally(ep, std::move(map[ep]), true); + return do_apply_state_locally(ep, std::move(map[ep]), false); }); }); @@ -1795,7 +1795,7 @@ future<> gossiper::handle_major_state_change(inet_address ep, endpoint_state eps endpoint_state_ptr eps_old = get_endpoint_state_ptr(ep); - if (!is_dead_state(eps) && shadow_round) { + if (!is_dead_state(eps) && !shadow_round) { if (_endpoint_state_map.contains(ep)) { logger.info("Node {} has restarted, now UP, status = {}", ep, get_gossip_status(eps)); } else { @@ -1805,7 +1805,7 @@ future<> gossiper::handle_major_state_change(inet_address ep, endpoint_state eps logger.trace("Adding endpoint state for {}, status = {}", ep, get_gossip_status(eps)); co_await replicate(ep, eps, pid); - if (!shadow_round) { + if (shadow_round) { co_return; } @@ -1915,7 +1915,7 @@ future<> gossiper::apply_new_states(inet_address addr, endpoint_state local_stat // being replicated to all shards. co_await replicate(addr, std::move(local_state), pid); - if (!shadow_round) { + if (shadow_round) { co_return; } From 245483f1bce70558de1b2dac0b738b59eaf43678 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 2 Jan 2025 13:19:35 +0200 Subject: [PATCH 174/397] topology coordinator: reject replace request if topology does not match Currently it should not happen because gossiper shadow round does similar check, but we want to drop states that propagate through raft from the gossiper eventually. --- service/storage_service.cc | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/service/storage_service.cc b/service/storage_service.cc index 78de47f1c82e..ea299d16115e 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -6783,6 +6783,13 @@ future storage_service::join_node_request_handler(join co_return result; } + if (replaced_it->second.datacenter != params.datacenter || replaced_it->second.rack != params.rack) { + result.result = join_node_request_result::rejected{ + .reason = fmt::format("Cannot replace node in {}/{} with node in {}/{}", replaced_it->second.datacenter, replaced_it->second.rack, params.datacenter, params.rack), + }; + co_return result; + } + auto is_zero_token = params.num_tokens == 0 && params.tokens_string.empty(); if (replaced_it->second.ring.value().tokens.empty() && !is_zero_token) { result.result = join_node_request_result::rejected{ From 0ea8cd2bb81dadb90b4f9e632255025addfc8238 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 31 Dec 2024 15:53:52 +0800 Subject: [PATCH 175/397] test/pylib/minio_server: use error level for fatal errors Previously fatal errors like missing Minio executable were logged at INFO level, which could be filtered out by log settings. Switch to ERROR level to ensure these critical issues are always visible to developers. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22084 --- test/pylib/minio_server.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/test/pylib/minio_server.py b/test/pylib/minio_server.py index 865522c0ec1f..ac45bb689fac 100755 --- a/test/pylib/minio_server.py +++ b/test/pylib/minio_server.py @@ -222,7 +222,7 @@ def print_environ(self): async def start(self): if self.srv_exe is None: - self.logger.info("Minio not installed, get it from https://dl.minio.io/server/minio/release/linux-amd64/minio and put into PATH") + self.logger.error("Minio not installed, get it from https://dl.minio.io/server/minio/release/linux-amd64/minio and put into PATH") return self.log_file = self.log_filename.open("wb") @@ -238,7 +238,7 @@ async def start(self): else: break else: - self.logger.info("Failed to start Minio server") + self.logger.error("Failed to start Minio server") return self.create_conf_file(self.address, self.port, self.access_key, self.secret_key, self.DEFAULT_REGION, self.config_file) @@ -261,7 +261,7 @@ async def start(self): await self.mc('admin', 'policy', 'attach', alias, 'test-policy', '--user', self.access_key) except Exception as e: - self.logger.info(f'MC failed: {e}') + self.logger.error(f'MC failed: {e}') await self.stop() async def stop(self): From e577f1d1414d640a68fd5adaef8b3220be72bdf7 Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Wed, 18 Dec 2024 15:50:03 +0100 Subject: [PATCH 176/397] test: Add test case for checking read repair diff calculation when having conflicting keys. The test updates two rows with keys that result in a Murmur3 hash collision, which is used to generate Scylla tokens. These tokens are involved in read repair diff calculations. Due to the identical token values, a hash map key collision occurs. Consequently, an incorrect value from the second row (with a different primary key) is then sent for writing as 'repaired', causing data corruption. --- .../test_conflicting_keys_read_repair.py | 82 +++++++++++++++++++ 1 file changed, 82 insertions(+) create mode 100644 test/topology_custom/test_conflicting_keys_read_repair.py diff --git a/test/topology_custom/test_conflicting_keys_read_repair.py b/test/topology_custom/test_conflicting_keys_read_repair.py new file mode 100644 index 000000000000..2c792dfee3c2 --- /dev/null +++ b/test/topology_custom/test_conflicting_keys_read_repair.py @@ -0,0 +1,82 @@ +# +# Copyright (C) 2024-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +import logging +import pytest +import asyncio +import time + +from cassandra import ConsistencyLevel # type: ignore +from cassandra.query import SimpleStatement # type: ignore +from test.pylib.manager_client import ManagerClient +from test.pylib.util import wait_for_cql_and_get_hosts + + +logger = logging.getLogger(__name__) + + +@pytest.mark.xfail(reason="https://github.com/scylladb/scylladb/issues/19101") +@pytest.mark.asyncio +async def test_read_repair_with_conflicting_hash_keys(request: pytest.FixtureRequest, manager: ManagerClient) -> None: + """ + Test that conflicting hash keys are handled correctly during read repair. + Issue https://github.com/scylladb/scylladb/issues/19101 + + 1. Create a new cluster with 3 nodes. + 2. Create a keyspace and a table with replication factor = 3. + 3. Stop one of the nodes. + 4. Add 2 rows that have primary keys causing a hash collision. + 5. Start the offline node. + 6. Run a SELECT query with ALL consistency level, forcing reading from all 3 nodes. + The node that's been offline will not have a value, causing a read repair. + Since difference calculation logic is using a token for it's hashmap key and the + token value is the same for both keys, this causes an incorrect diff calculation + and propagation to the node that was offline. + 7. Run the same SELECT query with ALL consistency level, forcing reading from all 3 nodes. + now there is also a conflict, since the node that was reset got an incorrect value as a + result of and prev step read repair. This incorrect value is newer than others, thus it + will be the result of reconciliation in case the diff calculation algorithm is using a + token as a key. + + """ + logger.info("Creating a new cluster") + srvs = await manager.servers_add(3) + cql, _ = await manager.get_ready_cql(srvs) + + await cql.run_async("CREATE KEYSPACE ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3};") + await cql.run_async("CREATE TABLE ks.t (pk bigint PRIMARY KEY, c int);") + + # Stop one of the nodes. + await manager.server_stop_gracefully(srvs[0].server_id) + + # Add rows with partition kays that cause murmur3 hash collision, token value [6874760189787677834]. + pk1 = -4818441857111425024 + pk2 = -8686612841249112064 + await cql.run_async(SimpleStatement(f"INSERT INTO ks.t (pk, c) VALUES ({pk1}, 111)", consistency_level=ConsistencyLevel.ONE)) + await cql.run_async(SimpleStatement(f"INSERT INTO ks.t (pk, c) VALUES ({pk2}, 222)", consistency_level=ConsistencyLevel.ONE)) + + # Start the offline node. + await manager.server_start(srvs[0].server_id, wait_others=2) + + # Run a SELECT query with ALL consistency level, forcing reading from all 3 nodes. + res = await cql.run_async(SimpleStatement("SELECT * FROM ks.t", consistency_level=ConsistencyLevel.ALL)) + + # Validate the results (should be OK). + assert len(res) == 2 + for row in res: + if (row.pk == pk1): + assert row.c == 111 + elif (row.pk == pk2): + assert row.c == 222 + + res = await cql.run_async(SimpleStatement("SELECT * FROM ks.t", consistency_level=ConsistencyLevel.ALL)) + + # Validate the results (will be wrong in case the diff calculation hash map uses tokens as keys). + assert len(res) == 2 + for row in res: + if (row.pk == pk1): + assert row.c == 111 + elif (row.pk == pk2): + assert row.c == 222 From 39785c6f4e265c736214dd186eb0eae58a9c92d1 Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Thu, 19 Dec 2024 17:13:55 +0100 Subject: [PATCH 177/397] storage_proxy/read_repair: Use `partition_key` instead of `token` key for mutation diff calculation hashmap. This update addresses an issue in the mutation diff calculation algorithm used during read repair. Previously, the algorithm used `token` as the hashmap key. Since `token` is calculated basing on the Murmur3 hash function, it could generate duplicate values for different partition keys, causing corruption in the affected rows' values. Fixes scylladb/scylladb#19101 --- service/storage_proxy.cc | 24 +++++++++---------- service/storage_proxy.hh | 4 +++- test/topology/util.py | 3 +-- .../test_conflicting_keys_read_repair.py | 1 - 4 files changed, 15 insertions(+), 17 deletions(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 6265191f121f..0b187c91c321 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -4398,7 +4398,7 @@ size_t storage_proxy::hint_to_dead_endpoints(std::unique_ptr& m } } -future> storage_proxy::schedule_repair(locator::effective_replication_map_ptr ermp, std::unordered_map>> diffs, db::consistency_level cl, tracing::trace_state_ptr trace_state, +future> storage_proxy::schedule_repair(locator::effective_replication_map_ptr ermp, mutations_per_partition_key_map diffs, db::consistency_level cl, tracing::trace_state_ptr trace_state, service_permit permit) { if (diffs.empty()) { return make_ready_future>(bo::success()); @@ -4704,7 +4704,7 @@ class data_read_resolver : public abstract_read_resolver { bool _all_reached_end = true; query::short_read _is_short_read; std::vector _data_results; - std::unordered_map>> _diffs; + mutations_per_partition_key_map _diffs; private: void on_timeout() override { fail_request(read_timeout_exception(_schema->ks_name(), _schema->cf_name(), _cl, response_count(), _targets_count, response_count() != 0)); @@ -4911,7 +4911,8 @@ class data_read_resolver : public abstract_read_resolver { return got_incomplete_information_across_partitions(s, cmd, last_row, rp, versions, is_reversed); } public: - data_read_resolver(schema_ptr schema, db::consistency_level cl, size_t targets_count, storage_proxy::clock_type::time_point timeout) : abstract_read_resolver(std::move(schema), cl, targets_count, timeout) { + data_read_resolver(schema_ptr schema, db::consistency_level cl, size_t targets_count, storage_proxy::clock_type::time_point timeout) : abstract_read_resolver(std::move(schema), cl, targets_count, timeout), + _diffs(10, partition_key::hashing(*_schema), partition_key::equality(*_schema)) { _data_results.reserve(targets_count); } void add_mutate_data(locator::host_id from, foreign_ptr> result) { @@ -5047,7 +5048,7 @@ class data_read_resolver : public abstract_read_resolver { bool has_diff = false; - // calculate differences + // Сalculate differences: iterate over the versions from all the nodes and calculate the difference with the reconciled result. for (auto z : std::views::zip(versions, reconciled_partitions)) { const mutation& m = std::get<1>(z).mut; for (const version& v : std::get<0>(z)) { @@ -5059,15 +5060,12 @@ class data_read_resolver : public abstract_read_resolver { has_diff = true; mdiff = mutation(schema, m.decorated_key(), std::move(diff)); } - if (auto [it, added] = _diffs[m.token()].try_emplace(v.from, std::move(mdiff)); !added) { - // should not really happen, but lets try to deal with it - if (mdiff) { - if (it->second) { - it->second.value().apply(std::move(mdiff.value())); - } else { - it->second = std::move(mdiff); - } - } + if (auto [it, added] = _diffs[m.key()].try_emplace(v.from, std::move(mdiff)); !added) { + // A collision could happen only in 2 cases: + // 1. We have 2 versions for the same node. + // 2. `versions` (and or) `reconciled_partitions` are not unique per partition key. + // Both cases are not possible unless there is a bug in the reconcilliation code. + on_internal_error(slogger, fmt::format("Partition key conflict, key: {}, node: {}, table: {}.", m.key(), v.from, schema->ks_name())); } co_await coroutine::maybe_yield(); } diff --git a/service/storage_proxy.hh b/service/storage_proxy.hh index 6689b68c5d8f..85756c3a0937 100644 --- a/service/storage_proxy.hh +++ b/service/storage_proxy.hh @@ -91,6 +91,8 @@ struct hint_wrapper; struct read_repair_mutation; using replicas_per_token_range = std::unordered_map>; +using mutations_per_partition_key_map = + std::unordered_map>, partition_key::hashing, partition_key::equality>; struct query_partition_key_range_concurrent_result { std::vector>> result; @@ -422,7 +424,7 @@ private: future> mutate_prepare(Range&& mutations, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit, db::allow_per_partition_rate_limit allow_limit); future> mutate_begin(unique_response_handler_vector ids, db::consistency_level cl, tracing::trace_state_ptr trace_state, std::optional timeout_opt = { }); future> mutate_end(future> mutate_result, utils::latency_counter, write_stats& stats, tracing::trace_state_ptr trace_state); - future> schedule_repair(locator::effective_replication_map_ptr ermp, std::unordered_map>> diffs, db::consistency_level cl, tracing::trace_state_ptr trace_state, service_permit permit); + future> schedule_repair(locator::effective_replication_map_ptr ermp, mutations_per_partition_key_map diffs, db::consistency_level cl, tracing::trace_state_ptr trace_state, service_permit permit); bool need_throttle_writes() const; void unthrottle(); void handle_read_error(std::variant failure, bool range); diff --git a/test/topology/util.py b/test/topology/util.py index aa3e4769bcf5..6b55d9c000e5 100644 --- a/test/topology/util.py +++ b/test/topology/util.py @@ -365,8 +365,7 @@ async def verify(): stream_to_timestamp = { stream: gen.time for gen in generations for stream in gen.streams} - # FIXME: Doesn't work with all_pages=True (https://github.com/scylladb/scylladb/issues/19101) - cdc_log = await cql.run_async(f"SELECT * FROM {ks_name}.tbl_scylla_cdc_log", all_pages=False) + cdc_log = await cql.run_async(f"SELECT * FROM {ks_name}.tbl_scylla_cdc_log", all_pages=True) for log_entry in cdc_log: assert log_entry.cdc_stream_id in stream_to_timestamp timestamp = stream_to_timestamp[log_entry.cdc_stream_id] diff --git a/test/topology_custom/test_conflicting_keys_read_repair.py b/test/topology_custom/test_conflicting_keys_read_repair.py index 2c792dfee3c2..6d01e0f2ebb5 100644 --- a/test/topology_custom/test_conflicting_keys_read_repair.py +++ b/test/topology_custom/test_conflicting_keys_read_repair.py @@ -17,7 +17,6 @@ logger = logging.getLogger(__name__) -@pytest.mark.xfail(reason="https://github.com/scylladb/scylladb/issues/19101") @pytest.mark.asyncio async def test_read_repair_with_conflicting_hash_keys(request: pytest.FixtureRequest, manager: ManagerClient) -> None: """ From 155480595ff2daea1efa524e4f0e29928668852f Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Fri, 20 Dec 2024 16:03:25 +0100 Subject: [PATCH 178/397] storage_proxy/read_repair: Remove redundant 'schema' parameter from `data_read_resolver::resolve` function. The `data_read_resolver` class inherits from `abstract_read_resolver`, which already includes the `schema_ptr _schema` member. Therefore, using a separate function parameter in `data_read_resolver::resolve` initialized with the same variable in `abstract_read_executor` is redundant. --- service/storage_proxy.cc | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 0b187c91c321..6a65281322ce 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -4957,7 +4957,7 @@ class data_read_resolver : public abstract_read_resolver { bool all_reached_end() const { return _all_reached_end; } - future> resolve(schema_ptr schema, const query::read_command& cmd, uint64_t original_row_limit, uint64_t original_per_partition_limit, + future> resolve(const query::read_command& cmd, uint64_t original_row_limit, uint64_t original_per_partition_limit, uint32_t original_partition_limit) { SCYLLA_ASSERT(_data_results.size()); @@ -4969,10 +4969,10 @@ class data_read_resolver : public abstract_read_resolver { co_return reconcilable_result(p->row_count(), p->partitions(), p->is_short_read()); } - const auto& s = *schema; + const auto& schema = *_schema; // return true if lh > rh - auto cmp = [&s](reply& lh, reply& rh) { + auto cmp = [&schema](reply& lh, reply& rh) { if (lh.result->partitions().size() == 0) { return false; // reply with empty partition array goes to the end of the sorted array } else if (rh.result->partitions().size() == 0) { @@ -4980,7 +4980,7 @@ class data_read_resolver : public abstract_read_resolver { } else { auto lhk = lh.result->partitions().back().mut().key(); auto rhk = rh.result->partitions().back().mut().key(); - return lhk.ring_order_tri_compare(s, rhk) > 0; + return lhk.ring_order_tri_compare(schema, rhk) > 0; } }; @@ -5010,7 +5010,7 @@ class data_read_resolver : public abstract_read_resolver { v.reserve(_targets_count); for (reply& r : _data_results) { auto pit = r.result->partitions().rbegin(); - if (pit != r.result->partitions().rend() && pit->mut().key().legacy_equal(s, max_key)) { + if (pit != r.result->partitions().rend() && pit->mut().key().legacy_equal(schema, max_key)) { bool reached_partition_end = pit->row_count() < cmd.slice.partition_row_limit(); v.emplace_back(r.from, std::move(*pit), r.reached_end, reached_partition_end); r.result->partitions().pop_back(); @@ -5031,11 +5031,11 @@ class data_read_resolver : public abstract_read_resolver { auto it = std::ranges::find_if(v, [] (auto&& ver) { return bool(ver.par); }); - auto m = mutation(schema, it->par->mut().key()); + auto m = mutation(_schema, it->par->mut().key()); for (const version& ver : v) { if (ver.par) { mutation_application_stats app_stats; - co_await apply_gently(m.partition(), *schema, ver.par->mut().partition(), *schema, app_stats); + co_await apply_gently(m.partition(), schema, ver.par->mut().partition(), schema, app_stats); } } auto live_row_count = m.live_row_count(); @@ -5053,26 +5053,26 @@ class data_read_resolver : public abstract_read_resolver { const mutation& m = std::get<1>(z).mut; for (const version& v : std::get<0>(z)) { auto diff = v.par - ? m.partition().difference(*schema, (co_await unfreeze_gently(v.par->mut(), schema)).partition()) - : mutation_partition(*schema, m.partition()); + ? m.partition().difference(schema, (co_await unfreeze_gently(v.par->mut(), _schema)).partition()) + : mutation_partition(schema, m.partition()); std::optional mdiff; if (!diff.empty()) { has_diff = true; - mdiff = mutation(schema, m.decorated_key(), std::move(diff)); + mdiff = mutation(_schema, m.decorated_key(), std::move(diff)); } if (auto [it, added] = _diffs[m.key()].try_emplace(v.from, std::move(mdiff)); !added) { // A collision could happen only in 2 cases: // 1. We have 2 versions for the same node. // 2. `versions` (and or) `reconciled_partitions` are not unique per partition key. // Both cases are not possible unless there is a bug in the reconcilliation code. - on_internal_error(slogger, fmt::format("Partition key conflict, key: {}, node: {}, table: {}.", m.key(), v.from, schema->ks_name())); + on_internal_error(slogger, fmt::format("Partition key conflict, key: {}, node: {}, table: {}.", m.key(), v.from, schema.ks_name())); } co_await coroutine::maybe_yield(); } } if (has_diff) { - if (got_incomplete_information(*schema, cmd, original_row_limit, original_per_partition_limit, + if (got_incomplete_information(schema, cmd, original_row_limit, original_per_partition_limit, original_partition_limit, reconciled_partitions, versions)) { co_return std::nullopt; } @@ -5361,7 +5361,7 @@ class abstract_read_executor : public enable_shared_from_thisresolve(_schema, *cmd, original_row_limit(), original_per_partition_row_limit(), original_partition_limit()); // reconciliation happens here + auto rr_opt = co_await data_resolver->resolve(*cmd, original_row_limit(), original_per_partition_row_limit(), original_partition_limit()); // reconciliation happens here // We generate a retry if at least one node reply with count live columns but after merge we have less // than the total number of column we are interested in (which may be < count on a retry). From 9d0f27e7c11d725416eb7701d44c53c717d0a677 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Sat, 4 Jan 2025 05:12:03 +0800 Subject: [PATCH 179/397] test.py: Throw only if scylla executable is not found Previously, we had inconsistent behavior around missing executables: - 561e88f0 added early failure if any executable was missing - 8b7a5ca8 added a partial skip for combined_test, but didn't properly handle build paths and artifacts This change: 1. Moves executable existence check to PythonTestSuite class 3. Eliminates redundant os.access() checks This allows running tests with a partial build while properly handling missing executables, particularly for the combined_test suite. In a succeeding change, we will correct the check for combined_tests. Refs scylladb/scylladb#19489 Refs scylladb/scylladb#22086 Signed-off-by: Kefu Chai --- test.py | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/test.py b/test.py index 4052150c082a..bc0d5792b2a5 100755 --- a/test.py +++ b/test.py @@ -93,14 +93,8 @@ def path_to(mode, *components): build_dir = 'build' if os.path.exists(os.path.join(build_dir, 'build.ninja')): *dir_components, basename = components - exe_path = os.path.join(build_dir, *dir_components, all_modes[mode], basename) - else: - exe_path = os.path.join(build_dir, mode, *components) - if not os.access(exe_path, os.F_OK): - raise FileNotFoundError(f"{exe_path} does not exist.") - elif not os.access(exe_path, os.X_OK): - raise PermissionError(f"{exe_path} is not executable.") - return exe_path + return os.path.join(build_dir, *dir_components, all_modes[mode], basename) + return os.path.join(build_dir, mode, *components) def ninja(target): @@ -487,6 +481,10 @@ class PythonTestSuite(TestSuite): def __init__(self, path, cfg: dict, options: argparse.Namespace, mode: str) -> None: super().__init__(path, cfg, options, mode) self.scylla_exe = path_to(self.mode, "scylla") + if not os.access(self.scylla_exe, os.F_OK): + raise FileNotFoundError(f"{self.scylla_exe} does not exist.") + if not os.access(self.scylla_exe, os.X_OK): + raise PermissionError(f"{self.scylla_exe} is not executable.") self.scylla_env = dict(self.base_env) if self.mode == "coverage": self.scylla_env.update(coverage.env(self.scylla_exe, distinct_id=self.name)) From 220cafe7c474837d64af1fb9c9b597aa1b711f12 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Sat, 4 Jan 2025 05:48:56 +0800 Subject: [PATCH 180/397] test.py: Fix path checking for combined_test executable Previously in 8b7a5ca88d, we checked for combined_test existence without the "build" component in the path. This caused the test suite to never find the executable, preventing the test cases' cache from being populated. Changes: 1. Use path_to() to check executable existence, which: - Includes the "build" component in path - Handles both CMake and configure.py build paths 2. Move existence check out of _generate_cache() for clarity This ensures combined_test and its included tests are properly discovered and run. Fixes scylladb/scylladb#22086 Signed-off-by: Kefu Chai --- test.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/test.py b/test.py index bc0d5792b2a5..1d715d722581 100755 --- a/test.py +++ b/test.py @@ -365,14 +365,9 @@ class BoostTestSuite(UnitTestSuite): _exec_name_cache: Dict[str, str] = dict() - def _generate_cache(self) -> None: - # Apply combined test only for test/boost - exe_path = pathlib.Path(self.mode, "test", self.name, 'combined_tests') - if self.name != 'boost' or not exe_path.exists(): - return - exe = path_to(self.mode, "test", self.name, 'combined_tests') + def _generate_cache(self, exec_path, exec_name) -> None: res = subprocess.run( - [exe, '--list_content'], + [exec_path, '--list_content'], check=True, capture_output=True, env=dict(os.environ, @@ -384,7 +379,7 @@ def _generate_cache(self) -> None: if not line.startswith(' '): testname = line.strip().rstrip('*') fqname = os.path.join(self.mode, self.name, testname) - self._exec_name_cache[fqname] = 'combined_tests' + self._exec_name_cache[fqname] = exec_name self._case_cache[fqname] = [] else: casename = line.strip().rstrip('*') @@ -394,7 +389,12 @@ def _generate_cache(self) -> None: def __init__(self, path, cfg: dict, options: argparse.Namespace, mode) -> None: super().__init__(path, cfg, options, mode) - self._generate_cache() + exe = path_to(self.mode, "test", self.name, 'combined_tests') + # Apply combined test only for test/boost, + # cache the tests only if the executable exists, so we can + # run test.py with a partially built tree + if self.name == 'boost' and os.path.exists(exe): + self._generate_cache(exe, 'combined_tests') async def create_test(self, shortname: str, casename: str, suite, args) -> None: fqname = os.path.join(self.mode, self.name, shortname) From c97325436237516fcec97eeb1f283674ea1fef1c Mon Sep 17 00:00:00 2001 From: "Raphael S. Carvalho" Date: Thu, 26 Dec 2024 11:41:58 -0300 Subject: [PATCH 181/397] Introduce incremental compaction strategy (ICS) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ICS is a compaction strategy that inherits size tiered properties -- therefore it's write optimized too -- but fixes its space overhead of 100% due to input files being only released on completion. That's achieved with the concept of sstable run (similar in concept to LCS levels) which breaks a large sstable into fixed-size chunks (1G by default), known as run fragments. ICS picks similar-sized runs for compaction, and fragments of those runs can be released incrementally as they're compacted, reducing the space overhead to about (number_of_input_runs * 1G). This allows user to increase storage density of nodes (from 50% to ~80%), reducing the cost of ownership. NOTE: test_system_schema_version_is_stable adjusted to account for batchlog using IncrementalCompactionStrategy contains: compaction/: added incremental_compaction_strategy.cc (.hh), incremental_backlog_tracker.cc (.hh) compaction/CMakeLists.txt: include ICS cc files configure.py: changes for ICS files, includes test db/legacy_schema_migrator.cc / db/schema_tables.cc: fallback to ICS when strategy is not supported db/system_keyspace: pick ICS for some system tables schema/schema.hh: ICS becomes default test/boost: Add incremental_compaction_test.cc test/boost/sstable_compaction_test.cc: ICS related changes test/cqlpy/test_compaction_strategy_validation.py: ICS related changes docs/architecture/compaction/compaction-strategies.rst: changes to ICS section docs/cql/compaction.rst: changes to ICS section docs/cql/ddl.rst: adds reference to ICS options docs/getting-started/system-requirements.rst: updates sentence mentioning ICS docs/kb/compaction.rst: changes to ICS section docs/kb/garbage-collection-ics.rst: add file docs/kb/index.rst: add reference to docs/operating-scylla/procedures/tips/production-readiness.rst: add ICS section some relevant commits throughout the ICS history: commit 434b97699b39c570d0d849d372bf64f418e5c692 Merge: 105586f747 30250749b8 Author: Paweł Dziepak Date: Tue Mar 12 12:14:23 2019 +0000 Merge "Introduce Incremental Compaction Strategy (ICS)" from Raphael " Introduce new compaction strategy which is essentially like size tiered but will work with the existing incremental compaction. Thus incremental compaction strategy. It works like size tiered, but each element composing a tier is a sstable run, meaning that the compaction strategy will look for N similar-sized sstable runs to compact, not just individual sstables. Parameters: * "sstable_size_in_mb": defines the maximum sstable (fragment) size composing a sstable run, which impacts directly the disk space requirement which is improved with incremental compaction. The lower the value the lower the space requirement for compaction because fragments involved will be released more frequently. * all others available in size tiered compaction strategy HOWTO ===== To change an existing table to use it, do: ALTER TABLE mykeyspace.mytable WITH compaction = {'class' : 'IncrementalCompactionStrategy'}; Set fragment size: ALTER TABLE mykeyspace.mytable WITH compaction = {'class' : 'IncrementalCompactionStrategy', 'sstable_size_in_mb' : 1000 } " commit 94ef3cd29a196bedbbeb8707e20fe78a197f30a1 Merge: dca89ce7a5 e08ef3e1a3 Author: Avi Kivity Date: Tue Sep 8 11:31:52 2020 +0300 Merge "Add feature to limit space amplification in Incremental Compaction" from Raphael " A new option, space_amplification_goal (SAG), is being added to ICS. This option will allow ICS user to set a goal on the space amplification (SA). It's not supposed to be an upper bound on the space amplification, but rather, a goal. This new option will be disabled by default as it doesn't benefit write-only (no overwrites) workloads and could hurt severely the write performance. The strategy is free to delay triggering this new behavior, in order to increase overall compaction efficiency. The graph below shows how this feature works in practice for different values of space_amplification_goal: https://user-images.githubusercontent.com/1409139/89347544-60b7b980-d681-11ea-87ab-e2fdc3ecb9f0.png When strategy finds space amplification crossed space_amplification_goal, it will work on reducing the SA by doing a cross-tier compaction on the two largest tiers. This feature works only on the two largest tiers, because taking into account others, could hurt the compaction efficiency which is based on the fact that the more similar-sized sstables are compacted together the higher the compaction efficiency will be. With SAG enabled, min_threshold only plays an important role on the smallest tiers, given that the second-largest tier could be compacted into the largest tier for a space_amplification_goal value < 2. By making the options space_amplification_goal and min_threshold independent, user will be able to tune write amplification and space amplification, based on the needs. The lower the space_amplification_goal the higher the write amplification, but by increasing the min threshold, the write amplification can be decreased to a desired amount. " commit 7d90911c5fb3fa891ad64a62147c3a6ca26d61b1 Author: Raphael S. Carvalho Date: Sat Oct 16 13:41:46 2021 -0300 compaction: ICS: Add garbage collection Today, ICS lacks an approach to persist expired tombstones in a timely manner, which is a problem because accumulation of tombstones are known to affecting latency considerably. For an expired tombstone to be purged, it has to reach the top of the LSM tree and hope that older overlapping data wasn't introduced at the bottom. The condition are there and must be satisfied to avoid data resurrection. STCS, today, has an inefficient garbage collection approach because it only picks a single sstable, which satisfies the tombstone density threshold and file staleness. That's a problem because overlapping data either on same tier or smaller tiers will prevent tombstones from being purged. Also, nothing is done to push the tombstones to the top of the tree, for the conditions to be eventually satisfied. Due to incremental compaction, ICS can more easily have an effecient GC by doing cross-tier compaction of relevant tiers. The trigger will be file staleness and tombstone density, which threshold values can be configured by tombstone_compaction_interval and tombstone_threshold, respectively. If ICS finds a tier which meets both conditions, then that tier and the larger[1] *and* closest-in-size[2] tier will be compacted together. [1]: A larger tier is picked because we want tombstones to eventually reach the top of the tree. [2]: It also has to be the closest-in-size tier as the smaller the size difference the higher the efficiency of the compaction. We want to minimize write amplification as much as possible. The staleness condition is there to prevent the same file from being picked over and over again in a short interval. With this approach, ICS will be continuously working to purge garbage while not hurting overall efficiency on a steady state, as same-tier compactions are prioritized. Signed-off-by: Raphael S. Carvalho Message-Id: <20211016164146.38010-1-raphaelsc@scylladb.com> Signed-off-by: Raphael S. Carvalho Closes scylladb/scylladb#22063 --- compaction/CMakeLists.txt | 2 + compaction/compaction_strategy.cc | 14 + compaction/compaction_strategy.hh | 5 + compaction/compaction_strategy_type.hh | 1 + compaction/incremental_backlog_tracker.cc | 128 +++++ compaction/incremental_backlog_tracker.hh | 61 ++ compaction/incremental_compaction_strategy.cc | 534 ++++++++++++++++++ compaction/incremental_compaction_strategy.hh | 112 ++++ configure.py | 3 + db/legacy_schema_migrator.cc | 6 +- db/schema_tables.cc | 6 +- db/system_keyspace.cc | 6 +- .../compaction/compaction-strategies.rst | 31 +- docs/cql/compaction.rst | 99 +++- docs/cql/ddl.rst | 2 +- docs/getting-started/system-requirements.rst | 2 +- docs/kb/compaction.rst | 45 +- docs/kb/index.rst | 1 + .../procedures/tips/production-readiness.rst | 11 + schema/schema.hh | 2 +- test/boost/CMakeLists.txt | 2 + test/boost/incremental_compaction_test.cc | 525 +++++++++++++++++ test/boost/schema_change_test.cc | 2 +- test/boost/sstable_compaction_test.cc | 53 ++ .../test_compaction_strategy_validation.py | 9 + 25 files changed, 1646 insertions(+), 16 deletions(-) create mode 100644 compaction/incremental_backlog_tracker.cc create mode 100644 compaction/incremental_backlog_tracker.hh create mode 100644 compaction/incremental_compaction_strategy.cc create mode 100644 compaction/incremental_compaction_strategy.hh create mode 100644 test/boost/incremental_compaction_test.cc diff --git a/compaction/CMakeLists.txt b/compaction/CMakeLists.txt index 6a8802b3b592..b758d5557db5 100644 --- a/compaction/CMakeLists.txt +++ b/compaction/CMakeLists.txt @@ -4,6 +4,8 @@ target_sources(compaction compaction.cc compaction_manager.cc compaction_strategy.cc + incremental_backlog_tracker.cc + incremental_compaction_strategy.cc leveled_compaction_strategy.cc size_tiered_compaction_strategy.cc task_manager_module.cc diff --git a/compaction/compaction_strategy.cc b/compaction/compaction_strategy.cc index 5abe49dd9780..4312e130caf5 100644 --- a/compaction/compaction_strategy.cc +++ b/compaction/compaction_strategy.cc @@ -29,8 +29,11 @@ #include "size_tiered_backlog_tracker.hh" #include "leveled_manifest.hh" #include "utils/to_string.hh" +#include "incremental_compaction_strategy.hh" +#include "sstables/sstable_set_impl.hh" logging::logger leveled_manifest::logger("LeveledManifest"); +logging::logger compaction_strategy_logger("CompactionStrategy"); using namespace sstables; @@ -172,6 +175,9 @@ void compaction_strategy_impl::validate_options_for_strategy_type(const std::map case compaction_strategy_type::time_window: time_window_compaction_strategy::validate_options(options, unchecked_options); break; + case compaction_strategy_type::incremental: + incremental_compaction_strategy::validate_options(options, unchecked_options); + break; default: break; } @@ -756,6 +762,9 @@ compaction_strategy make_compaction_strategy(compaction_strategy_type strategy, case compaction_strategy_type::time_window: impl = ::make_shared(options); break; + case compaction_strategy_type::incremental: + impl = make_shared(incremental_compaction_strategy(options)); + break; default: throw std::runtime_error("strategy not supported"); } @@ -770,6 +779,10 @@ future make_reshape_config(const sstables::storage& storage, res }; } +std::unique_ptr incremental_compaction_strategy::make_sstable_set(schema_ptr schema) const { + return std::make_unique(std::move(schema), false); +} + } namespace compaction { @@ -778,6 +791,7 @@ compaction_strategy_state compaction_strategy_state::make(const compaction_strat switch (cs.type()) { case compaction_strategy_type::null: case compaction_strategy_type::size_tiered: + case compaction_strategy_type::incremental: return compaction_strategy_state(default_empty_state{}); case compaction_strategy_type::leveled: return compaction_strategy_state(leveled_compaction_strategy_state{}); diff --git a/compaction/compaction_strategy.hh b/compaction/compaction_strategy.hh index 0ea40b167206..aaaaade7f4dd 100644 --- a/compaction/compaction_strategy.hh +++ b/compaction/compaction_strategy.hh @@ -17,6 +17,7 @@ struct mutation_source_metadata; class compaction_backlog_tracker; +extern logging::logger compaction_strategy_logger; using namespace compaction; @@ -69,6 +70,8 @@ public: return "LeveledCompactionStrategy"; case compaction_strategy_type::time_window: return "TimeWindowCompactionStrategy"; + case compaction_strategy_type::incremental: + return "IncrementalCompactionStrategy"; default: throw std::runtime_error("Invalid Compaction Strategy"); } @@ -85,6 +88,8 @@ public: return compaction_strategy_type::leveled; } else if (short_name == "TimeWindowCompactionStrategy") { return compaction_strategy_type::time_window; + } else if (short_name == "IncrementalCompactionStrategy") { + return compaction_strategy_type::incremental; } else { throw exceptions::configuration_exception(format("Unable to find compaction strategy class '{}'", name)); } diff --git a/compaction/compaction_strategy_type.hh b/compaction/compaction_strategy_type.hh index 87d4e8d073e8..b0a85a087ec1 100644 --- a/compaction/compaction_strategy_type.hh +++ b/compaction/compaction_strategy_type.hh @@ -17,6 +17,7 @@ enum class compaction_strategy_type { size_tiered, leveled, time_window, + incremental, }; enum class reshape_mode { strict, relaxed }; diff --git a/compaction/incremental_backlog_tracker.cc b/compaction/incremental_backlog_tracker.cc new file mode 100644 index 000000000000..ff5829a82657 --- /dev/null +++ b/compaction/incremental_backlog_tracker.cc @@ -0,0 +1,128 @@ +/* + * Copyright (C) 2019-present ScyllaDB + * + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "incremental_backlog_tracker.hh" +#include "sstables/sstables.hh" + +using namespace sstables; + +incremental_backlog_tracker::inflight_component incremental_backlog_tracker::compacted_backlog(const compaction_backlog_tracker::ongoing_compactions& ongoing_compactions) const { + inflight_component in; + for (auto& crp : ongoing_compactions) { + if (!_sstable_runs_contributing_backlog.contains(crp.first->run_identifier())) { + continue; + } + auto compacted = crp.second->compacted(); + in.total_bytes += compacted; + in.contribution += compacted * log4((crp.first->data_size())); + } + return in; +} + +incremental_backlog_tracker::backlog_calculation_result +incremental_backlog_tracker::calculate_sstables_backlog_contribution(const std::unordered_map& all, const incremental_compaction_strategy_options& options, unsigned threshold) { + int64_t total_backlog_bytes = 0; + float sstables_backlog_contribution = 0.0f; + std::unordered_set sstable_runs_contributing_backlog = {}; + + if (!all.empty()) { + auto freeze = [] (const sstable_run& run) { return make_lw_shared(run); }; + for (auto& bucket : incremental_compaction_strategy::get_buckets(boost::copy_range>(all | boost::adaptors::map_values | boost::adaptors::transformed(freeze)), options)) { + if (!incremental_compaction_strategy::is_bucket_interesting(bucket, threshold)) { + continue; + } + for (const frozen_sstable_run& run_ptr : bucket) { + auto& run = *run_ptr; + auto data_size = run.data_size(); + if (data_size > 0) { + total_backlog_bytes += data_size; + sstables_backlog_contribution += data_size * log4(data_size); + sstable_runs_contributing_backlog.insert((*run.all().begin())->run_identifier()); + } + } + } + } + return backlog_calculation_result{ + .total_backlog_bytes = total_backlog_bytes, + .sstables_backlog_contribution = sstables_backlog_contribution, + .sstable_runs_contributing_backlog = std::move(sstable_runs_contributing_backlog), + }; +} + +incremental_backlog_tracker::incremental_backlog_tracker(incremental_compaction_strategy_options options) : _options(std::move(options)) {} + +double incremental_backlog_tracker::backlog(const compaction_backlog_tracker::ongoing_writes& ow, const compaction_backlog_tracker::ongoing_compactions& oc) const { + inflight_component compacted = compacted_backlog(oc); + + // Bail out if effective backlog is zero + if (_total_backlog_bytes <= compacted.total_bytes) { + return 0; + } + + // Formula for each SSTable is (Si - Ci) * log(T / Si) + // Which can be rewritten as: ((Si - Ci) * log(T)) - ((Si - Ci) * log(Si)) + // + // For the meaning of each variable, please refer to the doc in size_tiered_backlog_tracker.hh + + // Sum of (Si - Ci) for all SSTables contributing backlog + auto effective_backlog_bytes = _total_backlog_bytes - compacted.total_bytes; + + // Sum of (Si - Ci) * log (Si) for all SSTables contributing backlog + auto sstables_contribution = _sstables_backlog_contribution - compacted.contribution; + // This is subtracting ((Si - Ci) * log (Si)) from ((Si - Ci) * log(T)), yielding the final backlog + auto b = (effective_backlog_bytes * log4(_total_bytes)) - sstables_contribution; + return b > 0 ? b : 0; +} + +// Removing could be the result of a failure of an in progress write, successful finish of a +// compaction, or some one-off operation, like drop +void incremental_backlog_tracker::replace_sstables(const std::vector& old_ssts, const std::vector& new_ssts) { + auto all = _all; + auto total_bytes = _total_bytes; + auto threshold = _threshold; + auto backlog_calculation_result = incremental_backlog_tracker::backlog_calculation_result{}; + for (auto&& sst : new_ssts) { + if (sst->data_size() > 0) { + // note: we don't expect failed insertions since each sstable will be inserted once + (void)all[sst->run_identifier()].insert(sst); + total_bytes += sst->data_size(); + // Deduce threshold from the last SSTable added to the set + threshold = sst->get_schema()->min_compaction_threshold(); + } + } + + bool exhausted_input_run = false; + for (auto&& sst : old_ssts) { + if (sst->data_size() > 0) { + auto run_identifier = sst->run_identifier(); + all[run_identifier].erase(sst); + if (all[run_identifier].all().empty()) { + all.erase(run_identifier); + exhausted_input_run = true; + } + total_bytes -= sst->data_size(); + } + } + // Backlog contribution will only be refreshed when an input SSTable run was exhausted by + // compaction, so to avoid doing it for each exhausted fragment, which would be both + // overkill and expensive. + if (exhausted_input_run) { + backlog_calculation_result = calculate_sstables_backlog_contribution(all, _options, threshold); + } + + // commit calculations + std::invoke([&] () noexcept { + _all = std::move(all); + _total_bytes = total_bytes; + _threshold = threshold; + + if (exhausted_input_run) { + _total_backlog_bytes = backlog_calculation_result.total_backlog_bytes; + _sstables_backlog_contribution = backlog_calculation_result.sstables_backlog_contribution; + _sstable_runs_contributing_backlog = std::move(backlog_calculation_result.sstable_runs_contributing_backlog); + } + }); +} diff --git a/compaction/incremental_backlog_tracker.hh b/compaction/incremental_backlog_tracker.hh new file mode 100644 index 000000000000..ca6e8b5a7c28 --- /dev/null +++ b/compaction/incremental_backlog_tracker.hh @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2019-present ScyllaDB + * + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include + +#include "compaction_backlog_manager.hh" +#include "incremental_compaction_strategy.hh" + +using namespace sstables; + +// The only difference to size tiered backlog tracker is that it will calculate +// backlog contribution using total bytes of each sstable run instead of total +// bytes of an individual sstable object. +class incremental_backlog_tracker final : public compaction_backlog_tracker::impl { + incremental_compaction_strategy_options _options; + int64_t _total_bytes = 0; + int64_t _total_backlog_bytes = 0; + unsigned _threshold = 0; + double _sstables_backlog_contribution = 0.0f; + std::unordered_set _sstable_runs_contributing_backlog; + std::unordered_map _all; + + struct inflight_component { + int64_t total_bytes = 0; + double contribution = 0; + }; + + inflight_component compacted_backlog(const compaction_backlog_tracker::ongoing_compactions& ongoing_compactions) const; + + struct backlog_calculation_result { + int64_t total_backlog_bytes; + float sstables_backlog_contribution; + std::unordered_set sstable_runs_contributing_backlog; + }; + +public: + static double log4(double x) { + static const double inv_log_4 = 1.0f / std::log(4); + return log(x) * inv_log_4; + } + + static backlog_calculation_result calculate_sstables_backlog_contribution(const std::unordered_map& all, + const incremental_compaction_strategy_options& options, unsigned threshold); + + incremental_backlog_tracker(incremental_compaction_strategy_options options); + + virtual double backlog(const compaction_backlog_tracker::ongoing_writes& ow, const compaction_backlog_tracker::ongoing_compactions& oc) const override; + + // Removing could be the result of a failure of an in progress write, successful finish of a + // compaction, or some one-off operation, like drop + virtual void replace_sstables(const std::vector& old_ssts, const std::vector& new_ssts) override; + + int64_t total_bytes() const { + return _total_bytes; + } +}; diff --git a/compaction/incremental_compaction_strategy.cc b/compaction/incremental_compaction_strategy.cc new file mode 100644 index 000000000000..57aacece3754 --- /dev/null +++ b/compaction/incremental_compaction_strategy.cc @@ -0,0 +1,534 @@ +/* + * Copyright (C) 2019 ScyllaDB + * + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "sstables/sstables.hh" +#include "sstables/sstable_set.hh" +#include "cql3/statements/property_definitions.hh" +#include "compaction.hh" +#include "compaction_manager.hh" +#include "incremental_compaction_strategy.hh" +#include "incremental_backlog_tracker.hh" +#include +#include +#include +#include + +namespace sstables { + +extern logging::logger clogger; + +static long validate_min_sstable_size(const std::map& options) { + auto tmp_value = compaction_strategy_impl::get_value(options, + incremental_compaction_strategy_options::MIN_SSTABLE_SIZE_KEY); + auto min_sstable_size = cql3::statements::property_definitions::to_long(incremental_compaction_strategy_options::MIN_SSTABLE_SIZE_KEY, + tmp_value, incremental_compaction_strategy_options::DEFAULT_MIN_SSTABLE_SIZE); + if (min_sstable_size < 0) { + throw exceptions::configuration_exception(fmt::format("{} value ({}) must be non negative", + incremental_compaction_strategy_options::MIN_SSTABLE_SIZE_KEY, min_sstable_size)); + } + return min_sstable_size; +} + +static long validate_min_sstable_size(const std::map& options, std::map& unchecked_options) { + auto min_sstable_size = validate_min_sstable_size(options); + unchecked_options.erase(incremental_compaction_strategy_options::MIN_SSTABLE_SIZE_KEY); + return min_sstable_size; +} + +static double validate_bucket_low(const std::map& options) { + auto tmp_value = compaction_strategy_impl::get_value(options, + incremental_compaction_strategy_options::BUCKET_LOW_KEY); + auto bucket_low = cql3::statements::property_definitions::to_double(incremental_compaction_strategy_options::BUCKET_LOW_KEY, + tmp_value, incremental_compaction_strategy_options::DEFAULT_BUCKET_LOW); + if (bucket_low <= 0.0 || bucket_low >= 1.0) { + throw exceptions::configuration_exception(fmt::format("{} value ({}) must be between 0.0 and 1.0", + incremental_compaction_strategy_options::BUCKET_LOW_KEY, bucket_low)); + } + return bucket_low; +} + +static double validate_bucket_low(const std::map& options, std::map& unchecked_options) { + auto bucket_low = validate_bucket_low(options); + unchecked_options.erase(incremental_compaction_strategy_options::BUCKET_LOW_KEY); + return bucket_low; +} + +static double validate_bucket_high(const std::map& options) { + auto tmp_value = compaction_strategy_impl::get_value(options, + incremental_compaction_strategy_options::BUCKET_HIGH_KEY); + auto bucket_high = cql3::statements::property_definitions::to_double(incremental_compaction_strategy_options::BUCKET_HIGH_KEY, + tmp_value, incremental_compaction_strategy_options::DEFAULT_BUCKET_HIGH); + if (bucket_high <= 1.0) { + throw exceptions::configuration_exception(fmt::format("{} value ({}) must be greater than 1.0", + incremental_compaction_strategy_options::BUCKET_HIGH_KEY, bucket_high)); + } + return bucket_high; +} + +static double validate_bucket_high(const std::map& options, std::map& unchecked_options) { + auto bucket_high = validate_bucket_high(options); + unchecked_options.erase(incremental_compaction_strategy_options::BUCKET_HIGH_KEY); + return bucket_high; +} + +static int validate_fragment_size(const std::map& options) { + auto tmp_value = compaction_strategy_impl::get_value(options, + incremental_compaction_strategy::FRAGMENT_SIZE_OPTION); + auto fragment_size_in_mb = cql3::statements::property_definitions::to_int(incremental_compaction_strategy::FRAGMENT_SIZE_OPTION, + tmp_value, incremental_compaction_strategy::DEFAULT_MAX_FRAGMENT_SIZE_IN_MB); + if (fragment_size_in_mb < 100) { + clogger.warn("SStable size of {}MB is configured. The value may lead to high memory overhead due to sstables proliferation.", fragment_size_in_mb); + } + return fragment_size_in_mb; +} + +static int validate_fragment_size(const std::map& options, std::map& unchecked_options) { + auto fragment_size_in_mb = validate_fragment_size(options); + unchecked_options.erase(incremental_compaction_strategy::FRAGMENT_SIZE_OPTION); + return fragment_size_in_mb; +} + +static std::optional validate_space_amplification_goal(const std::map& options) { + auto tmp_value = compaction_strategy_impl::get_value(options, + incremental_compaction_strategy::SPACE_AMPLIFICATION_GOAL_OPTION); + if (tmp_value) { + auto space_amplification_goal = cql3::statements::property_definitions::to_double(incremental_compaction_strategy::SPACE_AMPLIFICATION_GOAL_OPTION, + tmp_value, 0.0); + if (space_amplification_goal <= 1.0 || space_amplification_goal > 2.0) { + throw exceptions::configuration_exception(fmt::format("{} value ({}) must be greater than 1.0 and less than or equal to 2.0", + incremental_compaction_strategy::SPACE_AMPLIFICATION_GOAL_OPTION, space_amplification_goal)); + } + return space_amplification_goal; + } + return std::nullopt; +} + +static std::optional validate_space_amplification_goal(const std::map& options, std::map& unchecked_options) { + auto space_amplification_goal = validate_space_amplification_goal(options); + unchecked_options.erase(incremental_compaction_strategy::SPACE_AMPLIFICATION_GOAL_OPTION); + return space_amplification_goal; +} + +incremental_compaction_strategy_options::incremental_compaction_strategy_options(const std::map& options) { + min_sstable_size = validate_min_sstable_size(options); + bucket_low = validate_bucket_low(options); + bucket_high = validate_bucket_high(options); +} + +// options is a map of compaction strategy options and their values. +// unchecked_options is an analogical map from which already checked options are deleted. +// This helps making sure that only allowed options are being set. +void incremental_compaction_strategy_options::validate(const std::map& options, std::map& unchecked_options) { + validate_min_sstable_size(options, unchecked_options); + auto bucket_low = validate_bucket_low(options, unchecked_options); + auto bucket_high = validate_bucket_high(options, unchecked_options); + if (bucket_high <= bucket_low) { + throw exceptions::configuration_exception(fmt::format("{} value ({}) is less than or equal to the {} " + "value ({})", BUCKET_HIGH_KEY, bucket_high, BUCKET_LOW_KEY, bucket_low)); + } + validate_fragment_size(options, unchecked_options); + validate_space_amplification_goal(options, unchecked_options); + compaction_strategy_impl::validate_min_max_threshold(options, unchecked_options); +} + +uint64_t incremental_compaction_strategy::avg_size(std::vector& runs) const { + uint64_t n = 0; + + if (runs.empty()) { + return 0; + } + for (auto& r : runs) { + n += r->data_size(); + } + return n / runs.size(); +} + +bool incremental_compaction_strategy::is_bucket_interesting(const std::vector& bucket, size_t min_threshold) { + return bucket.size() >= min_threshold; +} + +bool incremental_compaction_strategy::is_any_bucket_interesting(const std::vector>& buckets, size_t min_threshold) const { + return boost::algorithm::any_of(buckets, [&] (const std::vector& bucket) { + return this->is_bucket_interesting(bucket, min_threshold); + }); +} + +std::vector +incremental_compaction_strategy::create_run_and_length_pairs(const std::vector& runs) { + + std::vector run_length_pairs; + run_length_pairs.reserve(runs.size()); + + for(auto& r_ptr : runs) { + auto& r = *r_ptr; + assert(r.data_size() != 0); + run_length_pairs.emplace_back(r_ptr, r.data_size()); + } + + return run_length_pairs; +} + +std::vector> +incremental_compaction_strategy::get_buckets(const std::vector& runs, const incremental_compaction_strategy_options& options) { + auto sorted_runs = create_run_and_length_pairs(runs); + + std::sort(sorted_runs.begin(), sorted_runs.end(), [] (sstable_run_and_length& i, sstable_run_and_length& j) { + return i.second < j.second; + }); + + using bucket_type = std::vector; + std::vector bucket_list; + std::vector bucket_average_size_list; + + for (auto& pair : sorted_runs) { + size_t size = pair.second; + + // look for a bucket containing similar-sized runs: + // group in the same bucket if it's w/in (bucket_low, bucket_high) of the average for this bucket, + // or this file and the bucket are all considered "small" (less than `minSSTableSize`) + if (!bucket_list.empty()) { + auto& bucket_average_size = bucket_average_size_list.back(); + + if ((size > (bucket_average_size * options.bucket_low) && size < (bucket_average_size * options.bucket_high)) || + (size < options.min_sstable_size && bucket_average_size < options.min_sstable_size)) { + auto& bucket = bucket_list.back(); + auto total_size = bucket.size() * bucket_average_size; + auto new_average_size = (total_size + size) / (bucket.size() + 1); + auto smallest_run_in_bucket = bucket[0]->data_size(); + + // SSTables are added in increasing size order so the bucket's + // average might drift upwards. + // Don't let it drift too high, to a point where the smallest + // SSTable might fall out of range. + if (size < options.min_sstable_size || smallest_run_in_bucket > new_average_size * options.bucket_low) { + bucket.push_back(pair.first); + bucket_average_size = new_average_size; + continue; + } + } + } + + // no similar bucket found; put it in a new one + bucket_type new_bucket = {pair.first}; + bucket_list.push_back(std::move(new_bucket)); + bucket_average_size_list.push_back(size); + } + + return bucket_list; +} + +std::vector +incremental_compaction_strategy::most_interesting_bucket(std::vector> buckets, + size_t min_threshold, size_t max_threshold) +{ + std::vector interesting_buckets; + interesting_buckets.reserve(buckets.size()); + + for (auto& bucket : buckets) { + bucket.resize(std::min(bucket.size(), max_threshold)); + if (is_bucket_interesting(bucket, min_threshold)) { + auto avg = avg_size(bucket); + interesting_buckets.push_back({ std::move(bucket), avg }); + } + } + + if (interesting_buckets.empty()) { + return std::vector(); + } + // Pick the bucket with more elements, as efficiency of same-tier compactions increases with number of files. + auto& max = *std::max_element(interesting_buckets.begin(), interesting_buckets.end(), + [] (sstable_run_bucket_and_length& i, sstable_run_bucket_and_length& j) { + return i.first.size() < j.first.size(); + }); + return std::move(max.first); +} + +compaction_descriptor +incremental_compaction_strategy::find_garbage_collection_job(const compaction::table_state& t, std::vector& buckets) { + auto worth_dropping_tombstones = [this, &t, now = db_clock::now()] (const sstable_run& run, gc_clock::time_point compaction_time) { + if (run.all().empty()) { + return false; + } + // for the purpose of checking if a run is stale, picking any fragment *composing the same run* + // will be enough as the difference in write time is acceptable. + auto run_write_time = (*run.all().begin())->data_file_write_time(); + // FIXME: hack to avoid infinite loop, get rid of it once the root cause is fixed. + // Refs #3571. + auto min_gc_compaction_interval = std::min(db_clock::duration(std::chrono::seconds(3600)), _tombstone_compaction_interval); + if ((now - min_gc_compaction_interval) < run_write_time) { + return false; + } + if (_unchecked_tombstone_compaction) { + return true; + } + auto run_max_timestamp = std::ranges::max(run.all() | std::views::transform([] (const shared_sstable& sstable) { + return sstable->get_stats_metadata().max_timestamp; + })); + bool satisfy_staleness = (now - _tombstone_compaction_interval) > run_write_time; + // Staleness condition becomes mandatory if memtable's data is possibly shadowed by tombstones. + if (run_max_timestamp >= t.min_memtable_timestamp() && !satisfy_staleness) { + return false; + } + // If interval is not satisfied, we still consider tombstone GC if the gain outweighs the increased frequency. + // By increasing threshold to a minimum of 0.5, we're only adding a maximum of 1 to write amp as we'll be halving + // the SSTable, containing garbage, on every GC round. + float actual_threshold = satisfy_staleness ? _tombstone_threshold : std::clamp(_tombstone_threshold * 2, 0.5f, 1.0f); + + return run.estimate_droppable_tombstone_ratio(compaction_time, t.get_tombstone_gc_state(), t.schema()) >= actual_threshold; + }; + auto compaction_time = gc_clock::now(); + auto can_garbage_collect = [&] (const size_bucket_t& bucket) { + return boost::algorithm::any_of(bucket, [&] (const frozen_sstable_run& r) { + return worth_dropping_tombstones(*r, compaction_time); + }); + }; + + // To make sure that expired tombstones are persisted in a timely manner, ICS will cross-tier compact + // two closest-in-size buckets such that tombstones will eventually reach the top of the LSM tree, + // making it possible to purge them. + + // Start from the largest tier as it's more likely to satisfy conditions for tombstones to be purged. + auto it = buckets.rbegin(); + for (; it != buckets.rend(); it++) { + if (can_garbage_collect(*it)) { + break; + } + } + if (it == buckets.rend()) { + clogger.debug("ICS: nothing to garbage collect in {} buckets for {}.{}", buckets.size(), t.schema()->ks_name(), t.schema()->cf_name()); + return compaction_descriptor(); + } + + size_bucket_t& first_bucket = *it; + std::vector input = std::move(first_bucket); + + if (buckets.size() >= 2) { + // If the largest tier needs GC, then compact it with the second largest. + // Any smaller tier needing GC will be compacted with the larger and closest-in-size one. + // It's done this way to reduce write amplification and satisfy conditions for purging tombstones. + it = it == buckets.rbegin() ? std::next(it) : std::prev(it); + + size_bucket_t& second_bucket = *it; + + input.reserve(input.size() + second_bucket.size()); + std::move(second_bucket.begin(), second_bucket.end(), std::back_inserter(input)); + } + clogger.debug("ICS: starting garbage collection on {} runs for {}.{}", input.size(), t.schema()->ks_name(), t.schema()->cf_name()); + + return compaction_descriptor(runs_to_sstables(std::move(input)), 0, _fragment_size); +} + +compaction_descriptor +incremental_compaction_strategy::get_sstables_for_compaction(table_state& t, strategy_control& control) { + auto candidates = control.candidates_as_runs(t); + + // make local copies so they can't be changed out from under us mid-method + size_t min_threshold = t.min_compaction_threshold(); + size_t max_threshold = t.schema()->max_compaction_threshold(); + + auto buckets = get_buckets(candidates); + + if (is_any_bucket_interesting(buckets, min_threshold)) { + std::vector most_interesting = most_interesting_bucket(std::move(buckets), min_threshold, max_threshold); + return sstables::compaction_descriptor(runs_to_sstables(std::move(most_interesting)), 0, _fragment_size); + } + // If we are not enforcing min_threshold explicitly, try any pair of sstable runs in the same tier. + if (!t.compaction_enforce_min_threshold() && is_any_bucket_interesting(buckets, 2)) { + std::vector most_interesting = most_interesting_bucket(std::move(buckets), 2, max_threshold); + return sstables::compaction_descriptor(runs_to_sstables(std::move(most_interesting)), 0, _fragment_size); + } + + // The cross-tier behavior is only triggered once we're done with all the pending same-tier compaction to + // increase overall efficiency. + if (control.has_ongoing_compaction(t)) { + return sstables::compaction_descriptor(); + } + + auto desc = find_garbage_collection_job(t, buckets); + if (!desc.sstables.empty()) { + return desc; + } + + if (_space_amplification_goal) { + if (buckets.size() < 2) { + return sstables::compaction_descriptor(); + } + // Let S0 be the size of largest tier + // Let S1 be the size of second-largest tier, + // SA will be (S0 + S1) / S0 + + // Don't try SAG if there's an ongoing compaction, because if largest tier is being compacted, + // SA would be calculated incorrectly, which may result in an unneeded cross-tier compaction. + + auto find_two_largest_tiers = [this] (std::vector&& buckets) -> std::tuple { + std::partial_sort(buckets.begin(), buckets.begin()+2, buckets.end(), [this] (size_bucket_t& i, size_bucket_t& j) { + return avg_size(i) > avg_size(j); // descending order + }); + return { std::move(buckets[0]), std::move(buckets[1]) }; + }; + + auto total_size = [] (const size_bucket_t& bucket) -> uint64_t { + return boost::accumulate(bucket | boost::adaptors::transformed(std::mem_fn(&sstable_run::data_size)), uint64_t(0)); + }; + + auto [s0, s1] = find_two_largest_tiers(std::move(buckets)); + uint64_t s0_size = total_size(s0), s1_size = total_size(s1); + double space_amplification = double(s0_size + s1_size) / s0_size; + + if (space_amplification > _space_amplification_goal) { + clogger.debug("ICS: doing cross-tier compaction of two largest tiers, to reduce SA {} to below SAG {}", + space_amplification, *_space_amplification_goal); + // Aims at reducing space amplification, to below SAG, by compacting together the two largest tiers + std::vector cross_tier_input = std::move(s0); + cross_tier_input.reserve(cross_tier_input.size() + s1.size()); + std::move(s1.begin(), s1.end(), std::back_inserter(cross_tier_input)); + + return sstables::compaction_descriptor(runs_to_sstables(std::move(cross_tier_input)), + 0, _fragment_size); + } + } + + return sstables::compaction_descriptor(); +} + +compaction_descriptor +incremental_compaction_strategy::get_major_compaction_job(table_state& t, std::vector candidates) { + if (candidates.empty()) { + return compaction_descriptor(); + } + return make_major_compaction_job(std::move(candidates), 0, _fragment_size); +} + +int64_t incremental_compaction_strategy::estimated_pending_compactions(table_state& t) const { + size_t min_threshold = t.schema()->min_compaction_threshold(); + size_t max_threshold = t.schema()->max_compaction_threshold(); + int64_t n = 0; + + for (auto& bucket : get_buckets(t.main_sstable_set().all_sstable_runs())) { + if (bucket.size() >= min_threshold) { + n += (bucket.size() + max_threshold - 1) / max_threshold; + } + } + return n; +} + +std::vector +incremental_compaction_strategy::runs_to_sstables(std::vector runs) { + return boost::accumulate(runs, std::vector(), [&] (std::vector&& v, const frozen_sstable_run& run_ptr) { + auto& run = *run_ptr; + v.insert(v.end(), run.all().begin(), run.all().end()); + return std::move(v); + }); +} + +std::vector +incremental_compaction_strategy::sstables_to_runs(std::vector sstables) { + std::unordered_map runs; + for (auto&& sst : sstables) { + // okay to ignore duplicates + (void)runs[sst->run_identifier()].insert(std::move(sst)); + } + auto freeze = [] (const sstable_run& run) { return make_lw_shared(run); }; + return boost::copy_range>(runs | boost::adaptors::map_values | boost::adaptors::transformed(freeze)); +} + +void incremental_compaction_strategy::sort_run_bucket_by_first_key(size_bucket_t& bucket, size_t max_elements, const schema_ptr& schema) { + std::partial_sort(bucket.begin(), bucket.begin() + max_elements, bucket.end(), [&schema](const frozen_sstable_run& a, const frozen_sstable_run& b) { + auto sst_first_key_less = [&schema] (const shared_sstable& sst_a, const shared_sstable& sst_b) { + return sst_a->get_first_decorated_key().tri_compare(*schema, sst_b->get_first_decorated_key()) <= 0; + }; + auto& a_first = *boost::min_element(a->all(), sst_first_key_less); + auto& b_first = *boost::min_element(b->all(), sst_first_key_less); + return a_first->get_first_decorated_key().tri_compare(*schema, b_first->get_first_decorated_key()) <= 0; + }); +} + +compaction_descriptor +incremental_compaction_strategy::get_reshaping_job(std::vector input, schema_ptr schema, reshape_config cfg) const { + auto mode = cfg.mode; + size_t offstrategy_threshold = std::max(schema->min_compaction_threshold(), 4); + size_t max_sstables = std::max(schema->max_compaction_threshold(), int(offstrategy_threshold)); + + if (mode == reshape_mode::relaxed) { + offstrategy_threshold = max_sstables; + } + + auto run_count = boost::copy_range>(input | boost::adaptors::transformed(std::mem_fn(&sstable::run_identifier))).size(); + if (run_count >= offstrategy_threshold && mode == reshape_mode::strict) { + std::sort(input.begin(), input.end(), [&schema] (const shared_sstable& a, const shared_sstable& b) { + return dht::ring_position(a->get_first_decorated_key()).less_compare(*schema, dht::ring_position(b->get_first_decorated_key())); + }); + // All sstables can be reshaped at once if the amount of overlapping will not cause memory usage to be high, + // which is possible because partitioned set is able to incrementally open sstables during compaction + if (sstable_set_overlapping_count(schema, input) <= max_sstables) { + compaction_descriptor desc(std::move(input), 0/* level */, _fragment_size); + desc.options = compaction_type_options::make_reshape(); + return desc; + } + } + + for (auto& bucket : get_buckets(sstables_to_runs(std::move(input)))) { + if (bucket.size() >= offstrategy_threshold) { + // preserve token contiguity by prioritizing runs with the lowest first keys. + if (bucket.size() > max_sstables) { + sort_run_bucket_by_first_key(bucket, max_sstables, schema); + bucket.resize(max_sstables); + } + compaction_descriptor desc(runs_to_sstables(std::move(bucket)), 0/* level */, _fragment_size); + desc.options = compaction_type_options::make_reshape(); + return desc; + } + } + + return compaction_descriptor(); +} + +std::vector +incremental_compaction_strategy::get_cleanup_compaction_jobs(table_state& t, std::vector candidates) const { + std::vector ret; + const auto& schema = t.schema(); + unsigned max_threshold = schema->max_compaction_threshold(); + + for (auto& bucket : get_buckets(sstables_to_runs(std::move(candidates)))) { + if (bucket.size() > max_threshold) { + // preserve token contiguity + sort_run_bucket_by_first_key(bucket, bucket.size(), schema); + } + auto it = bucket.begin(); + while (it != bucket.end()) { + unsigned remaining = std::distance(it, bucket.end()); + unsigned needed = std::min(remaining, max_threshold); + std::vector runs; + std::move(it, it + needed, std::back_inserter(runs)); + ret.push_back(compaction_descriptor(runs_to_sstables(std::move(runs)), 0/* level */, _fragment_size)); + std::advance(it, needed); + } + } + return ret; +} + +std::unique_ptr +incremental_compaction_strategy::make_backlog_tracker() const { + return std::make_unique(_options); +} + +incremental_compaction_strategy::incremental_compaction_strategy(const std::map& options) + : compaction_strategy_impl(options) + , _options(options) +{ + auto fragment_size_in_mb = validate_fragment_size(options); + _fragment_size = fragment_size_in_mb*1024*1024; + _space_amplification_goal = validate_space_amplification_goal(options); +} + +// options is a map of compaction strategy options and their values. +// unchecked_options is an analogical map from which already checked options are deleted. +// This helps making sure that only allowed options are being set. +void incremental_compaction_strategy::validate_options(const std::map& options, std::map& unchecked_options) { + incremental_compaction_strategy_options::validate(options, unchecked_options); +} + +} diff --git a/compaction/incremental_compaction_strategy.hh b/compaction/incremental_compaction_strategy.hh new file mode 100644 index 000000000000..16a9bf91fee9 --- /dev/null +++ b/compaction/incremental_compaction_strategy.hh @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2019 ScyllaDB + * + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "compaction_strategy_impl.hh" +#include "compaction.hh" +#include +#include +#include +#include +#include "size_tiered_compaction_strategy.hh" + +class incremental_backlog_tracker; + +namespace sstables { + +class incremental_compaction_strategy_options { +public: + static constexpr uint64_t DEFAULT_MIN_SSTABLE_SIZE = 50L * 1024L * 1024L; + static constexpr double DEFAULT_BUCKET_LOW = 0.5; + static constexpr double DEFAULT_BUCKET_HIGH = 1.5; + + static constexpr auto MIN_SSTABLE_SIZE_KEY = "min_sstable_size"; + static constexpr auto BUCKET_LOW_KEY = "bucket_low"; + static constexpr auto BUCKET_HIGH_KEY = "bucket_high"; +private: + uint64_t min_sstable_size = DEFAULT_MIN_SSTABLE_SIZE; + double bucket_low = DEFAULT_BUCKET_LOW; + double bucket_high = DEFAULT_BUCKET_HIGH; +public: + incremental_compaction_strategy_options(const std::map& options); + + incremental_compaction_strategy_options() { + min_sstable_size = DEFAULT_MIN_SSTABLE_SIZE; + bucket_low = DEFAULT_BUCKET_LOW; + bucket_high = DEFAULT_BUCKET_HIGH; + } + + static void validate(const std::map& options, std::map& unchecked_options); + + friend class incremental_compaction_strategy; +}; + +using sstable_run_and_length = std::pair; +using sstable_run_bucket_and_length = std::pair, uint64_t>; + +class incremental_compaction_strategy : public compaction_strategy_impl { + incremental_compaction_strategy_options _options; + + using size_bucket_t = std::vector; +public: + static constexpr int32_t DEFAULT_MAX_FRAGMENT_SIZE_IN_MB = 1000; + static constexpr auto FRAGMENT_SIZE_OPTION = "sstable_size_in_mb"; + static constexpr auto SPACE_AMPLIFICATION_GOAL_OPTION = "space_amplification_goal"; +private: + size_t _fragment_size = DEFAULT_MAX_FRAGMENT_SIZE_IN_MB*1024*1024; + std::optional _space_amplification_goal; + static std::vector create_run_and_length_pairs(const std::vector& runs); + + static std::vector> get_buckets(const std::vector& runs, const incremental_compaction_strategy_options& options); + + std::vector> get_buckets(const std::vector& runs) const { + return get_buckets(runs, _options); + } + + std::vector + most_interesting_bucket(std::vector> buckets, size_t min_threshold, size_t max_threshold); + + uint64_t avg_size(std::vector& runs) const; + + static bool is_bucket_interesting(const std::vector& bucket, size_t min_threshold); + + bool is_any_bucket_interesting(const std::vector>& buckets, size_t min_threshold) const; + + compaction_descriptor find_garbage_collection_job(const table_state& t, std::vector& buckets); + + static std::vector runs_to_sstables(std::vector runs); + static std::vector sstables_to_runs(std::vector sstables); + static void sort_run_bucket_by_first_key(size_bucket_t& bucket, size_t max_elements, const schema_ptr& schema); +public: + incremental_compaction_strategy() = default; + + incremental_compaction_strategy(const std::map& options); + + static void validate_options(const std::map& options, std::map& unchecked_options); + + virtual compaction_descriptor get_sstables_for_compaction(table_state& t, strategy_control& control) override; + + virtual std::vector get_cleanup_compaction_jobs(table_state& t, std::vector candidates) const override; + + virtual compaction_descriptor get_major_compaction_job(table_state& t, std::vector candidates) override; + + virtual int64_t estimated_pending_compactions(table_state& t) const override; + + virtual compaction_strategy_type type() const override { + return compaction_strategy_type::incremental; + } + + virtual std::unique_ptr make_backlog_tracker() const override; + + virtual compaction_descriptor get_reshaping_job(std::vector input, schema_ptr schema, reshape_config cfg) const override; + + virtual std::unique_ptr make_sstable_set(schema_ptr schema) const override; + + friend class ::incremental_backlog_tracker; +}; + +} diff --git a/configure.py b/configure.py index c89cd6c4ce62..03fe64ba8e54 100755 --- a/configure.py +++ b/configure.py @@ -491,6 +491,7 @@ def find_ninja(): 'test/boost/hashers_test', 'test/boost/hint_test', 'test/boost/idl_test', + 'test/boost/incremental_compaction_test', 'test/boost/index_reader_test', 'test/boost/input_stream_test', 'test/boost/intrusive_array_test', @@ -833,6 +834,8 @@ def find_ninja(): 'compaction/task_manager_module.cc', 'compaction/time_window_compaction_strategy.cc', 'compaction/compaction_manager.cc', + 'compaction/incremental_compaction_strategy.cc', + 'compaction/incremental_backlog_tracker.cc', 'sstables/integrity_checked_file_impl.cc', 'sstables/prepended_input_stream.cc', 'sstables/m_format_read_helpers.cc', diff --git a/db/legacy_schema_migrator.cc b/db/legacy_schema_migrator.cc index d3cbc8ac6576..33e7d8f948e0 100644 --- a/db/legacy_schema_migrator.cc +++ b/db/legacy_schema_migrator.cc @@ -385,9 +385,9 @@ class migrator { try { builder.set_compaction_strategy(sstables::compaction_strategy::type(strategy)); } catch (const exceptions::configuration_exception& e) { - // If compaction strategy class isn't supported, fallback to size tiered. - mlogger.warn("Falling back to size-tiered compaction strategy after the problem: {}", e.what()); - builder.set_compaction_strategy(sstables::compaction_strategy_type::size_tiered); + // If compaction strategy class isn't supported, fallback to incremental. + mlogger.warn("Falling back to incremental compaction strategy after the problem: {}", e.what()); + builder.set_compaction_strategy(sstables::compaction_strategy_type::incremental); } } if (td.has("compaction_strategy_options")) { diff --git a/db/schema_tables.cc b/db/schema_tables.cc index b7ab716466f4..fcb87626be85 100644 --- a/db/schema_tables.cc +++ b/db/schema_tables.cc @@ -2056,9 +2056,9 @@ static void prepare_builder_from_table_row(const schema_ctxt& ctxt, schema_build builder.set_compaction_strategy(sstables::compaction_strategy::type(i->second)); map.erase(i); } catch (const exceptions::configuration_exception& e) { - // If compaction strategy class isn't supported, fallback to size tiered. - slogger.warn("Falling back to size-tiered compaction strategy after the problem: {}", e.what()); - builder.set_compaction_strategy(sstables::compaction_strategy_type::size_tiered); + // If compaction strategy class isn't supported, fallback to incremental. + slogger.warn("Falling back to incremental compaction strategy after the problem: {}", e.what()); + builder.set_compaction_strategy(sstables::compaction_strategy_type::incremental); } } if (map.contains("max_threshold")) { diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 1304601472d8..59c2b2ad73a1 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -808,7 +808,7 @@ schema_ptr system_keyspace::v3::batches() { // FIXME: the original Java code also had: //.copy(new LocalPartitioner(TimeUUIDType.instance)) builder.set_gc_grace_seconds(0); - builder.set_compaction_strategy(sstables::compaction_strategy_type::size_tiered); + builder.set_compaction_strategy(sstables::compaction_strategy_type::incremental); builder.set_compaction_strategy_options({{"min_threshold", "2"}}); builder.with_hash_version(); return builder.build(schema_builder::compact_storage::no); @@ -1292,7 +1292,7 @@ schema_ptr system_keyspace::legacy::hints() { "*DEPRECATED* hints awaiting delivery" ); builder.set_gc_grace_seconds(0); - builder.set_compaction_strategy(sstables::compaction_strategy_type::size_tiered); + builder.set_compaction_strategy(sstables::compaction_strategy_type::incremental); builder.set_compaction_strategy_options({{"enabled", "false"}}); builder.with(schema_builder::compact_storage::yes); builder.with_hash_version(); @@ -1318,7 +1318,7 @@ schema_ptr system_keyspace::legacy::batchlog() { "*DEPRECATED* batchlog entries" ); builder.set_gc_grace_seconds(0); - builder.set_compaction_strategy(sstables::compaction_strategy_type::size_tiered); + builder.set_compaction_strategy(sstables::compaction_strategy_type::incremental); builder.set_compaction_strategy_options({{"min_threshold", "2"}}); builder.with(schema_builder::compact_storage::no); builder.with_hash_version(); diff --git a/docs/architecture/compaction/compaction-strategies.rst b/docs/architecture/compaction/compaction-strategies.rst index 5f05c88b797d..347ea9f4453a 100644 --- a/docs/architecture/compaction/compaction-strategies.rst +++ b/docs/architecture/compaction/compaction-strategies.rst @@ -70,7 +70,36 @@ Set the parameters for :ref:`Leveled Compaction `_ for details. +.. versionadded:: 2019.1.4 Scylla Enterprise + +ICS principles of operation are similar to those of STCS, merely replacing the increasingly larger SSTables in each tier, by increasingly longer SSTable runs, modeled after LCS runs, but using larger fragment size of 1 GB, by default. + +Compaction is triggered when there are two or more runs of roughly the same size. These runs are incrementally compacted with each other, producing a new SSTable run, while incrementally releasing space as soon as each SSTable in the input run is processed and compacted. This method eliminates the high temporary space amplification problem of STCS by limiting the overhead to twice the (constant) fragment size, per shard. + +Incremental Compaction Strategy benefits +---------------------------------------- +* Greatly reduces the temporary space amplification which is typical of STCS, resulting in more disk space being available for storing user data. +* The space requirement for a major compaction with ICS is almost non-existent given that the operation can release fragments at roughly same rate it produces new ones. + +If you look at the following screenshot the green line shows how disk usage behaves under ICS when major compaction is issued. + +.. image:: /architecture/compaction/screenshot.png + +Incremental Compaction Strategy disadvantages +---------------------------------------------- + +* Since ICS principles of operation are the same as STCS, its disadvantages are similar to STCS's, except for the temporary space amplification issue. + +Namely: + +* Continuously modifying existing rows results in each row being split across several SSTables, making reads slow, which doesn’t happen in Leveled compaction. +* Obsolete data (overwritten or deleted columns) may accumulate across tiers, wasting space, for a long time, until it is finally merged. This can be mitigated by running major compaction from time to time. + +**To implement this strategy** + +Set the parameters for :ref:`Incremental Compaction `. + +For more information, see the :ref:`Compaction KB Article `. .. _TWCS1: diff --git a/docs/cql/compaction.rst b/docs/cql/compaction.rst index 2de7686e71c4..3407220abb35 100644 --- a/docs/cql/compaction.rst +++ b/docs/cql/compaction.rst @@ -170,7 +170,104 @@ LCS options Incremental Compaction Strategy (ICS) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -ICS is only available in ScyllaDB Enterprise. See the `ScyllaDB Enetrpise documentation `_ for details. +.. versionadded:: 2019.1.4 Scylla Enterprise + +When using ICS, SSTable runs are put in different buckets depending on their size. +When an SSTable run is bucketed, the average size of the runs in the bucket is compared to the new run, as well as the ``bucket_high`` and ``bucket_low`` levels. + + +The database compares each SSTable-run size to the average of all SSTable-run sizes on all buckets in the node. +It calculates ``bucket_low * avg_bucket_size`` and ``bucket_high * avg_bucket_size`` and then compares the result with the ``average SSTable-run size``. +The conditions set for ``bucket_high`` and ``bucket_low`` dictate if successive runs will be added to the same bucket. +When compaction begins it merges SSTable runs whose size in KB are within ``[average-size * bucket_low]`` and ``[average-size * bucket_high]``. + + +Once there are multiple runs in a bucket, minor compaction begins. +The minimum number of SSTable runs that triggers minor compaction is either 2 or ``min_threshold``, if the ``compaction_enforce_min_threshold`` +configuration option is set in the scylla.yaml configuration file. + +.. _ics-options: + +ICS options +~~~~~~~~~~~~ + +The following options only apply to IncrementalCompactionStrategy: + +.. code-block:: cql + + compaction = { + 'class' : 'IncrementalCompactionStrategy', + 'bucket_high' : factor, + 'bucket_low' : factor, + 'min_sstable_size' : int, + 'min_threshold' : num_sstables, + 'max_threshold' : num_sstables, + 'sstable_size_in_mb' : int, + 'space_amplification_goal' : double} + +===== + +``bucket_high`` (default: 1.5) + A new SSTable is added to the bucket if the SSTable size is **less than** + bucket_high * the average size of that bucket (and if the bucket_low condition also holds). + For example, if **'bucket_high = 1.5'** and the **SSTable size = 14MB**, does the SSTable belong to a bucket with an average size of 10MB? + Yes, because the **SSTable size = 14**, which is **less** than **'bucket_high' * average bucket size = 15**. + So, the SSTable will be added to the bucket, and the bucket’s average size will be recalculated. + +===== + +``bucket_low`` (default: 0.5) + A new SSTable is added to the bucket if the SSTable size is **more than** + bucket_low * the average size of that bucket (and if the bucket_high condition also holds). + For example, if **'bucket_high = 0.5'** and the **SSTable size is 10MB**, does the SSTable belong to a bucket with an average size of 15MB? + Yes, because the **SSTable size = 10**, which is **more** than **'bucket_low' * average bucket size = 7.5**. + So, the SSTable will be added to the bucket, and the bucket’s average size will be recalculated. + +===== + +``min_sstable_size`` (default: 50) + All SSTables smaller than this number of megabytes are put into the same bucket. + + Unlike Apache Cassandra, scylla uses **uncompressed** size when bucketing similar-sized tiers together. + Since compaction works on uncompressed data, SSTables containing similar amounts of data should be compacted together, even when they have different compression ratios. + +===== + +``min_threshold`` (default: 4) + Minimum number of SSTable runs that need to belong to the same size bucket before compaction is triggered on that bucket. + + .. note:: Enforcement of ``min_threshold`` is controlled by the ``compaction_enforce_min_threshold`` configuration option in the scylla.yaml configuration settings. + By default, ``compaction_enforce_min_threshold=false``, meaning the Incremental Compaction Strategy will compact any bucket containing 2 or more SSTable runs. + Otherwise, if ``compaction_enforce_min_threshold=true``, the value of ``min_threshold`` is considered and only those buckets that contain at + least ``min_threshold`` SSTable runs will be compacted. + +===== + +``max_threshold`` (default: 32) + Maximum number of SSTables that will be compacted together in one compaction step. + +===== + +``sstable_size_in_mb`` (default: 1000) + This is the target size in megabytes, that will be used as the goal for an SSTable size (fragment size) following a compression. + +.. _SAG: + +===== + +``space_amplification_goal`` (default: null) + +:label-tip:`ScyllaDB Enterprise` + + .. versionadded:: 2020.1.6 + + This is a threshold of the ratio of the sum of the sizes of the two largest tiers to the size of the largest tier, + above which ICS will automatically compact the second largest and largest tiers together to eliminate stale data that may have been overwritten, expired, or deleted. + The space_amplification_goal is given as a double-precision floating point number that must be greater than 1.0. + + For example, if **'space_amplification_goal = 1.25'** and the largest tier holds **1000GB**, + when the second-largest tier accumulates SSTables with the total size of 250GB or more, + the ``space_amplification_goal`` threshold is crossed and all the SSTables in the largest and second-largest tiers will be compacted together. ===== diff --git a/docs/cql/ddl.rst b/docs/cql/ddl.rst index 0307d3e1d69a..fd53ea405e2e 100644 --- a/docs/cql/ddl.rst +++ b/docs/cql/ddl.rst @@ -756,7 +756,7 @@ Custom strategy can be provided by specifying the full class name as a :ref:`str `. All default strategies support a number of common options, as well as options specific to -the strategy chosen (see the section corresponding to your strategy for details: :ref:`STCS `, :ref:`LCS `, and :ref:`TWCS `). +the strategy chosen (see the section corresponding to your strategy for details: :ref:`STCS `, :ref:`LCS `, :ref:`ICS `, and :ref:`TWCS `). .. _cql-compression-options: diff --git a/docs/getting-started/system-requirements.rst b/docs/getting-started/system-requirements.rst index a0498551d047..30c2a00a050c 100644 --- a/docs/getting-started/system-requirements.rst +++ b/docs/getting-started/system-requirements.rst @@ -99,7 +99,7 @@ Time-window Compaction Strategy (TWCS) 50% 70% Incremental Compaction Strategy (ICS) 70% 80% ====================================== =========== ============ -Use the default ICS unless you'll have a clear understanding that another strategy is better for your use case. More on :doc:`choosing a Compaction Strategy `. +Use the default ICS unless you'll have a clear understanding that another strategy is better for your use case. More on :doc:`choosing a Compaction Strategy `. In order to maintain a high level of service availability, keep 50% to 20% free disk space at all times! .. _system-requirements-network: diff --git a/docs/kb/compaction.rst b/docs/kb/compaction.rst index d641b6701dcf..c14792efcae9 100644 --- a/docs/kb/compaction.rst +++ b/docs/kb/compaction.rst @@ -117,7 +117,50 @@ Likewise, when :term:`bootstrapping` a new node, SSTables are streame Incremental Compaction Strategy (ICS) :label-tip:`ScyllaDB Enterprise` ------------------------------------------------------------------------ -ICS is only available in ScyllaDB Enterprise. See the `ScyllaDB Enetrpise documentation `_ for details. + +.. versionadded:: 2019.1.4 + +One of the issues with Size-tiered compaction is that it needs temporary space because SSTables are not removed until they are fully compacted. ICS takes a different approach and splits each large SSTable into a run of sorted, fixed-size (by default 1 GB) SSTables (a.k.a. fragments) in the same way that LCS does, except it treats the entire run and not the individual SSTables as the sizing file for STCS. As the run-fragments are small, the SSTables compact quickly, allowing individual SSTables to be removed as soon as they are compacted. This approach uses low amounts of memory and temporary disk space. + +ICS uses the same philosophy as STCS, where the SSTables are sorted in buckets according to their size. However, unlike STCS, ICS compaction uses SSTable runs as input, and produces a new run as output. It doesn't matter if a run is composed of only one fragment that could have come from STCS migration. From an incremental compaction perspective, everything is a run. + +The strategy works as follows: + +#. ICS looks for candidates for compaction that are similar in size. These candidates are called ``Input Runs``. + + * The input runs may contain one or more SSTables each. + +#. ICS compacts two or more similar-sized input runs into a single ``Output run`` (* See note_ ). +#. Incremental Compaction progressively works on two or more fragments at a time, one from each input run. + + * It reads mutations from all input fragments and merges them together into a single output fragment. + * As long as the resulting fragment is smaller than the ``sstable_size_in_mb``, no further action is needed. + * If the fragment is larger than the ``sstable_size_in_mb``: + + 1. Stop when the size threshold is reached, and seal the output fragment. + 2. Create a new run fragment and continue compacting the remaining input fragments, until the size threshold is reached. + 3. When an input fragment is exhausted, take it out of the list of SSTables to compact, and delete it from disk. + 4. Repeat until there are no input fragments left. + +#. Take all of the output fragments and feed them back into compaction as an SSTable run. +#. Stop when all fragments from input runs were exhausted and released. + +.. _note: +.. note:: To prevent data resurrection in case scylla crashes in the middle of compaction, ICS may possibly write an auxiliary run containing purgeable tombstones in addition to the output run containing live data. + These tombstones are kept on disk while there are SSTables containing data that the tombstones may shadow. Once compaction is done, deleting all shadowed data from all SSTables, the purgeable tombstones are purged and the SSTables holding them are removed from storage. + +.. image:: ics-incremental-compaction.png + +Incremental compaction as a solution for temporary space overhead in STCS +......................................................................... + +We fixed the temporary space overhead on STCS by applying the incremental compaction approach to it, which resulted in the creation of Incremental Compaction Strategy (ICS). The compacted SSTables, that become increasingly larger over time with STCS, are replaced with sorted runs of SSTable fragments, together called “SSTable runs” – which is a concept borrowed from Leveled Compaction Strategy (LCS). + +Each fragment is a roughly fixed size (aligned to partition boundaries) SSTable and it holds a unique range of keys, a portion of the whole SSTable run. Note that as the SSTable-runs in ICS hold exactly the same data as the corresponding SSTables created by STCS, they become increasingly longer over time (holding more fragments), in the same way that SSTables grow in size with STCS, yet the ICS SSTable fragments’ size remains the same. + +For example, when compacting two SSTables (or SSTable runs) holding 7GB each: instead of writing up to 14GB into a single SSTable file, we’ll break the output SSTable into a run of 14 x 1GB fragments (fragment size is 1GB by default). + +.. image:: compaction-incremental.png .. _time-window-compactionstrategy-twcs: diff --git a/docs/kb/index.rst b/docs/kb/index.rst index 980102e09391..77acbccc8fbb 100644 --- a/docs/kb/index.rst +++ b/docs/kb/index.rst @@ -41,6 +41,7 @@ Knowledge Base * :doc:`Increase Cache to Avoid Non-paged Queries ` - How to increase the ``permissions_cache_max_entries`` setting. * :doc:`How to Safely Increase the Replication Factor ` * :doc:`Facts about TTL, Compaction, and gc_grace_seconds ` + * :doc:`Efficient Tombstone Garbage Collection in ICS ` **Note**: The KB article for social readers has been *removed*. Instead, please look at lessons on `ScyllaDB University `_ or the `Care Pet example `_ diff --git a/docs/operating-scylla/procedures/tips/production-readiness.rst b/docs/operating-scylla/procedures/tips/production-readiness.rst index 823aeb6c5e73..1e1f93250c33 100644 --- a/docs/operating-scylla/procedures/tips/production-readiness.rst +++ b/docs/operating-scylla/procedures/tips/production-readiness.rst @@ -22,6 +22,17 @@ Choose a Compaction Strategy Each workload may require a specific strategy. Refer to :doc:`Choose a Compaction Strategy ` for details. +Incremental Compaction Strategy (ICS) +..................................... + +We highly recommend using ICS (the default setting) for any table that you have. +You will have much less Space Amplification with ICS as it only requires 25% additional storage, as opposed to STCS which requires 50% more. + +.. note:: ICS is the default compaction strategy setting for Scylla Enterprise versions 2020.1 and higher. + +* Refer to :ref:`Incremental Compaction Strategy ` for an overview of the benefits. +* Refer to :ref:`Incremental Compaction Strategy Overview ` for a description of how it works. + Resiliency ---------- diff --git a/schema/schema.hh b/schema/schema.hh index 05da1d6616c2..76f2d1918d63 100644 --- a/schema/schema.hh +++ b/schema/schema.hh @@ -551,7 +551,7 @@ private: int32_t _memtable_flush_period = 0; ::speculative_retry _speculative_retry = ::speculative_retry(speculative_retry::type::PERCENTILE, 0.99); // This is the compaction strategy that will be used by default on tables which don't have one explicitly specified. - sstables::compaction_strategy_type _compaction_strategy = sstables::compaction_strategy_type::size_tiered; + sstables::compaction_strategy_type _compaction_strategy = sstables::compaction_strategy_type::incremental; std::map _compaction_strategy_options; bool _compaction_enabled = true; ::caching_options _caching_options; diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index c6f9290a7c92..481518803008 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -114,6 +114,8 @@ add_scylla_test(hint_test add_scylla_test(idl_test KIND BOOST LIBRARIES idl) +add_scylla_test(incremental_compaction_test + KIND SEASTAR) add_scylla_test(index_reader_test KIND SEASTAR) add_scylla_test(input_stream_test diff --git a/test/boost/incremental_compaction_test.cc b/test/boost/incremental_compaction_test.cc new file mode 100644 index 000000000000..0a79d5fa1e9b --- /dev/null +++ b/test/boost/incremental_compaction_test.cc @@ -0,0 +1,525 @@ +/* + * Copyright (C) 2019 ScyllaDB + * + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include "sstables/sstables.hh" +#include "compaction/incremental_compaction_strategy.hh" +#include "schema/schema.hh" +#include "replica/database.hh" +#include "compaction/compaction_manager.hh" +#include "sstable_test.hh" +#include "sstables/metadata_collector.hh" +#include "test/lib/tmpdir.hh" +#include "cell_locking.hh" +#include "test/lib/mutation_reader_assertions.hh" +#include "test/lib/key_utils.hh" +#include "service/storage_proxy.hh" +#include "test/lib/sstable_run_based_compaction_strategy_for_tests.hh" +#include "dht/i_partitioner.hh" +#include "dht/murmur3_partitioner.hh" +#include "db/large_data_handler.hh" +#include "db/config.hh" + +#include "test/lib/sstable_utils.hh" +#include "test/lib/test_services.hh" + +using namespace sstables; + +static mutation_reader sstable_reader(reader_permit permit, shared_sstable sst, schema_ptr s) { + return sst->as_mutation_source().make_reader_v2(s, std::move(permit), query::full_partition_range, s->full_slice()); + +} + +class strategy_control_for_test : public strategy_control { + bool _has_ongoing_compaction; +public: + explicit strategy_control_for_test(bool has_ongoing_compaction) noexcept : _has_ongoing_compaction(has_ongoing_compaction) {} + + bool has_ongoing_compaction(table_state& table_s) const noexcept override { + return _has_ongoing_compaction; + } + virtual std::vector candidates(table_state& t) const override { + return boost::copy_range>(*t.main_sstable_set().all()); + } + virtual std::vector candidates_as_runs(table_state& t) const override { + return t.main_sstable_set().all_sstable_runs(); + } +}; + +static std::unique_ptr make_strategy_control_for_test(bool has_ongoing_compaction) { + return std::make_unique(has_ongoing_compaction); +} + +SEASTAR_TEST_CASE(incremental_compaction_test) { + return sstables::test_env::do_with_async([&] (sstables::test_env& env) { + auto builder = schema_builder("tests", "incremental_compaction_test") + .with_column("id", utf8_type, column_kind::partition_key) + .with_column("value", int32_type) + .with_partitioner("org.apache.cassandra.dht.Murmur3Partitioner") + .with_sharder(smp::count, 0); + auto s = builder.build(); + + auto tmp = make_lw_shared(); + auto sst_gen = [&env, s, tmp] () mutable { + auto sst = env.make_sstable(s, tmp->path().string(), env.new_generation(), sstable_version_types::md, big); + return sst; + }; + + table_for_tests cf = env.make_table_for_tests(s, tmp->path().string()); + auto close_cf = deferred_stop(cf); + cf->set_compaction_strategy(sstables::compaction_strategy_type::size_tiered); + auto compact = [&, s] (std::vector all, auto replacer) -> std::vector { + auto desc = sstables::compaction_descriptor(std::move(all), 1, 0); + desc.enable_garbage_collection(cf->get_sstable_set()); + return compact_sstables(env, std::move(desc), cf, sst_gen, replacer).get().new_sstables; + }; + auto make_insert = [&] (auto p) { + auto key = p.key(); + mutation m(s, key); + m.set_clustered_cell(clustering_key::make_empty(), bytes("value"), data_value(int32_t(1)), 1 /* ts */); + BOOST_REQUIRE(m.decorated_key().token() == p.token()); + return m; + }; + + auto tokens = tests::generate_partition_keys(16, s, local_shard_only::yes, tests::key_size{8, 8}); + std::unordered_set sstables; + std::vector> observers; + sstables::sstable_run_based_compaction_strategy_for_tests cs; + + auto do_replace = [&] (const std::vector& old_sstables, const std::vector& new_sstables) { + for (auto& old_sst : old_sstables) { + BOOST_REQUIRE(sstables.count(old_sst)); + sstables.erase(old_sst); + } + for (auto& new_sst : new_sstables) { + BOOST_REQUIRE(!sstables.count(new_sst)); + sstables.insert(new_sst); + } + column_family_test(cf).rebuild_sstable_list(cf.as_table_state(), new_sstables, old_sstables).get(); + env.test_compaction_manager().propagate_replacement(cf.as_table_state(), old_sstables, new_sstables); + }; + + auto do_incremental_replace = [&] (auto old_sstables, auto new_sstables, auto& expected_sst, auto& closed_sstables_tracker) { + // that's because each sstable will contain only 1 mutation. + BOOST_REQUIRE(old_sstables.size() == 1); + BOOST_REQUIRE(new_sstables.size() == 1); + auto old_sstable = old_sstables.front(); + // check that sstable replacement follows token order + BOOST_REQUIRE(*expected_sst == old_sstable->generation()); + expected_sst++; + // check that previously released sstables were already closed + BOOST_REQUIRE(*closed_sstables_tracker == old_sstable->generation()); + + do_replace(old_sstables, new_sstables); + + observers.push_back(old_sstable->add_on_closed_handler([&] (sstable& sst) { + BOOST_TEST_MESSAGE(fmt::format("Closing sstable of generation {}", sst.generation())); + closed_sstables_tracker++; + })); + + BOOST_TEST_MESSAGE(fmt::format("Removing sstable of generation {}, refcnt: {}", old_sstables.front()->generation(), old_sstables.front().use_count())); + }; + + auto do_compaction = [&] (size_t expected_input, size_t expected_output) -> std::vector { + auto control = make_strategy_control_for_test(false); + auto desc = cs.get_sstables_for_compaction(cf.as_table_state(), *control); + + // nothing to compact, move on. + if (desc.sstables.empty()) { + return {}; + } + std::unordered_set run_ids; + bool incremental_enabled = std::any_of(desc.sstables.begin(), desc.sstables.end(), [&run_ids] (shared_sstable& sst) { + return !run_ids.insert(sst->run_identifier()).second; + }); + + BOOST_REQUIRE(desc.sstables.size() == expected_input); + auto sstable_run = boost::copy_range>(desc.sstables + | boost::adaptors::transformed([] (auto& sst) { return sst->generation(); })); + auto expected_sst = sstable_run.begin(); + auto closed_sstables_tracker = sstable_run.begin(); + auto replacer = [&] (compaction_completion_desc ccd) { + BOOST_REQUIRE(expected_sst != sstable_run.end()); + if (incremental_enabled) { + do_incremental_replace(std::move(ccd.old_sstables), std::move(ccd.new_sstables), expected_sst, closed_sstables_tracker); + } else { + do_replace(std::move(ccd.old_sstables), std::move(ccd.new_sstables)); + expected_sst = sstable_run.end(); + } + }; + + auto result = compact(std::move(desc.sstables), replacer); + + BOOST_REQUIRE_EQUAL(expected_output, result.size()); + BOOST_REQUIRE(expected_sst == sstable_run.end()); + return result; + }; + + // Generate 4 sstable runs composed of 4 fragments each after 4 compactions. + // All fragments non-overlapping. + for (auto i = 0U; i < tokens.size(); i++) { + auto sst = make_sstable_containing(sst_gen, { make_insert(tokens[i]) }); + sst->set_sstable_level(1); + BOOST_REQUIRE(sst->get_sstable_level() == 1); + column_family_test(cf).add_sstable(sst).get(); + sstables.insert(std::move(sst)); + do_compaction(4, 4); + } + BOOST_REQUIRE(sstables.size() == 16); + + // Generate 1 sstable run from 4 sstables runs of similar size + auto result = do_compaction(16, 16); + BOOST_REQUIRE(result.size() == 16); + for (auto i = 0U; i < tokens.size(); i++) { + assert_that(sstable_reader(env.semaphore().make_tracking_only_permit(s, "test reader", db::no_timeout, tracing::trace_state_ptr()), result[i], s)) + .produces(make_insert(tokens[i])) + .produces_end_of_stream(); + } + }); +} + +SEASTAR_THREAD_TEST_CASE(incremental_compaction_sag_test) { + auto builder = schema_builder("tests", "incremental_compaction_test") + .with_column("id", utf8_type, column_kind::partition_key) + .with_column("value", int32_type); + auto s = builder.build(); + + struct sag_test { + test_env& _env; + mutable table_for_tests _cf; + incremental_compaction_strategy _ics; + const unsigned min_threshold = 4; + const size_t data_set_size = 1'000'000'000; + + static incremental_compaction_strategy make_ics(double space_amplification_goal) { + std::map options; + options.emplace(sstring("space_amplification_goal"), sstring(std::to_string(space_amplification_goal))); + return incremental_compaction_strategy(options); + } + static replica::column_family::config make_table_config(test_env& env) { + auto config = env.make_table_config(); + config.compaction_enforce_min_threshold = true; + return config; + } + + sag_test(test_env& env, schema_ptr s, double space_amplification_goal) + : _env(env) + , _cf(env.make_table_for_tests(s)) + , _ics(make_ics(space_amplification_goal)) + { + } + + double space_amplification() const { + auto sstables = _cf->get_sstables(); + auto total = boost::accumulate(*sstables | boost::adaptors::transformed(std::mem_fn(&sstable::data_size)), uint64_t(0)); + return double(total) / data_set_size; + } + + shared_sstable make_sstable_with_size(size_t sstable_data_size) { + auto sst = _env.make_sstable(_cf->schema(), "/nowhere/in/particular", _env.new_generation(), sstable_version_types::md, big); + auto keys = tests::generate_partition_keys(2, _cf->schema(), local_shard_only::yes); + sstables::test(sst).set_values(keys[0].key(), keys[1].key(), stats_metadata{}, sstable_data_size); + return sst; + } + + void populate(double target_space_amplification) { + auto add_sstable = [this] (unsigned sst_data_size) { + auto sst = make_sstable_with_size(sst_data_size); + column_family_test(_cf).add_sstable(sst).get(); + }; + + add_sstable(data_set_size); + while (space_amplification() < target_space_amplification) { + add_sstable(data_set_size / min_threshold); + } + } + + void run() { + auto& table_s = _cf.as_table_state(); + auto control = make_strategy_control_for_test(false); + for (;;) { + auto desc = _ics.get_sstables_for_compaction(table_s, *control); + // no more jobs, bailing out... + if (desc.sstables.empty()) { + break; + } + auto total = boost::accumulate(desc.sstables | boost::adaptors::transformed(std::mem_fn(&sstable::data_size)), uint64_t(0)); + std::vector new_ssts = { make_sstable_with_size(std::min(total, data_set_size)) }; + column_family_test(_cf).rebuild_sstable_list(table_s, new_ssts, desc.sstables).get(); + } + } + + future<> stop() { + return _cf.stop(); + } + }; + + using SAG = double; + using TABLE_INITIAL_SA = double; + + auto with_sag_test = [&] (SAG sag, TABLE_INITIAL_SA initial_sa) { + test_env::do_with_async([&] (test_env& env) { + sag_test test(env, s, sag); + test.populate(initial_sa); + BOOST_REQUIRE(test.space_amplification() >= initial_sa); + test.run(); + BOOST_REQUIRE(test.space_amplification() <= sag); + test.stop().get(); + }).get(); + }; + + with_sag_test(SAG(1.25), TABLE_INITIAL_SA(1.5)); + with_sag_test(SAG(2), TABLE_INITIAL_SA(1.5)); + with_sag_test(SAG(1.5), TABLE_INITIAL_SA(1.75)); + with_sag_test(SAG(1.01), TABLE_INITIAL_SA(1.5)); + with_sag_test(SAG(1.5), TABLE_INITIAL_SA(1)); +} + +SEASTAR_TEST_CASE(basic_garbage_collection_test) { + return test_env::do_with_async([] (test_env& env) { + auto tmp = tmpdir(); + auto s = schema_builder("ks", "cf") + .with_column("p1", utf8_type, column_kind::partition_key) + .with_column("c1", utf8_type, column_kind::clustering_key) + .with_column("r1", utf8_type) + .build(); + + static constexpr float expired = 0.33; + // we want number of expired keys to be ~ 1.5*sstables::TOMBSTONE_HISTOGRAM_BIN_SIZE so as to + // test ability of histogram to return a good estimation after merging keys. + static int total_keys = std::ceil(sstables::TOMBSTONE_HISTOGRAM_BIN_SIZE/expired)*1.5; + + auto make_insert = [&] (bytes k, uint32_t ttl, uint32_t expiration_time) { + auto key = partition_key::from_exploded(*s, {k}); + mutation m(s, key); + auto c_key = clustering_key::from_exploded(*s, {to_bytes("c1")}); + auto live_cell = atomic_cell::make_live(*utf8_type, 0, bytes("a"), gc_clock::time_point(gc_clock::duration(expiration_time)), gc_clock::duration(ttl)); + m.set_clustered_cell(c_key, *s->get_column_definition("r1"), std::move(live_cell)); + return m; + }; + std::vector mutations; + mutations.reserve(total_keys); + + auto expired_keys = total_keys*expired; + auto now = gc_clock::now(); + for (auto i = 0; i < expired_keys; i++) { + // generate expiration time at different time points or only a few entries would be created in histogram + auto expiration_time = (now - gc_clock::duration(DEFAULT_GC_GRACE_SECONDS*2+i)).time_since_epoch().count(); + mutations.push_back(make_insert(to_bytes("expired_key" + to_sstring(i)), 1, expiration_time)); + } + auto remaining = total_keys-expired_keys; + auto expiration_time = (now + gc_clock::duration(3600)).time_since_epoch().count(); + for (auto i = 0; i < remaining; i++) { + mutations.push_back(make_insert(to_bytes("key" + to_sstring(i)), 3600, expiration_time)); + } + + table_for_tests cf = env.make_table_for_tests(s); + auto close_cf = deferred_stop(cf); + + auto creator = [&] { + auto sst = env.make_sstable(s, tmp.path().string(), env.new_generation(), sstables::get_highest_sstable_version(), big); + return sst; + }; + auto sst = make_sstable_containing(creator, std::move(mutations)); + column_family_test(cf).add_sstable(sst).get(); + + const auto& stats = sst->get_stats_metadata(); + BOOST_REQUIRE(stats.estimated_tombstone_drop_time.bin.size() == sstables::TOMBSTONE_HISTOGRAM_BIN_SIZE); + // Asserts that two keys are equal to within a positive delta + sstable_run run; + // FIXME: can we ignore return value of insert()? + (void)run.insert(sst); + BOOST_REQUIRE(std::fabs(run.estimate_droppable_tombstone_ratio(now, cf.as_table_state().get_tombstone_gc_state(), cf.schema()) - expired) <= 0.1); + + auto cd = sstables::compaction_descriptor({ sst }); + cd.enable_garbage_collection(cf->get_sstable_set()); + auto info = compact_sstables(env, std::move(cd), cf, creator).get(); + auto uncompacted_size = sst->data_size(); + BOOST_REQUIRE(info.new_sstables.size() == 1); + BOOST_REQUIRE(info.new_sstables.front()->estimate_droppable_tombstone_ratio(now, cf.as_table_state().get_tombstone_gc_state(), cf.schema()) == 0.0f); + BOOST_REQUIRE_CLOSE(info.new_sstables.front()->data_size(), uncompacted_size*(1-expired), 5); + auto control = make_strategy_control_for_test(false); + + // sstable satisfying conditions will be included + { + std::map options; + options.emplace("tombstone_threshold", "0.3f"); + // that's needed because sstable with droppable data should be old enough. + options.emplace("tombstone_compaction_interval", "1"); + sleep(2s).get(); + auto cs = sstables::make_compaction_strategy(sstables::compaction_strategy_type::incremental, options); + auto descriptor = cs.get_sstables_for_compaction(cf.as_table_state(), *control); + BOOST_REQUIRE(descriptor.sstables.size() == 1); + BOOST_REQUIRE(descriptor.sstables.front() == sst); + } + + // sstable with droppable ratio of 0.3 won't be included due to threshold + { + std::map options; + options.emplace("tombstone_threshold", "0.5f"); + auto cs = sstables::make_compaction_strategy(sstables::compaction_strategy_type::incremental, options); + auto descriptor = cs.get_sstables_for_compaction(cf.as_table_state(), *control); + BOOST_REQUIRE(descriptor.sstables.size() == 0); + } + // sstable which was recently created won't be included due to min interval + { + std::map options; + options.emplace("tombstone_compaction_interval", "3600"); + auto cs = sstables::make_compaction_strategy(sstables::compaction_strategy_type::incremental, options); + sstables::test(sst).set_data_file_write_time(db_clock::now()); + auto descriptor = cs.get_sstables_for_compaction(cf.as_table_state(), *control); + BOOST_REQUIRE(descriptor.sstables.size() == 0); + } + // sstable which should not be included because of droppable ratio of 0.3, will actually be included + // because the droppable ratio check has been disabled with unchecked_tombstone_compaction set to true + { + std::map options; + options.emplace("tombstone_compaction_interval", "3600"); + options.emplace("tombstone_threshold", "0.5f"); + options.emplace("unchecked_tombstone_compaction", "true"); + auto cs = sstables::make_compaction_strategy(sstables::compaction_strategy_type::incremental, options); + sstables::test(sst).set_data_file_write_time(db_clock::now() - std::chrono::seconds(7200)); + auto descriptor = cs.get_sstables_for_compaction(cf.as_table_state(), *control); + BOOST_REQUIRE(descriptor.sstables.size() == 1); + } + }); +} + +SEASTAR_TEST_CASE(ics_reshape_test) { + static constexpr unsigned disjoint_sstable_count = 256; + + return test_env::do_with_async([] (test_env& env) { + auto builder = schema_builder("tests", "ics_reshape_test") + .with_column("id", utf8_type, column_kind::partition_key) + .with_column("cl", ::timestamp_type, column_kind::clustering_key) + .with_column("value", int32_type); + builder.set_compaction_strategy(sstables::compaction_strategy_type::incremental); + constexpr unsigned target_sstable_size_in_mb = 1000; + std::map opts = { + {"sstable_size_in_mb", to_sstring(target_sstable_size_in_mb)}, + }; + auto cs = sstables::make_compaction_strategy(sstables::compaction_strategy_type::incremental, opts); + builder.set_compaction_strategy_options(std::move(opts)); + auto s = builder.build(); + + auto tokens = tests::generate_partition_keys(disjoint_sstable_count, s, local_shard_only::yes); + + auto make_row = [&](unsigned token_idx) { + auto key = tokens[token_idx].key(); + + mutation m(s, key); + auto value = 1; + auto next_ts = 1; + auto c_key = clustering_key::from_exploded(*s, {::timestamp_type->decompose(next_ts)}); + m.set_clustered_cell(c_key, bytes("value"), data_value(int32_t(value)), next_ts); + return m; + }; + + auto tmp = tmpdir(); + + auto sst_gen = [&env, s, &tmp]() { + return env.make_sstable(s, tmp.path().string(), env.new_generation(), sstables::sstable::version_types::md, big); + }; + + { + unsigned sstable_count = s->max_compaction_threshold() * 2; + + std::vector sstables; + sstables.reserve(sstable_count); + for (unsigned i = 0; i < sstable_count; i++) { + auto sst = make_sstable_containing(sst_gen, {make_row(0)}); + sstables.push_back(std::move(sst)); + } + + auto ret = cs.get_reshaping_job(sstables, s, reshape_config{.mode = reshape_mode::strict}); + BOOST_REQUIRE(ret.sstables.size() == unsigned(s->max_compaction_threshold())); + BOOST_REQUIRE(ret.max_sstable_bytes == target_sstable_size_in_mb*1024*1024); + } + + { + // create set of 256 disjoint ssts and expect that stcs reshape allows them all to be compacted at once + + std::vector sstables; + sstables.reserve(disjoint_sstable_count); + for (unsigned i = 0; i < disjoint_sstable_count; i++) { + auto sst = make_sstable_containing(sst_gen, {make_row(i)}); + sstables.push_back(std::move(sst)); + } + + BOOST_REQUIRE(cs.get_reshaping_job(sstables, s, reshape_config{.mode = reshape_mode::strict}).sstables.size() == disjoint_sstable_count); + } + + { + // create a single run of 256 sstables and expect that reshape will say there's nothing to do. + + run_id sstable_run_id = run_id::create_random_id(); + std::vector sstables; + sstables.reserve(disjoint_sstable_count); + for (unsigned i = 0; i < disjoint_sstable_count; i++) { + auto sst = make_sstable_containing(sst_gen, {make_row(i)}); + sstables::test(sst).set_run_identifier(sstable_run_id); + sstables.push_back(std::move(sst)); + } + + BOOST_REQUIRE(cs.get_reshaping_job(sstables, s, reshape_config{.mode = reshape_mode::strict}).sstables.size() == 0); + } + + { + // create set of 256 overlapping ssts and expect that stcs reshape allows only 32 to be compacted at once + + std::vector sstables; + sstables.reserve(disjoint_sstable_count); + for (unsigned i = 0; i < disjoint_sstable_count; i++) { + auto sst = make_sstable_containing(sst_gen, {make_row(0)}); + sstables.push_back(std::move(sst)); + } + + BOOST_REQUIRE(cs.get_reshaping_job(sstables, s, reshape_config{.mode = reshape_mode::strict}).sstables.size() == uint64_t(s->max_compaction_threshold())); + } + }); +} + +SEASTAR_TEST_CASE(gc_tombstone_with_grace_seconds_test) { + return test_env::do_with_async([](test_env &env) { + auto gc_grace_seconds = 5; + auto schema = schema_builder("tests", "gc_tombstone_with_grace_seconds_test") + .with_column("id", utf8_type, column_kind::partition_key) + .with_column("value", byte_type) + .set_gc_grace_seconds(gc_grace_seconds).build(); + auto sst_factory = env.make_sst_factory(schema); + + auto now = gc_clock::now(); + // set the expiration time to (now - gc_grace_seconds), so that the tombstone is GC'able when compaction is run + auto expiration_time = (now - gc_clock::duration(gc_grace_seconds)).time_since_epoch().count(); + mutation mut(schema, tests::generate_partition_key(schema, local_shard_only::yes)); + auto live_cell = atomic_cell::make_live(*byte_type, 0, to_bytes("a"), gc_clock::time_point(gc_clock::duration(expiration_time)), gc_clock::duration(1)); + mut.set_clustered_cell(clustering_key::make_empty(), *schema->get_column_definition("value"), std::move(live_cell)); + auto sst = make_sstable_containing(env.make_sst_factory(schema), {mut}); + + table_for_tests cf = env.make_table_for_tests(schema); + auto close_cf = deferred_stop(cf); + column_family_test(cf).add_sstable(sst).get(); + + std::map options; + // reduce tombstone_compaction_interval to make droppable data old enough for GC. + options.emplace("tombstone_compaction_interval", "1"); + forward_jump_clocks(std::chrono::seconds{1}); + auto control = make_strategy_control_for_test(false); + auto cs = sstables::make_compaction_strategy(sstables::compaction_strategy_type::incremental, options); + auto descriptor = cs.get_sstables_for_compaction(cf.as_table_state(), *control); + BOOST_REQUIRE_EQUAL(descriptor.sstables.size(), 1); + BOOST_REQUIRE_EQUAL(descriptor.sstables.front(), sst); + }); +} diff --git a/test/boost/schema_change_test.cc b/test/boost/schema_change_test.cc index 3b6d8ba40f5b..3f7a7f7c4e00 100644 --- a/test/boost/schema_change_test.cc +++ b/test/boost/schema_change_test.cc @@ -1158,7 +1158,7 @@ SEASTAR_TEST_CASE(test_system_schema_version_is_stable) { // If you changed the schema of system.batchlog then this is expected to fail. // Just replace expected version with the new version. - BOOST_REQUIRE_EQUAL(s->version(), table_schema_version(utils::UUID("9621f170-f101-3459-a8d3-f342c83ad86e"))); + BOOST_REQUIRE_EQUAL(s->version(), table_schema_version(utils::UUID("776f1766-8688-3d52-908b-a5228900dc00"))); }); } diff --git a/test/boost/sstable_compaction_test.cc b/test/boost/sstable_compaction_test.cc index 5524dfdd37a0..2c1e0b2b1f1c 100644 --- a/test/boost/sstable_compaction_test.cc +++ b/test/boost/sstable_compaction_test.cc @@ -39,6 +39,8 @@ #include "partition_slice_builder.hh" #include "compaction/time_window_compaction_strategy.hh" #include "compaction/leveled_compaction_strategy.hh" +#include "compaction/incremental_backlog_tracker.hh" +#include "compaction/size_tiered_backlog_tracker.hh" #include "test/lib/mutation_assertions.hh" #include "counters.hh" #include "test/lib/simple_schema.hh" @@ -5082,6 +5084,50 @@ SEASTAR_TEST_CASE(twcs_single_key_reader_through_compound_set_test) { }); } +SEASTAR_TEST_CASE(basic_ics_controller_correctness_test) { + return test_env::do_with_async([] (test_env& env) { + static constexpr uint64_t default_fragment_size = 1UL*1024UL*1024UL*1024UL; + + auto s = simple_schema().schema(); + + auto backlog = [&] (compaction_backlog_tracker backlog_tracker, uint64_t max_fragment_size) { + table_for_tests cf = env.make_table_for_tests(); + auto stop_cf = defer([&] { cf.stop().get(); }); + + uint64_t current_sstable_size = default_fragment_size; + uint64_t data_set_size = 0; + static constexpr uint64_t target_data_set_size = 1000UL*1024UL*1024UL*1024UL; + + while (data_set_size < target_data_set_size) { + auto run_identifier = sstables::run_id::create_random_id(); + + auto expected_fragments = std::max(1UL, current_sstable_size / max_fragment_size); + uint64_t fragment_size = std::max(default_fragment_size, current_sstable_size / expected_fragments); + auto tokens = tests::generate_partition_keys(expected_fragments, s, local_shard_only::yes); + + for (auto i = 0UL; i < expected_fragments; i++) { + auto sst = sstable_for_overlapping_test(env, cf->schema(), tokens[i].key(), tokens[i].key()); + sstables::test(sst).set_data_file_size(fragment_size); + sstables::test(sst).set_run_identifier(run_identifier); + backlog_tracker.replace_sstables({}, {std::move(sst)}); + } + data_set_size += current_sstable_size; + current_sstable_size *= 2; + } + + return backlog_tracker.backlog(); + }; + + sstables::incremental_compaction_strategy_options ics_options; + auto ics_backlog = backlog(compaction_backlog_tracker(std::make_unique(ics_options)), default_fragment_size); + sstables::size_tiered_compaction_strategy_options stcs_options; + auto stcs_backlog = backlog(compaction_backlog_tracker(std::make_unique(stcs_options)), std::numeric_limits::max()); + + // don't expect ics and stcs to yield different backlogs for the same workload. + BOOST_CHECK_CLOSE(ics_backlog, stcs_backlog, 0.0001); + }); +} + SEASTAR_TEST_CASE(test_major_does_not_miss_data_in_memtable) { return test_env::do_with_async([] (test_env& env) { auto builder = schema_builder("tests", "test_major_does_not_miss_data_in_memtable") @@ -5253,6 +5299,10 @@ SEASTAR_TEST_CASE(simple_backlog_controller_test_leveled) { return run_controller_test(sstables::compaction_strategy_type::leveled); } +SEASTAR_TEST_CASE(simple_backlog_controller_test_incremental) { + return run_controller_test(sstables::compaction_strategy_type::incremental); +} + SEASTAR_TEST_CASE(test_compaction_strategy_cleanup_method) { return test_env::do_with_async([] (test_env& env) { constexpr size_t all_files = 64; @@ -5339,6 +5389,9 @@ SEASTAR_TEST_CASE(test_compaction_strategy_cleanup_method) { // LCS: Check that 1 jobs is returned for all non-overlapping files in level 1, as incremental compaction can be employed // to limit memory usage and space requirement. run_cleanup_strategy_test(sstables::compaction_strategy_type::leveled, 64, empty_opts, 0ms, 1); + + // ICS: Check that 2 jobs are returned for a size tier containing 2x more files (single-fragment runs) than max threshold. + run_cleanup_strategy_test(sstables::compaction_strategy_type::incremental, 32); }); } diff --git a/test/cqlpy/test_compaction_strategy_validation.py b/test/cqlpy/test_compaction_strategy_validation.py index e11a20ab3e84..4366b33f4fe7 100644 --- a/test/cqlpy/test_compaction_strategy_validation.py +++ b/test/cqlpy/test_compaction_strategy_validation.py @@ -49,6 +49,14 @@ def test_time_window_compaction_strategy_options(cql, table1): def test_leveled_compaction_strategy_options(cql, table1): assert_throws(cql, table1, r"sstable_size_in_mb value \(-5\) must be positive|sstable_size_in_mb must be larger than 0, but was -5", "ALTER TABLE %s WITH compaction = { 'class' : 'LeveledCompactionStrategy', 'sstable_size_in_mb' : -5 }") +def test_incremental_compaction_strategy_options(cql, table1): + assert_throws(cql, table1, r"min_sstable_size value \(-1\) must be non negative", "ALTER TABLE %s WITH compaction = { 'class' : 'IncrementalCompactionStrategy', 'min_sstable_size' : -1 }") + assert_throws(cql, table1, r"bucket_low value \(0\) must be between 0.0 and 1.0", "ALTER TABLE %s WITH compaction = { 'class' : 'IncrementalCompactionStrategy', 'bucket_low' : 0.0 }") + assert_throws(cql, table1, r"bucket_low value \(1.3\) must be between 0.0 and 1.0", "ALTER TABLE %s WITH compaction = { 'class' : 'IncrementalCompactionStrategy', 'bucket_low' : 1.3 }") + assert_throws(cql, table1, r"bucket_high value \(0.7\) must be greater than 1.0", "ALTER TABLE %s WITH compaction = { 'class' : 'IncrementalCompactionStrategy', 'bucket_high' : 0.7 }") + assert_throws(cql, table1, r"space_amplification_goal value \(2.2\) must be greater than 1.0 and less than or equal to 2.0", "ALTER TABLE %s WITH compaction = { 'class' : 'IncrementalCompactionStrategy', 'space_amplification_goal' : 2.2 }") + assert_throws(cql, table1, r"min_threshold value \(1\) must be bigger or equal to 2", "ALTER TABLE %s WITH compaction = { 'class' : 'IncrementalCompactionStrategy', 'min_threshold' : 1 }") + def test_not_allowed_options(cql, table1): def scylla_error(**kwargs): template = "Invalid compaction strategy options {{{}}} for chosen strategy type" @@ -63,3 +71,4 @@ def scylla_error(**kwargs): assert_throws(cql, table1, rf"{scylla_error(abc=-54.54)}|Properties specified \[abc\] are not understood by SizeTieredCompactionStrategy", "ALTER TABLE %s WITH compaction = { 'class' : 'SizeTieredCompactionStrategy', 'abc' : -54.54 }") assert_throws(cql, table1, rf"{scylla_error(dog=3)}||Properties specified \[dog\] are not understood by TimeWindowCompactionStrategy", "ALTER TABLE %s WITH compaction = { 'class' : 'TimeWindowCompactionStrategy', 'dog' : 3 }") assert_throws(cql, table1, rf"{scylla_error(compaction_window_size=4)}|Properties specified \[compaction_window_size\] are not understood by LeveledCompactionStrategy", "ALTER TABLE %s WITH compaction = { 'class' : 'LeveledCompactionStrategy', 'compaction_window_size' : 4 }") + assert_throws(cql, table1, rf"{scylla_error(cold_reads_to_omit=0.5)}|Properties specified \[cold_reads_to_omit\] are not understood by IncrementalCompactionStrategy", "ALTER TABLE %s WITH compaction = { 'class' : 'IncrementalCompactionStrategy', 'cold_reads_to_omit' : 0.5 }") From 569f8e924676f02fe853b7c42a83142785fab259 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Fri, 3 Jan 2025 16:06:50 +0800 Subject: [PATCH 182/397] treewide: fix misspellings these misspellings were identified by codespell. let's fix them. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22154 --- message/messaging_service.cc | 12 ++++++------ reader_concurrency_semaphore_group.cc | 2 +- replica/database.cc | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/message/messaging_service.cc b/message/messaging_service.cc index 31e8692aa57b..eb7b814d8fcc 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -785,8 +785,8 @@ messaging_service::get_rpc_client_idx(messaging_verb verb) { // 2. We are running in a scheduling group that is not assigned to one of the // static tenants (e.g $system) // If this scheduling group is of one of the system's static statement tenants we - // whould have caught it in the loop above. - // The other posibility is that we are running in a scheduling group belongs to + // would have caught it in the loop above. + // The other possibility is that we are running in a scheduling group belongs to // a service level, maybe a deleted one, this is why it is possible that we will // not find the service level name. @@ -852,8 +852,8 @@ messaging_service::scheduling_group_for_isolation_cookie(const sstring& isolatio // We first check if this is a statement isolation cookie - if it is, we will search for the // appropriate service level in the service_level_controller since in can be that - // _scheduling_info_for_connection_index is not yet updated (drop readd case for example) - // in the future we will only fall back here for new service levels that havn't been referenced + // _scheduling_info_for_connection_index is not yet updated (drop read case for example) + // in the future we will only fall back here for new service levels that haven't been referenced // before. // It is safe to assume that an unknown connection type can be rejected since a connection // with an unknown purpose on the inbound side is useless. @@ -910,9 +910,9 @@ messaging_service::scheduling_group_for_isolation_cookie(const sstring& isolatio // create a new service level (internally), it will naturally catch up eventually and by creating it here we prevent // an rpc connection for a valid service level to permanently get stuck in the default service level scheduling group. // If we can't create the service level (we already have too many service levels), we will reject the connection by returning - // an exeptional future. + // an exceptional future. qos::service_level_options slo; - // We put here the minimal ammount of shares for this service level to be functional. When the node catches up it will + // We put here the minimal amount of shares for this service level to be functional. When the node catches up it will // be either deleted or the number of shares and other configuration options will be updated. slo.shares.emplace(1000); slo.shares_name.emplace(service_level_name); diff --git a/reader_concurrency_semaphore_group.cc b/reader_concurrency_semaphore_group.cc index fcc5ea78948f..d3a37ea3f632 100644 --- a/reader_concurrency_semaphore_group.cc +++ b/reader_concurrency_semaphore_group.cc @@ -8,7 +8,7 @@ #include "reader_concurrency_semaphore_group.hh" -// Calling adjust is serialized since 2 adjustments can't happen simultaneosly, +// Calling adjust is serialized since 2 adjustments can't happen simultaneously, // if they did the behaviour would be undefined. future<> reader_concurrency_semaphore_group::adjust() { return with_semaphore(_operations_serializer, 1, [this] () { diff --git a/replica/database.cc b/replica/database.cc index 234f2991bef2..1f1a0a9d307d 100644 --- a/replica/database.cc +++ b/replica/database.cc @@ -2253,7 +2253,7 @@ future<> database::start(sharded& sl_controller) if (!_reader_concurrency_semaphores_group.get_or_null(_dbcfg.statement_scheduling_group)) { // This is super ugly, we need to either force the database to use system scheduling group for non-user queries // or, if we have user queries running on this scheduling group make it's definition more robust (what runs in it). - // Another ugly thing here is that we have to have a pre-existing knowladge about the shares ammount this group was + // Another ugly thing here is that we have to have a pre-existing knowledge about the shares amount this group was // built with. I think we should have a followup that makes this more robust. _reader_concurrency_semaphores_group.add_or_update(_dbcfg.statement_scheduling_group, 1000); _view_update_read_concurrency_semaphores_group.add_or_update(_dbcfg.statement_scheduling_group, 1000); @@ -2269,7 +2269,7 @@ future<> database::start(sharded& sl_controller) for (auto&& service_level_record : service_levels) { auto service_level = sl_controller.local().get_service_level(service_level_record.first); if (service_level.slo.shares_name && *service_level.slo.shares_name != qos::service_level_controller::default_service_level_name) { - // We know slo.shares is valid becuse we know that slo.shares_name is valid + // We know slo.shares is valid because we know that slo.shares_name is valid _reader_concurrency_semaphores_group.add_or_update(service_level.sg, std::get(service_level.slo.shares)); _view_update_read_concurrency_semaphores_group.add_or_update(service_level.sg, std::get(service_level.slo.shares)); } From 879c0a3bd673c1a57e1fb7c930212758604d850a Mon Sep 17 00:00:00 2001 From: Nadav Har'El Date: Sun, 5 Jan 2025 19:33:55 +0200 Subject: [PATCH 183/397] test/alternator: move "cql" fixture to test/alternator/conftest.py Most Alternator test use only the DynamoDB API, not CQL. Tests in test_cql_rbac.py did need CQL to set up roles and RBAC, so this file introduced a "cql" fixture to make CQL requests. A recently-introduced test/alternator/test_service_levels.py also needs access to CQL - it currently uses it for misguided reasons but the next patch will need it for creating a role and a service level. So instead of duplicating this fixture, let's move this fixture into test/alternator/conftest.py that all Alternator tests can share. The next patch will clean up this duplication in test_service_levels.py and the other mistakes it introduced. Signed-off-by: Nadav Har'El --- test/alternator/conftest.py | 35 ++++++++++++++++++++++++++++++++ test/alternator/test_cql_rbac.py | 33 ------------------------------ 2 files changed, 35 insertions(+), 33 deletions(-) diff --git a/test/alternator/conftest.py b/test/alternator/conftest.py index 0bf69410ae93..5fff5beefd54 100644 --- a/test/alternator/conftest.py +++ b/test/alternator/conftest.py @@ -392,3 +392,38 @@ def xfail_tablets(request, has_tablets): def skip_tablets(has_tablets): if has_tablets: pytest.skip("Test may crash when Alternator tables use tablets") + +# Alternator tests normally use only the DynamoDB API. However, a few tests +# need to use CQL to set up Scylla-only features such as service levels or +# CQL-based RBAC (see test_service_levels.py and test_cql_rbac.py), and +# the "cql" fixture enables using CQL. +# If we're not testing Scylla, or the CQL port is not available on the same +# IP address as the Alternator IP address, a test using this fixture will +# be skipped with a message about the CQL API not being available. +@pytest.fixture(scope="session") +def cql(dynamodb): + from cassandra.auth import PlainTextAuthProvider + from cassandra.cluster import Cluster, ConsistencyLevel, ExecutionProfile, EXEC_PROFILE_DEFAULT, NoHostAvailable + from cassandra.policies import RoundRobinPolicy + if is_aws(dynamodb): + pytest.skip('Scylla-only CQL API not supported by AWS') + url = dynamodb.meta.client._endpoint.host + host, = re.search(r'.*://([^:]*):', url).groups() + profile = ExecutionProfile( + load_balancing_policy=RoundRobinPolicy(), + consistency_level=ConsistencyLevel.LOCAL_QUORUM, + serial_consistency_level=ConsistencyLevel.LOCAL_SERIAL) + cluster = Cluster(execution_profiles={EXEC_PROFILE_DEFAULT: profile}, + contact_points=[host], + port=9042, + protocol_version=4, + auth_provider=PlainTextAuthProvider(username='cassandra', password='cassandra'), + ) + try: + ret = cluster.connect() + # "BEGIN BATCH APPLY BATCH" is the closest to do-nothing I could find + ret.execute("BEGIN BATCH APPLY BATCH") + except NoHostAvailable: + pytest.skip('Could not connect to Scylla-only CQL API') + yield ret + cluster.shutdown() diff --git a/test/alternator/test_cql_rbac.py b/test/alternator/test_cql_rbac.py index aa66af3143c1..b7a7906206bf 100644 --- a/test/alternator/test_cql_rbac.py +++ b/test/alternator/test_cql_rbac.py @@ -19,45 +19,12 @@ from contextlib import contextmanager from functools import cache -from cassandra.auth import PlainTextAuthProvider -from cassandra.cluster import Cluster, ConsistencyLevel, ExecutionProfile, EXEC_PROFILE_DEFAULT, NoHostAvailable -from cassandra.policies import RoundRobinPolicy import re from .util import is_aws, unique_table_name, random_string, new_test_table from .test_gsi_updatetable import wait_for_gsi, wait_for_gsi_gone from .test_gsi import assert_index_query -# This file is all about testing RBAC as configured via CQL, so we need to -# connect to CQL to set these tests up. The "cql" fixture below enables that. -# If we're not testing Scylla, or the CQL port is not available on the same -# IP address as the Alternator IP address, a test using this fixture will -# be skipped with a message about the CQL API not being available. -@pytest.fixture(scope="module") -def cql(dynamodb): - if is_aws(dynamodb): - pytest.skip('Scylla-only CQL API not supported by AWS') - url = dynamodb.meta.client._endpoint.host - host, = re.search(r'.*://([^:]*):', url).groups() - profile = ExecutionProfile( - load_balancing_policy=RoundRobinPolicy(), - consistency_level=ConsistencyLevel.LOCAL_QUORUM, - serial_consistency_level=ConsistencyLevel.LOCAL_SERIAL) - cluster = Cluster(execution_profiles={EXEC_PROFILE_DEFAULT: profile}, - contact_points=[host], - port=9042, - protocol_version=4, - auth_provider=PlainTextAuthProvider(username='cassandra', password='cassandra'), - ) - try: - ret = cluster.connect() - # "BEGIN BATCH APPLY BATCH" is the closest to do-nothing I could find - ret.execute("BEGIN BATCH APPLY BATCH") - except NoHostAvailable: - pytest.skip('Could not connect to Scylla-only CQL API') - yield ret - cluster.shutdown() - # new_role() is a context manager for temporarily creating a new role with # a unique name and returning its name and the secret key needed to connect # to it with the DynamoDB API. From e919794db810aea1924a0ad24fa51771338ade7c Mon Sep 17 00:00:00 2001 From: Nadav Har'El Date: Sun, 5 Jan 2025 19:40:14 +0200 Subject: [PATCH 184/397] test/alternator: fix mistakes introduced with test_service_levels.py This patch undoes multiple mistakes done when introducing the test for service levels in pull request #22031: 1. The PR introduced in test/alternator/run and test/alternator/suite.yaml a permanent role and service level that the service-level test is supposed to use. This was a mistake - the test can create the service level for its own use, using CQL, it does not need to assume such a service level already exists. It's important to fix this to allow the service level test to run against an installation of Scylla not set up by our own scripts. Moreover, while the code in suite.yaml was correct, the code in "run" was incorrect (used an outdated keyspace name). This patch removes that incorrect code. 2. The PR introduced a duplicate "cql" fixture, copied verbatim from test_cql_rbac.py (including a comment that was correct only in the latter file :-)). Let's de-duplicate it, using the fixture that I moved to conftest.py in the previous patch. 3. The PR used temporary_grant(). This needelessly complicated the test and added even more duplicate code, and this patch removes all that stuff. This test is about service levels, not RBAC and "grant". This test should just use a superuser role that has the permissions to do everything, and don't need to be granted specific permissions. Signed-off-by: Nadav Har'El --- test/alternator/run | 13 --- test/alternator/suite.yaml | 5 -- test/alternator/test_service_levels.py | 105 ++++++++----------------- 3 files changed, 34 insertions(+), 89 deletions(-) diff --git a/test/alternator/run b/test/alternator/run index 797edf142c17..64131dad6de7 100755 --- a/test/alternator/run +++ b/test/alternator/run @@ -115,19 +115,6 @@ run.wait_for_services(pid, [ lambda: check_alternator(alternator_url), ]) -# Set up the the proper authentication credentials needed by the Alternator -# test. Currently this can only be done through CQL, which is why above we -# needed to make sure CQL is available. -cluster = run.get_cql_cluster(ip) -cql = cluster.connect() - -# Additional role and service level are created to test the feature properly (alternator doesn't have it's own API to set it up so we need to use CQL). -cql.execute("INSERT INTO system_auth_v2.roles (role, salted_hash) VALUES ('alternator_custom_sl', 'secret_pass')") -cql.execute("CREATE SERVICE LEVEL sl_alternator") -cql.execute("ATTACH SERVICE LEVEL sl_alternator TO alternator_custom_sl") - -cluster.shutdown() - # Finally run pytest: success = run.run_pytest(sys.path[0], ['--url', alternator_url] + sys.argv[1:]) diff --git a/test/alternator/suite.yaml b/test/alternator/suite.yaml index 9264015b1e77..ce11ad8094ea 100644 --- a/test/alternator/suite.yaml +++ b/test/alternator/suite.yaml @@ -1,10 +1,5 @@ type: Python pool_size: 6 -prepare_cql: - - INSERT INTO system.roles (role, can_login, salted_hash) VALUES ('alternator_custom_sl', true, 'secret_pass') - - CREATE SERVICE LEVEL sl_alternator - - ATTACH SERVICE LEVEL sl_alternator TO alternator_custom_sl - run_first: - test_streams - test_scan diff --git a/test/alternator/test_service_levels.py b/test/alternator/test_service_levels.py index 9689d57b068e..848c0e74207a 100644 --- a/test/alternator/test_service_levels.py +++ b/test/alternator/test_service_levels.py @@ -3,7 +3,6 @@ # SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest -from test.alternator.util import random_string, is_aws from test.alternator.conftest import new_dynamodb_session from test.alternator.test_metrics import metrics, get_metrics, check_increases_metric from contextlib import contextmanager @@ -13,83 +12,47 @@ import time import re -# Quote an identifier if it needs to be double-quoted in CQL. Quoting is -# *not* needed if the identifier matches [a-z][a-z0-9_]*, otherwise it does. -# double-quotes ('"') in the string are doubled. -def maybe_quote(identifier): - if re.match('^[a-z][a-z0-9_]*$', identifier): - return identifier - return '"' + identifier.replace('"', '""') + '"' +from .util import random_string, is_aws, unique_table_name +from .test_cql_rbac import new_role, new_dynamodb -# Convenience context manager for temporarily GRANTing some permission and -# then revoking it. +# new_service_level() is a context manager for temporarily creating a new +# service level with a unique name and attaching it to the given role. +# The fixture returns the new service level's name. @contextmanager -def temporary_grant(cql, permission, resource, role): - role = maybe_quote(role) - cql.execute(f"GRANT {permission} ON {resource} TO {role}") +def new_service_level(cql, role): + # The service level name is not a table's name but it doesn't matter. + # Because our unique_table_name() uses (deliberately) a non-lower-case + # character, the role name has to be quoted in double quotes when used + # in CQL below. + sl = unique_table_name() + cql.execute(f'CREATE SERVICE LEVEL "{sl}"') + cql.execute(f'ATTACH SERVICE LEVEL "{sl}" TO "{role}"') try: - yield + yield sl finally: - cql.execute(f"REVOKE {permission} ON {resource} FROM {role}") - -# Convenience function for getting the full CQL table name (ksname.cfname) -# for the given Alternator table. This uses our insider knowledge that -# table named "x" is stored in keyspace called "alternator_x", and if we -# ever change this we'll need to change this function too. -def cql_table_name(tab): - return maybe_quote('alternator_' + tab.name) + '.' + maybe_quote(tab.name) - -# This file is all about testing RBAC as configured via CQL, so we need to -# connect to CQL to set these tests up. The "cql" fixture below enables that. -# If we're not testing Scylla, or the CQL port is not available on the same -# IP address as the Alternator IP address, a test using this fixture will -# be skipped with a message about the CQL API not being available. -@pytest.fixture(scope="module") -def cql(dynamodb): - if is_aws(dynamodb): - pytest.skip('Scylla-only CQL API not supported by AWS') - url = dynamodb.meta.client._endpoint.host - host, = re.search(r'.*://([^:]*):', url).groups() - profile = ExecutionProfile( - load_balancing_policy=RoundRobinPolicy(), - consistency_level=ConsistencyLevel.LOCAL_QUORUM, - serial_consistency_level=ConsistencyLevel.LOCAL_SERIAL) - cluster = Cluster(execution_profiles={EXEC_PROFILE_DEFAULT: profile}, - contact_points=[host], - port=9042, - protocol_version=4, - auth_provider=PlainTextAuthProvider(username='cassandra', password='cassandra'), - ) - try: - ret = cluster.connect() - # "BEGIN BATCH APPLY BATCH" is the closest to do-nothing I could find - ret.execute("BEGIN BATCH APPLY BATCH") - except NoHostAvailable: - pytest.skip('Could not connect to Scylla-only CQL API') - yield ret - cluster.shutdown() + cql.execute(f'DROP SERVICE LEVEL "{sl}"') def test_service_level_metrics(test_table, request, dynamodb, cql, metrics): print("Please make sure authorization is enforced in your Scylla installation: alternator_enforce_authorization: true") p = random_string() c = random_string() _ = get_metrics(metrics) - # Use additional user created by test/alternator/run to execute write under sl_alternator service level. - ses = new_dynamodb_session(request, dynamodb, user='alternator_custom_sl') - # service_level_controler acts asynchronously in a loop so we can fail metric check - # if it hasn't processed service level update yet. It can take as long as 10 seconds. - started = time.time() - timeout = 30 - while True: - try: - with temporary_grant(cql, 'MODIFY', cql_table_name(test_table), 'alternator_custom_sl'): - with check_increases_metric(metrics, - ['scylla_storage_proxy_coordinator_write_latency_count'], - {'scheduling_group_name': 'sl:sl_alternator'}): - ses.meta.client.put_item(TableName=test_table.name, Item={'p': p, 'c': c}) - break # no exception, test passed - except: - if time.time() - started > timeout: - raise - else: - time.sleep(0.5) # retry + with new_role(cql, superuser=True) as (role, key): + with new_service_level(cql, role) as sl: + with new_dynamodb(dynamodb, role, key) as ses: + # service_level_controler acts asynchronously in a loop so we can fail metric check + # if it hasn't processed service level update yet. It can take as long as 10 seconds. + started = time.time() + timeout = 30 + while True: + try: + with check_increases_metric(metrics, + ['scylla_storage_proxy_coordinator_write_latency_count'], + {'scheduling_group_name': f'sl:{sl}'}): + ses.meta.client.put_item(TableName=test_table.name, Item={'p': p, 'c': c}) + break # no exception, test passed + except: + if time.time() - started > timeout: + raise + else: + time.sleep(0.5) # retry From 7390116620836713b13bad219b4258f050b3125c Mon Sep 17 00:00:00 2001 From: Amnon Heiman Date: Thu, 19 Dec 2024 16:12:09 +0200 Subject: [PATCH 185/397] alternator/test_returnconsumedcapacity.py: update item This patch adds tests for return consumed capacity for update_item. The tests cover: a simple update for a small object, a missing item, an update with a very large attribute (where the attribute itself is more than 1KB), and an update of a big item that uses read-before-write. --- .../alternator/test_returnconsumedcapacity.py | 64 +++++++++++++++++++ 1 file changed, 64 insertions(+) diff --git a/test/alternator/test_returnconsumedcapacity.py b/test/alternator/test_returnconsumedcapacity.py index 4bb520d4eefc..46f19a7b239b 100644 --- a/test/alternator/test_returnconsumedcapacity.py +++ b/test/alternator/test_returnconsumedcapacity.py @@ -227,3 +227,67 @@ def test_long_delete(test_table): test_table.put_item(Item={'p': p, 'c': c, 'att': val, 'another': val2}, ReturnConsumedCapacity='TOTAL') response = test_table.delete_item(Key={'p': p, 'c': c}, ReturnConsumedCapacity='TOTAL', ReturnValues='ALL_OLD') assert 3 == response['ConsumedCapacity']["CapacityUnits"] + +# The simple update item validates that when updating a short item in a table +# we will get 1 WCU +def test_simple_update_item(test_table_sb): + p = random_string() + val = random_string() + val1 = random_string() + c = random_bytes() + test_table_sb.put_item(Item={'p': p, 'c': c, 'att': val}) + response = test_table_sb.update_item(Key={'p': p, 'c': c}, + UpdateExpression='SET att = :val1', + ExpressionAttributeValues={':val1': val1}, ReturnConsumedCapacity='TOTAL') + assert 'ConsumedCapacity' in response + assert 1 == response['ConsumedCapacity']["CapacityUnits"] + + +# The simple update missing item validates that when trying to update non-exist item +# we will get 1 WCU +def test_simple_update_missing_item(test_table_sb): + p = random_string() + val1 = random_string() + c = random_bytes() + response = test_table_sb.update_item(Key={'p': p, 'c': c}, + UpdateExpression='SET att = :val1', + ExpressionAttributeValues={':val1': val1}, ReturnConsumedCapacity='TOTAL') + assert 'ConsumedCapacity' in response + assert 1 == response['ConsumedCapacity']["CapacityUnits"] + +# The test validates the length of the values passed to update is taking into account +# when calculating the WCU +def test_update_item_long_attr(test_table_sb): + p = random_string() + val = random_string() + c = random_bytes() + test_table_sb.put_item(Item={'p': p, 'c': c, 'att': val}, ReturnConsumedCapacity='TOTAL') + combined_keys = "pcatt" # Takes all the keys and make one single string out of them + total_length = len(p) + len(c) + len(combined_keys) + + val1 = 'a' * (2*KB + 1 - total_length) # val1 is a string that makes the total message length equals 2KB +1 + response = test_table_sb.update_item(Key={'p': p, 'c': c}, + UpdateExpression='SET att = :val1', + ExpressionAttributeValues={':val1': val1}, ReturnConsumedCapacity='TOTAL') + assert 'ConsumedCapacity' in response + assert 3 == response['ConsumedCapacity']["CapacityUnits"] + +# Validates that when the old value is returned the WCU takes +# Its size into account in the WCU calculation. +# WCU is calculated based on 1KB block size. +# The test uses Return value so that the API +# would take the previous item length into account +def test_long_update(test_table): + p = random_string() + c = random_string() + val = random_string() + combined_keys = "pcattanother" # Takes all the keys and make one single string out of them + total_length = len(p) + len(c) + len(val) + len(combined_keys) + + val2 = 'a' * (1 + 2*KB - total_length) # val2 is a string that makes the total message length equals to 2KB+1 + test_table.put_item(Item={'p': p, 'c': c, 'att': val, 'another': val2}) + val1 = 'a' # we replace the long string of val2 with a short string + response = test_table.update_item(Key={'p': p, 'c': c}, UpdateExpression='SET another = :val1', + ExpressionAttributeValues={':val1': val1}, + ReturnConsumedCapacity='TOTAL', ReturnValues='ALL_OLD') + assert 3 == response['ConsumedCapacity']["CapacityUnits"] From 9396c2ee6c66e897dc6146d8236dc1322aa1e132 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 6 Jan 2025 14:22:19 +0800 Subject: [PATCH 186/397] api: include "smaller" header Previously, `api/service_levels.hh` includes `api/api.hh` for accessing symbols like `api/http_context`. but these symbols are already available in a "smaller" header -- `api/api_init.hh`. so, in order to improve the build efficiency, let's include smaller headers in favor of "larger" ones. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22178 --- api/service_levels.hh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/api/service_levels.hh b/api/service_levels.hh index e2e3993774dd..c84753e7decf 100644 --- a/api/service_levels.hh +++ b/api/service_levels.hh @@ -8,10 +8,10 @@ #pragma once -#include "api.hh" +#include "api/api_init.hh" namespace api { void set_service_levels(http_context& ctx, httpd::routes& r, cql_transport::controller& ctl, sharded& qp); -} \ No newline at end of file +} From 4632e217e3e9349169a47494565c58e3b12141ec Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Thu, 2 Jan 2025 18:39:04 +0200 Subject: [PATCH 187/397] cql3: grammar: simplify unaliasedSelector production The return variable s only gets a value by assignment from the temporary tmp. Make tmp the return value instead. Closes scylladb/scylladb#22151 --- cql3/Cql.g | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/cql3/Cql.g b/cql3/Cql.g index 628c823737f1..fd6ef1a89e8d 100644 --- a/cql3/Cql.g +++ b/cql3/Cql.g @@ -449,8 +449,7 @@ selector returns [shared_ptr s] : us=unaliasedSelector (K_AS c=ident { alias = c; })? { $s = ::make_shared(std::move(us), alias); } ; -unaliasedSelector returns [uexpression s] - @init { uexpression tmp; } +unaliasedSelector returns [uexpression tmp] : ( c=cident { tmp = unresolved_identifier{std::move(c)}; } | K_COUNT '(' countArgument ')' { tmp = make_count_rows_function_expression(); } | K_WRITETIME '(' c=cident ')' { tmp = column_mutation_attribute{column_mutation_attribute::attribute_kind::writetime, @@ -463,7 +462,6 @@ unaliasedSelector returns [uexpression s] ( '.' fi=cident { tmp = field_selection{std::move(tmp), std::move(fi)}; } | '[' sub=term ']' { tmp = subscript{std::move(tmp), std::move(sub)}; } )* - { $s = tmp; } ; selectionFunctionArgs returns [std::vector a] From 8873a4e1aa555c693cb158432f6355e83d6a0cb9 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 31 Dec 2024 16:09:33 +0800 Subject: [PATCH 188/397] test.py: pass "count" to re.sub() with kwarg since Python 3.13, passing count to `re.sub()` as positional argument has been deprecated. and when runnint `test.py` with Python 3.13, we have following warning: ``` /home/kefu/dev/scylladb/./test.py:1540: DeprecationWarning: 'count' is passed as positional argument args.modes = re.sub(r'.* List configured modes\n(.*)\n', r'\1', ``` see also https://github.com/python/cpython/issues/56166 in order to silence this distracting warning, let's pass `count` using kwarg. this change was created in the same spirit of c3be4a36af. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22085 --- test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test.py b/test.py index 1d715d722581..9c539976c4e1 100755 --- a/test.py +++ b/test.py @@ -1543,7 +1543,7 @@ def parse_cmd_line() -> argparse.Namespace: # [1/1] List configured modes # debug release dev args.modes = re.sub(r'.* List configured modes\n(.*)\n', r'\1', - out, 1, re.DOTALL).split("\n")[-1].split(' ') + out, count=1, flags=re.DOTALL).split("\n")[-1].split(' ') except Exception: print(palette.fail("Failed to read output of `ninja mode_list`: please run ./configure.py first")) raise From 047ce1364198385319d117ccc536523195205391 Mon Sep 17 00:00:00 2001 From: Anna Stuchlik Date: Mon, 6 Jan 2025 10:29:09 +0800 Subject: [PATCH 189/397] doc: add a new KB article about tombstone garbage collection in ICS Signed-off-by: Kefu Chai Closes scylladb/scylladb#22174 --- docs/kb/garbage-collection-ics.rst | 61 ++++++++++++++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 docs/kb/garbage-collection-ics.rst diff --git a/docs/kb/garbage-collection-ics.rst b/docs/kb/garbage-collection-ics.rst new file mode 100644 index 000000000000..cff6b0684dce --- /dev/null +++ b/docs/kb/garbage-collection-ics.rst @@ -0,0 +1,61 @@ +============================================== +Efficient Tombstone Garbage Collection in ICS +============================================== + +Background +------------ + +Accumulating droppable tombstones is a known problem in LSM-tree, which can severely impact read latency. + +To avoid accumulating droppable tombstones, the compaction process must purge them in a timely manner. However, a droppable +tombstone cannot be purged unless it's compacted with the data it shadows. For example, let's assume that a droppable +tombstone sits on level 1, whereas its shadowed data is on level 2. You'll only be able to purge that tombstone once compaction +promotes it to level 2. + +Size-tiered compaction inefficiently handles this problem with a process known as tombstone compaction. The process works as follows: + +#. First, it searches for an SSTable with droppable tombstone density higher than N% (estimates density using tombstone histogram, which lives in Statistics.db). + + * The search starts from the highest level, as droppable tombstones in high levels are less likely to find shadowed data. + * The threshold can be configured through the ``tombstone_threshold`` option (the default is 0.2). +#. If it finds such an SSTable, it compacts that single file individually, hoping that the droppable tombstones can be purged. + + * The ``tombstone_compaction_interval`` option prevents the data in an SSTable from going through the procedure multiple times in a short interval. + + +The process above has several drawbacks; for example: + +* It doesn't take into account out-of-order writes, so shadowed data may be sitting in lower levels rather than higher ones. +* It can take an unpredictable amount of time for a droppable tombstone to reach the shadowed data counterpart in the LSM tree. + +Making Garbage Collection Efficient in ICS +--------------------------------------------- + +As a remedy to the known problem described above, a new process was introduced to ScyllaDB with `this commit `_. +The process inherits the cross-tier compaction idea from SAG, but instead of using a space-amplification-based trigger, +it uses a tombstone-density trigger instead. It can co-exist with SAG, if enabled. + +The only similarity to STCS 'tombstone compaction' is the trigger, i.e., **when** to start the garbage collection procedure. The difference is **how** it is performed. + +Instead of individually compacting an SSTable run (as it happens in STCS), ICS picks a tier where any of the runs meet +the tombstone density threshold and compacts that tier with the next bigger tier (or the next smaller one if the largest tier +was picked), resulting in cross-tier compaction. + +When the cross-tier approach is used, it’s a matter of time for the droppable tombstones to reach the data they shadow in the LSM tree. +As a result, ICS can promptly purge droppable tombstones. + +To maintain overall compaction efficiency, GC is only considered when same-tier compactions have been completed, as efficiency +is a function of # of files and their relative sizes. That’s also why ICS picks tiers closest in size for the cross-tier +procedure. + +How to Use It +--------------- + +ICS garbage collection is enabled by default starting from version 2021.1.9. + +As in STCS, you can use the compaction options ``tombstone_threshold`` and ``tombstone_compaction_interval`` to tweak the behavior +of the GC process. + +See the ScyllaDB documentation for a full description of :doc:`Compaction `. + + From d3f3e2a6c87fb5059b1ae27da86d5a866c61f7df Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 10 Dec 2024 09:24:49 +0800 Subject: [PATCH 190/397] .github: add more subdirectories to CLEANER_DIR in order to prevent future inclusion of unused headers, let's include - mutation_writer - node_ops - redis - replica subdirectories to CLEANER_DIR, so that this workflow can identify the regressions in future. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22050 --- .github/workflows/iwyu.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/iwyu.yaml b/.github/workflows/iwyu.yaml index 1c11011a0c96..a386e12afecc 100644 --- a/.github/workflows/iwyu.yaml +++ b/.github/workflows/iwyu.yaml @@ -11,7 +11,7 @@ env: CLEANER_OUTPUT_PATH: build/clang-include-cleaner.log # the "idl" subdirectory does not contain C++ source code. the .hh files in it are # supposed to be processed by idl-compiler.py, so we don't check them using the cleaner - CLEANER_DIRS: test/unit exceptions alternator api auth cdc compaction db dht gms index lang message mutation + CLEANER_DIRS: test/unit exceptions alternator api auth cdc compaction db dht gms index lang message mutation mutation_writer node_ops redis replica permissions: {} From a91e03710ac59e0d09b3337f35eb1abf0aeb71a6 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Fri, 3 Jan 2025 13:04:18 +0100 Subject: [PATCH 191/397] repair: check tasks local to given shard Currently task_manager_module::is_aborted checks the tasks local to caller's shard on a given shard. Fix the method to check the task map local to the given shard. Fixes: #22156. Closes scylladb/scylladb#22161 --- repair/repair.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/repair/repair.cc b/repair/repair.cc index f31e777e1f61..67b68eb80643 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -531,9 +531,10 @@ size_t repair::task_manager_module::nr_running_repair_jobs() { } future repair::task_manager_module::is_aborted(const tasks::task_id& uuid, shard_id shard) { - return smp::submit_to(shard, [&] () { - auto it = get_local_tasks().find(uuid); - return it != get_local_tasks().end() && it->second->abort_requested(); + return get_task_manager().container().invoke_on(shard, [name = get_name(), uuid] (tasks::task_manager& tm) { + auto module = tm.find_module(name); + auto it = module->get_local_tasks().find(uuid); + return it != module->get_local_tasks().end() && it->second->abort_requested(); }); } From e4463b11af804cff5fbd2fa96d6b7b35b857034a Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 31 Dec 2024 13:48:36 +0800 Subject: [PATCH 192/397] treewide: replace boost::algorithm::join() with fmt::join() Replace usages of `boost::algorithm::join()` with `fmt::join()` to improve performance and reduce dependency on Boost. `fmt::join()` allows direct formatting of ranges and tuples with custom separators without creating intermediate strings. When formatting comma-separated values into another string, fmt::join() avoids the overhead of temporary string creation that `boost::algorithm::join()` requires. This change also helps streamline our dependencies by leveraging the existing fmt library instead of Boost.Algorithm. To avoid the ambiguity, some caller sites were updated to call `seastar::format()` explicitly. See also - boost::algorithm::join(): https://www.boost.org/doc/libs/1_87_0/doc/html/string_algo/reference.html#doxygen.join_8hpp - fmt::join(): https://fmt.dev/11.0/api/#ranges-api Signed-off-by: Kefu Chai Closes scylladb/scylladb#22082 --- auth/default_authorizer.cc | 1 - auth/resource.cc | 3 +-- auth/service.cc | 6 ++---- auth/standard_role_manager.cc | 3 +-- cql3/statements/authorization_statement.cc | 3 --- cql3/util.cc | 2 -- db/view/view.cc | 5 ++--- index/secondary_index.cc | 1 - main.cc | 7 +++---- service/qos/service_level_controller.cc | 5 ++--- service/raft/raft_group0_client.cc | 3 +-- service/storage_service.cc | 1 - test/boost/cql_query_test.cc | 1 - test/boost/user_types_test.cc | 1 - test/lib/data_model.cc | 4 +--- test/lib/mutation_source_test.cc | 1 - test/lib/random_schema.cc | 11 +++++------ test/perf/perf_fast_forward.cc | 1 - tools/lua_sstable_consumer.cc | 1 - tools/scylla-nodetool.cc | 1 - tools/scylla-sstable.cc | 1 - tools/scylla-types.cc | 4 +--- 22 files changed, 19 insertions(+), 47 deletions(-) diff --git a/auth/default_authorizer.cc b/auth/default_authorizer.cc index 260896e989d4..13fa66abfc18 100644 --- a/auth/default_authorizer.cc +++ b/auth/default_authorizer.cc @@ -16,7 +16,6 @@ extern "C" { #include } -#include #include #include #include diff --git a/auth/resource.cc b/auth/resource.cc index 0e486bd6613d..6e6d26707ca6 100644 --- a/auth/resource.cc +++ b/auth/resource.cc @@ -15,7 +15,6 @@ #include #include -#include #include #include @@ -148,7 +147,7 @@ resource::resource(functions_resource_t, std::string_view keyspace, std::string_ } sstring resource::name() const { - return boost::algorithm::join(_parts, "/"); + return fmt::to_string(fmt::join(_parts, "/")); } std::optional resource::parent() const { diff --git a/auth/service.cc b/auth/service.cc index 94174b0f9ec1..18c6f798b8b4 100644 --- a/auth/service.cc +++ b/auth/service.cc @@ -14,7 +14,6 @@ #include "auth/service.hh" #include -#include #include #include @@ -875,7 +874,6 @@ future<> migrate_to_auth_v2(db::system_keyspace& sys_ks, ::service::raft_group0_ for (const auto& col : schema->all_columns()) { col_names.push_back(col.name_as_cql_string()); } - auto col_names_str = boost::algorithm::join(col_names, ", "); sstring val_binders_str = "?"; for (size_t i = 1; i < col_names.size(); ++i) { val_binders_str += ", ?"; @@ -891,10 +889,10 @@ future<> migrate_to_auth_v2(db::system_keyspace& sys_ks, ::service::raft_group0_ } } auto muts = co_await qp.get_mutations_internal( - format("INSERT INTO {}.{} ({}) VALUES ({})", + seastar::format("INSERT INTO {}.{} ({}) VALUES ({})", db::system_keyspace::NAME, cf_name, - col_names_str, + fmt::join(col_names, ", "), val_binders_str), internal_distributed_query_state(), ts, diff --git a/auth/standard_role_manager.cc b/auth/standard_role_manager.cc index 8a3d4df43058..867a481ff983 100644 --- a/auth/standard_role_manager.cc +++ b/auth/standard_role_manager.cc @@ -12,7 +12,6 @@ #include #include -#include #include #include #include @@ -330,7 +329,7 @@ standard_role_manager::alter(std::string_view role_name, const role_config_updat assignments.push_back(sstring("can_login = ") + (*u.can_login ? "true" : "false")); } - return boost::algorithm::join(assignments, ", "); + return fmt::to_string(fmt::join(assignments, ", ")); }; return require_record(_qp, role_name).then([this, role_name, &u, &mc](record) { diff --git a/cql3/statements/authorization_statement.cc b/cql3/statements/authorization_statement.cc index a8fccdc8a903..ea3ca4712f68 100644 --- a/cql3/statements/authorization_statement.cc +++ b/cql3/statements/authorization_statement.cc @@ -13,9 +13,6 @@ #include "auth/resource.hh" #include "cql3/query_processor.hh" #include "exceptions/exceptions.hh" -#include -#include -#include #include "db/cql_type_parser.hh" #include "auth/common.hh" diff --git a/cql3/util.cc b/cql3/util.cc index c1105c172020..190b928d689c 100644 --- a/cql3/util.cc +++ b/cql3/util.cc @@ -8,8 +8,6 @@ #include "util.hh" #include "cql3/expr/expr-utils.hh" -#include - #ifdef DEBUG #include diff --git a/db/view/view.cc b/db/view/view.cc index 57ef4958d1ea..dd68531e6492 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -16,7 +16,6 @@ #include #include -#include #include #include @@ -2656,7 +2655,7 @@ future<> view_builder::migrate_to_v2(locator::token_metadata_ptr tmptr, db::syst co_await utils::get_local_injector().inject("view_builder_pause_in_migrate_v2", utils::wait_for_message(5min)); auto col_names = schema->all_columns() | std::views::transform([] (const auto& col) {return col.name_as_cql_string(); }) | std::ranges::to>(); - auto col_names_str = boost::algorithm::join(col_names, ", "); + auto col_names_str = fmt::to_string(fmt::join(col_names, ", ")); sstring val_binders_str = "?"; for (size_t i = 1; i < col_names.size(); ++i) { val_binders_str += ", ?"; @@ -2698,7 +2697,7 @@ future<> view_builder::migrate_to_v2(locator::token_metadata_ptr tmptr, db::syst auto row_ts = row.get_as("ts"); auto muts = co_await qp.get_mutations_internal( - format("INSERT INTO {}.{} ({}) VALUES ({})", + seastar::format("INSERT INTO {}.{} ({}) VALUES ({})", db::system_keyspace::NAME, db::system_keyspace::VIEW_BUILD_STATUS_V2, col_names_str, diff --git a/index/secondary_index.cc b/index/secondary_index.cc index c96e013413f0..a5494ee068d7 100644 --- a/index/secondary_index.cc +++ b/index/secondary_index.cc @@ -13,7 +13,6 @@ #include "cql3/statements/index_target.hh" #include -#include #include #include "exceptions/exceptions.hh" diff --git a/main.cc b/main.cc index cd4da43c5214..8b73496bfcfd 100644 --- a/main.cc +++ b/main.cc @@ -118,8 +118,6 @@ #include "utils/shared_dict.hh" #include "message/dictionary_service.hh" -#include - seastar::metrics::metric_groups app_metrics; using namespace std::chrono_literals; @@ -533,8 +531,9 @@ std::string format_parsed_options(const std::vector& opts) { return opt.string_key; } - return (opt.string_key.empty() ? "(positional) " : fmt::format("{}: ", opt.string_key)) + - boost::algorithm::join(opt.value, " "); + return fmt::format("{}{}", + opt.string_key.empty() ? "(positional) " : fmt::format("{}: ", opt.string_key), + fmt::join(opt.value, " ")); }), ", ") ); } diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index 3399c8ab4afa..f5ba500ba02f 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -8,7 +8,6 @@ #include "cql3/util.hh" #include "utils/assert.hh" -#include #include #include @@ -824,7 +823,7 @@ future<> service_level_controller::migrate_to_v2(size_t nodes_count, db::system_ auto col_names = schema->all_columns() | std::views::transform([] (const auto& col) {return col.name_as_cql_string(); }) | std::ranges::to>(); - auto col_names_str = boost::algorithm::join(col_names, ", "); + auto col_names_str = fmt::to_string(fmt::join(col_names, ", ")); sstring val_binders_str = "?"; for (size_t i = 1; i < col_names.size(); ++i) { val_binders_str += ", ?"; @@ -844,7 +843,7 @@ future<> service_level_controller::migrate_to_v2(size_t nodes_count, db::system_ } auto muts = co_await qp.get_mutations_internal( - format("INSERT INTO {}.{} ({}) VALUES ({})", + seastar::format("INSERT INTO {}.{} ({}) VALUES ({})", db::system_keyspace::NAME, db::system_keyspace::SERVICE_LEVELS_V2, col_names_str, diff --git a/service/raft/raft_group0_client.cc b/service/raft/raft_group0_client.cc index c52284b43f57..9fc59bcf5732 100644 --- a/service/raft/raft_group0_client.cc +++ b/service/raft/raft_group0_client.cc @@ -12,7 +12,6 @@ #include #include "raft_group0_client.hh" #include "raft_group_registry.hh" -#include #include "frozen_schema.hh" #include "schema_mutations.hh" @@ -589,7 +588,7 @@ future<> group0_batch::commit(::service::raft_group0_client& group0_client, seas if (!_guard) { on_internal_error(logger, "group0_batch: trying to announce without guard"); } - auto description = boost::algorithm::join(_descriptions, "; "); + auto description = fmt::to_string(fmt::join(_descriptions, "; ")); // common case, don't bother with generators as we would have only 1-2 mutations, // when producer expects substantial number or size of mutations it should use generator if (_generators.size() == 0) { diff --git a/service/storage_service.cc b/service/storage_service.cc index fed207321ae3..bb9e8ea906c2 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -114,7 +114,6 @@ #include #include -#include #include #include diff --git a/test/boost/cql_query_test.cc b/test/boost/cql_query_test.cc index 95a7308081c3..ff4a7878ce7f 100644 --- a/test/boost/cql_query_test.cc +++ b/test/boost/cql_query_test.cc @@ -7,7 +7,6 @@ */ -#include #include #include #include diff --git a/test/boost/user_types_test.cc b/test/boost/user_types_test.cc index 3236ada1c351..9246fbcd832a 100644 --- a/test/boost/user_types_test.cc +++ b/test/boost/user_types_test.cc @@ -20,7 +20,6 @@ #include "db/config.hh" #include -#include BOOST_AUTO_TEST_SUITE(user_types_test) diff --git a/test/lib/data_model.cc b/test/lib/data_model.cc index 4afab72ea0e1..38a30286af1c 100644 --- a/test/lib/data_model.cc +++ b/test/lib/data_model.cc @@ -11,8 +11,6 @@ #include "utils/assert.hh" #include "test/lib/data_model.hh" -#include - #include "schema/schema_builder.hh" #include "concrete_types.hh" @@ -356,7 +354,7 @@ void table_description::rename_clustering_column(const sstring& from, const sstr table_description::table table_description::build() const { auto s = build_schema(); - return { boost::algorithm::join(_change_log, "\n"), s, build_mutations(s) }; + return { fmt::to_string(fmt::join(_change_log, "\n")), s, build_mutations(s) }; } } diff --git a/test/lib/mutation_source_test.cc b/test/lib/mutation_source_test.cc index 4a29661e6785..37cd7c07a9ea 100644 --- a/test/lib/mutation_source_test.cc +++ b/test/lib/mutation_source_test.cc @@ -27,7 +27,6 @@ #include "test/lib/key_utils.hh" #include "test/lib/log.hh" #include "test/lib/reader_concurrency_semaphore.hh" -#include #include "types/user.hh" #include "types/map.hh" #include "types/list.hh" diff --git a/test/lib/random_schema.cc b/test/lib/random_schema.cc index 9b731b0abd81..58537ac35a43 100644 --- a/test/lib/random_schema.cc +++ b/test/lib/random_schema.cc @@ -8,7 +8,6 @@ #include -#include #include #include @@ -1016,22 +1015,22 @@ sstring random_schema::cql() const { std::move(cols.begin(), cols.end(), std::back_inserter(col_specs)); } - sstring primary_key; + std::string primary_key; auto partition_column_names = column_names(_schema, column_kind::partition_key); auto clustering_key_names = column_names(_schema, column_kind::clustering_key); if (!clustering_key_names.empty()) { - primary_key = format("({}), {}", boost::algorithm::join(partition_column_names, ", "), boost::algorithm::join(clustering_key_names, ", ")); + primary_key = fmt::format("({}), {}", fmt::join(partition_column_names, ", "), fmt::join(clustering_key_names, ", ")); } else { - primary_key = format("{}", boost::algorithm::join(partition_column_names, ", ")); + primary_key = fmt::format("{}", fmt::join(partition_column_names, ", ")); } // FIXME include the clustering column orderings - return format( + return seastar::format( "{}\nCREATE TABLE {}.{} (\n\t{}\n\tPRIMARY KEY ({}))", udts_str, _schema->ks_name(), _schema->cf_name(), - boost::algorithm::join(col_specs, ",\n\t"), + fmt::join(col_specs, ",\n\t"), primary_key); } diff --git a/test/perf/perf_fast_forward.cc b/test/perf/perf_fast_forward.cc index f99bba2fbc00..d7aa1cb4f2d6 100644 --- a/test/perf/perf_fast_forward.cc +++ b/test/perf/perf_fast_forward.cc @@ -11,7 +11,6 @@ #include "utils/assert.hh" #include #include -#include #include #include #include diff --git a/tools/lua_sstable_consumer.cc b/tools/lua_sstable_consumer.cc index 2be97e117d06..e1e7c570d178 100644 --- a/tools/lua_sstable_consumer.cc +++ b/tools/lua_sstable_consumer.cc @@ -7,7 +7,6 @@ */ #include "utils/assert.hh" -#include #include #include #include diff --git a/tools/scylla-nodetool.cc b/tools/scylla-nodetool.cc index 6b7133ba6a28..930e640078a9 100644 --- a/tools/scylla-nodetool.cc +++ b/tools/scylla-nodetool.cc @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/tools/scylla-sstable.cc b/tools/scylla-sstable.cc index 31f33c3c01c2..0595cc7b54f1 100644 --- a/tools/scylla-sstable.cc +++ b/tools/scylla-sstable.cc @@ -7,7 +7,6 @@ */ #include -#include #include #include #include diff --git a/tools/scylla-types.cc b/tools/scylla-types.cc index 394f900c3d1b..d187a926087a 100644 --- a/tools/scylla-types.cc +++ b/tools/scylla-types.cc @@ -6,8 +6,6 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include -#include #include #include @@ -96,7 +94,7 @@ sstring to_printable_string(const compound_type& type, bytes_view for (size_t i = 0; i != values.size(); ++i) { printable_values.emplace_back(types.at(i)->to_string(values.at(i))); } - return format("({})", boost::algorithm::join(printable_values, ", ")); + return seastar::format("({})", fmt::join(printable_values, ", ")); } struct printing_visitor { From 66a5e7f672c37876f5e4f10f98312a057197c505 Mon Sep 17 00:00:00 2001 From: David Garcia Date: Tue, 31 Dec 2024 15:41:09 +0000 Subject: [PATCH 193/397] docs: update Sphinx configuration for unified repository publishing This change is related to the unification of enterprise and open-source repositories. The Sphinx configuration is updated to build documentation either for `docs.scylladb.com/manual` or `opensource.docs.scylladb.com`, depending on the flag passed to Sphinx. By default, it will build docs for `docs.scylladb.com/manual`. If the `opensource` flag is passed, it will build docs for `opensource.docs.scylladb.com`, with a different set of versions. This change will prepare the configuration to publish to `docs.scylladb.com/manual` while allowing the option to keep publishing and editing docs with a different multiversion configuration. Note that this change will continue publishing docs to `opensource.docs.scylladb.com` for now since the `opensource` flag is being passed in the `gh-pages.yml` branch. chore: remove comment chore: update project name Closes scylladb/scylladb#22089 --- docs/Makefile | 23 +++++++++++------------ docs/conf.py | 24 +++++++++++++++++++++--- 2 files changed, 32 insertions(+), 15 deletions(-) diff --git a/docs/Makefile b/docs/Makefile index 4f026010ec31..0a6055b32a82 100644 --- a/docs/Makefile +++ b/docs/Makefile @@ -7,7 +7,7 @@ PAPER := BUILDDIR := _build SOURCEDIR := . PREVIEW_HOST := 127.0.0.1 -FLAG := opensource +FLAG := manual CONF_PATH := ./ # Internal variables @@ -50,57 +50,56 @@ clean: # Generate output commands .PHONY: dirhtml dirhtml: setup - $(SPHINXBUILD) -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml + FLAG=$(FLAG) $(SPHINXBUILD) -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml @echo @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." .PHONY: singlehtml singlehtml: setup - $(SPHINXBUILD) -b singlehtml $(ALLSPHINXOPTS) $(BUILDDIR)/singlehtml + FLAG=$(FLAG) $(SPHINXBUILD) -b singlehtml $(ALLSPHINXOPTS) $(BUILDDIR)/singlehtml @echo @echo "Build finished. The HTML page is in $(BUILDDIR)/singlehtml." .PHONY: epub epub: setup - $(SPHINXBUILD) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub + FLAG=$(FLAG) $(SPHINXBUILD) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub @echo @echo "Build finished. The epub file is in $(BUILDDIR)/epub." .PHONY: epub3 epub3: setup - $(SPHINXBUILD) -b epub3 $(ALLSPHINXOPTS) $(BUILDDIR)/epub3 + FLAG=$(FLAG) $(SPHINXBUILD) -b epub3 $(ALLSPHINXOPTS) $(BUILDDIR)/epub3 @echo @echo "Build finished. The epub3 file is in $(BUILDDIR)/epub3." .PHONY: multiversion multiversion: setup - $(POETRY) run sphinx-multiversion $(SOURCEDIR) $(BUILDDIR)/dirhtml $(PROD_OPTS) + FLAG=$(FLAG) $(POETRY) run sphinx-multiversion $(SOURCEDIR) $(BUILDDIR)/dirhtml $(PROD_OPTS) @echo @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." .PHONY: redirects redirects: setup - $(POETRY) run redirects-cli fromfile --yaml-file ./_utils/redirects.yaml --output-dir $(BUILDDIR)/dirhtml + FLAG=$(FLAG) $(POETRY) run redirects-cli fromfile --yaml-file ./_utils/redirects.yaml --output-dir $(BUILDDIR)/dirhtml @echo @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." # Preview commands .PHONY: preview preview: setup - $(POETRY) run sphinx-autobuild -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml --host $(PREVIEW_HOST) --port 5500 --ignore *.csv --ignore *.json --ignore *.yaml + FLAG=$(FLAG) $(POETRY) run sphinx-autobuild -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml --host $(PREVIEW_HOST) --port 5500 --ignore *.csv --ignore *.json --ignore *.yaml .PHONY: multiversionpreview multiversionpreview: multiversion - $(POETRY) run python -m http.server 5500 --directory $(BUILDDIR)/dirhtml + FLAG=$(FLAG) $(POETRY) run python -m http.server 5500 --directory $(BUILDDIR)/dirhtml # Test commands .PHONY: test test: setup - $(SPHINXBUILD) -b dirhtml $(TESTSPHINXOPTS) $(BUILDDIR)/dirhtml + FLAG=$(FLAG) $(SPHINXBUILD) -b dirhtml $(TESTSPHINXOPTS) $(BUILDDIR)/dirhtml @echo @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." .PHONY: linkcheck linkcheck: setup - $(SPHINXBUILD) -b linkcheck $(SOURCEDIR) $(BUILDDIR)/linkcheck - + FLAG=$(FLAG) $(SPHINXBUILD) -b linkcheck $(SOURCEDIR) $(BUILDDIR)/linkcheck diff --git a/docs/conf.py b/docs/conf.py index 580367582613..c8b2f12e3d09 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -11,12 +11,15 @@ sys.path.insert(0, os.path.abspath("..")) # -- Global variables +FLAG = os.getenv('FLAG', 'manual') +# Set the project name +PROJECT = "ScyllaDB Manual" # Set the base URL for the documentation site. -BASE_URL = 'https://opensource.docs.scylladb.com' +BASE_URL = 'https://docs.scylladb.com/manual' # Build documentation for the following tags and branches. TAGS = [] -BRANCHES = ["master", "branch-5.1", "branch-5.2", "branch-5.4", "branch-6.0", "branch-6.1", "branch-6.2"] +BRANCHES = ["master", "branch-6.2"] # Set the latest version. LATEST_VERSION = "branch-6.2" # Set which versions are not released yet. @@ -24,6 +27,21 @@ # Set which versions are deprecated. DEPRECATED_VERSIONS = [""] +if FLAG == 'opensource': + # Set the project name + PROJECT = "ScyllaDB Open Source" + # Set the base URL for the documentation site. + BASE_URL = 'https://opensource.docs.scylladb.com' + # Build documentation for the following tags and branches. + TAGS = [] + BRANCHES = ["master", "branch-5.1", "branch-5.2", "branch-5.4", "branch-6.0", "branch-6.1", "branch-6.2"] + # Set the latest version. + LATEST_VERSION = "branch-6.2" + # Set which versions are not released yet. + UNSTABLE_VERSIONS = ["master"] + # Set which versions are deprecated. + DEPRECATED_VERSIONS = [""] + # -- General configuration # Add any Sphinx extension module names here, as strings. @@ -55,7 +73,7 @@ master_doc = "index" # General information about the project. -project = "ScyllaDB Open Source" +project = PROJECT copyright = str(date.today().year) + ", ScyllaDB. All rights reserved." author = u"ScyllaDB Project Contributors" From 173fad296aa5cb9c836a79d57fcee4361f0a79d1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Thu, 19 Dec 2024 14:58:10 +0200 Subject: [PATCH 194/397] tools/schema_loader.cc: remove duplicate include of short_streams.hh Closes scylladb/scylladb#21982 --- tools/schema_loader.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/tools/schema_loader.cc b/tools/schema_loader.cc index c416938dd560..433118592cf9 100644 --- a/tools/schema_loader.cc +++ b/tools/schema_loader.cc @@ -11,7 +11,6 @@ #include #include #include -#include #include #include From b74565e83fac845912e7e983bff8b61d2e54ab8b Mon Sep 17 00:00:00 2001 From: Yaron Kaikov Date: Sun, 29 Dec 2024 10:50:57 +0200 Subject: [PATCH 195/397] dist/common/scripts/scylla_raid_setup: reduce XFS metadata overhead The block size of 1k is significantly increasing metadata overhead with xfs since it reserves space upfront for btree expansion. With CRC disabled, this reservation doesn't happen. Smaller btree blocks reduce the fanout factor, increasing btree height and the reservation size. So block size implies a trade-off between write amplification and metadata size. Bigger blocks, smaller metadata, more write ampl. Smaller blocks, more metadata, and less write ampl. Let's disable both `rmapbt` and `relink` since we replicate data, and we can afford to rebuild a replica on local corruption. Fixes: https://github.com/scylladb/scylladb/issues/22028 Closes scylladb/scylladb#22072 --- dist/common/scripts/scylla_raid_setup | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dist/common/scripts/scylla_raid_setup b/dist/common/scripts/scylla_raid_setup index 479f8aa851a8..6a90b6feb3d3 100755 --- a/dist/common/scripts/scylla_raid_setup +++ b/dist/common/scripts/scylla_raid_setup @@ -229,7 +229,7 @@ if __name__ == '__main__': # and it also cannot be smaller than the sector size. block_size = max(1024, sector_size) run('udevadm settle', shell=True, check=True) - run(f'mkfs.xfs -b size={block_size} {fsdev} -K', shell=True, check=True) + run(f'mkfs.xfs -b size={block_size} {fsdev} -K -m rmapbt=0 -m reflink=0', shell=True, check=True) run('udevadm settle', shell=True, check=True) if is_debian_variant(): From f7fd55146d51b45c677a57be4a76f2aab2f4e8c1 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 6 Jan 2025 21:45:12 +0800 Subject: [PATCH 196/397] compaction: do not include unused headers these unused includes are identified by clang-include-cleaner. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22188 --- compaction/incremental_backlog_tracker.cc | 1 + compaction/incremental_compaction_strategy.cc | 1 + compaction/incremental_compaction_strategy.hh | 5 ----- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/compaction/incremental_backlog_tracker.cc b/compaction/incremental_backlog_tracker.cc index ff5829a82657..85263d7508b8 100644 --- a/compaction/incremental_backlog_tracker.cc +++ b/compaction/incremental_backlog_tracker.cc @@ -6,6 +6,7 @@ #include "incremental_backlog_tracker.hh" #include "sstables/sstables.hh" +#include using namespace sstables; diff --git a/compaction/incremental_compaction_strategy.cc b/compaction/incremental_compaction_strategy.cc index 57aacece3754..f07806b1aeaf 100644 --- a/compaction/incremental_compaction_strategy.cc +++ b/compaction/incremental_compaction_strategy.cc @@ -11,6 +11,7 @@ #include "compaction_manager.hh" #include "incremental_compaction_strategy.hh" #include "incremental_backlog_tracker.hh" +#include #include #include #include diff --git a/compaction/incremental_compaction_strategy.hh b/compaction/incremental_compaction_strategy.hh index 16a9bf91fee9..b388ed0beb46 100644 --- a/compaction/incremental_compaction_strategy.hh +++ b/compaction/incremental_compaction_strategy.hh @@ -7,12 +7,7 @@ #pragma once #include "compaction_strategy_impl.hh" -#include "compaction.hh" -#include -#include #include -#include -#include "size_tiered_compaction_strategy.hh" class incremental_backlog_tracker; From 353b522ca00372bf3ac1dd15fec38f2f43050cd2 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 31 Dec 2024 14:31:36 +0800 Subject: [PATCH 197/397] treewide: migrate from boost::adaptors::reversed to std::views::reverse now that we are allowed to use C++23. we now have the luxury of using `std::views::reverse`. - replace `boost::adaptors::transformed` with `std::views::transform` - remove unused `#include ` this change is part of our ongoing effort to modernize our codebase and reduce external dependencies where possible. Signed-off-by: Kefu Chai --- compaction/size_tiered_compaction_strategy.cc | 3 +-- cql3/selection/selection.cc | 1 - db/commitlog/commitlog.cc | 2 -- multishard_mutation_query.cc | 2 -- mutation/range_tombstone_list.cc | 4 ++-- service/qos/service_level_controller.cc | 3 +-- test/boost/cql_query_test.cc | 1 - test/lib/sstable_utils.cc | 1 - test/lib/test_services.cc | 3 +-- tools/scylla-nodetool.cc | 3 +-- utils/dynamic_bitset.cc | 7 +++---- 11 files changed, 9 insertions(+), 21 deletions(-) diff --git a/compaction/size_tiered_compaction_strategy.cc b/compaction/size_tiered_compaction_strategy.cc index 933c5fcbd8f8..bc1139921506 100644 --- a/compaction/size_tiered_compaction_strategy.cc +++ b/compaction/size_tiered_compaction_strategy.cc @@ -11,7 +11,6 @@ #include "size_tiered_compaction_strategy.hh" #include "cql3/statements/property_definitions.hh" -#include #include namespace sstables { @@ -241,7 +240,7 @@ size_tiered_compaction_strategy::get_sstables_for_compaction(table_state& table_ // ratio is greater than threshold. // prefer oldest sstables from biggest size tiers because they will be easier to satisfy conditions for // tombstone purge, i.e. less likely to shadow even older data. - for (auto&& sstables : buckets | boost::adaptors::reversed) { + for (auto&& sstables : buckets | std::views::reverse) { // filter out sstables which droppable tombstone ratio isn't greater than the defined threshold. auto e = boost::range::remove_if(sstables, [this, compaction_time, &table_s] (const sstables::shared_sstable& sst) -> bool { return !worth_dropping_tombstones(sst, compaction_time, table_s); diff --git a/cql3/selection/selection.cc b/cql3/selection/selection.cc index 1a1ee5dd53bf..39cccdb5ec8a 100644 --- a/cql3/selection/selection.cc +++ b/cql3/selection/selection.cc @@ -10,7 +10,6 @@ #include #include -#include #include "cql3/selection/selection.hh" #include "cql3/selection/raw_selector.hh" diff --git a/db/commitlog/commitlog.cc b/db/commitlog/commitlog.cc index a81405acd31a..bf52b7bddca8 100644 --- a/db/commitlog/commitlog.cc +++ b/db/commitlog/commitlog.cc @@ -13,8 +13,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/multishard_mutation_query.cc b/multishard_mutation_query.cc index 3fbccd4de2e6..d16fb567437c 100644 --- a/multishard_mutation_query.cc +++ b/multishard_mutation_query.cc @@ -19,8 +19,6 @@ #include #include -#include - #include logging::logger mmq_log("multishard_mutation_query"); diff --git a/mutation/range_tombstone_list.cc b/mutation/range_tombstone_list.cc index 4701959c3d9c..7f6d49b45e50 100644 --- a/mutation/range_tombstone_list.cc +++ b/mutation/range_tombstone_list.cc @@ -6,10 +6,10 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ -#include #include "range_tombstone_list.hh" #include "utils/assert.hh" #include "utils/allocation_strategy.hh" +#include #include range_tombstone_list::range_tombstone_list(const range_tombstone_list& x) @@ -386,7 +386,7 @@ void range_tombstone_list::reverter::update(range_tombstones_type::iterator it, } void range_tombstone_list::reverter::revert() noexcept { - for (auto&& rt : _ops | boost::adaptors::reversed) { + for (auto&& rt : _ops | std::views::reverse) { seastar::visit(rt, [this] (auto& op) { op.undo(_s, _dst); }); diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index f5ba500ba02f..9c1ebeb6c6d7 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -8,7 +8,6 @@ #include "cql3/util.hh" #include "utils/assert.hh" -#include #include #include @@ -325,7 +324,7 @@ future<> service_level_controller::update_effective_service_levels_cache() { // role1 in `sorted` vector. // That's why if we iterate over the vector in reversed order, we will visit the roles from the bottom // and we can use already calculated effective service levels for all of the subroles. - for (auto& role: sorted | boost::adaptors::reversed) { + for (auto& role: sorted | std::views::reverse) { std::optional sl_options; if (auto sl_name_it = attributes.find(role); sl_name_it != attributes.end()) { diff --git a/test/boost/cql_query_test.cc b/test/boost/cql_query_test.cc index ff4a7878ce7f..036fe376d0d8 100644 --- a/test/boost/cql_query_test.cc +++ b/test/boost/cql_query_test.cc @@ -9,7 +9,6 @@ #include #include -#include #include #include diff --git a/test/lib/sstable_utils.cc b/test/lib/sstable_utils.cc index 67522bf870e8..cd4aa9443b83 100644 --- a/test/lib/sstable_utils.cc +++ b/test/lib/sstable_utils.cc @@ -13,7 +13,6 @@ #include "dht/i_partitioner.hh" #include "dht/murmur3_partitioner.hh" #include -#include #include "sstables/version.hh" #include "test/lib/mutation_reader_assertions.hh" #include "test/lib/reader_concurrency_semaphore.hh" diff --git a/test/lib/test_services.cc b/test/lib/test_services.cc index f1c47fc4cecb..24117211158c 100644 --- a/test/lib/test_services.cc +++ b/test/lib/test_services.cc @@ -20,7 +20,6 @@ #include "utils/assert.hh" #include "utils/overloaded_functor.hh" #include -#include #include #include #include @@ -521,7 +520,7 @@ static sstring toc_filename(const sstring& dir, schema_ptr schema, sstables::gen } future test_env::reusable_sst(schema_ptr schema, sstring dir, sstables::generation_type generation) { - for (auto v : boost::adaptors::reverse(all_sstable_versions)) { + for (auto v : std::views::reverse(all_sstable_versions)) { if (co_await file_exists(toc_filename(dir, schema, generation, v))) { co_return co_await reusable_sst(schema, dir, generation, v); } diff --git a/tools/scylla-nodetool.cc b/tools/scylla-nodetool.cc index 930e640078a9..cca28596c1f6 100644 --- a/tools/scylla-nodetool.cc +++ b/tools/scylla-nodetool.cc @@ -21,7 +21,6 @@ #include #include #include -#include #include #include #include @@ -4525,7 +4524,7 @@ operation_func get_operation_function(const operation& op) noexcept { name.pop_back(); // Check suboperations. - for (auto n : name | boost::adaptors::reversed) { + for (auto n : name | std::views::reverse) { action = action.suboperation_funcs.at(n); } diff --git a/utils/dynamic_bitset.cc b/utils/dynamic_bitset.cc index eef6813ecc92..e631f4168ec8 100644 --- a/utils/dynamic_bitset.cc +++ b/utils/dynamic_bitset.cc @@ -8,8 +8,7 @@ #include #include -#include -#include +#include #include "utils/dynamic_bitset.hh" #include "seastarx.hh" @@ -43,7 +42,7 @@ void dynamic_bitset::clear(size_t n) noexcept { size_t dynamic_bitset::find_first_set() const noexcept { size_t pos = 0; - for (auto& vv : _bits | boost::adaptors::reversed) { + for (auto& vv : _bits | std::views::reverse) { auto v = vv[pos]; pos *= bits_per_int; if (v) { @@ -94,7 +93,7 @@ size_t dynamic_bitset::find_next_set(size_t n) const noexcept size_t dynamic_bitset::find_last_set() const noexcept { size_t pos = 0; - for (auto& vv : _bits | boost::adaptors::reversed) { + for (auto& vv : _bits | std::views::reverse) { auto v = vv[pos]; pos *= bits_per_int; if (v) { From d984f27b23d8d5da8cd258361c619b061cd468ef Mon Sep 17 00:00:00 2001 From: Dani Tweig Date: Tue, 24 Dec 2024 15:46:40 +0200 Subject: [PATCH 198/397] Create urgent_issue_reminder.yml Closes scylladb/scylladb#22042 --- .github/workflows/urgent_issue_reminder.yml | 58 +++++++++++++++++++++ 1 file changed, 58 insertions(+) create mode 100644 .github/workflows/urgent_issue_reminder.yml diff --git a/.github/workflows/urgent_issue_reminder.yml b/.github/workflows/urgent_issue_reminder.yml new file mode 100644 index 000000000000..cedbb70749bb --- /dev/null +++ b/.github/workflows/urgent_issue_reminder.yml @@ -0,0 +1,58 @@ +name: Urgent Issue Reminder + +on: + schedule: + - cron: '10 8 * * 1' # Runs every Monday at 8 AM + +jobs: + reminder: + runs-on: ubuntu-latest + + steps: + - name: Send reminders + uses: actions/github-script@v7 + with: + script: | + const labelFilters = ['P0', 'P1', 'Field-Tier1','status/release blocker', 'status/regression']; + const excludingLabelFilters = ['documentation']; + const daysInactive = 7; + const now = new Date(); + + // Fetch open issues + const issues = await github.rest.issues.listForRepo({ + owner: context.repo.owner, + repo: context.repo.repo, + state: 'open' + }); + console.log("Looking for issues with labels:"+labelFilters+", excluding labels:"+excludingLabelFilters+ ", inactive for more than "+daysInactive+" days."); + for (const issue of issues.data) { + // Check if issue has any of the specified labels + const hasFilteredLabel = issue.labels.some(label => labelFilters.includes(label.name)); + const hasExcludingLabel = issue.labels.some(label => excludingLabelFilters.includes(label.name)); + if (hasExcludingLabel) continue; + if (!hasFilteredLabel) continue; + + // Check for inactivity + const lastUpdated = new Date(issue.updated_at); + const diffInDays = (now - lastUpdated) / (1000 * 60 * 60 * 24); + console.log("Issue #"+issue.number+"; Days inactive:"+diffInDays); + if (diffInDays > daysInactive) { + if (issue.assignees.length > 0) { + console.log("==>> Alert about issue #"+issue.number); + const assigneesLogins = issue.assignees.map(assignee => `@${assignee.login}`).join(', '); + await github.rest.issues.createComment({ + owner: context.repo.owner, + repo: context.repo.repo, + issue_number: issue.number, + body: `${assigneesLogins}, This urgent issue had no activity for more than ${daysInactive} days. Please check its status.\n CC @mykaul @dani-tweig` + }); + } else { + await github.rest.issues.createComment({ + owner: context.repo.owner, + repo: context.repo.repo, + issue_number: issue.number, + body: `This urgent issue had no activity for more than ${daysInactive} days. Please check its status.\n CC @mykaul @dani-tweig` + }); + } + } + } From 8d824a564f79d8d7c2a66f6be9339c16d3959a98 Mon Sep 17 00:00:00 2001 From: Anna Stuchlik Date: Tue, 26 Nov 2024 17:06:17 +0100 Subject: [PATCH 199/397] doc: add troubleshooting removal with --autoremove-ubuntu This commit adds a troubleshooting article on removing ScyllaDB with the --autoremove option. Fixes https://github.com/scylladb/scylladb/issues/21408 Closes scylladb/scylladb#21697 --- docs/troubleshooting/autoremove-ubuntu.rst | 38 +++++++++++++++++++ docs/troubleshooting/index.rst | 2 + docs/troubleshooting/install-remove/index.rst | 13 +++++++ 3 files changed, 53 insertions(+) create mode 100644 docs/troubleshooting/autoremove-ubuntu.rst create mode 100644 docs/troubleshooting/install-remove/index.rst diff --git a/docs/troubleshooting/autoremove-ubuntu.rst b/docs/troubleshooting/autoremove-ubuntu.rst new file mode 100644 index 000000000000..a7932343b523 --- /dev/null +++ b/docs/troubleshooting/autoremove-ubuntu.rst @@ -0,0 +1,38 @@ +Removing ScyllaDB with the "--autoremove" option on Ubuntu breaks system packages +====================================================================================== + +Problem +^^^^^^^ + +Running ``apt purge scylla --autoremove`` marks most system packages for +removal. + +.. code:: + + root@myserv:~# apt purge scylla --autoremove + Reading package lists... Done + Building dependency tree... Done + Reading state information... Done + The following packages will be REMOVED: + apport-symptoms* bc* bcache-tools* bolt* btrfs-progs* byobu* cloud-guest-utils* cloud-init* cloud-initramfs-copymods* cloud-initramfs-dyn-netconf* cryptsetup* cryptsetup-initramfs* dmeventd* eatmydata* ethtool* fdisk* fonts-ubuntu-console* fwupd* fwupd-signed* gdisk* gir1.2-packagekitglib-1.0* git* git-man* kpartx* landscape-common* libaio1* libappstream4* libatasmart4* libblockdev-crypto2* libblockdev-fs2* + libblockdev-loop2* libblockdev-part-err2* libblockdev-part2* libblockdev-swap2* libblockdev-utils2* libblockdev2* libdevmapper-event1.02.1* libeatmydata1* liberror-perl* libfdisk1* libfwupd2* libfwupdplugin5* libgcab-1.0-0* libgpgme11* libgstreamer1.0-0* libgusb2* libinih1* libintl-perl* libintl-xs-perl* libjcat1* libjson-glib-1.0-0* libjson-glib-1.0-common* liblvm2cmd2.03* libmbim-glib4* libmbim-proxy* + libmm-glib0* libmodule-find-perl* libmodule-scandeps-perl* libmspack0* libpackagekit-glib2-18* libparted-fs-resize0* libproc-processtable-perl* libqmi-glib5* libqmi-proxy* libsgutils2-2* libsmbios-c2* libsort-naturally-perl* libstemmer0d* libtcl8.6* libterm-readkey-perl* libudisks2-0* liburcu8* libutempter0* libvolume-key1* libxmlb2* libxmlsec1* libxmlsec1-openssl* libxslt1.1* lvm2* lxd-agent-loader* mdadm* + modemmanager* motd-news-config* multipath-tools* needrestart* open-vm-tools* overlayroot* packagekit* packagekit-tools* pastebinit* patch* pollinate* python3-apport* python3-certifi* python3-chardet* python3-configobj* python3-debconf* python3-debian* python3-json-pointer* python3-jsonpatch* python3-jsonschema* python3-magic* python3-newt* python3-packaging* python3-pexpect* python3-problem-report* + python3-ptyprocess* python3-pyrsistent* python3-requests* python3-software-properties* python3-systemd* python3-xkit* run-one* sbsigntool* screen* scylla* scylla-conf* scylla-cqlsh* scylla-kernel-conf* scylla-node-exporter* scylla-python3* scylla-server* secureboot-db* sg3-utils* sg3-utils-udev* software-properties-common* sosreport* tcl* tcl8.6* thin-provisioning-tools* tmux* ubuntu-drivers-common* udisks2* + unattended-upgrades* update-notifier-common* usb-modeswitch* usb-modeswitch-data* xfsprogs* zerofree* + 0 upgraded, 0 newly installed, 139 to remove and 0 not upgraded. + +Cause +^^^^^^^ + +This problem may occur on Ubuntu 22.04 or earlier. It is caused by +the ``systemd-coredump`` package installed with the ``scylla_setup`` script. +Installing ``systemd-coredump`` results in removing ``apport`` and ``ubuntu-server``. +In turn, the ``--autoremove`` option marks for removal all packages installed +by ``ubuntu-server dependencies``. + + +Solution +^^^^^^^^^^ + +Do not run the ``--autoremove`` option when removing ScyllaDB. \ No newline at end of file diff --git a/docs/troubleshooting/index.rst b/docs/troubleshooting/index.rst index 848066927934..7e9ed5306b82 100644 --- a/docs/troubleshooting/index.rst +++ b/docs/troubleshooting/index.rst @@ -14,6 +14,7 @@ Troubleshooting ScyllaDB storage/index CQL/index monitor/index + install-remove/index ScyllaDB's troubleshooting section contains articles which are targeted to pinpoint and answer problems with ScyllaDB. For broader issues and workarounds, consult the :doc:`Knowledge base `. @@ -33,6 +34,7 @@ Keep your versions up-to-date. The two latest versions are supported. Also, alwa * :doc:`Data Storage and SSTables ` * :doc:`CQL errors ` * :doc:`ScyllaDB Monitoring and ScyllaDB Manager ` + * :doc:`Installation and Removal ` Also check out the `Monitoring lesson `_ on ScyllaDB University, which covers how to troubleshoot different issues when running a ScyllaDB cluster. diff --git a/docs/troubleshooting/install-remove/index.rst b/docs/troubleshooting/install-remove/index.rst new file mode 100644 index 000000000000..a85acf8a910d --- /dev/null +++ b/docs/troubleshooting/install-remove/index.rst @@ -0,0 +1,13 @@ +Installation and Removal +=========================== + +.. toctree:: + :hidden: + :maxdepth: 2 + + Removing ScyllaDB on Ubuntu breaks system packages + + + +* :doc:`Removing ScyllaDB with the "--autoremove" option on Ubuntu breaks system packages ` + \ No newline at end of file From 0617564123d3b858185f9c45f8f0afa8896d59ac Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Thu, 5 Dec 2024 17:16:12 +0200 Subject: [PATCH 200/397] db/commitlog: make the commit log hard limit mandatory mark the config parameter --commitlog-use-hard-size-limit as deprecated so the default 'true' is always used, making the hard limit mandatory. Fixes scylladb/scylladb#16471 Closes scylladb/scylladb#21804 --- db/commitlog/commitlog.cc | 2 +- db/commitlog/commitlog.hh | 2 +- db/config.cc | 4 ++-- test/boost/commitlog_test.cc | 1 + 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/db/commitlog/commitlog.cc b/db/commitlog/commitlog.cc index bf52b7bddca8..02e3af861d0b 100644 --- a/db/commitlog/commitlog.cc +++ b/db/commitlog/commitlog.cc @@ -104,7 +104,7 @@ db::commitlog::config db::commitlog::config::from_db_config(const db::config& cf c.mode = cfg.commitlog_sync() == "batch" ? sync_mode::BATCH : sync_mode::PERIODIC; c.extensions = &cfg.extensions(); c.use_o_dsync = cfg.commitlog_use_o_dsync(); - c.allow_going_over_size_limit = !cfg.commitlog_use_hard_size_limit(); + c.allow_going_over_size_limit = false; if (cfg.commitlog_flush_threshold_in_mb() >= 0) { c.commitlog_flush_threshold_in_mb = cfg.commitlog_flush_threshold_in_mb(); diff --git a/db/commitlog/commitlog.hh b/db/commitlog/commitlog.hh index 50c6cbadf588..df79c6ae6b7b 100644 --- a/db/commitlog/commitlog.hh +++ b/db/commitlog/commitlog.hh @@ -108,7 +108,7 @@ public: bool use_o_dsync = false; bool warn_about_segments_left_on_disk_after_shutdown = true; - bool allow_going_over_size_limit = true; + bool allow_going_over_size_limit = false; bool allow_fragmented_entries = false; // The base segment ID to use. diff --git a/db/config.cc b/db/config.cc index 1bc964a0f240..9010ecb5583d 100644 --- a/db/config.cc +++ b/db/config.cc @@ -616,8 +616,8 @@ db::config::config(std::shared_ptr exts) "Threshold for commitlog disk usage. When used disk space goes above this value, Scylla initiates flushes of memtables to disk for the oldest commitlog segments, removing those log segments. Adjusting this affects disk usage vs. write latency. Default is (approximately) commitlog_total_space_in_mb - *commitlog_segment_size_in_mb.") , commitlog_use_o_dsync(this, "commitlog_use_o_dsync", value_status::Used, true, "Whether or not to use O_DSYNC mode for commitlog segments IO. Can improve commitlog latency on some file systems.\n") - , commitlog_use_hard_size_limit(this, "commitlog_use_hard_size_limit", value_status::Used, true, - "Whether or not to use a hard size limit for commitlog disk usage. Default is true. Enabling this can cause latency spikes, whereas the default can lead to occasional disk usage peaks.\n") + , commitlog_use_hard_size_limit(this, "commitlog_use_hard_size_limit", value_status::Deprecated, true, + "Whether or not to use a hard size limit for commitlog disk usage. Default is true. Enabling this can cause latency spikes, whereas disabling this can lead to occasional disk usage peaks.\n") , commitlog_use_fragmented_entries(this, "commitlog_use_fragmented_entries", value_status::Used, true, "Whether or not to allow commitlog entries to fragment across segments, allowing for larger entry sizes.\n") /** diff --git a/test/boost/commitlog_test.cc b/test/boost/commitlog_test.cc index c3ea81cfcd38..26c11b7669a6 100644 --- a/test/boost/commitlog_test.cc +++ b/test/boost/commitlog_test.cc @@ -309,6 +309,7 @@ SEASTAR_TEST_CASE(test_commitlog_delete_when_over_disk_limit) { cfg.commitlog_segment_size_in_mb = max_size_mb; cfg.commitlog_total_space_in_mb = 1; cfg.commitlog_sync_period_in_ms = 1; + cfg.allow_going_over_size_limit = true; return cl_test(cfg, [](commitlog& log) { auto sem = make_lw_shared(0); auto segments = make_lw_shared>(); From 2ac9ed20737a36bdeb41d048294344d2971896dd Mon Sep 17 00:00:00 2001 From: Emil Maskovsky Date: Tue, 3 Dec 2024 17:52:22 +0100 Subject: [PATCH 201/397] raft: test the limited voters feature Test the limited voters feature by creating a cluster with 3 DCs, one of them disproportionately larger than the others. The raft majority should not be lost in case the large DC goes down. Fixes: scylladb/scylla#21915 Refs: scylladb/scylla#18793 Closes scylladb/scylladb#21901 --- test/topology_custom/test_raft_voters.py | 88 ++++++++++++++++++++++++ 1 file changed, 88 insertions(+) create mode 100644 test/topology_custom/test_raft_voters.py diff --git a/test/topology_custom/test_raft_voters.py b/test/topology_custom/test_raft_voters.py new file mode 100644 index 000000000000..4fef7a72b7fd --- /dev/null +++ b/test/topology_custom/test_raft_voters.py @@ -0,0 +1,88 @@ +# +# Copyright (C) 2024-present ScyllaDB +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# + +import asyncio +import logging +import pytest + +from cassandra.policies import WhiteListRoundRobinPolicy + +from test.pylib.manager_client import ManagerClient +from test.pylib.rest_client import read_barrier +from test.topology.conftest import cluster_con + + +@pytest.mark.asyncio +@pytest.mark.xfail(reason='issue #18793') +# Make sure the algorithm works with different number of nodes. +# Here with the "num_nodes == 1" we test that we'll only have one voter per DC, despite DC having two nodes +# (the DC1 must not have 2 voters otherwise losing it would result in the raft majority loss). +@pytest.mark.parametrize('num_nodes', [1, 3]) +async def test_raft_voters_multidc_kill_dc(manager: ManagerClient, num_nodes: int): + """ + Test the basic functionality of limited voters in a multi-DC cluster. + + All DCs should now have the same number of voters so the majority shouldn't be lost when one DC out of 3 goes out, + even if the DC is disproportionally larger than the other two. + + Arrange: + - create 2 smaller DCs and one large DC + - the sum of nodes in the smaller DCs is equal to the number of nodes of the large DC + Act: + - kill all the nodes in the large DC + - this would cause the loss of majority in the cluster without the limited voters feature + Assert: + - test the group0 didn't lose the majority by sending a read barrier request to one of the smaller DCs + - this should work as the large DC should now have the same number of voters as each of the smaller DCs + (so the majority shouldn't be lost in this scenario) + """ + + config = { + 'endpoint_snitch': 'GossipingPropertyFileSnitch', + } + dc_setup = [ + { + 'property_file': {'dc': 'dc1', 'rack': 'rack1'}, + # The large DC has 2x the number of nodes of the smaller DCs + 'num_nodes': 2 * num_nodes, + }, + { + 'property_file': {'dc': 'dc2', 'rack': 'rack2'}, + 'num_nodes': num_nodes, + }, + { + 'property_file': {'dc': 'dc3', 'rack': 'rack3'}, + 'num_nodes': num_nodes, + }, + ] + + # Arrange: create DCs / servers + + dc_servers = [] + for dc in dc_setup: + logging.info(f"Creating {dc['property_file']['dc']} with {dc['num_nodes']} nodes") + dc_servers.append(await manager.servers_add(dc['num_nodes'], config=config, + property_file=dc['property_file'])) + + assert len(dc_servers) == len(dc_setup) + + logging.info('Creating connections to all DCs') + dc_cqls = [] + for servers in dc_servers: + dc_cqls.append(cluster_con([servers[0].ip_addr], 9042, False, + load_balancing_policy=WhiteListRoundRobinPolicy([servers[0].ip_addr])).connect()) + + assert len(dc_cqls) == len(dc_servers) + + # Act: Kill all nodes in dc1 + + logging.info('Killing all nodes in dc1') + await asyncio.gather(*(manager.server_stop_gracefully(srv.server_id) for srv in dc_servers[0])) + + # Assert: Verify that the majority has not been lost (we can change the topology) + + logging.info('Executing read barrier') + await read_barrier(manager.api, dc_servers[1][0].ip_addr) From baaee28c072c5635bdbb919185bc960588f7e488 Mon Sep 17 00:00:00 2001 From: Asias He Date: Mon, 6 Jan 2025 10:39:30 +0800 Subject: [PATCH 202/397] storage_service: Add tablet migration log So that both mutation and file streaming will have the same log for tablet streaming which simplifies the dtest checking. Closes scylladb/scylladb#22176 --- service/storage_service.cc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/service/storage_service.cc b/service/storage_service.cc index bb9e8ea906c2..6f9e19f0451b 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -6112,7 +6112,9 @@ future<> storage_service::stream_tablet(locator::global_tablet_id tablet) { ranges_per_endpoint[r.host].emplace_back(range); } streamer->add_rx_ranges(table.schema()->ks_name(), std::move(ranges_per_endpoint)); + slogger.debug("Streaming for tablet migration of {} started table={}.{} range={}", tablet, table.schema()->ks_name(), table.schema()->cf_name(), range); co_await streamer->stream_async(); + slogger.info("Streaming for tablet migration of {} finished table={}.{} range={}", tablet, table.schema()->ks_name(), table.schema()->cf_name(), range); } // If new pending tablet replica needs splitting, streaming waits for it to complete. From 748d30a34d6b3bc55fc5e896559cf396f3ea1329 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Thu, 19 Dec 2024 14:40:11 +0200 Subject: [PATCH 203/397] tools: toolchain: simplify non-emulated build procedure Avoid using temporary names and instead treat the final image tag as a temporary. The new procedure is more or less remote-final := local-x86_64 local-aarch64 += remote-final remote-final := local-aarch64 (which now contains the x86_64 image too) Closes scylladb/scylladb#21981 --- tools/toolchain/README.md | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tools/toolchain/README.md b/tools/toolchain/README.md index d74df4bbf3df..a8b27f697c31 100644 --- a/tools/toolchain/README.md +++ b/tools/toolchain/README.md @@ -112,14 +112,11 @@ another using native systems, which requires access to aarch64 and s390x machine 1. check out the branch containing the new toolchain name 2. Run `git submodule update --init --recursive` to make sure all the submodules are synchronized - 3. Run `podman build --no-cache --pull --tag mytag-arch -f tools/toolchain/Dockerfile .`, where mytag-arch is a new, unique tag that is different for x86 and ARM. - 4. Push the resulting images to a personal docker repository. + 3. Run `tools/toolchain/prepare --clang-build-mode INSTALL_FROM --clang-archive --disable-multiarch`. This should complete relatively quickly. 4. Now, create a multiarch image with the following: - 1. Pull the two images with `podman pull`. Let's call the two tags - `mytag-x86` and `mytag-arm`. - 2. Create the new toolchain manifest with `podman manifest create $( Date: Thu, 19 Dec 2024 15:17:40 +0800 Subject: [PATCH 204/397] repair: Remove repair_task_info only when repair is finished In case of error, repair will be moved into the end_repair stage. We should not remove repair_task_info in this case because the repair task requested by the user is not finished yet. To fix, we should remove repair_task_info at the end of repair stage. Tests are added to ensure failed repair is not reported as finished. Closes scylladb/scylladb#21973 --- service/topology_coordinator.cc | 2 +- test/pylib/rest_client.py | 4 +- .../test_tablet_repair_scheduler.py | 43 ++++++++++++++++++- 3 files changed, 45 insertions(+), 4 deletions(-) diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index f9c2f734062a..8a1a1b4aa008 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -1579,6 +1579,7 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { rtlogger.debug("Will set tablet {} stage to {}", gid, locator::tablet_transition_stage::end_repair); auto update = get_mutation_builder() .set_stage(last_token, locator::tablet_transition_stage::end_repair) + .del_repair_task_info(last_token) .del_session(last_token); if (valid) { auto time = tinfo.repair_task_info.sched_time; @@ -1593,7 +1594,6 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { _tablets.erase(gid); updates.emplace_back(get_mutation_builder() .del_transition(last_token) - .del_repair_task_info(last_token) .build()); } } diff --git a/test/pylib/rest_client.py b/test/pylib/rest_client.py index 0298023b6f4e..e5ae15817313 100644 --- a/test/pylib/rest_client.py +++ b/test/pylib/rest_client.py @@ -273,8 +273,8 @@ async def del_tablet_replica(self, node_ip: str, ks: str, table: str, host: Host "token": str(token) }) - async def tablet_repair(self, node_ip: str, ks: str, table: str, token : int) -> None: - await self.client.post(f"/storage_service/tablets/repair", host=node_ip, params={ + async def tablet_repair(self, node_ip: str, ks: str, table: str, token : int, timeout: Optional[float] = None) -> None: + await self.client.post(f"/storage_service/tablets/repair", host=node_ip, timeout=timeout, params={ "ks": ks, "table": table, "tokens": str(token) diff --git a/test/topology_custom/test_tablet_repair_scheduler.py b/test/topology_custom/test_tablet_repair_scheduler.py index b3eec4959edd..6079de8984b0 100644 --- a/test/topology_custom/test_tablet_repair_scheduler.py +++ b/test/topology_custom/test_tablet_repair_scheduler.py @@ -6,7 +6,7 @@ from test.pylib.manager_client import ManagerClient from test.topology.conftest import skip_mode -from test.pylib.repair import load_tablet_repair_time, create_table_insert_data_for_repair +from test.pylib.repair import load_tablet_repair_time, create_table_insert_data_for_repair, get_tablet_task_id from test.pylib.rest_client import inject_error_one_shot import pytest @@ -112,3 +112,44 @@ async def test_tablet_repair_error_and_retry(manager: ManagerClient): await inject_error_one_shot_on(manager, "repair_tablet_fail_on_rpc_call", servers) await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token) await inject_error_off(manager, "repair_tablet_fail_on_rpc_call", servers) + +@pytest.mark.asyncio +@skip_mode('release', 'error injections are not supported in release mode') +async def test_tablet_repair_error_not_finish(manager: ManagerClient): + servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager) + + token = -1 + # Repair should not finish with error + await inject_error_on(manager, "repair_tablet_fail_on_rpc_call", servers) + try: + await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token, timeout=10) + assert False # Check the tablet repair is not supposed to finish + except TimeoutError: + logger.info("Repair timeout as expected") + await inject_error_off(manager, "repair_tablet_fail_on_rpc_call", servers) + +@pytest.mark.asyncio +@skip_mode('release', 'error injections are not supported in release mode') +async def test_tablet_repair_error_delete(manager: ManagerClient): + servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager) + + token = -1 + async def repair_task(): + await inject_error_on(manager, "repair_tablet_fail_on_rpc_call", servers) + # Check failed repair request can be deleted + await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token, timeout=900) + + async def del_repair_task(): + tablet_task_id = None + while tablet_task_id == None: + tablet_task_id = await get_tablet_task_id(cql, hosts[0], table_id, token) + status = None + while status == None: + try: + status = await manager.api.get_task_status(servers[0].ip_addr, tablet_task_id) + except: + status == None + await manager.api.abort_task(servers[0].ip_addr, tablet_task_id) + + await asyncio.gather(repair_task(), del_repair_task()); + await inject_error_off(manager, "repair_tablet_fail_on_rpc_call", servers) From d719f423e5d2057a67ae3a8a354d93b6e5daab9f Mon Sep 17 00:00:00 2001 From: Asias He Date: Mon, 2 Dec 2024 20:01:12 +0800 Subject: [PATCH 205/397] config: Enable enable_small_table_optimization_for_rbno by default Since the problematic dtests are with the enable_small_table_optimization_for_rbno turn off now, we can enable the flag by default. https://github.com/scylladb/scylla-dtest/pull/5383 Refs: #19131 Closes scylladb/scylladb#21861 --- db/config.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/db/config.cc b/db/config.cc index 9010ecb5583d..90c5e76679eb 100644 --- a/db/config.cc +++ b/db/config.cc @@ -1060,7 +1060,7 @@ db::config::config(std::shared_ptr exts) , repair_multishard_reader_enable_read_ahead(this, "repair_multishard_reader_enable_read_ahead", liveness::LiveUpdate, value_status::Used, false, "The multishard reader has a read-ahead feature to improve latencies of range-scans. This feature can be detrimental when the multishard reader is used under repair, as is the case in repair in mixed-shard clusters." " This know allows disabling this read-ahead (default), this can help the performance of mixed-shard repair (including RBNO).") - , enable_small_table_optimization_for_rbno(this, "enable_small_table_optimization_for_rbno", liveness::LiveUpdate, value_status::Used, false, "Set true to enable small table optimization for repair based node operations") + , enable_small_table_optimization_for_rbno(this, "enable_small_table_optimization_for_rbno", liveness::LiveUpdate, value_status::Used, true, "Set true to enable small table optimization for repair based node operations") , ring_delay_ms(this, "ring_delay_ms", value_status::Used, 30 * 1000, "Time a node waits to hear from other nodes before joining the ring in milliseconds. Same as -Dcassandra.ring_delay_ms in cassandra.") , shadow_round_ms(this, "shadow_round_ms", value_status::Used, 300 * 1000, "The maximum gossip shadow round time. Can be used to reduce the gossip feature check time during node boot up.") , fd_max_interval_ms(this, "fd_max_interval_ms", value_status::Used, 2 * 1000, "The maximum failure_detector interval time in milliseconds. Interval larger than the maximum will be ignored. Larger cluster may need to increase the default.") From 115005d86388956029a206be168527c8e528594c Mon Sep 17 00:00:00 2001 From: Emil Maskovsky Date: Wed, 11 Dec 2024 16:12:46 +0100 Subject: [PATCH 206/397] raft: refactor the voters api to allow enabling voters The raft voters api implementation only allowed to make a node to be a non-voter, but for the "limited voters" feature we need to also have the option to make the node a voter (from within the topology coordinator). Modifying the api to allow both adding and removing voters. This in particular tries to simplify the API by not having to add another set of new functions to make a voter, but having a single setter that allows to modify the node configuration to either become a voter or a non-voter. Fixes: scylladb/scylladb#21914 Refs: scylladb/scylladb#18793 Closes scylladb/scylladb#21899 --- service/raft/raft_group0.cc | 52 +++++++++++++++++---------------- service/raft/raft_group0.hh | 19 +++++++----- service/storage_service.cc | 8 ++--- service/topology_coordinator.cc | 6 ++-- 4 files changed, 45 insertions(+), 40 deletions(-) diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index f6dc84a72f6e..1b41c62db073 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -547,17 +547,17 @@ future<> raft_group0::join_group0(std::vector seeds, shared_p group0_log.info("server {} joined group 0 with group id {}", my_id, group0_id); } -shared_ptr raft_group0::make_legacy_handshaker(bool can_vote) { +shared_ptr raft_group0::make_legacy_handshaker(can_vote can_vote) { struct legacy_handshaker : public group0_handshaker { service::raft_group0& _group0; netw::messaging_service& _ms; - bool _can_vote; + service::can_vote _can_vote; - legacy_handshaker(service::raft_group0& group0, netw::messaging_service& ms, bool can_vote) - : _group0(group0) - , _ms(ms) - , _can_vote(can_vote) - {} + legacy_handshaker(service::raft_group0& group0, netw::messaging_service& ms, service::can_vote can_vote) + : _group0(group0) + , _ms(ms) + , _can_vote(can_vote) { + } future<> pre_server_start(const group0_info& info) override { // Nothing to do in this step @@ -569,7 +569,8 @@ shared_ptr raft_group0::make_legacy_handshaker(bool auto my_id = _group0.load_my_id(); raft::server_address my_addr{my_id, {}}; try { - co_await ser::group0_rpc_verbs::send_group0_modify_config(&_ms, locator::host_id{g0_info.id.uuid()}, timeout, g0_info.group0_id, {{my_addr, _can_vote}}, {}); + co_await ser::group0_rpc_verbs::send_group0_modify_config( + &_ms, locator::host_id{g0_info.id.uuid()}, timeout, g0_info.group0_id, {{my_addr, static_cast(_can_vote)}}, {}); co_return true; } catch (std::runtime_error& e) { group0_log.warn("failed to modify config at peer {}: {}. Retrying.", g0_info.id, e.what()); @@ -795,30 +796,31 @@ future<> raft_group0::become_nonvoter(abort_source& as, std::optional raft_group0::make_nonvoter(raft::server_id node, abort_source& as, std::optional timeout) { - co_return co_await make_nonvoters({node}, as, timeout); +future<> raft_group0::set_voter_status(raft::server_id node, can_vote can_vote, abort_source& as, std::optional timeout) { + co_return co_await set_voters_status({node}, can_vote, as, timeout); } -future<> raft_group0::make_nonvoters(const std::unordered_set& nodes, abort_source& as, - std::optional timeout) -{ +future<> raft_group0::set_voters_status( + const std::unordered_set& nodes, can_vote can_vote, abort_source& as, std::optional timeout) { if (!(co_await raft_upgrade_complete())) { - on_internal_error(group0_log, "called make_nonvoters before Raft upgrade finished"); + on_internal_error(group0_log, "called set_voter_status before Raft upgrade finished"); } if (nodes.empty()) { co_return; } - group0_log.info("making servers {} non-voters...", nodes); + const std::string_view status_str = can_vote ? "voters" : "non-voters"; + + group0_log.info("making servers {} {}...", nodes, status_str); - co_await make_raft_config_nonvoter(nodes, as, timeout); + co_await modify_raft_voter_status(nodes, can_vote, as, timeout); - group0_log.info("servers {} are now non-voters.", nodes); + group0_log.info("servers {} are now {}.", nodes, status_str); } future<> raft_group0::leave_group0() { @@ -904,9 +906,8 @@ future raft_group0::wait_for_raft() { co_return true; } -future<> raft_group0::make_raft_config_nonvoter(const std::unordered_set& ids, abort_source& as, - std::optional timeout) -{ +future<> raft_group0::modify_raft_voter_status( + const std::unordered_set& ids, can_vote can_vote, abort_source& as, std::optional timeout) { static constexpr auto max_retry_period = std::chrono::seconds{1}; auto retry_period = std::chrono::milliseconds{10}; @@ -915,14 +916,15 @@ future<> raft_group0::make_raft_config_nonvoter(const std::unordered_set add; add.reserve(ids.size()); - std::transform(ids.begin(), ids.end(), std::back_inserter(add), - [] (raft::server_id id) { return raft::config_member{{id, {}}, false}; }); + std::transform(ids.begin(), ids.end(), std::back_inserter(add), [can_vote] (raft::server_id id) { + return raft::config_member{{id, {}}, static_cast(can_vote)}; + }); try { co_await _raft_gr.group0_with_timeouts().modify_config(std::move(add), {}, &as, timeout); co_return; } catch (const raft::commit_status_unknown& e) { - group0_log.info("make_raft_config_nonvoter({}): modify_config returned \"{}\", retrying", ids, e); + group0_log.info("modify_raft_voter_config({}): modify_config returned \"{}\", retrying", ids, e); } retry_period *= 2; if (retry_period > max_retry_period) { @@ -1649,7 +1651,7 @@ future<> raft_group0::do_upgrade_to_group0(group0_upgrade_state start_state, ser if (!joined_group0()) { upgrade_log.info("Joining group 0..."); - auto handshaker = make_legacy_handshaker(true); // Voter + auto handshaker = make_legacy_handshaker(can_vote::yes); // Voter co_await join_group0(co_await _sys_ks.load_peers(), std::move(handshaker), ss, qp, mm, _sys_ks, topology_change_enabled); } else { upgrade_log.info( diff --git a/service/raft/raft_group0.hh b/service/raft/raft_group0.hh index 257ead64fe7e..3c6415bfa77d 100644 --- a/service/raft/raft_group0.hh +++ b/service/raft/raft_group0.hh @@ -27,6 +27,9 @@ class migration_manager; class raft_group0_client; class storage_service; +struct can_vote_tag {}; +using can_vote = bool_class; + // Wrapper for `discovery` which persists the learned peers on disk. class persistent_discovery { discovery _discovery; @@ -220,18 +223,17 @@ public: // `wait_for_raft` must've also been called earlier and returned `true`. future<> become_nonvoter(abort_source& as, std::optional timeout = std::nullopt); - // Make the given server, other than us, a non-voter in group 0. + // Set the voter status of the given server, other than us, in group 0. // // Assumes we've finished the startup procedure (`setup_group0()` finished earlier). // `wait_for_raft` must've also been called earlier and returned `true`. - future<> make_nonvoter(raft::server_id, abort_source&, std::optional timeout = std::nullopt); + future<> set_voter_status(raft::server_id, can_vote, abort_source&, std::optional timeout = std::nullopt); - // Make the given servers, other than us, a non-voter in group 0. + // Set the voter status of the given servers, other than us, in group 0. // // Assumes we've finished the startup procedure (`setup_group0()` finished earlier). // `wait_for_raft` must've also been called earlier and returned `true`. - future<> make_nonvoters(const std::unordered_set&, abort_source&, - std::optional timeout = std::nullopt); + future<> set_voters_status(const std::unordered_set&, can_vote, abort_source&, std::optional timeout = std::nullopt); // Remove ourselves from group 0. // @@ -271,7 +273,7 @@ public: // It is meant to be used as a fallback when a proper handshake procedure // cannot be used (e.g. when completing the upgrade or group0 procedures // or when joining an old cluster which does not support JOIN_NODE RPC). - shared_ptr make_legacy_handshaker(bool can_vote); + shared_ptr make_legacy_handshaker(can_vote can_vote); // Waits until all upgrade to raft group 0 finishes and all nodes switched // to use_post_raft_procedures. @@ -376,9 +378,10 @@ private: // (we could then try restarting the server internally). future<> start_server_for_group0(raft::group_id group0_id, service::storage_service& ss, cql3::query_processor& qp, service::migration_manager& mm, bool topology_change_enabled); - // Make the given server a non-voter in Raft group 0 configuration. + // Modify the given server voter status in Raft group 0 configuration. // Retries on raft::commit_status_unknown. - future<> make_raft_config_nonvoter(const std::unordered_set&, abort_source& as, std::optional timeout = std::nullopt); + future<> modify_raft_voter_status( + const std::unordered_set&, can_vote, abort_source& as, std::optional timeout = std::nullopt); // Returns true if raft is enabled future use_raft(); diff --git a/service/storage_service.cc b/service/storage_service.cc index 6f9e19f0451b..242050b92782 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -1812,7 +1812,7 @@ future<> storage_service::join_topology(sharded // if the node is bootstrapped the function will do nothing since we already created group0 in main.cc ::shared_ptr handshaker = raft_topology_change_enabled() ? ::make_shared(*this, join_params) - : _group0->make_legacy_handshaker(false); + : _group0->make_legacy_handshaker(can_vote::no); co_await _group0->setup_group0(_sys_ks.local(), initial_contact_nodes, std::move(handshaker), raft_replace_info, *this, _qp, _migration_manager.local(), raft_topology_change_enabled()); @@ -4115,7 +4115,7 @@ future<> storage_service::raft_removenode(locator::host_id host_id, locator::hos } try { // Make non voter during request submission for better HA - co_await _group0->make_nonvoters(ignored_ids, _group0_as, raft_timeout{}); + co_await _group0->set_voters_status(ignored_ids, can_vote::no, _group0_as, raft_timeout{}); co_await _group0->client().add_entry(std::move(g0_cmd), std::move(guard), _group0_as, raft_timeout{}); } catch (group0_concurrent_modification&) { rtlogger.info("removenode: concurrent operation is detected, retrying."); @@ -4198,7 +4198,7 @@ future<> storage_service::removenode(locator::host_id host_id, locator::host_id_ // but before removing it group 0, group 0's availability won't be reduced. if (is_group0_member && ss._group0->is_member(raft_id, true)) { slogger.info("removenode[{}]: making node {} a non-voter in group 0", uuid, raft_id); - ss._group0->make_nonvoter(raft_id, ss._group0_as).get(); + ss._group0->set_voter_status(raft_id, can_vote::no, ss._group0_as).get(); slogger.info("removenode[{}]: made node {} a non-voter in group 0", uuid, raft_id); } @@ -6859,7 +6859,7 @@ future storage_service::join_node_request_handler(join try { // Make replaced node and ignored nodes non voters earlier for better HA - co_await _group0->make_nonvoters(ignored_nodes_from_join_params(params), _group0_as, raft_timeout{}); + co_await _group0->set_voters_status(ignored_nodes_from_join_params(params), can_vote::no, _group0_as, raft_timeout{}); co_await _group0->client().add_entry(std::move(g0_cmd), std::move(guard), _group0_as, raft_timeout{}); break; } catch (group0_concurrent_modification&) { diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index 8a1a1b4aa008..f2b3a5a68a11 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -2139,7 +2139,7 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { // FIXME: removenode may be aborted and the already dead node can be resurrected. We should consider // restoring its voter state on the recovery path. if (node.rs->state == node_state::removing) { - co_await _group0.make_nonvoter(node.id, _as); + co_await _group0.set_voter_status(node.id, can_vote::no, _as); } // If we decommission a node when the number of nodes is even, we make it a non-voter early. @@ -2156,7 +2156,7 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { "giving up leadership"); co_await step_down_as_nonvoter(); } else { - co_await _group0.make_nonvoter(node.id, _as); + co_await _group0.set_voter_status(node.id, can_vote::no, _as); } } } @@ -2164,7 +2164,7 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { // We make a replaced node a non-voter early, just like a removed node. auto replaced_node_id = parse_replaced_node(node.req_param); if (_group0.is_member(replaced_node_id, true)) { - co_await _group0.make_nonvoter(replaced_node_id, _as); + co_await _group0.set_voter_status(replaced_node_id, can_vote::no, _as); } } utils::get_local_injector().inject("crash_coordinator_before_stream", [] { abort(); }); From d04f376227475c4dd72faf4729e805bc31bb6454 Mon Sep 17 00:00:00 2001 From: Wojciech Mitros Date: Mon, 2 Dec 2024 22:16:55 +0100 Subject: [PATCH 207/397] mv: add an experimental feature for creating views using tablets We still have a number of issues to be solved for views with tablets. Until they are fixed, we should prevent users from creating them, and use the vnode-based views instead. This patch prepares the feature for enabling views with tablets. The feature is disabled by default, but currently it has no effect. After all tests are adjusted to use the feature, we should depend on the feature for deciding whether we can create materialized views in tablet-enabled keyspaces. The unit tests are adjusted to enable this feature explicitly, and it's also added to the scylla sstable tool config - this tool treats all tables as if they were tablet-based (surprisingly, with SimpleStrategy), so for it to work on views, the new feature must be enabled. Refs scylladb/scylladb#21832 Closes scylladb/scylladb#21833 --- cql3/statements/create_view_statement.cc | 1 + db/config.cc | 1 + db/config.hh | 1 + gms/feature_service.cc | 3 +++ gms/feature_service.hh | 1 + test/cqlpy/run.py | 1 + test/cqlpy/suite.yaml | 1 + test/pylib/scylla_cluster.py | 3 ++- test/rest_api/suite.yaml | 1 + tools/schema_loader.cc | 2 ++ 10 files changed, 14 insertions(+), 1 deletion(-) diff --git a/cql3/statements/create_view_statement.cc b/cql3/statements/create_view_statement.cc index da7f1ac40dbe..592a6da434c6 100644 --- a/cql3/statements/create_view_statement.cc +++ b/cql3/statements/create_view_statement.cc @@ -29,6 +29,7 @@ #include "gms/feature_service.hh" #include "db/view/view.hh" #include "service/migration_manager.hh" +#include "replica/database.hh" namespace cql3 { diff --git a/db/config.cc b/db/config.cc index 90c5e76679eb..481ec3c4e96d 100644 --- a/db/config.cc +++ b/db/config.cc @@ -1447,6 +1447,7 @@ std::map db::experimental_feature {"broadcast-tables", feature::BROADCAST_TABLES}, {"keyspace-storage-options", feature::KEYSPACE_STORAGE_OPTIONS}, {"tablets", feature::UNUSED}, + {"views-with-tablets", feature::VIEWS_WITH_TABLETS} }; } diff --git a/db/config.hh b/db/config.hh index 64f9ffce3a43..6e001c832a45 100644 --- a/db/config.hh +++ b/db/config.hh @@ -109,6 +109,7 @@ struct experimental_features_t { ALTERNATOR_STREAMS, BROADCAST_TABLES, KEYSPACE_STORAGE_OPTIONS, + VIEWS_WITH_TABLETS }; static std::map map(); // See enum_option. static std::vector> all(); diff --git a/gms/feature_service.cc b/gms/feature_service.cc index 00f22866c164..009224591fde 100644 --- a/gms/feature_service.cc +++ b/gms/feature_service.cc @@ -82,6 +82,9 @@ feature_config feature_config_from_db_config(const db::config& cfg, std::set do_load_schemas(const db::config& cfg, std::string_view gms::feature_service feature_service(gms::feature_config_from_db_config(cfg)); feature_service.enable(feature_service.supported_feature_set()).get(); + feature_service.views_with_tablets.enable(); + sharded token_metadata; auto my_address = gms::inet_address("localhost"); From 9f639b176fe43ab4b93f5200ce4280a57bf83a68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 27 Nov 2024 13:00:40 +0100 Subject: [PATCH 208/397] db/config: increase the default value of internode_compression_zstd_min_message_size from 0 to 1024 Usually, the smaller the messsage, the higher the CPU cost per each network byte saved by compression, so it often makes sense to reserve heavier compression for bigger messages (where it can make the biggest impact for a given CPU budget) and use ligher compression for smaller messages. There is a knob -- internode_compression_zstd_min_message_size -- which excludes RPC messages below certain size from being compressed with zstd. We arbitrarily set its default to 0 bytes before. Now we want to arbitrarily set it to 1024 bytes. This is based purely on intuition and isn't backed by any solid data. Fixes scylladb/scylla-enterprise#4731 Closes scylladb/scylla-enterprise#4990 Closes scylladb/scylladb#22204 --- db/config.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/db/config.cc b/db/config.cc index 481ec3c4e96d..d6adb1e34aca 100644 --- a/db/config.cc +++ b/db/config.cc @@ -830,7 +830,7 @@ db::config::config(std::shared_ptr exts) "ZSTD compression of RPC will consume at most this fraction of each internode_compression_zstd_longterm_cpu_quota_refresh_period_ms time slice.") , internode_compression_zstd_longterm_cpu_quota_refresh_period_ms(this, "internode_compression_zstd_longterm_cpu_quota_refresh_period_ms", liveness::LiveUpdate, value_status::Used, 10000, "Advanced. ZSTD compression of RPC will consume at most internode_compression_zstd_max_longterm_cpu_fraction (plus one message) of in each time slice of this length.") - , internode_compression_zstd_min_message_size(this, "internode_compression_zstd_min_message_size", liveness::LiveUpdate, value_status::Used, 0, + , internode_compression_zstd_min_message_size(this, "internode_compression_zstd_min_message_size", liveness::LiveUpdate, value_status::Used, 1024, "Minimum RPC message size which can be compressed with ZSTD. Messages smaller than this threshold will always be compressed with LZ4. " "ZSTD has high per-message overhead, and might be a bad choice for small messages. This knob allows for some experimentation with that. ") , internode_compression_zstd_max_message_size(this, "internode_compression_zstd_max_message_size", liveness::LiveUpdate, value_status::Used, std::numeric_limits::max(), From e51b2075daccf4a52c07c7b7fc903124ed0aaa82 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 7 Jan 2025 07:01:29 +0800 Subject: [PATCH 209/397] docs/kb: correct referenced git sha1 and version number in 047ce136, we cherry-picked the change adding garbage-collection-ics.rst to the document. but it was still referencing the git sha1 and version number in enterprise. this change updates kb/garbage-collection-ics.rst, so that it * references the git commit sha1 in this repo * do not reference the version introducing this feature, as per Anna Stuchlik > As a rule, we should avoid documenting when something was > introduced or set as a default because our documentation > was versioned. Per-version information should be listed in > the release notes. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22195 --- docs/kb/garbage-collection-ics.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/kb/garbage-collection-ics.rst b/docs/kb/garbage-collection-ics.rst index cff6b0684dce..10419613c5da 100644 --- a/docs/kb/garbage-collection-ics.rst +++ b/docs/kb/garbage-collection-ics.rst @@ -31,7 +31,7 @@ The process above has several drawbacks; for example: Making Garbage Collection Efficient in ICS --------------------------------------------- -As a remedy to the known problem described above, a new process was introduced to ScyllaDB with `this commit `_. +As a remedy to the known problem described above, a new process was introduced to ScyllaDB with `this commit `_. The process inherits the cross-tier compaction idea from SAG, but instead of using a space-amplification-based trigger, it uses a tombstone-density trigger instead. It can co-exist with SAG, if enabled. @@ -51,7 +51,7 @@ procedure. How to Use It --------------- -ICS garbage collection is enabled by default starting from version 2021.1.9. +ICS garbage collection is enabled by default. As in STCS, you can use the compaction options ``tombstone_threshold`` and ``tombstone_compaction_interval`` to tweak the behavior of the GC process. From 23729beeb5dd15c2c1c942ea4100a912d5211c38 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 6 Jan 2025 10:54:13 +0800 Subject: [PATCH 210/397] docs: remove "ScyllaDB Enterprise" labels remove the "ScyllaDB Enterprise" labels in document. because there is no need to differentiate ScyllaDB Enterprise from its OSS variant, let's stop adding the "ScyllaDB Enterprise" labels to enterprise-only features. this helps to reduce the confusion. as we are still in the process of porting the enterprise features to this repo, this change does not fix scylladb/scylladb#22175. we will review the document again when completing the migration. we also take this opportunity to stop referencing "Enterprise" in the changed paragraph. Refs scylladb/scylladb#22175 Signed-off-by: Kefu Chai Closes scylladb/scylladb#22177 --- docs/cql/compaction.rst | 2 -- docs/features/workload-prioritization.rst | 2 -- docs/kb/compaction.rst | 6 +++--- .../config-change/advanced-internode-compression.rst | 2 -- docs/operating-scylla/security/security-checklist.rst | 4 ++-- 5 files changed, 5 insertions(+), 11 deletions(-) diff --git a/docs/cql/compaction.rst b/docs/cql/compaction.rst index 3407220abb35..c9a281a3f9c1 100644 --- a/docs/cql/compaction.rst +++ b/docs/cql/compaction.rst @@ -257,8 +257,6 @@ The following options only apply to IncrementalCompactionStrategy: ``space_amplification_goal`` (default: null) -:label-tip:`ScyllaDB Enterprise` - .. versionadded:: 2020.1.6 This is a threshold of the ratio of the sum of the sizes of the two largest tiers to the size of the largest tier, diff --git a/docs/features/workload-prioritization.rst b/docs/features/workload-prioritization.rst index b2b17cf40b19..86fb28a51654 100644 --- a/docs/features/workload-prioritization.rst +++ b/docs/features/workload-prioritization.rst @@ -2,8 +2,6 @@ Workload Prioritization ======================== -:label-tip:`ScyllaDB Enterprise` - In a typical database there are numerous workloads running at the same time. Each workload type dictates a different acceptable level of latency and throughput. For example, consider the following two workloads: diff --git a/docs/kb/compaction.rst b/docs/kb/compaction.rst index c14792efcae9..87495c01e592 100644 --- a/docs/kb/compaction.rst +++ b/docs/kb/compaction.rst @@ -43,7 +43,7 @@ A compaction strategy is what determines which of the SSTables will be compacted * `Size-tiered compaction strategy (STCS)`_ - (default setting) triggered when the system has enough similarly sized SSTables. * `Leveled compaction strategy (LCS)`_ - the system uses small, fixed-size (by default 160 MB) SSTables divided into different levels and lowers both Read and Space Amplification. -* :ref:`Incremental compaction strategy (ICS) ` - :label-tip:`ScyllaDB Enterprise` Uses runs of sorted, fixed size (by default 1 GB) SSTables in a similar way that LCS does, organized into size-tiers, similar to STCS size-tiers. If you are an Enterprise customer ICS is an updated strategy meant to replace STCS. It has the same read and write amplification, but has lower space amplification due to the reduction of temporary space overhead is reduced to a constant manageable level. +* :ref:`Incremental compaction strategy (ICS) ` - Uses runs of sorted, fixed size (by default 1 GB) SSTables in a similar way that LCS does, organized into size-tiers, similar to STCS size-tiers. ICS is an updated strategy meant to replace STCS. It has the same read and write amplification, but has lower space amplification due to the reduction of temporary space overhead is reduced to a constant manageable level. * `Time-window compaction strategy (TWCS)`_ - designed for time series data and puts data in time order. TWCS uses STCS to prevent accumulating SSTables in a window not yet closed. When the window closes, TWCS works towards reducing the SSTables in a time window to one. How to Set a Compaction Strategy @@ -115,8 +115,8 @@ Likewise, when :term:`bootstrapping` a new node, SSTables are streame .. _incremental-compaction-strategy-ics: -Incremental Compaction Strategy (ICS) :label-tip:`ScyllaDB Enterprise` ------------------------------------------------------------------------- +Incremental Compaction Strategy (ICS) +------------------------------------- .. versionadded:: 2019.1.4 diff --git a/docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst b/docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst index 101917bcfa01..aff3101a2328 100644 --- a/docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst +++ b/docs/operating-scylla/procedures/config-change/advanced-internode-compression.rst @@ -2,8 +2,6 @@ Advanced Internode (RPC) Compression ========================================== -:label-tip:`Available with the Premium plan` - Internode (RPC) compression controls whether traffic between nodes is compressed. If enabled, it reduces network bandwidth usage. diff --git a/docs/operating-scylla/security/security-checklist.rst b/docs/operating-scylla/security/security-checklist.rst index a71e712de49f..3e65c444d451 100644 --- a/docs/operating-scylla/security/security-checklist.rst +++ b/docs/operating-scylla/security/security-checklist.rst @@ -41,8 +41,8 @@ Configure ScyllaDB to use TLS/SSL for all the connections. Use TLS/SSL to encryp * :doc:`Encryption Data in Transit Node to Node ` -Encryption at Rest :label-tip:`ScyllaDB Enterprise` -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +Encryption at Rest +~~~~~~~~~~~~~~~~~~ Encryption at Rest is available in `ScyllaDB Enterprise `_. Encryption at Rest protects the privacy of your user's data, reduces the risk of data breaches, and helps meet regulatory requirements. From e6efaa3b7361ffbfaec7e683cd78b8bb0d999662 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Mon, 30 Dec 2024 10:23:33 +0200 Subject: [PATCH 211/397] Update seastar submodule * seastar 3133ecdd...a9bef537 (24): > file: add file_system_space > future: avoid inheriting from future payload type > treewide: include fmt/ostream.h for using fmt::print() > build: remove messages used for debugging > demos: Rename websocket demo to websocket_server demo > demos: Add a way to set port from cmd line in websocket demo > tls: Add optional builder + future-wait to cert reload callback + expose rebuild > rwlock: add try_hold_{read,write}_lock methods > json: add moving push to json_list > github: add a step to build "check-include-style" > build: add a target for checking include style > scheduling_group: use map for key configs instead of vector > scheduling_group: fix indentation > scheduling_group: fix race between scheduling group and key creation > http: Make request writing functions public > http: Expose connection_factory implementations > metrics: Use separate type for shared metadata > file: unexpected throw from inside noexcept > metrics: Internalize metric label sets > thread: optimize maybe_yield > reactor: fix crash in pending registration task after poller dtor > net: Fix ipv6 socket_address comparision > reactor, linux-aio: factor out get_smp_count() lambda > reactor, linux-aio: restore "available_aio" meaning after "reserve_iocbs" Fixed usage of seastar metric label sets due to: scylladb/seastar@733420d57 Merge 'metrics: Internalize metric label sets' from Stephan Dollberg Closes scylladb/scylladb#22076 --- seastar | 2 +- test/boost/group0_test.cc | 2 +- test/boost/query_processor_test.cc | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/seastar b/seastar index 3133ecdd6599..a9bef537696b 160000 --- a/seastar +++ b/seastar @@ -1 +1 @@ -Subproject commit 3133ecdd6599469672f1d7cbffdd92606f2b132c +Subproject commit a9bef537696b556104d42848629756a8a4b198ca diff --git a/test/boost/group0_test.cc b/test/boost/group0_test.cc index 5a6fc27f7daf..15f52ddbb7d6 100644 --- a/test/boost/group0_test.cc +++ b/test/boost/group0_test.cc @@ -46,7 +46,7 @@ SEASTAR_TEST_CASE(test_abort_server_on_background_error) { auto get_metric_ui64 = [&](sstring name) { const auto& value_map = seastar::metrics::impl::get_value_map(); const auto& metric_family = value_map.at("raft_group0_" + name); - const auto& registered_metric = metric_family.at({{"shard", "0"}}); + const auto& registered_metric = metric_family.at(make_lw_shared({{"shard", "0"}})); return (*registered_metric)().ui(); }; diff --git a/test/boost/query_processor_test.cc b/test/boost/query_processor_test.cc index 96b6e42247f3..e1d72dbe3447 100644 --- a/test/boost/query_processor_test.cc +++ b/test/boost/query_processor_test.cc @@ -184,8 +184,8 @@ std::unordered_map get_query_metrics() { const auto values = all_metrics->values[distance(cbegin(all_metadata), qp_group)]; std::vector labels; for (const auto& metric : qp_group->metrics) { - const auto found = metric.id.labels().find("consistency_level"); - BOOST_REQUIRE(found != metric.id.labels().cend()); + const auto found = metric.labels().find("consistency_level"); + BOOST_REQUIRE(found != metric.labels().cend()); labels.push_back(found->second); } BOOST_REQUIRE(values.size() == level_count); From de8253b98af6e34b1b125a7f1ff148cb13323f2b Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Wed, 1 Jan 2025 12:11:35 +0200 Subject: [PATCH 212/397] types: explicitly instantiate map_type_impl::deserialize() The definition of the template is in a source translation unit, but there are also uses outside the translation unit. Without lto/pgo it worked due to the definition in the translation unit, but with lto/pgo we can presume the definition was inlined, so callers outside the translation unit did not have anything to link with. Fix by explicitly instantiating the template function. Closes scylladb/scylladb#22136 --- types/types.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/types/types.cc b/types/types.cc index 62928c8543d9..5284d92a0e8c 100644 --- a/types/types.cc +++ b/types/types.cc @@ -1194,6 +1194,7 @@ map_type_impl::deserialize(View in) const { return make_value(std::move(m)); } template data_value map_type_impl::deserialize<>(ser::buffer_view) const; +template data_value map_type_impl::deserialize<>(managed_bytes_view) const; template static void validate_aux(const map_type_impl& t, View v) { From f41b030fddb350199a6cc0db7a8879c0c488afc5 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 7 Jan 2025 11:58:58 +0800 Subject: [PATCH 213/397] repair: do not include unused header this unused include was identifier by clang-include-cleaner. after auditing task_manager_module.hh, the report has been confirmed. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22200 --- repair/task_manager_module.hh | 1 - 1 file changed, 1 deletion(-) diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh index 01611468a7f3..c801ca1c3e83 100644 --- a/repair/task_manager_module.hh +++ b/repair/task_manager_module.hh @@ -8,7 +8,6 @@ #pragma once -#include "gms/inet_address.hh" #include "node_ops/node_ops_ctl.hh" #include "repair/repair.hh" #include "streaming/stream_reason.hh" From a2436f139f400199763bbeba9a48f2544a1ee9aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Tue, 7 Jan 2025 10:05:57 -0500 Subject: [PATCH 214/397] docs/dev: review-checklist.md: expand the guide for good commit log Closes scylladb/scylladb#22214 --- docs/dev/review-checklist.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/dev/review-checklist.md b/docs/dev/review-checklist.md index fe8c0c3b62ef..91475738444b 100644 --- a/docs/dev/review-checklist.md +++ b/docs/dev/review-checklist.md @@ -29,6 +29,17 @@ guidelines: introduces it. - Patches should be self-contained and do one thing. - Each patch should have a descriptive commit log. + The patch subject should have to following form: "module: changes". + The "module" is the module (usually a directory) that the changes apply + to. Multiple ones can be listed, concatenated with ",". If a patch + applies to the entire tree, use the "tree:" prefix. + In certain cases, the "module: " prefix can be omitted altogether, e.g. + when changing files in the root directory. + The "changes" is a short summary of the changes (similar to a title). + Avoid generic patch titles like: "sstables: fix the bug", instead make + it specific to the patch: "sstables: close file-descriptor on error". + Try to limit lines in the commit log to 72 chars width. This is not + always possible, especially for the subject line. - The patch description should not assume the reviewer is an expert in the matter. It should include enough context that a non-expert can understand, on their way to becoming From 35316a40c8699007fd1e6bc60d4525d6fd60c6cc Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Mon, 16 Dec 2024 09:58:33 +0200 Subject: [PATCH 215/397] service/storage_proxy: consider all replicas participating in write for MV backpressure replica writes are delayed according to the view update backlog in order to apply backpressure and reduce the rate of incoming base writes when the backlog is large, allowing slow replicas to catch up. previously the backlog calculation considered only the pending targets, excluding targets that replied successfuly, probably due to confusion in the code. instead, we want to consider the backlog of all the targets participating in the write. Fixes scylladb/scylladb#21672 Closes scylladb/scylladb#21935 --- service/storage_proxy.cc | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 6265191f121f..851457ca8e0a 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -1510,7 +1510,7 @@ class abstract_write_response_handler : public seastar::enable_shared_from_this< size_t _total_block_for = 0; db::write_type _type; std::unique_ptr _mutation_holder; - host_id_vector_replica_set _targets; // who we sent this mutation to + host_id_vector_replica_set _targets; // who we sent this mutation to and still pending response // added dead_endpoints as a member here as well. This to be able to carry the info across // calls in helper methods in a convenient way. Since we hope this will be empty most of the time // it should not be a huge burden. (flw) @@ -1528,6 +1528,7 @@ class abstract_write_response_handler : public seastar::enable_shared_from_this< timer _expire_timer; service_permit _permit; // holds admission permit until operation completes db::per_partition_rate_limit::info _rate_limit_info; + db::view::update_backlog _view_backlog; // max view update backlog of all participating targets protected: virtual bool waited_for(locator::host_id from) = 0; @@ -1549,7 +1550,7 @@ class abstract_write_response_handler : public seastar::enable_shared_from_this< , _effective_replication_map_ptr(std::move(erm)) , _trace_state(trace_state), _cl(cl), _type(type), _mutation_holder(std::move(mh)), _targets(std::move(targets)), _dead_endpoints(std::move(dead_endpoints)), _stats(stats), _expire_timer([this] { timeout_cb(); }), _permit(std::move(permit)), - _rate_limit_info(rate_limit_info) { + _rate_limit_info(rate_limit_info), _view_backlog(max_backlog()) { // original comment from cassandra: // during bootstrap, include pending endpoints in the count // or we may fail the consistency level guarantees (see #833, #8058) @@ -1738,8 +1739,7 @@ class abstract_write_response_handler : public seastar::enable_shared_from_this< // Calculates how much to delay completing the request. The delay adds to the request's inherent latency. template void delay(tracing::trace_state_ptr trace, Func&& on_resume) { - auto backlog = max_backlog(); - auto delay = db::view::calculate_view_update_throttling_delay(backlog, _expire_timer.get_timeout(), _proxy->data_dictionary().get_config().view_flow_control_delay_limit_in_ms()); + auto delay = db::view::calculate_view_update_throttling_delay(_view_backlog, _expire_timer.get_timeout(), _proxy->data_dictionary().get_config().view_flow_control_delay_limit_in_ms()); stats().last_mv_flow_control_delay = delay; stats().mv_flow_control_delay += delay.count(); if (delay.count() == 0) { @@ -1749,7 +1749,7 @@ class abstract_write_response_handler : public seastar::enable_shared_from_this< ++stats().throttled_base_writes; ++stats().total_throttled_base_writes; tracing::trace(trace, "Delaying user write due to view update backlog {}/{} by {}us", - backlog.get_current_bytes(), backlog.get_max_bytes(), delay.count()); + _view_backlog.get_current_bytes(), _view_backlog.get_max_bytes(), delay.count()); // Waited on indirectly. (void)sleep_abortable(delay).finally([self = shared_from_this(), on_resume = std::forward(on_resume)] { --self->stats().throttled_base_writes; From 866520ff894ef88dafefa2835ee72859739791e5 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Wed, 8 Jan 2025 14:27:52 +0800 Subject: [PATCH 216/397] test.py: Defer Scylla executable check until test execution Move the Scylla executable existence check from PythonTestSuite's constructor to test execution time. This allows running unit tests that don't depend on the scylla executable without building it first. Previously, PythonTestSuite's constructor would fail if the Scylla executable was missing, preventing even unrelated unit tests from running. Now, only tests that actually require Scylla will fail if the executable is missing. Fixes scylladb/scylladb#22168 Refs scylladb/scylladb#19486 Signed-off-by: Kefu Chai Closes scylladb/scylladb#22224 --- test.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/test.py b/test.py index 9c539976c4e1..e0c7e44665ff 100755 --- a/test.py +++ b/test.py @@ -481,10 +481,6 @@ class PythonTestSuite(TestSuite): def __init__(self, path, cfg: dict, options: argparse.Namespace, mode: str) -> None: super().__init__(path, cfg, options, mode) self.scylla_exe = path_to(self.mode, "scylla") - if not os.access(self.scylla_exe, os.F_OK): - raise FileNotFoundError(f"{self.scylla_exe} does not exist.") - if not os.access(self.scylla_exe, os.X_OK): - raise PermissionError(f"{self.scylla_exe} is not executable.") self.scylla_env = dict(self.base_env) if self.mode == "coverage": self.scylla_env.update(coverage.env(self.scylla_exe, distinct_id=self.name)) @@ -587,6 +583,13 @@ async def add_test(self, shortname, casename) -> None: test = PythonTest(self.next_id((shortname, self.suite_key)), shortname, casename, self) self.tests.append(test) + async def run(self, test: 'Test', options: argparse.Namespace): + if not os.access(self.scylla_exe, os.F_OK): + raise FileNotFoundError(f"{self.scylla_exe} does not exist.") + if not os.access(self.scylla_exe, os.X_OK): + raise PermissionError(f"{self.scylla_exe} is not executable.") + return await super().run(test, options) + class CQLApprovalTestSuite(PythonTestSuite): """Run CQL commands against a single Scylla instance""" From d0a3311ced3bc7f67399823eab9ea67debb532c7 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 7 Jan 2025 11:58:43 +0800 Subject: [PATCH 217/397] locator: do not include unused headers these unused includes were identifier by clang-include-cleaner. after auditing these source files, all of the reports have been confirmed. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22199 --- locator/abstract_replication_strategy.hh | 2 -- locator/tablet_sharder.hh | 2 +- locator/topology.hh | 1 - service/storage_service.hh | 1 + test/boost/cdc_test.cc | 1 + test/boost/network_topology_strategy_test.cc | 1 + test/boost/tablets_test.cc | 1 + 7 files changed, 5 insertions(+), 4 deletions(-) diff --git a/locator/abstract_replication_strategy.hh b/locator/abstract_replication_strategy.hh index 46f469f80ea2..34ed725f2ce2 100644 --- a/locator/abstract_replication_strategy.hh +++ b/locator/abstract_replication_strategy.hh @@ -12,11 +12,9 @@ #include #include #include "gms/inet_address.hh" -#include "locator/snitch_base.hh" #include "locator/token_range_splitter.hh" #include "dht/token-sharding.hh" #include "token_metadata.hh" -#include "snitch_base.hh" #include "utils/maybe_yield.hh" #include "utils/sequenced_set.hh" #include "utils/simple_hashers.hh" diff --git a/locator/tablet_sharder.hh b/locator/tablet_sharder.hh index 922fea6aa91a..9b52a8fb29c7 100644 --- a/locator/tablet_sharder.hh +++ b/locator/tablet_sharder.hh @@ -11,7 +11,7 @@ #include "dht/token-sharding.hh" #include "locator/tablets.hh" #include "locator/token_metadata.hh" -#include "utils/to_string.hh" +#include namespace locator { diff --git a/locator/topology.hh b/locator/topology.hh index db5921e761fe..95b5c5eeed65 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -15,7 +15,6 @@ #include #include #include -#include #include #include diff --git a/service/storage_service.hh b/service/storage_service.hh index 0d44ed864c0c..ecf3ad7c8f96 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -20,6 +20,7 @@ #include "service/qos/service_level_controller.hh" #include "service/topology_guard.hh" #include "locator/abstract_replication_strategy.hh" +#include "locator/snitch_base.hh" #include "locator/tablets.hh" #include "locator/tablet_metadata_guard.hh" #include "inet_address_vectors.hh" diff --git a/test/boost/cdc_test.cc b/test/boost/cdc_test.cc index 75f74d8b2794..cb17e9e012e3 100644 --- a/test/boost/cdc_test.cc +++ b/test/boost/cdc_test.cc @@ -14,6 +14,7 @@ #include #include #include +#include #include "cdc/log.hh" #include "cdc/cdc_options.hh" diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index d80ddf27a87d..58cbb556d739 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -11,6 +11,7 @@ #include "gms/inet_address.hh" #include "inet_address_vectors.hh" #include "locator/types.hh" +#include "locator/snitch_base.hh" #include "utils/assert.hh" #include "utils/UUID_gen.hh" #include "utils/sequenced_set.hh" diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 07ceaf04cac8..3e4c241810f7 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -30,6 +30,7 @@ #include "locator/tablet_replication_strategy.hh" #include "locator/tablet_sharder.hh" #include "locator/load_sketch.hh" +#include "locator/snitch_base.hh" #include "utils/UUID_gen.hh" #include "utils/error_injection.hh" #include "utils/to_string.hh" From 7ed89266b30f782c0584131888b025a0de9b5f8f Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 10:37:39 +0000 Subject: [PATCH 218/397] config/config_file: Add exports and definitions of config_type_for<> Required for implementors. Other than config.cc. --- db/config.cc | 3 +++ utils/config_file.hh | 17 +++++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/db/config.cc b/db/config.cc index d6adb1e34aca..6e5410d5c2dd 100644 --- a/db/config.cc +++ b/db/config.cc @@ -144,6 +144,9 @@ const config_type config_type_for = config_type("string", value_to_ template <> const config_type config_type_for> = config_type("string list", value_to_json>); +template <> +const config_type config_type_for>> = config_type("string map map", value_to_json>>); + template <> const config_type config_type_for> = config_type("string map", value_to_json>); diff --git a/utils/config_file.hh b/utils/config_file.hh index 771fa77e4a3d..547c75d98dbc 100644 --- a/utils/config_file.hh +++ b/utils/config_file.hh @@ -53,6 +53,21 @@ public: template extern const config_type config_type_for; +template<> +extern const config_type config_type_for; + +template<> +extern const config_type config_type_for; + +template<> +extern const config_type config_type_for; + +template<> +extern const config_type config_type_for>; + +template<> +extern const config_type config_type_for>>; + class config_file { static thread_local unsigned s_shard_id; struct any_value { @@ -271,6 +286,8 @@ public: config_file(std::initializer_list = {}); config_file(const config_file&) = delete; + virtual ~config_file() = default; + void add(cfg_ref, std::unique_ptr value); void add(std::initializer_list); void add(const std::vector &); From 9f06a0e3a30dcb0f761e14fbf85e640fbf5f6d31 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 10:38:20 +0000 Subject: [PATCH 219/397] sstables: add get_shared_components accessor To access the shared components. --- sstables/sstables.hh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sstables/sstables.hh b/sstables/sstables.hh index 61eabb93ff6e..820435e5af04 100644 --- a/sstables/sstables.hh +++ b/sstables/sstables.hh @@ -783,6 +783,10 @@ private: public: future<> read_toc() noexcept; + shareable_components& get_shared_components() const { + return *_components; + } + schema_ptr get_schema() const { return _schema; } From 511326882ab0c9cb372b979b477f95c23d269448 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 10:39:28 +0000 Subject: [PATCH 220/397] schema/migration_manager: Add schema validate Validates schema before announce. To ensure all extensions are happy. --- schema/schema.hh | 3 +++ service/migration_manager.cc | 13 +++++++++++++ 2 files changed, 16 insertions(+) diff --git a/schema/schema.hh b/schema/schema.hh index 76f2d1918d63..ce66a9a59ac5 100644 --- a/schema/schema.hh +++ b/schema/schema.hh @@ -473,6 +473,9 @@ class partition_slice; class schema_extension { public: virtual ~schema_extension() {}; + virtual future<> validate(const schema&) const { + return make_ready_future<>(); + } virtual bytes serialize() const = 0; virtual bool is_placeholder() const { return false; diff --git a/service/migration_manager.cc b/service/migration_manager.cc index 21d0e730c5b0..0a1acd0f12f8 100644 --- a/service/migration_manager.cc +++ b/service/migration_manager.cc @@ -626,6 +626,13 @@ std::vector prepare_new_keyspace_announcement(replica::database& db, l return db::schema_tables::make_create_keyspace_mutations(db.features().cluster_schema_features(), ksm, timestamp); } +static +future<> validate(schema_ptr schema) { + return do_for_each(schema->extensions(), [schema](auto & p) { + return p.second->validate(*schema); + }); +} + static future> include_keyspace( storage_proxy& sp, const keyspace_metadata& keyspace, std::vector mutations) { // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631). @@ -656,6 +663,7 @@ static future> do_prepare_new_column_family_announcement(s } future> prepare_new_column_family_announcement(storage_proxy& sp, schema_ptr cfm, api::timestamp_type timestamp) { + return validate(cfm).then([&sp, cfm, timestamp] { try { auto& db = sp.get_db().local(); auto ksm = db.find_keyspace(cfm->ks_name()).metadata(); @@ -663,6 +671,7 @@ future> prepare_new_column_family_announcement(storage_pro } catch (const replica::no_such_keyspace& e) { throw exceptions::configuration_exception(format("Cannot add table '{}' to non existing keyspace '{}'.", cfm->cf_name(), cfm->ks_name())); } + }); } future<> prepare_new_column_family_announcement(std::vector& mutations, @@ -677,6 +686,7 @@ future<> prepare_new_column_family_announcement(std::vector& mutations future> prepare_column_family_update_announcement(storage_proxy& sp, schema_ptr cfm, std::vector view_updates, api::timestamp_type ts) { warn(unimplemented::cause::VALIDATION); + co_await validate(cfm); try { auto& db = sp.local_db(); auto&& old_schema = db.find_column_family(cfm->ks_name(), cfm->cf_name()).schema(); // FIXME: Should we lookup by id? @@ -826,6 +836,7 @@ future> prepare_type_drop_announcement(storage_proxy& sp, } future> prepare_new_view_announcement(storage_proxy& sp, view_ptr view, api::timestamp_type ts) { + return validate(view).then([&sp, view = std::move(view), ts] { auto& db = sp.local_db(); try { auto keyspace = db.find_keyspace(view->ks_name()).metadata(); @@ -846,9 +857,11 @@ future> prepare_new_view_announcement(storage_proxy& sp, v return make_exception_future>( exceptions::configuration_exception(format("Cannot add view '{}' to non existing keyspace '{}'.", view->cf_name(), view->ks_name()))); } + }); } future> prepare_view_update_announcement(storage_proxy& sp, view_ptr view, api::timestamp_type ts) { + co_await validate(view); auto db = sp.data_dictionary(); try { auto&& keyspace = db.find_keyspace(view->ks_name()).metadata(); From e734fc11ec166d38f726f628c0b1a0080bdf3f58 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 10:40:41 +0000 Subject: [PATCH 221/397] cql_test_env: Add optional query timeout Some tests need queries to actually fail. --- test/lib/cql_test_env.cc | 7 ++++--- test/lib/cql_test_env.hh | 2 ++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index c1c906859ec4..0a6fd3f920ed 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -52,6 +52,7 @@ #include "message/messaging_service.hh" #include "gms/gossiper.hh" #include "gms/feature_service.hh" +#include "service/qos/service_level_controller.hh" #include "db/system_keyspace.hh" #include "db/system_distributed_keyspace.hh" #include "db/sstables-format-selector.hh" @@ -172,8 +173,8 @@ class single_node_cql_env : public cql_test_env { struct core_local_state { service::client_state client_state; - core_local_state(auth::service& auth_service, qos::service_level_controller& sl_controller) - : client_state(service::client_state::external_tag{}, auth_service, &sl_controller, infinite_timeout_config) + core_local_state(auth::service& auth_service, qos::service_level_controller& sl_controller, timeout_config timeout) + : client_state(service::client_state::external_tag{}, auth_service, &sl_controller, timeout) { client_state.set_login(auth::authenticated_user(testing_superuser)); } @@ -1070,7 +1071,7 @@ class single_node_cql_env : public cql_test_env { _group0_client = &group0_client; - _core_local.start(std::ref(_auth_service), std::ref(_sl_controller)).get(); + _core_local.start(std::ref(_auth_service), std::ref(_sl_controller), cfg_in.query_timeout.value_or(infinite_timeout_config)).get(); auto stop_core_local = defer([this] { _core_local.stop().get(); }); if (!local_db().has_keyspace(ks_name)) { diff --git a/test/lib/cql_test_env.hh b/test/lib/cql_test_env.hh index 22e75273536c..92e33ed620fe 100644 --- a/test/lib/cql_test_env.hh +++ b/test/lib/cql_test_env.hh @@ -98,6 +98,8 @@ public: bool ms_listen = false; bool run_with_raft_recovery = false; + std::optional query_timeout; + cql_test_config(); cql_test_config(const cql_test_config&); cql_test_config(shared_ptr); From 2f1731c55151b8173427074ba72cd19bb30ad7ea Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Mon, 6 Jan 2025 17:03:11 +0100 Subject: [PATCH 222/397] test: Include parent test name in `ScyllaClusterManager` log file names. Add the test file name to `ScyllaClusterManager` log file names alongside the test function name. This avoids race conditions when tests with the same function names are executed simultaneously. Fixes scylladb/scylladb#21807 Backport: not needed since this is a fix in the testing scripts. Closes scylladb/scylladb#22192 --- test/pylib/scylla_cluster.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 6f1f71115e7f..5e8d718e9e58 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -1311,7 +1311,8 @@ async def _before_test(self, test_case_name: str) -> str: self.current_test_case_full_name = f'{self.test_uname}::{test_case_name}' root_logger = logging.getLogger() # file handler file name should be consistent with topology/conftest.py:manager test_py_log_test variable - self.test_case_log_fh = logging.FileHandler(f"{self.base_dir}/{test_case_name}.log") + parent_test_name = self.test_uname.replace('/', '_') + self.test_case_log_fh = logging.FileHandler(f"{self.base_dir}/{parent_test_name}_{test_case_name}_cluster.log") self.test_case_log_fh.setLevel(root_logger.getEffectiveLevel()) # to have the custom formatter with a timestamp that used in a test.py but for each testcase's log, we need to # extract it from the root logger and apply to the handler From 462a10c4f6e08e9098c1344a449f335df2412c8d Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 6 Jan 2025 21:03:48 +0800 Subject: [PATCH 223/397] test.py: do not repeat "combined_tests" instead of repeating "combined_tests", let's define a variable for it. less repeating this way. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22185 --- test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/test.py b/test.py index e0c7e44665ff..f104d0d41b9e 100755 --- a/test.py +++ b/test.py @@ -389,12 +389,13 @@ def _generate_cache(self, exec_path, exec_name) -> None: def __init__(self, path, cfg: dict, options: argparse.Namespace, mode) -> None: super().__init__(path, cfg, options, mode) - exe = path_to(self.mode, "test", self.name, 'combined_tests') + exec_name = 'combined_tests' + exec_path = path_to(self.mode, "test", self.name, exec_name) # Apply combined test only for test/boost, # cache the tests only if the executable exists, so we can # run test.py with a partially built tree - if self.name == 'boost' and os.path.exists(exe): - self._generate_cache(exe, 'combined_tests') + if self.name == 'boost' and os.path.exists(exec_path): + self._generate_cache(exec_path, exec_name) async def create_test(self, shortname: str, casename: str, suite, args) -> None: fqname = os.path.join(self.mode, self.name, shortname) From 1728f9c983f3a6070da10c6c6ac0170a5ae488d0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Tue, 7 Jan 2025 15:10:37 +0100 Subject: [PATCH 224/397] utils/dict_trainer: silence an ERROR log when raft is aborted during dict publication The dict publication routine might throw raft::request_aborted when the node is aborted. This doesn't deserve an ERROR log. Let's demote the log printed in this case from ERROR to DEBUG. Fixes scylladb/scylladb#22081 Closes scylladb/scylladb#22211 --- utils/dict_trainer.cc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/utils/dict_trainer.cc b/utils/dict_trainer.cc index c3639eb03e9a..b187ec2ab489 100644 --- a/utils/dict_trainer.cc +++ b/utils/dict_trainer.cc @@ -14,6 +14,7 @@ #include "utils/alien_worker.hh" #include "utils/shared_dict.hh" #include "utils/hashers.hh" +#include "raft/raft.hh" #include using namespace seastar; @@ -152,6 +153,8 @@ seastar::future<> dict_training_loop::start( dict_trainer_logger.debug("dict_training_loop: publishing..."); co_await emit(dict_data); dict_trainer_logger.debug("dict_training_loop: published..."); + } catch (const raft::request_aborted&) { + dict_trainer_logger.debug("dict_training_loop: raft aborted while publishing"); } catch (...) { if (_cancelled.abort_requested()) { dict_trainer_logger.debug("dict_training_loop: cancelled"); From 224dc340897165bbb87bc8c466f3d56624142a85 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Wed, 8 Jan 2025 13:36:51 +0200 Subject: [PATCH 225/397] utils: introduce from_chars_exactly() This is a replacement for boost::lexical_cast (but without its long dependency chain). It wraps std::from_chars(), providing a less flexible but also more concise interface. --- utils/from_chars_exactly.hh | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 utils/from_chars_exactly.hh diff --git a/utils/from_chars_exactly.hh b/utils/from_chars_exactly.hh new file mode 100644 index 000000000000..f21bf00cc382 --- /dev/null +++ b/utils/from_chars_exactly.hh @@ -0,0 +1,26 @@ +// Copyright (C) 2025-present ScyllaDB +// SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + +#pragma once + +#include +#include +#include + +namespace utils { + +// Parses a string into a number, throwing an exception if the entire string is not consumed +// or a conversion error happens. The exception is created by calling the provided callable. +template +requires std::integral || std::floating_point +T +from_chars_exactly(std::string_view str, std::invocable auto&& make_exception) { + T result; + auto [ptr, ec] = std::from_chars(str.data(), str.data() + str.size(), result); + if (ptr != str.data() + str.size() || ec != std::errc{}) { + throw make_exception(str); + } + return result; +} + +} From 9ff64736916e5f6a8d12101857b470ccc1826335 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Tue, 24 Dec 2024 19:01:18 +0200 Subject: [PATCH 226/397] error_injection: replace boost::lexical_cast with std::from_chars Replace boost with a standard facility; this reduces dependencies as lexical_cast depends on boost ranges. As a side effect the exception error message is improved. --- raft/server.cc | 1 + utils/error_injection.hh | 9 +++++---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/raft/server.cc b/raft/server.cc index 2a745d3828a4..3def36fffcf8 100644 --- a/raft/server.cc +++ b/raft/server.cc @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include diff --git a/utils/error_injection.hh b/utils/error_injection.hh index 125525cc2097..cee42fe05c73 100644 --- a/utils/error_injection.hh +++ b/utils/error_injection.hh @@ -9,6 +9,7 @@ #pragma once #include "utils/assert.hh" +#include "utils/from_chars_exactly.hh" #include #include #include @@ -27,9 +28,6 @@ #include #include -#include - - namespace utils { // Exception thrown by enabled error injection @@ -163,7 +161,10 @@ class error_injection { if constexpr (std::is_same_v) { return s; } else { - return boost::lexical_cast(s.data(), s.size()); + return utils::from_chars_exactly(s, [&] (std::string_view s) { + return std::runtime_error(fmt::format("Failed to convert injected value [{}] for parameter [{}], injection [{}]", + s, name, injection_name)); + }); } } }; From 723518c3902be15224d8fa3cd7fdf253517e9f45 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 10:49:08 +0000 Subject: [PATCH 227/397] EAR: port the ear feature from enterprise Bulk transfer of EAR functionality. Includes all providers etc. Could maybe break up into smaller blocks, but once it gets down to the core of it, would require messing with code instead of just moving. So this is it. Note: KMIP support is disabled unless you happen to have the kmipc SDK in your scylla dir. Adds optional encryption of sstables and commitlog, using block level file encryption. Provides key sourcing from various sources, such as local files or popular KMS systems. --- CMakeLists.txt | 2 + cmake/Findkmip.cmake | 53 + configure.py | 41 +- ent/CMakeLists.txt | 1 + ent/encryption/CMakeLists.txt | 44 + ent/encryption/encrypted_file_impl.cc | 555 ++++++++++ ent/encryption/encrypted_file_impl.hh | 24 + ent/encryption/encryption.cc | 1040 ++++++++++++++++++ ent/encryption/encryption.hh | 196 ++++ ent/encryption/encryption_config.cc | 164 +++ ent/encryption/encryption_config.hh | 33 + ent/encryption/encryption_exceptions.hh | 55 + ent/encryption/gcp_host.cc | 1031 +++++++++++++++++ ent/encryption/gcp_host.hh | 80 ++ ent/encryption/gcp_key_provider.cc | 77 ++ ent/encryption/gcp_key_provider.hh | 25 + ent/encryption/kmip_host.cc | 1222 +++++++++++++++++++++ ent/encryption/kmip_host.hh | 80 ++ ent/encryption/kmip_key_provider.cc | 119 ++ ent/encryption/kmip_key_provider.hh | 40 + ent/encryption/kms_host.cc | 1164 ++++++++++++++++++++ ent/encryption/kms_host.hh | 80 ++ ent/encryption/kms_key_provider.cc | 71 ++ ent/encryption/kms_key_provider.hh | 37 + ent/encryption/local_file_provider.cc | 292 +++++ ent/encryption/local_file_provider.hh | 41 + ent/encryption/replicated_key_provider.cc | 477 ++++++++ ent/encryption/replicated_key_provider.hh | 39 + ent/encryption/symmetric_key.cc | 396 +++++++ ent/encryption/symmetric_key.hh | 154 +++ ent/encryption/system_key.cc | 65 ++ ent/encryption/system_key.hh | 34 + 32 files changed, 7731 insertions(+), 1 deletion(-) create mode 100644 cmake/Findkmip.cmake create mode 100644 ent/CMakeLists.txt create mode 100644 ent/encryption/CMakeLists.txt create mode 100644 ent/encryption/encrypted_file_impl.cc create mode 100644 ent/encryption/encrypted_file_impl.hh create mode 100644 ent/encryption/encryption.cc create mode 100644 ent/encryption/encryption.hh create mode 100644 ent/encryption/encryption_config.cc create mode 100644 ent/encryption/encryption_config.hh create mode 100644 ent/encryption/encryption_exceptions.hh create mode 100644 ent/encryption/gcp_host.cc create mode 100644 ent/encryption/gcp_host.hh create mode 100644 ent/encryption/gcp_key_provider.cc create mode 100644 ent/encryption/gcp_key_provider.hh create mode 100644 ent/encryption/kmip_host.cc create mode 100644 ent/encryption/kmip_host.hh create mode 100644 ent/encryption/kmip_key_provider.cc create mode 100644 ent/encryption/kmip_key_provider.hh create mode 100644 ent/encryption/kms_host.cc create mode 100644 ent/encryption/kms_host.hh create mode 100644 ent/encryption/kms_key_provider.cc create mode 100644 ent/encryption/kms_key_provider.hh create mode 100644 ent/encryption/local_file_provider.cc create mode 100644 ent/encryption/local_file_provider.hh create mode 100644 ent/encryption/replicated_key_provider.cc create mode 100644 ent/encryption/replicated_key_provider.hh create mode 100644 ent/encryption/symmetric_key.cc create mode 100644 ent/encryption/symmetric_key.hh create mode 100644 ent/encryption/system_key.cc create mode 100644 ent/encryption/system_key.hh diff --git a/CMakeLists.txt b/CMakeLists.txt index 1a19edda5e5e..cd32630f9158 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -267,6 +267,7 @@ add_subdirectory(compaction) add_subdirectory(cql3) add_subdirectory(data_dictionary) add_subdirectory(dht) +add_subdirectory(ent) add_subdirectory(gms) add_subdirectory(idl) add_subdirectory(index) @@ -308,6 +309,7 @@ set(scylla_libs cql3 data_dictionary dht + encryption gms idl index diff --git a/cmake/Findkmip.cmake b/cmake/Findkmip.cmake new file mode 100644 index 000000000000..02b916971109 --- /dev/null +++ b/cmake/Findkmip.cmake @@ -0,0 +1,53 @@ +# +# Copyright 2024-present ScyllaDB +# + +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# + +set(kmip_ver "2.1.0t") + +cmake_host_system_information( + RESULT distrib_id QUERY DISTRIB_ID) +if(distrib_id MATCHES "centos|fedora|rhel") + set(kmip_distrib "rhel84") +else() + message(FATAL_ERROR "Could not locate kmipc library for ${distrib_id}") +endif() + +if(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64") + set(kmip_arch "aarch64") +elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") + set(kmip_arch "64") +elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "(powerpc|ppc)64le") + set(kmip_arch "ppc64le") +endif() + +set(kmip_ROOT "${PROJECT_SOURCE_DIR}/kmipc/kmipc-${kmip_ver}-${kmip_distrib}_${kmip_arch}") +find_library(kmip_LIBRARY + NAMES kmip + HINTS ${kmip_ROOT}/lib) + +find_path(kmip_INCLUDE_DIR + NAMES kmip.h + HINTS ${kmip_ROOT}/include) + +mark_as_advanced( + kmip_LIBRARY + kmip_INCLUDE_DIR) + +find_package_handle_standard_args(kmip + REQUIRED_VARS + kmip_LIBRARY + kmip_INCLUDE_DIR) + +if(kmip_FOUND) + if (NOT TARGET KMIP::kmipc) + add_library(KMIP::kmipc UNKNOWN IMPORTED) + set_target_properties(KMIP::kmipc PROPERTIES + INTERFACE_INCLUDE_DIRECTORIES "${kmip_INCLUDE_DIR}" + IMPORTED_LINK_INTERFACE_LANGUAGES "C" + IMPORTED_LOCATION "${kmip_LIBRARY}") + endif() +endif() diff --git a/configure.py b/configure.py index 03fe64ba8e54..5e930ff6db22 100755 --- a/configure.py +++ b/configure.py @@ -1123,6 +1123,19 @@ def find_ninja(): 'utils/arch/powerpc/crc32-vpmsum/crc32_wrapper.cc', 'querier.cc', 'mutation_writer/multishard_writer.cc', + 'ent/encryption/encryption_config.cc', + 'ent/encryption/encryption.cc', + 'ent/encryption/symmetric_key.cc', + 'ent/encryption/local_file_provider.cc', + 'ent/encryption/replicated_key_provider.cc', + 'ent/encryption/system_key.cc', + 'ent/encryption/encrypted_file_impl.cc', + 'ent/encryption/kmip_host.cc', + 'ent/encryption/kmip_key_provider.cc', + 'ent/encryption/kms_host.cc', + 'ent/encryption/kms_key_provider.cc', + 'ent/encryption/gcp_host.cc', + 'ent/encryption/gcp_key_provider.cc', 'multishard_mutation_query.cc', 'reader_concurrency_semaphore.cc', 'sstables_loader.cc', @@ -2000,7 +2013,7 @@ def query_seastar_flags(pc_file, use_shared_libs, link_static_cxx=False): pkgs.append('lua53' if have_pkg('lua53') else 'lua') -libs = ' '.join([maybe_static(args.staticyamlcpp, '-lyaml-cpp'), '-latomic', '-lz', '-lsnappy', +libs = ' '.join([maybe_static(args.staticyamlcpp, '-lyaml-cpp'), '-latomic', '-lz', '-lsnappy', '-lcrypto', ' -lstdc++fs', ' -lcrypt', ' -lcryptopp', ' -lpthread', # Must link with static version of libzstd, since # experimental APIs that we use are only present there. @@ -2022,6 +2035,32 @@ def query_seastar_flags(pc_file, use_shared_libs, link_static_cxx=False): if args.staticcxx: user_ldflags += " -static-libstdc++" +kmip_lib_ver = '1.9.2a'; + +def kmiplib(): + os_ids = get_os_ids() + for id in os_ids: + if id in { 'centos', 'fedora', 'rhel' }: + return 'rhel84' + print('Could not resolve libkmip.a for platform {}'.format(os_ids)) + sys.exit(1) + +def target_cpu(): + cpu, _, _ = subprocess.check_output([cxx, '-dumpmachine']).decode('utf-8').partition('-') + return cpu + +def kmip_arch(): + arch = target_cpu() + if arch == 'x86_64': + return '64' + return arch + +kmipc_dir = f'kmipc/kmipc-2.1.0t-{kmiplib()}_{kmip_arch()}' +kmipc_lib = f'{kmipc_dir}/lib/libkmip.a' +libs += ' -lboost_filesystem' +if os.path.exists(kmipc_lib): + libs += f' {kmipc_lib}' + user_cflags += f' -I{kmipc_dir}/include -DHAVE_KMIP' def get_extra_cxxflags(mode, mode_config, cxx, debuginfo): cxxflags = [] diff --git a/ent/CMakeLists.txt b/ent/CMakeLists.txt new file mode 100644 index 000000000000..7ca6eb6617c0 --- /dev/null +++ b/ent/CMakeLists.txt @@ -0,0 +1 @@ +add_subdirectory(encryption) diff --git a/ent/encryption/CMakeLists.txt b/ent/encryption/CMakeLists.txt new file mode 100644 index 000000000000..2bee22124b6a --- /dev/null +++ b/ent/encryption/CMakeLists.txt @@ -0,0 +1,44 @@ +include(add_whole_archive) + +find_package(cpp-jwt REQUIRED) +find_package(kmip) + +add_library(scylla_encryption STATIC) +target_sources(scylla_encryption + PRIVATE + encrypted_file_impl.cc + encryption.cc + encryption_config.cc + gcp_host.cc + gcp_key_provider.cc + kmip_host.cc + kmip_key_provider.cc + kms_host.cc + kms_key_provider.cc + local_file_provider.cc + replicated_key_provider.cc + symmetric_key.cc + system_key.cc) +target_include_directories(scylla_encryption + PUBLIC + ${CMAKE_SOURCE_DIR}) +target_link_libraries(scylla_encryption + PUBLIC + Seastar::seastar + PRIVATE + cql3 + utils + cpp-jwt::cpp-jwt) +if(kmip_FOUND) + target_link_libraries(scylla_encryption + PRIVATE + KMIP::kmipc) + target_compile_definitions(scylla_encryption + PUBLIC + HAVE_KMIP) +endif() + +check_headers(check-headers scylla_encryption + GLOB_RECURSE ${CMAKE_CURRENT_SOURCE_DIR}/*.hh) + +add_whole_archive(encryption scylla_encryption) diff --git a/ent/encryption/encrypted_file_impl.cc b/ent/encryption/encrypted_file_impl.cc new file mode 100644 index 000000000000..7c3cf86d49e1 --- /dev/null +++ b/ent/encryption/encrypted_file_impl.cc @@ -0,0 +1,555 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include +#include + +#include "symmetric_key.hh" +#include "encryption.hh" +#include "utils/serialization.hh" +#include "encrypted_file_impl.hh" + +namespace encryption { + +static inline bool is_aligned(size_t n, size_t a) { + return (n & (a - 1)) == 0; +} + +/** + * Very simple block-encrypting file wrapper. + * + * Uses user provided symmetric key + ESSIV block IV calculation + * to encrypt data. + * + * The essiv block key is created by generating the SHA256 hash + * of the provided data encryption key bytes, truncated to block_key_len/8 + * and generating an AES/ECB key using this data. + * + * The file is divided in N blocks of `block_size` size. + * Each block is encrypted (unpadded) with the provided key and + * block mode, using an IV derived by (essiv): + * + * bytes tmp[] = { 0, ..., uint64_t-little-endian() } + * iv = block_key->encrypt(tmp); + * + * All encryption is done unpadded. To handle file sizes we use + * a slightly shaky scheme: + * + * Since all writes are assumed to be done by us, and must be aligned, + * we can assume in turn that any resizing should be made by our truncate + * method. If we attept to truncate to a size not a multiple of our + * _key_ block size (typically 16), we add the same size to the actual + * truncation size. + * On read we then check the file size. If we're reading from a file + * with unaliged size, we know there are key-block-size junk at the end. + * We can align down the last decryption call to match block size, then + * discard the excessive bytes from the result. + * + * If we're in a read/write situation, we need to keep size updated, and + * we could possibly race with disk op/continuations. + * But we are really only for ro/wo cases. + */ +class encrypted_file_impl : public seastar::file_impl { + file _file; + ::shared_ptr _key; + ::shared_ptr _block_key; + bytes _hash_salt; + + std::optional _file_length; + + // this is somewhat large, but we assume this is for bulky stuff like sstables/commitlog + // so large alignment should be preferable to reaclculating block IV too often. + static constexpr size_t block_size = 4096; + static constexpr size_t block_key_len = 256; + + class my_file_handle_impl; + friend class my_file_handle_impl; + + bytes iv_for(uint64_t pos) const; + + using mode = symmetric_key::mode; + + temporary_buffer transform(uint64_t, const void* buffer, size_t len, mode); + size_t transform(uint64_t, const void* buffer, size_t len, void*, mode); + + future<> verify_file_length(); + void maybe_set_length(uint64_t); + void clear_length(); + + static ::shared_ptr generate_block_key(::shared_ptr); + +public: + encrypted_file_impl(file, ::shared_ptr); + + future write_dma(uint64_t pos, const void* buffer, size_t len, io_intent*) override; + future write_dma(uint64_t pos, std::vector iov, io_intent*) override; + future read_dma(uint64_t pos, void* buffer, size_t len, io_intent*) override; + future read_dma(uint64_t pos, std::vector iov, io_intent*) override; + + + future<> flush() override { + return _file.flush(); + } + future stat(void) override; + future<> truncate(uint64_t length) override; + future<> discard(uint64_t offset, uint64_t length) override { + return _file.discard(offset, length); + } + future<> allocate(uint64_t position, uint64_t length) override { + return _file.allocate(position, length); + } + future size(void) override; + future<> close() override { + return _file.close(); + } + std::unique_ptr dup() override; + + subscription list_directory(std::function (directory_entry de)> next) override { + return _file.list_directory(std::move(next)); + } + future> dma_read_bulk(uint64_t offset, size_t range_size, io_intent*) override; +}; + +/** + * Note: ESSIV block iv generation implementation. + * See: http://securityevaluators.com/knowledge/papers/fde_whitepaper_draft_20170627.pdf + * + * We generate a key based on the sha256 of the data key, then encrypt each block number + * using this to get per-block IV. + * The key is AES-256, using ECB (non-iv) encryption + * + */ +::shared_ptr encrypted_file_impl::generate_block_key(::shared_ptr key) { + auto hash = calculate_sha256(key->key()); + hash.resize(block_key_len / 8); + return ::make_shared(key_info{"AES/ECB", block_key_len }, hash); +} + +encrypted_file_impl::encrypted_file_impl(file f, ::shared_ptr key) + : _file(std::move(f)) + , _key(std::move(key)) + , _block_key(generate_block_key(_key)) +{ + _memory_dma_alignment = std::max(_file.memory_dma_alignment(), block_size); + _disk_read_dma_alignment = std::max(_file.disk_read_dma_alignment(), block_size); + _disk_write_dma_alignment = std::max(_file.disk_write_dma_alignment(), block_size); +} + +static future calculate_file_length(const file& f, size_t key_block_size) { + return f.size().then([key_block_size](uint64_t s) { + if (!is_aligned(s, key_block_size)) { + if (s < key_block_size) { + throw std::domain_error(fmt::format("file size {}, expected 0 or at least {}", s, key_block_size)); + } + s -= key_block_size; + } + return s; + }); +} + +future<> encrypted_file_impl::verify_file_length() { + if (_file_length) { + return make_ready_future(); + } + return calculate_file_length(_file, _key->block_size()).then([this](uint64_t s) { + _file_length = s; + }); +} + +void encrypted_file_impl::maybe_set_length(uint64_t s) { + if (s > _file_length.value_or(0)) { + _file_length = s; + } +} + +void encrypted_file_impl::clear_length() { + _file_length = std::nullopt; +} + +bytes encrypted_file_impl::iv_for(uint64_t pos) const { + assert(!(pos & (block_size - 1))); + + // #658. ECB block mode has no IV. Bad for security, + // but must handle. + size_t iv_len = _key->iv_len(); + if (iv_len == 0) { + return bytes{}; + } + + assert(iv_len >= _key->block_size()); + assert(iv_len >= sizeof(uint64_t)); + + bytes b(bytes::initialized_later(), std::max(iv_len, _block_key->block_size())); + std::fill(b.begin(), b.end() - sizeof(uint64_t), 0); + + // write block pos as little endian IV-len integer + auto block = pos / block_size; + write_le(reinterpret_cast(b.end()) - sizeof(uint64_t), block); + + // encrypt the encoded block number to build an IV + _block_key->encrypt_unpadded(b.data(), b.size(), b.data()); + + b.resize(iv_len); + + return b; +} + +size_t encrypted_file_impl::transform(uint64_t pos, const void* buffer, size_t len, void* dst, mode m) { + assert(!(pos & (block_size - 1))); + assert(_file_length || m == mode::encrypt); + + auto o = reinterpret_cast(dst); + auto i = reinterpret_cast(buffer); + auto l = _file_length.value_or(std::numeric_limits::max()); + auto b = _key->block_size(); + + size_t off = 0; + + for (; off < len; off += block_size) { + auto iv = iv_for(pos + off); + auto rem = std::min(block_size, len - off); + + if (rem < block_size || ((pos + off + rem) > l && m == symmetric_key::mode::decrypt)) { + // truncated block. should be the last one. + if (m != symmetric_key::mode::decrypt) { + throw std::invalid_argument("Output data not aligned"); + } + _key->transform_unpadded(m, i + off, align_down(rem, b), o + off, iv.data()); + return l - pos; + } + _key->transform_unpadded(m, i + off, block_size, o + off, iv.data()); + } + + return off; +} + +temporary_buffer encrypted_file_impl::transform(uint64_t pos, const void* buffer, size_t len, mode m) { + assert(!(len & (block_size - 1))); + auto tmp = temporary_buffer::aligned(_file.memory_dma_alignment(), len); + auto s = transform(pos, buffer, len, tmp.get_write(), m); + tmp.trim(s); + return tmp; +} + +future encrypted_file_impl::write_dma(uint64_t pos, const void* buffer, size_t len, io_intent* intent) { + assert(!(len & (block_size - 1))); + auto tmp = transform(pos, buffer, len, mode::encrypt); + assert(tmp.size() == len); // writing + auto p = tmp.get(); + return _file.dma_write(pos, p, len, intent).then([this, tmp = std::move(tmp), pos](size_t s) { + maybe_set_length(pos + s); + return s; + }); +} + +future encrypted_file_impl::write_dma(uint64_t pos, std::vector iov, io_intent* intent) { + std::vector> tmp; + tmp.reserve(iov.size()); + size_t n = 0; + for (auto& i : iov) { + assert(!(i.iov_len & (block_size - 1))); + + tmp.emplace_back(transform(pos + n, i.iov_base, i.iov_len, mode::encrypt)); + assert(tmp.back().size() == i.iov_len); // writing + n += i.iov_len; + i = iovec{ tmp.back().get_write(), tmp.back().size() }; + } + return _file.dma_write(pos, std::move(iov), intent).then([this, tmp = std::move(tmp), pos](size_t s) { + maybe_set_length(pos + s); + return s; + }); +} + +future encrypted_file_impl::read_dma(uint64_t pos, void* buffer, size_t len, io_intent* intent) { + assert(!(len & (block_size - 1))); + return verify_file_length().then([this, pos, buffer, len, intent] { + return _file.dma_read(pos, buffer, len, intent).then([this, pos, buffer](size_t len) { + return transform(pos, buffer, len, buffer, mode::decrypt); + }); + }); +} + +future encrypted_file_impl::read_dma(uint64_t pos, std::vector iov, io_intent* intent) { + return verify_file_length().then([this, pos, iov = std::move(iov), intent]() mutable { + auto f = _file.dma_read(pos, iov, intent); + return f.then([this, pos, iov = std::move(iov)](size_t len) mutable { + size_t off = 0; + for (auto& i : iov) { + off += transform(pos + off, i.iov_base, i.iov_len, i.iov_base, mode::decrypt); + } + return off; + }); + }); +} + +future> encrypted_file_impl::dma_read_bulk(uint64_t offset, size_t range_size, io_intent* intent) { + return verify_file_length().then([this, offset, range_size, intent]() mutable { + auto front = offset & (block_size - 1); + offset -= front; + range_size += front; + // enterprise #925 + // If caller is clever and asks for the last chunk of file + // explicitly (as in offset = N, range_size = size() - N), + // or any other unaligned size, we need to add enough padding + // to get the actual full block to decode. + auto block_size = align_up(range_size, _key->block_size()); + return _file.dma_read_bulk(offset, block_size, intent).then([this, offset, front, range_size](temporary_buffer result) { + auto s = transform(offset, result.get(), result.size(), result.get_write(), mode::decrypt); + // never give back more than asked for. + result.trim(std::min(s, range_size)); + result.trim_front(front); + return result; + }); + }); +} + +future<> encrypted_file_impl::truncate(uint64_t length) { + return size().then([this, length](uint64_t s) { + if (s >= length) { + auto kb = _key->block_size(); + auto n = length; + if (!is_aligned(length, kb)) { + n += kb; + } + return _file.truncate(n).then([this, length] { + _file_length = length; + }); + } + + // crap. we need to pad zeros. But zeros here means + // encrypted zeros. So we must do this surprisingly + // expensively, by actually writing said zeros block + // by block. Anyone hoping for sparse files is now + // severely disappointed! + + auto buf_size = align_up(std::min(length, 32 * block_size), block_size); + auto aligned_size = align_down(s, block_size); + + temporary_buffer buf(buf_size); + std::fill(buf.get_write(), buf.get_write() + buf_size, 0); + + struct trunc { + temporary_buffer buf; + uint64_t aligned_size; + uint64_t size; + uint64_t length; + }; + + return do_with(trunc{std::move(buf), aligned_size, s, length}, [this](trunc & t) { + return repeat([this, &t] { + if (t.aligned_size >= t.length) { + return make_ready_future(stop_iteration::yes); + } + auto n = std::min(t.buf.size(), align_up(size_t(t.length - t.aligned_size), block_size)); + if (t.aligned_size < t.size) { + return read_dma(t.aligned_size, t.buf.get_write(), n, nullptr).then([&, n](size_t r) mutable { + auto rem = size_t(t.size - t.aligned_size); + auto ar = align_up(r, block_size); + assert(ar <= t.buf.size()); + if (rem < ar) { + std::fill(t.buf.get_write() + rem, t.buf.get_write() + ar, 0); + } + return write_dma(t.aligned_size, t.buf.get(), ar, nullptr).then([&, n](size_t w) { + t.aligned_size += w; + // #1869. On btrfs, we get the buffer potentially clobbered up to "n" (max read amount) + // even when "r" (actual bytes read) is less. + std::fill(t.buf.get_write(), t.buf.get_write() + n, 0); + return make_ready_future(stop_iteration::no); + }); + }); + } + return write_dma(t.aligned_size, t.buf.get(), n, nullptr).then([&](size_t w) { + t.aligned_size += w; + return make_ready_future(stop_iteration::no); + }); + }); + }).then([this, length] { + return truncate(length); + });; + }); +} + +future encrypted_file_impl::stat() { + return _file.stat().then([this](struct stat s) { + return verify_file_length().then([this, s]() mutable { + s.st_size = *_file_length; + return s; + }); + }); +} + +future encrypted_file_impl::size() { + return verify_file_length().then([this] { + return *_file_length; + }); +} + + +std::unique_ptr encrypted_file_impl::dup() { + class my_file_handle_impl : public seastar::file_handle_impl { + seastar::file_handle _handle; + key_info _info; + bytes _key; + public: + my_file_handle_impl(seastar::file_handle h, const key_info& info, const bytes& key) + : _handle(std::move(h)) + , _info(info) + , _key(key) + {} + std::unique_ptr clone() const override { + return std::make_unique(_handle, _info, _key); + } + seastar::shared_ptr to_file() && override { + return seastar::make_shared(_handle.to_file(), ::make_shared(_info, _key)); + } + }; + + return std::make_unique(_file.dup(), _key->info(), _key->key()); +} + +shared_ptr make_encrypted_file(file f, ::shared_ptr k) { + return ::make_shared(std::move(f), std::move(k)); +} + +class indirect_encrypted_file_impl : public file_impl { + ::shared_ptr _impl; + file _f; + size_t _key_block_size; + get_key_func _get; + + future<> get() { + if (_impl) { + return make_ready_future<>(); + } + return _get().then([this](::shared_ptr k) { + // #978 could be running the getting more than once. + // Only write _impl once though + if (!_impl) { + _impl = make_encrypted_file(_f, std::move(k)); + } + }); + } +public: + indirect_encrypted_file_impl(file f, size_t key_block_size, get_key_func get) + : _f(f), _key_block_size(key_block_size), _get(std::move(get)) + {} + + future write_dma(uint64_t pos, const void* buffer, size_t len, io_intent* intent) override { + return get().then([this, pos, buffer, len, intent]() { + return _impl->write_dma(pos, buffer, len, intent); + }); + } + future write_dma(uint64_t pos, std::vector iov, io_intent* intent) override { + return get().then([this, pos, iov = std::move(iov), intent]() mutable { + return _impl->write_dma(pos, std::move(iov), intent); + }); + } + future read_dma(uint64_t pos, void* buffer, size_t len, io_intent* intent) override { + return get().then([this, pos, buffer, len, intent]() { + return _impl->read_dma(pos, buffer, len, intent); + }); + } + future read_dma(uint64_t pos, std::vector iov, io_intent* intent) override { + return get().then([this, pos, iov = std::move(iov), intent]() mutable { + return _impl->read_dma(pos, std::move(iov), intent); + }); + } + future> dma_read_bulk(uint64_t offset, size_t range_size, io_intent* intent) override { + return get().then([this, offset, range_size, intent]() { + return _impl->dma_read_bulk(offset, range_size, intent); + }); + } + future<> flush(void) override { + if (_impl) { + return _impl->flush(); + } + return _f.flush(); + } + future stat(void) override { + if (_impl) { + return _impl->stat(); + } + return _f.stat().then([this](struct stat s) { + return calculate_file_length(_f, _key_block_size).then([s](uint64_t fs) mutable { + s.st_size = fs; + return s; + }); + }); + } + future<> truncate(uint64_t length) override { + if (_impl) { + return _impl->truncate(length); + } + return _f.truncate(length); + } + future<> discard(uint64_t offset, uint64_t length) override { + if (_impl) { + return _impl->discard(offset, length); + } + return _f.discard(offset, length); + } + future<> allocate(uint64_t position, uint64_t length) override { + if (_impl) { + return _impl->allocate(position, length); + } + return _f.allocate(position, length); + } + future size(void) override { + if (_impl) { + return _impl->size(); + } + return calculate_file_length(_f, _key_block_size); + } + future<> close() override { + if (_impl) { + return _impl->close(); + } + return _f.close(); + } + std::unique_ptr dup() override { + if (_impl) { + return _impl->dup(); + } + class my_file_handle_impl : public seastar::file_handle_impl { + seastar::file_handle _handle; + size_t _key_block_size; + get_key_func _get; + public: + my_file_handle_impl(seastar::file_handle h, size_t key_block_size, get_key_func get) + : _handle(std::move(h)) + , _key_block_size(key_block_size) + , _get(std::move(get)) + {} + std::unique_ptr clone() const override { + return std::make_unique(_handle, _key_block_size, _get); + } + seastar::shared_ptr to_file() && override { + return make_delayed_encrypted_file(_handle.to_file(), _key_block_size, _get); + } + }; + return std::make_unique(_f.dup(), _key_block_size, _get); + } + + subscription list_directory(std::function (directory_entry de)> next) override { + if (_impl) { + return _impl->list_directory(std::move(next)); + } + return _f.list_directory(std::move(next)); + } +}; + +shared_ptr make_delayed_encrypted_file(file f, size_t key_block_size, get_key_func get) { + return ::make_shared(std::move(f), key_block_size, std::move(get)); +} + + +} + diff --git a/ent/encryption/encrypted_file_impl.hh b/ent/encryption/encrypted_file_impl.hh new file mode 100644 index 000000000000..f4ecc7719b2f --- /dev/null +++ b/ent/encryption/encrypted_file_impl.hh @@ -0,0 +1,24 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include + +#include "symmetric_key.hh" + +namespace encryption { + +class symmetric_key; + +shared_ptr make_encrypted_file(file, ::shared_ptr); + +using get_key_func = std::function>()>; + +shared_ptr make_delayed_encrypted_file(file, size_t, get_key_func); +} diff --git a/ent/encryption/encryption.cc b/ent/encryption/encryption.cc new file mode 100644 index 000000000000..0031b068433c --- /dev/null +++ b/ent/encryption/encryption.cc @@ -0,0 +1,1040 @@ +/* + * Copyright (C) 2015 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include "utils/to_string.hh" + +#include "compress.hh" +#include "encryption.hh" +#include "symmetric_key.hh" +#include "local_file_provider.hh" +#include "replicated_key_provider.hh" +#include "kmip_key_provider.hh" +#include "kmip_host.hh" +#include "kms_key_provider.hh" +#include "kms_host.hh" +#include "gcp_key_provider.hh" +#include "gcp_host.hh" +#include "bytes.hh" +#include "utils/class_registrator.hh" +#include "cql3/query_processor.hh" +#include "db/extensions.hh" +#include "db/system_keyspace.hh" +#include "serializer.hh" +#include "serializer_impl.hh" +#include "schema/schema.hh" +#include "sstables/sstables.hh" +#include "service/storage_service.hh" +#include "service/migration_manager.hh" +#include "db/commitlog/commitlog_extensions.hh" +#include "encrypted_file_impl.hh" +#include "encryption_config.hh" +#include "utils/UUID_gen.hh" +#include "init.hh" + +static seastar::logger logg{"encryption"}; + +namespace encryption { + +static constexpr auto REPLICATED_KEY_PROVIDER_FACTORY = "ReplicatedKeyProviderFactory"; +static constexpr auto LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY = "LocalFileSystemKeyProviderFactory"; +static constexpr auto KMIP_KEY_PROVIDER_FACTORY = "KmipKeyProviderFactory"; +static constexpr auto KMS_KEY_PROVIDER_FACTORY = "KmsKeyProviderFactory"; +static constexpr auto GCP_KEY_PROVIDER_FACTORY = "GcpKeyProviderFactory"; + +bytes base64_decode(const sstring& s, size_t off, size_t len) { + if (off >= s.size()) { + throw std::out_of_range("Invalid offset"); + } + len = std::min(len, s.size() - off); + auto n = (len / 4) * 3; + bytes b{bytes::initialized_later(), n}; + + // EVP_DecodeBlock does not handle padding well (i.e. it returns + // data with actual padding. This is not what we want, since + // we need to allow zeros in data. + // Must thus do decoding the hard way... + + std::unique_ptr ctxt(EVP_ENCODE_CTX_new(), &EVP_ENCODE_CTX_free); + + ::EVP_DecodeInit(ctxt.get()); + + int outl = 0; + auto r = ::EVP_DecodeUpdate(ctxt.get(), reinterpret_cast(b.data()), &outl, reinterpret_cast(s.data() + off), + int(len)); + if (r < 0) { + throw std::invalid_argument("Could not decode: " + s); + } + + int outl2 = 0; + r = ::EVP_DecodeFinal(ctxt.get(), reinterpret_cast(b.data() + outl), &outl2); + if (r < 0) { + throw std::invalid_argument("Could not decode: " + s); + } + b.resize(outl + outl2); + return b; +} + +sstring base64_encode(const bytes& b, size_t off, size_t len) { + if (off >= b.size()) { + throw std::out_of_range("Invalid offset"); + } + len = std::min(len, b.size() - off); + auto n = ((len + 2) / 3) * 4; + sstring s{sstring::initialized_later(), n}; + auto r = EVP_EncodeBlock(reinterpret_cast(s.data()), + reinterpret_cast(b.data() + off), int(len)); + if (r < 0) { + throw std::invalid_argument("Could not encode"); + } + s.resize(r); + return s; +} + +bytes calculate_md5(const bytes& b, size_t off, size_t len) { + if (off >= b.size()) { + throw std::out_of_range("Invalid offset"); + } + len = std::min(len, b.size() - off); + bytes res{bytes::initialized_later(), MD5_DIGEST_LENGTH}; +#if OPENSSL_VERSION_NUMBER >= (3<<28) + EVP_MD_CTX *md5 = EVP_MD_CTX_new(); + EVP_DigestInit_ex(md5, EVP_md5(), nullptr); + EVP_DigestUpdate(md5, b.data() + off, len); + EVP_DigestFinal_ex(md5, reinterpret_cast(res.data()), nullptr); + EVP_MD_CTX_free(md5); +#else + MD5(reinterpret_cast(b.data() + off), len, reinterpret_cast(res.data())); +#endif + return res; +} + +bytes calculate_sha256(bytes_view b) { + bytes res{bytes::initialized_later(), SHA256_DIGEST_LENGTH}; + SHA256(reinterpret_cast(b.data()), b.size(), reinterpret_cast(res.data())); + return res; +} + +bytes calculate_sha256(const bytes& b, size_t off, size_t len) { + if (off >= b.size()) { + throw std::out_of_range("Invalid offset"); + } + len = std::min(len, b.size() - off); + return calculate_sha256(bytes_view(b.data() + off, len)); +} + +bytes hmac_sha256(bytes_view msg, bytes_view key) { + bytes res{bytes::initialized_later(), SHA256_DIGEST_LENGTH}; + + unsigned length; + HMAC(EVP_sha256(), + key.data(), key.size(), + reinterpret_cast(msg.data()), msg.size(), + reinterpret_cast(res.data()), &length); + return res; +} + +future> read_text_file_fully(const sstring& filename) { + return open_file_dma(filename, open_flags::ro).then([](file f) { + return f.size().then([f](size_t s) { + return do_with(make_file_input_stream(f), [s](input_stream& in) { + return in.read_exactly(s).then([](temporary_buffer buf) { + return make_ready_future>(std::move(buf)); + }).finally([&in] { + return in.close(); + }); + }); + }); + }); +} + +future<> write_text_file_fully(const sstring& filename, temporary_buffer buf) { + return open_file_dma(filename, open_flags::wo|open_flags::create).then([buf = std::move(buf)](file f) mutable { + return make_file_output_stream(f).then([buf = std::move(buf)] (output_stream out) mutable { + return do_with(std::move(out), [buf = std::move(buf)](output_stream& out) mutable { + auto p = buf.get(); + auto s = buf.size(); + return out.write(p, s).finally([&out, buf = std::move(buf)] { + return out.close(); + }); + }); + }); + }); +} + +future<> write_text_file_fully(const sstring& filename, const sstring& s) { + return write_text_file_fully(filename, temporary_buffer(s.data(), s.size())); +} + +std::optional parse_expiry(std::optional in) { + if (!in) { + return std::nullopt; + } + size_t idx = 0; + auto n = std::stoll(*in, &idx); // we assume seconds + + if (idx != 0) { + auto unit = in->substr(idx); + if (unit == "ms") { + return std::chrono::milliseconds(n); + } else if (unit == "h") { + return std::chrono::duration_cast(std::chrono::hours(n)); + } else if (unit == "d") { + return std::chrono::duration_cast(std::chrono::days(n)); + } else if (unit == "s") { + // ok + } else if (unit != "") { + throw std::invalid_argument("Unsupported time unit: " + unit); + } + } + return std::chrono::duration_cast(std::chrono::seconds(n)); +} + + +static const sstring namespace_prefix = "com.datastax.bdp.cassandra.crypto."; +static const sstring encryption_attribute = "scylla_encryption_options"; + +static inline const sstring key_id_attribute = "scylla_key_id"; +static inline const sstring encrypted_components_attribute = "encrypted_components"; + +static inline const sstables::disk_string encryption_attribute_ds{ + bytes{encryption_attribute.begin(), encryption_attribute.end()} +}; +static inline const sstables::disk_string key_id_attribute_ds{ + bytes{key_id_attribute.begin(), key_id_attribute.end()} +}; +static inline const sstables::disk_string encrypted_components_attribute_ds{ + bytes{encrypted_components_attribute.begin(), encrypted_components_attribute.end()} +}; + +key_info get_key_info(const options& map) { + opt_wrapper opts(map); + + auto cipher_name = opts(CIPHER_ALGORITHM).value_or("AES/CBC/PKCS5Padding"); + auto key_strength = std::stoul(opts(SECRET_KEY_STRENGTH).value_or("128")); + // todo: static constexpr auto KMIP_KEY_PROVIDER_FACTORY = "KmipKeyProviderFactory"; + return key_info{ std::move(cipher_name), unsigned(key_strength) }; +} + +std::ostream& operator<<(std::ostream& os, const key_provider& p) { + p.print(os); + return os; +} + +sstring encryption_context::maybe_decrypt_config_value(const sstring& s) const { + shared_ptr k = get_config_encryption_key(); + if (!s.empty() && k != nullptr) { + auto b = base64_decode(s); + auto iv = calculate_sha256(k->key()); + iv.resize(k->block_size(), 0); + bytes dst(bytes::initialized_later(), b.size()); + auto len = k->decrypt(b.data(), b.size(), dst.data(), dst.size(), iv.data()); + return sstring(dst.begin(), dst.begin() + len); + } + return s; +} + +class encryption_schema_extension; + +class encryption_context_impl : public encryption_context { + // poor mans per-thread instance variable. We need a lookup map + // per shard, so preallocate it, much like a "sharded" thing would, + // but without all the fancy start/stop stuff. + // Allows this object to be effectively stateless, except for the + // objects in the maps. + std::vector>> _per_thread_provider_cache; + std::vector>> _per_thread_system_key_cache; + std::vector>> _per_thread_kmip_host_cache; + std::vector>> _per_thread_kms_host_cache; + std::vector>> _per_thread_gcp_host_cache; + std::vector> _per_thread_global_user_extension; + std::unique_ptr _cfg; + sharded* _qp;; + sharded* _mm; + sharded* _db; + sharded* _ss; + shared_ptr _cfg_encryption_key; + bool _allow_per_table_encryption; +public: + encryption_context_impl(std::unique_ptr cfg, const service_set& services) + : _per_thread_provider_cache(smp::count) + , _per_thread_system_key_cache(smp::count) + , _per_thread_kmip_host_cache(smp::count) + , _per_thread_kms_host_cache(smp::count) + , _per_thread_gcp_host_cache(smp::count) + , _per_thread_global_user_extension(smp::count) + , _cfg(std::move(cfg)) + , _qp(find_or_null(services)) + , _mm(find_or_null(services)) + , _db(find_or_null(services)) + , _ss(find_or_null(services)) + , _allow_per_table_encryption(_cfg->allow_per_table_encryption()) + {} + + template + static sharded* find_or_null(const service_set& services) { + try { + return std::addressof(services.find()); + } catch (std::out_of_range&) { + // TODO: would be great if we could verify we are in tool mode here. + return nullptr; + } + } + + shared_ptr get_provider(const options& map) override { + opt_wrapper opts(map); + + auto provider_class = opts(KEY_PROVIDER); + if (!provider_class) { + provider_class = opts(SECRET_KEY_PROVIDER_FACTORY_CLASS).value_or(REPLICATED_KEY_PROVIDER_FACTORY); + } + if (provider_class->empty() || ::strcasecmp(provider_class->c_str(), "none") == 0) { + return {}; + } + static const std::unordered_map> providers = [] { + std::unordered_map> map; + + map[REPLICATED_KEY_PROVIDER_FACTORY] = std::make_unique(); + map[LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY] = std::make_unique(); + map[KMIP_KEY_PROVIDER_FACTORY] = std::make_unique(); + map[KMS_KEY_PROVIDER_FACTORY] = std::make_unique(); + map[GCP_KEY_PROVIDER_FACTORY] = std::make_unique(); + + return map; + }(); + + unqualified_name qn(namespace_prefix, *provider_class); + + try { + return providers.at(qn)->get_provider(*this, map); + } catch (std::out_of_range&) { + throw std::invalid_argument("Unknown provider: " + *provider_class); + } + } + shared_ptr get_cached_provider(const sstring& id) const override { + auto& cache = _per_thread_provider_cache[this_shard_id()]; + auto i = cache.find(id); + if (i != cache.end()) { + return i->second; + } + return {}; + } + void cache_provider(const sstring& id, shared_ptr p) override { + _per_thread_provider_cache[this_shard_id()][id] = std::move(p); + } + + shared_ptr get_system_key(const sstring& name) override { + auto& cache = _per_thread_system_key_cache[this_shard_id()]; + auto i = cache.find(name); + if (i != cache.end()) { + return i->second; + } + + shared_ptr k; + + if (kmip_system_key::is_kmip_path(name)) { + k = make_shared(*this, name); + } else { + k = make_shared(*this, name); + } + + if (k != nullptr) { + cache[name] = k; + } + + return k; + } + + shared_ptr get_kmip_host(const sstring& host) override { + auto& cache = _per_thread_kmip_host_cache[this_shard_id()]; + auto i = cache.find(host); + if (i != cache.end()) { + return i->second; + } + + auto j = _cfg->kmip_hosts().find(host); + if (j != _cfg->kmip_hosts().end()) { + auto result = ::make_shared(*this, host, j->second); + cache.emplace(host, result); + return result; + } + + throw std::invalid_argument("No such host: "+ host); + } + + shared_ptr get_kms_host(const sstring& host) override { + auto& cache = _per_thread_kms_host_cache[this_shard_id()]; + auto i = cache.find(host); + if (i != cache.end()) { + return i->second; + } + + auto j = _cfg->kms_hosts().find(host); + if (j != _cfg->kms_hosts().end()) { + auto result = ::make_shared(*this, host, j->second); + cache.emplace(host, result); + return result; + } + + throw std::invalid_argument("No such host: "+ host); + } + + shared_ptr get_gcp_host(const sstring& host) override { + auto& cache = _per_thread_gcp_host_cache[this_shard_id()]; + auto i = cache.find(host); + if (i != cache.end()) { + return i->second; + } + + auto j = _cfg->gcp_hosts().find(host); + if (j != _cfg->gcp_hosts().end()) { + auto result = ::make_shared(*this, host, j->second); + cache.emplace(host, result); + return result; + } + + throw std::invalid_argument("No such host: "+ host); + } + + + const encryption_config& config() const override { + return *_cfg; + } + shared_ptr get_config_encryption_key() const override { + return _cfg_encryption_key; + } + future<> load_config_encryption_key(const sstring & name) { + return get_system_key(name)->get_key().then([this](auto k) { + _cfg_encryption_key = std::move(k); + }); + } + /** + * This looks like checking too late, but since these are only used by + * replicated provider, they will be checked very early anyway, unless + * running tool mode, in which case they don't exist. + */ + template + T& check_service_object(T* t) const { + if (t == nullptr) { + throw std::runtime_error(fmt::format("Service {} not registered", typeid(T).name())); + } + return *t; + } + distributed& get_query_processor() const override { + return check_service_object(_qp); + } + distributed& get_storage_service() const override { + return check_service_object(_ss); + } + distributed& get_database() const override { + return check_service_object(_db); + } + distributed& get_migration_manager() const override { + return check_service_object(_mm); + } + + future<> start() override { + if (_qp && _ss && _db && _mm) { + co_await replicated_key_provider_factory::on_started(get_database().local(), get_migration_manager().local()); + } + } + future<> stop() override { + return smp::invoke_on_all([this]() -> future<> { + for (auto&& [id, h] : _per_thread_kmip_host_cache[this_shard_id()]) { + co_await h->disconnect(); + } + _per_thread_provider_cache[this_shard_id()].clear(); + _per_thread_system_key_cache[this_shard_id()].clear(); + _per_thread_kmip_host_cache[this_shard_id()].clear(); + _per_thread_kms_host_cache[this_shard_id()].clear(); + _per_thread_gcp_host_cache[this_shard_id()].clear(); + _per_thread_global_user_extension[this_shard_id()] = {}; + }); + } + + void add_global_user_encryption(shared_ptr ext) { + _per_thread_global_user_extension[this_shard_id()] = std::move(ext); + } + + shared_ptr get_global_user_encryption() const { + return _per_thread_global_user_extension[this_shard_id()]; + } + bool allow_per_table_encryption() const { + return _allow_per_table_encryption; + } +}; + +class encryption_schema_extension; + +std::ostream& operator<<(std::ostream& os, const encryption_schema_extension& ext); + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +namespace encryption { + +class encryption_schema_extension : public schema_extension { + key_info _info; + shared_ptr _provider; + std::map _options; + std::optional _key_block_size; + + friend std::ostream& operator<<(std::ostream&, const encryption_schema_extension&); +public: + encryption_schema_extension(key_info, shared_ptr, std::map); + + using extension_ptr = ::shared_ptr; + + static extension_ptr create(encryption_context_impl&, std::map); + static extension_ptr create(encryption_context_impl&, const bytes&); + + static extension_ptr parse(encryption_context_impl& ctxt, db::extensions::schema_ext_config cfg) { + struct { + encryption_context_impl& _ctxt; + + extension_ptr operator()(const sstring&) const { + throw std::invalid_argument("Malformed extension"); + } + extension_ptr operator()(const std::map& opts) const { + return create(_ctxt, opts); + } + extension_ptr operator()(const bytes& v) const { + return create(_ctxt, v); + } + } v{ctxt}; + + auto res = std::visit(v, cfg); + // Note: We always allow _disbling_ per-table encryption, i.e. if user encryption is active, we fall back to node-local + if (res && !ctxt.allow_per_table_encryption() && ctxt.get_global_user_encryption()) { + throw std::invalid_argument(fmt::format("Node global user encryption is active and per-table encryption attributes have been prohibited ({})", *res)); + } + return res; + } + + static options parse_options(const bytes& v) { + return ser::deserialize_from_buffer(v, std::type_identity(), 0); + } + + future<::shared_ptr> key_for_read(opt_bytes id) const { + return _provider->key(_info, std::move(id)).then([](std::tuple k_id) { + return std::get<0>(std::move(k_id)); + }); + } + future, opt_bytes>> key_for_write(opt_bytes id = {}) const { + return _provider->key(_info, std::move(id)); + } + + bytes serialize() const override { + return ser::serialize_to_buffer(_options, 0); + } + future<> validate(const schema& s) const override { + try { + co_await _provider->validate(); + auto k = co_await key_for_write(); + logg.info("Added encryption extension to {}.{}", s.ks_name(), s.cf_name()); + logg.info(" Options: {}", _options); + logg.info(" Key Algorithm: {}", _info); + logg.info(" Provider: {}", *_provider); + + auto problems = std::get<0>(k)->validate_exact_info_result(); + if (!problems.empty()) { + logg.warn("{}", problems); + } + } catch (...) { + std::throw_with_nested(exceptions::configuration_exception((std::stringstream{} << "Validation failed:" << std::current_exception()).str())); + } + } + + bool should_delay_read(const opt_bytes& id) { + return _provider->should_delay_read(id); + } + size_t key_block_size() { + if (!_key_block_size) { + _key_block_size = symmetric_key(_info).block_size(); + } + return *_key_block_size; + } +}; + +std::ostream& operator<<(std::ostream& os, const encryption_schema_extension& ext) { + fmt::print(os, "{}, alg={}, provider={}", ext._options, ext._info, *ext._provider); + return os; +} + +encryption_schema_extension::encryption_schema_extension(key_info info, shared_ptr provider, std::map options) + : _info(std::move(info)) + , _provider(std::move(provider)) + , _options(std::move(options)) +{} + +::shared_ptr encryption_schema_extension::create(encryption_context_impl& ctxt, const bytes& v) { + auto map = parse_options(v); + return create(ctxt, map); +} + +::shared_ptr encryption_schema_extension::create(encryption_context_impl& ctxt, std::map map) { + key_info info = get_key_info(map); + auto provider = ctxt.get_provider(map); + if (!provider) { + return {}; + } + return ::make_shared(std::move(info), std::move(provider), std::move(map)); +} + +class encryption_file_io_extension : public sstables::file_io_extension { + ::shared_ptr _ctxt; +public: + encryption_file_io_extension(::shared_ptr ctxt) + : _ctxt(std::move(ctxt)) + {} + + attr_value_map get_attributes(const sstables::sstable& sst) const override { + auto& sc = sst.get_shared_components(); + if (!sc.scylla_metadata) { + return {}; + } + auto* exta = sc.scylla_metadata->get_extension_attributes(); + if (!exta) { + return {}; + } + + auto i = exta->map.find(encryption_attribute_ds); + if (i == exta->map.end()) { + return {}; + } + auto opts = encryption_schema_extension::parse_options(i->second.value); + + if (exta->map.count(key_id_attribute_ds)) { + auto id = exta->map.at(key_id_attribute_ds).value; + auto id_str = id.size() == utils::UUID::serialized_size() + ? sstring(fmt::format("{}", utils::UUID_gen::get_UUID(id))) + : to_hex(id) + ; + opts["key_id"] = std::move(id_str); + } + + if (exta->map.count(encrypted_components_attribute_ds)) { + std::vector ccs; + ccs.reserve(9); + auto mask = ser::deserialize_from_buffer(exta->map.at(encrypted_components_attribute_ds).value, std::type_identity{}, 0); + for (auto c : { sstables::component_type::Index, + sstables::component_type::CompressionInfo, + sstables::component_type::Data, + sstables::component_type::Summary, + sstables::component_type::Digest, + sstables::component_type::CRC, + sstables::component_type::Filter, + sstables::component_type::Statistics, + sstables::component_type::TemporaryStatistics, + }) { + if (mask & int(c)) { + ccs.emplace_back(c); + } + } + opts["components"] = fmt::to_string(fmt::join(ccs, ", ")); + } else { + opts["components"] = "Data"; + } + attr_value_map res; + res["encryption_info"] = std::move(opts); + return res; + } + + future wrap_file(sstables::sstable& sst, sstables::component_type type, file f, open_flags flags) override { + switch (type) { + case sstables::component_type::Scylla: + case sstables::component_type::TemporaryTOC: + case sstables::component_type::TOC: + co_return file{}; + default: + break; + } + + if (flags == open_flags::ro) { + // open existing. check read opts. + auto& sc = sst.get_shared_components(); + if (sc.scylla_metadata) { + auto* exta = sc.scylla_metadata->get_extension_attributes(); + if (exta) { + auto i = exta->map.find(encryption_attribute_ds); + // note: earlier builds of encryption extension would only encrypt data component, + // so iff we are opening old sstables we need to check if this component is actually + // encrypted. We use a bitmask attribute for this. + + bool ok = i != exta->map.end(); + if (ok && type != sstables::component_type::Data) { + ok = exta->map.count(encrypted_components_attribute_ds) && + (ser::deserialize_from_buffer(exta->map.at(encrypted_components_attribute_ds).value, std::type_identity{}, 0) & (1 << int(type))); + } + + if (ok) { + auto esx = encryption_schema_extension::create(*_ctxt, i->second.value); + opt_bytes id; + + if (exta->map.count(key_id_attribute_ds)) { + id = exta->map.at(key_id_attribute_ds).value; + } + + if (esx->should_delay_read(id)) { + logg.debug("Encrypted sstable component {} using delayed opening {} (id: {})", sst.component_basename(type), *esx, id); + + co_return make_delayed_encrypted_file(f, esx->key_block_size(), [esx, comp = sst.component_basename(type), id = std::move(id)] { + logg.trace("Delayed component {} using {} (id: {}) resolve", comp, *esx, id); + return esx->key_for_read(id); + }); + } + + logg.debug("Open encrypted sstable component {} using {} (id: {})", sst.component_basename(type), *esx, id); + + auto k = co_await esx->key_for_read(std::move(id)); + co_return make_encrypted_file(f, std::move(k)); + } + } + } + } else { + auto s = sst.get_schema(); + shared_ptr esx; + auto e = s->extensions().find(encryption_attribute); + // #4844 - don't allow schema encryption to be used for writing + // iff it is disallowed by config -> placeholder here + // (See schema_tables.cc::prepare_builder_from_table_row - if an extension + // is unavailable/non-creatable at load time a dummy object is inserted + // ) + if (e != s->extensions().end() && !e->second->is_placeholder()) { + esx = static_pointer_cast(e->second); + } else if (!is_system_keyspace(s->ks_name())) { + esx = _ctxt->get_global_user_encryption(); + } + if (esx) { + auto& sc = sst.get_shared_components(); + if (!sc.scylla_metadata) { + sc.scylla_metadata.emplace(); + } + auto& ext = sc.scylla_metadata->get_or_create_extension_attributes(); + opt_bytes id; + + // We are writing more than one component. If we used a named key before + // we need to make sure we use the exact same one for all components, + // even if something like KMIP key invalidation replaced it. + // This will also speed up key lookup in some cases, as both repl + // and kmip cache id bound keys. + if (ext.map.count(key_id_attribute_ds)) { + id = ext.map.at(key_id_attribute_ds).value; + } + + logg.debug("Write encrypted sstable component {} using {} (id: {})", sst.component_basename(type), *esx, id); + + /** + * #3954 We can be (and are) called with two components simultaneously (hello index, data). + * If this case we could block on the below "key" call and iff provider has certain cache behaviour (hello replicated) + * or caches expire, we could end up with different keys for respective components, leading to one + * of the components ending up unreadable. + */ + for (;;) { + auto [k, k_id] = co_await esx->key_for_write(std::move(id)); + + if (k_id && ext.map.count(key_id_attribute_ds)) { + id = ext.map.at(key_id_attribute_ds).value; + if (k_id != id) { + continue; + } + } + + id = std::move(k_id); + + if (!ext.map.count(encryption_attribute_ds)) { + ext.map.emplace(encryption_attribute_ds, sstables::disk_string{esx->serialize()}); + } + if (id) { + ext.map.emplace(key_id_attribute_ds, sstables::disk_string{*id}); + } + if (type != sstables::component_type::Data) { + uint32_t mask = 0; + if (ext.map.count(encrypted_components_attribute_ds)) { + mask = ser::deserialize_from_buffer(ext.map.at(encrypted_components_attribute_ds).value, std::type_identity{}, 0); + } + mask |= (1 << int(type)); + // just a marker. see above + ext.map[encrypted_components_attribute_ds] = sstables::disk_string{ser::serialize_to_buffer(mask, 0)}; + } + co_return make_encrypted_file(f, std::move(k)); + } + } + } + + co_return file{}; + } +}; + +std::string encryption_provider(const sstables::sstable& sst) { + auto& sc = sst.get_shared_components(); + if (!sc.scylla_metadata) { + return {}; + } + auto* exta = sc.scylla_metadata->get_extension_attributes(); + if (!exta) { + return {}; + } + + auto i = exta->map.find(encryption_attribute_ds); + if (i == exta->map.end()) { + return {}; + } + auto options = encryption_schema_extension::parse_options(i->second.value); + opt_wrapper opts(options); + + return opts(KEY_PROVIDER).value_or(std::string{}); +} + +namespace bfs = std::filesystem; + +class encryption_commitlog_file_extension : public db::commitlog_file_extension { + const ::shared_ptr _ctxt; + const options _opts; + + static const inline std::regex prop_expr = std::regex("^([^=]+)=(\\S+)$"); + static const inline sstring id_key = "key_id"; + static const inline std::string end_of_file_mark = "#-- end of file"; + +public: + encryption_commitlog_file_extension(::shared_ptr ctxt, options opts) + : _ctxt(ctxt) + , _opts(std::move(opts)) + {} + sstring config_name(const sstring& filename) const { + bfs::path p(filename); + auto dir = p.parent_path(); + auto file = p.filename(); + return (dir / bfs::path("." + file.string())).string(); + } + future wrap_file(const sstring& filename, file f, open_flags flags) override { + auto cfg_file = config_name(filename); + + if (flags == open_flags::ro) { + return file_exists(cfg_file).then([=, this](bool exists) { + if (!exists) { + // #1681 if file system errors caused the options file to simply not exist, + // we can at least hope that the file itself is not very encrypted either. + // But who knows. Will probably cause data corruption. + logg.info("Commitlog segment {} has no encryption info. Opening unencrypted.", filename); + return make_ready_future(std::move(f)); + } + return read_text_file_fully(cfg_file).then([f, this, filename](temporary_buffer buf) { + std::istringstream ss(std::string(buf.begin(), buf.end())); + options opts; + std::string line; + bool has_eof = false; + while (std::getline(ss, line)) { + std::smatch m; + if (std::regex_match(line, m, prop_expr)) { + auto k = m[1].str(); + auto v = m[2].str(); + opts[k] = v; + } else if (line == end_of_file_mark) { + has_eof = true; + } + } + + // #1682 - if we crashed while writing the options file, + // it is quite possible that we are eventually trying to + // open + replay an (empty) CL file, but cannot read the + // properties now, since _our_ metadata is empty/truncated + if (!has_eof) { + // just return the unwrapped file. + logg.info("Commitlog segment {} has incomplete encryption info. Opening unencrypted.", filename); + return make_ready_future(std::move(f)); + } + opt_bytes id; + if (opts.count(id_key)) { + id = base64_decode(opts[id_key]); + } + + auto provider = _ctxt->get_provider(opts); + + logg.debug("Open commitlog segment {} using {} (id: {})", filename, *provider, id); + auto info = make_shared(get_key_info(opts)); + return provider->key(*info, id).then([f, info](std::tuple, opt_bytes> k) { + return make_ready_future(make_encrypted_file(f, std::get<0>(k))); + }); + }); + }); + } else { + auto provider = _ctxt->get_provider(_opts); + auto info = make_shared(get_key_info(_opts)); + return provider->key(*info).then([f, this, info, cfg_file, filename, &provider = *provider](std::tuple, opt_bytes> k_id) { + auto&& k = std::get<0>(k_id); + auto&& id = std::get<1>(k_id); + std::ostringstream ss; + for (auto&p : _opts) { + ss << p.first << "=" << p.second << std::endl; + } + if (id) { + ss << id_key << "=" << base64_encode(*id) << std::endl; + } + ss << end_of_file_mark << std::endl; + + logg.debug("Creating commitlog segment {} using {} (id: {})", filename, provider, id); + + return write_text_file_fully(cfg_file, ss.str()).then([f, k] { + return make_ready_future(make_encrypted_file(f, k)); + }); + }); + } + } + future<> before_delete(const sstring& filename) override { + auto cfg_file = config_name(filename); + return file_exists(cfg_file).then([cfg_file](bool b) { + return b ? remove_file(cfg_file) : make_ready_future(); + }); + } +}; + +future> register_extensions(const db::config&, std::unique_ptr cfg_in, db::extensions& exts, const ::service_set& services) { + auto& cfg = *cfg_in; + auto ctxt = ::make_shared(std::move(cfg_in), services); + // Note: extensions are immutable and shared across shards. + // Object in them must be stateless. We anchor the context in the + // extension objects, and while it is not as such 100% stateless, + // it is close enough. + exts.add_schema_extension(encryption_attribute, [ctxt](auto v) { + return encryption_schema_extension::parse(*ctxt, std::move(v)); + }); + exts.add_sstable_file_io_extension(encryption_attribute, std::make_unique(ctxt)); + + auto maybe_get_options = [&](const utils::config_file::string_map& map, const sstring& what) -> std::optional { + options opts(map.begin(), map.end()); + opt_wrapper sie(opts); + if (!::strcasecmp(sie("enabled").value_or("false").c_str(), "false")) { + return std::nullopt; + } + // commitlog/system table encryption/global user encryption should not use replicated keys, + // We default to local keys, but KMIP/KMS is ok as well (better in fact). + opts[KEY_PROVIDER] = sie(KEY_PROVIDER).value_or(LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY); + if (opts[KEY_PROVIDER] == LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY && !sie(SECRET_KEY_FILE)) { + // system encryption uses different key folder than user tables. + // explicitly set the key file path + opts[SECRET_KEY_FILE] = (bfs::path(cfg.system_key_directory()) / bfs::path("system") / bfs::path(sie("key_name").value_or("system_table_keytab"))).string(); + } + // forbid replicated. we cannot guarantee being able to open sstables on populate + if (opts[KEY_PROVIDER] == REPLICATED_KEY_PROVIDER_FACTORY) { + throw std::invalid_argument("Replicated provider is not allowed for " + what); + } + return opts; + }; + + future<> f = make_ready_future<>(); + + auto opts = maybe_get_options(cfg.system_info_encryption(), "system table encryption"); + + if (opts) { + logg.info("Adding system info encryption using {}", *opts); + + exts.add_commitlog_file_extension(encryption_attribute, std::make_unique(ctxt, *opts)); + + // modify schemas for tables holding sensitive data to use encryption w. key described + // by the opts. + // since schemas are duplicated across shards, we must call to each shard and augument + // them all. + // Since we are in pre-init phase, this should be safe. + f = f.then([opts = *opts, &exts] { + return smp::invoke_on_all([opts = make_lw_shared(opts), &exts] () mutable { + auto& f = exts.schema_extensions().at(encryption_attribute); + for (auto& s : { db::system_keyspace::paxos(), db::system_keyspace::batchlog(), db::system_keyspace::dicts() }) { + exts.add_extension_to_schema(s, encryption_attribute, f(*opts)); + } + }); + }); + } + + if (cfg.config_encryption_active()) { + f = f.then([&cfg, ctxt] { + return ctxt->load_config_encryption_key(cfg.config_encryption_key_name()); + }); + } + + + if (!cfg.kmip_hosts().empty()) { + // only pre-create on shard 0. + f = f.then([&cfg, ctxt] { + return parallel_for_each(cfg.kmip_hosts(), [ctxt](auto& p) { + auto host = ctxt->get_kmip_host(p.first); + return host->connect(); + }); + }); + } + + if (!cfg.kms_hosts().empty()) { + // only pre-create on shard 0. + f = f.then([&cfg, ctxt] { + return parallel_for_each(cfg.kms_hosts(), [ctxt](auto& p) { + auto host = ctxt->get_kms_host(p.first); + return host->init(); + }); + }); + } + + if (!cfg.gcp_hosts().empty()) { + // only pre-create on shard 0. + f = f.then([&cfg, ctxt] { + return parallel_for_each(cfg.gcp_hosts(), [ctxt](auto& p) { + auto host = ctxt->get_gcp_host(p.first); + return host->init(); + }); + }); + } + + replicated_key_provider_factory::init(exts); + + auto user_opts = maybe_get_options(cfg.user_info_encryption(), "user table encryption"); + + if (user_opts) { + logg.info("Adding user info encryption using {}", *user_opts); + + f = f.then([user_opts = *user_opts, ctxt] { + return smp::invoke_on_all([user_opts = make_lw_shared(user_opts), ctxt]() { + auto ext = encryption_schema_extension::create(*ctxt, *user_opts); + ctxt->add_global_user_encryption(std::move(ext)); + }); + }); + } + + return f.then([ctxt]() -> ::shared_ptr { + return ctxt; + }); +} + +} + diff --git a/ent/encryption/encryption.hh b/ent/encryption/encryption.hh new file mode 100644 index 000000000000..d70fd3ae19d7 --- /dev/null +++ b/ent/encryption/encryption.hh @@ -0,0 +1,196 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include + +#include +#include +#include +#include + +#include +#include + +#include "../../bytes.hh" +#include "../../compress.hh" + +class service_set; + +namespace replica { +class database; +} + +namespace db { +class config; +class extensions; +} + +namespace cql3 { +class query_processor; +} +namespace service { +class storage_service; +class migration_manager; +} + +namespace sstables { + class sstable; +} + +namespace encryption { +inline const sstring KEY_PROVIDER = "key_provider"; +inline const sstring SECRET_KEY_PROVIDER_FACTORY_CLASS = "secret_key_provider_factory_class"; +inline const sstring SECRET_KEY_FILE = "secret_key_file"; +inline const sstring SYSTEM_KEY_FILE = "system_key_file"; +inline const sstring CIPHER_ALGORITHM = "cipher_algorithm"; +inline const sstring SECRET_KEY_STRENGTH = "secret_key_strength"; + +inline const sstring HOST_NAME = "kmip_host"; +inline const sstring TEMPLATE_NAME = "template_name"; +inline const sstring KEY_NAMESPACE = "key_namespace"; + +bytes base64_decode(const sstring&, size_t off = 0, size_t n = sstring::npos); +sstring base64_encode(const bytes&, size_t off = 0, size_t n = bytes::npos); +bytes calculate_md5(const bytes&, size_t off = 0, size_t n = bytes::npos); +bytes calculate_sha256(const bytes&, size_t off = 0, size_t n = bytes::npos); +bytes calculate_sha256(bytes_view); +bytes hmac_sha256(bytes_view msg, bytes_view key); + +future> read_text_file_fully(const sstring&); +future<> write_text_file_fully(const sstring&, temporary_buffer); +future<> write_text_file_fully(const sstring&, const sstring&); + +std::optional parse_expiry(std::optional); + +class symmetric_key; +struct key_info; + +using options = std::map; +using opt_bytes = std::optional; +using key_ptr = shared_ptr; + +/** + * wrapper for "options" (map) to provide an + * interface returning empty optionals for + * non-available values. Makes query simpler + * and allows .value_or(...)-statements, which + * are neat for default values. + * + * In the long run one could contemplate + * using non-std maps with similar built-in + * functionality for all our various configs + * in the system, but for now we are firmly + * entrenched in map + */ +template +class map_wrapper { + const Map& _options; +public: + using mapped_type = typename Map::mapped_type; + using key_type = typename Map::key_type; + + map_wrapper(const Map& opts) + : _options(opts) + {} + + std::optional operator()(const key_type& k) const { + auto i = _options.find(k); + if (i != _options.end()) { + return i->second; + } + return std::nullopt; + } +}; + +using opt_wrapper = map_wrapper; + +key_info get_key_info(const options&); + +class encryption_context; + +class key_provider { +public: + virtual ~key_provider() + {} + virtual future> key(const key_info&, opt_bytes = {}) = 0; + virtual future<> validate() const { + return make_ready_future<>(); + } + virtual bool should_delay_read(const opt_bytes&) const { + return false; + } +private: + friend std::ostream& operator<<(std::ostream&, const key_provider&); + virtual void print(std::ostream&) const = 0; +}; + +std::ostream& operator<<(std::ostream&, const key_provider&); + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +namespace encryption { + +class key_provider_factory { +public: + virtual ~key_provider_factory() + {} + virtual shared_ptr get_provider(encryption_context& c, const options&) = 0; +}; + +class encryption_config; +class system_key; +class kmip_host; +class kms_host; +class gcp_host; + +/** + * Context is a singleton object, shared across shards. I.e. even though there are obvious mutating + * calls in it, it guarantees thread/shard safety. + * + * Why is this not a sharded thingamajing? Because its own instance methods need to send itself + * as a shard-safe object forwards, and thus need to know that same shard, which breaks the circle of + * ownership and stuff. + */ +class encryption_context { +public: + virtual ~encryption_context() = default; + virtual shared_ptr get_provider(const options&) = 0; + virtual shared_ptr get_system_key(const sstring&) = 0; + virtual shared_ptr get_kmip_host(const sstring&) = 0; + virtual shared_ptr get_kms_host(const sstring&) = 0; + virtual shared_ptr get_gcp_host(const sstring&) = 0; + + virtual shared_ptr get_cached_provider(const sstring& id) const = 0; + virtual void cache_provider(const sstring& id, shared_ptr) = 0; + + virtual const encryption_config& config() const = 0; + virtual shared_ptr get_config_encryption_key() const = 0; + + virtual distributed& get_query_processor() const = 0; + virtual distributed& get_storage_service() const = 0; + virtual distributed& get_database() const = 0; + virtual distributed& get_migration_manager() const = 0; + + sstring maybe_decrypt_config_value(const sstring&) const; + + virtual future<> start() = 0; + virtual future<> stop() = 0; +}; + +future> +register_extensions(const db::config&, std::unique_ptr, db::extensions&, const ::service_set&); + +// for testing +std::string encryption_provider(const sstables::sstable&); +} + diff --git a/ent/encryption/encryption_config.cc b/ent/encryption/encryption_config.cc new file mode 100644 index 000000000000..46e819025d2f --- /dev/null +++ b/ent/encryption/encryption_config.cc @@ -0,0 +1,164 @@ +/* + * Copyright (C) 2015 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include + +#include "db/config.hh" +#include "utils/config_file_impl.hh" + +#include "init.hh" +#include "encryption_config.hh" +#include "encryption.hh" + +#include + +encryption::encryption_config::encryption_config() + : config_file() +// BEGIN entry definitions + + , system_key_directory(this, "system_key_directory", value_status::Used, "/etc/scylla/conf/resources/system_keys", + R"foo(The directory where system keys are kept + +This directory should have 700 permissions and belong to the scylla user)foo") + + , config_encryption_active(this, "config_encryption_active", value_status::Used, false, "") + + , config_encryption_key_name(this, "config_encryption_key_name", value_status::Used, "system_key", + "Set to the local encryption key filename or KMIP key URL to use for configuration file property value decryption") + + , system_info_encryption(this, "system_info_encryption", value_status::Used, + { { "enabled", "false" }, { CIPHER_ALGORITHM, + "AES/CBC/PKCS5Padding" }, { + SECRET_KEY_STRENGTH, "128" }, + }, + R"foo(System information encryption settings + +If enabled, system tables that may contain sensitive information (system.batchlog, +system.paxos), hints files and commit logs are encrypted with the +encryption settings below. + +When enabling system table encryption on a node with existing data, run +`nodetool upgradesstables -a` on the listed tables to encrypt existing data. + +When tracing is enabled, sensitive info will be written into the tables in the +system_traces keyspace. Those tables should be configured to encrypt their data +on disk. + +It is recommended to use remote encryption keys from a KMIP server when using +Transparent Data Encryption (TDE) features. +Local key support is provided when a KMIP server is not available. + +See the scylla documentation for available key providers and their properties. +)foo") + , kmip_hosts(this, "kmip_hosts", value_status::Used, { }, + R"foo(KMIP host(s). + +The unique name of kmip host/cluster that can be referenced in table schema. + +host.yourdomain.com={ hosts=[, ...], keyfile=/path/to/keyfile, truststore=/path/to/truststore.pem, key_cache_millis=, timeout= }:... + +The KMIP connection management only supports failover, so all requests will go through a +single KMIP server. There is no load balancing, as no KMIP servers (at the time of this writing) +support read replication, or other strategies for availability. + +Hosts are tried in the order they appear here. Add them in the same sequence they'll fail over in. + +KMIP requests will fail over/retry 'max_command_retries' times (default 3) + +)foo") + , kms_hosts(this, "kms_hosts", value_status::Used, { }, + R"foo(KMS host(s). + +The unique name of kms host that can be referenced in table schema. + +host.yourdomain.com={ endpoint=, aws_access_key_id=, aws_secret_access_key=, aws_profile, aws_region=, aws_use_ec2_credentials, aws_use_ec2_region=, aws_assume_role_arn=, master_key=, keyfile=/path/to/keyfile, truststore=/path/to/truststore.pem, key_cache_millis=, timeout= }:... + +Actual connection can be either an explicit endpoint (:), or selected automatic via aws_region. + +If aws_use_ec2_region is true, regions is instead queried from EC2 metadata. + +Authentication can be explicit with aws_access_key_id and aws_secret_access_key. Either secret or both can be ommitted +in which case the provider will try to read them from AWS credentials in ~/.aws/credentials + +If aws_use_ec2_credentials is true, authentication is instead queried from EC2 metadata. + +If aws_assume_role_arn is set, scylla will issue an AssumeRole command and use the resulting security token for key operations. + +master_key is an AWS KMS key id or alias from which all keys used for actual encryption of scylla data will be derived. +This key must be pre-created with access policy allowing the above AWS id Encrypt, Decrypt and GenerateDataKey operations. + +)foo") + , gcp_hosts(this, "gcp_hosts", value_status::Used, { }, + R"foo(Google Compute Engine KMS host(s). + +The unique name of GCP kms host that can be referenced in table schema. + +gcp_project_id=, gcp_location=, master_key=, gcp_credentials_file=, gcp_impersonate_service_account=,keyfile=/path/to/keyfile, truststore=/path/to/truststore.pem, key_cache_millis=, timeout= }:... + +Authentication can be explicit with auth_file or by resolving default credentials (see google docs). + +If use_gcp_machine_credentials is true, authentication is instead queried from GCP metadata. + +auth_file can contain either a user, service or impersonated service account. + +master_key is an GCP KMS key name from which all keys used for actual encryption of scylla data will be derived. +This key must be pre-created with access policy allowing the above credentials Encrypt and Decrypt operations. + +)foo") + , user_info_encryption(this, "user_info_encryption", value_status::Used, + { { "enabled", "false" }, { CIPHER_ALGORITHM, + "AES/CBC/PKCS5Padding" }, { + SECRET_KEY_STRENGTH, "128" }, + }, + R"foo(Global user table encryption settings. If enabled, all user tables + will be encrypted using the provided settings, unless overridden + by table scylla_encryption_options.)foo") + , allow_per_table_encryption(this, "allow_per_table_encryption", value_status::Used, true, + "If 'user_info_encryption` is enabled this controls whether specifying per-table encryption using create/alter table is allowed" + ) + + +// END entry definitions +{} + +static class : public configurable { + std::unordered_map> _cfgs; + +public: + void append_options(db::config& cfg, boost::program_options::options_description_easy_init& init) override { + // While it is fine for normal execution to have just one, static (us) encryption config, + // it does not work well with unit testing, where we repeatedly create new cql_test_envs etc, + // since new config values will not be overwritten due to the actual named_values being shared here. + // Fix this (temporarily) by simply keeping a local map cfg->ecfg and using these. + // TODO: improve this by allowing db::config to hold named sub->configs (mapping config file objects). + if (_cfgs.count(&cfg)) { + throw std::runtime_error("Config already processed"); + } + auto& ccfg = _cfgs.emplace(&cfg, std::make_unique()).first->second; + // hook into main scylla.yaml. + cfg.add(ccfg->values()); + } + future initialize_ex(const boost::program_options::variables_map& opts, const db::config& cfg, db::extensions& exts, const service_set& services) override { + auto ccfg = _cfgs.count(&cfg) ? std::move(_cfgs.at(&cfg)) : std::make_unique(); + _cfgs.erase(&cfg); + auto ctxt = co_await encryption::register_extensions(cfg, std::move(ccfg), exts, services); + co_return [ctxt](system_state e) -> future<> { + switch (e) { + case system_state::started: + co_await ctxt->start(); + break; + case system_state::stopped: + co_await ctxt->stop(); + break; + default: + break; + } + }; + } +} cfg; diff --git a/ent/encryption/encryption_config.hh b/ent/encryption/encryption_config.hh new file mode 100644 index 000000000000..ced6f8ecf177 --- /dev/null +++ b/ent/encryption/encryption_config.hh @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "../../utils/config_file.hh" + +namespace encryption { + +class encryption_config : public utils::config_file { +public: + encryption_config(); + + typedef std::unordered_map string_string_map; + + named_value system_key_directory; + named_value config_encryption_active; + named_value config_encryption_key_name; + named_value system_info_encryption; + named_value kmip_hosts; + named_value kms_hosts; + named_value gcp_hosts; + named_value user_info_encryption; + named_value allow_per_table_encryption; +}; + +} diff --git a/ent/encryption/encryption_exceptions.hh b/ent/encryption/encryption_exceptions.hh new file mode 100644 index 000000000000..7f551a7418ab --- /dev/null +++ b/ent/encryption/encryption_exceptions.hh @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2024 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "db/extensions.hh" + +namespace encryption { + +using base_error = db::extension_storage_exception; + +class permission_error : public db::extension_storage_permission_error { +public: + using mybase = db::extension_storage_permission_error; + using mybase::mybase; +}; + +class configuration_error : public db::extension_storage_misconfigured { +public: + using mybase = db::extension_storage_misconfigured; + using mybase::mybase; +}; + +class service_error : public base_error { +public: + using base_error::base_error; +}; + +class missing_resource_error : public db::extension_storage_resource_unavailable { +public: + using mybase = db::extension_storage_resource_unavailable; + using mybase::mybase; +}; + +// #4970 - not 100% correct, but network errors are +// generally intermittent/recoverable. Mark as a non-isolating +// error. +class network_error : public missing_resource_error { +public: + using missing_resource_error::missing_resource_error; +}; + +class malformed_response_error : public service_error { +public: + using service_error::service_error; +}; + +} + diff --git a/ent/encryption/gcp_host.cc b/ent/encryption/gcp_host.cc new file mode 100644 index 000000000000..23b2c3964471 --- /dev/null +++ b/ent/encryption/gcp_host.cc @@ -0,0 +1,1031 @@ +/* + * Copyright (C) 2024 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + +#define CPP_JWT_USE_VENDORED_NLOHMANN_JSON +#include + +#include +#include +#include +#include "utils/to_string.hh" + +#include "gcp_host.hh" +#include "encryption.hh" +#include "encryption_exceptions.hh" +#include "symmetric_key.hh" +#include "utils/hash.hh" +#include "utils/loading_cache.hh" +#include "utils/UUID.hh" +#include "utils/UUID_gen.hh" +#include "utils/rjson.hh" +#include "marshal_exception.hh" +#include "db/config.hh" + +using namespace std::chrono_literals; +using namespace std::string_literals; + +logger gcp_log("gcp"); + +namespace encryption { +bool operator==(const gcp_host::credentials_source& k1, const gcp_host::credentials_source& k2) { + return k1.gcp_credentials_file == k2.gcp_credentials_file && k1.gcp_impersonate_service_account == k2.gcp_impersonate_service_account; +} +} + +template<> +struct fmt::formatter { + constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); } + auto format(const encryption::gcp_host::credentials_source& d, fmt::format_context& ctxt) const { + return fmt::format_to(ctxt.out(), "{{ gcp_credentials_file = {}, gcp_impersonate_service_account = {} }}", d.gcp_credentials_file, d.gcp_impersonate_service_account); + } +}; + +template<> +struct std::hash { + size_t operator()(const encryption::gcp_host::credentials_source& a) const { + return utils::tuple_hash{}(std::tie(a.gcp_credentials_file, a.gcp_impersonate_service_account)); + } +}; + +class encryption::gcp_host::impl { +public: + // set a rather long expiry. normal KMS policies are 365-day rotation of keys. + // we can do with 10 minutes. CMH. maybe even longer. + // (see comments below on what keys are here) + static inline constexpr std::chrono::milliseconds default_expiry = 600s; + static inline constexpr std::chrono::milliseconds default_refresh = 1200s; + + impl(encryption_context& ctxt, const std::string& name, const host_options& options) + : _ctxt(ctxt) + , _name(name) + , _options(options) + , _attr_cache(utils::loading_cache_config{ + .max_size = std::numeric_limits::max(), + .expiry = options.key_cache_expiry.value_or(default_expiry), + .refresh = options.key_cache_refresh.value_or(default_refresh)}, gcp_log, std::bind_front(&impl::create_key, this)) + , _id_cache(utils::loading_cache_config{ + .max_size = std::numeric_limits::max(), + .expiry = options.key_cache_expiry.value_or(default_expiry), + .refresh = options.key_cache_refresh.value_or(default_refresh)}, gcp_log, std::bind_front(&impl::find_key, this)) + {} + ~impl() = default; + + future<> init(); + const host_options& options() const { + return _options; + } + + future, id_type>> get_or_create_key(const key_info&, const option_override* = nullptr); + future> get_key_by_id(const id_type&, const key_info&, const option_override* = nullptr); + + using scopes_type = std::string; // space separated. avoids some transforms. makes other easy. +private: + class httpclient; + using key_and_id_type = std::tuple, id_type>; + + struct attr_cache_key { + credentials_source src; + std::string master_key; + key_info info; + bool operator==(const attr_cache_key& v) const = default; + }; + + friend struct fmt::formatter; + + struct attr_cache_key_hash { + size_t operator()(const attr_cache_key& k) const { + return utils::tuple_hash()(std::tie(k.master_key, k.src, k.info.len)); + } + }; + + struct id_cache_key { + credentials_source src; + id_type id; + bool operator==(const id_cache_key& v) const = default; + }; + + friend struct fmt::formatter; + + struct id_cache_key_hash { + size_t operator()(const id_cache_key& k) const { + return utils::tuple_hash()(std::tie(k.id, k.src)); + } + }; + + future create_key(const attr_cache_key&); + future find_key(const id_cache_key&); + + using timeout_clock = std::chrono::system_clock; + using timestamp_type = typename timeout_clock::time_point; + + struct access_token; + struct user_credentials; + struct service_account_credentials; + struct impersonated_service_account_credentials; + struct compute_engine_credentials{}; + + struct google_credentials; + + + struct access_token { + access_token() = default; + access_token(const rjson::value&); + + std::string token; + timestamp_type expiry; + scopes_type scopes; + + bool empty() const; + bool expired() const; + }; + + struct user_credentials { + user_credentials(const rjson::value&); + + std::string client_id; + std::string client_secret; + std::string refresh_token; + std::string access_token; + std::string quota_project_id; + }; + + using p_key = std::unique_ptr; + + struct service_account_credentials { + service_account_credentials(const rjson::value&); + + std::string client_id; + std::string client_email; + std::string private_key_id; + std::string private_key_pkcs8; + std::string token_server_uri; + std::string project_id; + std::string quota_project_id; + }; + + struct impersonated_service_account_credentials { + impersonated_service_account_credentials(std::string principal, google_credentials&&); + impersonated_service_account_credentials(const rjson::value&); + + std::vector delegates; + std::vector scopes; + std::string quota_project_id; + std::string iam_endpoint_override; + std::string target_principal; + + std::unique_ptr source_credentials; + access_token token; + }; + + using credentials_variant = std::variant< + user_credentials, + service_account_credentials, + impersonated_service_account_credentials, + compute_engine_credentials + >; + + struct google_credentials { + google_credentials(google_credentials&&) = default; + google_credentials(credentials_variant&& c) + : credentials(std::move(c)) + {} + google_credentials& operator=(google_credentials&&) = default; + credentials_variant credentials; + access_token token; + }; + + google_credentials from_data(std::string_view) const; + google_credentials from_data(const temporary_buffer& buf) const { + return from_data(std::string_view(buf.get(), buf.size())); + } + future from_file(const std::string& path) const { + auto buf = co_await read_text_file_fully(path); + co_return from_data(std::string_view(buf.get(), buf.size())); + } + + future get_default_credentials(); + + future get_access_token(const google_credentials&, const scopes_type& scopes) const; + + future<> refresh(google_credentials&, const scopes_type&) const; + + using key_values = std::initializer_list>; + + static std::string body(key_values kv); + + future send_request(std::string_view uri, std::string body, std::string_view content_type, httpd::operation_type = httpd::operation_type::GET, key_values headers = {}) const; + future send_request(std::string_view uri, const rjson::value& body, httpd::operation_type = httpd::operation_type::GET, key_values headers = {}) const; + future<> send_request(std::string_view uri, std::string body, std::string_view content_type, const std::function&, httpd::operation_type = httpd::operation_type::GET, key_values headers = {}) const; + + static std::tuple parse_key(std::string_view); + + future gcp_auth_post_with_retry(std::string_view uri, const rjson::value& body, const credentials_source&); + + encryption_context& _ctxt; + std::string _name; + host_options _options; + + std::unordered_map _cached_credentials; + + utils::loading_cache, attr_cache_key_hash> _attr_cache; + utils::loading_cache, id_cache_key_hash> _id_cache; + shared_ptr _creds; + std::unordered_map> _cache; + bool _initialized = false; + bool _checked_is_on_gce = false; + bool _is_on_gce = false; +}; + +template +static T get_option(const encryption::gcp_host::option_override* oov, std::optional C::* f, const T& def) { + if (oov) { + return (oov->*f).value_or(def); + } + return {}; +}; + +future, encryption::gcp_host::id_type>> encryption::gcp_host::impl::get_or_create_key(const key_info& info, const option_override* oov) { + attr_cache_key key { + .src = { + .gcp_credentials_file = get_option(oov, &option_override::gcp_credentials_file, _options.gcp_credentials_file), + .gcp_impersonate_service_account = get_option(oov, &option_override::gcp_impersonate_service_account, _options.gcp_impersonate_service_account), + }, + .master_key = get_option(oov, &option_override::master_key, _options.master_key), + .info = info, + }; + + if (key.master_key.empty()) { + throw configuration_error("No master key set in gcp host config or encryption attributes"); + } + try { + co_return co_await _attr_cache.get(key); + } catch (base_error&) { + throw; + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(fmt::format("get_or_create_key: {}", e.what()))); + } catch (...) { + std::throw_with_nested(service_error(fmt::format("get_or_create_key: {}", std::current_exception()))); + } +} + +future> encryption::gcp_host::impl::get_key_by_id(const id_type& id, const key_info& info, const option_override* oov) { + // note: since KMS does not really have any actual "key" association of id -> key, + // we only cache/query raw bytes of some length. (See below). + // Thus keys returned are always new objects. But they are not huge... + id_cache_key key { + .src = { + .gcp_credentials_file = get_option(oov, &option_override::gcp_credentials_file, _options.gcp_credentials_file), + .gcp_impersonate_service_account = get_option(oov, &option_override::gcp_impersonate_service_account, _options.gcp_impersonate_service_account), + }, + .id = id, + }; + try { + auto data = co_await _id_cache.get(key); + co_return make_shared(info, data); + } catch (base_error&) { + throw; + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(fmt::format("get_key_by_id: {}", e.what()))); + } catch (...) { + std::throw_with_nested(service_error(fmt::format("get_key_by_id: {}", std::current_exception()))); + } +} + +static const char CREDENTIAL_ENV_VAR[] = "GOOGLE_APPLICATION_CREDENTIALS"; +static const char WELL_KNOWN_CREDENTIALS_FILE[] = "application_default_credentials.json"; +static const char CLOUDSDK_CONFIG_DIRECTORY[] = "gcloud"; + +static const char USER_FILE_TYPE[] = "authorized_user"; +static const char SERVICE_ACCOUNT_FILE_TYPE[] = "service_account"; +static const char IMPERSONATED_SERVICE_ACCOUNT_FILE_TYPE[] = "impersonated_service_account"; + +static const char GCE_METADATA_HOST_ENV_VAR[] = "GCE_METADATA_HOST"; + +static const char DEFAULT_METADATA_SERVER_URL[] = "http://metadata.google.internal";; + +static const char METADATA_FLAVOR[] = "Metadata-Flavor"; +static const char GOOGLE[] = "Google"; + +static const char TOKEN_SERVER_URI[] = "https://oauth2.googleapis.com/token"; + +static const char AUTHORIZATION[] = "Authorization"; + +static const char KMS_SCOPE[] = "https://www.googleapis.com/auth/cloudkms"; +static const char CLOUD_PLATFORM_SCOPE[] = "https://www.googleapis.com/auth/cloud-platform"; + +//static const char[] CLOUD_SHELL_ENV_VAR = "DEVSHELL_CLIENT_PORT"; +//static const char[] SKIP_APP_ENGINE_ENV_VAR = "GOOGLE_APPLICATION_CREDENTIALS_SKIP_APP_ENGINE"; +//static const char[] NO_GCE_CHECK_ENV_VAR = "NO_GCE_CHECK"; +//static const char[] GCE_METADATA_HOST_ENV_VAR = "GCE_METADATA_HOST"; + +bool encryption::gcp_host::impl::access_token::empty() const { + return token.empty(); +} + +bool encryption::gcp_host::impl::access_token::expired() const { + if (empty()) { + return true; + } + return timeout_clock::now() >= this->expiry; +} + +encryption::gcp_host::impl::user_credentials::user_credentials(const rjson::value& v) + : client_id(rjson::get(v, "client_id")) + , client_secret(rjson::get(v, "client_secret")) + , refresh_token(rjson::get(v, "refresh_token")) + , quota_project_id(rjson::get_opt(v, "refresh_token").value_or("")) +{} + +encryption::gcp_host::impl::service_account_credentials::service_account_credentials(const rjson::value& v) + : client_id(rjson::get(v, "client_id")) + , client_email(rjson::get(v, "client_email")) + , private_key_id(rjson::get(v, "private_key_id")) + , private_key_pkcs8(rjson::get(v, "private_key")) + , token_server_uri([&] { + auto token_uri = rjson::get_opt(v, "token_uri"); + if (token_uri) { + // TODO: verify uri + return *token_uri; + } + return std::string{}; + }()) + , project_id(rjson::get_opt(v, "project_id").value_or("")) + , quota_project_id(rjson::get_opt(v, "refresh_token").value_or("")) +{} + + +encryption::gcp_host::impl::impersonated_service_account_credentials::impersonated_service_account_credentials(std::string principal, google_credentials&& c) + : target_principal(std::move(principal)) + , source_credentials(std::make_unique(std::move(c))) +{} + +encryption::gcp_host::impl::impersonated_service_account_credentials::impersonated_service_account_credentials(const rjson::value& v) + : delegates([&] { + std::vector res; + auto tmp = rjson::find(v, "delegates"); + if (tmp) { + if (!tmp->IsArray()) { + throw configuration_error("Malformed json"); + } + + for (const auto& d : tmp->GetArray()) { + res.emplace_back(std::string(rjson::to_string_view(d))); + } + } + return res; + }()) + , quota_project_id(rjson::get_opt(v, "quota_project_id").value_or("")) + , target_principal([&] { + auto url = rjson::get(v, "service_account_impersonation_url"); + + auto si = url.find_last_of('/'); + auto ei = url.find(":generateAccessToken"); + + if (si != std::string::npos && ei != std::string::npos && si < ei) { + return url.substr(si + 1, ei - si - 1); + } + throw configuration_error( "Unable to determine target principal from service account impersonation URL."); + }()) + , source_credentials([&]() -> decltype(source_credentials) { + auto& scjson = rjson::get(v, "source_credentials"); + auto type = rjson::get(scjson, "type"); + + if (type == USER_FILE_TYPE) { + return std::make_unique(user_credentials(scjson)); + } else if (type == SERVICE_ACCOUNT_FILE_TYPE) { + return std::make_unique(service_account_credentials(scjson)); + } + throw configuration_error(fmt::format("A credential of type {} is not supported as source credential for impersonation.", type)); + }()) +{} + +encryption::gcp_host::impl::google_credentials +encryption::gcp_host::impl::from_data(std::string_view content) const { + auto json = rjson::parse(content); + auto type = rjson::get_opt(json, "type"); + + if (!type) { + throw configuration_error("Error reading credentials from stream, 'type' field not specified."); + } + if (type == USER_FILE_TYPE) { + return google_credentials(user_credentials(json)); + } + if (type == SERVICE_ACCOUNT_FILE_TYPE) { + return google_credentials(service_account_credentials(json)); + } + if (type == IMPERSONATED_SERVICE_ACCOUNT_FILE_TYPE) { + return google_credentials(impersonated_service_account_credentials(json)); + } + throw configuration_error(fmt::format( + "Error reading credentials from stream, 'type' value '{}' not recognized. Expecting '{}', '{}' or '{}'." + , type, USER_FILE_TYPE, SERVICE_ACCOUNT_FILE_TYPE, IMPERSONATED_SERVICE_ACCOUNT_FILE_TYPE)); +} + +static std::string get_metadata_server_url() { + auto meta_host = std::getenv(GCE_METADATA_HOST_ENV_VAR); + auto token_uri = meta_host ? std::string("http://") + meta_host : DEFAULT_METADATA_SERVER_URL; + return token_uri; +} + +future +encryption::gcp_host::impl::get_default_credentials() { + auto credentials_path = std::getenv(CREDENTIAL_ENV_VAR); + + if (credentials_path != nullptr && strlen(credentials_path)) { + gcp_log.debug("Attempting to load credentials from file: {}", credentials_path); + + try { + co_return co_await from_file(credentials_path); + } catch (...) { + std::throw_with_nested(configuration_error(fmt::format( + "Error reading credential file from environment variable {}, value '{}'" + , CREDENTIAL_ENV_VAR + , credentials_path + )) + ); + } + } + + { + std::string well_known_file; + auto env_path = std::getenv("CLOUDSDK_CONFIG"); + if (env_path) { + well_known_file = fmt::format("~/{}/{}", env_path, WELL_KNOWN_CREDENTIALS_FILE); + } else { + well_known_file = fmt::format("~/.config/{}/{}", CLOUDSDK_CONFIG_DIRECTORY, WELL_KNOWN_CREDENTIALS_FILE); + } + + if (co_await seastar::file_exists(well_known_file)) { + gcp_log.debug("Attempting to load credentials from well known file: {}", well_known_file); + try { + co_return co_await from_file(well_known_file); + } catch (...) { + std::throw_with_nested(configuration_error(fmt::format( + "Error reading credential file from location {}" + , well_known_file + )) + ); + } + } + } + + { + // Then try Compute Engine and GAE 8 standard environment + gcp_log.debug("Attempting to load credentials from GCE"); + + auto is_on_gce = [this]() -> future { + if (_checked_is_on_gce) { + co_return _is_on_gce; + } + + auto token_uri = get_metadata_server_url(); + + for (int i = 1; i <= 3; ++i) { + try { + co_await send_request(token_uri, std::string{}, "", [&](const http::reply& rep, std::string_view) { + _checked_is_on_gce = true; + _is_on_gce = rep.get_header(METADATA_FLAVOR) == GOOGLE; + }, httpd::operation_type::GET, { { METADATA_FLAVOR, GOOGLE } }); + if (_checked_is_on_gce) { + co_return _is_on_gce;; + } + } catch (...) { + // TODO: handle timeout + break; + } + } + + auto linux_path = "/sys/class/dmi/id/product_name"; + if (co_await seastar::file_exists(linux_path)) { + auto f = file_desc::open(linux_path, O_RDONLY | O_CLOEXEC); + char buf[128] = {}; + f.read(buf, 128); + _is_on_gce = std::string_view(buf).find(GOOGLE) == 0; + } + + _checked_is_on_gce = true; + co_return _is_on_gce; + }; + + if (co_await is_on_gce()) { + co_return compute_engine_credentials{}; + } + } + + throw configuration_error("Could not determine initial credentials"); +} + +template +static void for_each_scope(const encryption::gcp_host::impl::scopes_type& s, Func&& f) { + size_t i = 0; + while(i < s.size()) { + auto j = s.find(' ', i + 1); + f(s.substr(i, j - i)); + i = j; + } +} + +encryption::gcp_host::impl::access_token::access_token(const rjson::value& json) + : token(rjson::get(json, "access_token")) + , expiry(timeout_clock::now() + std::chrono::seconds(rjson::get(json, "expires_in"))) + , scopes(rjson::get_opt(json, "scope").value_or("")) +{} + +std::string encryption::gcp_host::impl::body(key_values kv) { + std::ostringstream ss; + std::string_view sep = ""; + for (auto& [k, v] : kv) { + ss << sep << k << "=" << http::internal::url_encode(v); + sep = "&"; + } + return ss.str(); +} + +future encryption::gcp_host::impl::send_request(std::string_view uri, const rjson::value& body, httpd::operation_type op, key_values headers) const { + return send_request(uri, rjson::print(body), "application/json", op, std::move(headers)); +} + +future encryption::gcp_host::impl::send_request(std::string_view uri, std::string body, std::string_view content_type, httpd::operation_type op, key_values headers) const { + rjson::value v; + co_await send_request(uri, std::move(body), content_type, [&](const http::reply& rep, std::string_view s) { + if (rep._status != http::reply::status_type::ok) { + gcp_log.trace("Got unexpected reponse ({})", rep._status); + for (auto& [k, v] : rep._headers) { + gcp_log.trace("{}: {}", k, v); + } + gcp_log.trace("{}", s); + throw httpd::unexpected_status_error(rep._status); + } + v = rjson::parse(s); + }, op, std::move(headers)); + co_return v; +} + +future<> encryption::gcp_host::impl::send_request(std::string_view uri, std::string body, std::string_view content_type, const std::function& handler, httpd::operation_type op, key_values headers) const { + // Extremely simplified URI parsing. Does not handle any params etc. But we do not expect such here. + static boost::regex simple_url(R"foo((https?):\/\/([^\/:]+)(:\d+)?(\/.*)?)foo"); + + boost::smatch m; + std::string tmp(uri); + if (!boost::regex_match(tmp, m, simple_url)) { + throw std::invalid_argument(fmt::format("Could not parse URI {}", uri)); + } + + auto scheme = m[1].str(); + auto host = m[2].str(); + auto port = m[3].str(); + auto path = m[4].str(); + + auto addr = co_await net::dns::resolve_name(host, net::inet_address::family::INET /* CMH our client does not handle ipv6 well?*/); + auto certs = scheme == "https" + ? ::make_shared() + : shared_ptr() + ; + if (certs) { + if (!_options.priority_string.empty()) { + certs->set_priority_string(_options.priority_string); + } else { + certs->set_priority_string(db::config::default_tls_priority); + } + if (!_options.certfile.empty()) { + co_await certs->set_x509_key_file(_options.certfile, _options.keyfile, seastar::tls::x509_crt_format::PEM); + } + if (!_options.truststore.empty()) { + co_await certs->set_x509_trust_file(_options.truststore, seastar::tls::x509_crt_format::PEM); + } else { + co_await certs->set_system_trust(); + } + } + + uint16_t pi = port.empty() ? (certs ? 443 : 80) : uint16_t(std::stoi(port.substr(1))); + auto client = certs + ? http::experimental::client(socket_address(addr, pi), std::move(certs), host) + : http::experimental::client(socket_address(addr, pi)) + ; + if (path.empty()) { + path = "/"; + } + + gcp_log.trace("Resolved {} -> {}:{}{}", uri, addr, pi, path); + + auto req = http::request::make(op, host, path); + + for (auto& [k, v] : headers) { + req._headers[sstring(k)] = sstring(v); + } + + if (!body.empty()) { + if (content_type.empty()) { + content_type = "application/x-www-form-urlencoded"; + } + req.write_body("", std::move(body)); + req.set_mime_type(sstring(content_type)); + } + + gcp_log.trace("Sending {} request to {} ({}): {}", content_type, uri, headers, body); + + co_await client.make_request(std::move(req), [&] (const http::reply& rep, input_stream&& in) -> future<> { + auto&lh = handler; + auto lin = std::move(in); + auto result = co_await util::read_entire_stream_contiguous(lin); + gcp_log.trace("Got reponse {}: {}", int(rep._status), result); + lh(rep, result); + }); + + co_await client.close(); +} + + +future<> encryption::gcp_host::impl::refresh(google_credentials& c, const scopes_type& scopes) const { + if (!c.token.expired() && c.token.scopes == scopes) { + co_return; + } + c.token = co_await get_access_token(c, scopes); +} + +future +encryption::gcp_host::impl::get_access_token(const google_credentials& creds, const scopes_type& scope) const { + co_return co_await std::visit(overloaded_functor { + [&](const user_credentials& c) -> future { + assert(!c.refresh_token.empty()); + auto json = co_await send_request(TOKEN_SERVER_URI, body({ + { "client_id", c.client_id }, + { "client_secret", c.client_secret }, + { "refresh_token", c.refresh_token }, + { "grant_type", "grant_type" }, + }), "", httpd::operation_type::POST); + + co_return access_token{ json }; + }, + [&](const service_account_credentials& c) -> future { + using namespace jwt::params; + + jwt::jwt_object obj{algorithm("RS256"), secret(c.private_key_pkcs8), headers({{"kid", c.private_key_id }})}; + + auto uri = c.token_server_uri.empty() ? TOKEN_SERVER_URI : c.token_server_uri; + obj.add_claim("iss", c.client_email) + .add_claim("iat", timeout_clock::now()) + .add_claim("exp", timeout_clock::now() + std::chrono::seconds(3600)) + .add_claim("scope", scope) + .add_claim("aud", uri) + ; + auto sign = obj.signature(); + + auto json = co_await send_request(uri, body({ + { "grant_type", "urn:ietf:params:oauth:grant-type:jwt-bearer" }, + { "assertion", sign } + }), "", httpd::operation_type::POST); + co_return access_token{ json }; + }, + [&](const impersonated_service_account_credentials& c) -> future { + auto json_body = rjson::empty_object(); + auto scopes = rjson::empty_array(); + for_each_scope(scope, [&](std::string s) { + rjson::push_back(scopes, rjson::from_string(s)); + }); + + rjson::add(json_body, "scope", std::move(scopes)); + + if (!c.delegates.empty()) { + auto delegates = rjson::empty_array(); + for (auto& d : c.delegates) { + rjson::push_back(delegates, rjson::from_string(d)); + } + rjson::add(json_body, "delegates", std::move(delegates)); + } + + rjson::add(json_body, "lifetime", "3600s"); + + co_await refresh(*c.source_credentials, CLOUD_PLATFORM_SCOPE); + + auto endpoint = c.iam_endpoint_override.empty() + ? fmt::format("https://iamcredentials.googleapis.com/v1/projects/-/serviceAccounts/{}:generateAccessToken", c.target_principal) + : c.iam_endpoint_override + ; + auto json = co_await send_request(endpoint, json_body, httpd::operation_type::POST, { + { AUTHORIZATION, fmt::format("Bearer {}", c.source_credentials->token.token) }, + }); + + struct tm tmp; + ::strptime(rjson::get(json, "expireTime").data(), "%FT%TZ", &tmp); + + access_token a; + + a.expiry = timeout_clock::from_time_t(::mktime(&tmp)); + a.scopes = scope; + a.token = rjson::get(json, "accessToken"); + + co_return a; + }, + [this](const compute_engine_credentials& c) -> future { + auto meta_uri = get_metadata_server_url(); + auto token_uri = meta_uri + "/computeMetadata/v1/instance/service-accounts/default/token"; + try { + auto json = co_await send_request(token_uri, std::string{}, "", httpd::operation_type::GET, { { METADATA_FLAVOR, GOOGLE } }); + co_return access_token{ json }; + } catch (...) { + std::throw_with_nested(service_error("Unexpected Error code trying to get security access token from Compute Engine metadata for the default service account")); + } + } + }, creds.credentials); +} + +future encryption::gcp_host::impl::gcp_auth_post_with_retry(std::string_view uri, const rjson::value& body, const credentials_source& src) { + auto i = _cached_credentials.find(src); + if (i == _cached_credentials.end()) { + try { + auto c = !src.gcp_credentials_file.empty() + ? co_await from_file(src.gcp_credentials_file) + : co_await get_default_credentials() + ; + if (!src.gcp_credentials_file.empty()) { + gcp_log.trace("Loaded credentials from {}", src.gcp_credentials_file); + } + if (!src.gcp_impersonate_service_account.empty()) { + c = google_credentials(impersonated_service_account_credentials(src.gcp_impersonate_service_account, std::move(c))); + } + i = _cached_credentials.emplace(src, std::move(c)).first; + } catch (...) { + gcp_log.warn("Error resolving credentials for {}: {}", src, std::current_exception()); + throw; + } + } + + assert(i != _cached_credentials.end()); // should either be set now or we threw. + + auto& creds = i->second; + + int retries = 0; + + for (;;) { + try { + co_await this->refresh(creds, KMS_SCOPE); + } catch (...) { + std::throw_with_nested(permission_error("Error refreshing credentials")); + } + + try { + auto res = co_await send_request(uri, body, httpd::operation_type::POST, { + { AUTHORIZATION, fmt::format("Bearer {}", creds.token.token) }, + }); + co_return res; + } catch (httpd::unexpected_status_error& e) { + gcp_log.debug("{}: Got unexpected response: {}", uri, e.status()); + if (e.status() == http::reply::status_type::unauthorized && retries++ < 3) { + // refresh access token and retry. + continue; + } + if (e.status() == http::reply::status_type::unauthorized) { + std::throw_with_nested(permission_error(std::string(uri))); + } + std::throw_with_nested(service_error(std::string(uri))); + } catch (...) { + std::throw_with_nested(network_error(std::string(uri))); + } + } +} + +static constexpr char GCP_KMS_QUERY_TEMPLATE[] = "https://cloudkms.googleapis.com/v1/projects/{}/locations/{}/keyRings/{}/cryptoKeys/{}:{}"; + +future<> encryption::gcp_host::impl::init() { + if (_initialized) { + co_return; + } + + if (!_options.master_key.empty()) { + gcp_log.debug("Looking up master key"); + + attr_cache_key k{ + .src = _options, + .master_key = _options.master_key, + .info = key_info{ .alg = "AES", .len = 128 }, + }; + co_await create_key(k); + gcp_log.debug("Master key exists"); + } else { + gcp_log.info("No default master key configured. Not verifying."); + } + _initialized = true; +} + +std::tuple encryption::gcp_host::impl::parse_key(std::string_view spec) { + auto i = spec.find_last_of('/'); + if (i == std::string_view::npos) { + throw std::invalid_argument(fmt::format("Invalid master key spec '{}'. Must be in format /", spec)); + } + return std::make_tuple(std::string(spec.substr(0, i)), std::string(spec.substr(i + 1))); +} + +future encryption::gcp_host::impl::create_key(const attr_cache_key& k) { + auto& info = k.info; + + /** + * Google GCP KMS does allow us to create keys, but like AWS this would + * force us to deal with permissions and assignments etc. We instead + * require a pre-prepared key. + * + * Like AWS, we cannot get the actual key out, nor can we really bulk + * encrypt/decrypt things. So we do just like with AWS KMS, and generate + * a data key, and encrypt it as the key ID. + * + * For ID -> key, we simply split the ID into the encrypted key part, and + * the master key name part, decrypt the first using the second (AWS KMS Decrypt), + * and create a local key using the result. + * + * Data recovery: + * Assuming you have data encrypted using a KMS generated key, you will have + * metadata detailing algorithm, key length etc (see sstable metadata, and key info). + * Metadata will also include a byte blob representing the ID of the encryption key. + * For GCP KMS, the ID will actually be a text string: + * :: + * + * I.e. something like: + * mykeyring:mykey:e56sadfafa3324ff=/wfsdfwssdf + * + * The actual data key can be retreived by doing a KMS "Decrypt" of the data blob part + * using the KMS key referenced by the key ID. This gives back actual key data that can + * be used to create a symmetric_key with algo, length etc as specified by metadata. + * + */ + + // avoid creating too many keys and too many calls. If we are not shard 0, delegate there. + if (this_shard_id() != 0) { + auto [data, id] = co_await smp::submit_to(0, [this, k]() -> future> { + auto host = _ctxt.get_gcp_host(_name); + auto [key, id] = co_await host->_impl->_attr_cache.get(k); + co_return std::make_tuple(key != nullptr ? key->key() : bytes{}, id); + }); + co_return key_and_id_type{ + data.empty() ? nullptr : make_shared(info, data), + id + }; + } + + // note: since external keys are _not_ stored, + // there is nothing we can "look up" or anything. Always + // new key here. + + gcp_log.debug("Creating new key: {}", info); + + auto [keyring, keyname] = parse_key(k.master_key); + + auto key = make_shared(info); + auto url = fmt::format(GCP_KMS_QUERY_TEMPLATE, + _options.gcp_project_id, + _options.gcp_location, + keyring, + keyname, + "encrypt" + ); + auto query = rjson::empty_object(); + rjson::add(query, "plaintext", std::string(base64_encode(key->key()))); + + auto response = co_await gcp_auth_post_with_retry(url, query, k.src); + auto cipher = rjson::get(response, "ciphertext"); + auto data = base64_decode(cipher); + + auto sid = fmt::format("{}/{}:{}", keyring, keyname, cipher); + bytes id(sid.begin(), sid.end()); + + gcp_log.trace("Created key id {}", sid); + + co_return key_and_id_type{ key, id }; +} + +future encryption::gcp_host::impl::find_key(const id_cache_key& k) { + // avoid creating too many keys and too many calls. If we are not shard 0, delegate there. + if (this_shard_id() != 0) { + co_return co_await smp::submit_to(0, [this, k]() -> future { + auto host = _ctxt.get_gcp_host(_name); + auto bytes = co_await host->_impl->_id_cache.get(k); + co_return bytes; + }); + } + + // See create_key. ID consists of :. + // master id can contain ':', but blob will not. + // (we are being wasteful, and keeping the base64 encoding - easier to read) + std::string_view id(reinterpret_cast(k.id.data()), k.id.size()); + gcp_log.debug("Finding key: {}", id); + + auto pos = id.find_last_of(':'); + auto pos2 = id.find_last_of('/', pos - 1); + if (pos == id_type::npos || pos2 == id_type::npos || pos2 >= pos) { + throw std::invalid_argument(fmt::format("Not a valid key id: {}", id)); + } + + std::string keyring(id.begin(), id.begin() + pos2); + std::string keyname(id.begin() + pos2 + 1, id.begin() + pos); + std::string enc(id.begin() + pos + 1, id.end()); + + auto url = fmt::format(GCP_KMS_QUERY_TEMPLATE, + _options.gcp_project_id, + _options.gcp_location, + keyring, + keyname, + "decrypt" + ); + auto query = rjson::empty_object(); + rjson::add(query, "ciphertext", enc); + + auto response = co_await gcp_auth_post_with_retry(url, query, k.src); + auto data = base64_decode(rjson::get(response, "plaintext")); + + // we know nothing about key type etc, so just return data. + co_return data; +} + +encryption::gcp_host::gcp_host(encryption_context& ctxt, const std::string& name, const host_options& options) + : _impl(std::make_unique(ctxt, name, options)) +{} + +encryption::gcp_host::gcp_host(encryption_context& ctxt, const std::string& name, const std::unordered_map& map) + : gcp_host(ctxt, name, [&map] { + host_options opts; + map_wrapper> m(map); + + opts.master_key = m("master_key").value_or(""); + + opts.gcp_project_id = m("gcp_project_id").value_or(""); + opts.gcp_location = m("gcp_location").value_or(""); + + opts.gcp_credentials_file = m("gcp_credentials_file").value_or(""); + opts.gcp_impersonate_service_account = m("gcp_impersonate_service_account").value_or(""); + + opts.certfile = m("certfile").value_or(""); + opts.keyfile = m("keyfile").value_or(""); + opts.truststore = m("truststore").value_or(""); + opts.priority_string = m("priority_string").value_or(""); + + opts.key_cache_expiry = parse_expiry(m("key_cache_expiry")); + opts.key_cache_refresh = parse_expiry(m("key_cache_refresh")); + + return opts; + }()) +{} + +encryption::gcp_host::~gcp_host() = default; + +future<> encryption::gcp_host::init() { + return _impl->init(); +} + +const encryption::gcp_host::host_options& encryption::gcp_host::options() const { + return _impl->options(); +} + +future, encryption::gcp_host::id_type>> encryption::gcp_host::get_or_create_key(const key_info& info, const option_override* oov) { + return _impl->get_or_create_key(info, oov); +} + +future> encryption::gcp_host::get_key_by_id(const id_type& id, const key_info& info, const option_override* oov) { + return _impl->get_key_by_id(id, info, oov); +} + +template<> +struct fmt::formatter { + constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); } + auto format(const encryption::gcp_host::impl::attr_cache_key& d, fmt::format_context& ctxt) const { + return fmt::format_to(ctxt.out(), "{},{},{}", d.master_key, d.src.gcp_credentials_file, d.src.gcp_impersonate_service_account); + } +}; + +template<> +struct fmt::formatter { + constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); } + auto format(const encryption::gcp_host::impl::id_cache_key& d, fmt::format_context& ctxt) const { + return fmt::format_to(ctxt.out(), "{},{},{}", d.id, d.src.gcp_credentials_file, d.src.gcp_impersonate_service_account); + } +}; diff --git a/ent/encryption/gcp_host.hh b/ent/encryption/gcp_host.hh new file mode 100644 index 000000000000..8764c0d24a7d --- /dev/null +++ b/ent/encryption/gcp_host.hh @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2024 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include +#include +#include + +#include +#include + +#include "symmetric_key.hh" + +namespace encryption { + +class encryption_context; +struct key_info; + +class gcp_host { +public: + class impl; + + template + struct t_credentials_source { + // Path to credentials JSON file (exported from gcloud console) + T gcp_credentials_file; + // Optional service account (email address) to impersonate + T gcp_impersonate_service_account; + }; + + using credentials_source = t_credentials_source; + + struct host_options : public credentials_source { + std::string gcp_project_id; + std::string gcp_location; + + // GCP KMS Key to encrypt data keys with. Format: / + std::string master_key; + + // tls. if unspeced, use system for https + // GCP does not (afaik?) allow certificate auth + // but we keep the option available just in case. + std::string certfile; + std::string keyfile; + std::string truststore; + std::string priority_string; + + std::optional key_cache_expiry; + std::optional key_cache_refresh; + }; + + using id_type = bytes; + + gcp_host(encryption_context&, const std::string& name, const host_options&); + gcp_host(encryption_context&, const std::string& name, const std::unordered_map&); + ~gcp_host(); + + future<> init(); + const host_options& options() const; + + struct option_override : public t_credentials_source> { + std::optional master_key; + }; + + future, id_type>> get_or_create_key(const key_info&, const option_override* = nullptr); + future> get_key_by_id(const id_type&, const key_info&, const option_override* = nullptr); +private: + std::unique_ptr _impl; +}; + +} diff --git a/ent/encryption/gcp_key_provider.cc b/ent/encryption/gcp_key_provider.cc new file mode 100644 index 000000000000..1c00b969acd2 --- /dev/null +++ b/ent/encryption/gcp_key_provider.cc @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2024 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include + +#include "gcp_key_provider.hh" +#include "gcp_host.hh" + +namespace encryption { + +class gcp_key_provider : public key_provider { +public: + gcp_key_provider(::shared_ptr gcp_host, std::string name, gcp_host::option_override oov) + : _gcp_host(std::move(gcp_host)) + , _name(std::move(name)) + , _oov(std::move(oov)) + {} + future> key(const key_info& info, opt_bytes id) override { + if (id) { + return _gcp_host->get_key_by_id(*id, info, &_oov).then([id](key_ptr k) { + return make_ready_future>(std::tuple(k, id)); + }); + } + return _gcp_host->get_or_create_key(info, &_oov).then([](std::tuple k_id) { + return make_ready_future>(k_id); + }); + } + void print(std::ostream& os) const override { + os << _name; + } +private: + ::shared_ptr _gcp_host; + std::string _name; + gcp_host::option_override _oov; +}; + +shared_ptr gcp_key_provider_factory::get_provider(encryption_context& ctxt, const options& map) { + opt_wrapper opts(map); + auto gcp_host = opts("gcp_host"); + + + gcp_host::option_override oov { + .master_key = opts("master_key"), + }; + + oov.gcp_credentials_file = opts("gcp_credentials_file"); + oov.gcp_impersonate_service_account = opts("gcp_impersonate_service_account"); + + if (!gcp_host) { + throw std::invalid_argument("gcp_host must be provided"); + } + + auto host = ctxt.get_gcp_host(*gcp_host); + auto id = gcp_host.value() + + ":" + oov.master_key.value_or(host->options().master_key) + + ":" + oov.gcp_credentials_file.value_or(host->options().gcp_credentials_file) + + ":" + oov.gcp_impersonate_service_account.value_or(host->options().gcp_impersonate_service_account) + ; + + auto provider = ctxt.get_cached_provider(id); + + if (!provider) { + provider = ::make_shared(host, *gcp_host, std::move(oov)); + ctxt.cache_provider(id, provider); + } + + return provider; +} + +} diff --git a/ent/encryption/gcp_key_provider.hh b/ent/encryption/gcp_key_provider.hh new file mode 100644 index 000000000000..76ddb674f654 --- /dev/null +++ b/ent/encryption/gcp_key_provider.hh @@ -0,0 +1,25 @@ +/* + * Copyright (C) 2024 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "encryption.hh" +#include "system_key.hh" + +namespace encryption { + +class gcp_key_provider_factory : public key_provider_factory { +public: + shared_ptr get_provider(encryption_context&, const options&) override; +}; + +/** + * See comment for AWS KMS regarding system key support. + */ +} diff --git a/ent/encryption/kmip_host.cc b/ent/encryption/kmip_host.cc new file mode 100644 index 000000000000..1e2bf5dd0b00 --- /dev/null +++ b/ent/encryption/kmip_host.cc @@ -0,0 +1,1222 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#ifdef HAVE_KMIP + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include + +// workaround cryptsoft sdk issue: +#define strcasestr kmip_strcasestr +#include +#include +#undef strcasestr + +#include "kmip_host.hh" +#include "encryption.hh" +#include "encryption_exceptions.hh" +#include "symmetric_key.hh" +#include "utils/hash.hh" +#include "utils/loading_cache.hh" +#include "utils/UUID.hh" +#include "utils/UUID_gen.hh" +#include "marshal_exception.hh" +#include "db/config.hh" + +using namespace std::chrono_literals; + +static logger kmip_log("kmip"); +static constexpr uint16_t kmip_port = 5696u; +// default for command execution/failover retry. +static constexpr int default_num_cmd_retry = 5; +static constexpr int min_num_cmd_retry = 2; +static constexpr auto base_backoff_time = 100ms; + +std::ostream& operator<<(std::ostream& os, KMIP* kmip) { + auto* s = KMIP_dump_str(kmip, KMIP_DUMP_FORMAT_DEFAULT); + os << s; + free(s); + return os; +} + +static void kmip_logger(void *cb_arg, unsigned char *str, unsigned long len) { + // kmipc likes to write a log of white space and newlines. Skip these. + std::string_view v(reinterpret_cast(str), len); + if (std::find_if(v.begin(), v.end(), [](char c) { return !::isspace(c); }) == v.end()) { + return; + } + kmip_log.trace("kmipcmd: {}", v); +} + +namespace encryption { + +bool operator==(const kmip_host::key_options& l, const kmip_host::key_options& r) { + return std::tie(l.template_name, l.key_namespace) == std::tie(r.template_name, r.key_namespace); +} + +class kmip_error_category : public std::error_category { +public: + constexpr kmip_error_category() noexcept : std::error_category{} {} + const char * name() const noexcept { + return "KMIP"; + } + std::string message(int error) const { + return KMIP_error2string(error); + } +}; + +static const kmip_error_category kmip_errorc; + +class kmip_error : public std::system_error { +public: + kmip_error(int res) + : system_error(res, kmip_errorc) + {} + kmip_error(int res, const std::string& msg) + : system_error(res, kmip_errorc, msg) + {} +}; + +// Checks a gnutls return value. +// < 0 -> error. +static void kmip_chk(int res, KMIP_CMD * cmd = nullptr) { + if (res != KMIP_ERROR_NONE) { + int status=0, reason=0; + char* message = nullptr; + + if (KMIP_CMD_get_result(cmd, &status, &reason, &message) == KMIP_ERROR_NONE) { + auto* ctxt = cmd != nullptr ? KMIP_CMD_get_ctx(cmd) : "(unknown cmd)"; + auto s = fmt::format("{}: status={}, reason={}, message={}", + ctxt, + KMIP_RESULT_STATUS_to_string(status, 0, nullptr), + KMIP_RESULT_REASON_to_string(reason, 0, nullptr), + message ? message : "" + ); + throw kmip_error(res, s); + } + throw kmip_error(res); + } +} + + +class kmip_host::impl { +public: + struct kmip_key_info { + key_info info; + key_options options; + bool operator==(const kmip_key_info& i) const { + return info == i.info && options == i.options; + } + friend std::ostream& operator<<(std::ostream& os, const kmip_key_info& info) { + return os << info.info << ":" << info.options; + } + }; + struct kmip_key_info_hash { + size_t operator()(const kmip_key_info& i) const { + return utils::tuple_hash()( + std::tie(i.info.alg, i.info.len, + i.options.template_name, + i.options.key_namespace)); + } + }; + + using key_and_id_type = std::tuple, id_type>; + + inline static constexpr std::chrono::milliseconds default_expiry = 30s; + inline static constexpr std::chrono::milliseconds default_refresh = 100s; + inline static constexpr uintptr_t max_hosts = 1<<8; + + inline static constexpr size_t def_max_pooled_connections_per_host = 8; + + impl(encryption_context& ctxt, const sstring& name, const host_options& options) + : _ctxt(ctxt), _name(name), _options(options), _attr_cache( + utils::loading_cache_config{ + .max_size = std::numeric_limits::max(), + .expiry = options.key_cache_expiry.value_or( + default_expiry), + .refresh = options.key_cache_refresh.value_or(default_refresh)}, + kmip_log, + std::bind(&impl::create_key, this, + std::placeholders::_1)), + _id_cache( + utils::loading_cache_config{ + .max_size = std::numeric_limits::max(), + .expiry = options.key_cache_expiry.value_or( + default_expiry), + .refresh = options.key_cache_refresh.value_or(default_refresh), + }, + kmip_log, + std::bind(&impl::find_key, this, + std::placeholders::_1)), + _max_retry(std::max(size_t(min_num_cmd_retry), options.max_command_retries.value_or(default_num_cmd_retry))) + { + if (_options.hosts.size() > max_hosts) { + throw std::invalid_argument("Too many hosts"); + } + + KMIP_CMD_set_default_logfile(nullptr, nullptr); // disable logfile + KMIP_CMD_set_default_logger(kmip_logger, nullptr); // send logs to us instead + } + + future<> connect(); + future<> disconnect(); + future, id_type>> get_or_create_key(const key_info&, const key_options& = {}); + future> get_key_by_id(const id_type&, const std::optional& = {}); + + id_type kmip_id_to_id(const sstring&) const; + sstring id_to_kmip_string(const id_type&) const; +private: + future create_key(const kmip_key_info&); + future> find_key(const id_type&); + future> find_matching_keys(const kmip_key_info&, std::optional max = {}); + + static shared_ptr ensure_compatible_key(shared_ptr, const key_info&); + + template + class kmip_handle; + class kmip_cmd; + class kmip_data_list; + class connection; + + std::tuple make_attributes(const kmip_key_info&, bool include_template = true) const; + + union userdata { + void * ptr; + const char* host; + }; + + friend std::ostream& operator<<(std::ostream& os, const impl& me) { + fmt::print(os, "{}", me._name); + return os; + } + + using con_ptr = ::shared_ptr; + using opt_int = std::optional; + + template + future do_cmd(kmip_cmd, Func &&); + template + future do_cmd(KMIP_CMD*, con_ptr, Func&, bool retain_connection_after_command = false); + + future get_connection(KMIP_CMD*); + future get_connection(const sstring&); + future<> clear_connections(const sstring& host); + + void release(KMIP_CMD*, con_ptr, bool retain_connection = false); + + size_t max_pooled_connections_per_host() const { + return _options.max_pooled_connections_per_host.value_or(def_max_pooled_connections_per_host); + } + bool is_current_host(const sstring& host) { + return host == _options.hosts.at(_index % _options.hosts.size()); + } + + encryption_context& _ctxt; + sstring _name; + host_options _options; + utils::loading_cache, + kmip_key_info_hash> _attr_cache; + + utils::loading_cache, 2, + utils::loading_cache_reload_enabled::yes, + utils::simple_entry_size<::shared_ptr>> _id_cache; + + using connections = std::deque; + using host_to_connections = std::unordered_map; + + host_to_connections _host_connections; + // current default host. If a host fails, incremented and + // we try another in the host ip list. + size_t _index = 0; + size_t _max_retry = default_num_cmd_retry; +}; + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +namespace encryption { + +class kmip_host::impl::connection { +public: + connection(const sstring& host, host_options& options) + : _host(host) + , _options(options) + {} + ~connection() + {} + + const sstring& host() const { + return _host; + } + + void attach(KMIP_CMD*); + + future<> connect(); + future<> wait_for_io(); + future<> close(); +private: + static int io_callback(KMIP*, void*, int, void*, unsigned int, unsigned int*); + + int send(void*, unsigned int, unsigned int*); + int recv(void*, unsigned int, unsigned int*); + + friend std::ostream& operator<<(std::ostream& os, const connection& me) { + return os << me._host; + } + + sstring _host; + host_options& _options; + output_stream _output; + input_stream _input; + seastar::connected_socket _socket; + std::optional> _in_buffer; + std::optional> _pending; +}; + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +namespace encryption { + +future<> kmip_host::impl::connection::connect() { + auto cred = ::make_shared(); + auto f = make_ready_future(); + + kmip_log.debug("connecting {}", _host); + + if (!_options.priority_string.empty()) { + cred->set_priority_string(_options.priority_string); + } else { + cred->set_priority_string(db::config::default_tls_priority); + } + + if (!_options.certfile.empty()) { + f = f.then([this, cred] { + return cred->set_x509_key_file(_options.certfile, _options.keyfile, seastar::tls::x509_crt_format::PEM); + }); + } + if (!_options.truststore.empty()) { + f = f.then([this, cred] { + return cred->set_x509_trust_file(_options.truststore, seastar::tls::x509_crt_format::PEM); + }); + } + return f.then([this, cred] { + // TODO, find if we should do hostname verification + // TODO: connect all failovers already? + + auto i = _host.find_last_of(':'); + auto name = _host.substr(0, i); + auto port = i != sstring::npos ? std::stoul(_host.substr(i + 1)) : kmip_port; + + return seastar::net::dns::resolve_name(name).then([this, cred, port](seastar::net::inet_address addr) { + return seastar::tls::connect(cred, seastar::ipv4_addr{addr, uint16_t(port)}).then([this](seastar::connected_socket s) { + kmip_log.debug("Successfully connected {}", _host); + // #998 Set keepalive to try avoiding connection going stale inbetween commands. + s.set_keepalive_parameters(net::tcp_keepalive_params{60s, 60s, 10}); + s.set_keepalive(true); + _input = s.input(); + _output = s.output(); + }); + }); + }); +} + +future<> kmip_host::impl::connection::wait_for_io() { + kmip_log.trace("{}: Waiting...", *this); + auto o = std::exchange(_pending, std::nullopt); + return o ? std::move(*o) : make_ready_future(); +} + +int kmip_host::impl::connection::send(void* data, unsigned int len, unsigned int*) { + if (_pending) { + kmip_log.trace("{}: operation pending...", *this); + return KMIP_ERROR_RETRY; + } + kmip_log.trace("{}: Sending {} bytes", *this, len); + + auto f = _output.write(reinterpret_cast(data), len).then([this] { + kmip_log.trace("{}: send done. flushing...", *this); + return _output.flush(); + }); + // if the call failed already, we still want to + // drop back to "wait_for_io()", because we cannot throw + // exceptions through the kmipc code frames. + if (!f.available() || f.failed()) { + _pending.emplace(std::move(f)); + } + return KMIP_ERROR_NONE; +} + +int kmip_host::impl::connection::recv(void* data, unsigned int len, unsigned int* outlen) { + kmip_log.trace("{}: Waiting for data ({})", *this, len); + for (;;) { + if (_in_buffer) { + auto n = std::min(unsigned(_in_buffer->size()), len); + *outlen = n; + kmip_log.trace("{}: returning {} ({}) bytes", *this, n, _in_buffer->size()); + std::copy(_in_buffer->begin(), _in_buffer->begin() + n, reinterpret_cast(data)); + _in_buffer->trim_front(n); + if (_in_buffer->empty()) { + _in_buffer = std::nullopt; + } + // #998 cryptsoft example returns error on EOF. + if (n == 0) { + return KMIP_ERROR_IO; + } + break; + } + + if (_pending) { + kmip_log.trace("{}: operation pending...", *this); + return KMIP_ERROR_RETRY; + } + + kmip_log.trace("{}: issue read", *this); + auto f = _input.read().then([this](temporary_buffer buf) { + kmip_log.trace("{}: got {} bytes", *this, buf.size()); + _in_buffer = std::move(buf); + }); + + // if the call failed already, we still want to + // drop back to "wait_for_io()", because we cannot throw + // exceptions through the kmipc code frames. + if (!f.available() || f.failed()) { + _pending.emplace(std::move(f)); + } + } + return KMIP_ERROR_NONE; +} + +int kmip_host::impl::connection::io_callback(KMIP *kmip, void *cb_arg, int op, void *data, unsigned int len, unsigned int *outlen) { + auto* conn = reinterpret_cast(cb_arg); + try { + switch(op) { + default: + return KMIP_ERROR_NOT_SUPPORTED; + case KMIP_IO_CMD_SEND: + return conn->send(data, len, outlen); + case KMIP_IO_CMD_RECV: + return conn->recv(data, len, outlen); + } + } catch (...) { + kmip_log.warn("Error in KMIP IO: {}", std::current_exception()); + return KMIP_ERROR_IO; + } +} + +void kmip_host::impl::connection::attach(KMIP_CMD* cmd) { + kmip_log.trace("{} Attach: {}", *this, reinterpret_cast(cmd)); + if (cmd == nullptr) { + return; + } + + if (!_options.username.empty()) { + kmip_chk( + KMIP_CMD_set_credential_username(cmd, + const_cast(_options.username.c_str()), + const_cast(_options.password.c_str()))); + } + + /* because we haven't passed in anything to the KMIP_CMD layer + * that would provide it with the protocol version details we + * have to separately indicate that here + */ + kmip_chk(KMIP_CMD_set_lib_protocol(cmd, KMIP_LIB_PROTOCOL_KMIP1)); + /* handle all IO via the callback */ + kmip_chk( + KMIP_CMD_set_io_cb(cmd, &connection::io_callback, + reinterpret_cast(this))); +} + +future<> kmip_host::impl::connection::close() { + return _output.close().finally([this] { + return _input.close(); + }); +} + +template +class kmip_host::impl::kmip_handle { +public: + kmip_handle(T * ptr) + : _ptr(ptr, FreeFunc) + {} + kmip_handle(kmip_handle&&) = default; + kmip_handle& operator=(kmip_handle&&) = default; + + T* get() const { + return _ptr.get(); + } + operator T*() const { + return _ptr.get(); + } + explicit operator bool() const { + return _ptr != nullptr; + } +private: + using ptr_type = std::unique_ptr; + ptr_type _ptr; +}; + +class kmip_host::impl::kmip_cmd : public kmip_handle { +public: + kmip_cmd(int flags = KMIP_CMD_FLAGS_DEFAULT|KMIP_CMD_FLAGS_LOG|KMIP_CMD_FLAGS_LOG_XML) + : kmip_handle([flags] { + KMIP_CMD* cmd; + kmip_chk(KMIP_CMD_new_ex(flags, nullptr, &cmd)); + return cmd; + }()) + {} + kmip_cmd(kmip_cmd&&) = default; + kmip_cmd& operator=(kmip_cmd&&) = default; + + friend std::ostream& operator<<(std::ostream& os, const kmip_cmd& cmd) { + return os << KMIP_CMD_get_request(cmd); + } +}; + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +namespace encryption { + +class kmip_host::impl::kmip_data_list : public kmip_handle { +public: + kmip_data_list(int flags = KMIP_DATA_LIST_FLAGS_DEFAULT) + : kmip_handle([flags] { + KMIP_DATA_LIST* kdl; + kmip_chk(KMIP_DATA_LIST_new(flags, &kdl)); + return kdl; + }()) + {} + kmip_data_list(kmip_data_list&&) = default; + kmip_data_list& operator=(kmip_data_list&&) = default; +}; + +/** + * Clears and releases a connection cp. Release connection after. + * If retain_connection is true, the connection is only cleared of command data and + * can be reused by caller, otherwise it is either added to the connection pool + * or dropped. +*/ +void kmip_host::impl::release(KMIP_CMD* cmd, con_ptr cp, bool retain_connection) { + auto i = _host_connections.find(cp->host()); + userdata u; + u.host = i->first.c_str(); + if (cmd) { + KMIP_CMD_set_userdata(cmd, u.ptr); + } + if (!retain_connection && is_current_host(i->first) && max_pooled_connections_per_host() > i->second.size()) { + i->second.emplace_back(std::move(cp)); + } +} + +/** + * Run a function on a KMIP command using connection cp. Release connection after. + * If retain_connection_after_command is true, the connection is only cleared of command data and + * can be reused by caller. +*/ +template +future kmip_host::impl::do_cmd(KMIP_CMD* cmd, con_ptr cp, Func& f, bool retain_connection_after_command) { + cp->attach(cmd); + + return repeat_until_value([this, cmd, &f, cp, retain_connection_after_command] { + int res = f(cmd); + switch (res) { + case KMIP_ERROR_RETRY: + return cp->wait_for_io().then([] { + return opt_int(); + }).handle_exception([cp](auto ep) { + // get here if we had any wire exceptions below. + // make sure to force flush and stuff here as well. + return cp->close().then_wrapped([ep = std::move(ep)](auto f) mutable { + try { + f.get(); + } catch (...) { + } + return make_exception_future(std::move(ep)); + }); + }); + case 0: + release(cmd, cp, retain_connection_after_command); + return make_ready_future(res); + default: + // error. connection is dicarded. close it. + return cp->close().then_wrapped([cp, res](auto f) { + // ignore any exception thrown from the close. + // ensure we provide the kmip error instead. + try { + f.get(); + } catch (...) { + } + return make_ready_future(res); + }); + } + }).finally([cp] {}); +} + +template +future kmip_host::impl::do_cmd(kmip_cmd cmd_in, Func && f) { + kmip_log.trace("{}: begin do_cmd", *this, cmd_in); + KMIP_CMD* cmd = cmd_in; + + // #998 Need to do retry loop, because we can have either timed out connection, + // lost it (connected server went down) or some other network error. + return do_with(std::move(f), [this, cmd](Func& f) { + return repeat_until_value([this, cmd, &f, retry = _max_retry]() mutable { + --retry; + return get_connection(cmd).handle_exception([this, cmd, retry](std::exception_ptr ep) { + if (retry) { + // failed to connect. do more serious backing off. + // we only retry this once, since get_connection + // will either give back cached connections, + // or explicitly try all avail hosts. + // In the first case, we will do the lower retry + // loop if something is stale/borked, the latter is + // more or less dead. + auto sleeptime = base_backoff_time * (_max_retry - retry); + kmip_log.debug("{}: Connection failed. backoff {}", *this, std::chrono::duration_cast(sleeptime).count()); + return seastar::sleep(sleeptime).then([this, cmd] { + kmip_log.debug("{}: retrying...", *this); + return get_connection(cmd); + }); + } + return make_exception_future(std::move(ep)); + }).then([this, cmd, &f, retry](con_ptr cp) mutable { + auto host = cp->host(); + auto res = do_cmd(cmd, std::move(cp), f); + kmip_log.trace("{}: request {}", *this, fmt::ptr(KMIP_CMD_get_request(cmd))); + return res.then([this, retry, host = std::move(host)](int res) { + if (res == KMIP_ERROR_IO) { + kmip_log.debug("{}: request error {}", *this, kmip_errorc.message(res)); + if (retry) { + // do some backing off unless this is the first retry, which + // might be a stale connection. Clear out all caches for the + // current host first, then retry. + auto f = clear_connections(host); + if (retry != (_max_retry - 1)) { + f = f.then([this] { + auto sleeptime = base_backoff_time; + kmip_log.debug("{}: backoff {}ms", *this, std::chrono::duration_cast(sleeptime).count()); + return seastar::sleep(sleeptime); + }); + } + return f.then([this] { + kmip_log.debug("{}: retrying...", *this); + return opt_int{}; + }); + } + } + return make_ready_future(res); + }); + }); + }); + }).then([this, cmd = std::move(cmd_in)](int res) mutable { + kmip_chk(res, cmd); + kmip_log.trace("{}: result {}", *this, fmt::ptr(KMIP_CMD_get_response(cmd))); + return std::move(cmd); + }); +} + +future kmip_host::impl::get_connection(const sstring& host) { + // TODO: if a pooled connection is stale, the command run will fail, + // and the connection will be discarded. Would be good if we could detect this case + // and re-run command with a new connection. Maybe always verify connection, even if + // it is old? + auto& q = _host_connections[host]; + + if (!q.empty()) { + auto cp = q.front(); + q.pop_front(); + return make_ready_future<::shared_ptr>(cp); + } + + auto cp = ::make_shared(host, _options); + kmip_log.trace("{}: connecting to {}", *this, host); + return cp->connect().then([this, cp, host] { + kmip_log.trace("{}: verifying {}", *this, host); + kmip_cmd cmd; + static auto connection_query = [](KMIP_CMD* cmd) { + static const std::array query_options = { + KMIP_QUERY_FUNCTION_QUERY_OPERATIONS, + KMIP_QUERY_FUNCTION_QUERY_OBJECTS, + }; + return KMIP_CMD_query(cmd, const_cast(query_options.data()), unsigned(query_options.size())); + }; + // when/if this succeeds, it will push the connection onto the available stack + auto f = do_cmd(cmd, cp, connection_query, true /* keep cp */); + return f.then([this, host, cmd = std::move(cmd), cp](int res) { + kmip_chk(res, cmd); + kmip_log.trace("{}: connected {}", *this, host); + return cp; + }); + }); +} + + +future kmip_host::impl::get_connection(KMIP_CMD* cmd) { + userdata u{ KMIP_CMD_get_userdata(cmd) }; + if (u.host != nullptr) { + return get_connection(u.host).then([](con_ptr cp) { + return cp; + }); + } + + using con_ptr = ::shared_ptr; + using con_opt = std::optional; + + return repeat_until_value([this, i = size_t(0)]() mutable { + if (i++ == _options.hosts.size()) { + throw missing_resource_error("Could not connect to any server"); + } + auto& host = _options.hosts[_index % _options.hosts.size()]; + return get_connection(host).then([](con_ptr cp) { + return con_opt(std::move(cp)); + }).handle_exception([this, host](auto) { + ++_index; + // if we fail one host, clear out any + // caches for it just in case. + return clear_connections(host).then([] { + return con_opt(); + }); + }); + }); +} + +future<> kmip_host::impl::clear_connections(const sstring& host) { + auto q = std::exchange(_host_connections[host], {}); + return parallel_for_each(q.begin(), q.end(), [](con_ptr c) { + return c->close().handle_exception([c](auto ep) { + // ignore exceptions + }); + }); +} + +future<> kmip_host::impl::connect() { + return do_for_each(_options.hosts, [this](const sstring& host) { + return get_connection(host).then([this](auto cp) { + release(nullptr, cp); + }); + }); +} + +future<> kmip_host::impl::disconnect() { + return do_for_each(_options.hosts, [this](const sstring& host) { + return clear_connections(host); + }); +} + +static unsigned from_str(unsigned (*f)(char*, int, int*), const sstring& s, const sstring& what) { + int found = 0; + auto res = f(const_cast(s.c_str()), CODE2STR_FLAG_STR_CASE, &found); + if (!found) { + throw std::invalid_argument(format("Unsupported {}: {}", what, s)); + } + return res; +} + +std::tuple kmip_host::impl::make_attributes(const kmip_key_info& info, bool include_template) const { + kmip_data_list kdl_attrs; + + if (!info.options.template_name.empty()) { + kmip_chk(KMIP_DATA_LIST_add_attr_str_by_tag(kdl_attrs, + KMIP_TAG_TEMPLATE, + const_cast(info.options.template_name.c_str())) + ); + } + if (!info.options.key_namespace.empty()) { + kmip_chk(KMIP_DATA_LIST_add_attr_str(kdl_attrs, + const_cast("x-key-namespace"), + const_cast(info.options.key_namespace.c_str())) + ); + } + sstring type, mode, padd; + std::tie(type, mode, padd) = parse_key_spec_and_validate_defaults(info.info.alg); + + try { + auto crypt_alg = from_str(&KMIP_string_to_CRYPTOGRAPHIC_ALGORITHM, type, "cryptographic algorithm"); + return std::make_tuple(std::move(kdl_attrs), crypt_alg); + } catch (std::invalid_argument& e) { + std::throw_with_nested(std::invalid_argument("Invalid algorithm: " + info.info.alg)); + } +} + +kmip_host::id_type kmip_host::impl::kmip_id_to_id(const sstring& s) const { + try { + // #2205 - we previously made all ID:s into uuids (because the literal functions + // are called KMIP_CMD_get_uuid etc). This has issues with Keysecure which apparently + // does _not_ give back UUID format strings, but "other" things. + // Could just always store ascii as bytes instead, but that would now + // break existing installations, so we check for UUID, and if it does not + // match we encode it. + utils::UUID uuid(s); + return uuid.serialize(); + } catch (marshal_exception&) { + // very simple exncoding scheme: add a "len" byte at the end. + // iff byte size of id + 1 (len) equals 16 (length of UUID), + // add a padding byte. + size_t len = s.size() + 1; + if (len == 16) { + ++len; + } + bytes res(len, 0); + std::copy(s.begin(), s.end(), res.begin()); + res.back() = int8_t(len - s.size()); + return res; + } +} + +sstring kmip_host::impl::id_to_kmip_string(const id_type& id) const { + // see comment above for encoding scheme. + if (id.size() == 16) { + // if byte size is UUID it must be a UUID. No "old" id:s are + // not, and we never encode non-uuid as 16 bytes. + auto uuid = utils::UUID_gen::get_UUID(id); + return fmt::format("{}", uuid); + } + auto len = id.size() - id.back(); + return sstring(id.begin(), id.begin() + len); +} + +future kmip_host::impl::create_key(const kmip_key_info& info) { + if (this_shard_id() == 0) { + // #1039 First try looking for existing keys on server + return find_matching_keys(info, 1).then([this, info](std::vector ids) { + if (!ids.empty()) { + // got it + return get_key_by_id(ids.front(), info.info).then([id = ids.front()](shared_ptr k) { + return key_and_id_type(std::move(k), id); + }); + } + + kmip_log.debug("{}: Creating key {}", _name, info); + + auto kdl_attrs_crypt_alg = make_attributes(info); + auto&& kdl_attrs = std::get<0>(kdl_attrs_crypt_alg); + auto&& crypt_alg = std::get<1>(kdl_attrs_crypt_alg); + + // TODO: this is inefficient. We can probably put this in a single batch. + kmip_cmd cmd; + KMIP_CMD_set_ctx(cmd, const_cast("Create key")); + + return do_cmd(std::move(cmd), [info, kdl_attrs = std::move(kdl_attrs), crypt_alg](KMIP_CMD* cmd) { + return KMIP_CMD_create_smpl(cmd, KMIP_OBJECT_TYPE_SYMMETRIC_KEY, + crypt_alg, + KMIP_CRYPTOGRAPHIC_USAGE_ENCRYPT|KMIP_CRYPTOGRAPHIC_USAGE_DECRYPT, + int(info.info.len), + KMIP_DATA_LIST_attrs(kdl_attrs), KMIP_DATA_LIST_n_attrs(kdl_attrs) + ); + }).then([this, info](kmip_cmd cmd) { + /* now get the details (the value of the key) */ + char* new_id; + kmip_chk(KMIP_CMD_get_uuid(cmd, 0, &new_id), cmd); + sstring uuid(new_id); + + kmip_log.debug("{}: Created {}:{}", _name, info, uuid); + + KMIP_CMD_set_ctx(cmd, const_cast("activate")); + + return do_cmd(std::move(cmd), [new_id](KMIP_CMD* cmd) { + return KMIP_CMD_activate(cmd, new_id); + }).then([this, info, uuid](kmip_cmd cmd) { + auto id = kmip_id_to_id(uuid); + kmip_log.debug("{}: Activated {}", _name, uuid); + return get_key_by_id(id, info.info).then([id](auto k) { + return key_and_id_type(k, id); + }); + }); + }); + }); + } + + return smp::submit_to(0, [this, info] { + return _ctxt.get_kmip_host(_name)->get_or_create_key(info.info, info.options).then([](std::tuple, id_type> k_id) { + auto&& [k, id] = k_id; + return make_ready_future>(std::tuple(k->info(), k->key(), id)); + }); + }).then([](std::tuple info_b_id) { + auto&& [info, b, id] = info_b_id; + return make_ready_future(key_and_id_type(make_shared(info, b), id)); + }); +} + +future> kmip_host::impl::find_matching_keys(const kmip_key_info& info, std::optional max) { + kmip_log.debug("{}: Finding matching key {}", _name, info); + + auto [kdl_attrs, crypt_alg] = make_attributes(info, false); + + static const char kmip_tag_cryptographic_length[] = KMIP_TAG_CRYPTOGRAPHIC_LENGTH_STR; + static const char kmip_tag_cryptographic_usage_mask[] = KMIP_TAG_CRYPTOGRAPHIC_USAGE_MASK_STR; + + // #1079. Query mask apparently ignores things like cryptographic + // attribute set of options, instead we must specify the query + // as a list of attributes. + kmip_chk(KMIP_DATA_LIST_add_attr_enum_by_tag(kdl_attrs, + KMIP_TAG_OBJECT_TYPE, + KMIP_OBJECT_TYPE_SYMMETRIC_KEY) + ); + kmip_chk(KMIP_DATA_LIST_add_attr_enum_by_tag(kdl_attrs, + KMIP_TAG_CRYPTOGRAPHIC_ALGORITHM, + int(crypt_alg)) + ); + kmip_chk(KMIP_DATA_LIST_add_attr_int(kdl_attrs, + // our kmip sdk is broken/const-challenged + const_cast(kmip_tag_cryptographic_length), + int(info.info.len)) + ); + kmip_chk(KMIP_DATA_LIST_add_attr_enum_by_tag(kdl_attrs, + KMIP_TAG_STATE, + KMIP_STATE_ACTIVE) + ); + kmip_chk(KMIP_DATA_LIST_add_attr_int(kdl_attrs, + const_cast(kmip_tag_cryptographic_usage_mask), + KMIP_CRYPTOGRAPHIC_USAGE_ENCRYPT|KMIP_CRYPTOGRAPHIC_USAGE_DECRYPT) + ); + + kmip_cmd cmd; + KMIP_CMD_set_ctx(cmd, const_cast("Find matching key")); + + std::unique_ptr mp; + int* maxp = nullptr; + if (max) { + mp = std::make_unique(*max); + maxp = mp.get(); + } + + return do_cmd(std::move(cmd), [kdl_attrs = std::move(kdl_attrs), maxp](KMIP_CMD* cmd) { + return KMIP_CMD_locate(cmd, maxp, nullptr, KMIP_DATA_LIST_attrs(kdl_attrs), KMIP_DATA_LIST_n_attrs(kdl_attrs)); + }).then([this, info, mp = std::move(mp)](kmip_cmd cmd) { + std::vector result; + + for (int i = 0; ; ++i) { + char* new_id; + auto err = KMIP_CMD_get_uuid(cmd, i, &new_id); + if (err == KMIP_ERROR_NOT_FOUND) { + break; + } + kmip_chk(err, cmd); + result.emplace_back(kmip_id_to_id(new_id)); + } + + kmip_log.debug("{}: Found {} matching keys {}", _name, result.size(), info); + + return result; + }); +} + +future> kmip_host::impl::find_key(const id_type& id) { + if (this_shard_id() == 0) { + kmip_cmd cmd; + KMIP_CMD_set_ctx(cmd, const_cast("Find key")); + + auto uuid = id_to_kmip_string(id); + kmip_log.debug("{}: Finding {}", _name, uuid); + + // Batch operation. Nothing is sent/received until xmit below + kmip_chk(KMIP_CMD_batch_start(cmd)); + kmip_chk(KMIP_CMD_set_batch_order(cmd, 1)); + { + int key_format_type = KMIP_KEY_FORMAT_TYPE_RAW; + kmip_chk(KMIP_CMD_get(cmd, const_cast(uuid.c_str()), &key_format_type, nullptr, nullptr)); + } + kmip_chk(KMIP_CMD_get_attributes(cmd, const_cast(uuid.c_str()), nullptr, 0)); + + return do_cmd(std::move(cmd), [](KMIP_CMD* cmd) { + return KMIP_CMD_batch_xmit(cmd); + }).then([this, uuid](kmip_cmd cmd) { + auto nb = KMIP_CMD_get_batch_count(cmd); + if (nb != 2) { + throw malformed_response_error("Invalid batch count in response: " + std::to_string(nb)); + } + + sstring alg; + sstring mode; + sstring padd; + + // "Get" result + auto kdl_res = KMIP_CMD_get_batch(cmd, 0); + + /* get a reference to the key material (the actual key value) */ + unsigned char* key; + unsigned int keylen; + kmip_chk(KMIP_DATA_LIST_get_data(kdl_res, KMIP_TAG_KEY_MATERIAL, 0, &key, &keylen)); + + auto tag_to_string = [](auto f, auto val) { + int found; + auto p = f(val, CODE2STR_FLAG_STR_CASE, &found); + if (!found) { + throw malformed_response_error("Invalid tag: " + std::to_string(val)); + } + return sstring(p); + }; + + int crypto_alg; + kmip_chk(KMIP_DATA_LIST_get_32(kdl_res, KMIP_TAG_CRYPTOGRAPHIC_ALGORITHM, 0, &crypto_alg)); + alg = tag_to_string(&KMIP_CRYPTOGRAPHIC_ALGORITHM_to_string, crypto_alg); + + // "Attribute list" result + // This will apparently most of the time _not_ contain the info we want, + // depending on server, but we record as much as we can anyway. + // The actual resulting keys used will be based on external config. Only + // key data and verifying that it is compatible with said info is + // important for us. + auto kdl_attr = KMIP_CMD_get_batch(cmd, 1); + + unsigned int attr_count = 0; + kmip_chk(KMIP_DATA_LIST_get_count(kdl_attr, KMIP_TAG_ATTRIBUTE, &attr_count)); + + for (unsigned int i = 0; i < attr_count; i++) { + KMIP_DATA *attr = nullptr; + int n_attr = 0; + + kmip_chk(KMIP_DATA_LIST_get_struct(kdl_attr, KMIP_TAG_ATTRIBUTE, i, &attr, &n_attr, NULL)); + + + KMIP_DATA *attr_val = nullptr; + kmip_chk(KMIP_DATA_get(attr, n_attr,KMIP_TAG_ATTRIBUTE_VALUE, 0, &attr_val)); + + switch (attr_val->tag) { + case KMIP_TAG_BLOCK_CIPHER_MODE: + mode = tag_to_string(&KMIP_BLOCK_CIPHER_MODE_to_string, attr_val->data32); + break; + case KMIP_TAG_PADDING_METHOD: + padd = tag_to_string(&KMIP_PADDING_METHOD_to_string, attr_val->data32); + break; + default: + break; + } + } + + if (alg.empty()) { + throw configuration_error("Could not find algorithm"); + } + if (mode.empty() != padd.empty()) { + throw configuration_error("Invalid block mode/padding"); + } + + auto str = mode.empty() || padd.empty() ? alg : alg + "/" + mode + "/" + padd; + key_info derived_info{ str, keylen*8}; + + kmip_log.trace("{}: Found {}:{} {}", _name, uuid, derived_info.alg, derived_info.len); + + return make_shared(derived_info, bytes(key, key + keylen)); + }); + } + + return smp::submit_to(0, [this, id] { + return _ctxt.get_kmip_host(_name)->get_key_by_id(id).then([](shared_ptr k) { + return make_ready_future>(std::tuple(k->info(), k->key())); + }); + }).then([](std::tuple info_b) { + auto&& [info, b] = info_b; + return make_shared(info, b); + }); +} + +shared_ptr kmip_host::impl::ensure_compatible_key(shared_ptr k, const key_info& info) { + // keys we get back are typically void + // of block mode/padding info (because this is meaningless + // from the standpoint of the kmip server). + // Check and re-init the actual key used based + // on what the user wants so we adhere to block mode etc. + if (!info.compatible(k->info())) { + throw malformed_response_error(fmt::format("Incompatible key: {}", k->info())); + } + if (k->info() != info) { + k = ::make_shared(info, k->key()); + } + return k; +} + +[[noreturn]] +static void translate_kmip_error(const kmip_error& e) { + switch (e.code().value()) { + case KMIP_ERROR_BAD_CONNECT: case KMIP_ERROR_IO: + std::throw_with_nested(network_error(e.what())); + case KMIP_ERROR_BAD_PROTOCOL: + std::throw_with_nested(configuration_error(e.what())); + case KMIP_ERROR_NOT_FOUND: + std::throw_with_nested(missing_resource_error(e.what())); + case KMIP_ERROR_AUTH_FAILED: case KMIP_ERROR_CERT_AUTH_FAILED: + std::throw_with_nested(permission_error(e.what())); + default: + std::throw_with_nested(service_error(e.what())); + } +} + +future, kmip_host::id_type>> kmip_host::impl::get_or_create_key(const key_info& info, const key_options& opts) { + kmip_log.debug("{}: Lookup key {}:{}", _name, info, opts); + try { + auto linfo = info; + auto kinfo = co_await _attr_cache.get(kmip_key_info{info, opts}); + co_return std::tuple(ensure_compatible_key(std::get<0>(kinfo), linfo), std::get<1>(kinfo)); + } catch (kmip_error& e) { + translate_kmip_error(e); + } catch (base_error&) { + throw; + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(fmt::format("get_or_create_key: {}", e.what()))); + } catch (...) { + std::throw_with_nested(service_error(fmt::format("get_or_create_key: {}", std::current_exception()))); + } +} + +future> kmip_host::impl::get_key_by_id(const id_type& id, const std::optional& info) { + try { + auto linfo = info; // maintain on stack + auto k = co_await _id_cache.get(id); + if (linfo) { + k = ensure_compatible_key(k, *linfo); + } + co_return k; + } catch (kmip_error& e) { + translate_kmip_error(e); + } catch (base_error&) { + throw; + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(fmt::format("get_key_by_id: {}", e.what()))); + } catch (...) { + std::throw_with_nested(service_error(fmt::format("get_key_by_id: {}", std::current_exception()))); + } +} + +kmip_host::kmip_host(encryption_context& ctxt, const sstring& name, const std::unordered_map& map) + : kmip_host(ctxt, name, [&ctxt, &map] { + host_options opts; + map_wrapper> m(map); + + try { + static const std::regex wsc("\\s*,\\s*"); // comma+whitespace + + std::string hosts = m("hosts").value(); + + auto i = std::sregex_token_iterator(hosts.begin(), hosts.end(), wsc, -1); + auto e = std::sregex_token_iterator(); + + std::for_each(i, e, [&](const std::string & s) { + opts.hosts.emplace_back(s); + }); + } catch (std::bad_optional_access&) { + throw std::invalid_argument("No KMIP host names provided"); + } + + opts.certfile = m("certificate").value_or(""); + opts.keyfile = m("keyfile").value_or(""); + opts.truststore = m("truststore").value_or(""); + opts.priority_string = m("priority_string").value_or(""); + + opts.username = m("username").value_or(""); + opts.password = ctxt.maybe_decrypt_config_value(m("password").value_or("")); + + if (m("max_command_retries")) { + opts.max_command_retries = std::stoul(*m("max_command_retries")); + } + + opts.key_cache_expiry = parse_expiry(m("key_cache_expiry")); + opts.key_cache_refresh = parse_expiry(m("key_cache_refresh")); + + return opts; + }()) +{} + +kmip_host::kmip_host(encryption_context& ctxt, const sstring& name, const host_options& opts) + : _impl(std::make_unique(ctxt, name, opts)) +{} + +kmip_host::~kmip_host() = default; + +future<> kmip_host::connect() { + return _impl->connect(); +} + +future<> kmip_host::disconnect() { + return _impl->disconnect(); +} + +future, kmip_host::id_type>> kmip_host::get_or_create_key(const key_info& info, const key_options& opts) { + return _impl->get_or_create_key(info, opts); +} + +future> kmip_host::get_key_by_id(const id_type& id, std::optional info) { + return _impl->get_key_by_id(id, info); +} + +future> kmip_host::get_key_by_name(const sstring& name) { + return _impl->get_key_by_id(_impl->kmip_id_to_id(name)); +} + +std::ostream& operator<<(std::ostream& os, const kmip_host::key_options& opts) { + return os << opts.template_name << ":" << opts.key_namespace; +} + +} + +#else + +#include "kmip_host.hh" + +namespace encryption { + +class kmip_host::impl { +}; + +kmip_host::kmip_host(encryption_context& ctxt, const sstring& name, const std::unordered_map& map) { + throw std::runtime_error("KMIP support not enabled"); +} + +kmip_host::kmip_host(encryption_context& ctxt, const sstring& name, const host_options& opts) { + throw std::runtime_error("KMIP support not enabled"); +} + +kmip_host::~kmip_host() = default; + +future<> kmip_host::connect() { + throw std::runtime_error("KMIP support not enabled"); +} + +future<> kmip_host::disconnect() { + throw std::runtime_error("KMIP support not enabled"); +} + +future, kmip_host::id_type>> kmip_host::get_or_create_key(const key_info& info, const key_options& opts) { + throw std::runtime_error("KMIP support not enabled"); +} + +future> kmip_host::get_key_by_id(const id_type& id, std::optional info) { + throw std::runtime_error("KMIP support not enabled"); +} + +future> kmip_host::get_key_by_name(const sstring& name) { + throw std::runtime_error("KMIP support not enabled"); +} + +std::ostream& operator<<(std::ostream& os, const kmip_host::key_options& opts) { + return os << opts.template_name << ":" << opts.key_namespace; +} + +} + +#endif diff --git a/ent/encryption/kmip_host.hh b/ent/encryption/kmip_host.hh new file mode 100644 index 000000000000..05963e278490 --- /dev/null +++ b/ent/encryption/kmip_host.hh @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include + +#include "../../bytes.hh" + +#include "symmetric_key.hh" + +namespace encryption { + +class symmetric_key; +class encryption_context; +struct key_info; + +class kmip_host { +public: + struct host_options { + std::vector hosts; + + sstring username; + sstring password; + + sstring certfile; + sstring keyfile; + sstring truststore; + sstring priority_string; + + std::optional key_cache_expiry; + std::optional key_cache_refresh; + + std::optional max_pooled_connections_per_host; + std::optional max_command_retries; + }; + struct key_options { + sstring template_name; + sstring key_namespace; + }; + using id_type = bytes; + + kmip_host(encryption_context&, const sstring& name, const host_options&); + kmip_host(encryption_context&, const sstring& name, const std::unordered_map&); + ~kmip_host(); + + future<> connect(); + future<> disconnect(); + future, id_type>> get_or_create_key(const key_info&, const key_options& = {}); + future> get_key_by_id(const id_type&, std::optional = std::nullopt); + + /** for system key(s) */ + future> get_key_by_name(const sstring&); + +private: + class impl; + std::unique_ptr _impl; +}; + +std::ostream& operator<<(std::ostream&, const kmip_host::key_options&); + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/ent/encryption/kmip_key_provider.cc b/ent/encryption/kmip_key_provider.cc new file mode 100644 index 000000000000..6d98a217c090 --- /dev/null +++ b/ent/encryption/kmip_key_provider.cc @@ -0,0 +1,119 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include + +#include "utils/UUID.hh" +#include "utils/UUID_gen.hh" + +#include "kmip_key_provider.hh" +#include "kmip_host.hh" + +namespace encryption { + +class kmip_key_provider : public key_provider { +public: + kmip_key_provider(::shared_ptr kmip_host, kmip_host::key_options kopts, sstring name) + : _kmip_host(std::move(kmip_host)) + , _kopts(std::move(kopts)) + , _name(std::move(name)) + {} + future> key(const key_info& info, opt_bytes id) override { + if (id) { + return _kmip_host->get_key_by_id(*id, info).then([id](key_ptr k) { + return make_ready_future>(std::tuple(k, id)); + }); + } + return _kmip_host->get_or_create_key(info, _kopts).then([](std::tuple k_id) { + return make_ready_future>(k_id); + }); + } + void print(std::ostream& os) const override { + os << _name; + if (!_kopts.key_namespace.empty()) { + os << ", namespace=" << _kopts.key_namespace; + } + if (!_kopts.template_name.empty()) { + os << ", template=" << _kopts.template_name; + } + } + +private: + ::shared_ptr _kmip_host; + kmip_host::key_options _kopts; + sstring _name; +}; + + +shared_ptr kmip_key_provider_factory::get_provider(encryption_context& ctxt, const options& map) { + opt_wrapper opts(map); + auto host = opts(HOST_NAME); + if (!host) { + throw std::invalid_argument("kmip_host must be provided"); + } + kmip_host::key_options kopts = { + opts(TEMPLATE_NAME).value_or(""), + opts(KEY_NAMESPACE).value_or(""), + }; + + auto cache_key = *host + ":" + boost::lexical_cast(kopts); + auto provider = ctxt.get_cached_provider(cache_key); + + if (!provider) { + provider = ::make_shared(ctxt.get_kmip_host(*host), std::move(kopts), *host); + ctxt.cache_provider(cache_key, provider); + } + + return provider; +} + +static std::optional> parse_kmip_host_and_path(const sstring & s) { + static const std::regex kmip_ex("kmip://([^/]+)/([\\w/]+)"); + + std::match_results m; + if (std::regex_match(s.begin(), s.end(), m, kmip_ex)) { + return std::make_pair(sstring(m[1]), sstring(m[2])); + } + return std::nullopt; +} + +kmip_system_key::kmip_system_key(encryption_context& ctxt, const sstring& s) { + auto p = parse_kmip_host_and_path(s); + if (!p) { + throw std::invalid_argument("Not a kmip path: " + s); + } + + _host = ctxt.get_kmip_host(p->first); + _name = p->second; +} + +kmip_system_key::~kmip_system_key() = default; + +bool kmip_system_key::is_kmip_path(const sstring& s) { + return parse_kmip_host_and_path(s) != std::nullopt; +} + +future> kmip_system_key::get_key() { + if (_key) { + return make_ready_future>(_key); + } + return _host->get_key_by_name(_name).then([this](shared_ptr k) { + _key = k; + return k; + }); +} + +const sstring& kmip_system_key::name() const { + return _name; +} + + +} + diff --git a/ent/encryption/kmip_key_provider.hh b/ent/encryption/kmip_key_provider.hh new file mode 100644 index 000000000000..c383b7fb0f08 --- /dev/null +++ b/ent/encryption/kmip_key_provider.hh @@ -0,0 +1,40 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#pragma once + +#include "encryption.hh" +#include "system_key.hh" + +namespace encryption { + +class kmip_key_provider_factory : public key_provider_factory { +public: + shared_ptr get_provider(encryption_context&, const options&) override; +}; + +class kmip_host; + +class kmip_system_key : public system_key { + shared_ptr _key; + shared_ptr _host; + sstring _name; +public: + kmip_system_key(encryption_context&, const sstring&); + ~kmip_system_key(); + + static bool is_kmip_path(const sstring&); + + future> get_key() override; + const sstring& name() const override; + bool is_local() const override { + return false; + } +}; + +} diff --git a/ent/encryption/kms_host.cc b/ent/encryption/kms_host.cc new file mode 100644 index 000000000000..2827f54efa3a --- /dev/null +++ b/ent/encryption/kms_host.cc @@ -0,0 +1,1164 @@ +/* + * Copyright (C) 2022 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include "utils/to_string.hh" + +#include "kms_host.hh" +#include "encryption.hh" +#include "encryption_exceptions.hh" +#include "symmetric_key.hh" +#include "utils/hash.hh" +#include "utils/loading_cache.hh" +#include "utils/UUID.hh" +#include "utils/UUID_gen.hh" +#include "utils/rjson.hh" +#include "marshal_exception.hh" +#include "db/config.hh" + +template struct fmt::formatter> : fmt::ostream_formatter {}; +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +using namespace std::chrono_literals; +using namespace std::string_literals; + +logger kms_log("kms"); + +class kms_error : public std::exception { + std::string _type, _msg; +public: + kms_error(std::string_view type, std::string_view msg) + : _type(type) + , _msg(fmt::format("{}: {}", type, msg)) + {} + const std::string& type() const { + return _type; + } + const char* what() const noexcept override { + return _msg.c_str(); + } +}; + +namespace kms_errors { + [[maybe_unused]] static const char* AccessDeniedException = "AccessDeniedException"; + [[maybe_unused]] static const char* IncompleteSignature = "IncompleteSignature"; + [[maybe_unused]] static const char* InternalFailure = "InternalFailure"; + [[maybe_unused]] static const char* InvalidAction = "InvalidAction"; + [[maybe_unused]] static const char* InvalidClientTokenId = "InvalidClientTokenId"; + [[maybe_unused]] static const char* InvalidParameterCombination = "InvalidParameterCombination"; + [[maybe_unused]] static const char* InvalidParameterValue = "InvalidParameterValue"; + [[maybe_unused]] static const char* InvalidQueryParameter = "InvalidQueryParameter"; + [[maybe_unused]] static const char* MalformedQueryString = "MalformedQueryString"; + [[maybe_unused]] static const char* MissingAction = "MissingAction"; + [[maybe_unused]] static const char* MissingAuthenticationToken = "MissingAuthenticationToken"; + [[maybe_unused]] static const char* MissingParameter = "MissingParameter"; + [[maybe_unused]] static const char* NotAuthorized = "NotAuthorized"; + [[maybe_unused]] static const char* OptInRequired = "OptInRequired"; + [[maybe_unused]] static const char* RequestExpired = "RequestExpired"; + [[maybe_unused]] static const char* ServiceUnavailable = "ServiceUnavailable"; + [[maybe_unused]] static const char* ThrottlingException = "ThrottlingException"; + [[maybe_unused]] static const char* ValidationError = "ValidationError"; + [[maybe_unused]] static const char* DependencyTimeoutException = "DependencyTimeoutException"; + [[maybe_unused]] static const char* InvalidArnExceptio = "InvalidArnException"; + [[maybe_unused]] static const char* KMSInternalException = "KMSInternalException"; + [[maybe_unused]] static const char* NotFoundException = "NotFoundException"; + [[maybe_unused]] static const char* AlreadyExistsException = "AlreadyExistsException"; +} + +namespace beast = boost::beast; // from +// Note: switch http -> bhttp to deal with namespace ambiguity. +namespace bhttp = beast::http; // from +namespace shttp = seastar::http; + +static std::string to_lower(std::string_view s) { + std::string tmp(s.size(), 0); + std::transform(s.begin(), s.end(), tmp.begin(), ::tolower); + return tmp; +} + +static bool is_true(std::string_view s) { + auto tmp = to_lower(s); + return tmp == "true" || tmp == "1" || tmp == "yes" || tmp == "on"; +} + +class encryption::kms_host::impl { +public: + // set a rather long expiry. normal KMS policies are 365-day rotation of keys. + // we can do with 10 minutes. CMH. maybe even longer. + // (see comments below on what keys are here) + static inline constexpr std::chrono::milliseconds default_expiry = 600s; + static inline constexpr std::chrono::milliseconds default_refresh = 1200s; + + impl(encryption_context& ctxt, const std::string& name, const host_options& options) + : _ctxt(ctxt) + , _name(name) + , _options(options) + , _attr_cache(utils::loading_cache_config{ + .max_size = std::numeric_limits::max(), + .expiry = options.key_cache_expiry.value_or(default_expiry), + .refresh = options.key_cache_refresh.value_or(default_refresh)}, kms_log, std::bind(&impl::create_key, this, std::placeholders::_1)) + , _id_cache(utils::loading_cache_config{ + .max_size = std::numeric_limits::max(), + .expiry = options.key_cache_expiry.value_or(default_expiry), + .refresh = options.key_cache_refresh.value_or(default_refresh)}, kms_log, std::bind(&impl::find_key, this, std::placeholders::_1)) + { + // check if we have an explicit endpoint set. + if (!_options.endpoint.empty()) { + static std::regex simple_url(R"foo((https?):\/\/(?:([\w\.]+)|\[([\w:]+)\]):?(\d+)?\/?)foo"); + std::transform(_options.endpoint.begin(), _options.endpoint.end(), _options.endpoint.begin(), ::tolower); + std::smatch m; + if (!std::regex_match(_options.endpoint, m, simple_url)) { + throw std::invalid_argument(fmt::format("Could not parse URL: {}", _options.endpoint)); + } + _options.https = m[1].str() == "https"; + _options.host = m[2].length() > 0 ? m[2].str() : m[3].str(); + _options.port = m[4].length() > 0 ? std::stoi(m[4].str()) : 0; + } + if (_options.endpoint.empty() && _options.host.empty() && _options.aws_region.empty() && !_options.aws_use_ec2_region) { + throw std::invalid_argument("No AWS region or endpoint specified"); + } + if (_options.port == 0) { + _options.port = _options.https ? 443 : 80; + } + if (_options.aws_profile.empty()) { + auto profile = std::getenv("AWS_PROFILE"); + if (profile) { + _options.aws_profile = profile; + } else { + _options.aws_profile = "default"; + } + } + kms_log.trace("Added KMS node {}={}", name, _options.endpoint.empty() + ? (_options.host.empty() ? _options.aws_region : _options.host) + : _options.endpoint + ); + } + ~impl() = default; + + future<> init(); + const host_options& options() const { + return _options; + } + + future, id_type>> get_or_create_key(const key_info&, const option_override* = nullptr); + future> get_key_by_id(const id_type&, const key_info&, const option_override* = nullptr); +private: + class httpclient; + using key_and_id_type = std::tuple, id_type>; + + struct attr_cache_key { + std::string master_key; + std::string aws_assume_role_arn; + key_info info; + + bool operator==(const attr_cache_key& v) const = default; + friend std::ostream& operator<<(std::ostream& os, const attr_cache_key& k) { + fmt::print(os, "{}", std::tie(k.master_key, k.aws_assume_role_arn, k.info)); + return os; + } + }; + + struct attr_cache_key_hash { + size_t operator()(const attr_cache_key& k) const { + return utils::tuple_hash()(std::tie(k.master_key, k.aws_assume_role_arn, k.info.len)); + } + }; + + struct id_cache_key { + id_type id; + std::string aws_assume_role_arn; + bool operator==(const id_cache_key& v) const = default; + friend std::ostream& operator<<(std::ostream& os, const id_cache_key& k) { + fmt::print(os, "{{{}, {}}}", k.id, k.aws_assume_role_arn); + return os; + } + }; + + struct id_cache_key_hash { + size_t operator()(const id_cache_key& k) const { + return utils::tuple_hash()(std::tie(k.id, k.aws_assume_role_arn)); + } + }; + + struct aws_query; + using result_type = bhttp::response; + + future post(aws_query); + future post(std::string_view target, std::string_view aws_assume_role_arn, const rjson::value& query); + + future create_key(const attr_cache_key&); + future find_key(const id_cache_key&); + + encryption_context& _ctxt; + std::string _name; + host_options _options; + utils::loading_cache, attr_cache_key_hash> _attr_cache; + utils::loading_cache, id_cache_key_hash> _id_cache; + shared_ptr _creds; + std::unordered_map> _cache; + bool _initialized = false; +}; + +template <> struct fmt::formatter : fmt::ostream_formatter {}; +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +/** + * Not in seastar. Because nowhere near complete, thought through or + * capable of dealing with anything but tiny aws messages. + * + * TODO: formalize and move to seastar + */ +class encryption::kms_host::impl::httpclient { +public: + httpclient(std::string host, uint16_t port, shared_ptr = {}); + + httpclient& add_header(std::string_view key, std::string_view value); + void clear_headers(); + + using result_type = kms_host::impl::result_type; + using request_type = bhttp::request; + + future send(); + + using method_type = bhttp::verb; + + void method(method_type); + void content(std::string_view); + void target(std::string_view); + + request_type& request() { + return _req; + } + const request_type& request() const { + return _req; + } + const std::string& host() const { + return _host; + } + uint16_t port() const { + return _port; + } +private: + + std::string _host; + uint16_t _port; + shared_ptr _creds; + request_type _req; +}; + +encryption::kms_host::impl::httpclient::httpclient(std::string host, uint16_t port, shared_ptr creds) + : _host(std::move(host)) + , _port(port) + , _creds(std::move(creds)) +{} + +encryption::kms_host::impl::httpclient& encryption::kms_host::impl::httpclient::add_header(std::string_view key, std::string_view value) { + _req.set(beast::string_view(key.data(), key.size()), beast::string_view(value.data(), value.size())); + return *this; +} + +void encryption::kms_host::impl::httpclient::clear_headers() { + _req.clear(); +} + +future encryption::kms_host::impl::httpclient::send() { + auto addr = co_await net::dns::resolve_name(_host); + socket_address sa(addr, _port); + connected_socket s = co_await (_creds + ? tls::connect(_creds, sa) + : seastar::connect(sa) + ); + + s.set_keepalive(true); + s.set_nodelay(true); + + auto out = s.output(); + auto in = s.input(); + + bhttp::serializer ser(_req); + + beast::error_code ec; + std::exception_ptr ex; + + bhttp::parser p(result_type{}); + + try { + while (!ser.is_done()) { + future<> f = make_ready_future<>(); + ser.next(ec, [&](beast::error_code& ec, auto&& buffers) { + for (auto const buffer : beast::buffers_range (buffers)) { + f = f.then([&out, data = buffer.data(), size = buffer.size()] { + return out.write(static_cast(data), size); + }); + } + ser.consume(beast::buffer_bytes(buffers)); + }); + + co_await std::move(f); + + if (ec.failed()) { + break; + } + } + + co_await out.flush(); + + p.eager(true); + p.skip(false); + + if (!ec.failed()) { + while (!p.is_done()) { + auto buf = co_await in.read(); + if (buf.empty()) { + break; + } + // parse + boost::asio::const_buffer wrap(buf.get(), buf.size()); + p.put(wrap, ec); + if (ec.failed() && ec != bhttp::error::need_more) { + break; + } + ec.clear(); + } + } + } catch (...) { + ex = std::current_exception(); + } + + try { + co_await out.close(); + } catch (...) { + if (!ex) { + ex = std::current_exception(); + } + } + try { + co_await in.close(); + } catch (...) { + if (!ex) { + ex = std::current_exception(); + } + } + + if (ec.failed()) { + throw std::system_error(ec); + } + if (ex) { + std::rethrow_exception(ex); + } + + co_return p.release(); +} + +void encryption::kms_host::impl::httpclient::method(method_type m) { + _req.method(m); +} + +void encryption::kms_host::impl::httpclient::content(std::string_view body) { + _req.body().assign(body.begin(), body.end()); + _req.set(bhttp::field::content_length, std::to_string(_req.body().size())); +} + +void encryption::kms_host::impl::httpclient::target(std::string_view target) { + _req.target(std::string(target)); +} + +static std::string get_option(const encryption::kms_host::option_override* oov, std::optional encryption::kms_host::option_override::* f, const std::string& def) { + if (oov) { + return (oov->*f).value_or(def); + } + return {}; +}; + +[[noreturn]] +static void translate_kms_error(const kms_error& e) { + using namespace kms_errors; + using namespace encryption; + + if (e.type() == AccessDeniedException || e.type() == MissingAuthenticationToken || e.type() == NotAuthorized) { + std::throw_with_nested(permission_error(e.what())); + } + if (e.type() == OptInRequired || e.type() == InvalidClientTokenId || e.type() == InvalidAction) { + std::throw_with_nested(configuration_error(e.what())); + } + if (e.type() == NotFoundException || e.type() == DependencyTimeoutException) { + std::throw_with_nested(missing_resource_error(e.what())); + } + std::throw_with_nested(service_error(e.what())); +} + +future, encryption::kms_host::id_type>> encryption::kms_host::impl::get_or_create_key(const key_info& info, const option_override* oov) { + attr_cache_key key { + .master_key = get_option(oov, &option_override::master_key, _options.master_key), + .aws_assume_role_arn = get_option(oov, &option_override::aws_assume_role_arn, _options.aws_assume_role_arn), + .info = info, + }; + + if (key.master_key.empty() && _options.master_key.empty()) { + throw configuration_error("No master key set in kms host config or encryption attributes"); + } + try { + co_return co_await _attr_cache.get(key); + } catch (kms_error& e) { + translate_kms_error(e); + } catch (base_error&) { + throw; + } catch (std::system_error& e) { + std::throw_with_nested(network_error(e.what())); + } catch (...) { + std::throw_with_nested(service_error(fmt::format("get_key_by_id: {}", std::current_exception()))); + } +} + +future> encryption::kms_host::impl::get_key_by_id(const id_type& id, const key_info& info, const option_override* oov) { + // note: since KMS does not really have any actual "key" associtation of id -> key, + // we only cache/query raw bytes of some length. (See below). + // Thus keys returned are always new objects. But they are not huge... + id_cache_key key { + .id = id, + .aws_assume_role_arn = get_option(oov, &option_override::aws_assume_role_arn, _options.aws_assume_role_arn), + }; + try { + auto data = co_await _id_cache.get(key); + co_return make_shared(info, data); + } catch (kms_error& e) { + translate_kms_error(e); + } catch (base_error&) { + throw; + } catch (std::system_error& e) { + std::throw_with_nested(network_error(e.what())); + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(fmt::format("get_key_by_id: {}", e.what()))); + } catch (...) { + std::throw_with_nested(service_error(fmt::format("get_key_by_id: {}", std::current_exception()))); + } +} + +std::string make_aws_host(std::string_view aws_region, std::string_view service) { + static const char AWS_GLOBAL[] = "aws-global"; + static const char US_EAST_1[] = "us-east-1"; // US East (N. Virginia) + static const char CN_NORTH_1[] = "cn-north-1"; // China (Beijing) + static const char CN_NORTHWEST_1[] = "cn-northwest-1"; // China (Ningxia) + static const char US_ISO_EAST_1[] = "us-iso-east-1"; // US ISO East + static const char US_ISOB_EAST_1[] = "us-isob-east-1"; // US ISOB East (Ohio) + + // Fallback to us-east-1 if global endpoint does not exists. + auto region = aws_region == AWS_GLOBAL ? US_EAST_1 : aws_region; + + std::stringstream ss; + ss << service << "." << region; + + if (region == CN_NORTH_1 || region == CN_NORTHWEST_1) { + ss << ".amazonaws.com.cn"; + } else if (region == US_ISO_EAST_1) { + ss << ".c2s.ic.gov"; + } else if (region == US_ISOB_EAST_1) { + ss << ".sc2s.sgov.gov"; + } else { + ss << ".amazonaws.com"; + } + return ss.str(); +} + +struct encryption::kms_host::impl::aws_query { + std::string_view host; + + std::string_view service; + std::string_view target; + std::string_view content_type; + std::string_view content; + + std::string_view aws_access_key_id; + std::string_view aws_secret_access_key; + std::string_view security_token; + + uint16_t port; +}; + +future encryption::kms_host::impl::post(std::string_view target, std::string_view aws_assume_role_arn, const rjson::value& query) { + static auto get_response_error = [](const result_type& res) -> std::string { + switch (res.result()) { + case bhttp::status::unauthorized: case bhttp::status::forbidden: return "AccessDenied"; + case bhttp::status::not_found: return "ResourceNotFound"; + case bhttp::status::too_many_requests: return "SlowDown"; + case bhttp::status::internal_server_error: return "InternalError"; + case bhttp::status::service_unavailable: return "ServiceUnavailable"; + case bhttp::status::request_timeout: case bhttp::status::gateway_timeout: + case bhttp::status::network_connect_timeout_error: + return "RequestTimeout"; + default: + return format("{}", res.result()); + } + }; + + static auto query_ec2_meta = [](std::string_view target, std::string token = {}) -> future> { + static auto get_env_def = [](std::string_view var, std::string_view def) { + auto val = std::getenv(var.data()); + return val ? std::string_view(val) : def; + }; + + static const std::string ec2_meta_host(get_env_def("AWS_EC2_METADATA_ADDRESS", "169.254.169.254")); + static const int ec2_meta_port = std::stoi(get_env_def("AWS_EC2_METADATA_PORT", "80").data()); + + kms_log.debug("Query ec2 metadata"); + + httpclient client(ec2_meta_host, ec2_meta_port); + + static constexpr auto X_AWS_EC2_METADATA_TOKEN_TTL_SECONDS = "X-aws-ec2-metadata-token-ttl-seconds"; + static constexpr auto X_AWS_EC2_METADATA_TOKEN = "X-aws-ec2-metadata-token"; + static constexpr const char* HOST_HEADER = "host"; + + static auto logged_send = [](httpclient& client) -> future { + kms_log.trace("Request: {}", client.request()); + result_type res; + try { + res = co_await client.send(); + } catch (std::system_error& e) { + std::throw_with_nested(network_error(fmt::format("Error sending to host {}:{}: {}", client.host(), client.port(), e.what()))); + } catch (std::exception& e) { + std::throw_with_nested(service_error(fmt::format("Error sending to host {}:{}: {}", client.host(), client.port(), e.what()))); + } + kms_log.trace("Result: status={}, response={}", res.result_int(), res); + if (res.result() != bhttp::status::ok) { + throw kms_error(get_response_error(res), "EC2 metadata query"); + } + co_return res; + }; + + client.add_header(HOST_HEADER, ec2_meta_host); + + if (token.empty()) { + client.add_header(X_AWS_EC2_METADATA_TOKEN_TTL_SECONDS, "21600"); + client.method(httpclient::method_type::put); + client.target("/latest/api/token"); + + + auto res = co_await logged_send(client); + + if (res.result() != bhttp::status::ok) { + throw kms_error(get_response_error(res), "EC2 metadata token query"); + } + + token = res.body(); + client.clear_headers(); + } + + client.add_header(X_AWS_EC2_METADATA_TOKEN, token); + client.add_header(HOST_HEADER, ec2_meta_host); + client.method(httpclient::method_type::get); + client.target(target); + + auto res = co_await logged_send(client); + co_return std::make_tuple(std::move(res), token); + }; + + std::string gtoken; + + if (_options.aws_region.empty() && _options.host.empty()) { + assert(_options.aws_use_ec2_region); + httpclient::result_type res; + std::tie(res, gtoken) = co_await query_ec2_meta("/latest/meta-data/placement/region"); + _options.aws_region = res.body(); + } + + if (_options.host.empty()) { + // resolve region -> endpoint + assert(!_options.aws_region.empty()); + _options.host = make_aws_host(_options.aws_region, "kms"); + } + + auto should_resolve_options_credentials = [this] { + if (_options.aws_use_ec2_credentials) { + return false; + } + return _options.aws_access_key_id.empty() || _options.aws_secret_access_key.empty(); + }; + + // if we did not get full auth info in config, we can try to + // retrieve it from environment + if (should_resolve_options_credentials()) { + auto key_id = std::getenv("AWS_ACCESS_KEY_ID"); + auto key = std::getenv("AWS_SECRET_ACCESS_KEY"); + if (_options.aws_access_key_id.empty() && key_id) { + kms_log.debug("No aws id specified. Using environment AWS_ACCESS_KEY_ID"); + _options.aws_access_key_id = key_id; + } + if (_options.aws_secret_access_key.empty() && key) { + kms_log.debug("No aws secret specified. Using environment AWS_SECRET_ACCESS_KEY"); + _options.aws_secret_access_key = key; + } + } + + // if we did not get full auth info in config or env, we can try to + // retrieve it from ~/.aws/credentials + if (should_resolve_options_credentials()) { + auto home = std::getenv("HOME"); + if (home) { + auto credentials = std::string(home) + "/.aws/credentials"; + auto credentials_exists = co_await seastar::file_exists(credentials); + if (credentials_exists) { + kms_log.debug("No aws id/secret specified. Trying to read credentials from {}", credentials); + try { + auto buf = co_await read_text_file_fully(credentials); + std::string profile; + + static std::regex cred_line(R"foo(\s*\[(?:profile\s+)?(\w+)\]|([^\s]+)\s*=\s*([^\s]+)\s*\n)foo"); + std::cregex_iterator i(buf.get(), buf.get() + buf.size(), cred_line), e; + + std::string id, secret; + while (i != e) { + if ((*i)[1].length() > 0) { + profile = (*i)[1].str(); + kms_log.trace("Found profile {} ({})", profile, credentials); + } else if (profile == _options.aws_profile) { + std::string key((*i)[2].str()); + std::string val((*i)[3].str()); + if (key == "aws_access_key_id") { + id = val; + } else if (key == "aws_secret_access_key") { + secret = val; + } + } + ++i; + } + + if (!id.empty() && !_options.aws_access_key_id.empty() && id != _options.aws_access_key_id) { + throw configuration_error(fmt::format("Mismatched aws id: {} != {}", id, _options.aws_access_key_id)); + } + if (!id.empty() && _options.aws_access_key_id.empty()) { + _options.aws_access_key_id = id; + } + if (!secret.empty() && _options.aws_secret_access_key.empty()) { + _options.aws_secret_access_key = secret; + } + if (_options.aws_access_key_id.empty() || _options.aws_secret_access_key.empty()) { + throw configuration_error(fmt::format("Could not find credentials for profile {}", _options.aws_profile)); + } + kms_log.debug("Read credentials from {} ({}:{}{})", credentials, _options.aws_access_key_id + , _options.aws_secret_access_key.substr(0, 2) + , std::string(_options.aws_secret_access_key.size()-2, '-') + ); + } catch (...) { + kms_log.debug("Could not read credentials: {}", std::current_exception()); + } + } + } + } + + auto aws_access_key_id = _options.aws_access_key_id; + auto aws_secret_access_key = _options.aws_secret_access_key; + auto session = ""s; + + if (_options.aws_use_ec2_credentials) { + auto [res, token] = co_await query_ec2_meta("/latest/meta-data/iam/security-credentials/", gtoken); + auto role = res.body(); + + std::tie(res, std::ignore) = co_await query_ec2_meta("/latest/meta-data/iam/security-credentials/" + role, token); + auto body = rjson::parse(std::string_view(res.body().data(), res.body().size())); + + try { + aws_access_key_id = rjson::get(body, "AccessKeyId"); + aws_secret_access_key = rjson::get(body, "SecretAccessKey"); + session = rjson::get(body, "Token"); + } catch (rjson::malformed_value&) { + std::throw_with_nested(kms_error("AccessDenied", fmt::format("Code={}, Message={}" + , rjson::get_opt(body, "Code") + , rjson::get_opt(body, "Message") + ))); + } + } + + // Note: allowing user code to potentially reset aws_assume_role_arn='' -> no assumerole. + // Not 100% sure this is needed. + + if (!aws_assume_role_arn.empty()) { + auto sts_host = make_aws_host(_options.aws_region, "sts"); + auto now = db_clock::now(); + auto rs_id = utils::UUID_gen::get_time_UUID(std::chrono::system_clock::time_point(now.time_since_epoch())); + auto role_session = fmt::format("ScyllaDB-{}", rs_id); + + kms_log.debug("Assume role: {} (RoleSessionID={})", aws_assume_role_arn, role_session); + + auto res = co_await post(aws_query{ + .host = sts_host, + .service = "sts", + .content_type = "application/x-www-form-urlencoded; charset=utf-8", + .content = "Action=AssumeRole&Version=2011-06-15&RoleArn=" + + shttp::internal::url_encode(aws_assume_role_arn) + + "&RoleSessionName=" + role_session, + .aws_access_key_id = aws_access_key_id, + .aws_secret_access_key = aws_secret_access_key, + .security_token = session, + .port = _options.port, + }); + + if (res.result() != bhttp::status::ok) { + throw kms_error(get_response_error(res), "AssumeRole"); + } + + rapidxml::xml_document<> doc; + try { + doc.parse<0>(res.body().data()); + + using node_type = rapidxml::xml_node; + static auto get_xml_node = [](node_type* node, const char* what) { + auto res = node->first_node(what); + if (!res) { + throw kms_error("XML parse error", what); + } + return res; + }; + + auto arrsp = get_xml_node(&doc, "AssumeRoleResponse"); + auto arres = get_xml_node(arrsp, "AssumeRoleResult"); + auto creds = get_xml_node(arres, "Credentials"); + auto keyid = get_xml_node(creds, "AccessKeyId"); + auto key = get_xml_node(creds, "SecretAccessKey"); + auto token = get_xml_node(creds, "SessionToken"); + + aws_access_key_id = keyid->value(); + aws_secret_access_key = key->value(); + session = token->value(); + + } catch (const rapidxml::parse_error& e) { + std::throw_with_nested(kms_error("XML parse error", "AssumeRole")); + } + } + + auto res = co_await post(aws_query{ + .host = _options.host, + .service = "kms", + .target = target, + .content_type = "application/x-amz-json-1.1", + .content = rjson::print(query), + .aws_access_key_id = aws_access_key_id, + .aws_secret_access_key = aws_secret_access_key, + .security_token = session, + .port = _options.port, + }); + + auto body = rjson::empty_object(); + + if (!res.body().empty()) { + try { + body = rjson::parse(std::string_view(res.body().data(), res.body().size())); + } catch (...) { + if (res.result() == bhttp::status::ok) { + throw; + } + // assume non-json formatted error. fall back to parsing below + } + } + + if (res.result() != bhttp::status::ok) { + // try to format as good an error as we can. + static const char* message_lc_header = "message"; + static const char* message_cc_header = "Message"; + static const char* error_type_header = "x-amzn-ErrorType"; + static const char* type_header = "__type"; + + auto o = rjson::get_opt(body, message_lc_header); + if (!o) { + o = rjson::get_opt(body, message_cc_header); + } + auto msg = o.value_or("Unknown error"); + + o = rjson::get_opt(body, error_type_header); + if (!o) { + o = rjson::get_opt(body, type_header); + } + // this should never happen with aws, but... + auto type = o ? *o : get_response_error(res); + + throw kms_error(type, msg); + } + + co_return body; +} + +// helper to build AWS request and parse result. +future encryption::kms_host::impl::post(aws_query query) { + auto creds = _creds; + // if we are https, we need at least a credentials object that says "use system trust" + if (!creds && _options.https) { + creds = ::make_shared(); + + if (!_options.priority_string.empty()) { + creds->set_priority_string(_options.priority_string); + } else { + creds->set_priority_string(db::config::default_tls_priority); + } + + if (!_options.certfile.empty()) { + co_await creds->set_x509_key_file(_options.certfile, _options.keyfile, seastar::tls::x509_crt_format::PEM); + } + if (!_options.truststore.empty()) { + co_await creds->set_x509_trust_file(_options.truststore, seastar::tls::x509_crt_format::PEM); + } else { + co_await creds->set_system_trust(); + } + _creds = creds; + } + + // some of this could be shared with alternator + static constexpr const char* CONTENT_TYPE_HEADER = "content-type"; + static constexpr const char* HOST_HEADER = "host"; + static constexpr const char* X_AWS_DATE_HEADER = "X-Amz-Date"; + static constexpr const char* AWS_AUTHORIZATION_HEADER = "authorization"; + static constexpr const char* AMZ_SDK_INVOCATION_ID = "amz-sdk-invocation-id"; + static constexpr const char* X_AMZ_SECURITY_TOKEN = "X-Amz-Security-Token"; + + static constexpr const char* AMZ_TARGET_HEADER = "x-amz-target"; + static constexpr const char* AWS_HMAC_SHA256 = "AWS4-HMAC-SHA256"; + static constexpr const char* AWS4_REQUEST = "aws4_request"; + static constexpr const char* SIGNING_KEY = "AWS4"; + static constexpr const char* CREDENTIAL = "Credential"; + static constexpr const char* SIGNATURE = "Signature"; + static constexpr const char* SIGNED_HEADERS = "SignedHeaders"; + [[maybe_unused]] static constexpr const char* ACTION_HEADER = "Action"; + + static constexpr const char* ISO_8601_BASIC = "{:%Y%m%dT%H%M%SZ}"; + static constexpr const char* SIMPLE_DATE_FORMAT_STR = "{:%Y%m%d}"; + static constexpr auto NEWLINE = '\n'; + + auto now = db_clock::now(); + auto req_id = utils::UUID_gen::get_time_UUID(std::chrono::system_clock::time_point(now.time_since_epoch())); + + kms_log.trace("Building request: {} ({}:{}) {}", query.target, query.host, query.port, req_id); + + httpclient client(std::string(query.host), query.port, std::move(creds)); + + auto t_now = fmt::gmtime(db_clock::to_time_t(now)); + auto timestamp = fmt::format(ISO_8601_BASIC, t_now); + + // see https://docs.aws.amazon.com/general/latest/gr/sigv4-create-canonical-request.html + // see AWS SDK. + // see https://docs.aws.amazon.com/general/latest/gr/sigv4-signed-request-examples.html + std::stringstream signedHeadersStream; + std::stringstream canonicalRequestStream; + + canonicalRequestStream + << "POST" << NEWLINE + << "/" << NEWLINE << NEWLINE + ; + + auto add_signed_header = [&](std::string_view name, std::string_view value) { + client.add_header(name, value); + auto lname = to_lower(name); + canonicalRequestStream << lname << ":" << value << NEWLINE; + if (signedHeadersStream.tellp() != 0) { + signedHeadersStream << ';'; + } + signedHeadersStream << lname; + }; + + // headers must be sorted! + + add_signed_header(CONTENT_TYPE_HEADER, query.content_type); + add_signed_header(HOST_HEADER, query.host); + add_signed_header(X_AWS_DATE_HEADER, timestamp); + if (!query.target.empty()) { + add_signed_header(AMZ_TARGET_HEADER, "TrentService."s + std::string(query.target)); + } + + if (!query.security_token.empty()) { + //add_signed_header(X_AMZ_SECURITY_TOKEN, query.security_token); + client.add_header(X_AMZ_SECURITY_TOKEN, query.security_token); + } + + client.add_header(AMZ_SDK_INVOCATION_ID, fmt::format("{}", req_id)); + client.add_header("Accept-Encoding", "identity"); + client.add_header("Accept", "*/*"); + + auto make_hash = [&](std::string_view s) { + auto sha256 = calculate_sha256(bytes_view(reinterpret_cast(s.data()), s.size())); + auto hash = to_hex(sha256); + return hash; + }; + + auto hash = make_hash(query.content); + + auto signedHeadersValue = signedHeadersStream.str(); + canonicalRequestStream << NEWLINE << signedHeadersValue << NEWLINE << hash; + auto canonicalRequestString = canonicalRequestStream.str(); + auto canonicalRequestHash = make_hash(canonicalRequestString); + + kms_log.trace("Canonical request: {}", canonicalRequestString); + + auto simpleDate = fmt::format(SIMPLE_DATE_FORMAT_STR, t_now); + + std::stringstream stringToSignStream; + stringToSignStream << AWS_HMAC_SHA256 << NEWLINE + << timestamp << NEWLINE + << simpleDate << "/" << _options.aws_region << "/" + << query.service << "/" << AWS4_REQUEST << NEWLINE + << canonicalRequestHash + ; + auto stringToSign = stringToSignStream.str(); + + // these log messages intentionally made to mimic aws sdk/boto3 + kms_log.trace("StringToSign: {}", stringToSign); + + std::string finalSignature; + + { + auto tobv = [](std::string_view s) { + return bytes_view(reinterpret_cast(s.data()), s.size()); + }; + + auto signingKey = SIGNING_KEY + std::string(query.aws_secret_access_key); + auto kDate = hmac_sha256(tobv(simpleDate), tobv(signingKey)); + auto kRegion = hmac_sha256(tobv(_options.aws_region), kDate); + auto kService = hmac_sha256(tobv(query.service), kRegion); + auto hashResult = hmac_sha256(tobv(AWS4_REQUEST), kService); + auto finalHash = hmac_sha256(tobv(stringToSign), hashResult); + finalSignature = to_hex(finalHash); + } + + std::stringstream authStream; + authStream << AWS_HMAC_SHA256 << " " + << CREDENTIAL << "=" << query.aws_access_key_id << "/" << simpleDate << "/" << _options.aws_region + << "/" << query.service << "/" << AWS4_REQUEST << ", " << SIGNED_HEADERS + << "=" << signedHeadersValue << ", " << SIGNATURE << "=" << finalSignature + ; + + auto awsAuthString = authStream.str(); + + client.add_header(AWS_AUTHORIZATION_HEADER, awsAuthString); + client.target("/"); + client.content(query.content); + client.method(httpclient::method_type::post); + + kms_log.trace("Request: {}", client.request()); + + auto res = co_await client.send(); + + kms_log.trace("Result: status={}, response={}", res.result_int(), res); + + co_return res; +} + +static std::optional make_opt(const std::string& s) { + if (s.empty()) { + return std::nullopt; + } + return s; +} + +future<> encryption::kms_host::impl::init() { + if (_initialized) { + co_return; + } + + if (!_options.master_key.empty()) { + kms_log.debug("Looking up master key"); + auto query = rjson::empty_object(); + rjson::add(query, "KeyId", _options.master_key); + auto response = co_await post("DescribeKey", _options.aws_assume_role_arn, query); + kms_log.debug("Master key exists"); + } else { + kms_log.info("No default master key configured. Not verifying."); + } + _initialized = true; +} + +future encryption::kms_host::impl::create_key(const attr_cache_key& k) { + auto& master_key = k.master_key; + auto& aws_assume_role_arn = k.aws_assume_role_arn; + auto& info = k.info; + + /** + * AWS KMS does _not_ allow us to actually have "named keys" that can be used externally, + * i.e. exported to us, here, for bulk encryption. + * All named keys are 100% internal, the only options we have is using the + * "GenerateDataKey" API. This creates a new (epiphermal) key, encrypts it + * using a named (internal) key, and gives us both raw and encrypted blobs + * for usage as a local key. + * To be able to actually re-use this key again, on decryption of data, + * we employ the strategy recommended (https://docs.aws.amazon.com/kms/latest/APIReference/API_GenerateDataKey.html) + * namely actually embedding the encrypted key in the key ID associated with + * the locally encrypted data. So ID:s become pretty big. + * + * For ID -> key, we simply split the ID into the encrypted key part, and + * the master key name part, decrypt the first using the second (AWS KMS Decrypt), + * and create a local key using the result. + * + * Data recovery: + * Assuming you have data encrypted using a KMS generated key, you will have + * metadata detailing algorithm, key length etc (see sstable metadata, and key info). + * Metadata will also include a byte blob representing the ID of the encryption key. + * For KMS, the ID will actually be a text string: + * : + * + * I.e. something like: + * 761f258a-e2e9-40b3-8891-602b1b8b947e:e56sadfafa3324ff=/wfsdfwssdf + * or + * arn:aws:kms:us-east-1:797456418907:key/761f258a-e2e9-40b3-8891-602b1b8b947e:e56sadfafa3324ff=/wfsdfwssdf + * + * (last colon is separator) + * + * The actual data key can be retreived by doing a KMS "Decrypt" of the data blob part + * using the KMS key referenced by the key ID. This gives back actual key data that can + * be used to create a symmetric_key with algo, length etc as specified by metadata. + * + */ + + // avoid creating too many keys and too many calls. If we are not shard 0, delegate there. + if (this_shard_id() != 0) { + auto [data, id] = co_await smp::submit_to(0, [this, info, master_key, aws_assume_role_arn]() -> future> { + auto host = _ctxt.get_kms_host(_name); + option_override oov { + .master_key = make_opt(master_key), + .aws_assume_role_arn = make_opt(aws_assume_role_arn), + }; + auto [k, id] = co_await host->_impl->get_or_create_key(info, &oov); + co_return std::make_tuple(k != nullptr ? k->key() : bytes{}, id); + }); + co_return key_and_id_type{ + data.empty() ? nullptr : make_shared(info, data), + id + }; + } + + // note: since external keys are _not_ stored, + // there is nothing we can "look up" or anything. Always + // new key here. + + kms_log.debug("Creating new key: {}", info); + + auto query = rjson::empty_object(); + + rjson::add(query, "KeyId", std::string(master_key.begin(), master_key.end())); + rjson::add(query, "NumberOfBytes", info.len/8); + + auto response = co_await post("GenerateDataKey", aws_assume_role_arn, query); + auto data = base64_decode(rjson::get(response, "Plaintext")); + auto enc = rjson::get(response, "CiphertextBlob"); + auto kid = rjson::get(response, "KeyId"); + + try { + auto key = make_shared(info, data); + bytes id(kid.size() + 1 + enc.size(), 0); + auto i = std::copy(kid.begin(), kid.end(), id.begin()); + *i++ = ':'; + std::copy(enc.begin(), enc.end(), i); + + co_return key_and_id_type{ key, id }; + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(e.what())); + } +} + +future encryption::kms_host::impl::find_key(const id_cache_key& k) { + // avoid creating too many keys and too many calls. If we are not shard 0, delegate there. + if (this_shard_id() != 0) { + co_return co_await smp::submit_to(0, [this, k]() -> future { + auto host = _ctxt.get_kms_host(_name); + auto bytes = co_await host->_impl->_id_cache.get(k); + co_return bytes; + }); + } + + // See create_key. ID consists of :. + // master id can (and will) contain ':', but blob will not. + // (we are being wasteful, and keeping the base64 encoding - easier to read) + auto& id = k.id; + auto pos = id.find_last_of(':'); + if (pos == id_type::npos) { + throw std::invalid_argument(format("Not a valid key id: {}", id)); + } + + kms_log.debug("Finding key: {}", id); + + std::string kid(id.begin(), id.begin() + pos); + std::string enc(id.begin() + pos + 1, id.end()); + + auto query = rjson::empty_object(); + rjson::add(query, "CiphertextBlob", enc); + rjson::add(query, "KeyId", kid); + + auto response = co_await post("Decrypt", k.aws_assume_role_arn, query); + auto data = base64_decode(rjson::get(response, "Plaintext")); + + // we know nothing about key type etc, so just return data. + co_return data; +} + +encryption::kms_host::kms_host(encryption_context& ctxt, const std::string& name, const host_options& options) + : _impl(std::make_unique(ctxt, name, options)) +{} + +encryption::kms_host::kms_host(encryption_context& ctxt, const std::string& name, const std::unordered_map& map) + : kms_host(ctxt, name, [&map] { + host_options opts; + map_wrapper> m(map); + + opts.aws_access_key_id = m("aws_access_key_id").value_or(""); + opts.aws_secret_access_key = m("aws_secret_access_key").value_or(""); + opts.aws_region = m("aws_region").value_or(""); + opts.aws_profile = m("aws_profile").value_or(""); + opts.aws_assume_role_arn = m("aws_assume_role_arn").value_or(""); + opts.aws_use_ec2_credentials = is_true(m("aws_use_ec2_credentials").value_or("false")); + opts.aws_use_ec2_region = is_true(m("aws_use_ec2_region").value_or("false")); + + // use "endpoint" semantics to match AWS configs. + opts.endpoint = m("endpoint").value_or(""); + opts.host = m("host").value_or(""); + opts.port = std::stoi(m("port").value_or("0")); + + opts.master_key = m("master_key").value_or(""); + opts.certfile = m("certfile").value_or(""); + opts.keyfile = m("keyfile").value_or(""); + opts.truststore = m("truststore").value_or(""); + opts.priority_string = m("priority_string").value_or(""); + + opts.key_cache_expiry = parse_expiry(m("key_cache_expiry")); + opts.key_cache_refresh = parse_expiry(m("key_cache_refresh")); + return opts; + }()) +{} + +encryption::kms_host::~kms_host() = default; + +future<> encryption::kms_host::init() { + return _impl->init(); +} + +const encryption::kms_host::host_options& encryption::kms_host::options() const { + return _impl->options(); +} + +future, encryption::kms_host::id_type>> encryption::kms_host::get_or_create_key(const key_info& info, const option_override* oov) { + return _impl->get_or_create_key(info, oov); +} + +future> encryption::kms_host::get_key_by_id(const id_type& id, const key_info& info, const option_override* oov) { + return _impl->get_key_by_id(id, info, oov); +} + diff --git a/ent/encryption/kms_host.hh b/ent/encryption/kms_host.hh new file mode 100644 index 000000000000..ac9fd1de4182 --- /dev/null +++ b/ent/encryption/kms_host.hh @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2022 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include +#include +#include + +#include +#include + +#include "symmetric_key.hh" + +namespace encryption { + +class encryption_context; +struct key_info; + +class kms_host { +public: + struct host_options { + std::string endpoint; + // or... + std::string host; + uint16_t port; + bool https = true; + // auth + std::string aws_access_key_id; + std::string aws_secret_access_key; + std::string aws_region; + std::string aws_profile; + std::string aws_assume_role_arn; + + bool aws_use_ec2_credentials; + bool aws_use_ec2_region; + + // key to use for keys + std::string master_key; + // tls. if unspeced, use system for https + // AWS does not (afaik?) allow certificate auth + // but we keep the option available just in case. + std::string certfile; + std::string keyfile; + std::string truststore; + std::string priority_string; + + std::optional key_cache_expiry; + std::optional key_cache_refresh; + }; + using id_type = bytes; + + kms_host(encryption_context&, const std::string& name, const host_options&); + kms_host(encryption_context&, const std::string& name, const std::unordered_map&); + ~kms_host(); + + future<> init(); + const host_options& options() const; + + struct option_override { + std::optional master_key; + std::optional aws_assume_role_arn; + }; + + future, id_type>> get_or_create_key(const key_info&, const option_override* = nullptr); + future> get_key_by_id(const id_type&, const key_info&, const option_override* = nullptr); +private: + class impl; + std::unique_ptr _impl; +}; + +} diff --git a/ent/encryption/kms_key_provider.cc b/ent/encryption/kms_key_provider.cc new file mode 100644 index 000000000000..3140de151390 --- /dev/null +++ b/ent/encryption/kms_key_provider.cc @@ -0,0 +1,71 @@ +/* + * Copyright (C) 2022 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include + +#include "kms_key_provider.hh" +#include "kms_host.hh" + +namespace encryption { + +class kms_key_provider : public key_provider { +public: + kms_key_provider(::shared_ptr kms_host, std::string name, kms_host::option_override oov) + : _kms_host(std::move(kms_host)) + , _name(std::move(name)) + , _oov(std::move(oov)) + {} + future> key(const key_info& info, opt_bytes id) override { + if (id) { + return _kms_host->get_key_by_id(*id, info, &_oov).then([id](key_ptr k) { + return make_ready_future>(std::tuple(k, id)); + }); + } + return _kms_host->get_or_create_key(info, &_oov).then([](std::tuple k_id) { + return make_ready_future>(k_id); + }); + } + void print(std::ostream& os) const override { + os << _name; + } +private: + ::shared_ptr _kms_host; + std::string _name; + kms_host::option_override _oov; +}; + +shared_ptr kms_key_provider_factory::get_provider(encryption_context& ctxt, const options& map) { + opt_wrapper opts(map); + auto kms_host = opts("kms_host"); + kms_host::option_override oov { + .master_key = opts("master_key"), + .aws_assume_role_arn = opts("aws_assume_role_arn"), + }; + + if (!kms_host) { + throw std::invalid_argument("kms_host must be provided"); + } + + auto host = ctxt.get_kms_host(*kms_host); + auto id = kms_host.value() + + ":" + oov.master_key.value_or(host->options().master_key) + + ":" + oov.aws_assume_role_arn.value_or(host->options().aws_assume_role_arn) + ; + auto provider = ctxt.get_cached_provider(id); + + if (!provider) { + provider = ::make_shared(host, *kms_host, std::move(oov)); + ctxt.cache_provider(id, provider); + } + + return provider; +} + +} diff --git a/ent/encryption/kms_key_provider.hh b/ent/encryption/kms_key_provider.hh new file mode 100644 index 000000000000..ae0cf0eebab7 --- /dev/null +++ b/ent/encryption/kms_key_provider.hh @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2022 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "encryption.hh" +#include "system_key.hh" + +namespace encryption { + +class kms_key_provider_factory : public key_provider_factory { +public: + shared_ptr get_provider(encryption_context&, const options&) override; +}; + +/** + * As it stands today, given system_key api (gives keys), and + * what it is used for (config encryption), we cannot provide + * a KMS system key. This is because: + * + * a.) KMS does not allow us to store a named object (key) in a secure(ish) way. + * We can encrypt/decrypt and create one-off keys for local usage, which are + * encoded in their own ID (see kms_host), but having a unique key from + * a "path" is not possible. Esp. due to key rotation, encrypted data preamble + * etc. We could keep the encrypted key material in a local file, then decrypt + * it using a named key on startup, but given b.) it is dubious if this is useful. + * b.) System keys are only used for config encryption. The authentication config for + * AWS/KMS access is typically one of the things that should be encrypted. Thus + * we would create a big chicken and egg problem here. + */ +} diff --git a/ent/encryption/local_file_provider.cc b/ent/encryption/local_file_provider.cc new file mode 100644 index 000000000000..2fe1a3e9fa3a --- /dev/null +++ b/ent/encryption/local_file_provider.cc @@ -0,0 +1,292 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#include +#include +#include + +#include + +#include +#include + +#include +#include +#include +#include + +#include "local_file_provider.hh" +#include "symmetric_key.hh" +#include "encryption.hh" +#include "encryption_exceptions.hh" +#include "encryption_config.hh" +#include "db/config.hh" + +namespace encryption { + +namespace bfs = std::filesystem; + +const sstring default_key_file_path = (bfs::path(db::config::get_conf_dir()) / "data_encryption_keys").string(); + +static const key_info system_key_info{ "System", 0 }; + +class local_file_provider : public key_provider { +public: + local_file_provider(encryption_context& ctxt, const bfs::path& path, bool must_exist = false) + : local_file_provider(ctxt, sstring(bfs::absolute(path).string()), must_exist) + {} + local_file_provider(encryption_context& ctxt, const sstring& path, bool must_exist = false) + : _ctxt(ctxt) + , _path(path) + , _sem(1) + , _must_exist(must_exist) + {} + future> key(const key_info& info, opt_bytes = {}) override { + // TODO: assert options -> my key + auto i = _keys.find(info); + if (i != _keys.end()) { + return make_ready_future>(std::tuple(i->second, std::nullopt)); + } + return load_or_create(info).then([](key_ptr k) { + return make_ready_future>(std::tuple(k, std::nullopt)); + }); + } + future<> validate() const override { + auto f = make_ready_future<>(); + if (!_must_exist) { + return f; + } + // if we must exist, we don't change. Ok to open from all shards. + return f.then([this] { + return open_file_dma(_path, open_flags::ro).then([](file f) { + return f.close(); + }); + }).handle_exception([this](auto ep) { + try { + std::rethrow_exception(ep); + } catch (...) { + std::throw_with_nested(missing_resource_error("Could not read '" + _path + "'")); + } + }); + } + + const sstring& path() const { + return _path; + } + void print(std::ostream& os) const override { + os << "key=" << _path; + } + +private: + future load_or_create(const key_info&); + future load_or_create_local(const key_info&); + future<> read_key_file(); + future write_key_file(key_info); + + std::unordered_map _keys; + encryption_context& _ctxt; + sstring _path; + semaphore _sem; + bool _read_file = false; + bool _must_exist = false; +}; + +shared_ptr local_file_provider_factory::find(encryption_context& ctxt, const sstring& path) { + auto p = ctxt.get_cached_provider(path); + if (!p) { + p = make_shared(ctxt, path); + ctxt.cache_provider(path, p); + } + return p; +} + +shared_ptr local_file_provider_factory::get_provider(encryption_context& ctxt, const options& map) { + opt_wrapper opts(map); + return find(ctxt, opts(SECRET_KEY_FILE).value_or(default_key_file_path)); +} + +future +local_file_provider::load_or_create(const key_info& info) { + // if someone uses a system key as a table key, we could still race + // here. but that is a user error, so ignore + if (this_shard_id() == 0 || &info == &system_key_info) { + return load_or_create_local(info); + } + + struct data { + bytes key; + key_info info; + }; + + /** + * Key files are singular. Not sharded. This would be ok if we only read from them. + * But in keeping with dse compat, we don't. So rather than dealing with lock files + * or whatnot, we simply say that a single file is handled by a single key object, + * and only on shard 0. So if we are not shard 0, we call to there, find our + * counterpart object (local_file_provider_factory::find), and as him about the + * key data instead. He in turn will sync on his semaphore. + * + * The downside is that we are not resilient against multiple processes messing + * with the key file, but neither is dse + */ + return do_with(data{bytes(bytes::initialized_later(), info.len/8), info}, [this](data& i) { + return smp::submit_to(0, [this, &i]{ + auto kp = static_pointer_cast(local_file_provider_factory::find(_ctxt, _path)); + auto f = kp->load_or_create_local(i.info); + return f.then([&i, kp](key_ptr k) { + auto& kd = k->key(); + i.key.resize(kd.size()); + std::copy(kd.begin(), kd.end(), i.key.begin()); + }); + }).then([this, &i] { + auto k = make_shared(i.info, i.key); + _keys.emplace(i.info, k); + return make_ready_future(std::move(k)); + }); + }); +} + +future +local_file_provider::load_or_create_local(const key_info& info) { + if (_keys.count(info)) { + return make_ready_future(_keys.at(info)); + } + return read_key_file().then([this, info] { + if (_keys.count(info)) { + return make_ready_future(_keys.at(info)); + } + if (info == system_key_info) { + if (_keys.size() != 1) { + _keys.clear(); + return make_exception_future(std::invalid_argument("System key must contain exactly one entry")); + } + auto k = _keys.begin()->second; + _keys.clear(); + _keys.emplace(info, k); + return make_ready_future(k); + } + // create it. + return write_key_file(info); + }); +} + +future<> local_file_provider::read_key_file() { + if (_read_file) { + return make_ready_future(); + } + + // #1923 - a key can have a descriptor string line "AES:128:" iff user relies on + // defaults. Must match this as well. + static const std::regex key_line_expr(R"foo((\w+(?:\/\w+)?(?:\/\w+)?)\:(\d+)\:(\S+)\s*)foo"); + + return with_semaphore(_sem, 1, [this] { + // could do this twice, but it is only reading + return read_text_file_fully(_path).then([this](temporary_buffer buf) { + auto i = std::cregex_iterator(buf.begin(), buf.end(), key_line_expr); + auto e = std::cregex_iterator(); + + while (i != e) { + std::cmatch m = *i; + auto alg = m[1].str(); + auto len = std::stoul(m[2].str()); + auto key = m[3].str(); + + auto info = key_info{alg, unsigned(len)}; + if (!_keys.count(info)) { + auto kb = base64_decode(key); + auto k = make_shared(info, kb); + _keys.emplace(info, std::move(k)); + } + ++i; + } + _read_file = true; + }).handle_exception([this](auto ep) { + try { + std::rethrow_exception(ep); + } catch (std::system_error& e) { + if (e.code() == std::error_code(ENOENT, std::system_category())) { + if (!_must_exist) { + return; + } + std::throw_with_nested(configuration_error("Key file '" + _path + "' does not exist")); + } + std::throw_with_nested(service_error("read_key_file")); + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(fmt::format("read_key_file: {}", e.what()))); + } catch (...) { + std::throw_with_nested(service_error(fmt::format("read_key_file: {}", std::current_exception()))); + } + }); + }); +} + +future local_file_provider::write_key_file(key_info info) { + return with_semaphore(_sem, 1, [this, info] { + // we can get here more than once if shards race. + // however, we only need to use/write the first key matching + // the required info. + if (_keys.count(info)) { + return make_ready_future(_keys.at(info)); + } + + auto k = make_shared(info); + + std::ostringstream ss; + for (auto& p : _keys) { + ss << p.first.alg << ":" << p.first.len << ":" << base64_encode(p.second->key()) << std::endl; + } + ss << info.alg << ":" << info.len << ":" << base64_encode(k->key()) << std::endl; + auto s = ss.str(); + auto tmpnam = _path + ".tmp"; + auto f = make_ready_future<>(); + if (!_must_exist) { + f = seastar::recursive_touch_directory((bfs::path(tmpnam).remove_filename()).string()); + } + return f.then([this, tmpnam, s] { + return write_text_file_fully(tmpnam, s).then([this, tmpnam] { + return rename_file(tmpnam, _path); + }); + }).then([this, k, info] { + // don't cache until written + _keys[info] = k; + return make_ready_future(k); + }); + }).handle_exception([this](auto ep) -> key_ptr{ + try { + std::rethrow_exception(ep); + } catch (...) { + std::throw_with_nested(service_error("Could not write key file '" + _path + "'")); + } + }); +} + +local_system_key::local_system_key(encryption_context& ctxt, const sstring& path) + : _provider(make_shared(ctxt, bfs::path(ctxt.config().system_key_directory()) / bfs::path(path), true)) +{} + +local_system_key::~local_system_key() +{} + +future> local_system_key::get_key() { + return _provider->key(system_key_info).then([](std::tuple k_id) { + return make_ready_future>(std::get<0>(std::move(k_id))); + }); +} + +future<> local_system_key::validate() const { + // first, just validate the file provider itself + co_await _provider->validate(); + // second, do an early load of the actual key to ensure file contents. + co_await _provider->key(system_key_info); +} + +const sstring& local_system_key::name() const { + return _provider->path(); +} + +} diff --git a/ent/encryption/local_file_provider.hh b/ent/encryption/local_file_provider.hh new file mode 100644 index 000000000000..9378e413c8a9 --- /dev/null +++ b/ent/encryption/local_file_provider.hh @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "encryption.hh" +#include "system_key.hh" + +namespace encryption { + +const extern sstring default_key_file_path; + +class local_file_provider; + +class local_file_provider_factory : public key_provider_factory { +public: + static shared_ptr find(encryption_context&, const sstring& path); + shared_ptr get_provider(encryption_context&, const options&) override; +}; + +class local_system_key : public system_key { + shared_ptr _provider; +public: + local_system_key(encryption_context&, const sstring&); + ~local_system_key(); + + future> get_key() override; + future<> validate() const override; + const sstring& name() const override; + bool is_local() const override { + return true; + } +}; + +} diff --git a/ent/encryption/replicated_key_provider.cc b/ent/encryption/replicated_key_provider.cc new file mode 100644 index 000000000000..9a06950fcc09 --- /dev/null +++ b/ent/encryption/replicated_key_provider.cc @@ -0,0 +1,477 @@ +/* + * Copyright (C) 2015 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#include +#include +#include +#include + +#include + +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include "utils/to_string.hh" + +#include "replicated_key_provider.hh" +#include "encryption.hh" +#include "encryption_exceptions.hh" +#include "local_file_provider.hh" +#include "symmetric_key.hh" +#include "replica/database.hh" +#include "cql3/query_processor.hh" +#include "cql3/untyped_result_set.hh" +#include "utils/UUID.hh" +#include "utils/UUID_gen.hh" +#include "utils/hash.hh" +#include "service/storage_service.hh" +#include "service/migration_manager.hh" +#include "compaction/compaction_manager.hh" +#include "replica/distributed_loader.hh" +#include "schema/schema_builder.hh" +#include "db/system_keyspace.hh" +#include "db/extensions.hh" +#include "locator/everywhere_replication_strategy.hh" + +namespace encryption { + +static auto constexpr KSNAME = "system_replicated_keys"; +static auto constexpr TABLENAME = "encrypted_keys"; + +static logger log("replicated_key_provider"); + +using utils::UUID; + +class replicated_key_provider : public key_provider { +public: + static constexpr int8_t version = 0; + /** + * Header: + * 1 byte version + * 16 bytes UUID of key + * 16 bytes MD5 of UUID + */ + static const size_t header_size = 33; + + struct key_id { + key_info info; + opt_bytes id; + + key_id(key_info k, opt_bytes b = {}) + : info(std::move(k)) + , id(std::move(b)) + {} + bool operator==(const key_id& v) const { + return info == v.info && id == v.id; + } + }; + + struct key_id_hash { + size_t operator()(const key_id& id) const { + return utils::tuple_hash()(std::tie(id.info.alg, id.info.len, id.id)); + } + }; + + replicated_key_provider(encryption_context& ctxt, shared_ptr system_key, shared_ptr local_provider) + : _ctxt(ctxt) + , _system_key(std::move(system_key)) + , _local_provider(std::move(local_provider)) + {} + + + future> key(const key_info&, opt_bytes = {}) override; + future<> validate() const override; + future<> maybe_initialize_tables(); + static future<> do_initialize_tables(::replica::database& db, service::migration_manager&); + + bool should_delay_read(const opt_bytes& id) const override { + if (!id || _initialized) { + return false; + } + if (!_initialized) { + return true; + } + auto& qp = _ctxt.get_query_processor(); + // This check should be ok, and even somewhat redundant. "Initialized" above + // will only be set once we've generated/queried a key not passing through here + // (i.e. a key for write _or_ commit log (should we allow this)). This can only be + // done if: + // a.) Encryption was already set up, thus table existed and we waited + // for distributed_tables in "ensure_populated" + // b.) Encryption was added. In which case we are way past bootstrap + // and can receive user commands. + // c.) System table/commit log write, with either first use of this provider, + // in which case we're creating the table (here at least) - thus fine, + // or again, we've waited through "ensure_populated", so keys are + // readble. At worst, we create a few extra keys. + // Note: currently c.) is not relevant, as we don't support system/commitlog + // encryption using repl_prov. + return !qp.local_is_initialized(); + } + + void print(std::ostream& os) const override { + os << "system_key=" << _system_key->name() << ", local=" << *_local_provider; + } + +private: + void store_key(const key_id&, const UUID&, key_ptr); + + static opt_bytes decode_id(const opt_bytes&); + static bytes encode_id(const UUID&); + + future> get_key(const key_info&, opt_bytes = {}); + + future load_or_create(const key_info&); + future load_or_create_local(const key_info&); + future<> read_key_file(); + future<> write_key_file(); + + template + future<::shared_ptr> query(sstring, Args&& ...); + + future<> force_blocking_flush(); + + encryption_context& _ctxt; + shared_ptr _system_key; + shared_ptr _local_provider; + std::unordered_map, key_id_hash> _keys; + + bool _initialized = false; + bool _use_cache = true; + + friend class replicated_key_provider_factory; + + static const utils::UUID local_fallback_uuid; + static const bytes local_fallback_id; + static const bytes_view local_fallback_bytes; +}; + +using namespace std::chrono_literals; + +static const timeout_config rkp_db_timeout_config { + 5s, 5s, 5s, 5s, 5s, 5s, 5s, +}; + +static service::query_state& rkp_db_query_state() { + static thread_local service::client_state cs(service::client_state::internal_tag{}, rkp_db_timeout_config); + static thread_local service::query_state qs(cs, empty_service_permit()); + return qs; +} + +template +future<::shared_ptr> replicated_key_provider::query(sstring q, Args&& ...params) { + auto mode = co_await _ctxt.get_storage_service().local().get_operation_mode(); + if (mode != service::storage_service::mode::STARTING) { + co_return co_await _ctxt.get_query_processor().local().execute_internal(q, { std::forward(params)...}, cql3::query_processor::cache_internal::no); + } + co_return co_await _ctxt.get_query_processor().local().execute_internal(q, db::consistency_level::ONE, rkp_db_query_state(), { std::forward(params)...}, cql3::query_processor::cache_internal::no); +} + +future<> replicated_key_provider::force_blocking_flush() { + return _ctxt.get_database().invoke_on_all([](replica::database& db) { + // if (!Boolean.getBoolean("cassandra.unsafesystem")) + replica::column_family& cf = db.find_column_family(KSNAME, TABLENAME); + return cf.flush(); + }); +} + +void replicated_key_provider::store_key(const key_id& id, const UUID& uuid, key_ptr k) { + if (!_use_cache) { + return; + } + _keys[id] = std::make_pair(uuid, k); + if (!id.id) { + _keys[key_id(id.info, uuid.serialize())] = std::make_pair(uuid, k); + } +} + +opt_bytes replicated_key_provider::decode_id(const opt_bytes& b) { + if (b) { + auto i = b->begin(); + auto v = *i++; + if (v == version && b->size() == 33) { + bytes id(i + 1, i + 1 + 16); + bytes md(i + 1 + 16, b->end()); + if (calculate_md5(id) == md) { + return id; + } + } + } + return std::nullopt; +} + +bytes replicated_key_provider::encode_id(const UUID& uuid) { + bytes b{bytes::initialized_later(), header_size}; + auto i = b.begin(); + *i++ = version; + uuid.serialize(i); + auto md = calculate_md5(b, 1, 16); + std::copy(md.begin(), md.end(), i); + return b; +} + +const utils::UUID replicated_key_provider::local_fallback_uuid(0u, 0u); // not valid! +const bytes replicated_key_provider::local_fallback_id = encode_id(local_fallback_uuid); +const bytes_view replicated_key_provider::local_fallback_bytes(local_fallback_id.data() + 1, 16); + +future> replicated_key_provider::key(const key_info& info, opt_bytes input) { + opt_bytes id; + + if (input) { //reading header? + auto v = *input; + if (v[0] == version) { + bytes bid(v.begin() + 1, v.begin() + 1 + 16); + bytes md(v.begin() + 1 + 16, v.begin() + 1 + 32); + if (calculate_md5(bid) == md) { + id = bid; + } + } + } + + bool try_local = id == local_fallback_bytes; + + // if the id indicates the key came from local fallback, don't even + // try keyspace lookup. + if (!try_local) { + try { + auto [uuid, k] = co_await get_key(info, std::move(id)); + co_return std::make_tuple(k, encode_id(uuid)); + } catch (std::invalid_argument& e) { + std::throw_with_nested(configuration_error(e.what())); + } catch (...) { + auto ep = std::current_exception(); + log.warn("Exception looking up key {}: {}", info, ep); + if (_local_provider) { + try { + std::rethrow_exception(ep); + } catch (replica::no_such_keyspace&) { + } catch (exceptions::invalid_request_exception&) { + } catch (exceptions::read_failure_exception&) { + } catch (...) { + std::throw_with_nested(service_error(fmt::format("key: {}", std::current_exception()))); + } + if (!id) { + try_local = true; + } + } + if (!try_local) { + std::throw_with_nested(service_error(fmt::format("key: {}", std::current_exception()))); + } + } + } + + log.warn("Falling back to local key {}", info); + auto [k, nid] = co_await _local_provider->key(info, id); + if (nid && nid != id) { + // local provider does not give ids. + throw malformed_response_error("Expected null id back from local provider"); + } + co_return std::make_tuple(k, local_fallback_id); +} + +future> replicated_key_provider::get_key(const key_info& info, opt_bytes opt_id) { + if (!_initialized) { + co_await maybe_initialize_tables(); + } + + key_id id(info, std::move(opt_id)); + auto i = _keys.find(id); + if (i != _keys.end()) { + co_return std::tuple(i->second.first, i->second.second); + } + + // TODO: origin does non-cql acquire of all available keys from + // replicas in the "host_ids" table iff we get here during boot. + // For now, ignore this and assume that if we have a sstable with + // key X, we should have a local replica of X as well, given + // the "everywhere strategy of the keys table. + + auto cipher = info.alg.substr(0, info.alg.find('/')); // e.g. "AES" + + UUID uuid; + shared_ptr res; + + if (id.id) { + uuid = utils::UUID_gen::get_UUID(*id.id); + log.debug("Finding key {} ({})", uuid, info); + auto s = fmt::format("SELECT * FROM {}.{} WHERE key_file=? AND cipher=? AND strength=? AND key_id=?;", KSNAME, TABLENAME); + res = co_await query(std::move(s), _system_key->name(), cipher, int32_t(id.info.len), uuid); + + // if we find nothing, and we actually queried a specific key (by uuid), we've failed. + if (res->empty()) { + log.debug("Could not find key {}", id.id); + throw std::runtime_error(fmt::format("Unable to find key for cipher={} strength={} id={}", cipher, id.info.len, uuid)); + } + } else { + log.debug("Finding key ({})", info); + auto s = fmt::format("SELECT * FROM {}.{} WHERE key_file=? AND cipher=? AND strength=? LIMIT 1;", KSNAME, TABLENAME); + res = co_await query(std::move(s), _system_key->name(), cipher, int32_t(id.info.len)); + } + + // otoh, if we don't need a specific key, we can just create a new one (writing a sstable) + if (res->empty()) { + uuid = utils::UUID_gen::get_time_UUID(); + + log.debug("No key found. Generating {}", uuid); + + auto k = make_shared(id.info); + store_key(id, uuid, k); + + auto b = co_await _system_key->encrypt(k->key()); + auto ks = base64_encode(b); + log.trace("Inserting generated key {}", uuid); + co_await query(fmt::format("INSERT INTO {}.{} (key_file, cipher, strength, key_id, key) VALUES (?, ?, ?, ?, ?)", + KSNAME, TABLENAME), _system_key->name(), cipher, int32_t(id.info.len), uuid, ks + ); + log.trace("Flushing key table"); + co_await force_blocking_flush(); + + co_return std::tuple(uuid, k); + } + + // found it + auto& row = res->one(); + uuid = row.get_as("key_id"); + auto ks = row.get_as("key"); + auto kb = base64_decode(ks); + auto b = co_await _system_key->decrypt(kb); + auto k = make_shared(id.info, b); + store_key(id, uuid, k); + + co_return std::tuple(uuid, k); +} + +future<> replicated_key_provider::validate() const { + try { + co_await _system_key->validate(); + } catch (...) { + std::throw_with_nested(std::invalid_argument(fmt::format("Could not validate system key: {}", _system_key->name()))); + } + if (_local_provider){ + co_await _local_provider->validate(); + } +} + +schema_ptr encrypted_keys_table() { + static thread_local auto schema = [] { + auto id = generate_legacy_id(KSNAME, TABLENAME); + return schema_builder(KSNAME, TABLENAME, std::make_optional(id)) + .with_column("key_file", utf8_type, column_kind::partition_key) + .with_column("cipher", utf8_type, column_kind::partition_key) + .with_column("strength", int32_type, column_kind::clustering_key) + .with_column("key_id", timeuuid_type, column_kind::clustering_key) + .with_column("key", utf8_type) + .with_hash_version() + .build(); + }(); + return schema; +} + +future<> replicated_key_provider::maybe_initialize_tables() { + if (!_initialized) { + co_await do_initialize_tables(_ctxt.get_database().local(), _ctxt.get_migration_manager().local()); + _initialized = true; + } +} + +future<> replicated_key_provider::do_initialize_tables(::replica::database& db, service::migration_manager& mm) { + if (db.has_schema(KSNAME, TABLENAME)) { + co_return; + } + + log.debug("Creating keyspace and table"); + if (!db.has_keyspace(KSNAME)) { + auto group0_guard = co_await mm.start_group0_operation(); + auto ts = group0_guard.write_timestamp(); + try { + auto ksm = keyspace_metadata::new_keyspace( + KSNAME, + "org.apache.cassandra.locator.EverywhereStrategy", + {}, + std::nullopt, + true); + co_await mm.announce(service::prepare_new_keyspace_announcement(db, ksm, ts), std::move(group0_guard), fmt::format("encryption at rest: create keyspace {}", KSNAME)); + } catch (exceptions::already_exists_exception&) { + } + } + auto group0_guard = co_await mm.start_group0_operation(); + auto ts = group0_guard.write_timestamp(); + try { + co_await mm.announce(co_await service::prepare_new_column_family_announcement(mm.get_storage_proxy(), encrypted_keys_table(), ts), std::move(group0_guard), + fmt::format("encryption at rest: create table {}.{}", KSNAME, TABLENAME)); + } catch (exceptions::already_exists_exception&) { + } + auto& ks = db.find_keyspace(KSNAME); + auto& rs = ks.get_replication_strategy(); + // should perhaps check name also.. + if (rs.get_type() != locator::replication_strategy_type::everywhere_topology) { + // TODO: reset to everywhere + repair. + } +} + +const size_t replicated_key_provider::header_size; + +replicated_key_provider_factory::replicated_key_provider_factory() +{} + +replicated_key_provider_factory::~replicated_key_provider_factory() +{} + +namespace bfs = std::filesystem; + +shared_ptr replicated_key_provider_factory::get_provider(encryption_context& ctxt, const options& map) { + opt_wrapper opts(map); + auto system_key_name = opts(SYSTEM_KEY_FILE).value_or("system_key"); + if (system_key_name.find('/') != sstring::npos) { + throw std::invalid_argument("system_key cannot contain '/'"); + } + + auto system_key = ctxt.get_system_key(system_key_name); + auto local_key_file = bfs::absolute(bfs::path(opts(SECRET_KEY_FILE).value_or(default_key_file_path))); + + if (system_key->is_local() && bfs::absolute(bfs::path(system_key->name())) == local_key_file) { + throw std::invalid_argument("system key and local key cannot be the same"); + } + + auto name = system_key->name() + ":" + local_key_file.string(); + auto debug = opts("DEBUG"); + if (debug) { + name = name + ":" + *debug; + } + auto p = ctxt.get_cached_provider(name); + if (!p) { + auto rp = seastar::make_shared(ctxt, std::move(system_key), local_file_provider_factory::find(ctxt, local_key_file.string())); + ctxt.cache_provider(name, rp); + + if (debug && debug->find("nocache") != sstring::npos) { + log.debug("Turn off cache"); + rp->_use_cache = false; + } + p = std::move(rp); + } + + return p; +} + +void replicated_key_provider_factory::init(db::extensions& exts) { + exts.add_extension_internal_keyspace(KSNAME); +} + +future<> replicated_key_provider_factory::on_started(::replica::database& db, service::migration_manager& mm) { + return replicated_key_provider::do_initialize_tables(db, mm); +} + +} diff --git a/ent/encryption/replicated_key_provider.hh b/ent/encryption/replicated_key_provider.hh new file mode 100644 index 000000000000..cd20208e9e71 --- /dev/null +++ b/ent/encryption/replicated_key_provider.hh @@ -0,0 +1,39 @@ +/* + * Copyright (C) 2015 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "encryption.hh" + +namespace db { +class extensions; +} + +namespace replica { +class database; +} + +namespace service { +class migration_manager; +} + +namespace encryption { + +class replicated_key_provider_factory : public key_provider_factory { +public: + replicated_key_provider_factory(); + ~replicated_key_provider_factory(); + + shared_ptr get_provider(encryption_context&, const options&) override; + + static void init(db::extensions&); + static future<> on_started(::replica::database&, service::migration_manager&); +}; + +} diff --git a/ent/encryption/symmetric_key.cc b/ent/encryption/symmetric_key.cc new file mode 100644 index 000000000000..7f3a267f9b5b --- /dev/null +++ b/ent/encryption/symmetric_key.cc @@ -0,0 +1,396 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#include +#include +#include + +#include +#include +#include + +#if OPENSSL_VERSION_NUMBER >= (3<<28) +# include +#endif + +#include +#include + +#include "symmetric_key.hh" +#include "utils/hash.hh" + +namespace { +struct openssl_env { + OSSL_PROVIDER* legacy_provider = nullptr; + OSSL_PROVIDER* default_provider = nullptr; + openssl_env() { + OpenSSL_add_all_ciphers(); +#if OPENSSL_VERSION_NUMBER >= (3<<28) + legacy_provider = OSSL_PROVIDER_load(NULL, "legacy"); + default_provider = OSSL_PROVIDER_load(NULL, "default"); +#endif + } + ~openssl_env() { + OSSL_PROVIDER_unload(legacy_provider); + OSSL_PROVIDER_unload(default_provider); + } +}; +static const openssl_env ossl_env; +} + +std::ostream& encryption::operator<<(std::ostream& os, const key_info& info) { + return os << info.alg << ":" << info.len; +} + +static void throw_evp_error(std::string msg) { + auto e = ERR_get_error(); + if (e != 0) { + char buf[512]; + ERR_error_string_n(e, buf, sizeof(buf)); + msg += "(" + std::string(buf) + ")"; + } + throw std::runtime_error(msg); +} + +bool encryption::key_info::compatible(const key_info& rhs) const { + sstring malg, halg; + std::tie(malg, std::ignore, std::ignore) = parse_key_spec(alg); + std::tie(halg, std::ignore, std::ignore) = parse_key_spec(rhs.alg); + if (malg != halg) { + return false; + } + // If lengths differ we need to actual create keys to + // check what the true lengths are. Since openssl and + // java designators count different for DES etc. + if (len != rhs.len) { + symmetric_key k1(*this); + symmetric_key k2(rhs); + if (k1.key().size() != k2.key().size()) { + return false; + } + } + return true; +} + +std::tuple +encryption::parse_key_spec(const sstring& alg) { + static const std::regex alg_exp(R"foo(^(\w+)(?:\/(\w+))?(?:\/(\w+))?$)foo"); + + std::cmatch m; + if (!std::regex_match(alg.begin(), alg.end(), m, alg_exp)) { + throw std::invalid_argument("Invalid algorithm string: " + alg); + } + + auto type = m[1].str(); + auto mode = m[2].str(); + auto padd = m[3].str(); + + std::transform(type.begin(), type.end(), type.begin(), ::tolower); + std::transform(mode.begin(), mode.end(), mode.begin(), ::tolower); + std::transform(padd.begin(), padd.end(), padd.begin(), ::tolower); + + static const std::string padding = "padding"; + if (padd.size() > padding.size() && std::equal(padding.rbegin(), padding.rend(), padd.rbegin())) { + padd.resize(padd.size() - padding.size()); + } + + return std::make_tuple(type, mode, padd); +} + +std::tuple encryption::parse_key_spec_and_validate_defaults(const sstring& alg) { + auto [type, mode, padd] = parse_key_spec(alg); + + // openssl AND kmip server(s?) does not allow missing block mode. so default one. + if (mode.empty()) { + mode = "cbc"; + } + + // OpenSSL only supports one form of padding. We used to just allow + // non-empty string -> pkcs5/pcks7. Better to verify + // (note: pcks5 is sortof a misnomeanor here, as in the Sun world, it + // sort of means "pkcs7 with automatic block size" - which is pretty + // much how things are in the OpenSSL universe as well) + if (padd == "no") { + padd = ""; + } + if (!padd.empty() && padd != "pkcs5" && padd != "pkcs" && padd != "pkcs7") { + throw std::invalid_argument("non-supported padding option: " + padd); + } + + return { type, mode, padd }; +} + +encryption::symmetric_key::symmetric_key(const key_info& info, const bytes& key) + : _ctxt(EVP_CIPHER_CTX_new(), &EVP_CIPHER_CTX_free) + , _info(info) + , _key(key) +{ + if (!_ctxt) { + throw std::bad_alloc(); + } + + sstring type, mode, padd; + std::tie(type, mode, padd) = parse_key_spec_and_validate_defaults(info.alg); + + // Note: we are using some types here that are explicitly marked as "unsupported - placeholder" + // in gnutls. + + // camel case vs. dash + if (type == "desede") { + type = "des-ede"; + // and 168-bits desede is ede3 in openssl... + if (info.len > 16*8) { + type = "des-ede3"; + } + } + + auto str = fmt::format("{}-{}-{}", type, info.len, mode); + auto cipher = EVP_get_cipherbyname(str.c_str()); + + if (!cipher) { + str = fmt::format("{}-{}", type, mode); + cipher = EVP_get_cipherbyname(str.c_str()); + } + if (!cipher) { + str = fmt::format("{}-{}", type, info.len); + cipher = EVP_get_cipherbyname(str.c_str()); + } + if (!cipher) { + str = type; + cipher = EVP_get_cipherbyname(str.c_str()); + } + if (!cipher) { + throw_evp_error("Invalid algorithm: " + info.alg); + } + + size_t len = EVP_CIPHER_key_length(cipher); + + if ((_info.len/8) != len) { + if (!EVP_CipherInit_ex(*this, cipher, nullptr, nullptr, nullptr, 0)) { + throw_evp_error("Could not initialize cipher"); + } + auto dlen = _info.len/8; + // Openssl describes des-56 length as 64 (counts parity), + // des-ede-112 as 128 etc... + // do some special casing... + if ((type == "des" || type == "des-ede" || type == "des-ede3") && (dlen & 7) != 0) { + dlen = align_up(dlen, 8u); + } + // if we had to find a cipher without explicit key length (like rc2), + // try to set the key length to the desired strength. + if (!EVP_CIPHER_CTX_set_key_length(*this, dlen)) { + throw_evp_error(fmt::format("Invalid length {} for resolved type {} (wanted {})", len*8, str, _info.len)); + } + + len = EVP_CIPHER_key_length(cipher); + } + + + if (_key.empty()) { + _key.resize(len); + if (!RAND_bytes(reinterpret_cast(_key.data()), _key.size())) { + throw_evp_error(fmt::format("Could not generate key: {}", info.alg)); + } + } + if (_key.size() < len) { + throw std::invalid_argument(fmt::format("Invalid key data length {} for resolved type {} ({})", _key.size()*8, str, len*8)); + } + + if (!EVP_CipherInit_ex(*this, cipher, nullptr, + reinterpret_cast(_key.data()), nullptr, + 0)) { + throw_evp_error("Could not initialize cipher from key materiel"); + } + + _iv_len = EVP_CIPHER_CTX_iv_length(*this); + _block_size = EVP_CIPHER_CTX_block_size(*this); + _padding = !padd.empty(); + +} + +std::string encryption::symmetric_key::validate_exact_info_result() const { + auto [types, modes, padds] = parse_key_spec(_info.alg); + + auto cipher = EVP_CIPHER_CTX_get0_cipher(*this); + auto len = EVP_CIPHER_key_length(cipher); + auto mode = EVP_CIPHER_get_mode(cipher); + + std::ostringstream ss; + + if (unsigned(len)*8 != align_up(_info.len, 16u)) { + ss << "Length " << len*8 << " differs from requested " << _info.len << std::endl; + } + + static std::unordered_map openssl_modes({ + { EVP_CIPH_ECB_MODE, "ecb" }, + { EVP_CIPH_CBC_MODE, "cbc" }, + { EVP_CIPH_CFB_MODE, "cfb" }, + { EVP_CIPH_OFB_MODE, "ofb" }, + { EVP_CIPH_CTR_MODE, "ctr" }, + { EVP_CIPH_GCM_MODE, "cgm" }, + { EVP_CIPH_CCM_MODE, "ccm" }, + { EVP_CIPH_XTS_MODE, "xts" }, + { EVP_CIPH_WRAP_MODE, "wrap"}, + { EVP_CIPH_OCB_MODE, "ocb" }, + { EVP_CIPH_SIV_MODE, "siv" }, + }); + + auto i = openssl_modes.find(mode); + if (i != openssl_modes.end() && i->second != modes) { + ss << _info << ": " << "Block mode " << i->second << " differers from requested " << modes << std::endl; + } + + if ((!padds.empty() && padds != "no") != _padding) { + ss << _info << ": " << "Padding (" << bool(_padding) << " differs from requested " << padds << std::endl; + } + + return ss.str(); +} + +void encryption::symmetric_key::generate_iv_impl(uint8_t* dst, size_t s) const { + if (s < _iv_len) { + throw std::invalid_argument("Buffer underflow"); + } + if (!RAND_bytes(dst, s)) { + throw_evp_error("Could not generate initialization vector"); + } +} + +void encryption::symmetric_key::transform_unpadded_impl(const uint8_t* input, + size_t input_len, uint8_t* output, const uint8_t* iv, mode m) const { + if (!EVP_CipherInit_ex(*this, nullptr, nullptr, + reinterpret_cast(_key.data()), iv, int(m))) { + throw_evp_error("Could not initialize cipher (transform)"); + } + if (!EVP_CIPHER_CTX_set_padding(*this, 0)) { + throw_evp_error("Could not disable padding"); + } + + if (input_len & (_block_size - 1)) { + throw std::invalid_argument("Data must be aligned to 'blocksize'"); + } + + int outl = 0; + auto res = m == mode::decrypt ? + EVP_DecryptUpdate(*this, output, &outl, input, + int(input_len)) : + EVP_EncryptUpdate(*this, output, &outl, input, + int(input_len)); + + if (!res || outl != int(input_len)) { + throw std::runtime_error("transformation failed"); + } +} + +size_t encryption::symmetric_key::decrypt_impl(const uint8_t* input, + size_t input_len, uint8_t* output, size_t output_len, + const uint8_t* iv) const { + if (!EVP_CipherInit_ex(*this, nullptr, nullptr, + reinterpret_cast(_key.data()), iv, 0)) { + throw_evp_error("Could not initialize cipher (decrypt)"); + } + if (!EVP_CIPHER_CTX_set_padding(*this, int(_padding))) { + throw_evp_error("Could not initialize padding"); + } + + // normal case, caller provides output enough to deal with any padding. + // in padding case, max out size is input_len - 1. + if (input_len <= output_len) { + // one go. + int outl = 0; + int finl = 0; + if (!EVP_DecryptUpdate(*this, output, &outl, input, int(input_len))) { + throw_evp_error("decryption failed"); + } + if (!EVP_DecryptFinal(*this, output + outl, &finl)) { + throw_evp_error("decryption failed"); + } + + return outl + finl; + } + + // meh. must provide block padding. + constexpr size_t local_buf_size = 1024; + + static thread_local std::vector cached_buf; + + if (cached_buf.size() < local_buf_size + _block_size) [[unlikely]] { + cached_buf.resize(local_buf_size + _block_size); + } + + auto buf = cached_buf.data(); + size_t res = 0; + while (input_len) { + auto n = std::min(input_len, local_buf_size); + int outl = 0; + if (!EVP_DecryptUpdate(*this, buf, &outl, input, int(n))) { + throw std::runtime_error("decryption failed"); + } + if (n < local_buf_size) { + // last block + int finl = 0; + if (!EVP_DecryptFinal(*this, buf + outl, &finl)) { + throw std::runtime_error("decryption failed"); + } + outl += finl; + } + if ((res + outl) > output_len) { + throw std::invalid_argument("Output buffer too small"); + } + output = std::copy(buf, buf + outl, output); + res += outl; + input_len -= n; + input += n; + } + + return res; +} + +size_t encryption::symmetric_key::encrypted_size(size_t n) const { + // encryption always adds padding. So if n is multiple of blocksize + // the size is n + blocksize. But if its not, things are "better"... + return _block_size + align_down(n, _block_size); +} + +size_t encryption::symmetric_key::encrypt_impl(const uint8_t* input, + size_t input_len, uint8_t* output, size_t output_len, + const uint8_t* iv) const { + if (output_len < encrypted_size(input_len)) { + throw std::invalid_argument("Insufficient buffer"); + } + + if (!EVP_CipherInit_ex(*this, nullptr, nullptr, + reinterpret_cast(_key.data()), iv, 1)) { + throw_evp_error("Could not initialize cipher (encrypt)"); + } + if (!EVP_CIPHER_CTX_set_padding(*this, int(_padding))) { + throw_evp_error("Could not initialize padding"); + } + + int outl = 0; + int finl = 0; + if (!EVP_EncryptUpdate(*this, output, &outl, input, int(input_len))) { + throw_evp_error("encryption failed"); + } + if (!EVP_EncryptFinal(*this, output + outl, &finl)) { + throw_evp_error("encryption failed"); + } + return outl + finl; +} + +bool encryption::operator==(const key_info& k1, const key_info& k2) { + return k1.alg == k2.alg && k1.len == k2.len; +} + +bool encryption::operator!=(const key_info& k1, const key_info& k2) { + return !(k1 == k2); +} + +size_t encryption::key_info_hash::operator()(const key_info& e) const { + return utils::tuple_hash()(std::tie(e.alg, e.len)); +} diff --git a/ent/encryption/symmetric_key.hh b/ent/encryption/symmetric_key.hh new file mode 100644 index 000000000000..4a752e396f28 --- /dev/null +++ b/ent/encryption/symmetric_key.hh @@ -0,0 +1,154 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include +#include +#include + +#include "../../bytes.hh" + +// forward declare openssl evp. +extern "C" { +struct evp_cipher_ctx_st; +} + +namespace encryption { + +struct key_info { + sstring alg; + unsigned len; + + bool compatible(const key_info&) const; +}; + +bool operator==(const key_info& k1, const key_info& k2); +bool operator!=(const key_info& k1, const key_info& k2); +std::ostream& operator<<(std::ostream&, const key_info&); + +struct key_info_hash { + size_t operator()(const key_info& e) const; +}; + +std::tuple parse_key_spec(const sstring&); + +// shared between key & kmip +std::tuple parse_key_spec_and_validate_defaults(const sstring&); + +class symmetric_key { + std::unique_ptr _ctxt; + key_info _info; + bytes _key; + unsigned _iv_len = 0; + unsigned _block_size = 0; + bool _padding = true; + + operator evp_cipher_ctx_st *() const { + return _ctxt.get(); + } + + void generate_iv_impl(uint8_t* dst, size_t) const; + size_t decrypt_impl(const uint8_t* input, size_t input_len, uint8_t* output, + size_t output_len, const uint8_t* iv) const; + size_t encrypt_impl(const uint8_t* input, size_t input_len, uint8_t* output, + size_t output_len, const uint8_t* iv) const; + +public: + symmetric_key(const key_info& info, const bytes& key = { }); + + const key_info& info() const { + return _info; + } + const bytes& key() const { + return _key; + } + size_t iv_len() const { + return _iv_len; + } + size_t block_size() const { + return _block_size; + } + + /** + * Evaluates whether or not the key info provided resulted in + * the exact same result from openssl, i.e. whether the combination + * of alg/block mode/padding etc was actually fully valid (or our + * heuristics have issues) + */ + std::string validate_exact_info_result() const; + + /** + * Write a random IV to dst. Must be iv_len() sized or larger + */ + template + void generate_iv(T* dst, size_t s) const { + static_assert(sizeof(T) == sizeof(uint8_t) && std::is_integral_v); + generate_iv_impl(reinterpret_cast(dst), s); + } + + // returns minimal buffer size required to encrypt n bytes. I.e. + // block alignment + size_t encrypted_size(size_t n) const; + + template + size_t decrypt(const T* input, size_t input_len, V* output, + size_t output_len, const I* iv = nullptr) const { + static_assert(sizeof(T) == sizeof(uint8_t) && std::is_integral_v); + return decrypt_impl(reinterpret_cast(input), input_len, + reinterpret_cast(output), output_len, + reinterpret_cast(iv)); + } + template + size_t encrypt(const T* input, size_t input_len, V* output, + size_t output_len, const I* iv = nullptr) const { + static_assert(sizeof(T) == sizeof(uint8_t) && std::is_integral_v); + return encrypt_impl(reinterpret_cast(input), input_len, + reinterpret_cast(output), output_len, + reinterpret_cast(iv)); + } + + enum class mode { + decrypt, encrypt, + }; + template + void transform_unpadded(mode m, const T* input, size_t input_len, V* output, + const I* iv = nullptr) const { + static_assert(sizeof(T) == sizeof(uint8_t) && std::is_integral_v); + return transform_unpadded_impl(reinterpret_cast(input), + input_len, reinterpret_cast(output), + reinterpret_cast(iv), m); + } + template + void encrypt_unpadded(const T* input, size_t input_len, V* output, + const I* iv = nullptr) const { + static_assert(sizeof(T) == sizeof(uint8_t) && std::is_integral_v); + return transform_unpadded_impl(reinterpret_cast(input), + input_len, reinterpret_cast(output), + reinterpret_cast(iv), mode::encrypt); + } + template + void decrypt_unpadded(const T* input, size_t input_len, V* output, + const I* iv = nullptr) const { + static_assert(sizeof(T) == sizeof(uint8_t) && std::is_integral_v); + return transform_unpadded_impl(reinterpret_cast(input), + input_len, reinterpret_cast(output), + reinterpret_cast(iv), mode::decrypt); + } + +private: + void transform_unpadded_impl(const uint8_t* input, size_t input_len, + uint8_t* output, const uint8_t* iv, mode) const; +}; + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/ent/encryption/system_key.cc b/ent/encryption/system_key.cc new file mode 100644 index 000000000000..bb8b8456395e --- /dev/null +++ b/ent/encryption/system_key.cc @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2015 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#include +#include +#include +#include + +#include +#include + +#include + +#include "symmetric_key.hh" +#include "system_key.hh" + +future<> encryption::system_key::validate() const { + return make_ready_future<>(); +} + +future encryption::system_key::decrypt(const sstring& s) { + auto b = base64_decode(s); + return decrypt(b).then([](bytes b) { + return make_ready_future(sstring(b.begin(), b.end())); + }); +} + +future encryption::system_key::encrypt(const sstring& s) { + return encrypt(bytes(s.begin(), s.end())).then([](bytes b) { + return make_ready_future(base64_encode(b)); + }); +} + +future encryption::system_key::encrypt(const bytes& b) { + return get_key().then([b](shared_ptr k) { + auto i = k->iv_len(); + auto n = k->encrypted_size(b.size()); + bytes res(bytes::initialized_later(), n + i); + k->generate_iv(reinterpret_cast(res.data()), i); + n = k->encrypt(reinterpret_cast(b.data()), b.size() + , reinterpret_cast(res.data()) + i, res.size() - i + , reinterpret_cast(res.data())); + res.resize(n + i); + return make_ready_future(std::move(res)); + }); + +} + +future encryption::system_key::decrypt(const bytes& b) { + return get_key().then([b](shared_ptr k) { + auto i = k->iv_len(); + bytes res(bytes::initialized_later(), b.size() - i); + auto n = k->decrypt(reinterpret_cast(b.data()) + i, + b.size() - i, reinterpret_cast(res.data()), + res.size(), reinterpret_cast(b.data())); + res.resize(n); + return make_ready_future(std::move(res)); + }); +} + diff --git a/ent/encryption/system_key.hh b/ent/encryption/system_key.hh new file mode 100644 index 000000000000..8d6ac32b2609 --- /dev/null +++ b/ent/encryption/system_key.hh @@ -0,0 +1,34 @@ +/* + * Copyright (C) 2015 ScyllaDB + * + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "encryption.hh" +#include "../../bytes.hh" + +namespace encryption { + +class symmetric_key; + +class system_key { +public: + virtual ~system_key() {} + virtual future> get_key() = 0; + virtual const sstring& name() const = 0; + virtual bool is_local() const = 0; + virtual future<> validate() const; + + future encrypt(const sstring&); + future decrypt(const sstring&); + future encrypt(const bytes&); + future decrypt(const bytes&); +}; + +} + From ee62b61c84ff9a43006daaba06261d1d145966fe Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 16:03:06 +0000 Subject: [PATCH 228/397] tmpdir: shorten test tempdir path To make certain python tests work in CI --- test/lib/tmpdir.cc | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/test/lib/tmpdir.cc b/test/lib/tmpdir.cc index 738dcdf90fd1..fce9519dcb01 100644 --- a/test/lib/tmpdir.cc +++ b/test/lib/tmpdir.cc @@ -31,9 +31,27 @@ tmpdir::sweeper::~sweeper() { } } -tmpdir::tmpdir() - : _path(fs::temp_directory_path() / fs::path(fmt::format(FMT_STRING("scylla-{}"), utils::make_random_uuid()))) { - fs::create_directories(_path); +tmpdir::tmpdir() { + auto tmp = fs::temp_directory_path(); + for (;;) { + // Reduce the path length of the created tmp dir. This might seem + // silly when running with base TMPDIR=/tmp or similar, but + // in a lot of CI testing, TMPDIR will be a loooooong path into + // jenkins workdirs or similar -> this path will be 100+ chars long. + // Again, this should most often not be a problem, _but_ if we + // for example run something like a sub process of a python server, + // which will try to create various unix sockets et al for its + // operations, the TMPDIR base for this must not exceed 107 chars. + // Note: converting UUID to string first, because for some reason + // our UUID formatter does not respect width/precision. Feel free to + // change once it does. + _path = tmp / fmt::format("scylla-{:.8}", fmt::to_string(utils::make_random_uuid())); + // Note: this is a slight improvement also, in that we ensure the dir + // we use is actually created by us. + if (fs::create_directories(_path)) { + break; + } + } } tmpdir::tmpdir(tmpdir&& other) noexcept : _path(std::exchange(other._path, {})) {} From c596ae6eb1ab881f80fa93a5fad1e79065a5b3c4 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 11:50:16 +0000 Subject: [PATCH 229/397] tests: Add EAR tests Adds the migrated EAR/encryption tests. Note: Until scylla CI is updated to provide all the proper ENV vars, some tests will not execute. --- configure.py | 3 + test/boost/CMakeLists.txt | 9 + test/boost/encrypted_file_test.cc | 265 ++++++ test/boost/encryption_at_rest_test.cc | 1098 +++++++++++++++++++++++++ test/boost/kmip_wrapper.py | 95 +++ test/boost/symmetric_key_test.cc | 221 +++++ test/resource/certs/cacert.pem | 82 ++ test/resource/certs/scylla.pem | 57 ++ 8 files changed, 1830 insertions(+) create mode 100644 test/boost/encrypted_file_test.cc create mode 100644 test/boost/encryption_at_rest_test.cc create mode 100644 test/boost/kmip_wrapper.py create mode 100644 test/boost/symmetric_key_test.cc create mode 100644 test/resource/certs/cacert.pem create mode 100644 test/resource/certs/scylla.pem diff --git a/configure.py b/configure.py index 5e930ff6db22..35c1f10039b9 100755 --- a/configure.py +++ b/configure.py @@ -475,6 +475,8 @@ def find_ninja(): 'test/boost/double_decker_test', 'test/boost/duration_test', 'test/boost/dynamic_bitset_test', + 'test/boost/encrypted_file_test', + 'test/boost/encryption_at_rest_test', 'test/boost/enum_option_test', 'test/boost/enum_set_test', 'test/boost/estimated_histogram_test', @@ -554,6 +556,7 @@ def find_ninja(): 'test/boost/token_metadata_test', 'test/boost/top_k_test', 'test/boost/transport_test', + 'test/boost/symmetric_key_test', 'test/boost/types_test', 'test/boost/utf8_test', 'test/boost/vint_serialization_test', diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index 481518803008..1d95d935a34d 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -80,6 +80,15 @@ add_scylla_test(duration_test add_scylla_test(dynamic_bitset_test KIND BOOST LIBRARIES utils) +add_scylla_test(encrypted_file_test + KIND SEASTAR + LIBRARIES + encryption) +add_scylla_test(encryption_at_rest_test + KIND SEASTAR + LIBRARIES + Boost::filesystem + encryption) add_scylla_test(enum_option_test KIND BOOST) add_scylla_test(enum_set_test diff --git a/test/boost/encrypted_file_test.cc b/test/boost/encrypted_file_test.cc new file mode 100644 index 000000000000..62c6042e3f76 --- /dev/null +++ b/test/boost/encrypted_file_test.cc @@ -0,0 +1,265 @@ +/* + * Copyright (C) 2016 ScyllaDB + */ + + + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +#include "ent/encryption/encryption.hh" +#include "ent/encryption/symmetric_key.hh" +#include "ent/encryption/encrypted_file_impl.hh" +#include "test/lib/tmpdir.hh" +#include "test/lib/random_utils.hh" +#include "test/lib/exception_utils.hh" + +using namespace encryption; + +static tmpdir dir; + +static future>> make_file(const sstring& name, open_flags mode, ::shared_ptr k = nullptr) { + file f = co_await open_file_dma(sstring(dir.path() / std::string(name)), mode); + if (k == nullptr) { + key_info info{"AES/CBC", 256}; + k = ::make_shared(info); + } + co_return std::tuple(file(make_encrypted_file(f, k)), k); +} + +static temporary_buffer generate_random(size_t n, size_t align) { + auto tmp = temporary_buffer::aligned(align, align_up(n, align)); + auto data = tests::random::get_sstring(n); + std::copy(data.begin(), data.end(), tmp.get_write()); + return tmp; +} + +static future<> test_random_data_disk(size_t n) { + auto name = "test_rand_" + std::to_string(n); + auto t = co_await make_file(name, open_flags::rw|open_flags::create); + auto f = std::get<0>(t); + std::exception_ptr ex = nullptr; + + try { + auto k = std::get<1>(t); + auto a = f.memory_dma_alignment(); + auto buf = generate_random(n, a); + auto w = co_await f.dma_write(0, buf.get(), buf.size()); + + co_await f.flush(); + if (n != buf.size()) { + co_await f.truncate(n); + } + + BOOST_REQUIRE_EQUAL(w, buf.size()); + + auto k2 = ::make_shared(k->info(), k->key()); + auto f2 = std::get<0>(co_await make_file(name, open_flags::ro, k2)); + + auto tmp = temporary_buffer::aligned(a, buf.size()); + auto n2 = co_await f2.dma_read(0, tmp.get_write(), tmp.size()); + + BOOST_REQUIRE_EQUAL(n2, n); + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp.get(), tmp.get() + n2, buf.get(), buf.get() + n2); + } catch (...) { + ex = std::current_exception(); + } + + co_await f.close(); + if (ex) { + std::rethrow_exception(ex); + } +} + +static void test_random_data(size_t n) { + auto buf = generate_random(n, 8); + + + // first, verify padded. + { + key_info info{"AES/CBC/PKCSPadding", 256}; + auto k = ::make_shared(info); + + bytes b(bytes::initialized_later(), k->iv_len()); + k->generate_iv(b.data(), k->iv_len()); + + temporary_buffer tmp(n + k->block_size()); + k->encrypt(buf.get(), buf.size(), tmp.get_write(), tmp.size(), b.data()); + + auto bytes = k->key(); + auto k2 = ::make_shared(info, bytes); + + temporary_buffer tmp2(n + k->block_size()); + k2->decrypt(tmp.get(), tmp.size(), tmp2.get_write(), tmp2.size(), b.data()); + + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp2.get(), tmp2.get() + n, buf.get(), buf.get() + n); + } + + // unpadded + { + key_info info{"AES/CBC", 256}; + auto k = ::make_shared(info); + + bytes b(bytes::initialized_later(), k->iv_len()); + k->generate_iv(b.data(), k->iv_len()); + + temporary_buffer tmp(n); + k->encrypt_unpadded(buf.get(), buf.size(), tmp.get_write(), b.data()); + + auto bytes = k->key(); + auto k2 = ::make_shared(info, bytes); + + temporary_buffer tmp2(buf.size()); + k2->decrypt_unpadded(tmp.get(), tmp.size(), tmp2.get_write(), b.data()); + + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp2.get(), tmp2.get() + n, buf.get(), buf.get() + n); + } +} + + +BOOST_AUTO_TEST_CASE(test_encrypting_data_128) { + test_random_data(128); +} + +BOOST_AUTO_TEST_CASE(test_encrypting_data_4k) { + test_random_data(4*1024); +} + + +SEASTAR_TEST_CASE(test_encrypted_file_data_4k) { + return test_random_data_disk(4*1024); +} + +SEASTAR_TEST_CASE(test_encrypted_file_data_16k) { + return test_random_data_disk(16*1024); +} + +SEASTAR_TEST_CASE(test_encrypted_file_data_unaligned) { + return test_random_data_disk(16*1024 - 3); +} + +SEASTAR_TEST_CASE(test_encrypted_file_data_unaligned2) { + return test_random_data_disk(16*1024 - 4092); +} + +SEASTAR_TEST_CASE(test_short) { + auto name = "test_short"; + file f = co_await open_file_dma(sstring(dir.path() / name), open_flags::rw|open_flags::create); + co_await f.truncate(1); + co_await f.close(); + + auto t = co_await make_file(name, open_flags::ro); + f = std::get<0>(t); + std::exception_ptr ex = nullptr; + + try { + temporary_buffer buf(f.memory_dma_alignment()); + + BOOST_REQUIRE_EXCEPTION( + co_await f.dma_read(0, buf.get_write(), buf.size()), + std::domain_error, + exception_predicate::message_contains("file size 1, expected 0 or at least 16") + ); + } catch (...) { + ex = std::current_exception(); + } + + co_await f.close(); + if (ex) { + std::rethrow_exception(ex); + } +} + +SEASTAR_TEST_CASE(test_truncating_empty) { + auto name = "test_truncating_empty"; + auto t = co_await make_file(name, open_flags::rw|open_flags::create); + auto f = std::get<0>(t); + auto k = std::get<1>(t); + auto s = 64 * f.memory_dma_alignment(); + + co_await f.truncate(s); + + temporary_buffer buf(s); + auto n = co_await f.dma_read(0, buf.get_write(), buf.size()); + + co_await f.close(); + + BOOST_REQUIRE_EQUAL(s, n); + + for (auto c : buf) { + BOOST_REQUIRE_EQUAL(c, 0); + } +} + +SEASTAR_TEST_CASE(test_truncating_extend) { + auto name = "test_truncating_extend"; + auto t = co_await make_file(name, open_flags::rw|open_flags::create); + auto f = std::get<0>(t); + auto k = std::get<1>(t); + auto a = f.memory_dma_alignment(); + auto s = 32 * a; + auto buf = generate_random(s, a); + auto w = co_await f.dma_write(0, buf.get(), buf.size()); + + co_await f.flush(); + BOOST_REQUIRE_EQUAL(s, w); + + for (size_t i = 1; i < 64; ++i) { + // truncate smaller, unaligned + auto l = w - i; + auto r = w + 8 * a; + co_await f.truncate(l); + BOOST_REQUIRE_EQUAL(l, (co_await f.stat()).st_size); + + { + auto tmp = temporary_buffer::aligned(a, align_up(l, a)); + auto n = co_await f.dma_read(0, tmp.get_write(), tmp.size()); + + BOOST_REQUIRE_EQUAL(l, n); + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp.get(), tmp.get() + l, buf.get(), buf.get() + l); + + auto k = align_down(l, a); + + while (k > 0) { + n = co_await f.dma_read(0, tmp.get_write(), k); + + BOOST_REQUIRE_EQUAL(k, n); + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp.get(), tmp.get() + k, buf.get(), buf.get() + k); + + n = co_await f.dma_read(k, tmp.get_write(), tmp.size()); + BOOST_REQUIRE_EQUAL(l - k, n); + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp.get(), tmp.get() + n, buf.get() + k, buf.get() + k + n); + + k -= a; + } + } + + co_await f.truncate(r); + BOOST_REQUIRE_EQUAL(r, (co_await f.stat()).st_size); + + auto tmp = temporary_buffer::aligned(a, align_up(r, a)); + auto n = co_await f.dma_read(0, tmp.get_write(), tmp.size()); + + BOOST_REQUIRE_EQUAL(r, n); + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp.get(), tmp.get() + l, buf.get(), buf.get() + l); + + while (l < r) { + BOOST_REQUIRE_EQUAL(tmp[l], 0); + ++l; + } + } + + co_await f.close(); +} + diff --git a/test/boost/encryption_at_rest_test.cc b/test/boost/encryption_at_rest_test.cc new file mode 100644 index 000000000000..acf06d3564b8 --- /dev/null +++ b/test/boost/encryption_at_rest_test.cc @@ -0,0 +1,1098 @@ +/* + * Copyright (C) 2016 ScyllaDB + */ + + + +#include +#include +#include +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include "ent/encryption/encryption.hh" +#include "ent/encryption/symmetric_key.hh" +#include "ent/encryption/local_file_provider.hh" +#include "test/lib/tmpdir.hh" +#include "test/lib/random_utils.hh" +#include "test/lib/cql_test_env.hh" +#include "test/lib/cql_assertions.hh" +#include "db/config.hh" +#include "db/extensions.hh" +#include "db/commitlog/commitlog.hh" +#include "db/commitlog/commitlog_replayer.hh" +#include "init.hh" +#include "sstables/sstables.hh" +#include "cql3/untyped_result_set.hh" +#include "utils/rjson.hh" +#include "replica/database.hh" +#include "service/client_state.hh" + +using namespace encryption; +namespace fs = std::filesystem; + +using test_hook = std::function; + +struct test_provider_args { + const tmpdir& tmp; + std::string options; + std::string extra_yaml = {}; + unsigned n_tables = 1; + unsigned n_restarts = 1; + std::string explicit_provider = {}; + + test_hook before_create_table; + test_hook after_create_table; + test_hook on_insert_exception; + + std::optional timeout; +}; + +static void do_create_and_insert(cql_test_env& env, const test_provider_args& args, const std::string& pk, const std::string& v) { + for (auto i = 0u; i < args.n_tables; ++i) { + if (args.before_create_table) { + args.before_create_table(env); + } + if (args.options.empty()) { + env.execute_cql(fmt::format("create table t{} (pk text primary key, v text)", i)).get(); + } else { + env.execute_cql(fmt::format("create table t{} (pk text primary key, v text) WITH scylla_encryption_options={{{}}}", i, args.options)).get(); + } + + if (args.after_create_table) { + args.after_create_table(env); + } + try { + env.execute_cql(fmt::format("insert into ks.t{} (pk, v) values ('{}', '{}')", i, pk, v)).get(); + } catch (...) { + args.on_insert_exception(env); + throw; + } + } +} + +static future<> test_provider(const test_provider_args& args) { + auto make_config = [&] { + auto ext = std::make_shared(); + auto cfg = seastar::make_shared(ext); + cfg->data_file_directories({args.tmp.path().string()}); + + // Currently the test fails with consistent_cluster_management = true. See #2995. + cfg->consistent_cluster_management(false); + + if (!args.extra_yaml.empty()) { + boost::program_options::options_description desc; + boost::program_options::options_description_easy_init init(&desc); + configurable::append_all(*cfg, init); + cfg->read_from_yaml(args.extra_yaml); + } + + return std::make_tuple(cfg, ext); + }; + + std::string pk = "apa"; + std::string v = "ko"; + + { + auto [cfg, ext] = make_config(); + + co_await do_with_cql_env_thread([&] (cql_test_env& env) { + do_create_and_insert(env, args, pk, v); + }, cfg, {}, cql_test_init_configurables{ *ext }); + } + + for (auto rs = 0u; rs < args.n_restarts; ++rs) { + auto [cfg, ext] = make_config(); + + co_await do_with_cql_env_thread([&] (cql_test_env& env) { + for (auto i = 0u; i < args.n_tables; ++i) { + require_rows(env, fmt::format("select * from ks.t{}", i), {{utf8_type->decompose(pk), utf8_type->decompose(v)}}); + + auto provider = args.explicit_provider; + + // check that all sstables have the defined provider class (i.e. are encrypted using correct optons) + if (provider.empty() && args.options.find("'key_provider'") != std::string::npos) { + static std::regex ex(R"foo('key_provider'\s*:\s*'(\w+)')foo"); + + std::smatch m; + BOOST_REQUIRE(std::regex_search(args.options.begin(), args.options.end(), m, ex)); + provider = m[1].str(); + BOOST_REQUIRE(!provider.empty()); + } + if (!provider.empty()) { + env.db().invoke_on_all([&](replica::database& db) { + auto& cf = db.find_column_family("ks", "t" + std::to_string(i)); + auto sstables = cf.get_sstables_including_compacted_undeleted(); + + if (sstables) { + for (auto& t : *sstables) { + auto sst_provider = encryption::encryption_provider(*t); + BOOST_REQUIRE_EQUAL(provider, sst_provider); + } + } + }).get(); + } + } + }, cfg, {}, cql_test_init_configurables{ *ext }); + } +} + +static future<> test_provider(const std::string& options, const tmpdir& tmp, const std::string& extra_yaml = {}, unsigned n_tables = 1, unsigned n_restarts = 1, const std::string& explicit_provider = {}) { + test_provider_args args{ + .tmp = tmp, + .options = options, + .extra_yaml = extra_yaml, + .n_tables = n_tables, + .n_restarts = n_restarts, + .explicit_provider = explicit_provider + }; + co_await test_provider(args); +} + +SEASTAR_TEST_CASE(test_local_file_provider) { + tmpdir tmp; + auto keyfile = tmp.path() / "secret_key"; + co_await test_provider(fmt::format("'key_provider': 'LocalFileSystemKeyProviderFactory', 'secret_key_file': '{}', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", keyfile.string()), tmp); +} + +static future<> create_key_file(const fs::path& path, const std::vector& key_types) { + std::ostringstream ss; + + for (auto& info : key_types) { + symmetric_key k(info); + ss << info.alg << ":" << info.len << ":" << base64_encode(k.key()) << std::endl; + } + + auto s = ss.str(); + co_await seastar::recursive_touch_directory(fs::path(path).remove_filename().string()); + co_await write_text_file_fully(path.string(), s); +} + +static future<> do_test_replicated_provider(unsigned n_tables, unsigned n_restarts, const std::string& extra = {}, test_hook hook = {}) { + tmpdir tmp; + auto keyfile = tmp.path() / "secret_key"; + auto sysdir = tmp.path() / "system_keys"; + auto syskey = sysdir / "system_key"; + auto yaml = fmt::format("system_key_directory: {}", sysdir.string()); + + co_await create_key_file(syskey, { { "AES/CBC/PKCSPadding", 256 }}); + + BOOST_REQUIRE(fs::exists(syskey));; + + test_provider_args args{ + .tmp = tmp, + .options = fmt::format("'key_provider': 'ReplicatedKeyProviderFactory', 'system_key_file': 'system_key', 'secret_key_file': '{}','cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128{}", keyfile.string(), extra), + .extra_yaml = yaml, + .n_tables = n_tables, + .n_restarts = n_restarts, + .explicit_provider = {}, + .after_create_table = hook + }; + + co_await test_provider(args); + + BOOST_REQUIRE(fs::exists(tmp.path())); +} + +SEASTAR_TEST_CASE(test_replicated_provider) { + co_await do_test_replicated_provider(1, 1); +} + +SEASTAR_TEST_CASE(test_replicated_provider_many_tables) { + co_await do_test_replicated_provider(100, 5); +} + +using namespace std::chrono_literals; + +static const timeout_config rkp_db_timeout_config { + 5s, 5s, 5s, 5s, 5s, 5s, 5s, +}; + +static service::query_state& rkp_db_query_state() { + static thread_local service::client_state cs(service::client_state::internal_tag{}, rkp_db_timeout_config); + static thread_local service::query_state qs(cs, empty_service_permit()); + return qs; +} + +SEASTAR_TEST_CASE(test_replicated_provider_shutdown_failure) { + co_await do_test_replicated_provider(1, 1, ", 'DEBUG': 'nocache,novalidate'", [](cql_test_env& env) { + /** + * Try to remove all keys in replicated table. Note: we can't use truncate because we + * are not running any proper remotes. + */ + auto res = env.local_qp().execute_internal("select * from system_replicated_keys.encrypted_keys", + db::consistency_level::ONE, rkp_db_query_state(), {}, cql3::query_processor::cache_internal::no + ).get(); + for (auto& row : (*res)) { + auto key_file = row.get_as("key_file"); + auto cipher = row.get_as("cipher"); + auto strength = row.get_as("strength"); + auto uuid = row.get_as("key_id"); + + env.local_qp().execute_internal("delete from system_replicated_keys.encrypted_keys where key_file=? AND cipher=? AND strength=? AND key_id=?", + db::consistency_level::ONE, rkp_db_query_state(), + { key_file, cipher, strength, uuid }, + cql3::query_processor::cache_internal::no + ).get(); + } + }); +} + +static std::string get_var_or_default(const char* var, std::string_view def, bool* set) { + const char* val = std::getenv(var); + if (val == nullptr) { + *set = false; + return std::string(def); + } + *set = true; + return val; +} + +static std::string get_var_or_default(const char* var, std::string_view def) { + bool dummy; + return get_var_or_default(var, def, &dummy); +} + +static bool check_run_test(const char* var, bool defval = false) { + auto do_test = get_var_or_default(var, std::to_string(defval)); + + if (!strcasecmp(do_test.data(), "0") || !strcasecmp(do_test.data(), "false")) { + BOOST_TEST_MESSAGE(fmt::format("Skipping test. Set {}=1 to run", var)); + return false; + } + return true; +} + +static auto check_run_test_decorator(const char* var, bool def = false) { + return boost::unit_test::precondition(std::bind(&check_run_test, var, def)); +} + +#ifdef HAVE_KMIP + +struct kmip_test_info { + std::string host; + std::string cert; + std::string key; + std::string ca; + std::string prio; +}; + +namespace bp = boost::process; + +static future<> kmip_test_helper(const std::function(const kmip_test_info&, const tmpdir&)>& f) { + tmpdir tmp; + bool host_set = false; + bp::child python; + bp::group gp; + bp::ipstream is; + + std::future pykmip_status; + + static const char* def_resourcedir = "./test/resource/certs"; + const char* resourcedir = std::getenv("KMIP_RESOURCE_DIR"); + if (resourcedir == nullptr) { + resourcedir = def_resourcedir; + } + + kmip_test_info info { + .host = get_var_or_default("KMIP_HOST", "127.0.0.1", &host_set), + .cert = get_var_or_default("KMIP_CERT", fmt::format("{}/scylla.pem", resourcedir)), + .key = get_var_or_default("KMIP_KEY", fmt::format("{}/scylla.pem", resourcedir)), + .ca = get_var_or_default("KMIP_CA", fmt::format("{}/cacert.pem", resourcedir)), + .prio = get_var_or_default("KMIP_PRIO", "SECURE128:+RSA:-VERS-TLS1.0:-ECDHE-ECDSA") + }; + + auto cleanup = defer([&] { + if (python.running()) { + BOOST_TEST_MESSAGE("Stopping PyKMIP server"); // debug print. Why not. + gp.terminate(); + pykmip_status.get(); + } + }); + + // note: default kmip port = 5696; + + if (!host_set) { + // Note: we set `enable_tls_client_auth=False` - client cert is still validated, + // but we have note generated certs with "extended usage client OID", which + // pykmip will check for if this is true. + auto cfg = fmt::format(R"foo( +[server] +hostname=127.0.0.1 +port=1 +certificate_path={} +key_path={} +ca_path={} +auth_suite=TLS1.2 +policy_path={} +enable_tls_client_auth=False +logging_level=DEBUG +database_path={}/pykmip.db + )foo", info.cert, info.key, info.ca, tmp.path().string(), tmp.path().string()); + + auto cfgfile = fmt::format("{}/pykmip.conf", tmp.path().string()); + auto log = fmt::format("{}/pykmip.log", tmp.path().string()); + + { + std::ofstream of(cfgfile); + of << cfg; + } + + auto pyexec = bp::search_path("python"); + + BOOST_TEST_MESSAGE("Starting PyKMIP server"); // debug print. Why not. + + python = bp::child(pyexec, gp, + "test/boost/kmip_wrapper.py", + "-l", log, + "-f", cfgfile, + "-v", "DEBUG", + (bp::std_out & bp::std_err) > is, bp::std_in.close(), + bp::env["TMPDIR"]=tmp.path().string() + ); + + std::promise port_promise; + auto f = port_promise.get_future(); + + pykmip_status = std::async([&] { + static std::regex port_ex("Listening on (\\d+)"); + + std::string line; + bool b = false; + + do { + while (std::getline(is, line)) { + std::cout << line << std::endl; + std::smatch m; + if (!b && std::regex_match(line, m, port_ex)) { + port_promise.set_value(std::stoi(m[1].str())); + b = true; + } + } + } while (python.running()); + + if (!b) { + port_promise.set_value(-1); + } + }); + // arbitrary timeout of 20s for the server to make some output. Very generous. + if (f.wait_for(20s) == std::future_status::timeout) { + throw std::runtime_error("Could not start pykmip"); + } + auto port = f.get(); + if (port <= 0) { + throw std::runtime_error("Invalid port"); + } + // wait for port. + for (;;) { + try { + // TODO: seastar does not have a connect with timeout. That would be helpful here. But alas... + co_await seastar::connect(socket_address(net::inet_address("127.0.0.1"), port)); + BOOST_TEST_MESSAGE("PyKMIP server up and available"); // debug print. Why not. + break; + } catch (...) { + } + co_await sleep(100ms); + } + + info.host = fmt::format("127.0.0.1:{}", port); + } + + co_await f(info, tmp); +} + +SEASTAR_TEST_CASE(test_kmip_provider, *check_run_test_decorator("ENABLE_KMIP_TEST", true)) { + co_await kmip_test_helper([](const kmip_test_info& info, const tmpdir& tmp) -> future<> { + auto yaml = fmt::format(R"foo( + kmip_hosts: + kmip_test: + hosts: {0} + certificate: {1} + keyfile: {2} + truststore: {3} + priority_string: {4} + )foo" + , info.host, info.cert, info.key, info.ca, info.prio + ); + co_await test_provider("'key_provider': 'KmipKeyProviderFactory', 'kmip_host': 'kmip_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml); + }); +} + +#endif // HAVE_KMIP + +class fake_proxy { + seastar::server_socket _socket; + socket_address _address; + bool _go_on = true; + bool _do_proxy = true; + future<> _f; + + future<> run(std::string s) { + uint16_t port = 443u; + auto i = s.find_last_of(':'); + if (i != std::string::npos && i > 0 && s[i - 1] != ':') { // just check against ipv6... + port = std::stoul(s.substr(i + 1)); + s = s.substr(0, i); + } + + auto addr = co_await seastar::net::dns::resolve_name(s); + std::vector> work; + + while (_go_on) { + try { + auto client = co_await _socket.accept(); + auto dst = co_await seastar::connect(socket_address(addr, port)); + + auto f = [&]() -> future<> { + auto& s = client.connection; + auto& ldst = dst; + + auto do_io = [this](connected_socket& src, connected_socket& dst) -> future<> { + auto sin = src.input(); + auto dout = dst.output(); + // note: have to have differing conditions for proxying + // and shutdown, and need to check inside look, because + // kmip connector caches connection -> not new socket. + while (_go_on && _do_proxy && !sin.eof()) { + auto buf = co_await sin.read(); + if (_do_proxy) { + co_await dout.write(std::move(buf)); + co_await dout.flush(); + } + } + co_await dout.close(); + }; + + co_await when_all(do_io(s, ldst), do_io(ldst, s)); + }(); + + work.emplace_back(std::move(f)); + } catch (...) { + } + } + + for (auto&& f : work) { + co_await std::move(f); + } + } +public: + fake_proxy(std::string dst) + : _socket(seastar::listen(socket_address(0x7f000001, 0))) + , _address(_socket.local_address()) + , _f(run(std::move(dst))) + {} + + const socket_address& address() const { + return _address; + } + void enable(bool b) { + _do_proxy = b; + } + future<> stop() { + if (std::exchange(_go_on, false)) { + _socket.abort_accept(); + co_await std::move(_f); + } + } +}; + +#ifdef HAVE_KMIP + +SEASTAR_TEST_CASE(test_kmip_provider_multiple_hosts, *check_run_test_decorator("ENABLE_KMIP_TEST", true)) { + /** + * Tests for #3251. KMIP connector ends up in endless loop if using more than one + * fallover host. This is only in initial connection (in real life only in initial connection verification). + * + * We don't have access to more than one KMIP server for testing (at a time). + * Pretend to have failover by using a local proxy. + */ + co_await kmip_test_helper([](const kmip_test_info& info, const tmpdir& tmp) -> future<> { + fake_proxy proxy(info.host); + + auto host2 = boost::lexical_cast(proxy.address()); + + auto yaml = fmt::format(R"foo( + kmip_hosts: + kmip_test: + hosts: {0}, {5} + certificate: {1} + keyfile: {2} + truststore: {3} + priority_string: {4} + )foo" + , info.host, info.cert, info.key, info.ca, info.prio, host2 + ); + + std::exception_ptr ex; + + try { + co_await test_provider("'key_provider': 'KmipKeyProviderFactory', 'kmip_host': 'kmip_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml); + } catch (...) { + ex = std::current_exception(); + } + + co_await proxy.stop(); + + if (ex) { + std::rethrow_exception(ex); + } + }); +} + +#endif // HAVE_KMIP + +/* +Simple test of KMS provider. Still has some caveats: + + 1.) Uses aws CLI credentials for auth. I.e. you need to have a valid + ~/.aws/credentials for the user running the test. + 2.) I can't figure out a good way to set up a key "everyone" can access. So user needs + to have read/encrypt access to the key alias (default "alias/kms_encryption_test") + in the scylla AWS account. + + A "better" solution might be to create dummmy user only for KMS testing with only access + to a single key, and no other priviledges. But that seems dangerous as well. + + For this reason, this test is parameterized with env vars: + * ENABLE_KMS_TEST - set to non-zero (1/true) to run + * KMS_KEY_ALIAS - default "alias/kms_encryption_test" - set to key alias you have access to. + * KMS_AWS_REGION - default us-east-1 - set to whatever region your key is in. + +*/ +static future<> kms_test_helper(std::function(const tmpdir&, std::string_view, std::string_view, std::string_view)> f) { + auto kms_key_alias = get_var_or_default("KMS_KEY_ALIAS", "alias/kms_encryption_test"); + auto kms_aws_region = get_var_or_default("KMS_AWS_REGION", "us-east-1"); + auto kms_aws_profile = get_var_or_default("KMS_AWS_PROFILE", "default"); + + tmpdir tmp; + + co_await f(tmp, kms_key_alias, kms_aws_region, kms_aws_profile); +} + +SEASTAR_TEST_CASE(test_kms_provider, *check_run_test_decorator("ENABLE_KMS_TEST")) { + co_await kms_test_helper([](const tmpdir& tmp, std::string_view kms_key_alias, std::string_view kms_aws_region, std::string_view kms_aws_profile) -> future<> { + /** + * Note: NOT including any auth stuff here. The provider will pick up AWS credentials + * from ~/.aws/credentials + */ + auto yaml = fmt::format(R"foo( + kms_hosts: + kms_test: + master_key: {0} + aws_region: {1} + aws_profile: {2} + )foo" + , kms_key_alias, kms_aws_region, kms_aws_profile + ); + + co_await test_provider("'key_provider': 'KmsKeyProviderFactory', 'kms_host': 'kms_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml); + }); +} + +SEASTAR_TEST_CASE(test_kms_provider_with_master_key_in_cf, *check_run_test_decorator("ENABLE_KMS_TEST")) { + co_await kms_test_helper([](const tmpdir& tmp, std::string_view kms_key_alias, std::string_view kms_aws_region, std::string_view kms_aws_profile) -> future<> { + /** + * Note: NOT including any auth stuff here. The provider will pick up AWS credentials + * from ~/.aws/credentials + */ + auto yaml = fmt::format(R"foo( + kms_hosts: + kms_test: + aws_region: {1} + aws_profile: {2} + )foo" + , kms_key_alias, kms_aws_region, kms_aws_profile + ); + + // should fail + BOOST_REQUIRE_THROW( + co_await test_provider("'key_provider': 'KmsKeyProviderFactory', 'kms_host': 'kms_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml) + , std::exception + ); + + // should be ok + co_await test_provider(fmt::format("'key_provider': 'KmsKeyProviderFactory', 'kms_host': 'kms_test', 'master_key': '{}', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", kms_key_alias) + , tmp, yaml + ); + }); +} + + +SEASTAR_TEST_CASE(test_user_info_encryption) { + tmpdir tmp; + auto keyfile = tmp.path() / "secret_key"; + + auto yaml = fmt::format(R"foo( + user_info_encryption: + enabled: True + key_provider: LocalFileSystemKeyProviderFactory + secret_key_file: {} + cipher_algorithm: AES/CBC/PKCS5Padding + secret_key_strength: 128 + )foo" + , keyfile.string()); + + co_await test_provider({}, tmp, yaml, 4, 1, "LocalFileSystemKeyProviderFactory" /* verify encrypted even though no kp in options*/); +} + +SEASTAR_TEST_CASE(test_kms_provider_with_broken_algo, *check_run_test_decorator("ENABLE_KMS_TEST")) { + co_await kms_test_helper([](const tmpdir& tmp, std::string_view kms_key_alias, std::string_view kms_aws_region, std::string_view kms_aws_profile) -> future<> { + /** + * Note: NOT including any auth stuff here. The provider will pick up AWS credentials + * from ~/.aws/credentials + */ + auto yaml = fmt::format(R"foo( + kms_hosts: + kms_test: + master_key: {0} + aws_region: {1} + aws_profile: {2} + )foo" + , kms_key_alias, kms_aws_region, kms_aws_profile + ); + + try { + co_await test_provider("'key_provider': 'KmsKeyProviderFactory', 'kms_host': 'kms_test', 'cipher_algorithm':'', 'secret_key_strength': 128", tmp, yaml); + BOOST_FAIL("should not reach"); + } catch (exceptions::configuration_exception&) { + // ok + } + }); +} + +static auto make_commitlog_config(const test_provider_args& args, const std::unordered_map& scopts) { + auto ext = std::make_shared(); + auto cfg = seastar::make_shared(ext); + cfg->data_file_directories({args.tmp.path().string()}); + cfg->commitlog_sync("batch"); // just to make sure files are written + + // Currently the test fails with consistent_cluster_management = true. See #2995. + cfg->consistent_cluster_management(false); + + boost::program_options::options_description desc; + boost::program_options::options_description_easy_init init(&desc); + configurable::append_all(*cfg, init); + + std::ostringstream ss; + ss << "system_info_encryption:" << std::endl + << " enabled: true" << std::endl + << " cipher_algorithm: AES/CBC/PKCS5Padding" << std::endl + << " secret_key_strength: 128" << std::endl + ; + + for (auto& [k, v] : scopts) { + ss << " " << k << ": " << v << std::endl; + } + auto str = ss.str(); + cfg->read_from_yaml(str); + + if (!args.extra_yaml.empty()) { + cfg->read_from_yaml(args.extra_yaml); + } + + return std::make_tuple(cfg, ext); +} + +static future<> test_encrypted_commitlog(const test_provider_args& args, std::unordered_map scopts = {}) { + fs::path clback = args.tmp.path() / "commitlog_back"; + + std::string pk = "apa"; + std::string v = "ko"; + + + { + auto [cfg, ext] = make_commitlog_config(args, scopts); + + cql_test_config cqlcfg(cfg); + + if (args.timeout) { + cqlcfg.query_timeout = args.timeout; + } + + co_await do_with_cql_env_thread([&] (cql_test_env& env) { + do_create_and_insert(env, args, pk, v); + fs::copy(fs::path(cfg->commitlog_directory()), clback); + }, cqlcfg, {}, cql_test_init_configurables{ *ext }); + + } + + { + auto [cfg, ext] = make_commitlog_config(args, scopts); + + cql_test_config cqlcfg(cfg); + + if (args.timeout) { + cqlcfg.query_timeout = args.timeout; + } + + co_await do_with_cql_env_thread([&] (cql_test_env& env) { + // Fake commitlog replay using the files copied. + std::vector paths; + for (auto const& dir_entry : fs::directory_iterator{clback}) { + auto p = dir_entry.path(); + try { + db::commitlog::descriptor d(p); + paths.emplace_back(std::move(p)); + } catch (...) { + } + } + + BOOST_REQUIRE(!paths.empty()); + + auto rp = db::commitlog_replayer::create_replayer(env.db(), env.get_system_keyspace()).get(); + rp.recover(paths, db::commitlog::descriptor::FILENAME_PREFIX).get(); + + // not really checking anything, but make sure we did not break anything. + for (auto i = 0u; i < args.n_tables; ++i) { + require_rows(env, fmt::format("select * from ks.t{}", i), {{utf8_type->decompose(pk), utf8_type->decompose(v)}}); + } + }, cqlcfg, {}, cql_test_init_configurables{ *ext }); + } +} + +static future<> test_encrypted_commitlog(const tmpdir& tmp, std::unordered_map scopts = {}, const std::string& extra_yaml = {}, unsigned n_tables = 1) { + test_provider_args args{ + .tmp = tmp, + .extra_yaml = extra_yaml, + .n_tables = n_tables, + }; + + co_await test_encrypted_commitlog(args, std::move(scopts)); +} + +SEASTAR_TEST_CASE(test_commitlog_kms_encryption_with_slow_key_resolve, *check_run_test_decorator("ENABLE_KMS_TEST")) { + co_await kms_test_helper([](const tmpdir& tmp, std::string_view kms_key_alias, std::string_view kms_aws_region, std::string_view kms_aws_profile) -> future<> { + /** + * Note: NOT including any auth stuff here. The provider will pick up AWS credentials + * from ~/.aws/credentials + */ + auto yaml = fmt::format(R"foo( + kms_hosts: + kms_test: + master_key: {0} + aws_region: {1} + aws_profile: {2} + )foo" + , kms_key_alias, kms_aws_region, kms_aws_profile + ); + + co_await test_encrypted_commitlog(tmp, { { "key_provider", "KmsKeyProviderFactory" }, { "kms_host", "kms_test" } }, yaml); + }); +} + +#ifdef HAVE_KMIP + +SEASTAR_TEST_CASE(test_commitlog_kmip_encryption_with_slow_key_resolve, *check_run_test_decorator("ENABLE_KMIP_TEST")) { + co_await kmip_test_helper([](const kmip_test_info& info, const tmpdir& tmp) -> future<> { + auto yaml = fmt::format(R"foo( + kmip_hosts: + kmip_test: + hosts: {0} + certificate: {1} + keyfile: {2} + truststore: {3} + priority_string: {4} + )foo" + , info.host, info.cert, info.key, info.ca, info.prio + ); + co_await test_encrypted_commitlog(tmp, { { "key_provider", "KmipKeyProviderFactory" }, { "kmip_host", "kmip_test" } }, yaml); + }); +} + +#endif // HAVE_KMIP + +SEASTAR_TEST_CASE(test_user_info_encryption_dont_allow_per_table_encryption) { + tmpdir tmp; + auto keyfile = tmp.path() / "secret_key"; + + auto yaml = fmt::format(R"foo( + allow_per_table_encryption: false + user_info_encryption: + enabled: True + key_provider: LocalFileSystemKeyProviderFactory + secret_key_file: {} + cipher_algorithm: AES/CBC/PKCS5Padding + secret_key_strength: 128 + )foo" + , keyfile.string()); + + try { + co_await test_provider( + fmt::format("'key_provider': 'LocalFileSystemKeyProviderFactory', 'secret_key_file': '{}', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", keyfile.string()) + , tmp, yaml, 4, 1 + ); + BOOST_FAIL("Should not reach"); + } catch (std::invalid_argument&) { + // Ok. + } +} + +/* + Simple test of GCP cloudkms provider. Uses scylladb GCP project "scylla-kms-test" and keys therein. + + Note: the above text blobs are service account credentials, including private keys. + _Never_ give any real priviledges to these accounts, as we are obviously exposing them here. + + User1 is assumed to have permissions to encrypt/decrypt using the given key + User2 is assumed to _not_ have permissions to encrypt/decrypt using the given key, but permission to + impersonate User1. + + This test is parameterized with env vars: + * ENABLE_GCP_TEST - set to non-zero (1/true) to run + * GCP_USER_1_CREDENTIALS - set to credentials file for user1 + * GCP_USER_2_CREDENTIALS - set to credentials file for user2 + * GCP_KEY_NAME - set to / to override. + * GCP_PROJECT_ID - set to test project + * GCP_LOCATION - set to test location +*/ + +struct gcp_test_env { + std::string key_name; + std::string location; + std::string project_id; + std::string user_1_creds; + std::string user_2_creds; +}; + +static future<> gcp_test_helper(std::function(const tmpdir&, const gcp_test_env&)> f) { + gcp_test_env env { + .key_name = get_var_or_default("GCP_KEY_NAME", "test_ring/test_key"), + .location = get_var_or_default("GCP_LOCATION", "global"), + .project_id = get_var_or_default("GCP_PROJECT_ID", "scylla-kms-test"), + .user_1_creds = get_var_or_default("GCP_USER_1_CREDENTIALS", ""), + .user_2_creds = get_var_or_default("GCP_USER_2_CREDENTIALS", ""), + }; + + tmpdir tmp; + + if (env.user_1_creds.empty()) { + BOOST_ERROR("No 'GCP_USER_1_CREDENTIALS' provided"); + } + if (env.user_2_creds.empty()) { + BOOST_ERROR("No 'GCP_USER_2_CREDENTIALS' provided"); + } + + co_await f(tmp, env); +} + +SEASTAR_TEST_CASE(test_gcp_provider, *check_run_test_decorator("ENABLE_GCP_TEST")) { + co_await gcp_test_helper([](const tmpdir& tmp, const gcp_test_env& gcp) -> future<> { + auto yaml = fmt::format(R"foo( + gcp_hosts: + gcp_test: + master_key: {0} + gcp_project_id: {1} + gcp_location: {2} + gcp_credentials_file: {3} + )foo" + , gcp.key_name, gcp.project_id, gcp.location, gcp.user_1_creds + ); + + co_await test_provider("'key_provider': 'GcpKeyProviderFactory', 'gcp_host': 'gcp_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml); + }); +} + +SEASTAR_TEST_CASE(test_gcp_provider_with_master_key_in_cf, *check_run_test_decorator("ENABLE_GCP_TEST")) { + co_await gcp_test_helper([](const tmpdir& tmp, const gcp_test_env& gcp) -> future<> { + auto yaml = fmt::format(R"foo( + gcp_hosts: + gcp_test: + gcp_project_id: {1} + gcp_location: {2} + gcp_credentials_file: {3} + )foo" + , gcp.key_name, gcp.project_id, gcp.location, gcp.user_1_creds + ); + + // should fail + BOOST_REQUIRE_THROW( + co_await test_provider("'key_provider': 'GcpKeyProviderFactory', 'gcp_host': 'gcp_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml) + , std::exception + ); + + // should be ok + co_await test_provider(fmt::format("'key_provider': 'GcpKeyProviderFactory', 'gcp_host': 'gcp_test', 'master_key': '{}', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", gcp.key_name) + , tmp, yaml + ); + }); +} + +/** + * Verify that trying to access key materials with a user w/o permissions to encrypt/decrypt using cloudkms + * fails. +*/ +SEASTAR_TEST_CASE(test_gcp_provider_with_invalid_user, *check_run_test_decorator("ENABLE_GCP_TEST")) { + co_await gcp_test_helper([](const tmpdir& tmp, const gcp_test_env& gcp) -> future<> { + auto yaml = fmt::format(R"foo( + gcp_hosts: + gcp_test: + master_key: {0} + gcp_project_id: {1} + gcp_location: {2} + gcp_credentials_file: {3} + )foo" + , gcp.key_name, gcp.project_id, gcp.location, gcp.user_2_creds + ); + + // should fail + BOOST_REQUIRE_THROW( + co_await test_provider("'key_provider': 'GcpKeyProviderFactory', 'gcp_host': 'gcp_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml) + , std::exception + ); + }); +} + +/** + * Verify that impersonation of an allowed service account works. User1 can encrypt, but we run + * as User2. However, impersonating user1 will allow us do it ourselves. +*/ +SEASTAR_TEST_CASE(test_gcp_provider_with_impersonated_user, *check_run_test_decorator("ENABLE_GCP_TEST")) { + co_await gcp_test_helper([](const tmpdir& tmp, const gcp_test_env& gcp) -> future<> { + auto buf = co_await read_text_file_fully(sstring(gcp.user_1_creds)); + auto json = rjson::parse(std::string_view(buf.begin(), buf.end())); + auto user1 = rjson::get(json, "client_email"); + + auto yaml = fmt::format(R"foo( + gcp_hosts: + gcp_test: + master_key: {0} + gcp_project_id: {1} + gcp_location: {2} + gcp_credentials_file: {3} + gcp_impersonate_service_account: {4} + )foo" + , gcp.key_name, gcp.project_id, gcp.location, gcp.user_2_creds, user1 + ); + + co_await test_provider("'key_provider': 'GcpKeyProviderFactory', 'gcp_host': 'gcp_test', 'cipher_algorithm':'AES/CBC/PKCS5Padding', 'secret_key_strength': 128", tmp, yaml); + }); +} + +std::string make_aws_host(std::string_view aws_region, std::string_view service); + +using scopts_map = std::unordered_map; + +static future<> test_broken_encrypted_commitlog(const test_provider_args& args, scopts_map scopts = {}) { + std::string pk = "apa"; + std::string v = "ko"; + + { + auto [cfg, ext] = make_commitlog_config(args, scopts); + + cql_test_config cqlcfg(cfg); + + if (args.timeout) { + cqlcfg.query_timeout = args.timeout; + } + + co_await do_with_cql_env_thread([&] (cql_test_env& env) { + do_create_and_insert(env, args, pk, v); + }, cqlcfg, {}, cql_test_init_configurables{ *ext }); + } +} + +/** + * Tests that a network error in key resolution (in commitlog in this case) results in a non-fatal, non-isolating + * exception, i.e. an eventual write error. + */ +static future<> network_error_test_helper(const tmpdir& tmp, const std::string& host, std::function(const fake_proxy&)> make_opts) { + fake_proxy proxy(host); + + auto [scopts, yaml] = make_opts(proxy); + + test_provider_args args{ + .tmp = tmp, + .extra_yaml = yaml, + .n_tables = 10, + .before_create_table = [&](auto& env) { + // turn off proxy. all key resolution after this should fail + proxy.enable(false); + // wait for key cache expiry. + seastar::sleep(10ms).get(); + // ensure commitlog will create a new segment on write -> eventual write failure + env.db().invoke_on_all([](replica::database& db) { + return db.commitlog()->force_new_active_segment(); + }).get(); + }, + .on_insert_exception = [&](auto&&) { + // once we get the exception we have to enable key resolution again, + // otherwise we can't shut down cql test env. + proxy.enable(true); + }, + .timeout = timeout_config{ + // set really low write timeouts so we get a failure (timeout) + // when we fail to write to commitlog + 100ms, 100ms, 100ms, 100ms, 100ms, 100ms, 100ms + }, + }; + + BOOST_REQUIRE_THROW( + co_await test_broken_encrypted_commitlog(args, scopts); + , std::exception + ); + + co_await proxy.stop(); +} + +SEASTAR_TEST_CASE(test_kms_network_error, *check_run_test_decorator("ENABLE_KMS_TEST")) { + co_await kms_test_helper([](const tmpdir& tmp, std::string_view kms_key_alias, std::string_view kms_aws_region, std::string_view kms_aws_profile) -> future<> { + auto host = make_aws_host(kms_aws_region, "kms"); + + co_await network_error_test_helper(tmp, host, [&](const auto& proxy) { + auto yaml = fmt::format(R"foo( + kms_hosts: + kms_test: + master_key: {0} + aws_region: {1} + aws_profile: {2} + endpoint: https://{3} + key_cache_expiry: 1ms + )foo" + , kms_key_alias, kms_aws_region, kms_aws_profile, proxy.address() + ); + return std::make_tuple(scopts_map({ { "key_provider", "KmsKeyProviderFactory" }, { "kms_host", "kms_test" } }), yaml); + }); + }); +} + +#ifdef HAVE_KMIP + +SEASTAR_TEST_CASE(test_kmip_network_error, *check_run_test_decorator("ENABLE_KMIP_TEST")) { + co_await kmip_test_helper([](const kmip_test_info& info, const tmpdir& tmp) -> future<> { + co_await network_error_test_helper(tmp, info.host, [&](const auto& proxy) { + auto yaml = fmt::format(R"foo( + kmip_hosts: + kmip_test: + hosts: {0} + certificate: {1} + keyfile: {2} + truststore: {3} + priority_string: {4} + key_cache_expiry: 1ms + )foo" + , proxy.address(), info.cert, info.key, info.ca, info.prio + ); + return std::make_tuple(scopts_map({ { "key_provider", "KmipKeyProviderFactory" }, { "kmip_host", "kmip_test" } }), yaml); + }); + }); +} + +#endif // HAVE_KMIP + +// Note: cannot do the above test for gcp, because we can't use false endpoints there. Could mess with address resolution, +// but there is no infrastructure for that atm. diff --git a/test/boost/kmip_wrapper.py b/test/boost/kmip_wrapper.py new file mode 100644 index 000000000000..b6deee255ea0 --- /dev/null +++ b/test/boost/kmip_wrapper.py @@ -0,0 +1,95 @@ +import ssl +import sys + +from kmip.services import auth +from kmip.services.server.server import build_argument_parser +from kmip.services.server.server import KmipServer + +# Helper wrapper for running pykmip in scylla testing. Needed because TLS options +# (hardcoded) in pykmip are obsolete and will not work with connecting using gnutls +# of any modern variety. + +class TLS13AuthenticationSuite(auth.TLS12AuthenticationSuite): + """ + An authentication suite used to establish secure network connections. + + Supports TLS 1.3. More importantly, works with gnutls- + """ + def __init__(self, cipher_suites=None): + """ + Create a TLS12AuthenticationSuite object. + + Args: + cipher_suites (list): A list of strings representing the names of + cipher suites to use. Overrides the default set of cipher + suites. Optional, defaults to None. + """ + super().__init__(cipher_suites) + self._protocol = ssl.PROTOCOL_TLS_SERVER + +def main(): + # Build argument parser and parser command-line arguments. + parser = build_argument_parser() + opts, args = parser.parse_args(sys.argv[1:]) + + kwargs = {} + if opts.hostname: + kwargs['hostname'] = opts.hostname + if opts.port: + kwargs['port'] = opts.port + if opts.certificate_path: + kwargs['certificate_path'] = opts.certificate_path + if opts.key_path: + kwargs['key_path'] = opts.key_path + if opts.ca_path: + kwargs['ca_path'] = opts.ca_path + if opts.auth_suite: + kwargs['auth_suite'] = opts.auth_suite + if opts.config_path: + kwargs['config_path'] = opts.config_path + if opts.log_path: + kwargs['log_path'] = opts.log_path + if opts.policy_path: + kwargs['policy_path'] = opts.policy_path + if opts.ignore_tls_client_auth: + kwargs['enable_tls_client_auth'] = False + if opts.logging_level: + kwargs['logging_level'] = opts.logging_level + if opts.database_path: + kwargs['database_path'] = opts.database_path + + kwargs['live_policies'] = True + + # Create and start the server. + s = KmipServer(**kwargs) + # Fix TLS. Try to get this into mainline project, but that will take time... + s.auth_suite = TLS13AuthenticationSuite(s.auth_suite.ciphers) + # force port to zero -> select dynamically + s.config.settings['port'] = 0 + + def fake_wrap_ssl(sock, keyfile=None, certfile=None, + server_side=False, cert_reqs=ssl.CERT_NONE, + ssl_version=ssl.PROTOCOL_TLS, ca_certs=None, + do_handshake_on_connect=True, + suppress_ragged_eofs=True, + ciphers=None): + ctxt = ssl.SSLContext(protocol = ssl_version) + ctxt.load_cert_chain(certfile=certfile, keyfile=keyfile) + ctxt.verify_mode = cert_reqs + ctxt.load_verify_locations(cafile=ca_certs) + ctxt.set_ciphers(ciphers) + return ctxt.wrap_socket(sock, server_side=server_side + , do_handshake_on_connect=do_handshake_on_connect + , suppress_ragged_eofs=suppress_ragged_eofs) + + ssl.wrap_socket = fake_wrap_ssl + + print("Starting...") + + with s: + print("Listening on {}".format(s._socket.getsockname()[1])) + sys.stdout.flush() + s.serve() + +if __name__ == '__main__': + main() diff --git a/test/boost/symmetric_key_test.cc b/test/boost/symmetric_key_test.cc new file mode 100644 index 000000000000..891f9b00e77c --- /dev/null +++ b/test/boost/symmetric_key_test.cc @@ -0,0 +1,221 @@ +/* + * Copyright (C) 2016 ScyllaDB + */ + + + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include + +#include "ent/encryption/encryption.hh" +#include "ent/encryption/symmetric_key.hh" + +using namespace encryption; + +static temporary_buffer generate_random(size_t n, size_t align) { + std::random_device r; + std::default_random_engine e1(r()); + std::uniform_int_distribution dist('0', 'z'); + + auto tmp = temporary_buffer::aligned(align, align_up(n, align)); + std::generate(tmp.get_write(), tmp.get_write() + tmp.size(), std::bind(dist, std::ref(e1))); + return tmp; +} + +static void test_random_data(const sstring& desc, unsigned int bits) { + auto buf = generate_random(128, 8); + auto n = buf.size(); + + // first, verify padded. + { + key_info info{desc, bits}; + auto k = ::make_shared(info); + + bytes b(bytes::initialized_later(), k->iv_len()); + k->generate_iv(b.data(), k->iv_len()); + + temporary_buffer tmp(n + k->block_size()); + k->encrypt(buf.get(), buf.size(), tmp.get_write(), tmp.size(), b.data()); + + auto bytes = k->key(); + auto k2 = ::make_shared(info, bytes); + + temporary_buffer tmp2(n + k->block_size()); + k2->decrypt(tmp.get(), tmp.size(), tmp2.get_write(), tmp2.size(), b.data()); + + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp2.get(), tmp2.get() + n, buf.get(), buf.get() + n); + } + + // unpadded + { + auto desc2 = desc; + desc2.resize(desc.find_last_of('/')); + key_info info{desc2, bits}; + auto k = ::make_shared(info); + + bytes b(bytes::initialized_later(), k->iv_len()); + k->generate_iv(b.data(), k->iv_len()); + + temporary_buffer tmp(n); + k->encrypt_unpadded(buf.get(), buf.size(), tmp.get_write(), b.data()); + + auto bytes = k->key(); + auto k2 = ::make_shared(info, bytes); + + temporary_buffer tmp2(buf.size()); + k2->decrypt_unpadded(tmp.get(), tmp.size(), tmp2.get_write(), b.data()); + + BOOST_REQUIRE_EQUAL_COLLECTIONS(tmp2.get(), tmp2.get() + n, buf.get(), buf.get() + n); + } +} + + +SEASTAR_TEST_CASE(test_cipher_types) { + static const std::unordered_map> ciphers = { + { "AES/CBC/PKCS5Padding", { 128, 192, 256 } }, + { "AES/ECB/PKCS5Padding", { 128, 192, 256 } }, + { "DES/CBC/PKCS5Padding", { 56 } }, + { "DESede/CBC/PKCS5Padding", { 112, 168 } }, + { "Blowfish/CBC/PKCS5Padding", { 32, 64, 448 } }, + { "RC2/CBC/PKCS5Padding", { 40, 41, 64, 67, 120, 128 } }, + }; + + for (auto & p : ciphers) { + for (auto s : p.second) { + test_random_data(p.first, s); + } + } + return make_ready_future<>(); +} + +// OpenSSL only supports one form of padding. We used to just allow +// non-empty string -> pkcs5/pcks7. We now instead verify this to be +// within the "sane" limits, i.e. pkcs, pkcs5 or pkcs7. +// Check an non-exhaustive set of invalid padding options and verify +// we get an exception as expected. +// See below for test for valid strings. +SEASTAR_TEST_CASE(test_invalid_padding_options) { + static const std::unordered_map ciphers = { + { "AES/CBC/PKCSU", 128 }, + { "AES/ECB/Gris", 128 }, + { "DES/CBC/PKCS12Padding", 56 }, + { "DES/CBC/KorvPadding", 56 }, + { "DES/CBC/MUPadding", 56 }, + }; + for (auto& p : ciphers) { + try { + key_info info{p.first, p.second}; + symmetric_key k(info); + BOOST_ERROR("should not reach"); + } catch (...) { + // ok. + } + } + return make_ready_future<>(); +} + +SEASTAR_TEST_CASE(test_valid_padding_options) { + static const std::unordered_map ciphers = { + { "AES/CBC/PKCS", 128 }, + { "AES/CBC/PKCSPadding", 128 }, + { "AES/ECB/PKCS7Padding", 128 }, + { "AES/ECB/PKCS7", 128 }, + { "DES/CBC/PKCS5Padding", 56 }, + { "DES/CBC/PKCS5", 56 }, + { "AES/CBC/NoPadding", 128 }, + { "AES/ECB/NoPadding", 128 }, + { "DES/CBC/NoPadding", 56 }, + { "AES/CBC/No", 128 }, + { "AES/ECB/No", 128 }, + { "DES/CBC/No", 56 }, + }; + for (auto& p : ciphers) { + key_info info{p.first, p.second}; + symmetric_key k(info); + + auto errors = k.validate_exact_info_result(); + BOOST_REQUIRE_EQUAL(errors, std::string{}); + } + return make_ready_future<>(); +} + +SEASTAR_TEST_CASE(test_warn_adjusted_options) { + static const std::unordered_map> ciphers = { + // blowfish only supports CBC and will become CBC whatever you say + { "Blowfish/CFB/PKCS5Padding", { 32, 64, 448 } }, + { "Blowfish/XTS/PKCS5Padding", { 32, 64, 448 } }, + }; + for (auto& p : ciphers) { + for (auto s : p.second) { + auto alg = p.first; + key_info info{alg, s}; + symmetric_key k(info); + + auto errors = k.validate_exact_info_result(); + BOOST_REQUIRE_NE(errors, std::string{}); + } + } + return make_ready_future<>(); +} + +/** + * Verifies that when using defaults in a key, the key info returned is still + * equal to the input one (by bit and textually) + */ +SEASTAR_TEST_CASE(test_cipher_defaults) { + static const std::unordered_map> ciphers = { + { "AES/CBC/PKCS5Padding", { 128, 192, 256 } }, + { "AES/ECB/PKCS5Padding", { 128, 192, 256 } }, + { "DES/CBC/PKCS5Padding", { 56 } }, + { "DESede/CBC/PKCS5Padding", { 112, 168 } }, + { "Blowfish/CBC/PKCS5Padding", { 32, 64, 448 } }, + { "RC2/CBC/PKCS5Padding", { 40, 41, 64, 67, 120, 128 } }, + }; + + for (auto& p : ciphers) { + for (auto s : p.second) { + auto alg = p.first; + for (;;) { + key_info info{alg, s}; + symmetric_key k(info); + + BOOST_REQUIRE_EQUAL(info, k.info()); + BOOST_REQUIRE_EQUAL(boost::lexical_cast(info), boost::lexical_cast(k.info())); + + auto i = alg.find_last_of('/'); + if (i != sstring::npos) { + alg.resize(i); + continue; + } + // also verify that whatever we say (or don't say), we get a blockmode + // -> iv len > 0 + BOOST_CHECK_GT(k.iv_len(), 0); + + auto errors = k.validate_exact_info_result(); + if (i != sstring::npos) { + BOOST_REQUIRE_EQUAL(errors, std::string{}); + } else { + // Again, if we cut out block mode (i.e. only cipher name left) + // we will still force a block mode. Thus this should warn. + BOOST_REQUIRE_NE(errors, std::string{}); + } + + break; + } + } + } + return make_ready_future<>(); +} diff --git a/test/resource/certs/cacert.pem b/test/resource/certs/cacert.pem new file mode 100644 index 000000000000..9e6f920887ab --- /dev/null +++ b/test/resource/certs/cacert.pem @@ -0,0 +1,82 @@ +Certificate: + Data: + Version: 3 (0x2) + Serial Number: 1572441855 (0x5db98eff) + Signature Algorithm: sha256WithRSAEncryption + Issuer: C=US, O=HyTrust Inc., CN=HyTrust KeyControl Certificate Authority + Validity + Not Before: Jun 1 00:00:00 2011 GMT + Not After : Dec 31 23:59:59 2049 GMT + Subject: C=US, O=HyTrust Inc., CN=HyTrust KeyControl Certificate Authority + Subject Public Key Info: + Public Key Algorithm: rsaEncryption + Public-Key: (2048 bit) + Modulus: + 00:ce:97:5b:a1:30:b1:26:00:31:8a:aa:bd:a2:0c: + 19:1b:24:83:05:20:7a:63:ac:6c:ab:0f:80:24:47: + 2b:03:94:86:25:a4:10:51:fb:b6:e8:5e:33:db:e2: + 91:98:f3:2a:b1:78:ef:83:a5:f2:e1:79:36:44:06: + 3b:01:cd:1a:47:c1:74:89:2b:d3:3c:8d:f1:fb:c2: + 90:88:fe:18:d9:81:7b:2a:e4:67:61:87:17:23:38: + 29:5a:66:eb:d0:01:a6:da:c3:a5:7f:f5:d1:9e:f5: + 04:dc:1c:4a:62:2e:e2:5b:5f:22:56:61:fe:ba:66: + c2:ad:a9:51:43:9c:28:e4:8f:fa:05:12:fa:0d:a5: + 35:e3:2f:99:e8:a4:98:09:f9:e7:c8:e0:6c:a9:bd: + e9:59:b0:83:07:09:10:10:5b:aa:b5:72:3b:40:e6: + 38:f4:e3:f8:9a:55:8e:5e:ae:5c:3e:c3:08:34:13: + 9c:19:fc:65:07:ac:3f:98:ae:a0:d2:f8:1d:4c:bf: + cb:93:a7:e4:d6:37:84:0a:0c:3a:1f:86:f2:35:0c: + 2e:66:b0:9b:43:8e:bc:e4:b9:b0:bf:33:67:c2:97: + df:47:6c:65:cc:55:38:70:a9:39:27:60:e1:74:14: + 34:e9:a6:a0:b6:11:de:61:94:9a:6b:83:f1:84:8d: + 27:9f + Exponent: 65537 (0x10001) + X509v3 extensions: + X509v3 Subject Key Identifier: + 66:AD:DE:45:50:5D:54:68:1F:B0:56:00:65:FB:D1:F2:97:57:EF:6E + X509v3 Authority Key Identifier: + keyid:66:AD:DE:45:50:5D:54:68:1F:B0:56:00:65:FB:D1:F2:97:57:EF:6E + DirName:/C=US/O=HyTrust Inc./CN=HyTrust KeyControl Certificate Authority + serial:5D:B9:8E:FF + + X509v3 Basic Constraints: + CA:TRUE + Signature Algorithm: sha256WithRSAEncryption + ab:a9:19:07:07:f0:b9:74:e1:a8:49:db:bd:c1:21:fc:38:38: + 79:dd:2f:3e:59:be:96:79:1d:18:d4:5e:1f:31:47:fb:bd:d4: + 96:d7:be:87:7e:0d:e4:9e:7c:7a:36:c9:9a:5f:e5:63:38:33: + 68:cf:b1:92:d0:b8:81:1a:6f:23:27:d8:71:50:41:63:ce:5f: + 20:69:72:4c:cd:07:ab:35:58:fe:da:d5:26:1e:44:f4:97:e3: + ff:6c:80:db:31:17:13:52:6c:fb:68:34:71:11:af:b6:84:3b: + b1:5c:d3:67:25:e1:5a:31:a6:68:83:ec:c4:3e:e8:f6:08:60: + d0:2a:26:9e:fe:07:08:57:6e:9a:dd:6e:ba:a2:10:ab:2e:fd: + cd:52:a3:2f:e0:59:6d:33:39:05:ed:fd:ed:ac:b0:e7:98:5e: + f2:51:00:12:df:4c:8a:0c:e2:11:df:43:65:d0:f3:a1:85:59: + 6d:d4:bb:a0:97:f7:c7:40:63:b3:24:cf:ec:5e:9e:42:1b:cc: + e2:36:43:d1:83:79:11:48:3b:3d:db:c3:2a:03:4f:cd:53:2d: + 07:8d:0e:28:4a:a9:58:e0:27:c3:47:f6:ab:00:cd:fc:31:ed: + 99:b9:57:2e:2d:5a:79:5f:48:14:39:8b:0e:da:1c:a0:d6:4e: + d4:81:83:49 +-----BEGIN CERTIFICATE----- +MIID4jCCAsqgAwIBAgIEXbmO/zANBgkqhkiG9w0BAQsFADBXMQswCQYDVQQGEwJV +UzEVMBMGA1UEChMMSHlUcnVzdCBJbmMuMTEwLwYDVQQDEyhIeVRydXN0IEtleUNv +bnRyb2wgQ2VydGlmaWNhdGUgQXV0aG9yaXR5MB4XDTExMDYwMTAwMDAwMFoXDTQ5 +MTIzMTIzNTk1OVowVzELMAkGA1UEBhMCVVMxFTATBgNVBAoTDEh5VHJ1c3QgSW5j +LjExMC8GA1UEAxMoSHlUcnVzdCBLZXlDb250cm9sIENlcnRpZmljYXRlIEF1dGhv +cml0eTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAM6XW6EwsSYAMYqq +vaIMGRskgwUgemOsbKsPgCRHKwOUhiWkEFH7tuheM9vikZjzKrF474Ol8uF5NkQG +OwHNGkfBdIkr0zyN8fvCkIj+GNmBeyrkZ2GHFyM4KVpm69ABptrDpX/10Z71BNwc +SmIu4ltfIlZh/rpmwq2pUUOcKOSP+gUS+g2lNeMvmeikmAn558jgbKm96VmwgwcJ +EBBbqrVyO0DmOPTj+JpVjl6uXD7DCDQTnBn8ZQesP5iuoNL4HUy/y5On5NY3hAoM +Oh+G8jUMLmawm0OOvOS5sL8zZ8KX30dsZcxVOHCpOSdg4XQUNOmmoLYR3mGUmmuD +8YSNJ58CAwEAAaOBtTCBsjAdBgNVHQ4EFgQUZq3eRVBdVGgfsFYAZfvR8pdX724w +gYIGA1UdIwR7MHmAFGat3kVQXVRoH7BWAGX70fKXV+9uoVukWTBXMQswCQYDVQQG +EwJVUzEVMBMGA1UEChMMSHlUcnVzdCBJbmMuMTEwLwYDVQQDEyhIeVRydXN0IEtl +eUNvbnRyb2wgQ2VydGlmaWNhdGUgQXV0aG9yaXR5ggRduY7/MAwGA1UdEwQFMAMB +Af8wDQYJKoZIhvcNAQELBQADggEBAKupGQcH8Ll04ahJ273BIfw4OHndLz5ZvpZ5 +HRjUXh8xR/u91JbXvod+DeSefHo2yZpf5WM4M2jPsZLQuIEabyMn2HFQQWPOXyBp +ckzNB6s1WP7a1SYeRPSX4/9sgNsxFxNSbPtoNHERr7aEO7Fc02cl4VoxpmiD7MQ+ +6PYIYNAqJp7+BwhXbprdbrqiEKsu/c1Soy/gWW0zOQXt/e2ssOeYXvJRABLfTIoM +4hHfQ2XQ86GFWW3Uu6CX98dAY7Mkz+xenkIbzOI2Q9GDeRFIOz3bwyoDT81TLQeN +DihKqVjgJ8NH9qsAzfwx7Zm5Vy4tWnlfSBQ5iw7aHKDWTtSBg0k= +-----END CERTIFICATE----- diff --git a/test/resource/certs/scylla.pem b/test/resource/certs/scylla.pem new file mode 100644 index 000000000000..df1825ddcb40 --- /dev/null +++ b/test/resource/certs/scylla.pem @@ -0,0 +1,57 @@ +Bag Attributes + localKeyID: 12 1A 34 A0 C8 58 91 A4 E3 B6 7F 16 F0 31 05 81 AA 27 82 05 +subject=/C=US/O=HyTrust Inc./CN=scylla +issuer=/C=US/O=HyTrust Inc./CN=HyTrust KeyControl Certificate Authority +-----BEGIN CERTIFICATE----- +MIIDlTCCAn2gAwIBAgIFAPm5jwcwDQYJKoZIhvcNAQELBQAwVzELMAkGA1UEBhMC +VVMxFTATBgNVBAoTDEh5VHJ1c3QgSW5jLjExMC8GA1UEAxMoSHlUcnVzdCBLZXlD +b250cm9sIENlcnRpZmljYXRlIEF1dGhvcml0eTAgFw0yMTAxMDQxMjMwMTlaGA8y +MDk5MDEwMTEyMzAxOVowNTELMAkGA1UEBhMCVVMxFTATBgNVBAoTDEh5VHJ1c3Qg +SW5jLjEPMA0GA1UEAxMGc2N5bGxhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB +CgKCAQEAosGginu/B4eQKyB26ar6rqg8FTMoCTf56iBwQnrOt17YN++IitlEhFyB +X0QdA0z97jLIBckTCbieTg+CdtWDeCnM8IAjoN55C0Z2zBKH6cuPTTnDu0WaZY/8 +IafGwxcWllYpgQ3AiJFNK66QRIiiX+ejrS3+Co0PYWPzmSczWoxBgFhnXnTPE4ki +MLvZ+zY1iXt83NbQIw8yMUcL+RYK4RlACf3bPztOss98LmyntIkNkZL8GblLoZbc +AZc6udnDe1GuP+NlMO+1jPmyND/xz/kK2hkU4+yotBWVxM1lwpANnElsAHaRvthP +kGsjoVvZgEgg6MQX+iaDngjmiLtUlwIDAQABo4GHMIGEMAkGA1UdEwQCMAAwLAYJ +YIZIAYb4QgENBB8WHU9wZW5TU0wgR2VuZXJhdGVkIENlcnRpZmljYXRlMB0GA1Ud +DgQWBBTAgszBvUAggtuLWf2XucTzjLQeqDAfBgNVHSMEGDAWgBRmrd5FUF1UaB+w +VgBl+9Hyl1fvbjAJBgNVHREEAjAAMA0GCSqGSIb3DQEBCwUAA4IBAQBJA9q7q5XL +H7O/EiJYagWaqrO+AGVrg3DtkP0NTaumM/zYoOq9klZNwvsGZ88+L0hBQ8fj8O/l +rTobdM7eT5p4JiwfN+MB8zXuZ+XjL+kIpqFqPJdVDtBLPGINHP7itrUo9Uk/9XcW +JqPAwufEMN5X7iwN80aUVj6/iUiQ1yqXKQmERdoIPqiyHOEBwobwIAQ1bQKtEiBT +ZE0hdDdI+ZCrtJYOES4kpR4WI3997doVhEusNGBETCWMm2HoJ8xsk7fgVgnpYlat +95tGZ/ZR1Zaa+fXSm9adxJDCviG77pTZBa7nbzPoyG3wm76cTGSeHso8rvGVHsTh +AraNJqMQkZT5 +-----END CERTIFICATE----- +Bag Attributes + localKeyID: 12 1A 34 A0 C8 58 91 A4 E3 B6 7F 16 F0 31 05 81 AA 27 82 05 +Key Attributes: +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCiwaCKe78Hh5Ar +IHbpqvquqDwVMygJN/nqIHBCes63Xtg374iK2USEXIFfRB0DTP3uMsgFyRMJuJ5O +D4J21YN4KczwgCOg3nkLRnbMEofpy49NOcO7RZplj/whp8bDFxaWVimBDcCIkU0r +rpBEiKJf56OtLf4KjQ9hY/OZJzNajEGAWGdedM8TiSIwu9n7NjWJe3zc1tAjDzIx +Rwv5FgrhGUAJ/ds/O06yz3wubKe0iQ2RkvwZuUuhltwBlzq52cN7Ua4/42Uw77WM ++bI0P/HP+QraGRTj7Ki0FZXEzWXCkA2cSWwAdpG+2E+QayOhW9mASCDoxBf6JoOe +COaIu1SXAgMBAAECggEAefvDdmd+6obJH/mqBkIWtpbyyTTZOeeRUM/VM45VpovY +ZDwMS3zB5K5sbFlhoVqwKzo2PlfRBAUx6PGo56XpbkNuDYcMrQJxGGlMmnD9GtZt +ZgT4VUC6kopS/2p/BzMjw7N6UfZbqj+05htkl2kMwfKb8y04bPICaA0Aw3XlAGRS +WIrfsXW32/hStXJvt8YSveIUP/WbdyEefwcKsu44LFvaq+j85KH75KLyZQre6n/A +7A67WIs1AcjiBHq/nupNF9rZUFm5YzL/kOC+g9Klq6bIIlSu+2ULIjL0WLy5be8R +OdIipE+ENa5HpC5J8z9WfGCf8yTB8/V+pt2dxZ9T6QKBgQDV2R1/9IgphiNCjPSE +M7JbY+K0XJ5Mfq64h1apLjycsq93dUX+1OI/e/qlxKS6SpqCgEajKch/x16ylZeQ +BNuG3zBuFPJH/17MfjdWFAW5ZJmbsnM05u+IU4aN0+r8laddvyNcdgLS4O6WvGrL +z5Nxio0YLkZnCFBsjdUb5Fs00wKBgQDC1mViZp4yT1b21r070sYQ1w4wgLYfVNIF +rc8AFevGh35kFRe2fFWl7T28Zqhetj7ES2LK7H93o8GdT28RuHrwst/nYL+lClmn +37iHPuHwlqDussCxg5A49HWq2qEoZ5DF379b8FJgcBYxYBsu7LSmQNEgFtCdvD7y +8C3uHieWrQKBgQCoOK0OFOxvzcc6+Or4fDpXzhFuVFVqU1Vab4xqdabUlXOWgzhW +qFx6GCsp77CtozY8ZnAqthm0+r6xuR+K+Wc/h57vWabloCuQrdEV85Y1Kr/zTMN3 +4BqZoSr9srDtlUQdaNiGSYcbIDpPdVSFk3qnHJi1ZuGW92Fco4367P4aZQKBgFyZ +4V2/P/jRVJfEv/Oq3ZArZgcPZX/GpHsmfHeh84lL5HpUvAxzU5GlC+57LBK3s2VA +HxgrBvopzl+h3Twi3euAWIJzrSIXpTzwS5eb/26FaL+KHaNA0E8BgNtPRcEaV+hz +y1M7CSvkmeelscW/oqxRqhMCROxzB8gW9v1xP4eRAoGADhZOjOmsfbBD58ijzaOJ +i2bOjCPzkRsS0cxnVlILA7kWU9bPxOkHKmkZnaYIBJhAJsW9l1XA3mlS5gqf93EW +VivIJZb10wI/MeRuITxanWiHPDcbbyOjBgaj6gNBbyrS9rDordYdLRVOsmlwvvJT +a60BCRcu0IgS6uM0sKaojoc= +-----END PRIVATE KEY----- From f901beec8780c7ed46e2797333054726e0680626 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 12:33:07 +0000 Subject: [PATCH 230/397] tools: Add local-file-key-generator tool For generating key files for local provider --- configure.py | 3 +- main.cc | 1 + tools/CMakeLists.txt | 1 + tools/entry_point.hh | 1 + tools/scylla-local-file-key-generator.cc | 166 +++++++++++++++++++++++ 5 files changed, 171 insertions(+), 1 deletion(-) create mode 100644 tools/scylla-local-file-key-generator.cc diff --git a/configure.py b/configure.py index 35c1f10039b9..2d5b947823ea 100755 --- a/configure.py +++ b/configure.py @@ -1336,7 +1336,8 @@ def find_ninja(): scylla_raft_dependencies = scylla_raft_core + ['utils/uuid.cc', 'utils/error_injection.cc', 'utils/exceptions.cc'] -scylla_tools = ['tools/read_mutation.cc', +scylla_tools = ['tools/scylla-local-file-key-generator.cc', + 'tools/read_mutation.cc', 'tools/scylla-types.cc', 'tools/scylla-sstable.cc', 'tools/scylla-nodetool.cc', diff --git a/main.cc b/main.cc index 8b73496bfcfd..2eef9bf59914 100644 --- a/main.cc +++ b/main.cc @@ -2393,6 +2393,7 @@ int main(int ac, char** av) { {"types", tools::scylla_types_main, "a command-line tool to examine values belonging to scylla types"}, {"sstable", tools::scylla_sstable_main, "a multifunctional command-line tool to examine the content of sstables"}, {"nodetool", tools::scylla_nodetool_main, "a command-line tool to administer local or remote ScyllaDB nodes"}, + {"local-file-key-generator", tools::scylla_local_file_key_generator_main, "a command-line tool to generate encryption at rest keys"}, {"perf-fast-forward", perf::scylla_fast_forward_main, "run performance tests by fast forwarding the reader on this server"}, {"perf-row-cache-update", perf::scylla_row_cache_update_main, "run performance tests by updating row cache on this server"}, {"perf-tablets", perf::scylla_tablets_main, "run performance tests of tablet metadata management"}, diff --git a/tools/CMakeLists.txt b/tools/CMakeLists.txt index b763bcc30050..81ec0075599e 100644 --- a/tools/CMakeLists.txt +++ b/tools/CMakeLists.txt @@ -1,6 +1,7 @@ add_library(tools STATIC) target_sources(tools PRIVATE + scylla-local-file-key-generator.cc load_system_tablets.cc read_mutation.cc scylla-types.cc diff --git a/tools/entry_point.hh b/tools/entry_point.hh index fe2ef0ee73b8..682ac858448e 100644 --- a/tools/entry_point.hh +++ b/tools/entry_point.hh @@ -10,6 +10,7 @@ namespace tools { +int scylla_local_file_key_generator_main(int argc, char** argv); int scylla_types_main(int argc, char** argv); int scylla_sstable_main(int argc, char** argv); int scylla_nodetool_main(int argc, char** argv); diff --git a/tools/scylla-local-file-key-generator.cc b/tools/scylla-local-file-key-generator.cc new file mode 100644 index 000000000000..5f46262b124b --- /dev/null +++ b/tools/scylla-local-file-key-generator.cc @@ -0,0 +1,166 @@ +/* + * Copyright (C) 2020-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include +#include + +#include +#include "compound.hh" +#include "db/marshal/type_parser.hh" +#include "schema/schema_builder.hh" +#include "tools/utils.hh" +#include "dht/i_partitioner.hh" +#include "utils/managed_bytes.hh" +#include "ent/encryption/symmetric_key.hh" +#include "ent/encryption/local_file_provider.hh" + +using namespace seastar; +using namespace tools::utils; + +namespace bpo = boost::program_options; + +namespace std { +// required by boost::lexical_cast(vector), which is in turn used +// by boost::program_option for printing out the default value of an option +static std::ostream& operator<<(std::ostream& os, const std::vector& v) { + return os << fmt::format("{}", v); +} +} + +namespace { + +const auto app_name = "local-file-key-generator"; + +const std::vector global_options{ + typed_option("alg,a", "AES", "Key algorithm (i.e. AES, 3DES)"), + typed_option("block-mode,b", "CBC", "Algorithm block mode (i.e. CBC, EBC)"), + typed_option("padding,p", "PKCS5", "Algorithm padding method (i.e. PKCS5)"), + typed_option("length,l", 128, "Key length in bits (i.e. 128, 256)"), +}; + +const std::vector global_positional_options{ + typed_option>("files", "key path|key name", -1), +}; + +const std::vector operations = { + {"generate", "creates a new key and stores to a new file", +R"( +Generate a key suitable for a given algorithm and key length +and store to a file readable by scylla encryption at rest +local file key provider. +)"}, + {"append", "same as generate, but appends key to existing file", +R"( +Generate a key suitable for a given algorithm and key length +and append to an existing file readable by scylla encryption at rest +local file key provider. +)"}, +}; + +} + +namespace tools { + +using namespace encryption; +using namespace std::string_literals; +namespace fs = std::filesystem; + +int scylla_local_file_key_generator_main(int argc, char** argv) { + constexpr auto description_template = +R"(scylla-{} - a command-line tool to generate file-based encryption keys. + +Usage: scylla {} [--option1] [--option2] ... [key path|key name] + +Allows creating symmetric keys for use with scylla encryption at rest +local key file provider. + +Where can be one of: + +{} +)"; + + auto op_str = std::ranges::to(operations | std::views::transform([] (const operation& op) { + return fmt::format("* {} - {}\n{}", op.name(), op.summary(), op.description()); + }) | std::views::join_with('\n')); + tool_app_template::config app_cfg{ + .name = app_name, + .description = seastar::format(description_template, app_name, app_name, op_str), + .operations = std::move(operations), + .global_options = &global_options, + .global_positional_options = &global_positional_options, + }; + tool_app_template app(std::move(app_cfg)); + + return app.run_async(argc, argv, [] (const operation& op, const boost::program_options::variables_map& app_config) { + std::vector files; + + if (app_config.contains("files")) { + files = app_config["files"].as>(); + } + if (files.size() > 1) { + throw std::invalid_argument("Too many arguments"); + } + auto alg = app_config["alg"].as(); + auto mode = app_config["block-mode"].as(); + auto padd = app_config["padding"].as(); + auto len = app_config["length"].as(); + + if (!padd.ends_with("Padding")) { + padd = padd + "Padding"; + } + + auto java_sig = fmt::format("{}/{}/{}", alg, mode, padd); + + key_info info { + .alg = java_sig, .len = len + }; + + symmetric_key k(info); + auto key = k.key(); + auto hex = base64_encode(key); + auto line = fmt::format("{}:{}:{}", java_sig, len, hex); + auto key_name = "system_key"s; + + if (!files.empty()) { + fs::path f(files.front()); + if (fs::is_directory(f)) { + f = f / key_name; + } + if (!fs::exists(f)) { + auto p = f.parent_path(); + if (!p.empty()) { + fs::create_directories(p); + } + } + std::ios_base::openmode mode = std::ios_base::out; + if (op.name() == "append") { + mode |= std::ios_base::ate|std::ios_base::app; + } else { + mode |= std::ios_base::trunc; + } + + if (!fs::exists(f) || op.name() != "append") { + // create once so we can enforce proper + // permissions. (neither seastar or c++ io is great here) + std::ofstream os(f, mode); + } + + fs::permissions(f, fs::perms::owner_read|fs::perms::owner_write); + std::ofstream os(f, mode); + + os << line << std::endl; + } else { + std::cout << line << std::endl; + } + return 0; + }); +} + +} // namespace tools From 083f7353665e65976c52df659b7b059f8134fb94 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 12:33:58 +0000 Subject: [PATCH 231/397] main/build: Add p11-kit and initialize For p11 certification/validation --- CMakeLists.txt | 2 ++ cmake/Findp11-kit.cmake | 48 +++++++++++++++++++++++++++++++++++++++++ configure.py | 3 ++- main.cc | 21 ++++++++++++++++++ 4 files changed, 73 insertions(+), 1 deletion(-) create mode 100644 cmake/Findp11-kit.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index cd32630f9158..375f55e22af3 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -148,6 +148,7 @@ find_package(ICU COMPONENTS uc i18n REQUIRED) find_package(fmt 10.0.0 REQUIRED) find_package(libdeflate REQUIRED) find_package(libxcrypt REQUIRED) +find_package(p11-kit REQUIRED) find_package(Snappy REQUIRED) find_package(RapidJSON REQUIRED) find_package(xxHash REQUIRED) @@ -345,6 +346,7 @@ if(Scylla_ENABLE_LTO) endif() target_link_libraries(scylla PRIVATE + p11-kit::p11-kit Seastar::seastar absl::headers yaml-cpp::yaml-cpp diff --git a/cmake/Findp11-kit.cmake b/cmake/Findp11-kit.cmake new file mode 100644 index 000000000000..e9188ec1c7ca --- /dev/null +++ b/cmake/Findp11-kit.cmake @@ -0,0 +1,48 @@ +# +# Copyright 2023-present ScyllaDB +# + +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# +find_package(PkgConfig REQUIRED) + +pkg_check_modules(PC_p11_kit QUIET p11-kit-1) + +find_library(p11-kit_LIBRARY + NAMES p11-kit + PATH_SUFFIXES p11-kit-1 + HINTS + ${PC_p11_kit_LIBDIR} + ${PC_p11_kit_LIBRARY_DIRS}) + +find_path(p11-kit_INCLUDE_DIR + NAMES p11-kit/p11-kit.h + HINTS + ${PC_p11_kit_INCLUDEDIR} + ${PC_p11_kit_INCLUDE_DIRS}) + +mark_as_advanced( + p11-kit_LIBRARY + p11-kit_INCLUDE_DIR) + +include(FindPackageHandleStandardArgs) + +find_package_handle_standard_args(p11-kit + REQUIRED_VARS + p11-kit_LIBRARY + p11-kit_INCLUDE_DIR + VERSION_VAR PC_p11_kit_VERSION) + +if(p11-kit_FOUND) + set(p11-kit_LIBRARIES ${p11-kit_LIBRARY}) + set(p11-kit_INCLUDE_DIRS ${p11-kit_INCLUDE_DIR}) + if(NOT(TARGET p11-kit::p11-kit)) + add_library(p11-kit::p11-kit UNKNOWN IMPORTED) + + set_target_properties(p11-kit::p11-kit + PROPERTIES + IMPORTED_LOCATION ${p11-kit_LIBRARY} + INTERFACE_INCLUDE_DIRECTORIES ${p11-kit_INCLUDE_DIRS}) + endif() +endif() diff --git a/configure.py b/configure.py index 2d5b947823ea..956c59d970c7 100755 --- a/configure.py +++ b/configure.py @@ -2004,7 +2004,6 @@ def query_seastar_flags(pc_file, use_shared_libs, link_static_cxx=False): libs = f"-Wl,-rpath='{rpath}' {libs}" if link_static_cxx: libs = libs.replace('-lstdc++ ', '') - testing_libs = pkg_config(pc_file.replace('seastar.pc', 'seastar-testing.pc'), '--libs', '--static') return {'seastar_cflags': cflags, 'seastar_libs': libs, @@ -2028,6 +2027,8 @@ def query_seastar_flags(pc_file, use_shared_libs, link_static_cxx=False): '-ldeflate', ]) +args.user_cflags += " " + pkg_config('p11-kit-1', '--cflags') + if not args.staticboost: user_cflags += ' -DBOOST_ALL_DYN_LINK' diff --git a/main.cc b/main.cc index 2eef9bf59914..6b07eb118961 100644 --- a/main.cc +++ b/main.cc @@ -118,6 +118,12 @@ #include "utils/shared_dict.hh" #include "message/dictionary_service.hh" + +#define P11_KIT_FUTURE_UNSTABLE_API +extern "C" { +#include +} + seastar::metrics::metric_groups app_metrics; using namespace std::chrono_literals; @@ -2451,5 +2457,20 @@ int main(int ac, char** av) { return 0; } + // We have to override p11-kit config path before p11-kit initialization. + // And the initialization will invoke on seastar initalization, so it has to + // be before app.run() + // #3583 - need to potentially ensure this for tools as well, since at least + // sstable* might need crypto libraries. + auto scylla_path = fs::read_symlink(fs::path("/proc/self/exe")); // could just be argv[0] I guess... + auto p11_modules = scylla_path.parent_path().parent_path().append("share/p11-kit/modules"); + // Note: must be in scope for application lifetime. p11_kit_override_system_files does _not_ + // copy input strings. + auto p11_modules_str = p11_modules.string(); + // #3392 only do this if we are actually packaged and the path exists. + if (fs::exists(p11_modules)) { + ::p11_kit_override_system_files(NULL, NULL, p11_modules_str.c_str(), NULL, NULL); + } + return main_func(ac, av); } From 8e828f608dffefae5d837f4b9a74df075f874899 Mon Sep 17 00:00:00 2001 From: Calle Wilund Date: Wed, 8 Jan 2025 13:10:12 +0000 Subject: [PATCH 232/397] docs: Add EAR docs Merge docs relating to EAR. --- docs/dev/file_encryption.md | 76 ++ docs/operating-scylla/_common/tools_index.rst | 2 + .../nodetool-commands/upgradesstables.rst | 1 + .../procedures/backup-restore/restore.rst | 4 + .../cassandra-to-scylla-migration-process.rst | 13 + .../security/_common/security-index.rst | 1 + .../security/encryption-at-rest.rst | 859 ++++++++++++++++++ docs/operating-scylla/security/index.rst | 3 +- .../security/security-checklist.rst | 10 +- docs/reference/glossary.rst | 3 + 10 files changed, 967 insertions(+), 5 deletions(-) create mode 100644 docs/dev/file_encryption.md create mode 100644 docs/operating-scylla/security/encryption-at-rest.rst diff --git a/docs/dev/file_encryption.md b/docs/dev/file_encryption.md new file mode 100644 index 000000000000..824c9a10b3fe --- /dev/null +++ b/docs/dev/file_encryption.md @@ -0,0 +1,76 @@ +File level encryption in scylla enterprise +========================================== + +File encryption in scylla enterprise is done by "block-level" encryption via a `file_impl` implementation that transparently wraps file IO transforming data +to/from encrypted state. Refer to `encrypted_file_impl` in `ent/encryption/encrypted_file_impl.cc`. + +Encryption is algorithm-agnostic in that the wrapper supports any symmetric-key algorithm (block cipher) that is available in the OpenSSL EVP (envelope) +library. + +The wrapper uses a user-provided symmetric key coupled with ESSIV block initialization vector calculation. *NOTE*: the data file itself does *not* keep track of +the key used to encrypt data, thus an external meta data provider is required to map files to their keys, and is solely the user's responsibility. + +File block encryption does not use padding, since it relies on input data size and output data size being identical. + +The file is divided in `N` blocks of 4096 bytes size. Each 4KB block is encrypted with the provided key `K`, configured block cipher `B`, and block cipher +operating mode (usually CBC). Because 4KB is an integral multiple of any considerable block cipher's block size, no padding is necessary within any 4KB file +block. + +The initialization vector (IV) for the block cipher operating mode (usually CBC) of each 4KB file block is derived via +[ESSIV](https://en.wikipedia.org/wiki/Disk_encryption_theory#Encrypted_salt-sector_initialization_vector_(ESSIV)): + +- The user-provided data encryption key `K` is hashed with SHA256 to value `h` (32 bytes = 256 bits). + +- An AES256 block cipher is keyed with `h`. + +- For the particular file block number, a byte array is populated with 8 `NUL` bytes, followed by the little-endian representation of the `uint64_t` block + number (16 bytes = 128 bits). + +- The 16 byte array is encrypted with a single round (i.e., one ECB mode application) of the AES256 block cipher, to value `c` (16 bytes = 128 bits). + +- `c` is the IV of the block cipher `B` (truncated or zero-padded as required by the block size of `B`). + +``` +h := SHA256(K) +IV_B(block_number) := AES256_h(uint64_t(0) ‖ block_number_le64) +``` + + +Padding/truncation +================== + +All encryption is done unpadded. To handle file sizes we use a simplified padding scheme: + +Since all writes are assumed to be done by us, and must be aligned (scylla requirement), we can assume in turn that any resizing should be made by truncation. + +If a file is truncated to a size that is not a whole multiple of the `B` block cipher's block size (which is typically 16 bytes = 128 bits), then we increment +the actual truncation size by `B`'s block size. + + +``` + +----------- 16 bytes ----------+ + | | + +----------- 16 bytes ----------+ | + | | | | + v v v v ++--------------+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ +| n * 16 bytes | T | P | T' | ++--------------+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + ^ ^ + | | +requested truncation offset: n * 16 + 3 | + | + actual truncation offset: (n + 1) * 16 + 3 + +``` + +- This preserves, in its entirety, for decryption's sake, the ciphertext block (`T ‖ P`) that the user expects to be truncated. + +- It records the useful byte count (`size(T)`) of the final ciphertext block (`T ‖ P`) through the trailing misalignment (`size(T')`). The contents of `T'` are + irrelevant (it's a partial cipher block, so it cannot be decrypted); only its size matters. + +When reading an encrypted file, we check the file size. If we're reading from a file with an unaliged size (i.e., `size(T')` is nonzero), we know that the size +of the padding at the end is `B`'s block size. `T'` is discarded; the last complete cipher block (`T ‖ P`) is decrypted. After decryption, `P` is discarded as +well. File size query methods adjust the returned values accordingly. + +Non-empty files that are shorter than `B`'s block size are invalid -- they can never be created by the above-described padding scheme. diff --git a/docs/operating-scylla/_common/tools_index.rst b/docs/operating-scylla/_common/tools_index.rst index d12fbb6f20bd..ead541052148 100644 --- a/docs/operating-scylla/_common/tools_index.rst +++ b/docs/operating-scylla/_common/tools_index.rst @@ -8,6 +8,8 @@ * :doc:`cassandra-stress ` A tool for benchmarking and load testing a ScyllaDB and Cassandra clusters. * :doc:`SSTabledump ` * :doc:`SSTableMetadata ` +* configuration_encryptor - :doc:`encrypt at rest ` sensitive scylla configuration entries using system key. +* scylla local-file-key-generator - Generate a local file (system) key for :doc:`encryption at rest `, with the provided length, Key algorithm, Algorithm block mode and Algorithm padding method. * `scyllatop `_ - A terminal base top-like tool for scylladb collectd/prometheus metrics. * :doc:`scylla_dev_mode_setup` - run ScyllaDB in Developer Mode. * :doc:`perftune` - performance configuration. diff --git a/docs/operating-scylla/nodetool-commands/upgradesstables.rst b/docs/operating-scylla/nodetool-commands/upgradesstables.rst index e2f9b030cc9c..a052f59801d1 100644 --- a/docs/operating-scylla/nodetool-commands/upgradesstables.rst +++ b/docs/operating-scylla/nodetool-commands/upgradesstables.rst @@ -55,6 +55,7 @@ Additional References --------------------- .. include:: nodetool-index.rst +:doc:`Encryption at Rest ` diff --git a/docs/operating-scylla/procedures/backup-restore/restore.rst b/docs/operating-scylla/procedures/backup-restore/restore.rst index 71d8b92e633f..61ebf7d793c4 100644 --- a/docs/operating-scylla/procedures/backup-restore/restore.rst +++ b/docs/operating-scylla/procedures/backup-restore/restore.rst @@ -35,6 +35,10 @@ Procedure Repeat the following steps for each node in the cluster: -------------------------------------------------------- +.. note:: + + If you are restoring :doc:`encrypted backup files `, make sure Scylla Enterprise has the same keys used by Scylla to encrypt the data before starting the restore process. + .. note:: Best practise is **not** to restore :doc:`Materialized Views (MV) ` and :doc:`Secondary Indexes (SI) ` SSTables. diff --git a/docs/operating-scylla/procedures/cassandra-to-scylla-migration-process.rst b/docs/operating-scylla/procedures/cassandra-to-scylla-migration-process.rst index 546653926c68..2ec0f0a9a371 100644 --- a/docs/operating-scylla/procedures/cassandra-to-scylla-migration-process.rst +++ b/docs/operating-scylla/procedures/cassandra-to-scylla-migration-process.rst @@ -51,6 +51,19 @@ Procedure - Import schema to ScyllaDB: ``cqlsh [IP] --file 'adjusted_schema.cql'`` .. _`limitations and known issues section`: #notes-limitations-and-known-issues + + +.. note:: + + Scylla and Apache Cassandra :doc:`encrypted backup files ` are **not** compatible. + sstableloader does **not** support loading from encrypted files. + + If you need to migrate/restore from encrypted files: + + * Upload them to the original database + * Decrypted the table with ALTER TABLE + * Update the SSTables files with :doc:`upgradesstable ` + * Use sstableloader .. note:: diff --git a/docs/operating-scylla/security/_common/security-index.rst b/docs/operating-scylla/security/_common/security-index.rst index 33a8c81928ff..fae922e75a87 100644 --- a/docs/operating-scylla/security/_common/security-index.rst +++ b/docs/operating-scylla/security/_common/security-index.rst @@ -8,6 +8,7 @@ * :doc:`Encryption: Data in Transit Client to Node ` * :doc:`Encryption: Data in Transit Node to Node ` * :doc:`Generating a self-signed Certificate Chain Using openssl ` +* :doc:`Encryption at Rest ` diff --git a/docs/operating-scylla/security/encryption-at-rest.rst b/docs/operating-scylla/security/encryption-at-rest.rst new file mode 100644 index 000000000000..d5ae3fb5dfa9 --- /dev/null +++ b/docs/operating-scylla/security/encryption-at-rest.rst @@ -0,0 +1,859 @@ +================== +Encryption at Rest +================== + +Introduction +---------------------- + +ScyllaDB Enterprise protects your sensitive data with data-at-rest encryption. +It protects the privacy of your user's data, reduces the risk of data breaches, and helps meet regulatory requirements. +In particular, it provides an additional level of protection for your data persisted in storage or its backups. + +When ScyllaDB Enterprise Encryption at Rest is used together with Encryption in Transit (:doc:`Node to Node ` and :doc:`Client to Node `), you benefit from end to end data encryption. + +About Encryption at Rest +----------------------------- + +The following can be encrypted: + +* ScyllaDB persistent tables (SSTables) +* System level data, such as: + + - Commit logs + - Batches + - hints logs + - KMIP Password (part of scylla.yaml) + +Encryption at Rest works at table level granularity, so you can choose to encrypt only sensitive tables. For both system and table data, you can use different algorithms that are supported by `OpenSSL `_ in a file block encryption scheme. + +.. note:: SSTables of a particular table can have different encryption keys, use different encryption algorithms, or not be encrypted at all - at the same time. + +When is Data Encrypted? +======================== + +As SSTables are immutable, tables are encrypted only once, as a result of memtable flush, compaction, or upgrade (with :doc:`Nodetool upgradesstables `). + +Once a table is encrypted, all resulting SSTables are encrypted using the most current key and algorithm. +When you encrypt an existing table, the new SSTables are encrypted. The old SSTables which existed before the encryption are not updated. These tables are encrypted according to the same actions as described previously. + +When is Data Decrypted? +======================== + +When ScyllaDB reads an encrypted SSTable from disk, it fetches the encryption key's ID from the SSTable and uses it to extract the key and decrypt the data. +When ScyllaDB reads an encrypted system table, it fetches the system table encryption key location from the scylla.yaml file. It locates the key and uses it to extract the key and decrypt the data. + + +Encryption Key Types +---------------------- + +Two types of encryption keys are available: System Keys and Table Keys. + +System Keys +==================== + +System keys are used for encrypting system data, such as commit logs, hints, and/or other user table keys. When a Replicated Key Provider is used for encrypting SSTables, the table keys are stored in the encrypted_keys table, and the system key is used to encrypt the encrypted_keys table. The system key is stored as the contents of a local file and is encrypted with a single key that you provide. The default location of system keys is ``/etc/scylla/resources/system_keys/`` and can be changed with the ``system_key_directory`` option in scylla.yaml file. When a Local Key Provider is used for encrypting system info, you can provide your own key, or ScyllaDB can make one for you. + +.. _Replicated: + +Table Keys +=================== +Table keys are used for encrypting SSTables. Depending on your key provider, this key is stored in different locations: + +* Replicated Key Provider - encrypted_keys table +* KMIP Key Provider - KMIP server +* KMS Key Provider - AWS +* Local Key Provider - in a local file with multiple keys. You can provide your own key or ScyllaDB can make one for you. + +.. _ear-key-providers: + +.. note:: + + Encrypted SStables undergo a regular backup procedure. Ensure you keep your + encryption key available in case you need to restore from backup. + +Key Providers +---------------------- + +When encrypting the system tables or SSTables, you need to state which provider is holding your keys. You can use the following options: + +.. list-table:: + :widths: 33 33 33 + :header-rows: 1 + + * - Key Provider Name + - key_provider Name + - Description + * - Local Key Provider + - LocalFileSystemKeyProviderFactory (**default**) + - Stores the key on the same machine as the data. + * - Replicated Key Provider + - ReplicatedKeyProviderFactory + - Stores table keys in a ScyllaDB table where the table itself is encrypted using the system key (available from 2019.1.3) + * - KMIP Key Provider + - KmipKeyProviderFactory + - External key management server (available from 2019.1.3) + * - KMS Key Provider + - KmsKeyProviderFactory + - Uses key(s) provided by the AWS KMS service. + * - GCP Key Provider + - GcpKeyProviderFactory + - Used key(s) provided by the GCP KMS service. + + +About Local Key Storage +========================== + +Local keys are used for encrypting user data, such as SSTables. +Currently, this is the only option available for user data and, as such, is the default key storage manager. +With local key storage, keys are stored locally on disk in a text file. The location of this file is specified in the scylla.yaml. + +.. caution:: Care should be taken so that no unauthorized person can access the key data from the file system. Make sure that the owner of this file is the ``scylla`` user and that the file is **not** readable by **other users**, not accessible by **other roles**. + +You should also consider keeping the key directory on a network drive (using TLS for the file sharing) to avoid having keys and data on the same storage media, in case your storage is stolen or discarded. + +.. _ear-cipher-algorithms: + +Cipher Algorithms +---------------------- + +The following cipher_algorithims are available for use with ScyllaDB using `OpenSSL `_. Note that the default algorithm (AES/CBC/PKCS5Padding with key strength 128 ) is recommended. + +.. list-table:: + :widths: 70 30 + :header-rows: 1 + + * - cipher_algorithm + - secret_key_strength + * - AES/CBC/PKCS5Padding (**default**) + - 128 (**default**), 192, or 256 + * - AES/ECB/PKCS5Padding + - 128, 192, or 256 + * - Blowfish/CBC/PKCS5Padding + - 32-448 + +.. _ear-create-encryption-key: + +Create Encryption Keys +----------------------------- + +Depending on your key provider, you will either have the option of allowing ScyllaDB to generate an encryption key, or you will have to provide one: + +* KMIP Key Provider - you don't need to generate any key yourself +* KMS Key Provider - you must generate a key yourself in AWS +* Replicated Key Provider - you must generate a system key yourself +* Local Key Provider - If you do not generate your own secret key, ScyllaDB will create one for you + +When encrypting ScyllaDB config by ``configuration_encryptor``, you also need to generate a secret key and upload the key to all nodes. + + +Use the key generator script +================================ + +The Key Generator script generates a key in the directory of your choice. + +**Procedure** + + +#. Create (if it doesn't exist) a local directory for storing the key. Make sure that the owner of the directory is ``scylla`` and not another user. Make sure that the ``scylla`` user can read, write, and execute over the parent directory. Following this procedure makes ``/etc/scylla/encryption_keys/`` the parent directory of your keys. + + For example: + + .. code-block:: none + + sudo mkdir -p /etc/scylla/encryption_keys/system_keys + sudo chown -R scylla:scylla /etc/scylla/encryption_keys + sudo chmod -R 700 /etc/scylla/encryption_keys + +#. Create a key using the local file key generator script making sure that the keyfile owner is ``scylla`` and not another user. Run the command: + + .. code-block:: none + + sudo -u scylla /usr/bin/scylla local-file-key-generator [options] [key-path] + + Where: + + * ``-a,--alg `` - the encryption algorithm (e.g., AES) you want to use to encrypt the key + * ``-h,--help`` - displays the help menu + * ``-l,--length `` - the length of the encryption key in bits (i.e. 128, 256) + * ``-b,--block-mode `` - the encryption algorithm block mode (i.e. CBC, EBC) + * ``-p,--padding `` - the encryption algorithm padding method (i.e. PKCS5) + * ``key-path`` - is the directory you want to place the key into (/etc/scylla/encryption_keys, for example) + + And ```` is one of ``generate`` or ``append``, the first creating a new key file with the generated key, the latter + appending a new key of the required type to an existing file. + + For Example: + + To create a secret key and a system key using other encryption settings in a different location: + + .. code-block:: none + + sudo -u scylla /usr/bin/scylla local-file-key-generator generate -a AES -b ECB -p PKCS5 -l 192 /etc/scylla/encryption_keys/secret_key + sudo -u scylla /usr/bin/scylla local-file-key-generator generate -a AES -b CBC -p PKCS5 -l 128 /etc/scylla/encryption_keys/system_keys/system_key + + To display the secret key parameters: + + .. code-block:: none + + sudo cat /etc/scylla/encryption_keys/secret_key + + Returns: + + .. code-block:: none + + AES/ECB/PKCS5Padding:192:8stVxW5ypYhNxsnRVS1A6suKhk0sG4Tj + + To display the system key parameters: + + .. code-block:: none + + sudo cat /etc/scylla/encryption_keys/system_keys/system_key + + Returns: + + .. code-block:: none + + AES/CBC/PKCS5Padding:128:GGpOSxTGhtPRPLrNPYvVMQ== + + + Once you have created a key, copy the key to each node, using the procedure described in `Copy keys to nodes`_. + +Copy keys to nodes +====================== + +Every key you generate needs to be copied to the nodes for use in local key providers. + +**Procedure** + +#. Securely copy the key file, using ``scp`` or similar, to the same path on all nodes in the cluster. Make sure the key on each target node is moved to the same location as the source directory and that the target directory has the same permissions as the source directory. + +#. Repeat for all nodes in the cluster. + +.. _encryption-at-rest-set-kmip: + +Set the KMIP Host +---------------------- + +If you are using :term:`KMIP ` to encrypt tables or system information, add the KMIP server information to the ``scylla.yaml`` configuration file. + +#. Edit the ``scylla.yaml`` file located in ``/etc/scylla/`` and add the following in KMIP host(s) section: + + .. code-block:: yaml + + # + # kmip_hosts: + # : + # hosts: [, ...] + # certificate: (optional) + # keyfile: (optional; it is required if "certificate" is set) + # truststore: (optional) + # certficate_revocation_list: (optional) + # priority_string: + # username: (optional> + # password: (optional) + # max_command_retries: (optional; default 3) + # key_cache_expiry: + # key_cache_refresh: + # : + + Where: + + * ```` - The cluster name. + * ``hosts`` - The list of hosts specified by IP and port for the KMIP server. The KMIP connection management only supports failover, so all requests go through a single KMIP server. There is no load balancing, as currently no KMIP servers support read replication or other strategies for availability. Hosts are tried in the order they appear, and the next one in the list is tried if the previous one fails. The default number of retries is three, but you can customize it with "max_command_retries". + * ``certificate`` - The name of the certificate and path used to identify yourself to the KMIP server. + * ``keyfile`` - The name of the key used to identify yourself to the KMIP server. It is generated together with the certificate. + * ``truststore`` - The location and key for the truststore to present to the KMIP server. + * ``certficate_revocation_list`` - The path to a PEM-encoded certificate revocation list (CRL) - a list of issued certificates that have been revoked before their expiration date. + * ``priority_string`` - The KMIP TLS priority string. + * ``username`` - The KMIP server user name. + * ``password`` - The KMIP server password. + * ``max_command_retries`` - The number of attempts to connect to the KMIP server before trying the next host in the list. + * ``key_cache_expiry`` - Key cache expiry period, after which keys will be re-requested from server. Default is 600s. + * ``key_cache_refresh`` - Key cache refresh period - the frequency at which cache is checked for expired entries. Default is 1200s. + +#. Save the file. +#. Drain the node with :doc:`nodetool drain ` +#. Restart the scylla-server service. + +.. include:: /rst_include/scylla-commands-restart-index.rst + +.. _encryption-at-rest-set-kms: + +Set the KMS Host +---------------------- + +.. note:: KMS support is available since ScyllaDB Enterprise **2023.1.1**. + +If you are using AWS KMS to encrypt tables or system information, add the KMS information to the ``scylla.yaml`` configuration file. + +#. Edit the ``scylla.yaml`` file located in ``/etc/scylla/`` to add the following in KMS host(s) section: + + .. code-block:: yaml + + kms_hosts: + : + endpoint: http(s)://(:port) (optional if `aws_region` is specified) + aws_region: (optional if `endpoint` is specified) + aws_access_key_id: (optional) + aws_secret_access_key: (optional) + aws_profile: (optional) + aws_use_ec2_credentials: (bool : default false) + aws_use_ec2_region: (bool : default false) + aws_assume_role_arn: (optional) + master_key: (required) + certificate: (optional) + keyfile: (optional) + truststore: (optional) + priority_string: (optional) + key_cache_expiry: + key_cache_refresh: + # : + + Where: + + * ```` - The name to identify the KMS host. You have to provide this name to encrypt a :ref:`new ` or :ref:`existing ` table. + * ``endpoint`` - The explicit KMS host endpoint. If not provided, ``aws_region`` is used for connection. + * ``aws_region`` - An AWS region. If not provided, ``endpoint`` is used for connection. + * ``aws_access_key_id`` - AWS access key used for authentication. If not specified, the provider reads it from your AWS credentials. + * ``aws_secret_access_key`` - AWS secret access key used for authentication. If not specified, the provider reads it from your AWS credentials. + * ``aws_profile`` - AWS profile to use if reading credentials from file + * ``aws_use_ec2_credentials`` - If true, KMS queries will use the credentials provided by ec2 instance role metadata as initial access key. + * ``aws_use_ec2_region`` - If true, KMS queries will use the AWS region indicated by ec2 instance metadata. + * ``aws_assume_role_arn`` - If set, any KMS query will first attempt to assume this role. + * ``master_key`` - The ID or alias of your AWS KMS key. The key must be generated with an appropriate access policy so that the AWS user has permissions to read the key and encrypt data using that key. This parameter is required. + * ``certificate`` - The name of the certificate and the path used to identify yourself to the KMS server. + * ``keyfile`` - The name of the key for the certificate. It is generated together with the certificate. + * ``truststore`` - The location and key for the truststore to present to the KMS server. + * ``priority_string`` - The KMS TLS priority string. + * ``key_cache_expiry`` - Key cache expiry period, after which keys will be re-requested from server. Default is 600s. + * ``key_cache_refresh`` - Key cache refresh period - the frequency at which cache is checked for expired entries. Default is 1200s. + + .. note:: + + Note that either ``endpoint``, ``aws_region`` or ``aws_use_ec2_region`` must be set (one of them is required for connection). + + Example: + + .. code-block:: yaml + + kms_hosts: + my-kms1: + aws_use_ec2_credentials: true + aws_use_ec2_region: true + master_key: myorg/MyKey + +#. Save the file. +#. Drain the node with :doc:`nodetool drain ` +#. Restart the scylla-server service. + +.. include:: /rst_include/scylla-commands-restart-index.rst + +.. _encryption-at-rest-set-gcp: + +Set the GCP Host +---------------------- + +If you are using Google GCP KMS to encrypt tables or system information, add the GCP information to the ``scylla.yaml`` configuration file. + +#. Edit the ``scylla.yaml`` file located in ``/etc/scylla/`` to add the following in KMS host(s) section: + + .. code-block:: yaml + + gcp_hosts: + : + gcp_project_id: + gcp_location: + gcp_credentials_file: <(service) account json key file - authentication> + gcp_impersonate_service_account: + master_key: / - named GCP key for encrypting data keys (required) + certificate: (optional) + keyfile: (optional) + truststore: (optional) + priority_string: (optional) + key_cache_expiry: + key_cache_refresh: + # : + + Where: + + * ```` - The name to identify the GCP host. You have to provide this name to encrypt a :ref:`new ` or :ref:`existing ` table. + * ``gcp_project_id`` - The GCP project from which to retrieve key information. + * ``gcp_location`` - A GCP project location. + * ``gcp_credentials_file`` - GCP credentials file used for authentication. If not specified, the provider reads it from your GCP credentials. + * ``gcp_impersonate_service_account`` - An optional service account to impersonate when issuing key query calls. + * ``master_key`` - The / of your GCP KMS key. The key must be generated with an appropriate access policy so that the AWS user has permissions to read the key and encrypt data using that key. This parameter is required. + * ``certificate`` - The name of the certificate and the path used to identify yourself to the KMS server. + * ``keyfile`` - The name of the key for the certificate. It is generated together with the certificate. + * ``truststore`` - The location and key for the truststore to present to the KMS server. + * ``priority_string`` - The KMS TLS priority string. + * ``key_cache_expiry`` - Key cache expiry period, after which keys will be re-requested from server. Default is 600s. + * ``key_cache_refresh`` - Key cache refresh period - the frequency at which cache is checked for expired entries. Default is 1200s. + + Example: + + .. code-block:: yaml + + gcp_hosts: + my-gcp1: + gcp_project_id: myproject + gcp_location: global + master_key: mykeyring/mykey + +#. Save the file. +#. Drain the node with :doc:`nodetool drain ` +#. Restart the scylla-server service. + +.. include:: /rst_include/scylla-commands-restart-index.rst + +Encrypt Tables +----------------------------- + +.. note:: + + This feature is available since ScyllaDB Enterprise 2023.1.2. + +ScyllaDB allows you to enable or disable default encryption of tables. +When enabled, tables will be encrypted by default using the configuration +provided for the ``user_info_encryption`` option in the ``scylla.yaml`` file. + +You can override the default configuration when you CREATE TABLE or ALTER TABLE +with ``scylla_encryption_options``. See :ref:`Encrypt a Single Table ` +for details. + +**Before you Begin** + +Ensure you have an encryption key available: + +* If you are using AWS KMS, :ref:`set the KMS Host `. +* If you are using KMIP, :ref:`set the KMIP Host `. +* If you are using Google GCP KMS, :ref:`set the GCP Host `. +* If you want to create your own key, follow the procedure in :ref:`Create Encryption Keys `. +* If you do not create your own key, use the following procedure for ScyllaDB + to create a key for you (the default location ``/etc/scylla/data_encryption_keys`` may cause + permission issues; the following example creates a key in the directory ``/etc/scylla/encryption_keys``): + + .. code-block:: none + + sudo mkdir -p /etc/scylla/encryption_keys + sudo chown -R scylla:scylla /etc/scylla/encryption_keys + sudo chmod -R 700 /etc/scylla/encryption_keys + +**Procedure** + +Edit the ``scylla.yaml`` file located in ``/etc/scylla/`` and configure +the ``user_info_encryption`` option: + +.. code-block:: yaml + + user_info_encryption: + enabled: + cipher_algorithm: + secret_key_strength: + key_provider: + secret_key_file: + kmip_host: + kms_host: + gcp_host: + +Where: + +* ``enabled`` - Enables or disables default table encryption. Required. +* ``cipher_algorithm`` - One of the :ref:`cipher algorithms `. + If not provided, the default will be used. +* ``secret_key_strength`` - The length of the key in bytes ( determined by + the :ref:`cipher algorithms ` you choose). + If not provided, the default will be used. +* ``key_provider`` - The name of the key provider. See :ref:`Key Providers `. + Required. +* ``secret_key_file`` - The location of the key created by ScyllaDB (by default ``/etc/scylla/data_encryption_keys``). + Required if you use a ScyllaDB-generated key. +* ``kmip_host`` - The name of your :ref:`kmip_host ` group. + Required if you use KMIP. +* ``kms_host`` - The name of your :ref:`kms_host ` group. + Required if you use KMS. +* ``gcp_host`` - The name of your :ref:`gcp_host ` group. + Required if you use GCP. + +**Example** + +.. code-block:: yaml + + user_info_encryption: + enabled: true + cipher_algorithm: AES + secret_key_strength: 128 + key_provider: LocalFileSystemKeyProviderFactory + secret_key_file: scylla /etc/scylla/encryption_keys + +**Examples for KMS:** + +In the following example, the ``master_key`` configured for :ref:`kms_host ` will be used. + +.. code-block:: yaml + + user_info_encryption: + enabled: true + key_provider: KmsKeyProviderFactory + kms_host: my-kms1 + + +You can specify a different ``master_key`` than the one configured for :ref:`kms_host `: + + .. code-block:: yaml + + user_info_encryption: + enabled: true + key_provider: KmsKeyProviderFactory + kms_host: my-kms1 + master_key: myorg/SomeOtherKey + +.. _ear-create-table: + +Encrypt a Single Table +----------------------------- + +This procedure demonstrates how to encrypt a new table. + +**Before you Begin** + +* Make sure to `Set the KMIP Host`_ if you are using KMIP, or the the :ref:`KMS Host ` if you are using AWS KMS. + +* If you want to make your own key, use the procedure in `Create Encryption Keys`_ and skip to step 3. If you do not create your own key, ScyllaDB will create one for you in the ``secret_key_file`` path. If you are not creating your own key, start with step 1. + +**Procedure** + +#. By default, the encryption key is located in the ``/etc/scylla/`` directory, and the file is named ``data_encryption_keys``. If you want to save the key in a different directory, create one. This example will create encryption keys in a different directory (``/etc/scylla/encryption_keys``, for example), which ensures that the owner of this directory is ``scylla`` and not another user. + + .. note:: Using the default location results in a known permission issue (scylladb/scylla-tools-java#94), so it is recommended to use another location as described in the example. + + .. code-block:: none + + sudo mkdir -p /etc/scylla/encryption_keys + sudo chown -R scylla:scylla /etc/scylla/encryption_keys + sudo chmod -R 700 /etc/scylla/encryption_keys + +#. Create the keyspace if it doesn’t exist. + +#. Create the table using the ``CREATE TABLE`` CQL statement, adding any :ref:`additional options `. To encrypt the table, use the options for encryption below, remembering to set the ``secret_key_file `` to the same directory you created in step 1. + + .. code-block:: cql + + CREATE TABLE . (......) WITH + scylla_encryption_options = { + 'cipher_algorithm' : , + 'secret_key_strength' : , + 'key_provider': , + 'secret_key_file': + } + ; + + Where: + + * ``cipher_algorithm`` - The hashing algorithm which is to be used to create the key. See `Cipher Algorithms`_ for more information. + * ``secret_key_strength`` - The length of the key in bytes. This is determined by the cipher you choose. See `Cipher Algorithms`_ for more information. + * ``key_provider`` is the name or type of key provider. Refer to `Key Providers`_ for more information. + * ``secret_key_file`` - the location that ScyllaDB will store the key it creates (if one does not exist in this location) or the location of the key. By default the location is ``/etc/scylla/data_encryption_keys``. + + **Example:** + + Continuing the example from above, this command will instruct ScyllaDB to encrypt the table and will save the key in the location created in step 1. + + .. code-block:: cql + + CREATE TABLE data.atrest (pk text primary key, c0 int) WITH + scylla_encryption_options = { + 'cipher_algorithm' : 'AES/ECB/PKCS5Padding', + 'secret_key_strength' : 128, + 'key_provider': 'LocalFileSystemKeyProviderFactory', + 'secret_key_file': '/etc/scylla/encryption_keys/data_encryption_keys' + } + ; + + **Example for KMS:** + + .. code-block:: cql + + CREATE TABLE myks.mytable (......) WITH + scylla_encryption_options = { + 'cipher_algorithm' : 'AES/CBC/PKCS5Padding', + 'secret_key_strength' : 128, + 'key_provider': 'KmsKeyProviderFactory', + 'kms_host': 'my-kms1' + } + ; + + You can skip ``cipher_algorithm`` and ``secret_key_strength`` (the :ref:`defaults ` will be used): + + .. code-block:: cql + + CREATE TABLE myks.mytable (......) WITH + scylla_encryption_options = { + 'key_provider': 'KmsKeyProviderFactory', + 'kms_host': 'my-kms1' + } + ; + + You can specify a different master key than the one configured for ``kms_host`` in the ``scylla.yaml`` file: + + .. code-block:: cql + + CREATE TABLE myks.mytable (......) WITH + scylla_encryption_options = { + 'key_provider': 'KmsKeyProviderFactory', + 'kms_host': 'my-kms1', + 'master_key':'myorg/SomeOtherKey' + } + ; + + +#. From this point, every new SSTable created for the ``atrest`` table is encrypted, using the ``data_encryption_keys`` key located in ``/etc/scylla/encryption_keys/``. This table will remain encrypted with this key until you either change the key, change the key properties, or disable encryption. + +#. To ensure all SSTables for this table on every node are encrypted, run the :doc:`Nodetool upgradesstables ` command. If not, the SSTables remain unencrypted until they are compacted or flushed from MemTables. + + For Example: + + .. code-block:: none + + nodetool upgradesstables data atrest + +#. Your SSTables are encrypted. If you want to change the key at any point, use the `Update Encryption Properties of Existing Tables`_ procedure. Always keep your key in a safe location known to you. Do not lose it. See `When a Key is Lost`_. + +.. _ear-alter-table: + +Update Encryption Properties of Existing Tables +================================================== + +You can encrypt any existing table or use this procedure to change the cipher algorithm, key location or key strength or even disable encryption on a table. + +**Procedure** + +#. Edit the table properties to enable encryption of one table of your choosing. Use the properties explained in `Encrypt a Single Table`_ if needed. + + .. code-block:: cql + + ALTER TABLE . (......) WITH + scylla_encryption_options = { + 'cipher_algorithm' : , + 'secret_key_strength' : , + 'key_provider': , + 'secret_key_file': + } + ; + + + **Example:** + + Continuing the example from above, this command will instruct ScyllaDB to encrypt the table and will save the key in the location created in step 1. + + .. code-block:: cql + + ALTER TABLE data.atrest (pk text primary key, c0 int) WITH + scylla_encryption_options = { + 'cipher_algorithm' : 'AES/ECB/PKCS5Padding', + 'secret_key_strength' : 192, + 'key_provider': 'LocalFileSystemKeyProviderFactory', + 'secret_key_file': '/etc/scylla/encryption_keys/data_encryption_keys' + } + ; + + **Example for KMS:** + + .. code-block:: cql + + ALTER TABLE myks.mytable (......) WITH + scylla_encryption_options = { + 'cipher_algorithm' : 'AES/CBC/PKCS5Padding', + 'secret_key_strength' : 128, + 'key_provider': 'KmsKeyProviderFactory', + 'kms_host': 'my-kms1' + } + ; + +#. If you want to make sure that SSTables that existed before this change are also encrypted, you can either upgrade them using the ``nodetool upgradesstables`` command or wait until the next compaction. If you decide to wait, ScyllaDB will still be able to read the old unencrypted tables. If you change the key or remove encryption, ScyllaDB will still continue to read the old tables as long as you still have the key. If your data is encrypted and you do not have the key, your data is unreadable. + + * If you decide to upgrade all of your old SSTables run the :doc:`nodetool upgradesstables ` command. + + .. code-block:: none + + nodetool upgradesstables
+ + For example: + + .. code-block:: none + + nodetool upgradesstables ks test + + * Repeat this command on all nodes as nodetool runs locally. + +#. If you want to change the key or disable encryption, repeat the `Update Encryption Properties of Existing Tables`_ procedure using the examples below as reference. + +**Examples** + +To encrypt an existing table named test in keyspace ks: + +.. code-block:: cql + + ALTER TABLE ks.test WITH + scylla_encryption_options = { + 'cipher_algorithm' : 'AES/ECB/PKCS5Padding', + 'secret_key_strength' : 128, + 'key_provider': 'LocalFileSystemKeyProviderFactory', + 'secret_key_file': '/etc/scylla/encryption_keys/data_encryption_keys' + } + ; + + +To change the cipher algorithm from AES/ECB/PKCS5Padding to AES/ECB/PKCS5Padding and to change the key strength from 128 to 192 on an existing table: + +.. code-block:: cql + + ALTER TABLE ks.test WITH + scylla_encryption_options = { + 'cipher_algorithm' : 'AES/ECB/PKCS5Padding', + 'secret_key_strength' : 192, + 'key_provider': 'LocalFileSystemKeyProviderFactory', + 'secret_key_file': '/etc/scylla/encryption_keys/data_encryption_keys' + } + ; + +To disable encryption on an encrypted table named test in keyspace ks: + +.. code-block:: cql + + ALTER TABLE ks.test WITH + scylla_encryption_options = { 'key_provider' : 'none’ }; + + +Encrypt System Resources +--------------------------- + +System encryption is applied to semi-transient on-disk data, such as commit logs, batch logs, and hinted handoff data. +This ensures that all temporarily stored data is encrypted until fully persisted to final SSTable on disk. +Once this encryption is enabled, it is used for all system data. + + +**Procedure** + +#. Edit the scylla.yaml file - located in /etc/scylla/scylla.yaml and add the following: + + .. code-block:: none + + system_info_encryption: + enabled: + key_provider: (optional) + system_key_directory: + + Where: + + * ``enabled`` can be true or false. True is enabled; false is disabled. + + * ``key_provider`` is the name or type of key provider. Refer to `Key Providers`_ for more information. + + * ``cipher_algorithm`` is one of the supported `Cipher Algorithms`_. + + * ``secret_key_file`` is the name of the key file containing the secret key (key.pem, for example) + + Example: + + .. code-block:: none + + system_info_encryption: + enabled: True + cipher_algorithm: AES + secret_key_strength: 128 + key_provider: LocalFileSystemKeyProviderFactory + secret_key_file: /path/to/systemKey.pem + + Example for KMIP: + + .. code-block:: none + + system_info_encryption: + enabled: True + cipher_algorithm: AES + secret_key_strength: 128 + key_provider: KmipKeyProviderFactory + kmip_host: yourkmipServerIP.com + + Where ``kmip_host`` is the address for your KMIP server. + + Example for KMS: + + .. code-block:: none + + system_info_encryption: + enabled: True + cipher_algorithm: AES/CBC/PKCS5Padding + secret_key_strength: 128 + key_provider: KmsKeyProviderFactory + kms_host: myScylla + + Where ``kms_host`` is the unique name of the KMS host specified in the scylla.yaml file. + + Example for GCP: + + .. code-block:: none + + system_info_encryption: + enabled: True + cipher_algorithm: AES/CBC/PKCS5Padding + secret_key_strength: 128 + key_provider: GcpKeyProviderFactory + gcp_host: myScylla + + Where ``gcp_host`` is the unique name of the GCP host specified in the scylla.yaml file. + + +#. Do not close the yaml file. Change the system key directory location according to your settings. + + * ``system_key_directory`` is the location of the system key you created in `Create Encryption Keys`_. + + .. code-block:: none + + system_key_directory: /etc/scylla/encryption_keys/system_keys + +#. Save the file. +#. Drain the node with :doc:`nodetool drain ` +#. Restart the scylla-server service. + + .. include:: /rst_include/scylla-commands-restart-index.rst + + .. wasn't able to test this successfully + +.. Encrypt and Decrypt Configuration Files +.. ======================================= + +.. Using the Configuration Encryption tool, you can encrypt parts of the scylla.yaml file which contain encryption configuration settings. + +.. **Procedure** + +.. 1. Run the Configuration Encryption script: + +.. test code-block: none + +.. /bin/configuration_encryptor [options] [key-path] + +.. Where: + +.. * ``-c, --config`` - the path to the configuration file (/etc/scylla/scylla.yaml, for example) +.. * ``-d, --decrypt`` - decrypts the configuration file at the specified path +.. * ``-o, --output`` - (optional) writes the configuration file to a specified target. This can be the same location as the source file. +.. * ``-h. --help`` - help for this command + +.. For example: + +.. test code-block: none + +.. sudo -u scylla /bin/configuration_encryptor -c /etc/scylla/scylla.yaml /etc/scylla/encryption_keys/secret_key +.. end of test + +When a Key is Lost +---------------------- + +It is crucial to back up all of your encryption keys in a secure way. Keep a copy of all keys in a secure location. In the event that you do lose a key, your data encrypted with that key will be unreadable. + +Additional Resources +---------------------- + +* :doc:`nodetool upgradesstables ` +* :ref:`CREATE TABLE parameters ` diff --git a/docs/operating-scylla/security/index.rst b/docs/operating-scylla/security/index.rst index 6d9c4cf38072..171024b7eb5d 100644 --- a/docs/operating-scylla/security/index.rst +++ b/docs/operating-scylla/security/index.rst @@ -18,6 +18,7 @@ Security node-node-encryption generate-certificate saslauthd + encryption-at-rest .. panel-box:: @@ -50,6 +51,6 @@ Security * :doc:`Encryption: Data in Transit Client to Node ` * :doc:`Encryption: Data in Transit Node to Node ` * :doc:`Generating a self-signed Certificate Chain Using openssl ` - * `Encryption at Rest `_ available in `ScyllaDB Enterprise `_ + * :doc:`Encryption at Rest ` Also check out the `Security Features lesson `_ on ScyllaDB University. diff --git a/docs/operating-scylla/security/security-checklist.rst b/docs/operating-scylla/security/security-checklist.rst index a71e712de49f..737b07a82613 100644 --- a/docs/operating-scylla/security/security-checklist.rst +++ b/docs/operating-scylla/security/security-checklist.rst @@ -41,14 +41,16 @@ Configure ScyllaDB to use TLS/SSL for all the connections. Use TLS/SSL to encryp * :doc:`Encryption Data in Transit Node to Node ` -Encryption at Rest :label-tip:`ScyllaDB Enterprise` -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -Encryption at Rest is available in `ScyllaDB Enterprise `_. +Encryption at Rest +~~~~~~~~~~~~~~~~~~ +Encryption at Rest is available in a Scylla Enterprise 2019.1.1. Encryption at Rest protects the privacy of your user's data, reduces the risk of data breaches, and helps meet regulatory requirements. In particular, it provides an additional level of protection for your data persisted in storage or backup. -See `Encryption at Rest `_ for details. +See: + +* :doc:`Encryption at Rest ` Reduce the Network Exposure ~~~~~~~~~~~~~~~~~~~~~~~~~~~ diff --git a/docs/reference/glossary.rst b/docs/reference/glossary.rst index ae4c93d1f04d..627d2de732d1 100644 --- a/docs/reference/glossary.rst +++ b/docs/reference/glossary.rst @@ -58,6 +58,9 @@ Glossary Keyspace A collection of tables with attributes which define how data is replicated on nodes. See :doc:`Ring Architecture `. + Key Management Interoperability Protocol (KMIP) + :abbr:`KMIP (Key Management Interoperability Protocol)` is a communication protocol that defines message formats for storing keys on a key management server (KMIP server). You can use a KMIP server to protect your keys when using Encryption at Rest. See :doc:`Encryption at Rest`. + Leveled compaction strategy (LCS) :abbr:`LCS (Leveled compaction strategy)` uses small, fixed-size (by default 160 MB) SSTables divided into different levels. See :doc:`Compaction Strategies`. From 2a8ff478f0e2900843c47a2fa04f6d85d8edd21b Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Sun, 5 Jan 2025 14:58:01 +0200 Subject: [PATCH 233/397] view_builder: register listener for new views before reading views When starting the view builder, we find all existing views in `calculate_shard_build_step` and then register a listener for new views. Between these steps we may yield and create a new view, then we miss initializing the view build step for the new view, and we won't start building it. To fix this we first register the listener and then read existing views, so a view can't be missed. Fixes scylladb/scylladb#20338 Closes scylladb/scylladb#22184 --- db/view/view.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/db/view/view.cc b/db/view/view.cc index dd68531e6492..03e654d2dfb6 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -2052,8 +2052,8 @@ future<> view_builder::start_in_background(service::migration_manager& mm, utils co_await barrier.arrive_and_wait(); units.return_all(); - co_await calculate_shard_build_step(vbi); _mnotifier.register_listener(this); + co_await calculate_shard_build_step(vbi); _current_step = _base_to_build_step.begin(); // Waited on indirectly in stop(). (void)_build_step.trigger(); From b6b4b767dec4985bd5726aba638496272ebf6ef1 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Tue, 3 Dec 2024 16:44:06 +0100 Subject: [PATCH 234/397] locator: extend tablet_task_info to cover resize tasks --- locator/tablets.cc | 14 ++++++++++++++ locator/tablets.hh | 6 +++++- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/locator/tablets.cc b/locator/tablets.cc index cafe0fe22aa8..410a1fca1571 100644 --- a/locator/tablets.cc +++ b/locator/tablets.cc @@ -558,6 +558,8 @@ static const std::unordered_map tablet_task_type_to_n {locator::tablet_task_type::auto_repair, "auto_repair"}, {locator::tablet_task_type::migration, "migration"}, {locator::tablet_task_type::intranode_migration, "intranode_migration"}, + {locator::tablet_task_type::split, "split"}, + {locator::tablet_task_type::merge, "merge"}, }; static const std::unordered_map tablet_task_type_from_name = std::invoke([] { @@ -1175,3 +1177,15 @@ locator::tablet_task_info locator::tablet_task_info::make_intranode_migration_re auto tablet_task_id = locator::tablet_task_id(utils::UUID_gen::get_time_UUID()); return locator::tablet_task_info{locator::tablet_task_type::intranode_migration, tablet_task_id, db_clock::now(), sched_nr, db_clock::time_point()}; } + +locator::tablet_task_info locator::tablet_task_info::make_split_request() { + long sched_nr = 0; + auto tablet_task_id = locator::tablet_task_id(utils::UUID_gen::get_time_UUID()); + return locator::tablet_task_info{locator::tablet_task_type::split, tablet_task_id, db_clock::now(), sched_nr, db_clock::time_point()}; +} + +locator::tablet_task_info locator::tablet_task_info::make_merge_request() { + long sched_nr = 0; + auto tablet_task_id = locator::tablet_task_id(utils::UUID_gen::get_time_UUID()); + return locator::tablet_task_info{locator::tablet_task_type::merge, tablet_task_id, db_clock::now(), sched_nr, db_clock::time_point()}; +} diff --git a/locator/tablets.hh b/locator/tablets.hh index f4f6fbeef08b..46b1883c790c 100644 --- a/locator/tablets.hh +++ b/locator/tablets.hh @@ -148,7 +148,9 @@ enum class tablet_task_type { user_repair, auto_repair, migration, - intranode_migration + intranode_migration, + split, + merge }; sstring tablet_task_type_to_string(tablet_task_type); @@ -167,6 +169,8 @@ struct tablet_task_info { static tablet_task_info make_auto_repair_request(); static tablet_task_info make_migration_request(); static tablet_task_info make_intranode_migration_request(); + static tablet_task_info make_split_request(); + static tablet_task_info make_merge_request(); }; /// Stores information about a single tablet. From 18b829add82a06a4cee4a47bbb04883c9d211e6d Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 4 Dec 2024 11:13:04 +0100 Subject: [PATCH 235/397] replica: service: add resize_task_info static column to system.tablets Add resize_task_info static column to system.tablets. Set or delete resize_task_info value when the resize_decision is changed. Reflect the column content in tablet_map. --- gms/feature_service.hh | 1 + locator/tablets.cc | 8 +++++++ locator/tablets.hh | 3 +++ replica/tablet_mutation_builder.hh | 4 +++- replica/tablets.cc | 36 +++++++++++++++++++++++++++++- service/topology_coordinator.cc | 2 +- test/boost/tablets_test.cc | 3 ++- 7 files changed, 53 insertions(+), 4 deletions(-) diff --git a/gms/feature_service.hh b/gms/feature_service.hh index a4fc300ad304..c52337b9f788 100644 --- a/gms/feature_service.hh +++ b/gms/feature_service.hh @@ -150,6 +150,7 @@ public: gms::feature tablet_merge { *this, "TABLET_MERGE"sv }; gms::feature tablet_migration_virtual_task { *this, "TABLET_MIGRATION_VIRTUAL_TASK"sv }; + gms::feature tablet_resize_virtual_task { *this, "TABLET_RESIZE_VIRTUAL_TASK"sv }; // A feature just for use in tests. It must not be advertised unless // the "features_enable_test_feature" injection is enabled. diff --git a/locator/tablets.cc b/locator/tablets.cc index 410a1fca1571..29ac9e547e62 100644 --- a/locator/tablets.cc +++ b/locator/tablets.cc @@ -425,6 +425,10 @@ void tablet_map::set_resize_decision(locator::resize_decision decision) { _resize_decision = std::move(decision); } +void tablet_map::set_resize_task_info(tablet_task_info task_info) { + _resize_task_info = std::move(task_info); +} + void tablet_map::set_repair_scheduler_config(locator::repair_scheduler_config config) { _repair_scheduler_config = std::move(config); } @@ -606,6 +610,10 @@ const locator::resize_decision& tablet_map::resize_decision() const { return _resize_decision; } +const tablet_task_info& tablet_map::resize_task_info() const { + return _resize_task_info; +} + const locator::repair_scheduler_config& tablet_map::repair_scheduler_config() const { return _repair_scheduler_config; } diff --git a/locator/tablets.hh b/locator/tablets.hh index 46b1883c790c..4a68c1916fd0 100644 --- a/locator/tablets.hh +++ b/locator/tablets.hh @@ -400,6 +400,7 @@ private: size_t _log2_tablets; // log_2(_tablets.size()) std::unordered_map _transitions; resize_decision _resize_decision; + tablet_task_info _resize_task_info; repair_scheduler_config _repair_scheduler_config; /// Returns the largest token owned by tablet_id when the tablet_count is `1 << log2_tablets`. @@ -522,11 +523,13 @@ public: dht::token_range get_token_range_after_split(const token& t) const noexcept; const locator::resize_decision& resize_decision() const; + const tablet_task_info& resize_task_info() const; const locator::repair_scheduler_config& repair_scheduler_config() const; public: void set_tablet(tablet_id, tablet_info); void set_tablet_transition_info(tablet_id, tablet_transition_info); void set_resize_decision(locator::resize_decision); + void set_resize_task_info(tablet_task_info); void set_repair_scheduler_config(locator::repair_scheduler_config config); void clear_tablet_transition_info(tablet_id); void clear_transitions(); diff --git a/replica/tablet_mutation_builder.hh b/replica/tablet_mutation_builder.hh index ff7d9e98f230..97715248f158 100644 --- a/replica/tablet_mutation_builder.hh +++ b/replica/tablet_mutation_builder.hh @@ -39,13 +39,15 @@ public: tablet_mutation_builder& set_session(dht::token last_token, service::session_id); tablet_mutation_builder& del_session(dht::token last_token); tablet_mutation_builder& del_transition(dht::token last_token); - tablet_mutation_builder& set_resize_decision(locator::resize_decision); + tablet_mutation_builder& set_resize_decision(locator::resize_decision, const gms::feature_service&); tablet_mutation_builder& set_repair_scheduler_config(locator::repair_scheduler_config); tablet_mutation_builder& set_repair_time(dht::token last_token, db_clock::time_point repair_time); tablet_mutation_builder& set_repair_task_info(dht::token last_token, locator::tablet_task_info info); tablet_mutation_builder& del_repair_task_info(dht::token last_token); tablet_mutation_builder& set_migration_task_info(dht::token last_token, locator::tablet_task_info info, const gms::feature_service& features); tablet_mutation_builder& del_migration_task_info(dht::token last_token, const gms::feature_service& features); + tablet_mutation_builder& set_resize_task_info(locator::tablet_task_info info, const gms::feature_service& features); + tablet_mutation_builder& del_resize_task_info(const gms::feature_service& features); mutation build() { return std::move(_m); diff --git a/replica/tablets.cc b/replica/tablets.cc index b7d75451fbe2..46011b7a6e18 100644 --- a/replica/tablets.cc +++ b/replica/tablets.cc @@ -75,6 +75,7 @@ schema_ptr make_tablets_schema() { .with_column("repair_task_info", tablet_task_info_type) .with_column("repair_scheduler_config", repair_scheduler_config_type, column_kind::static_column) .with_column("migration_task_info", tablet_task_info_type) + .with_column("resize_task_info", tablet_task_info_type, column_kind::static_column) .with_hash_version() .build(); } @@ -126,6 +127,9 @@ tablet_map_to_mutation(const tablet_map& tablets, table_id id, const sstring& ke m.set_static_cell("table_name", data_value(table_name), ts); m.set_static_cell("resize_type", data_value(tablets.resize_decision().type_name()), ts); m.set_static_cell("resize_seq_number", data_value(int64_t(tablets.resize_decision().sequence_number)), ts); + if (features.tablet_resize_virtual_task && tablets.resize_task_info().is_valid()) { + m.set_static_cell("resize_task_info", tablet_task_info_to_data_value(tablets.resize_task_info()), ts); + } if (features.tablet_repair_scheduler) { m.set_static_cell("repair_scheduler_config", repair_scheduler_config_to_data_value(tablets.repair_scheduler_config()), ts); } @@ -212,9 +216,19 @@ tablet_mutation_builder::del_transition(dht::token last_token) { } tablet_mutation_builder& -tablet_mutation_builder::set_resize_decision(locator::resize_decision resize_decision) { +tablet_mutation_builder::set_resize_decision(locator::resize_decision resize_decision, const gms::feature_service& features) { _m.set_static_cell("resize_type", data_value(resize_decision.type_name()), _ts); _m.set_static_cell("resize_seq_number", data_value(int64_t(resize_decision.sequence_number)), _ts); + if (resize_decision.split_or_merge()) { + auto resize_task_info = std::holds_alternative(resize_decision.way) + ? locator::tablet_task_info::make_split_request() + : locator::tablet_task_info::make_merge_request(); + resize_task_info.sched_nr++; + resize_task_info.sched_time = db_clock::now(); + return set_resize_task_info(std::move(resize_task_info), features); + } else { + return del_resize_task_info(features); + } return *this; } @@ -260,6 +274,23 @@ tablet_mutation_builder::del_migration_task_info(dht::token last_token, const gm return *this; } +tablet_mutation_builder& +tablet_mutation_builder::set_resize_task_info(locator::tablet_task_info resize_task_info, const gms::feature_service& features) { + if (features.tablet_resize_virtual_task) { + _m.set_static_cell("resize_task_info", tablet_task_info_to_data_value(resize_task_info), _ts); + } + return *this; +} + +tablet_mutation_builder& +tablet_mutation_builder::del_resize_task_info(const gms::feature_service& features) { + if (features.tablet_resize_virtual_task) { + auto col = _s->get_column_definition("resize_task_info"); + _m.set_static_cell(*col, atomic_cell::make_dead(_ts, gc_clock::now())); + } + return *this; +} + mutation make_drop_tablet_map_mutation(table_id id, api::timestamp_type ts) { auto s = db::system_keyspace::tablets(); mutation m(s, partition_key::from_single_value(*s, @@ -546,6 +577,9 @@ struct tablet_metadata_builder { locator::resize_decision resize_decision(std::move(resize_type_name), resize_seq_number); current->map.set_resize_decision(std::move(resize_decision)); } + if (row.has("resize_task_info")) { + current->map.set_resize_task_info(deserialize_tablet_task_info(row.get_view("resize_task_info"))); + } if (row.has("repair_scheduler_config")) { auto config = deserialize_repair_scheduler_config(row.get_view("repair_scheduler_config")); diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index f2b3a5a68a11..bc25c3ca8aeb 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -1281,7 +1281,7 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { table_id, resize_decision.type_name(), resize_decision.sequence_number); out.emplace_back( replica::tablet_mutation_builder(guard.write_timestamp(), table_id) - .set_resize_decision(std::move(resize_decision)) + .set_resize_decision(std::move(resize_decision), _db.features()) .build()); } diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 3e4c241810f7..68b6eb2ea9f8 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -983,7 +983,7 @@ SEASTAR_TEST_CASE(test_mutation_builder) { tablet_replica {h2, 3}, }); b.del_transition(last_token); - b.set_resize_decision(resize_decision); + b.set_resize_decision(resize_decision, e.local_db().features()); e.local_db().apply({freeze(b.build())}, db::no_timeout).get(); } @@ -1006,6 +1006,7 @@ SEASTAR_TEST_CASE(test_mutation_builder) { expected_tmap.set_resize_decision(resize_decision); auto tm_from_disk = read_tablet_metadata(e.local_qp()).get(); + expected_tmap.set_resize_task_info(tm_from_disk.get_tablet_map(table1).resize_task_info()); BOOST_REQUIRE_EQUAL(expected_tmap, tm_from_disk.get_tablet_map(table1)); } }, tablet_cql_test_config()); From 5dfac9290c5820841679e5cf3936dbdcde939bc4 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 9 Jan 2025 13:43:54 +0100 Subject: [PATCH 236/397] tasks: drop noexcept from module::get_nodes --- node_ops/task_manager_module.cc | 2 +- node_ops/task_manager_module.hh | 2 +- tasks/task_manager.cc | 2 +- tasks/task_manager.hh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/node_ops/task_manager_module.cc b/node_ops/task_manager_module.cc index c3dd83fcf696..ae099b95a600 100644 --- a/node_ops/task_manager_module.cc +++ b/node_ops/task_manager_module.cc @@ -203,7 +203,7 @@ task_manager_module::task_manager_module(tasks::task_manager& tm, service::stora , _ss(ss) {} -std::set task_manager_module::get_nodes() const noexcept { +std::set task_manager_module::get_nodes() const { return std::ranges::join_view(std::to_array({ std::views::all(_ss._topology_state_machine._topology.normal_nodes), std::views::all(_ss._topology_state_machine._topology.transition_nodes)}) diff --git a/node_ops/task_manager_module.hh b/node_ops/task_manager_module.hh index 04be6161d4fb..bae4ff6a0ebc 100644 --- a/node_ops/task_manager_module.hh +++ b/node_ops/task_manager_module.hh @@ -67,7 +67,7 @@ private: public: task_manager_module(tasks::task_manager& tm, service::storage_service& ss) noexcept; - virtual std::set get_nodes() const noexcept override; + virtual std::set get_nodes() const override; }; } diff --git a/tasks/task_manager.cc b/tasks/task_manager.cc index 48cbb3004e72..5a19eb4c5326 100644 --- a/tasks/task_manager.cc +++ b/tasks/task_manager.cc @@ -527,7 +527,7 @@ const task_manager::tasks_collection& task_manager::module::get_tasks_collection return _tasks; } -std::set task_manager::module::get_nodes() const noexcept { +std::set task_manager::module::get_nodes() const { return {_tm.get_broadcast_address()}; } diff --git a/tasks/task_manager.hh b/tasks/task_manager.hh index 33d04cf80b2f..7c839f61bad2 100644 --- a/tasks/task_manager.hh +++ b/tasks/task_manager.hh @@ -332,7 +332,7 @@ public: tasks_collection& get_tasks_collection() noexcept; const tasks_collection& get_tasks_collection() const noexcept; // Returns a set of nodes on which some of virtual tasks on this module can have their children. - virtual std::set get_nodes() const noexcept; + virtual std::set get_nodes() const; future> get_stats(is_internal internal, std::function filter) const; void register_task(task_ptr task); From 3f6b9323620b39c8da0211920534b66114010816 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 11 Dec 2024 12:39:38 +0100 Subject: [PATCH 237/397] tasks: add task_manager::get_nodes Move an implementation of node_ops::task_manager_module::get_nodes to task_manager::get_nodes, so that it can be reused by other modules. --- node_ops/task_manager_module.cc | 9 +-------- service/storage_service.hh | 2 +- tasks/task_manager.cc | 16 ++++++++++++++++ tasks/task_manager.hh | 7 +++++++ 4 files changed, 25 insertions(+), 9 deletions(-) diff --git a/node_ops/task_manager_module.cc b/node_ops/task_manager_module.cc index ae099b95a600..4de9efbc7eaf 100644 --- a/node_ops/task_manager_module.cc +++ b/node_ops/task_manager_module.cc @@ -204,14 +204,7 @@ task_manager_module::task_manager_module(tasks::task_manager& tm, service::stora {} std::set task_manager_module::get_nodes() const { - return std::ranges::join_view(std::to_array({ - std::views::all(_ss._topology_state_machine._topology.normal_nodes), - std::views::all(_ss._topology_state_machine._topology.transition_nodes)}) - ) | std::views::transform([&ss = _ss] (auto& node) { - return ss.host2ip(locator::host_id{node.first.uuid()}); - }) | std::views::filter([&ss = _ss] (gms::inet_address ip) { - return ss._gossiper.is_alive(ip); - }) | std::ranges::to>(); + return get_task_manager().get_nodes(_ss); } } diff --git a/service/storage_service.hh b/service/storage_service.hh index ecf3ad7c8f96..bc89e727fdb0 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -1006,7 +1006,7 @@ private: friend class join_node_rpc_handshaker; friend class node_ops::node_ops_virtual_task; - friend class node_ops::task_manager_module; + friend class tasks::task_manager; friend class tablet_virtual_task; }; diff --git a/tasks/task_manager.cc b/tasks/task_manager.cc index 5a19eb4c5326..7014f2a32b36 100644 --- a/tasks/task_manager.cc +++ b/tasks/task_manager.cc @@ -20,6 +20,7 @@ #include "message/messaging_service.hh" #include "utils/assert.hh" #include "utils/overloaded_functor.hh" +#include "service/storage_service.hh" #include "tasks/task_handler.hh" #include "task_manager.hh" #include "tasks/virtual_task_hint.hh" @@ -491,6 +492,10 @@ task_manager& task_manager::module::get_task_manager() noexcept { return _tm; } +const task_manager& task_manager::module::get_task_manager() const noexcept { + return _tm; +} + abort_source& task_manager::module::abort_source() noexcept { return _as; } @@ -681,6 +686,17 @@ const task_manager::tasks_collection& task_manager::get_tasks_collection() const return _tasks; } +std::set task_manager::get_nodes(service::storage_service& ss) const { + return std::ranges::join_view(std::to_array({ + std::views::all(ss._topology_state_machine._topology.normal_nodes), + std::views::all(ss._topology_state_machine._topology.transition_nodes)}) + ) | std::views::transform([&ss] (auto& node) { + return ss.host2ip(locator::host_id{node.first.uuid()}); + }) | std::views::filter([&ss] (gms::inet_address ip) { + return ss._gossiper.is_alive(ip); + }) | std::ranges::to>(); +} + future> task_manager::get_virtual_task_children(task_id parent_id) { return container().map_reduce0([parent_id] (task_manager& tm) { return tm.get_local_tasks() | diff --git a/tasks/task_manager.hh b/tasks/task_manager.hh index 7c839f61bad2..92c23a0d0435 100644 --- a/tasks/task_manager.hh +++ b/tasks/task_manager.hh @@ -29,6 +29,10 @@ namespace repair { class task_manager_module; } +namespace service { +class storage_service; +} + namespace netw { class messaging_service; } @@ -322,6 +326,7 @@ public: uint64_t new_sequence_number() noexcept; task_manager& get_task_manager() noexcept; + const task_manager& get_task_manager() const noexcept; seastar::abort_source& abort_source() noexcept; gate& async_gate() noexcept; const std::string& get_name() const noexcept; @@ -385,6 +390,8 @@ public: const tasks_collection& get_tasks_collection() const noexcept; future> get_virtual_task_children(task_id parent_id); + std::set get_nodes(service::storage_service& ss) const; + module_ptr make_module(std::string name); void register_module(std::string name, module_ptr module); module_ptr find_module(std::string module_name); From a8d7f4d89a931ecead121bfa216779785f550988 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 11 Dec 2024 13:44:35 +0100 Subject: [PATCH 238/397] service: add service::task_manager_module::get_nodes --- service/storage_service.cc | 2 +- service/task_manager_module.cc | 7 ++++++- service/task_manager_module.hh | 6 +++++- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 242050b92782..c7be08edf470 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -193,7 +193,7 @@ storage_service::storage_service(abort_source& abort_source, , _group0(nullptr) , _node_ops_abort_thread(node_ops_abort_thread()) , _node_ops_module(make_shared(tm, *this)) - , _tablets_module(make_shared(tm)) + , _tablets_module(make_shared(tm, *this)) , _address_map(address_map) , _shared_token_metadata(stm) , _erm_factory(erm_factory) diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index 36e30dc2639f..c0a51395cccb 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -215,8 +215,13 @@ future> tablet_virtual_task::get_status_helper co_return std::nullopt; } -task_manager_module::task_manager_module(tasks::task_manager& tm) noexcept +task_manager_module::task_manager_module(tasks::task_manager& tm, service::storage_service& ss) noexcept : tasks::task_manager::module(tm, "tablets") + , _ss(ss) {} +std::set task_manager_module::get_nodes() const { + return get_task_manager().get_nodes(_ss); +} + } diff --git a/service/task_manager_module.hh b/service/task_manager_module.hh index 1fbce3aec3d2..c242c93e01a5 100644 --- a/service/task_manager_module.hh +++ b/service/task_manager_module.hh @@ -46,7 +46,11 @@ private: }; class task_manager_module : public tasks::task_manager::module { +private: + service::storage_service& _ss; public: - task_manager_module(tasks::task_manager& tm) noexcept; + task_manager_module(tasks::task_manager& tm, service::storage_service& ss) noexcept; + + std::set get_nodes() const override; }; } From 0df64e18fb571b3df3d451e60ad202b03ec7e57b Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 4 Dec 2024 12:30:21 +0100 Subject: [PATCH 239/397] service: extend tablet_virtual_task::get_stats Extend tablet_virtual_task::get_stats to list resize tasks. --- service/task_manager_module.cc | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index c0a51395cccb..7462eb52d965 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -25,6 +25,21 @@ static std::optional maybe_get_task_type(const locato return task_info.is_valid() && task_info.tablet_task_id.uuid() == task_id.uuid() ? std::make_optional(task_info.request_type) : std::nullopt; } +static sstring get_scope(locator::tablet_task_type task_type) { + switch (task_type) { + case locator::tablet_task_type::user_repair: + case locator::tablet_task_type::split: + case locator::tablet_task_type::merge: + return "table"; + case locator::tablet_task_type::auto_repair: + case locator::tablet_task_type::migration: + case locator::tablet_task_type::intranode_migration: + return "tablet"; + case locator::tablet_task_type::none: + on_internal_error(tasks::tmlogger, "attempted to get the scope for none task type"); + } +} + static std::optional maybe_make_task_stats(const locator::tablet_task_info& task_info, schema_ptr schema) { if (!task_info.is_valid()) { return std::nullopt; @@ -34,7 +49,7 @@ static std::optional maybe_make_task_stats(const locator::tab .task_id = tasks::task_id{task_info.tablet_task_id.uuid()}, .type = locator::tablet_task_type_to_string(task_info.request_type), .kind = tasks::task_kind::cluster, - .scope = task_info.is_user_repair_request() ? "table" : "tablet", + .scope = get_scope(task_info.request_type), .state = tasks::task_manager::task_state::running, .keyspace = schema->ks_name(), .table = schema->cf_name() @@ -134,6 +149,10 @@ future> tablet_virtual_task::get_stats() { auto schema = _ss._db.local().get_tables_metadata().get_table(table).schema(); std::unordered_map user_requests; std::unordered_map sched_num_sum; + auto resize_stats = maybe_make_task_stats(tmap.resize_task_info(), schema); + if (resize_stats) { + res.push_back(std::move(resize_stats.value())); + } co_await tmap.for_each_tablet([&] (locator::tablet_id tid, const locator::tablet_info& info) { auto repair_stats = maybe_make_task_stats(info.repair_task_info, schema); if (repair_stats) { From 78215d64d1e8d4e31f989769a1323b9f51522273 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 4 Dec 2024 12:51:09 +0100 Subject: [PATCH 240/397] service: extend tablet_virtual_task::contains Extend tablet_virtual_task::contains to check resize operations. Methods that do not support resize tasks return immediately if they are handling split or merge task. --- service/task_manager_module.cc | 31 ++++++++++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index 7462eb52d965..065a667a063d 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -60,14 +60,29 @@ static bool is_repair_task(const locator::tablet_task_type& task_type) { return task_type == locator::tablet_task_type::user_repair || task_type == locator::tablet_task_type::auto_repair; } +static bool is_migration_task(const locator::tablet_task_type& task_type) { + return task_type == locator::tablet_task_type::migration || task_type == locator::tablet_task_type::intranode_migration; +} + +static bool is_resize_task(const locator::tablet_task_type& task_type) { + return task_type == locator::tablet_task_type::split || task_type == locator::tablet_task_type::merge; +} + static bool tablet_id_provided(const locator::tablet_task_type& task_type) { - return !is_repair_task(task_type); + return is_migration_task(task_type); } future> tablet_virtual_task::contains(tasks::task_id task_id) const { auto tables = get_table_ids(); for (auto table : tables) { auto& tmap = _ss.get_token_metadata().tablets().get_tablet_map(table); + if (auto task_type = maybe_get_task_type(tmap.resize_task_info(), task_id); task_type.has_value()) { + co_return tasks::virtual_task_hint{ + .table_id = table, + .task_type = task_type.value(), + .tablet_id = std::nullopt, + }; + } std::optional tid = tmap.first_tablet(); for (const locator::tablet_info& info : tmap.tablets()) { auto task_type = maybe_get_task_type(info.repair_task_info, task_id).or_else([&] () { @@ -89,10 +104,18 @@ future> tablet_virtual_task::contains(ta future tablet_virtual_task::is_abortable(tasks::virtual_task_hint hint) const { auto task_type = hint.get_task_type(); + if (is_resize_task(task_type)) { + return make_ready_future(tasks::is_abortable::no); + } return make_ready_future(is_repair_task(task_type)); } future> tablet_virtual_task::get_status(tasks::task_id id, tasks::virtual_task_hint hint) { + auto task_type = hint.get_task_type(); + if (is_resize_task(task_type)) { + co_return std::nullopt; + } + utils::chunked_vector tablets; std::optional pending_replica; co_return co_await get_status_helper(id, tablets, std::move(hint), pending_replica); @@ -101,6 +124,9 @@ future> tablet_virtual_task::get_status(tasks: future> tablet_virtual_task::wait(tasks::task_id id, tasks::virtual_task_hint hint) { auto table = hint.get_table_id(); auto task_type = hint.get_task_type(); + if (is_resize_task(task_type)) { + co_return std::nullopt; + } auto tablet_id_opt = tablet_id_provided(task_type) ? std::make_optional(hint.get_tablet_id()) : std::nullopt; utils::chunked_vector tablets; @@ -134,6 +160,9 @@ future> tablet_virtual_task::wait(tasks::task_ future<> tablet_virtual_task::abort(tasks::task_id id, tasks::virtual_task_hint hint) noexcept { auto table = hint.get_table_id(); auto task_type = hint.get_task_type(); + if (is_resize_task(task_type)) { + co_return; + } if (!is_repair_task(task_type)) { on_internal_error(tasks::tmlogger, format("non-abortable task {} of type {} cannot be aborted", id, task_type)); From adf6b3f3ff075cdfb24814056d91a61010e8bc55 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 4 Dec 2024 13:49:12 +0100 Subject: [PATCH 241/397] service: extend tablet_virtual_task::get_status Extend tablet_virtual_task::get_status to cover resize tasks. --- service/task_manager_module.cc | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index 065a667a063d..4d2c1e399257 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -111,11 +111,6 @@ future tablet_virtual_task::is_abortable(tasks::virtual_tas } future> tablet_virtual_task::get_status(tasks::task_id id, tasks::virtual_task_hint hint) { - auto task_type = hint.get_task_type(); - if (is_resize_task(task_type)) { - co_return std::nullopt; - } - utils::chunked_vector tablets; std::optional pending_replica; co_return co_await get_status_helper(id, tablets, std::move(hint), pending_replica); @@ -219,7 +214,7 @@ std::vector tablet_virtual_task::get_table_ids() const { static void update_status(const locator::tablet_task_info& task_info, tasks::task_status& status, size_t& sched_nr) { sched_nr += task_info.sched_nr; status.type = locator::tablet_task_type_to_string(task_info.request_type); - status.scope = task_info.is_user_repair_request() ? "table" : "tablet"; + status.scope = get_scope(task_info.request_type); status.start_time = task_info.request_time; } @@ -245,7 +240,7 @@ future> tablet_virtual_task::get_status_helper } return make_ready_future(); }); - } else { // Migration task. + } else if (is_migration_task(task_type)) { // Migration task. auto tablet_id = hint.get_tablet_id(); pending_replica = tmap.get_tablet_transition_info(tablet_id)->pending_replica; auto& task_info = tmap.get_tablet_info(tablet_id).migration_task_info; @@ -253,6 +248,14 @@ future> tablet_virtual_task::get_status_helper update_status(task_info, res, sched_nr); tablets.push_back(tablet_id); } + } else { // Resize task. + auto& task_info = tmap.resize_task_info(); + if (task_info.tablet_task_id.uuid() == id.uuid()) { + update_status(task_info, res, sched_nr); + res.state = tasks::task_manager::task_state::running; + res.children = task_type == locator::tablet_task_type::split ? co_await get_children(get_module(), id) : std::vector{}; + co_return res; + } } if (!tablets.empty()) { From 24bbd161fdffeceb447566bcb7409f2ba408d374 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 11 Dec 2024 14:14:25 +0100 Subject: [PATCH 242/397] tasks: add suspended task state Add suspended task state. It will be used for revoke resize requests. --- api/api-doc/task_manager.json | 6 ++++-- tasks/task_manager.hh | 3 ++- test/topology_tasks/task_manager_types.py | 1 + 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/api/api-doc/task_manager.json b/api/api-doc/task_manager.json index 41c1b34b5cc7..f46c51e3086d 100644 --- a/api/api-doc/task_manager.json +++ b/api/api-doc/task_manager.json @@ -284,7 +284,8 @@ "created", "running", "done", - "failed" + "failed", + "suspended" ], "description":"The state of a task" }, @@ -352,7 +353,8 @@ "created", "running", "done", - "failed" + "failed", + "suspended" ], "description":"The state of the task" }, diff --git a/tasks/task_manager.hh b/tasks/task_manager.hh index 92c23a0d0435..b2015f319567 100644 --- a/tasks/task_manager.hh +++ b/tasks/task_manager.hh @@ -104,7 +104,8 @@ public: created, running, done, - failed + failed, + suspended }; enum class task_group { diff --git a/test/topology_tasks/task_manager_types.py b/test/topology_tasks/task_manager_types.py index 3726da72c93e..3d2e8e2f20e0 100644 --- a/test/topology_tasks/task_manager_types.py +++ b/test/topology_tasks/task_manager_types.py @@ -23,6 +23,7 @@ class State(StrEnum): running = "running" done = "done" failed = "failed" + suspended = "suspended" class TaskStats(NamedTuple): From 0c7bef6875da85f5def2ec9e0593e472406940c7 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 4 Dec 2024 17:29:23 +0100 Subject: [PATCH 243/397] service: extend tablet_virtual_task::wait Extend tablet_virtual_task::wait to support resize tasks. To decide what is a state of a finished resize virtual task (done or failed), the tablet count is checked. The task state is set to done, if the tablet count before resize is different than after. --- service/task_manager_module.cc | 29 ++++++++++++++++++----------- service/task_manager_module.hh | 2 +- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index 4d2c1e399257..dcca68c89fb2 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -113,20 +113,19 @@ future tablet_virtual_task::is_abortable(tasks::virtual_tas future> tablet_virtual_task::get_status(tasks::task_id id, tasks::virtual_task_hint hint) { utils::chunked_vector tablets; std::optional pending_replica; - co_return co_await get_status_helper(id, tablets, std::move(hint), pending_replica); + size_t tablet_count; + co_return co_await get_status_helper(id, tablets, std::move(hint), pending_replica, tablet_count); } future> tablet_virtual_task::wait(tasks::task_id id, tasks::virtual_task_hint hint) { auto table = hint.get_table_id(); auto task_type = hint.get_task_type(); - if (is_resize_task(task_type)) { - co_return std::nullopt; - } auto tablet_id_opt = tablet_id_provided(task_type) ? std::make_optional(hint.get_tablet_id()) : std::nullopt; utils::chunked_vector tablets; std::optional pending_replica; - auto status = co_await get_status_helper(id, tablets, std::move(hint), pending_replica); + size_t tablet_count; + auto status = co_await get_status_helper(id, tablets, std::move(hint), pending_replica, tablet_count); if (!status) { co_return std::nullopt; } @@ -134,19 +133,26 @@ future> tablet_virtual_task::wait(tasks::task_ tasks::tmlogger.info("tablet_virtual_task: wait until tablet operation is finished"); co_await _ss._topology_state_machine.event.wait([&] { auto& tmap = _ss.get_token_metadata().tablets().get_tablet_map(table); - if (tablet_id_opt.has_value()) { + if (is_resize_task(task_type)) { // Resize task. + return tmap.resize_task_info().tablet_task_id.uuid() != id.uuid(); + } else if (tablet_id_opt.has_value()) { // Migration task. return tmap.get_tablet_info(tablet_id_opt.value()).migration_task_info.tablet_task_id.uuid() != id.uuid(); + } else { // Repair task. + return std::all_of(tablets.begin(), tablets.end(), [&] (const locator::tablet_id& tablet) { + return tmap.get_tablet_info(tablet).repair_task_info.tablet_task_id.uuid() != id.uuid(); + }); } - return std::all_of(tablets.begin(), tablets.end(), [&] (const locator::tablet_id& tablet) { - return tmap.get_tablet_info(tablet).repair_task_info.tablet_task_id.uuid() != id.uuid(); - }); }); status->state = tasks::task_manager::task_state::done; // Failed repair task is retried. - if (!is_repair_task(task_type)) { + if (is_migration_task(task_type)) { auto& replicas = _ss.get_token_metadata().tablets().get_tablet_map(table).get_tablet_info(tablet_id_opt.value()).replicas; auto migration_failed = std::all_of(replicas.begin(), replicas.end(), [&] (const auto& replica) { return pending_replica.has_value() && replica != pending_replica.value(); }); status->state = migration_failed ? tasks::task_manager::task_state::failed : tasks::task_manager::task_state::done; + } else if (is_resize_task(task_type)) { + auto new_tablet_count = _ss.get_token_metadata().tablets().get_tablet_map(table).tablet_count(); + status->state = new_tablet_count == tablet_count ? tasks::task_manager::task_state::suspended : tasks::task_manager::task_state::done; + status->children = task_type == locator::tablet_task_type::split ? co_await get_children(get_module(), id) : std::vector{}; } status->end_time = db_clock::now(); // FIXME: Get precise end time. co_return status; @@ -218,7 +224,7 @@ static void update_status(const locator::tablet_task_info& task_info, tasks::tas status.start_time = task_info.request_time; } -future> tablet_virtual_task::get_status_helper(tasks::task_id id, utils::chunked_vector& tablets, tasks::virtual_task_hint hint, std::optional& pending_replica) { +future> tablet_virtual_task::get_status_helper(tasks::task_id id, utils::chunked_vector& tablets, tasks::virtual_task_hint hint, std::optional& pending_replica, size_t& tablet_count) { auto table = hint.get_table_id(); auto task_type = hint.get_task_type(); auto schema = _ss._db.local().get_tables_metadata().get_table(table).schema(); @@ -249,6 +255,7 @@ future> tablet_virtual_task::get_status_helper tablets.push_back(tablet_id); } } else { // Resize task. + tablet_count = tmap.tablet_count(); auto& task_info = tmap.resize_task_info(); if (task_info.tablet_task_id.uuid() == id.uuid()) { update_status(task_info, res, sched_nr); diff --git a/service/task_manager_module.hh b/service/task_manager_module.hh index c242c93e01a5..841bea896d99 100644 --- a/service/task_manager_module.hh +++ b/service/task_manager_module.hh @@ -42,7 +42,7 @@ public: virtual future> get_stats() override; private: std::vector get_table_ids() const; - future> get_status_helper(tasks::task_id id, utils::chunked_vector& tablets, tasks::virtual_task_hint hint, std::optional& pending_replica); + future> get_status_helper(tasks::task_id id, utils::chunked_vector& tablets, tasks::virtual_task_hint hint, std::optional& pending_replica, size_t& tablet_count); }; class task_manager_module : public tasks::task_manager::module { From 639470d2568f5f06dae92db90b544190c7802074 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 9 Jan 2025 16:20:24 +0100 Subject: [PATCH 244/397] service: retrun status_helper struct from tablet_virtual_task::get_status_helper --- service/task_manager_module.cc | 64 ++++++++++++++++++---------------- service/task_manager_module.hh | 3 +- 2 files changed, 36 insertions(+), 31 deletions(-) diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index dcca68c89fb2..ca0e67c731e0 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -17,6 +17,12 @@ namespace service { +struct status_helper { + tasks::task_status status; + utils::chunked_vector tablets; + std::optional pending_replica; +}; + tasks::task_manager::task_group tablet_virtual_task::get_group() const noexcept { return tasks::task_manager::task_group::tablets_group; } @@ -111,10 +117,10 @@ future tablet_virtual_task::is_abortable(tasks::virtual_tas } future> tablet_virtual_task::get_status(tasks::task_id id, tasks::virtual_task_hint hint) { - utils::chunked_vector tablets; - std::optional pending_replica; - size_t tablet_count; - co_return co_await get_status_helper(id, tablets, std::move(hint), pending_replica, tablet_count); + auto res = co_await get_status_helper(id, std::move(hint)); + co_return res.transform([] (status_helper s) { + return s.status; + }); } future> tablet_virtual_task::wait(tasks::task_id id, tasks::virtual_task_hint hint) { @@ -122,11 +128,9 @@ future> tablet_virtual_task::wait(tasks::task_ auto task_type = hint.get_task_type(); auto tablet_id_opt = tablet_id_provided(task_type) ? std::make_optional(hint.get_tablet_id()) : std::nullopt; - utils::chunked_vector tablets; - std::optional pending_replica; - size_t tablet_count; - auto status = co_await get_status_helper(id, tablets, std::move(hint), pending_replica, tablet_count); - if (!status) { + size_t tablet_count = _ss.get_token_metadata().tablets().get_tablet_map(table).tablet_count(); + auto res = co_await get_status_helper(id, std::move(hint)); + if (!res) { co_return std::nullopt; } @@ -138,24 +142,24 @@ future> tablet_virtual_task::wait(tasks::task_ } else if (tablet_id_opt.has_value()) { // Migration task. return tmap.get_tablet_info(tablet_id_opt.value()).migration_task_info.tablet_task_id.uuid() != id.uuid(); } else { // Repair task. - return std::all_of(tablets.begin(), tablets.end(), [&] (const locator::tablet_id& tablet) { + return std::all_of(res->tablets.begin(), res->tablets.end(), [&] (const locator::tablet_id& tablet) { return tmap.get_tablet_info(tablet).repair_task_info.tablet_task_id.uuid() != id.uuid(); }); } }); - status->state = tasks::task_manager::task_state::done; // Failed repair task is retried. + res->status.state = tasks::task_manager::task_state::done; // Failed repair task is retried. if (is_migration_task(task_type)) { auto& replicas = _ss.get_token_metadata().tablets().get_tablet_map(table).get_tablet_info(tablet_id_opt.value()).replicas; - auto migration_failed = std::all_of(replicas.begin(), replicas.end(), [&] (const auto& replica) { return pending_replica.has_value() && replica != pending_replica.value(); }); - status->state = migration_failed ? tasks::task_manager::task_state::failed : tasks::task_manager::task_state::done; + auto migration_failed = std::all_of(replicas.begin(), replicas.end(), [&] (const auto& replica) { return res->pending_replica.has_value() && replica != res->pending_replica.value(); }); + res->status.state = migration_failed ? tasks::task_manager::task_state::failed : tasks::task_manager::task_state::done; } else if (is_resize_task(task_type)) { auto new_tablet_count = _ss.get_token_metadata().tablets().get_tablet_map(table).tablet_count(); - status->state = new_tablet_count == tablet_count ? tasks::task_manager::task_state::suspended : tasks::task_manager::task_state::done; - status->children = task_type == locator::tablet_task_type::split ? co_await get_children(get_module(), id) : std::vector{}; + res->status.state = new_tablet_count == tablet_count ? tasks::task_manager::task_state::suspended : tasks::task_manager::task_state::done; + res->status.children = task_type == locator::tablet_task_type::split ? co_await get_children(get_module(), id) : std::vector{}; } - status->end_time = db_clock::now(); // FIXME: Get precise end time. - co_return status; + res->status.end_time = db_clock::now(); // FIXME: Get precise end time. + co_return res->status; } future<> tablet_virtual_task::abort(tasks::task_id id, tasks::virtual_task_hint hint) noexcept { @@ -224,11 +228,12 @@ static void update_status(const locator::tablet_task_info& task_info, tasks::tas status.start_time = task_info.request_time; } -future> tablet_virtual_task::get_status_helper(tasks::task_id id, utils::chunked_vector& tablets, tasks::virtual_task_hint hint, std::optional& pending_replica, size_t& tablet_count) { +future> tablet_virtual_task::get_status_helper(tasks::task_id id, tasks::virtual_task_hint hint) { + status_helper res; auto table = hint.get_table_id(); auto task_type = hint.get_task_type(); auto schema = _ss._db.local().get_tables_metadata().get_table(table).schema(); - tasks::task_status res{ + res.status = { .task_id = id, .kind = tasks::task_kind::cluster, .is_abortable = co_await is_abortable(std::move(hint)), @@ -241,32 +246,31 @@ future> tablet_virtual_task::get_status_helper co_await tmap.for_each_tablet([&] (locator::tablet_id tid, const locator::tablet_info& info) { auto& task_info = info.repair_task_info; if (task_info.tablet_task_id.uuid() == id.uuid()) { - update_status(task_info, res, sched_nr); - tablets.push_back(tid); + update_status(task_info, res.status, sched_nr); + res.tablets.push_back(tid); } return make_ready_future(); }); } else if (is_migration_task(task_type)) { // Migration task. auto tablet_id = hint.get_tablet_id(); - pending_replica = tmap.get_tablet_transition_info(tablet_id)->pending_replica; + res.pending_replica = tmap.get_tablet_transition_info(tablet_id)->pending_replica; auto& task_info = tmap.get_tablet_info(tablet_id).migration_task_info; if (task_info.tablet_task_id.uuid() == id.uuid()) { - update_status(task_info, res, sched_nr); - tablets.push_back(tablet_id); + update_status(task_info, res.status, sched_nr); + res.tablets.push_back(tablet_id); } } else { // Resize task. - tablet_count = tmap.tablet_count(); auto& task_info = tmap.resize_task_info(); if (task_info.tablet_task_id.uuid() == id.uuid()) { - update_status(task_info, res, sched_nr); - res.state = tasks::task_manager::task_state::running; - res.children = task_type == locator::tablet_task_type::split ? co_await get_children(get_module(), id) : std::vector{}; + update_status(task_info, res.status, sched_nr); + res.status.state = tasks::task_manager::task_state::running; + res.status.children = task_type == locator::tablet_task_type::split ? co_await get_children(get_module(), id) : std::vector{}; co_return res; } } - if (!tablets.empty()) { - res.state = sched_nr == 0 ? tasks::task_manager::task_state::created : tasks::task_manager::task_state::running; + if (!res.tablets.empty()) { + res.status.state = sched_nr == 0 ? tasks::task_manager::task_state::created : tasks::task_manager::task_state::running; co_return res; } // FIXME: Show finished tasks. diff --git a/service/task_manager_module.hh b/service/task_manager_module.hh index 841bea896d99..e32148abb43f 100644 --- a/service/task_manager_module.hh +++ b/service/task_manager_module.hh @@ -19,6 +19,7 @@ class tablet_replica; namespace service { +class status_helper; class storage_service; class tablet_virtual_task : public tasks::task_manager::virtual_task::impl { @@ -42,7 +43,7 @@ public: virtual future> get_stats() override; private: std::vector get_table_ids() const; - future> get_status_helper(tasks::task_id id, utils::chunked_vector& tablets, tasks::virtual_task_hint hint, std::optional& pending_replica, size_t& tablet_count); + future> get_status_helper(tasks::task_id id, tasks::virtual_task_hint hint); }; class task_manager_module : public tasks::task_manager::module { From 840bcdc158e176b0643d50fd3adea60de20932f6 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 4 Dec 2024 17:33:52 +0100 Subject: [PATCH 245/397] service: extend tablet_virtual_task::abort Set resize tasks as non abortable. --- service/task_manager_module.cc | 7 ------- 1 file changed, 7 deletions(-) diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index ca0e67c731e0..3262f8e9ec81 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -110,9 +110,6 @@ future> tablet_virtual_task::contains(ta future tablet_virtual_task::is_abortable(tasks::virtual_task_hint hint) const { auto task_type = hint.get_task_type(); - if (is_resize_task(task_type)) { - return make_ready_future(tasks::is_abortable::no); - } return make_ready_future(is_repair_task(task_type)); } @@ -165,10 +162,6 @@ future> tablet_virtual_task::wait(tasks::task_ future<> tablet_virtual_task::abort(tasks::task_id id, tasks::virtual_task_hint hint) noexcept { auto table = hint.get_table_id(); auto task_type = hint.get_task_type(); - if (is_resize_task(task_type)) { - co_return; - } - if (!is_repair_task(task_type)) { on_internal_error(tasks::tmlogger, format("non-abortable task {} of type {} cannot be aborted", id, task_type)); } From 5a948d3fac90be73d540aa44dd9b3607897ffd25 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 11 Dec 2024 11:59:14 +0100 Subject: [PATCH 246/397] tasks: initialize shard in task_info ctor Initialize shard in task_info constructor. All current usages do not care about the shard of an empty task_info. In the following patches we may need that for setting info about virtual task parent. --- tasks/types.hh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tasks/types.hh b/tasks/types.hh index 090ca740b747..bd9be78f09bc 100644 --- a/tasks/types.hh +++ b/tasks/types.hh @@ -18,7 +18,7 @@ struct task_info { task_id id; unsigned shard; - task_info() noexcept : id(task_id::create_null_id()) {} + task_info() noexcept : id(task_id::create_null_id()), shard(0) {} task_info(task_id id, unsigned parent_shard) noexcept : id(id), shard(parent_shard) {} operator bool() const noexcept { From 14dcaecc2988477e27dffbbfc4a78af656bd5152 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 12 Dec 2024 11:32:02 +0100 Subject: [PATCH 247/397] tasks: children of virtual tasks aren't internal by default Currently, streaming_task_impl is the only existing child of any virtual task. It overrides the is_internal definition so that it is non-internal even though it has a parent. This should apply to all children of all virtual tasks. Modify task_manager::task::impl::is_internal so that children of virtual tasks aren't internal by default. --- tasks/task_manager.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tasks/task_manager.cc b/tasks/task_manager.cc index 7014f2a32b36..26d29794040a 100644 --- a/tasks/task_manager.cc +++ b/tasks/task_manager.cc @@ -156,7 +156,7 @@ is_abortable task_manager::task::impl::is_abortable() const noexcept { } is_internal task_manager::task::impl::is_internal() const noexcept { - return tasks::is_internal(bool(_parent_id)); + return tasks::is_internal(_parent_id && _parent_kind != task_kind::cluster); } tasks::is_user_task task_manager::task::impl::is_user_task() const noexcept { From 7ef6900837459faa5b36151f1f2a0869ab3e79bc Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 5 Dec 2024 12:20:52 +0100 Subject: [PATCH 248/397] replica: service: pass parent info down to storage_group::split Pass task_info down to storage_group::split. In the following patches, it will be used to set the parent of offstrategy_compaction_task_executor and split_compaction_task_executor running as a part of the split. The task_info param will contain task info of a split virtual task. --- replica/compaction_group.hh | 4 ++-- replica/database.hh | 2 +- replica/table.cc | 18 +++++++++--------- service/storage_service.cc | 2 +- test/boost/sstable_set_test.cc | 2 +- test/boost/tablets_test.cc | 2 +- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/replica/compaction_group.hh b/replica/compaction_group.hh index 839ea13fccaa..ffa066a067c6 100644 --- a/replica/compaction_group.hh +++ b/replica/compaction_group.hh @@ -264,7 +264,7 @@ public: // 1) Flushes all memtables which were created in non-split mode, and waits for that to complete. // 2) Compacts all sstables which overlap with the split point // Returns a future which resolves when this process is complete. - future<> split(sstables::compaction_type_options::split opt); + future<> split(sstables::compaction_type_options::split opt, tasks::task_info tablet_split_task_info); // Make an sstable set spanning all sstables in the storage_group lw_shared_ptr make_sstable_set() const; @@ -368,7 +368,7 @@ public: virtual locator::table_load_stats table_load_stats(std::function tablet_filter) const noexcept = 0; virtual bool all_storage_groups_split() = 0; - virtual future<> split_all_storage_groups() = 0; + virtual future<> split_all_storage_groups(tasks::task_info tablet_split_task_info) = 0; virtual future<> maybe_split_compaction_group_of(size_t idx) = 0; virtual future> maybe_split_sstable(const sstables::shared_sstable& sst) = 0; virtual dht::token_range get_token_range_after_split(const dht::token&) const noexcept = 0; diff --git a/replica/database.hh b/replica/database.hh index 94fc42e372e5..63fefd98bbe2 100644 --- a/replica/database.hh +++ b/replica/database.hh @@ -586,7 +586,7 @@ public: // Precondition: table needs tablet splitting. // Returns true if all storage of table is ready for splitting. bool all_storage_groups_split(); - future<> split_all_storage_groups(); + future<> split_all_storage_groups(tasks::task_info tablet_split_task_info); // Splits compaction group of a single tablet, if and only if the underlying table has // split request emitted by coordinator (found in tablet metadata). diff --git a/replica/table.cc b/replica/table.cc index 270aa5be6197..0bc901b6b471 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -723,7 +723,7 @@ class single_storage_group_manager final : public storage_group_manager { }; } bool all_storage_groups_split() override { return true; } - future<> split_all_storage_groups() override { return make_ready_future(); } + future<> split_all_storage_groups(tasks::task_info tablet_split_task_info) override { return make_ready_future(); } future<> maybe_split_compaction_group_of(size_t idx) override { return make_ready_future(); } future> maybe_split_sstable(const sstables::shared_sstable& sst) override { return make_ready_future>(std::vector{sst}); @@ -856,7 +856,7 @@ class tablet_storage_group_manager final : public storage_group_manager { locator::table_load_stats table_load_stats(std::function tablet_filter) const noexcept override; bool all_storage_groups_split() override; - future<> split_all_storage_groups() override; + future<> split_all_storage_groups(tasks::task_info tablet_split_task_info) override; future<> maybe_split_compaction_group_of(size_t idx) override; future> maybe_split_sstable(const sstables::shared_sstable& sst) override; dht::token_range get_token_range_after_split(const dht::token& token) const noexcept override { @@ -977,7 +977,7 @@ future<> storage_group::remove_empty_merging_groups() { std::erase_if(_merging_groups, std::mem_fn(&compaction_group::empty)); } -future<> storage_group::split(sstables::compaction_type_options::split opt) { +future<> storage_group::split(sstables::compaction_type_options::split opt, tasks::task_info tablet_split_task_info) { if (set_split_mode()) { co_return; } @@ -1054,17 +1054,17 @@ sstables::compaction_type_options::split tablet_storage_group_manager::split_com }}; } -future<> tablet_storage_group_manager::split_all_storage_groups() { +future<> tablet_storage_group_manager::split_all_storage_groups(tasks::task_info tablet_split_task_info) { sstables::compaction_type_options::split opt = split_compaction_options(); - co_await for_each_storage_group_gently([opt] (storage_group& storage_group) { - return storage_group.split(opt); + co_await for_each_storage_group_gently([opt, tablet_split_task_info] (storage_group& storage_group) { + return storage_group.split(opt, tablet_split_task_info); }); } -future<> table::split_all_storage_groups() { +future<> table::split_all_storage_groups(tasks::task_info tablet_split_task_info) { auto holder = async_gate().hold(); - co_await _sg_manager->split_all_storage_groups(); + co_await _sg_manager->split_all_storage_groups(tablet_split_task_info); } future<> tablet_storage_group_manager::maybe_split_compaction_group_of(size_t idx) { @@ -1078,7 +1078,7 @@ future<> tablet_storage_group_manager::maybe_split_compaction_group_of(size_t id idx, schema()->ks_name(), schema()->cf_name())); } - return sg->split(split_compaction_options()); + return sg->split(split_compaction_options(), tasks::task_info{}); } future> diff --git a/service/storage_service.cc b/service/storage_service.cc index c7be08edf470..e2e3cd75a8c4 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -5418,7 +5418,7 @@ future<> storage_service::process_tablet_split_candidate(table_id table) noexcep auto split_all_compaction_groups = [&] () -> future<> { return _db.invoke_on_all([table] (replica::database& db) -> future<> { - return db.find_column_family(table).split_all_storage_groups(); + return db.find_column_family(table).split_all_storage_groups(tasks::task_info{}); }); }; diff --git a/test/boost/sstable_set_test.cc b/test/boost/sstable_set_test.cc index cef9b11d779c..d264337a4596 100644 --- a/test/boost/sstable_set_test.cc +++ b/test/boost/sstable_set_test.cc @@ -190,7 +190,7 @@ SEASTAR_TEST_CASE(test_tablet_sstable_set_copy_ctor) { } auto& cf = env.local_db().find_column_family("test_tablet_sstable_set_copy_ctor", "test"); auto& sgm = column_family_test::get_storage_group_manager(cf); - sgm->split_all_storage_groups().get(); + sgm->split_all_storage_groups(tasks::task_info{}).get(); auto tablet_sstable_set = replica::make_tablet_sstable_set(cf.schema(), *sgm.get(), locator::tablet_map(8)); auto tablet_sstable_set_copy = *tablet_sstable_set.get(); diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 68b6eb2ea9f8..4cc96bf715d3 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -2825,7 +2825,7 @@ SEASTAR_THREAD_TEST_CASE(basic_tablet_storage_splitting_test) { e.db().invoke_on_all([] (replica::database& db) { auto& table = db.find_column_family("ks", "cf"); testlog.info("sstable count: {}", table.sstables_count()); - return table.split_all_storage_groups(); + return table.split_all_storage_groups(tasks::task_info{}); }).get(); testlog.info("Verifying sstables are split..."); From 062f155fd69a5481cf74bf915caa62ab1c4efcbc Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 5 Dec 2024 13:25:29 +0100 Subject: [PATCH 249/397] replica: service: add split virtual task's children offstrategy_compaction_task_executor and split_compaction_task_executor running as a part of the split become children of a split virtual task. --- replica/table.cc | 7 ++++--- service/storage_service.cc | 7 +++++-- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/replica/table.cc b/replica/table.cc index 0bc901b6b471..56df0d085909 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -993,8 +993,8 @@ future<> storage_group::split(sstables::compaction_type_options::split opt, task auto holder = cg->async_gate().hold(); co_await cg->flush(); // Waits on sstables produced by repair to be integrated into main set; off-strategy is usually a no-op with tablets. - co_await cg->get_compaction_manager().perform_offstrategy(_main_cg->as_table_state(), tasks::task_info{}); - co_await cg->get_compaction_manager().perform_split_compaction(_main_cg->as_table_state(), std::move(opt), tasks::task_info{}); + co_await cg->get_compaction_manager().perform_offstrategy(_main_cg->as_table_state(), tablet_split_task_info); + co_await cg->get_compaction_manager().perform_split_compaction(_main_cg->as_table_state(), std::move(opt), tablet_split_task_info); } } @@ -1071,6 +1071,7 @@ future<> tablet_storage_group_manager::maybe_split_compaction_group_of(size_t id if (!tablet_map().needs_split()) { return make_ready_future<>(); } + tasks::task_info tablet_split_task_info{tasks::task_id{tablet_map().resize_task_info().tablet_task_id.uuid()}, 0}; auto& sg = _storage_groups[idx]; if (!sg) { @@ -1078,7 +1079,7 @@ future<> tablet_storage_group_manager::maybe_split_compaction_group_of(size_t id idx, schema()->ks_name(), schema()->cf_name())); } - return sg->split(split_compaction_options(), tasks::task_info{}); + return sg->split(split_compaction_options(), tablet_split_task_info); } future> diff --git a/service/storage_service.cc b/service/storage_service.cc index e2e3cd75a8c4..8f5a6a0268f2 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -5409,6 +5409,8 @@ future<> storage_service::load_tablet_metadata(const locator::tablet_metadata_ch } future<> storage_service::process_tablet_split_candidate(table_id table) noexcept { + tasks::task_info tablet_split_task_info; + auto all_compaction_groups_split = [&] () mutable { return _db.map_reduce0([table_ = table] (replica::database& db) { auto all_split = db.find_column_family(table_).all_storage_groups_split(); @@ -5417,8 +5419,8 @@ future<> storage_service::process_tablet_split_candidate(table_id table) noexcep }; auto split_all_compaction_groups = [&] () -> future<> { - return _db.invoke_on_all([table] (replica::database& db) -> future<> { - return db.find_column_family(table).split_all_storage_groups(tasks::task_info{}); + return _db.invoke_on_all([table, tablet_split_task_info] (replica::database& db) -> future<> { + return db.find_column_family(table).split_all_storage_groups(tablet_split_task_info); }); }; @@ -5434,6 +5436,7 @@ future<> storage_service::process_tablet_split_candidate(table_id table) noexcep release_guard(std::move(guard)); break; } + tablet_split_task_info.id = tasks::task_id{tmap.resize_task_info().tablet_task_id.uuid()}; if (co_await all_compaction_groups_split()) { slogger.debug("All compaction groups of table {} are split ready.", table); From 48e08437672ceef957437791b9dc37bf07561559 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 11 Dec 2024 15:48:20 +0100 Subject: [PATCH 250/397] test: test_tablet_tasks: generalize functions --- test/topology_tasks/test_tablet_tasks.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index dfd88a495c23..6925d4337016 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -5,6 +5,7 @@ # import asyncio +from typing import Optional import pytest from test.pylib.internal_types import ServerInfo @@ -24,23 +25,24 @@ async def disable_injection(manager: ManagerClient, servers: list[ServerInfo], i for server in servers: await manager.api.disable_injection(server.ip_addr, injection) -async def wait_tasks_created(tm: TaskManagerClient, server: ServerInfo, module_name: str, expected_number: int, type: str): +async def wait_tasks_created(tm: TaskManagerClient, server: ServerInfo, module_name: str, expected_number: int, type: str, table: Optional[str] = None): async def get_tasks(): - return [task for task in await tm.list_tasks(server.ip_addr, module_name) if task.kind == "cluster" and task.type == type and task.keyspace == "test"] + tasks = [task for task in await tm.list_tasks(server.ip_addr, module_name) if task.kind == "cluster" and task.type == type and task.keyspace == "test"] + return [task for task in tasks if not table or table == task.table] tasks = await get_tasks() while len(tasks) != expected_number: tasks = await get_tasks() return tasks -def check_task_status(status: TaskStatus, states: list[str], type: str, scope: str, abortable: bool): +def check_task_status(status: TaskStatus, states: list[str], type: str, scope: str, abortable: bool, keyspace: str = "test", table: str = "test", possible_child_num: list[int] = [0]): assert status.scope == scope assert status.kind == "cluster" assert status.type == type - assert status.keyspace == "test" - assert status.table == "test" + assert status.keyspace == keyspace + assert status.table == table assert status.is_abortable == abortable - assert not status.children_ids + assert len(status.children_ids) in possible_child_num assert status.state in states async def check_and_abort_repair_task(tm: TaskManagerClient, servers: list[ServerInfo], module_name: str): From 2ed4bad752b13d2f7280aec34bad449817181850 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 11 Dec 2024 16:05:58 +0100 Subject: [PATCH 251/397] test: add tests to check spilt and merge virtual tasks status --- test/topology_tasks/test_tablet_tasks.py | 72 ++++++++++++++++++++++++ 1 file changed, 72 insertions(+) diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index 6925d4337016..b3f1a6a01c40 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -5,6 +5,7 @@ # import asyncio +import random from typing import Optional import pytest @@ -310,3 +311,74 @@ async def wait_and_check_none(): await check_none() await asyncio.gather(repair_task(), wait_and_check_none()) + +async def prepare_split(manager: ManagerClient, server: ServerInfo, keyspace: str, table: str, keys: list[int]): + await manager.api.disable_tablet_balancing(server.ip_addr) + + cql = manager.get_cql() + insert = cql.prepare(f"INSERT INTO {keyspace}.{table}(pk, c) VALUES(?, ?)") + for pk in keys: + value = random.randbytes(1000) + cql.execute(insert, [pk, value]) + + await manager.api.flush_keyspace(server.ip_addr, keyspace) + +async def prepare_merge(manager: ManagerClient, server: ServerInfo, keyspace: str, table: str, keys: list[int]): + await manager.api.disable_tablet_balancing(server.ip_addr) + + cql = manager.get_cql() + await asyncio.gather(*[cql.run_async(f"DELETE FROM {keyspace}.{table} WHERE pk={k};") for k in keys]) + + await manager.api.flush_keyspace(server.ip_addr, keyspace) + +async def enable_tablet_balancing_and_wait(manager: ManagerClient, server: ServerInfo, message: str): + s1_log = await manager.server_open_log(server.server_id) + s1_mark = await s1_log.mark() + + await manager.api.enable_tablet_balancing(server.ip_addr) + + await s1_log.wait_for(message, from_mark=s1_mark) + +@pytest.mark.asyncio +@skip_mode('release', 'error injections are not supported in release mode') +async def test_tablet_resize_task(manager: ManagerClient): + module_name = "tablets" + tm = TaskManagerClient(manager.api) + cmdline = [ + '--target-tablet-size-in-bytes', '30000', + ] + servers = [await manager.server_add(cmdline=cmdline, config={ + 'error_injections_at_startup': ['short_tablet_stats_refresh_interval'] + })] + + await manager.api.disable_tablet_balancing(servers[0].ip_addr) + + cql = manager.get_cql() + keyspace = "test" + table1 = "test1" + table2 = "test2" + await cql.run_async(f"CREATE KEYSPACE {keyspace} WITH replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}} AND tablets = {{'initial': 1}};") + await cql.run_async(f"CREATE TABLE {keyspace}.{table1} (pk int PRIMARY KEY, c blob) WITH gc_grace_seconds=0 AND bloom_filter_fp_chance=1;") + await cql.run_async(f"CREATE TABLE {keyspace}.{table2} (pk int PRIMARY KEY, c blob) WITH gc_grace_seconds=0 AND bloom_filter_fp_chance=1;") + + total_keys = 60 + keys = range(total_keys) + await prepare_split(manager, servers[0], keyspace, table1, keys) + await enable_tablet_balancing_and_wait(manager, servers[0], "Detected tablet split for table") + await wait_tasks_created(tm, servers[0], module_name, 0, "split", table1) + + await prepare_split(manager, servers[0], keyspace, table2, keys) + await prepare_merge(manager, servers[0], keyspace, table1, keys[:-1]) + await manager.api.keyspace_compaction(servers[0].ip_addr, "test") + + injection = "tablet_split_finalization_postpone" + await enable_injection(manager, servers, injection) + await manager.api.enable_tablet_balancing(servers[0].ip_addr) + + async def wait_and_check_status(server, type, keyspace, table): + task = (await wait_tasks_created(tm, server, module_name, 1, type, table))[0] + status = await tm.get_task_status(server.ip_addr, task.task_id) + check_task_status(status, ["running"], type, "table", False, keyspace, table, [0, 1, 2]) + + await wait_and_check_status(servers[0], "split", keyspace, table2) + await wait_and_check_status(servers[0], "merge", keyspace, table1) From 50c9c0d898a063938aab31c5f4b26694d8680372 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Wed, 11 Dec 2024 16:56:02 +0100 Subject: [PATCH 252/397] test: add tests to check the list of resize virtual tasks --- test/topology_tasks/test_tablet_tasks.py | 60 ++++++++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index b3f1a6a01c40..428824c9035f 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -382,3 +382,63 @@ async def wait_and_check_status(server, type, keyspace, table): await wait_and_check_status(servers[0], "split", keyspace, table2) await wait_and_check_status(servers[0], "merge", keyspace, table1) + +@pytest.mark.asyncio +@skip_mode('release', 'error injections are not supported in release mode') +async def test_tablet_resize_list(manager: ManagerClient): + module_name = "tablets" + tm = TaskManagerClient(manager.api) + cmdline = [ + '--target-tablet-size-in-bytes', '30000', + ] + servers = [await manager.server_add(cmdline=cmdline, config={ + 'error_injections_at_startup': ['short_tablet_stats_refresh_interval'] + })] + + await manager.api.disable_tablet_balancing(servers[0].ip_addr) + + cql = manager.get_cql() + keyspace = "test" + table1 = "test1" + await cql.run_async(f"CREATE KEYSPACE {keyspace} WITH replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}} AND tablets = {{'initial': 1}};") + await cql.run_async(f"CREATE TABLE {keyspace}.{table1} (pk int PRIMARY KEY, c blob) WITH gc_grace_seconds=0 AND bloom_filter_fp_chance=1;") + + total_keys = 60 + keys = range(total_keys) + await prepare_split(manager, servers[0], keyspace, table1, keys) + + servers.append(await manager.server_add(cmdline=cmdline, config={ + 'error_injections_at_startup': ['short_tablet_stats_refresh_interval'] + })) + + s1_log = await manager.server_open_log(servers[0].server_id) + s1_mark = await s1_log.mark() + + injection = "tablet_split_finalization_postpone" + compaction_injection = "split_sstable_rewrite" + await enable_injection(manager, servers, injection) + await manager.api.enable_injection(servers[0].ip_addr, compaction_injection, one_shot=True) + + await manager.api.enable_tablet_balancing(servers[0].ip_addr) + task0 = (await wait_tasks_created(tm, servers[0], module_name, 1, "split", table1))[0] + task1 = (await wait_tasks_created(tm, servers[1], module_name, 1, "split", table1))[0] + + assert task0.task_id == task1.task_id + + for task in [task0, task1]: + assert task.state == "running" + assert task.type == "split" + assert task.kind == "cluster" + assert task.scope == "table" + assert task.table == table1 + assert task.keyspace == keyspace + + await s1_log.wait_for("split_sstable_rewrite: waiting", from_mark=s1_mark) + await manager.api.message_injection(servers[0].ip_addr, "split_sstable_rewrite") + + status1 = await tm.get_task_status(servers[1].ip_addr, task0.task_id) + status0 = await tm.get_task_status(servers[0].ip_addr, task0.task_id) + assert len(status0.children_ids) == 2 + assert status0.children_ids == status1.children_ids + + await disable_injection(manager, servers, injection) From b11c21e9018df472bd1e6d1a220bfc98f3e19a74 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 12 Dec 2024 11:23:38 +0100 Subject: [PATCH 253/397] test: add tests to check revoked resize virtual tasks The test is skipped in debug mode, because the preparation of revoke takes too long and wait request, which needs to be started before the preparation, hits timeout. --- test/topology_tasks/test_tablet_tasks.py | 47 ++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index 428824c9035f..d1a4201f5468 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -442,3 +442,50 @@ async def test_tablet_resize_list(manager: ManagerClient): assert status0.children_ids == status1.children_ids await disable_injection(manager, servers, injection) + + +@pytest.mark.asyncio +@skip_mode('release', 'error injections are not supported in release mode') +@skip_mode('debug', 'debug mode is too time-sensitive') +async def test_tablet_resize_revoked(manager: ManagerClient): + module_name = "tablets" + tm = TaskManagerClient(manager.api) + cmdline = [ + '--target-tablet-size-in-bytes', '30000', + ] + servers = [await manager.server_add(cmdline=cmdline, config={ + 'error_injections_at_startup': ['short_tablet_stats_refresh_interval'] + })] + + await manager.api.disable_tablet_balancing(servers[0].ip_addr) + + cql = manager.get_cql() + keyspace = "test" + table1 = "test1" + await cql.run_async(f"CREATE KEYSPACE {keyspace} WITH replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}} AND tablets = {{'initial': 1}};") + await cql.run_async(f"CREATE TABLE {keyspace}.{table1} (pk int PRIMARY KEY, c blob) WITH gc_grace_seconds=0 AND bloom_filter_fp_chance=1;") + + total_keys = 60 + keys = range(total_keys) + await prepare_split(manager, servers[0], keyspace, table1, keys) + + injection = "tablet_split_finalization_postpone" + await enable_injection(manager, servers, injection) + + await manager.api.enable_tablet_balancing(servers[0].ip_addr) + task0 = (await wait_tasks_created(tm, servers[0], module_name, 1, "split", table1))[0] + + log = await manager.server_open_log(servers[0].server_id) + mark = await log.mark() + + async def revoke_resize(log, mark): + await log.wait_for('tablet_virtual_task: wait until tablet operation is finished', from_mark=mark) + await asyncio.gather(*[cql.run_async(f"DELETE FROM {keyspace}.{table1} WHERE pk={k};") for k in keys]) + + await manager.api.flush_keyspace(servers[0].ip_addr, keyspace) + + async def wait_for_task(task_id): + status = await tm.wait_for_task(servers[0].ip_addr, task_id) + check_task_status(status, ["suspended"], "split", "table", False, keyspace, table1, [0, 1, 2]) + + await asyncio.gather(revoke_resize(log, mark), wait_for_task(task0.task_id)) From 1d46bdb1ad11d2e448639e11296efe225554783b Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 9 Jan 2025 17:51:23 +0100 Subject: [PATCH 254/397] test: boost: check resize_task_info in tablet_test.cc --- test/boost/tablets_test.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 4cc96bf715d3..e5d1be5677f0 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -283,6 +283,7 @@ SEASTAR_TEST_CASE(test_tablet_metadata_persistence) { decision.way = locator::resize_decision::split{}, decision.sequence_number = 1; tmap.set_resize_decision(decision); + tmap.set_resize_task_info(locator::tablet_task_info::make_split_request()); tm.set_tablet_map(table1, std::move(tmap)); } From d2ba45a01f78c6c8b4ed41818fa949d8f4bd8e0f Mon Sep 17 00:00:00 2001 From: Lakshmi Narayanan Sreethar Date: Thu, 9 Jan 2025 21:20:57 +0530 Subject: [PATCH 255/397] sstable_directory: reintroduce `get_shards_for_this_sstable()` Reintroduce `get_shards_for_this_sstable()` that was removed in commit ad375fbb. This will be used in the following patch to ensure that an sstable is loaded only once. Signed-off-by: Lakshmi Narayanan Sreethar --- sstables/sstable_directory.cc | 8 ++++++++ sstables/sstable_directory.hh | 4 ++++ 2 files changed, 12 insertions(+) diff --git a/sstables/sstable_directory.cc b/sstables/sstable_directory.cc index bae64522c34a..5915d77f7571 100644 --- a/sstables/sstable_directory.cc +++ b/sstables/sstable_directory.cc @@ -521,6 +521,14 @@ sstable_directory::load_foreign_sstables(sstable_entry_descriptor_vector info_ve }); } +future> sstable_directory::get_shards_for_this_sstable( + const sstables::entry_descriptor& desc, const data_dictionary::storage_options& storage_opts, process_flags flags) const { + auto sst = _manager.make_sstable(_schema, storage_opts, desc.generation, _state, desc.version, desc.format, gc_clock::now(), _error_handler_gen); + co_await sst->load_owner_shards(_sharder); + validate(sst, flags); + co_return sst->get_shards_for_this_sstable(); +} + future<> sstable_directory::remove_sstables(std::vector sstlist) { dirlog.debug("Removing {} SSTables", sstlist.size()); diff --git a/sstables/sstable_directory.hh b/sstables/sstable_directory.hh index 697ef5bc826f..08496c6e0401 100644 --- a/sstables/sstable_directory.hh +++ b/sstables/sstable_directory.hh @@ -202,6 +202,10 @@ private: future<> load_foreign_sstables(sstable_entry_descriptor_vector info_vec); + // Compute owner of shards for a particular SSTable. + future> get_shards_for_this_sstable( + const sstables::entry_descriptor& desc, const data_dictionary::storage_options& storage_opts, process_flags flags) const; + sstable_directory(sstables_manager& manager, schema_ptr schema, std::variant, const dht::sharder*> sharder, From 288f9b2b1586d0c955b140716de1024e7b2b7bb8 Mon Sep 17 00:00:00 2001 From: Piotr Smaron Date: Tue, 3 Dec 2024 21:58:28 +0100 Subject: [PATCH 256/397] Introduce LDAP role manager & saslauthd authenticator This PR extends authentication with 2 mechanisms: - a new role_manager subclass, which allows managing users via LDAP server, - a new authenticator, which delegates plaintext authentication to a running saslauthd daemon. The features have been ported from the enterprise repository with their test.py tests and the documentation as part of changing license to source available. Fixes: scylladb/scylla-enterprise#5000 Fixes: scylladb/scylla-enterprise#5001 Closes scylladb/scylladb#22030 --- CMakeLists.txt | 2 + auth/CMakeLists.txt | 8 +- auth/authenticator.cc | 2 + auth/authenticator.hh | 6 + auth/ldap_role_manager.cc | 345 +++++++++++ auth/ldap_role_manager.hh | 116 ++++ auth/saslauthd_authenticator.cc | 202 +++++++ auth/saslauthd_authenticator.hh | 68 +++ bytes.hh | 2 + cmake/FindOpenLDAP.cmake | 46 ++ conf/scylla.yaml | 4 + configure.py | 13 +- db/config.cc | 11 +- db/config.hh | 6 + docs/operating-scylla/security/index.rst | 3 +- .../security/ldap-authentication.rst | 52 ++ .../security/ldap-authorization.rst | 153 +++++ ent/CMakeLists.txt | 1 + ent/ldap/CMakeLists.txt | 17 + ent/ldap/ldap_connection.cc | 487 ++++++++++++++++ ent/ldap/ldap_connection.hh | 212 +++++++ test.py | 288 +++++++++- test/CMakeLists.txt | 1 + test/ldap/CMakeLists.txt | 11 + test/ldap/ldap_common.hh | 25 + test/ldap/ldap_connection_test.cc | 293 ++++++++++ test/ldap/role_manager_test.cc | 537 ++++++++++++++++++ test/ldap/saslauthd_authenticator_test.cc | 204 +++++++ test/ldap/suite.yaml | 3 + test/resource/slapd.conf | 16 + utils/sequential_producer.hh | 46 ++ 31 files changed, 3159 insertions(+), 21 deletions(-) create mode 100644 auth/ldap_role_manager.cc create mode 100644 auth/ldap_role_manager.hh create mode 100644 auth/saslauthd_authenticator.cc create mode 100644 auth/saslauthd_authenticator.hh create mode 100644 cmake/FindOpenLDAP.cmake create mode 100644 docs/operating-scylla/security/ldap-authentication.rst create mode 100644 docs/operating-scylla/security/ldap-authorization.rst create mode 100644 ent/CMakeLists.txt create mode 100644 ent/ldap/CMakeLists.txt create mode 100644 ent/ldap/ldap_connection.cc create mode 100644 ent/ldap/ldap_connection.hh create mode 100644 test/ldap/CMakeLists.txt create mode 100644 test/ldap/ldap_common.hh create mode 100644 test/ldap/ldap_connection_test.cc create mode 100644 test/ldap/role_manager_test.cc create mode 100644 test/ldap/saslauthd_authenticator_test.cc create mode 100644 test/ldap/suite.yaml create mode 100644 test/resource/slapd.conf create mode 100644 utils/sequential_producer.hh diff --git a/CMakeLists.txt b/CMakeLists.txt index 1a19edda5e5e..09d70c0c81bb 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -267,6 +267,7 @@ add_subdirectory(compaction) add_subdirectory(cql3) add_subdirectory(data_dictionary) add_subdirectory(dht) +add_subdirectory(ent) add_subdirectory(gms) add_subdirectory(idl) add_subdirectory(index) @@ -312,6 +313,7 @@ set(scylla_libs idl index lang + ldap locator message mutation diff --git a/auth/CMakeLists.txt b/auth/CMakeLists.txt index 8c7cd26ac442..9530b19e9ff1 100644 --- a/auth/CMakeLists.txt +++ b/auth/CMakeLists.txt @@ -1,4 +1,6 @@ include(add_whole_archive) +find_package(OpenLDAP REQUIRED + ldap) add_library(scylla_auth STATIC) target_sources(scylla_auth @@ -10,6 +12,7 @@ target_sources(scylla_auth certificate_authenticator.cc common.cc default_authorizer.cc + ldap_role_manager.cc password_authenticator.cc passwords.cc permission.cc @@ -18,6 +21,7 @@ target_sources(scylla_auth role_or_anonymous.cc roles-metadata.cc sasl_challenge.cc + saslauthd_authenticator.cc service.cc standard_role_manager.cc transitional.cc @@ -31,12 +35,14 @@ target_link_libraries(scylla_auth xxHash::xxhash PRIVATE absl::headers + OpenLDAP::ldap cql3 idl + ldap wasmtime_bindings libxcrypt::libxcrypt) add_whole_archive(auth scylla_auth) check_headers(check-headers scylla_auth - GLOB_RECURSE ${CMAKE_CURRENT_SOURCE_DIR}/*.hh) + GLOB_RECURSE ${CMAKE_CURRENT_SOURCE_DIR}/*.hh) \ No newline at end of file diff --git a/auth/authenticator.cc b/auth/authenticator.cc index 500556fa040c..908cb8ea6d06 100644 --- a/auth/authenticator.cc +++ b/auth/authenticator.cc @@ -14,6 +14,8 @@ const sstring auth::authenticator::USERNAME_KEY("username"); const sstring auth::authenticator::PASSWORD_KEY("password"); +const sstring auth::authenticator::SERVICE_KEY("service"); +const sstring auth::authenticator::REALM_KEY("realm"); future> auth::authenticator::authenticate(session_dn_func) const { return make_ready_future>(std::nullopt); diff --git a/auth/authenticator.hh b/auth/authenticator.hh index c0e6a51e4331..1a0f9f9be642 100644 --- a/auth/authenticator.hh +++ b/auth/authenticator.hh @@ -67,6 +67,12 @@ public: /// static const sstring PASSWORD_KEY; + /// Service for SASL authentication. + static const sstring SERVICE_KEY; + + /// Realm for SASL authentication. + static const sstring REALM_KEY; + using credentials_map = std::unordered_map; virtual ~authenticator() = default; diff --git a/auth/ldap_role_manager.cc b/auth/ldap_role_manager.cc new file mode 100644 index 000000000000..db5ef6d6a731 --- /dev/null +++ b/auth/ldap_role_manager.cc @@ -0,0 +1,345 @@ +/* + * Copyright (C) 2019 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "ldap_role_manager.hh" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common.hh" +#include "cql3/query_processor.hh" +#include "exceptions/exceptions.hh" +#include "seastarx.hh" +#include "service/raft/raft_group0_client.hh" +#include "utils/class_registrator.hh" +#include "db/config.hh" +#include "utils/exponential_backoff_retry.hh" + +namespace { + +logger mylog{"ldap_role_manager"}; // `log` is taken by math. + +struct url_desc_deleter { + void operator()(LDAPURLDesc *p) { + ldap_free_urldesc(p); + } +}; + +using url_desc_ptr = std::unique_ptr; + +url_desc_ptr parse_url(std::string_view url) { + LDAPURLDesc *desc = nullptr; + if (ldap_url_parse(url.data(), &desc)) { + mylog.error("error in ldap_url_parse({})", url); + } + return url_desc_ptr(desc); +} + +/// Extracts attribute \p attr from all entries in \p res. +std::vector get_attr_values(LDAP* ld, LDAPMessage* res, const char* attr) { + std::vector values; + mylog.debug("Analyzing search results"); + for (auto e = ldap_first_entry(ld, res); e; e = ldap_next_entry(ld, e)) { + struct deleter { + void operator()(berval** p) { ldap_value_free_len(p); } + void operator()(char* p) { ldap_memfree(p); } + }; + const std::unique_ptr dname(ldap_get_dn(ld, e)); + mylog.debug("Analyzing entry {}", dname.get()); + const std::unique_ptr vals(ldap_get_values_len(ld, e, attr)); + if (!vals) { + mylog.warn("LDAP entry {} has no attribute {}", dname.get(), attr); + continue; + } + for (size_t i = 0; vals.get()[i]; ++i) { + values.emplace_back(vals.get()[i]->bv_val, vals.get()[i]->bv_len); + } + } + mylog.debug("Done analyzing search results; extracted roles {}", values); + return values; +} + +const char* ldap_role_manager_full_name = "com.scylladb.auth.LDAPRoleManager"; + +} // anonymous namespace + +namespace auth { + +static const class_registrator< + role_manager, + ldap_role_manager, + cql3::query_processor&, + ::service::raft_group0_client&, + ::service::migration_manager&> registration(ldap_role_manager_full_name); + +ldap_role_manager::ldap_role_manager( + std::string_view query_template, std::string_view target_attr, std::string_view bind_name, std::string_view bind_password, + cql3::query_processor& qp, ::service::raft_group0_client& rg0c, ::service::migration_manager& mm) + : _std_mgr(qp, rg0c, mm), _group0_client(rg0c), _query_template(query_template), _target_attr(target_attr), _bind_name(bind_name) + , _bind_password(bind_password) + , _connection_factory(bind(std::mem_fn(&ldap_role_manager::reconnect), std::ref(*this))) { +} + +ldap_role_manager::ldap_role_manager(cql3::query_processor& qp, ::service::raft_group0_client& rg0c, ::service::migration_manager& mm) + : ldap_role_manager( + qp.db().get_config().ldap_url_template(), + qp.db().get_config().ldap_attr_role(), + qp.db().get_config().ldap_bind_dn(), + qp.db().get_config().ldap_bind_passwd(), + qp, + rg0c, + mm) { +} + +std::string_view ldap_role_manager::qualified_java_name() const noexcept { + return ldap_role_manager_full_name; +} + +const resource_set& ldap_role_manager::protected_resources() const { + return _std_mgr.protected_resources(); +} + +future<> ldap_role_manager::start() { + if (!parse_url(get_url("dummy-user"))) { // Just need host and port -- any user should do. + return make_exception_future( + std::runtime_error(fmt::format("error getting LDAP server address from template {}", _query_template))); + } + return _std_mgr.start(); +} + +using conn_ptr = lw_shared_ptr; + +future ldap_role_manager::connect() { + const auto desc = parse_url(get_url("dummy-user")); // Just need host and port -- any user should do. + if (!desc) { + co_return coroutine::exception(std::make_exception_ptr(std::runtime_error("connect attempted before a successful start"))); + } + net::inet_address host = co_await net::dns::resolve_name(desc->lud_host); + const socket_address addr(host, uint16_t(desc->lud_port)); + connected_socket sock = co_await seastar::connect(addr); + auto conn = make_lw_shared(std::move(sock)); + sstring error; + try { + ldap_msg_ptr response = co_await conn->simple_bind(_bind_name.c_str(), _bind_password.c_str()); + if (!response || ldap_msgtype(response.get()) != LDAP_RES_BIND) { + error = format("simple_bind error: {}", conn->get_error()); + } + } catch (...) { + error = format("connect error: {}", std::current_exception()); + } + if (!error.empty()) { + co_await conn->close(); + co_return coroutine::exception(std::make_exception_ptr(std::runtime_error(std::move(error)))); + } + co_return std::move(conn); +} + +future ldap_role_manager::reconnect() { + unsigned retries_left = 5; + using namespace std::literals::chrono_literals; + conn_ptr conn = co_await exponential_backoff_retry::do_until_value(1s, 32s, _as, [this, &retries_left] () -> future> { + if (!retries_left) { + co_return conn_ptr{}; + } + mylog.trace("reconnect() retrying ({} attempts left)", retries_left); + --retries_left; + try { + co_return co_await connect(); + } catch (...) { + mylog.error("error in reconnect: {}", std::current_exception()); + } + co_return std::nullopt; + }); + + mylog.trace("reconnect() finished backoff, conn={}", reinterpret_cast(conn.get())); + if (conn) { + co_return std::move(conn); + } + co_return coroutine::exception(std::make_exception_ptr(std::runtime_error("reconnect failed after 5 attempts"))); +} + +future<> ldap_role_manager::stop() { + _as.request_abort(); + return _std_mgr.stop().then([this] { return _connection_factory.stop(); }); +} + +future<> ldap_role_manager::create(std::string_view name, const role_config& config, ::service::group0_batch& mc) { + return _std_mgr.create(name, config, mc); +} + +future<> ldap_role_manager::drop(std::string_view name, ::service::group0_batch& mc) { + return _std_mgr.drop(name, mc); +} + +future<> ldap_role_manager::alter(std::string_view name, const role_config_update& config, ::service::group0_batch& mc) { + return _std_mgr.alter(name, config, mc); +} + +future<> ldap_role_manager::grant(std::string_view, std::string_view, ::service::group0_batch& mc) { + return make_exception_future<>(exceptions::invalid_request_exception("Cannot grant roles with LDAPRoleManager.")); +} + +future<> ldap_role_manager::revoke(std::string_view, std::string_view, ::service::group0_batch& mc) { + return make_exception_future<>(exceptions::invalid_request_exception("Cannot revoke roles with LDAPRoleManager.")); +} + +future ldap_role_manager::query_granted(std::string_view grantee_name, recursive_role_query) { + const auto url = get_url(grantee_name.data()); + auto desc = parse_url(url); + if (!desc) { + return make_exception_future(std::runtime_error(format("Error parsing URL {}", url))); + } + return _connection_factory.with_connection([this, desc = std::move(desc), grantee_name_ = sstring(grantee_name)] + (ldap_connection& conn) -> future { + sstring grantee_name = std::move(grantee_name_); + ldap_msg_ptr res = co_await conn.search(desc->lud_dn, desc->lud_scope, desc->lud_filter, desc->lud_attrs, + /*attrsonly=*/0, /*serverctrls=*/nullptr, /*clientctrls=*/nullptr, + /*timeout=*/nullptr, /*sizelimit=*/0); + mylog.trace("query_granted: got search results"); + const auto mtype = ldap_msgtype(res.get()); + if (mtype != LDAP_RES_SEARCH_ENTRY && mtype != LDAP_RES_SEARCH_RESULT && mtype != LDAP_RES_SEARCH_REFERENCE) { + mylog.error("ldap search yielded result {} of type {}", static_cast(res.get()), mtype); + co_return coroutine::exception(std::make_exception_ptr(std::runtime_error("ldap_role_manager: search result has wrong type"))); + } + std::vector values = get_attr_values(conn.get_ldap(), res.get(), _target_attr.c_str()); + auth::role_set valid_roles{grantee_name}; + + // Each value is a role to be granted. + co_await parallel_for_each(values, [this, &valid_roles] (const sstring& ldap_role) { + return _std_mgr.exists(ldap_role).then([&valid_roles, &ldap_role] (bool exists) { + if (exists) { + valid_roles.insert(ldap_role); + } else { + mylog.error("unrecognized role received from LDAP: {}", ldap_role); + } + }); + }); + + co_return std::move(valid_roles); + }); +} + +future +ldap_role_manager::query_all_directly_granted() { + role_to_directly_granted_map result; + auto roles = co_await query_all(); + for (auto& role: roles) { + auto granted_set = co_await query_granted(role, recursive_role_query::no); + for (auto& granted: granted_set) { + if (granted != role) { + result.insert({role, granted}); + } + } + } + co_return result; +} + +future ldap_role_manager::query_all() { + return _std_mgr.query_all(); +} + +future<> ldap_role_manager::create_role(std::string_view role_name) { + return smp::submit_to(0, [this, role_name] () -> future<> { + int retries = 10; + while (true) { + auto guard = co_await _group0_client.start_operation(_as, ::service::raft_timeout{}); + ::service::group0_batch batch(std::move(guard)); + auto cfg = role_config{.can_login = true}; + try { + co_await create(role_name, cfg, batch); + co_await std::move(batch).commit(_group0_client, _as, ::service::raft_timeout{}); + } catch (const role_already_exists&) { + // ok + } catch (const ::service::group0_concurrent_modification& ex) { + mylog.warn("Failed to auto-create role \"{}\" due to guard conflict.{}.", + role_name, retries ? " Retrying" : " Number of retries exceeded, giving up"); + if (retries--) { + continue; + } + throw; + } + break; + } + // make sure to wait until create mutations are applied locally + (void)(co_await _group0_client.start_operation(_as, ::service::raft_timeout{})); + }); +} + +future ldap_role_manager::exists(std::string_view role_name) { + bool exists = co_await _std_mgr.exists(role_name); + if (exists) { + co_return true; + } + role_set roles = co_await query_granted(role_name, recursive_role_query::yes); + // A role will get auto-created if it's already assigned any permissions. + // The role set will always contains at least a single entry (the role itself), + // so auto-creation is only triggered if at least one more external role is assigned. + if (roles.size() > 1) { + mylog.info("Auto-creating user {}", role_name); + try { + co_await create_role(role_name); + exists = true; + } catch (...) { + mylog.error("Failed to auto-create role {}: {}", role_name, std::current_exception()); + exists = false; + } + co_return exists; + } + mylog.debug("Role {} will not be auto-created", role_name); + co_return false; +} + +future ldap_role_manager::is_superuser(std::string_view role_name) { + return _std_mgr.is_superuser(role_name); +} + +future ldap_role_manager::can_login(std::string_view role_name) { + return _std_mgr.can_login(role_name); +} + +future> ldap_role_manager::get_attribute( + std::string_view role_name, std::string_view attribute_name) { + return _std_mgr.get_attribute(role_name, attribute_name); +} + +future ldap_role_manager::query_attribute_for_all(std::string_view attribute_name) { + return _std_mgr.query_attribute_for_all(attribute_name); +} + +future<> ldap_role_manager::set_attribute( + std::string_view role_name, std::string_view attribute_name, std::string_view attribute_value, ::service::group0_batch& mc) { + return _std_mgr.set_attribute(role_name, attribute_value, attribute_value, mc); +} + +future<> ldap_role_manager::remove_attribute(std::string_view role_name, std::string_view attribute_name, ::service::group0_batch& mc) { + return _std_mgr.remove_attribute(role_name, attribute_name, mc); +} + +sstring ldap_role_manager::get_url(std::string_view user) const { + return boost::replace_all_copy(_query_template, "{USER}", user); +} + +future> ldap_role_manager::describe_role_grants() { + // Since grants are performed by the ldap admin, we shouldn't echo them back + co_return std::vector(); +} + +future<> ldap_role_manager::ensure_superuser_is_created() { + // ldap is responsible for users + co_return; +} + +} // namespace auth diff --git a/auth/ldap_role_manager.hh b/auth/ldap_role_manager.hh new file mode 100644 index 000000000000..b65c87546833 --- /dev/null +++ b/auth/ldap_role_manager.hh @@ -0,0 +1,116 @@ +/* + * Copyright (C) 2019 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + + +#pragma once + +#include +#include +#include + +#include "bytes.hh" +#include "ent/ldap/ldap_connection.hh" +#include "standard_role_manager.hh" + +namespace auth { + +/// Queries an LDAP server for roles. +/// +/// Since LDAP grants and revokes roles, calling grant() and revoke() is disallowed. +/// +/// We query LDAP for a list of a particular user's roles, and the results must match roles that exist in the +/// database. Furthermore, the user must have already authenticated to Scylla, meaning it, too, exists in the +/// database. Therefore, some of the role_manager functionality is provided by a standard_role_manager under +/// the hood. For example, listing all roles or checking if the user can login cannot currently be determined +/// by querying LDAP, so they are delegated to the standard_role_manager. +class ldap_role_manager : public role_manager { + standard_role_manager _std_mgr; + ::service::raft_group0_client& _group0_client; + seastar::sstring _query_template; ///< LDAP URL dictating which query to make. + seastar::sstring _target_attr; ///< LDAP entry attribute containing the Scylla role name. + seastar::sstring _bind_name; ///< Username for LDAP simple bind. + seastar::sstring _bind_password; ///< Password for LDAP simple bind. + mutable ldap_reuser _connection_factory; // Potentially modified by query_granted(). + seastar::abort_source _as; + public: + ldap_role_manager( + std::string_view query_template, ///< LDAP query template as described in Scylla documentation. + std::string_view target_attr, ///< LDAP entry attribute containing the Scylla role name. + std::string_view bind_name, ///< LDAP bind credentials. + std::string_view bind_password, ///< LDAP bind credentials. + cql3::query_processor& qp, ///< Passed to standard_role_manager. + ::service::raft_group0_client& rg0c, ///< Passed to standard_role_manager. + ::service::migration_manager& mm ///< Passed to standard_role_manager. + ); + + /// Retrieves LDAP configuration entries from qp and invokes the other constructor. Required by + /// class_registrator. + ldap_role_manager(cql3::query_processor& qp, ::service::raft_group0_client& rg0c, ::service::migration_manager& mm); + + /// Thrown when query-template parsing fails. + struct url_error : public std::runtime_error { + using runtime_error::runtime_error; + }; + + std::string_view qualified_java_name() const noexcept override; + + const resource_set& protected_resources() const override; + + future<> start() override; + + future<> stop() override; + + future<> create(std::string_view, const role_config&, ::service::group0_batch& mc) override; + + future<> drop(std::string_view, ::service::group0_batch& mc) override; + + future<> alter(std::string_view, const role_config_update&, ::service::group0_batch& mc) override; + + future<> grant(std::string_view, std::string_view, ::service::group0_batch& mc) override; + + future<> revoke(std::string_view, std::string_view, ::service::group0_batch& mc) override; + + future query_granted(std::string_view, recursive_role_query) override; + + future query_all_directly_granted() override; + + future query_all() override; + + future exists(std::string_view) override; + + future is_superuser(std::string_view) override; + + future can_login(std::string_view) override; + + future> get_attribute(std::string_view, std::string_view) override; + + future query_attribute_for_all(std::string_view) override; + + future<> set_attribute(std::string_view, std::string_view, std::string_view, ::service::group0_batch& mc) override; + + future<> remove_attribute(std::string_view, std::string_view, ::service::group0_batch& mc) override; + + future> describe_role_grants() override; + private: + /// Connects to the LDAP server indicated by _query_template and executes LDAP bind using _bind_name and + /// _bind_password. Returns the resulting ldap_connection. + future> connect(); + + /// Invokes connect() repeatedly with backoff, until it succeeds or retry limit is reached. + future> reconnect(); + + /// Macro-expands _query_template, returning the result. + sstring get_url(std::string_view user) const; + + /// Used to auto-create roles returned by ldap. + future<> create_role(std::string_view role_name); + + future<> ensure_superuser_is_created() override; +}; + +} // namespace auth diff --git a/auth/saslauthd_authenticator.cc b/auth/saslauthd_authenticator.cc new file mode 100644 index 000000000000..c0ddb2ebd80d --- /dev/null +++ b/auth/saslauthd_authenticator.cc @@ -0,0 +1,202 @@ +/* + * Copyright (C) 2020 ScyllaDB + * + * Modified by ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "auth/saslauthd_authenticator.hh" + +#include +#include +#include +#include +#include +#include +#include +#include "common.hh" +#include "cql3/query_processor.hh" +#include "db/config.hh" +#include "utils/log.hh" +#include "seastarx.hh" +#include "utils/class_registrator.hh" + +namespace auth { + +static logging::logger mylog("saslauthd_authenticator"); + +// To ensure correct initialization order, we unfortunately need to use a string literal. +static const class_registrator< + authenticator, + saslauthd_authenticator, + cql3::query_processor&, + ::service::raft_group0_client&, + ::service::migration_manager&> saslauthd_auth_reg("com.scylladb.auth.SaslauthdAuthenticator"); + +saslauthd_authenticator::saslauthd_authenticator(cql3::query_processor& qp, ::service::raft_group0_client&, ::service::migration_manager&) + : _socket_path(qp.db().get_config().saslauthd_socket_path()) +{} + +future<> saslauthd_authenticator::start() { + return once_among_shards([this] { + return file_exists(_socket_path).then([this] (bool exists) { + if (!exists) { + mylog.warn("saslauthd socket file {} doesn't exist -- is saslauthd running?", _socket_path); + } + return make_ready_future(); + }); + }); +} + +future<> saslauthd_authenticator::stop() { return make_ready_future(); } + +std::string_view saslauthd_authenticator::qualified_java_name() const { + return "com.scylladb.auth.SaslauthdAuthenticator"; +} + +bool saslauthd_authenticator::require_authentication() const { + return true; +} + +authentication_option_set saslauthd_authenticator::supported_options() const { + return authentication_option_set{authentication_option::password, authentication_option::options}; +} + +authentication_option_set saslauthd_authenticator::alterable_options() const { + return supported_options(); +} + +namespace { + +// Note the saslauthd protocol description: +// https://github.com/cyrusimap/cyrus-sasl/blob/f769dde423e1b3ae8bfb35b826fca3d5f1e1f6fe/saslauthd/saslauthd-main.c#L74 + +constexpr size_t len_size = sizeof(htons(0)); + +char* pack(std::string_view s, char* p) { + uint16_t size = s.size(); + produce_be(p, size); + memcpy(p, s.data(), size); + return p + size; +} + +temporary_buffer make_saslauthd_message(const saslauthd_credentials& creds) { + temporary_buffer message( + creds.username.size() + creds.password.size() + creds.service.size() + creds.realm.size() + + 4 * len_size); + auto p = pack(creds.username, message.get_write()); + p = pack(creds.password, p); + p = pack(creds.service, p); + p = pack(creds.realm, p); + return message; +} + +/// An exception handler that reports saslauthd socket IO error. +future as_authentication_exception(std::exception_ptr ex) { + return make_exception_future( + exceptions::authentication_exception(format("saslauthd socket IO error: {}", ex))); +} + +} // anonymous namespace + +future authenticate_with_saslauthd(sstring saslauthd_socket_path, const saslauthd_credentials& creds) { + socket_address addr((unix_domain_addr(saslauthd_socket_path))); + // TODO: switch to seastar::connect() when it supports Unix domain sockets. + return engine().net().connect(addr).then([creds = std::move(creds)] (connected_socket s) { + return do_with( + s.input(), s.output(), + [creds = std::move(creds)] (input_stream& in, output_stream& out) { + return out.write(make_saslauthd_message(creds)).then([&in, &out] () mutable { + return out.flush().then([&in] () mutable { + return in.read_exactly(2).then([&in] (temporary_buffer len) mutable { + if (len.size() < 2) { + return make_exception_future( + exceptions::authentication_exception( + "saslauthd closed connection before completing response")); + } + const auto paylen = read_be(len.get()); + return in.read_exactly(paylen).then([paylen] (temporary_buffer resp) { + mylog.debug("saslauthd response: {}", std::string_view(resp.get(), resp.size())); + if (resp.size() != paylen) { + return make_exception_future( + exceptions::authentication_exception( + // We say "different" here, though we could just as well say + // "shorter". A longer response is cut to size by + // read_exactly(). + "saslauthd response length different than promised")); + } + bool ok = (resp.size() >= 2 && resp[0] == 'O' && resp[1] == 'K'); + return make_ready_future(ok); + }); + }).finally([&in] () mutable { return in.close(); }); + }).handle_exception(as_authentication_exception).finally([&out] () mutable { + return out.close(); + }); + }); + }); + }).handle_exception_type([] (std::system_error& e) { + return make_exception_future( + exceptions::authentication_exception(format("saslauthd socket connection error: {}", e.what()))); + }); +} + +future saslauthd_authenticator::authenticate(const credentials_map& credentials) const { + const auto username_found = credentials.find(USERNAME_KEY); + if (username_found == credentials.end()) { + throw exceptions::authentication_exception(format("Required key '{}' is missing", USERNAME_KEY)); + } + const auto password_found = credentials.find(PASSWORD_KEY); + if (password_found == credentials.end()) { + throw exceptions::authentication_exception(format("Required key '{}' is missing", PASSWORD_KEY)); + } + const auto service_found = credentials.find(SERVICE_KEY); + const auto realm_found = credentials.find(REALM_KEY); + + sstring username = username_found->second; + return authenticate_with_saslauthd(_socket_path, {username, password_found->second, + service_found == credentials.end() ? "" : service_found->second, + realm_found == credentials.end() ? "" : realm_found->second}).then([username] (bool ok) { + if (!ok) { + throw exceptions::authentication_exception("Incorrect credentials"); + } + return make_ready_future(username); + }); +} + +future<> saslauthd_authenticator::create(std::string_view role_name, const authentication_options& options, ::service::group0_batch& mc) { + if (!options.credentials) { + return make_ready_future<>(); + } + throw exceptions::authentication_exception("Cannot create passwords with SaslauthdAuthenticator"); +} + +future<> saslauthd_authenticator::alter(std::string_view role_name, const authentication_options& options, ::service::group0_batch& mc) { + if (!options.credentials) { + return make_ready_future<>(); + } + throw exceptions::authentication_exception("Cannot modify passwords with SaslauthdAuthenticator"); +} + +future<> saslauthd_authenticator::drop(std::string_view name, ::service::group0_batch& mc) { + throw exceptions::authentication_exception("Cannot delete passwords with SaslauthdAuthenticator"); +} + +future saslauthd_authenticator::query_custom_options(std::string_view role_name) const { + return make_ready_future(); +} + +const resource_set& saslauthd_authenticator::protected_resources() const { + static const resource_set empty; + return empty; +} + +::shared_ptr saslauthd_authenticator::new_sasl_challenge() const { + return ::make_shared([this](std::string_view username, std::string_view password) { + return this->authenticate(credentials_map{{USERNAME_KEY, sstring(username)}, {PASSWORD_KEY, sstring(password)}}); + }); +} + +} // namespace auth diff --git a/auth/saslauthd_authenticator.hh b/auth/saslauthd_authenticator.hh new file mode 100644 index 000000000000..18a7f86df1cf --- /dev/null +++ b/auth/saslauthd_authenticator.hh @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2020 ScyllaDB + * + * Modified by ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "auth/authenticator.hh" + +namespace cql3 { +class query_processor; +} + +namespace service { +class migration_manager; +class raft_group0_client; +} + +namespace auth { + +/// Delegates authentication to saslauthd. When this class is asked to authenticate, it passes the credentials +/// to saslauthd, gets its response, and allows or denies authentication based on that response. +class saslauthd_authenticator : public authenticator { + sstring _socket_path; ///< Path to the domain socket on which saslauthd is listening. +public: + saslauthd_authenticator(cql3::query_processor&, ::service::raft_group0_client&, ::service::migration_manager&); + + future<> start() override; + + future<> stop() override; + + std::string_view qualified_java_name() const override; + + bool require_authentication() const override; + + authentication_option_set supported_options() const override; + + authentication_option_set alterable_options() const override; + + future authenticate(const credentials_map& credentials) const override; + + future<> create(std::string_view role_name, const authentication_options& options, ::service::group0_batch& mc) override; + + future<> alter(std::string_view role_name, const authentication_options& options, ::service::group0_batch& mc) override; + + future<> drop(std::string_view role_name, ::service::group0_batch& mc) override; + + future query_custom_options(std::string_view role_name) const override; + + const resource_set& protected_resources() const override; + + ::shared_ptr new_sasl_challenge() const override; +}; + +/// A set of four credential strings that saslauthd expects. +struct saslauthd_credentials { + sstring username, password, service, realm; +}; + +future authenticate_with_saslauthd(sstring saslauthd_socket_path, const saslauthd_credentials& creds); + +} + diff --git a/bytes.hh b/bytes.hh index 1891cc6ae2c8..de2a05c3499a 100644 --- a/bytes.hh +++ b/bytes.hh @@ -20,6 +20,8 @@ #include "utils/mutable_view.hh" #include "utils/simple_hashers.hh" +using sstring_view = std::string_view; + inline bytes to_bytes(bytes&& b) { return std::move(b); } diff --git a/cmake/FindOpenLDAP.cmake b/cmake/FindOpenLDAP.cmake new file mode 100644 index 000000000000..294014f9c891 --- /dev/null +++ b/cmake/FindOpenLDAP.cmake @@ -0,0 +1,46 @@ +# +# Copyright 2024-present ScyllaDB +# + +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# +find_package(PkgConfig REQUIRED) + +foreach(component ${OpenLDAP_FIND_COMPONENTS}) + pkg_search_module(PC_${component} QUIET ${component}) + find_path (OpenLDAP_${component}_INCLUDE_DIR + NAMES lber.h + HINTS + ${PC_${component}_INCLUDEDIR} + ${PC_${component}_INCLUDE_DIRS}) + find_library(OpenLDAP_${component}_LIBRARY + NAMES ${component} + HINTS + ${PC_${component}_LIBDIR} + ${PC_${component}_LIBRARY_DIRS}) + list(APPEND OpenLDAP_INCLUDE_DIRS OpenLDAP_${component}_INCLUDE_DIR) + list(APPEND OpenLDAP_LIBRARIES OpenLDAP_${component}_LIBRARY) +endforeach() + +include(FindPackageHandleStandardArgs) +find_package_handle_standard_args(OpenLDAP + DEFAULT_MSG + ${OpenLDAP_INCLUDE_DIRS} + ${OpenLDAP_LIBRARIES}) + +mark_as_advanced( + ${OpenLDAP_INCLUDE_DIRS} + ${OpenLDAP_LIBRARIES}) + +if(OpenLDAP_FOUND) + foreach(component ${OpenLDAP_FIND_COMPONENTS}) + if(NOT TARGET OpenLDAP::${component}) + add_library(OpenLDAP::${component} UNKNOWN IMPORTED) + set_target_properties(OpenLDAP::${component} PROPERTIES + INTERFACE_INCLUDE_DIRECTORIES "${OpenLDAP_${component}_INCLUDE_DIR}" + IMPORTED_LINK_INTERFACE_LANGUAGES "C" + IMPORTED_LOCATION "${OpenLDAP_${component}_LIBRARY}") + endif() + endforeach() +endif() diff --git a/conf/scylla.yaml b/conf/scylla.yaml index 46becd020390..898581778afd 100644 --- a/conf/scylla.yaml +++ b/conf/scylla.yaml @@ -244,7 +244,11 @@ batch_size_fail_threshold_in_kb: 1024 # - com.scylladb.auth.TransitionalAuthenticator requires username/password pair # to authenticate in the same manner as PasswordAuthenticator, but improper credentials # result in being logged in as an anonymous user. Use for upgrading clusters' auth. +# - com.scylladb.auth.SaslauthdAuthenticator outsources authentication to a running saslauthd +# daemon. When using this authenticator, you must set the saslauthd_socket_path property to the +# Unix domain socket on which saslauthd is listening. # authenticator: AllowAllAuthenticator +# saslauthd_socket_path: /var/state/saslauthd/mux # Authorization backend, implementing IAuthorizer; used to limit access/provide permissions # Out of the box, Scylla provides org.apache.cassandra.auth.{AllowAllAuthorizer, diff --git a/configure.py b/configure.py index 03fe64ba8e54..225ea0e0b3d2 100755 --- a/configure.py +++ b/configure.py @@ -436,6 +436,12 @@ def find_ninja(): }, } +ldap_tests = set([ + 'test/ldap/ldap_connection_test', + 'test/ldap/role_manager_test', + 'test/ldap/saslauthd_authenticator_test' +]) + scylla_tests = set([ 'test/boost/combined_tests', 'test/boost/UUID_test', @@ -587,7 +593,7 @@ def find_ninja(): 'test/unit/row_cache_stress_test', 'test/unit/cross_shard_barrier_test', 'test/boost/address_map_test', -]) +]) | ldap_tests perf_tests = set([ 'test/perf/perf_mutation_readers', @@ -1103,11 +1109,13 @@ def find_ninja(): 'auth/permissions_cache.cc', 'auth/service.cc', 'auth/standard_role_manager.cc', + 'auth/ldap_role_manager.cc', 'auth/transitional.cc', 'auth/maintenance_socket_role_manager.cc', 'auth/role_or_anonymous.cc', 'auth/sasl_challenge.cc', 'auth/certificate_authenticator.cc', + 'auth/saslauthd_authenticator.cc', 'tracing/tracing.cc', 'tracing/trace_keyspace_helper.cc', 'tracing/trace_state.cc', @@ -1123,6 +1131,7 @@ def find_ninja(): 'utils/arch/powerpc/crc32-vpmsum/crc32_wrapper.cc', 'querier.cc', 'mutation_writer/multishard_writer.cc', + 'ent/ldap/ldap_connection.cc', 'multishard_mutation_query.cc', 'reader_concurrency_semaphore.cc', 'sstables_loader.cc', @@ -2001,7 +2010,7 @@ def query_seastar_flags(pc_file, use_shared_libs, link_static_cxx=False): libs = ' '.join([maybe_static(args.staticyamlcpp, '-lyaml-cpp'), '-latomic', '-lz', '-lsnappy', - ' -lstdc++fs', ' -lcrypt', ' -lcryptopp', ' -lpthread', + ' -lstdc++fs', ' -lcrypt', ' -lcryptopp', ' -lpthread', ' -lldap -llber', # Must link with static version of libzstd, since # experimental APIs that we use are only present there. maybe_static(True, '-lzstd'), diff --git a/db/config.cc b/db/config.cc index d6adb1e34aca..46cb9dc732ed 100644 --- a/db/config.cc +++ b/db/config.cc @@ -961,9 +961,10 @@ db::config::config(std::shared_ptr exts) "* org.apache.cassandra.auth.PasswordAuthenticator: Authenticates users with user names and hashed passwords stored in the system_auth.credentials table. If you use the default, 1, and the node with the lone replica goes down, you will not be able to log into the cluster because the system_auth keyspace was not replicated.\n" "* com.scylladb.auth.CertificateAuthenticator: Authenticates users based on TLS certificate authentication subject. Roles and permissions still need to be defined as normal. Super user can be set using the 'auth_superuser_name' configuration value. Query to extract role name from subject string is set using 'auth_certificate_role_queries'.\n" "* com.scylladb.auth.TransitionalAuthenticator: Wraps around the PasswordAuthenticator, logging them in if username/password pair provided is correct and treating them as anonymous users otherwise.\n" + "* com.scylladb.auth.SaslauthdAuthenticator : Use saslauthd for authentication.\n" "\n" "Related information: Internal authentication", - {"AllowAllAuthenticator", "PasswordAuthenticator", "CertificateAuthenticator", "org.apache.cassandra.auth.PasswordAuthenticator", "org.apache.cassandra.auth.AllowAllAuthenticator", "com.scylladb.auth.TransitionalAuthenticator", "com.scylladb.auth.CertificateAuthenticator"}) + {"AllowAllAuthenticator", "PasswordAuthenticator", "CertificateAuthenticator", "org.apache.cassandra.auth.PasswordAuthenticator", "com.scylladb.auth.SaslauthdAuthenticator", "org.apache.cassandra.auth.AllowAllAuthenticator", "com.scylladb.auth.TransitionalAuthenticator", "com.scylladb.auth.CertificateAuthenticator"}) , internode_authenticator(this, "internode_authenticator", value_status::Unused, "enabled", "Internode authentication backend. It implements org.apache.cassandra.auth.AllowAllInternodeAuthenticator to allows or disallow connections from peer nodes.") , authorizer(this, "authorizer", value_status::Used, "org.apache.cassandra.auth.AllowAllAuthorizer", @@ -977,7 +978,8 @@ db::config::config(std::shared_ptr exts) , role_manager(this, "role_manager", value_status::Used, "org.apache.cassandra.auth.CassandraRoleManager", "The role-management backend, used to maintain grants and memberships between roles." "The available role-managers are:\n" - "* CassandraRoleManager: Stores role data in the system_auth keyspace.") + "* org.apache.cassandra.auth.CassandraRoleManager: Stores role data in the system_auth keyspace;\n" + "* com.scylladb.auth.LDAPRoleManager: Fetches role data from an LDAP server.") , permissions_validity_in_ms(this, "permissions_validity_in_ms", liveness::LiveUpdate, value_status::Used, 10000, "How long permissions in cache remain valid. Depending on the authorizer, such as CassandraAuthorizer, fetching permissions can be resource intensive. Permissions caching is disabled when this property is set to 0 or when AllowAllAuthorizer is used. The cached value is considered valid as long as both its value is not older than the permissions_validity_in_ms " "and the cached value has been read at least once during the permissions_validity_in_ms time frame. If any of these two conditions doesn't hold the cached value is going to be evicted from the cache.\n" @@ -1253,6 +1255,11 @@ db::config::config(std::shared_ptr exts) , replication_strategy_warn_list(this, "replication_strategy_warn_list", liveness::LiveUpdate, value_status::Used, {locator::replication_strategy_type::simple}, "Controls which replication strategies to warn about when creating/altering a keyspace. Doesn't affect the pre-existing keyspaces.") , replication_strategy_fail_list(this, "replication_strategy_fail_list", liveness::LiveUpdate, value_status::Used, {}, "Controls which replication strategies are disallowed to be used when creating/altering a keyspace. Doesn't affect the pre-existing keyspaces.") , service_levels_interval(this, "service_levels_interval_ms", liveness::LiveUpdate, value_status::Used, 10000, "Controls how often service levels module polls configuration table") + , ldap_url_template(this, "ldap_url_template", value_status::Used, "", "LDAP URL template used by LDAPRoleManager for crafting queries.") + , ldap_attr_role(this, "ldap_attr_role", value_status::Used, "", "LDAP attribute containing Scylla role.") + , ldap_bind_dn(this, "ldap_bind_dn", value_status::Used, "", "Distinguished name used by LDAPRoleManager for binding to LDAP server.") + , ldap_bind_passwd(this, "ldap_bind_passwd", value_status::Used, "", "Password used by LDAPRoleManager for binding to LDAP server.") + , saslauthd_socket_path(this, "saslauthd_socket_path", value_status::Used, "", "UNIX domain socket on which saslauthd is listening.") , error_injections_at_startup(this, "error_injections_at_startup", error_injection_value_status, {}, "List of error injections that should be enabled on startup.") , topology_barrier_stall_detector_threshold_seconds(this, "topology_barrier_stall_detector_threshold_seconds", value_status::Used, 2, "Report sites blocking topology barrier if it takes longer than this.") , enable_tablets(this, "enable_tablets", value_status::Used, false, "Enable tablets for newly created keyspaces.") diff --git a/db/config.hh b/db/config.hh index 6e001c832a45..0a018250e64b 100644 --- a/db/config.hh +++ b/db/config.hh @@ -504,6 +504,12 @@ public: named_value service_levels_interval; + named_value ldap_url_template; + named_value ldap_attr_role; + named_value ldap_bind_dn; + named_value ldap_bind_passwd; + named_value saslauthd_socket_path; + seastar::logging_settings logging_settings(const log_cli::options&) const; const db::extensions& extensions() const; diff --git a/docs/operating-scylla/security/index.rst b/docs/operating-scylla/security/index.rst index 6d9c4cf38072..63ea2db83571 100644 --- a/docs/operating-scylla/security/index.rst +++ b/docs/operating-scylla/security/index.rst @@ -18,7 +18,8 @@ Security node-node-encryption generate-certificate saslauthd - + ldap-authentication + ldap-authorization .. panel-box:: :title: Security diff --git a/docs/operating-scylla/security/ldap-authentication.rst b/docs/operating-scylla/security/ldap-authentication.rst new file mode 100644 index 000000000000..953e70a37ec1 --- /dev/null +++ b/docs/operating-scylla/security/ldap-authentication.rst @@ -0,0 +1,52 @@ +=================== +LDAP Authentication +=================== + +.. toctree:: + :hidden: + + saslauthd + +:label-tip:`ScyllaDB Enterprise` + +.. versionadded:: 2021.1.2 + +Scylla supports user authentication via an LDAP server by leveraging the SaslauthdAuthenticator. +By configuring saslauthd correctly against your LDAP server, you enable Scylla to check the user’s credentials through it. + + +Configure saslauthd for LDAP +---------------------------- +**Before You Begin** + +This procedure requires you to install and configure saslauthd. +The general instructions are :doc:`here `. + +#. Follow all of the steps in :doc:`this procedure ` and use the code snippets below to list LDAP as the authentication mechanism. + +#. You must list LDAP as saslauthd’s authentication mechanism: + + .. tabs:: + + .. group-tab:: rpm-based distros + + Edit ``/etc/sysconfig/saslauthd`` and add: + + .. code-block:: none + + MECH=ldap + + .. group-tab:: deb-based distros + + Edit ``/etc/default/saslauthd`` and add: + + .. code-block:: none + + MECHANISMS=ldap + + +#. You also have to edit the /etc/saslauthd.conf file to provide adequate `parameter `_ values for your LDAP server. + + + + diff --git a/docs/operating-scylla/security/ldap-authorization.rst b/docs/operating-scylla/security/ldap-authorization.rst new file mode 100644 index 000000000000..f92df3c8d97f --- /dev/null +++ b/docs/operating-scylla/security/ldap-authorization.rst @@ -0,0 +1,153 @@ +===================================== +LDAP Authorization (Role Management) +===================================== + +:label-tip:`ScyllaDB Enterprise` + +.. versionadded:: 2021.1.2 + +Scylla Enterprise customers can manage and authorize users’ privileges via an :abbr:`LDAP (Lightweight Directory Access Protocol)` server. +LDAP is an open, vendor-neutral, industry-standard protocol for accessing and maintaining distributed user access control over a standard IP network. +If your users are already stored in an LDAP directory, you can now use the same LDAP server to regulate their roles in Scylla. + + +Introduction +------------ + +Scylla can use LDAP to manage which roles a user has. This behavior is triggered by setting the ``role_manager`` entry in scylla.yaml to *com.scylladb.auth.LDAPRoleManager*. +When this role manager is chosen, Scylla forbids ``GRANT`` and ``REVOKE`` role statements (CQL commands) as all users get their roles from the contents in the LDAP directory. + +.. _note: + +.. note:: Scylla still allows ``GRANT`` and ``REVOKE`` permission statements, such as ``GRANT permission ON resource TO role``, which are handled by the authorizer, not role manager. + This allows permissions to be granted to and revoked from LDAP-managed roles. In addition, if you have nested Scylla roles, LDAP authorization does not allow them. A role cannot be a member of another role. + In LDAP only login users can be members of a role. + +When LDAP Authorization is enabled and a Scylla user authenticates to Scylla, a query is sent to the LDAP server, whose response sets the user's roles for that login session. +The user keeps the granted roles until logout; any subsequent changes to the LDAP directory are only effective at the user's next login to Scylla. + +The precise form of the LDAP query is configured by Scylla administrator in the scylla.yaml configuration file. +This configuration takes the form of a query template which is defined in the scylla.yaml configuration file using the parameter ``ldap_url_template``. +The value of ``ldap_url_template`` parameter should contain a valid LDAP URL (e.g., as returned by the ldapurl utility from OpenLDAP) representing an LDAP query that returns entries for all the user's roles. +Scylla will replace the text ``{USER}`` in the URL with the user's Scylla username before querying LDAP. + +Workflow +-------- + +**Before you begin** +On your LDAP server, create LDAP directory entries for Scylla users and roles. + +**Workflow** + +#. :ref:`Create a Query Template ` +#. Ensure Scylla has the same users and roles as listed in the LDAP directory. +#. :ref:`Enable LDAP as the role manager in Scylla ` +#. Make Scylla reload the configuration (SIGHUP or restart) + +.. _example-template: + +Example: Query Template +======================= + +Use this example to create a query that will retrieve from your LDAP server the information you need to create a template. +For example, this template URL will query LDAP server at ``localhost:5000`` for all entries under ``base_dn`` that list the user's username as one of their ``uniqueMember`` attribute values: + +.. code-block:: none + + ldap://localhost:5000/base_dn?cn?sub?(uniqueMember={USER}) + +After Scylla queries LDAP and obtains the resulting entries, it looks for a particular attribute in each entry and uses that attribute's value as a Scylla role this user will have. +The name of this attribute can be configured in scylla.yaml by setting the ``ldap_attr_role`` parameter there. + +When the LDAP query returns multiple entries, multiple roles will be granted to the user. +Each role must already exist in Scylla, created via the :ref:`CREATE ROLE ` CQL command beforehand. + +For example, if the LDAP query returns the following results: + +.. code-block:: none + + # extended LDIF + # + # LDAPv3 + + # role1, example.com + dn: cn=role1,dc=example,dc=com + objectClass: groupOfUniqueNames + cn: role1 + scyllaName: sn1 + uniqueMember: uid=jsmith,ou=People,dc=example,dc=com + uniqueMember: uid=cassandra,ou=People,dc=example,dc=com + + # role2, example.com + dn: cn=role2,dc=example,dc=com + objectClass: groupOfUniqueNames + cn: role2 + scyllaName: sn2 + uniqueMember: uid=cassandra,ou=People,dc=example,dc=com + + # role3, example.com + dn: cn=role3,dc=example,dc=com + objectClass: groupOfUniqueNames + cn: role3 + uniqueMember: uid=jdoe,ou=People,dc=example,dc=com + +If ``ldap_attr_role`` is set to *cn*, then the resulting role set will be { role1, role2, role3 } (assuming, of course, that these roles already exist in Scylla). +However, if ``ldap_attr_role`` is set to *scyllaName*, then the resulting role set will be { sn1, sn2 }. +If an LDAP entry does not have the ``ldap_attr_role`` attribute, it is simply ignored. +Before Scylla attempts to query the LDAP server, it first performs an LDAP bind operation, to gain access to the directory information. +Scylla executes a simple bind with credentials configured in scylla.yaml. +The parameters ``ldap_bind_dn`` and ``ldap_bind_passwd`` must contain, respectively, the distinguished name and password that Scylla uses to perform the simple bind. + +.. _role-ldap: + +Enable LDAP Authorization +------------------------- + +Enables Scylla to use LDAP Authorization. LDAP will manage the roles, not Scylla. See :ref:`Note ` above + +#. Open the scylla.yaml file in an editor. The file is located in /etc/scylla/scylla.yaml by default. +#. Edit the ``role_manager`` section. Change the entry to ``com.scylladb.auth.LDAPRoleManager``. If this section does not exist, add it to the file. + Configure the parameters according to your organization's IT and Security Policy. + + .. code-block:: yaml + + role_manager: "com.scylladb.auth.LDAPRoleManager" + ldap_url_template: "ldap://localhost:123/dc=example,dc=com?cn?sub?(uniqueMember=uid={USER},ou=People,dc=example,dc=com)" + ldap_attr_role: "cn" + ldap_bind_dn: "cn=root,dc=example,dc=com" + ldap_bind_passwd: "secret" + +#. Restart the scylla-server service or kill the scylla process. + + .. include:: /rst_include/scylla-commands-restart-index.rst + +Disable LDAP Authorization +-------------------------- + +#. Open the scylla.yaml file in an editor. The file is located in /etc/scylla/scylla.yaml by default. +#. Comment out or delete the role_manager section. +#. Restart the scylla-server service or kill the scylla process. + + .. include:: /rst_include/scylla-commands-restart-index.rst + + +Troubleshooting +--------------- + +Before configuring Scylla, it is a good idea to validate the query template by manually ensuring that the LDAP server returns the correct entries when queried. +This can be accomplished by using an LDAP search tool such as `ldapsearch `_. + +If manual querying does not yield correct results, then Scylla cannot see correct results, either. +Try to adjust ldapsearch parameters until it returns the correct role entries for **one** user. + +Once that works as expected, you can use the `ldapurl `_ utility to transform the parameters into a URL providing a basis for the ldap_url_template. + +.. tip:: Always provide an explicit ``-s`` flag to both ``ldapsearch`` and ``ldapurl``; the default ``-s`` value differs among the two tools. + +Remember to replace the specific user name with ``{USER}`` in the URL template. +You can turn on debug logging in the LDAP role manager by passing the following argument to the Scylla executable: ``--logger-log-level ldap_role_manager=debug``. +This will make Scylla log useful additional details about the LDAP responses it receives. + +If ldapsearch yields expected results but Scylla queries do not, first check the host and port parts of the URL template and make sure both ldapsearch and +Scylla are actually querying the same LDAP server. +Then check the LDAP logs and see if there are any subtle differences between the logged queries of ldapsearch and Scylla. diff --git a/ent/CMakeLists.txt b/ent/CMakeLists.txt new file mode 100644 index 000000000000..181669ed726a --- /dev/null +++ b/ent/CMakeLists.txt @@ -0,0 +1 @@ +add_subdirectory(ldap) diff --git a/ent/ldap/CMakeLists.txt b/ent/ldap/CMakeLists.txt new file mode 100644 index 000000000000..ce65c3ece97e --- /dev/null +++ b/ent/ldap/CMakeLists.txt @@ -0,0 +1,17 @@ +find_package(OpenLDAP REQUIRED + ldap lber) +add_library(ldap STATIC) +target_sources(ldap + PRIVATE + ldap_connection.cc) +target_include_directories(ldap + PUBLIC + ${CMAKE_SOURCE_DIR}) +target_link_libraries(ldap + PUBLIC + Seastar::seastar + PRIVATE + OpenLDAP::ldap OpenLDAP::lber) + +check_headers(check-headers ldap + GLOB_RECURSE ${CMAKE_CURRENT_SOURCE_DIR}/*.hh) \ No newline at end of file diff --git a/ent/ldap/ldap_connection.cc b/ent/ldap/ldap_connection.cc new file mode 100644 index 000000000000..fb5134d49b8d --- /dev/null +++ b/ent/ldap/ldap_connection.cc @@ -0,0 +1,487 @@ +/* + * Copyright (C) 2019 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#define LDAP_DEPRECATED 1 + +#include "ldap_connection.hh" + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include "seastarx.hh" + +extern "C" { +// Declared in `ldap_pvt.h`, but this header is not usually installed by distributions even though +// it's considered public by upstream. +int ldap_init_fd(int, int, const char *, LDAP**); +} + +namespace { + +logger mylog{"ldap_connection"}; // `log` is taken by math. + +constexpr int failure_code{-1}, success_code{0}; // LDAP return codes. + +/// Helper function for Sockbuf_IO work. +ldap_connection* connection(Sockbuf_IO_Desc* sid) { + return reinterpret_cast(sid->sbiod_pvt); +} + +/// Sockbuf_IO setup function for ldap_connection. +int ssbi_setup(Sockbuf_IO_Desc* sid, void* arg) { + sid->sbiod_pvt = arg; // arg is ldap_connection, already set up. + return success_code; +} + +/// Sockbuf_IO remove function for ldap_connection. +int ssbi_remove(Sockbuf_IO_Desc* sid) { + return success_code; // ldap_connection will be destructed by its owner. +} + +void throw_if_failed(int status, const char* op, const ldap_connection& conn, int success = LDAP_SUCCESS) { + if (status != success) { + throw std::runtime_error(fmt::format("{} returned {}: {}", op, status, conn.get_error())); + } +} + +} // anonymous namespace + +std::mutex ldap_connection::_global_init_mutex; + +void ldap_connection::ldap_deleter::operator()(LDAP* ld) { + mylog.trace("ldap_deleter: invoking unbind"); + int status = ldap_unbind(ld); + if (status != LDAP_SUCCESS) { + mylog.error("ldap_unbind failed with status {}", status); + } + mylog.trace("ldap_deleter done"); +} + +ldap_connection::ldap_connection(seastar::connected_socket&& socket) : + _fd(file_desc::eventfd(0, EFD_NONBLOCK)) // Never ready for read, always ready for write. + , _socket(std::move(socket)) + , _input_stream(_socket.input()) + , _output_stream(_socket.output()) + , _status(status::up) + , _read_consumer(now()) + , _read_in_progress(false) + , _outstanding_write(now()) + , _currently_polling(false) { + // Proactively initiate Seastar read, before ldap_connection::read() is first called. + read_ahead(); + + // Libldap determines if we're ready for an sbi_write by polling _fd. We're always ready to + // accept an sbi_write because we chain writes as continuations. Therefore, _fd always polls + // ready to write. + // + // Libldap determines if we're ready for an sbi_read by first calling sbi_ctrl(DATA_READY); only + // if that returns false does libldap then poll _fd. We are ready to accept an sbi_read when a + // prior Seastar read completed successfully and delivered some data into our _read_buffer. + // Only sbi_ctrl() knows that, which is why _fd always polls NOT READY to read. + // + // NB: libldap never actually reads or writes directly to _fd. It reads and writes through the + // custom Sockbuf_IO we provide it. + static constexpr int LDAP_PROTO_EXT = 4; // From ldap_pvt.h, which isn't always available. + mylog.trace("constructor invoking ldap_init"); + LDAP* init_result; + { + std::lock_guard global_init_lock{_global_init_mutex}; + throw_if_failed(ldap_init_fd(_fd.get(), LDAP_PROTO_EXT, nullptr, &init_result), "ldap_init_fd", *this); + } + _ldap.reset(init_result); + static constexpr int opt_v3 = LDAP_VERSION3; + throw_if_failed( + ldap_set_option(_ldap.get(), LDAP_OPT_PROTOCOL_VERSION, &opt_v3), // Encouraged by ldap_set_option manpage. + "ldap_set_option protocol version", + *this, + LDAP_OPT_SUCCESS); + throw_if_failed( + ldap_set_option(_ldap.get(), LDAP_OPT_RESTART, LDAP_OPT_ON), // Retry on EINTR, rather than return error. + "ldap_set_option restart", + *this, + LDAP_OPT_SUCCESS); + throw_if_failed( + // Chasing referrals with this setup results in libldap crashing. + ldap_set_option(_ldap.get(), LDAP_OPT_REFERRALS, LDAP_OPT_OFF), + "ldap_set_option no referrals", + *this, + LDAP_OPT_SUCCESS); + + Sockbuf* sb; + throw_if_failed(ldap_get_option(_ldap.get(), LDAP_OPT_SOCKBUF, &sb), "ldap_get_option", *this, LDAP_OPT_SUCCESS); + mylog.trace("constructor adding Sockbuf_IO"); + throw_if_failed( + ber_sockbuf_add_io(sb, const_cast(&seastar_sbio), LBER_SBIOD_LEVEL_PROVIDER, this), + "ber_sockbuf_add_io", + *this); + mylog.trace("constructor done"); +} + +future<> ldap_connection::close() { + if (_status == status::down) { + mylog.error("close called while connection is down"); + return make_exception_future<>(std::runtime_error("double close() of ldap_connection")); + } + _ldap.reset(); // Sends one last message to the server before reclaiming memory. + return when_all( + _read_consumer.finally([this] { return _input_stream.close(); }) + .handle_exception([] (std::exception_ptr ep) { + mylog.error("Seastar input stream closing failed: {}", ep); + }), + _outstanding_write.finally([this] { return _output_stream.close(); }) + .handle_exception([] (std::exception_ptr ep) { + mylog.error("Seastar output stream closing failed: {}", ep); + }) + ).discard_result().then([this] { + shutdown(); + return make_ready_future<>(); + }); +} + +future ldap_connection::await_result(int msgid) { + mylog.trace("await_result({})", msgid); + + if (_status != status::up) { + mylog.error("await_result({}) error: connection is not up", msgid); + ldap_abandon_ext(get_ldap(), msgid, /*sctrls=*/nullptr, /*cctrls=*/nullptr); + return make_exception_future(std::runtime_error("ldap_connection status set to error")); + } + + try { + return _msgid_to_promise[msgid].get_future(); + } catch (...) { + auto ex = std::current_exception(); + mylog.error("await_result({}) error: {}", msgid, ex); + // Tell LDAP to abandon this msgid, since we failed to register it. + ldap_abandon_ext(get_ldap(), msgid, /*sctrls=*/nullptr, /*cctrls=*/nullptr); + return make_exception_future(ex); + } +} + +future ldap_connection::await_result(int status, int msgid) { + mylog.trace("await_result({}, {})", status, msgid); + if (status == LDAP_SUCCESS) { + return await_result(msgid); + } else { + const char* err = ldap_err2string(status); + mylog.trace("await_result({}, {}) reporting error {}", status, msgid, err); + return make_exception_future( + std::runtime_error(fmt::format("ldap operation error: {}", err))); + } +} + +future ldap_connection::simple_bind(const char *who, const char *passwd) { + mylog.trace("simple_bind({})", who); + if (_status != status::up) { + mylog.error("simple_bind({}) punting, connection down", who); + return make_exception_future( + std::runtime_error("bind operation attempted on a closed ldap_connection")); + } + const int msgid = ldap_simple_bind(get_ldap(), who, passwd); + if (msgid == -1) { + const auto err = get_error(); + mylog.error("ldap simple bind error: {}", err); + return make_exception_future( + std::runtime_error(fmt::format("ldap simple bind error: {}", err))); + } else { + mylog.trace("simple_bind: msgid {}", msgid); + return await_result(msgid); + } +} + +future ldap_connection::search( + char *base, + int scope, + char *filter, + char *attrs[], + int attrsonly, + LDAPControl **serverctrls, + LDAPControl **clientctrls, + struct timeval *timeout, + int sizelimit) { + mylog.trace("search"); + int msgid; + if (_status != status::up) { + mylog.error("search punting, connection down"); + return make_exception_future( + std::runtime_error("search operation attempted on a closed ldap_connection")); + } + const int status = ldap_search_ext( + get_ldap(), base, scope, filter, attrs, attrsonly, serverctrls, clientctrls, timeout, sizelimit, &msgid); + return await_result(status, msgid); +} + +sstring ldap_connection::get_error() const { + int result_code; + int status = ldap_get_option(get_ldap(), LDAP_OPT_RESULT_CODE, reinterpret_cast(&result_code)); + if (status != LDAP_OPT_SUCCESS) { + mylog.error("ldap_get_option returned {}", status); + return "error description unavailable"; + } + return ldap_err2string(result_code); +} + +int ldap_connection::sbi_ctrl(Sockbuf_IO_Desc* sid, int opt, void* arg) noexcept { + mylog.debug("sbi_ctrl({}/{}, {}, {})", static_cast(sid), sid->sbiod_pvt, opt, arg); + auto conn = connection(sid); + switch (opt) { + case LBER_SB_OPT_DATA_READY: + return !conn->_read_buffer.empty() + || conn->_status != status::up; // Let sbi_read proceed and report status; otherwise, LDAP loops forever. + case LBER_SB_OPT_GET_FD: + if (conn->_status == status::down) { + errno = ENOTCONN; + return -1; + } + *reinterpret_cast(arg) = conn->_fd.get(); + return 1; + } + return 0; +} + +ber_slen_t ldap_connection::sbi_read(Sockbuf_IO_Desc* sid, void* buffer, ber_len_t size) noexcept { + mylog.trace("sbi_read {}/{}", static_cast(sid), static_cast(sid->sbiod_pvt)); + try { + return connection(sid)->read(reinterpret_cast(buffer), size); + } catch (...) { + mylog.error("Unexpected error while reading: {}", std::current_exception()); + return failure_code; + } +} + +ber_slen_t ldap_connection::sbi_write(Sockbuf_IO_Desc* sid, void* buffer, ber_len_t size) noexcept { + mylog.trace("sbi_write {}/{}", static_cast(sid), static_cast(sid->sbiod_pvt)); + try { + return connection(sid)->write(reinterpret_cast(buffer), size); + } catch (...) { + mylog.error("Unexpected error while writing: {}", std::current_exception()); + return failure_code; + } +} + +int ldap_connection::sbi_close(Sockbuf_IO_Desc* sid) noexcept { + mylog.debug("sbi_close {}/{}", static_cast(sid), static_cast(sid->sbiod_pvt)); + // Leave actual closing to the owner of *this. Note sbi_close() will be invoked during + // ldap_unbind(), which also calls sbi_write() to convey one last message to the server. We + // remain open here, to try to communicate that message. + return success_code; +} + +const Sockbuf_IO ldap_connection::seastar_sbio{ + // Strictly speaking, designated initializers like this are not in the standard, but they're + // supported by both major compilers we use. + .sbi_setup = &ssbi_setup, + .sbi_remove = &ssbi_remove, + .sbi_ctrl = &ldap_connection::sbi_ctrl, + .sbi_read = &ldap_connection::sbi_read, + .sbi_write = &ldap_connection::sbi_write, + .sbi_close = &ldap_connection::sbi_close +}; + +void ldap_connection::read_ahead() { + if (_read_in_progress) { // Differs from _read_consumer.available(), because handle_exception adds a continuation. + mylog.warn("read_ahead called while a prior Seastar read is already in progress"); + return; + } + if (_input_stream.eof()) { + mylog.error("read_ahead encountered EOF"); + set_status(status::eof); + return; + } + mylog.trace("read_ahead"); + _read_in_progress = true; + mylog.trace("read_ahead invoking socket read"); + _read_consumer = _input_stream.read().then([this] (temporary_buffer b) { + if (b.empty()) { + mylog.debug("read_ahead received empty buffer; assuming EOF"); + set_status(status::eof); + return; + } + mylog.trace("read_ahead received data of size {}", b.size()); + if (!_read_buffer.empty()) { // Shouldn't happen; read_ahead's purpose is to replenish empty _read_buffer. + mylog.error("read_ahead dropping {} unconsumed bytes", _read_buffer.size()); + } + _read_buffer = std::move(b); + _read_in_progress = false; + poll_results(); + }).handle_exception([this] (std::exception_ptr ep) { + mylog.error("Seastar read failed: {}", ep); + set_status(status::err); + }); + mylog.trace("read_ahead done"); +} + +ber_slen_t ldap_connection::write(char const* b, ber_len_t size) { + mylog.trace("write({})", size); + switch (_status) { + case status::err: + mylog.trace("write({}) reporting error", size); + errno = ECONNRESET; + return -1; + case status::down: + mylog.trace("write({}) invoked after shutdown", size); + errno = ENOTCONN; + return -1; + case status::up: + case status::eof: + ; // Proceed. + } + _outstanding_write = _outstanding_write.then([this, buf = temporary_buffer(b, size)] () mutable { + if (_status != status::up) { + return make_ready_future<>(); + } + mylog.trace("write invoking socket write"); + return _output_stream.write(std::move(buf)).then([this] { + // Sockbuf_IO doesn't seem to have the notion of flushing the stream, so we flush after + // every write. + mylog.trace("write invoking flush"); + return _output_stream.flush(); + }).handle_exception([this] (std::exception_ptr ep) { + mylog.error("Seastar write failed: {}", ep); + set_status(status::err); + }); + }); + mylog.trace("write({}) done, status={}", size, _status); + return _status == status::up ? size : -1; // _status can be err here if _outstanding_write threw. +} + +ber_slen_t ldap_connection::read(char* b, ber_len_t size) { + mylog.trace("read({})", size); + switch (_status) { + case status::eof: + mylog.trace("read({}) reporting eof", size); + return 0; + case status::err: + mylog.trace("read({}) reporting error", size); + errno = ECONNRESET; + return -1; + case status::down: + mylog.trace("read({}) invoked after shutdown", size); + errno = ENOTCONN; + return -1; + case status::up: + ; // Proceed. + } + if (_read_buffer.empty()) { // Can happen because libldap doesn't always wait for data to be ready. + mylog.trace("read({}) found empty read buffer", size); + // Don't invoke read_ahead() here; it was already invoked as soon as _read_buffer was + // drained. In fact, its Seastar read might have actually completed, and the buffer is + // about to be filled by a waiting continuation. We DON'T want another read_ahead() before + // that data is consumed. + errno = EWOULDBLOCK; + return 0; + } + const auto byte_count = std::min(_read_buffer.size(), size); + std::copy_n(_read_buffer.begin(), byte_count, b); + _read_buffer.trim_front(byte_count); + if (_read_buffer.empty()) { + mylog.trace("read({}) replenishing buffer", size); + read_ahead(); + } + mylog.trace("read({}) returning {}", size, byte_count); + return byte_count; +} + +void ldap_connection::shutdown() { + mylog.trace("shutdown"); + set_status(status::down); + mylog.trace("shutdown: shutdown input"); + _socket.shutdown_input(); + mylog.trace("shutdown: shutdown output"); + _socket.shutdown_output(); + mylog.trace("shutdown done"); +} + +void ldap_connection::poll_results() { + mylog.trace("poll_results"); + if (!_ldap) { // Could happen during close(), which unbinds. + mylog.debug("poll_results: _ldap is null, punting"); + return; + } + if (_currently_polling) { + // This happens when ldap_result() calls read_ahead() and runs its inner continuation immediately. + mylog.debug("poll_results: _currently_polling somewhere up the call-stack, punting"); + return; + } + + // Ensure that _currently_polling is true until we return. + class flag_guard { + bool& _flag; + public: + flag_guard(bool& flag) : _flag(flag) { flag = true; } + ~flag_guard() { _flag = false; } + } guard(_currently_polling); + + LDAPMessage *result; + while (!_read_buffer.empty() && _status == status::up) { + static timeval zero_duration{}; + mylog.trace("poll_results: {} in buffer, invoking ldap_result", _read_buffer.size()); + const int status = ldap_result(get_ldap(), LDAP_RES_ANY, /*all=*/1, &zero_duration, &result); + if (status > 0) { + ldap_msg_ptr result_ptr(result); + const int id = ldap_msgid(result); + mylog.trace("poll_results: ldap_result returned status {}, id {}", status, id); + const auto found = _msgid_to_promise.find(id); + if (found == _msgid_to_promise.end()) { + mylog.error("poll_results: got valid result for unregistered id {}, dropping it", id); + ldap_msgfree(result); + } else { + found->second.set_value(std::move(result_ptr)); + _msgid_to_promise.erase(found); + } + } else if (status < 0) { + mylog.error("poll_results: ldap_result returned status {}, error: {}", status, get_error()); + set_status(status::err); + } + } + mylog.trace("poll_results done"); +} + +void ldap_connection::set_status(ldap_connection::status s) { + _status = s; + if (s != status::up) { + mylog.trace("set_status: signal result-waiting futures"); + for (auto& e : _msgid_to_promise) { + e.second.set_exception(std::runtime_error("ldap_connection status set to error")); + } + _msgid_to_promise.clear(); + } +} + +ldap_reuser::ldap_reuser(sequential_producer::factory_t&& f) + : _make_conn(std::move(f)), _reaper(now()) { +} + +void ldap_reuser::reap(conn_ptr& conn) { + if (!conn) { + return; + } + if (auto p = conn.release()) { // Safe to close, other fibers are done with it. + _reaper = _reaper.then([p = std::move(p)] () mutable { + return p->close().then_wrapped([p = std::move(p)] (future<> fut) { + if (fut.failed()) { + mylog.warn("failure closing dead ldap_connection: {}", fut.get_exception()); + } + /* p disposed here */ + }); + }); + } +} + +future<> ldap_reuser::stop() { + reap(_conn); + return std::move(_reaper); +} diff --git a/ent/ldap/ldap_connection.hh b/ent/ldap/ldap_connection.hh new file mode 100644 index 000000000000..50b2e17e0cb0 --- /dev/null +++ b/ent/ldap/ldap_connection.hh @@ -0,0 +1,212 @@ +/* + * Copyright (C) 2019 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + + +#pragma once + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "utils/sequential_producer.hh" + +/// Functor to invoke ldap_msgfree. +struct ldap_msg_deleter { + void operator()(LDAPMessage* p) { + ldap_msgfree(p); + } +}; + +using ldap_msg_ptr = std::unique_ptr; + +/// A connection to an LDAP server with custom networking over a Seastar socket. Constructor takes +/// a connected socket and generates an LDAP structure hooked up to it. The LDAP object is obtained +/// using get_ldap(); its custom networking is valid as long as its ldap_connection host is alive. +class ldap_connection { + seastar::file_desc _fd; ///< Libldap polls this to determine if we're ready for reading/writing. + seastar::connected_socket _socket; + seastar::input_stream _input_stream; ///< _socket's input. + seastar::output_stream _output_stream; ///< _socket's output. + seastar::temporary_buffer _read_buffer; ///< Everything read by Seastar but not yet consumed by Sockbuf_IO. + static std::mutex _global_init_mutex; ///< Mutex for protecting global ldap data initialization for older ldap libs - + /// ref:https://github.com/openldap/openldap/commit/877faea723ecc5721291b0b2f53e34f7921e0f7c + enum class status { + // Lowercase, to avoid inadvertently invoking macros: + up, ///< Connected, operating normally. + down, ///< Shut down. + eof, ///< Read encountered EOF, write should be OK. + err ///< IO error encountered. + }; +#if FMT_VERSION >= 9'00'00 + friend auto format_as(status s) { return fmt::underlying(s); } +#endif + status _status; ///< When not OK, all \c read() and \c write() calls will immediately return without action. + seastar::future<> _read_consumer; ///< Consumes Seastar read data. + bool _read_in_progress; ///< Is there a Seastar read in progress? + seastar::future<> _outstanding_write; ///< Captures Seastar write continuation. + /// When LDAP yields a result for one of these msgids, forward it to the corresponding promise: + std::unordered_map> _msgid_to_promise; + bool _currently_polling; ///< True iff poll_results() is in progress. + + /// Deallocates an LDAP structure. + struct ldap_deleter { + void operator()(LDAP*); + }; + std::unique_ptr _ldap; + + public: + /// Creates LDAP with custom Seastar networking. + ldap_connection(seastar::connected_socket&& socket); + + /// A pointer to the LDAP customized with Seastar IO. *this keeps ownership. + /// + /// \warning Do not call ldap_result() on this object, nor any ldap_* operations that generate + /// network traffic. This includes at least: + /// - ldap_abandon + /// - ldap_add + /// - ldap_bind + /// - ldap_compare + /// - ldap_delete + /// - ldap_extended_operation + /// - ldap_modify + /// - ldap_rename + /// - ldap_search + /// - ldap_unbind() + LDAP* get_ldap() const { return _ldap.get(); } + + /// Before destroying *this, user must wait on the future returned: + seastar::future<> close(); + + /// Performs LDAP simple bind operation. See man ldap_bind. + seastar::future simple_bind(const char *who, const char *passwd); + + /// Performs LDAP search operation. See man ldap_search. + seastar::future search( + char *base, + int scope, + char *filter, + char *attrs[], + int attrsonly, + LDAPControl **serverctrls, + LDAPControl **clientctrls, + struct timeval *timeout, + int sizelimit); + + /// The last error reported by an LDAP operation. + seastar::sstring get_error() const; + + /// Cannot be moved, since it spawns continuations that capture \c this. + ldap_connection(ldap_connection&&) = delete; + + bool is_live() const { return _status == status::up; } + + private: + // Sockbuf_IO functionality (see Sockbuf_IO manpage): + static int sbi_ctrl(Sockbuf_IO_Desc* sid, int option, void* value) noexcept; + static ber_slen_t sbi_read(Sockbuf_IO_Desc* sid, void* buffer, ber_len_t size) noexcept; + static ber_slen_t sbi_write(Sockbuf_IO_Desc* sid, void* buffer, ber_len_t size) noexcept; + static int sbi_close(Sockbuf_IO_Desc* sid) noexcept; + static const Sockbuf_IO seastar_sbio; + + /// Efficiently waits for ldap_result of msgid, returning the result object in the future. If + /// an error occurs at any point, the future will be exceptional. + /// + /// \warning You must call await_result() immediately after obtaining msgid from an ldap_* + /// function, without yielding to Seastar in between. Otherwise, the result may be dropped. + seastar::future await_result(int msgid); + + /// If status is LDAP_SUCCESS, returns await_result(msgid). Otherwise, returns an exceptional + /// future with the error report. + seastar::future await_result(int status, int msgid); + + /// Schedules a Seastar write of (copied) b[:size]. On success, returns size. On failure, + /// returns LDAP's failure code. + ber_slen_t write(char const* b, ber_len_t size); + + /// Consumes at most \p size bytes from what Seastar has read so far and writes them to b. On + /// success, returns the number of bytes so consumed (possibly less than \p size). On failure, + /// returns LDAP's failure code. + ber_slen_t read(char* b, ber_len_t size); + + /// Shuts down all internal state that can be shut down immediately. See also close(). + void shutdown(); + + /// Initiates a Seastar read that will procure data for \c read() to consume. Data consumption + /// will happen in a future captured by _read_consumer. If _read_consumer is currently active, + /// however (because the previous Seastar read hasn't been consumed yet), this method does + /// nothing. + void read_ahead(); + + /// Invokes ldap_result for all elements of _msgid_to_promise. For every ready result, fulfills + /// its promise and removes it from _msgid_to_promise. + void poll_results(); + + /// Sets _status to \p s. If s != status::up, sends an exception to each _msgid_to_promise + /// element, then clears _msgid_to_promise. + void set_status(ldap_connection::status s); +}; + +/// Reuses an ldap_connection as long as it is live, then transparently creates a new one, ad infinitum. Cleans up +/// all the created connections. +class ldap_reuser { + public: + using conn_ptr = seastar::lw_shared_ptr; + + private: + sequential_producer _make_conn; // TODO: This type can be a parameter. + conn_ptr _conn; + seastar::future<> _reaper; ///< Closes and deletes all connections produced. + + public: + ldap_reuser(sequential_producer::factory_t&& f); + ldap_reuser(ldap_reuser&&) = delete; // Spawns continuations that capture *this; don't move it, pls. + ldap_reuser& operator=(ldap_reuser&&) = delete; // Spawns continuations that capture *this; don't move it, pls. + + /// Must resolve before destruction. + seastar::future<> stop(); + + /// Invokes fn on a valid ldap_connection, managing its lifetime and validity. + template Func> + std::invoke_result_t with_connection(Func fn) { + if (_conn && _conn->is_live()) { + return invoke(std::move(fn)); + } else { + return _make_conn().then([this, fn = std::move(fn)] (conn_ptr&& conn) mutable { + if (_conn) { + if (!_conn->is_live()) { + reap(_conn); + } else { + reap(conn); // apparently lost the race + } + } + if (!_conn) { + _conn = std::move(conn); + } + _make_conn.clear(); // So _make_conn doesn't keep a shared-pointer copy that escapes reaping. + return invoke(std::move(fn)); + }); + } + } + + private: + /// Invokes fn on a copy of _conn and schedules its reaping. + template Func> + std::invoke_result_t invoke(Func fn) { + conn_ptr conn(_conn); + return fn(*conn).finally([this, conn = std::move(conn)] () mutable { reap(conn); }); + } + + /// Decreases conn reference count. If this is the last fiber using conn, closes and disposes of it. + void reap(conn_ptr& conn); +}; diff --git a/test.py b/test.py index f104d0d41b9e..18f9c64139b4 100755 --- a/test.py +++ b/test.py @@ -23,12 +23,15 @@ import shlex import shutil import signal +import socket import subprocess import sys import time import traceback import xml.etree.ElementTree as ET import yaml +from random import randint +from tempfile import TemporaryDirectory from abc import ABC, abstractmethod from io import StringIO @@ -59,6 +62,73 @@ 'coverage': 'Coverage'} debug_modes = {'debug', 'sanitize'} +LDAP_SERVER_CONFIGURATION_FILE = os.path.join(os.path.dirname(__file__), 'test', 'resource', 'slapd.conf') + +DEFAULT_ENTRIES = [ + """dn: dc=example,dc=com +objectClass: dcObject +objectClass: organization +dc: example +o: Example +description: Example directory. +""", + """dn: cn=root,dc=example,dc=com +objectClass: organizationalRole +cn: root +description: Directory manager. +""", + """dn: ou=People,dc=example,dc=com +objectClass: organizationalUnit +ou: People +description: Our people. +""", + """# Default superuser for Scylla +dn: uid=cassandra,ou=People,dc=example,dc=com +objectClass: organizationalPerson +objectClass: uidObject +cn: cassandra +ou: People +sn: cassandra +userid: cassandra +userPassword: cassandra +""", + """dn: uid=jsmith,ou=People,dc=example,dc=com +objectClass: organizationalPerson +objectClass: uidObject +cn: Joe Smith +ou: People +sn: Smith +userid: jsmith +userPassword: joeisgreat +""", + """dn: uid=jdoe,ou=People,dc=example,dc=com +objectClass: organizationalPerson +objectClass: uidObject +cn: John Doe +ou: People +sn: Doe +userid: jdoe +userPassword: pa55w0rd +""", + """dn: cn=role1,dc=example,dc=com +objectClass: groupOfUniqueNames +cn: role1 +uniqueMember: uid=jsmith,ou=People,dc=example,dc=com +uniqueMember: uid=cassandra,ou=People,dc=example,dc=com +""", + """dn: cn=role2,dc=example,dc=com +objectClass: groupOfUniqueNames +cn: role2 +uniqueMember: uid=cassandra,ou=People,dc=example,dc=com +""", + """dn: cn=role3,dc=example,dc=com +objectClass: groupOfUniqueNames +cn: role3 +uniqueMember: uid=jdoe,ou=People,dc=example,dc=com +""", +] + + def create_formatter(*decorators) -> Callable[[Any], str]: """Return a function which decorates its argument with the given color/style if stdout is a tty, and leaves intact otherwise.""" @@ -476,6 +546,19 @@ def junit_tests(self) -> Iterable['Test']: def boost_tests(self) -> Iterable['Tests']: return self.tests + +class LdapTestSuite(UnitTestSuite): + """TestSuite for ldap unit tests""" + + async def create_test(self, shortname, casename, suite, args): + test = LdapTest(self.next_id((shortname, self.suite_key)), shortname, suite, args) + self.tests.append(test) + + def junit_tests(self): + """Ldap tests produce an own XML output, so are not included in a junit report""" + return [] + + class PythonTestSuite(TestSuite): """A collection of Python pytests against a single Scylla instance""" @@ -739,6 +822,16 @@ async def run(self, options: argparse.Namespace) -> 'Test': def print_summary(self) -> None: pass + async def setup(self, port, options): + """ + Performs any necessary setup steps before running a test. + Returns (fn, txt, test_env) where: + fn - is a cleanup function to call unconditionally after the test stops running + txt - is failure-injection description. + test_env - is a dictionary containing environment variables map specific for the test + """ + return (lambda: 0, None,{}) + def check_log(self, trim: bool) -> None: """Check and trim logs and xml output for tests which have it""" if trim: @@ -887,6 +980,113 @@ def print_summary(self) -> None: print(read_log(self.log_filename)) +def can_connect(address, family=socket.AF_INET): + s = socket.socket(family) + try: + s.connect(address) + return True + except OSError as e: + if 'AF_UNIX path too long' in str(e): + raise OSError(e.errno, "{} ({})".format(str(e), address)) from None + else: + return False + except: + return False + + +def try_something_backoff(something): + sleep_time = 0.05 + while not something(): + if sleep_time > 30: + return False + time.sleep(sleep_time) + sleep_time *= 2 + return True + +def make_saslauthd_conf(port, instance_path): + """Creates saslauthd.conf with appropriate contents under instance_path. Returns the path to the new file.""" + saslauthd_conf_path = os.path.join(instance_path, 'saslauthd.conf') + with open(saslauthd_conf_path, 'w') as f: + f.write('ldap_servers: ldap://localhost:{}\nldap_search_base: dc=example,dc=com'.format(port)) + return saslauthd_conf_path + + +class LdapTest(BoostTest): + """A unit test which can produce its own XML output, and needs an ldap server""" + + def __init__(self, test_no, shortname, args, suite): + super().__init__(test_no, shortname, args, suite, None, False, None) + + async def setup(self, port, options): + instances_root = os.path.join(options.tmpdir, self.mode, 'ldap_instances'); + instance_path = os.path.join(os.path.abspath(instances_root), str(port)) + slapd_pid_file = os.path.join(instance_path, 'slapd.pid') + saslauthd_socket_path = TemporaryDirectory() + os.makedirs(instance_path, exist_ok=True) + # This will always fail because it lacks the permissions to read the default slapd data + # folder but it does create the instance folder so we don't want to fail here. + try: + subprocess.check_output(['slaptest', '-f', LDAP_SERVER_CONFIGURATION_FILE, '-F', instance_path], + stderr=subprocess.DEVNULL) + except: + pass + # Set up failure injection. + proxy_name = 'p{}'.format(port) + subprocess.check_output([ + 'toxiproxy-cli', 'c', proxy_name, + '--listen', 'localhost:{}'.format(port + 2), '--upstream', 'localhost:{}'.format(port)]) + # Sever the connection after byte_limit bytes have passed through: + byte_limit = options.byte_limit if options.byte_limit else randint(0, 2000) + subprocess.check_output(['toxiproxy-cli', 't', 'a', proxy_name, '-t', 'limit_data', '-n', 'limiter', + '-a', 'bytes={}'.format(byte_limit)]) + # Change the data folder in the default config. + replace_expression = 's/olcDbDirectory:.*/olcDbDirectory: {}/g'.format( + os.path.abspath(instance_path).replace('/', r'\/')) + subprocess.check_output( + ['find', instance_path, '-type', 'f', '-exec', 'sed', '-i', replace_expression, '{}', ';']) + # Change the pid file to be kept with the instance. + replace_expression = 's/olcPidFile:.*/olcPidFile: {}/g'.format( + os.path.abspath(slapd_pid_file).replace('/', r'\/')) + subprocess.check_output( + ['find', instance_path, '-type', 'f', '-exec', 'sed', '-i', replace_expression, '{}', ';']) + # Put the test data in. + cmd = ['slapadd', '-F', instance_path] + subprocess.check_output( + cmd, input='\n\n'.join(DEFAULT_ENTRIES).encode('ascii'), stderr=subprocess.STDOUT) + # Set up the server. + SLAPD_URLS='ldap://:{}/ ldaps://:{}/'.format(port, port + 1) + def can_connect_to_slapd(): + return can_connect(('127.0.0.1', port)) and can_connect(('127.0.0.1', port + 1)) and can_connect(('127.0.0.1', port + 2)) + def can_connect_to_saslauthd(): + return can_connect(os.path.join(saslauthd_socket_path.name, 'mux'), socket.AF_UNIX) + slapd_proc = subprocess.Popen(['prlimit', '-n1024', 'slapd', '-F', instance_path, '-h', SLAPD_URLS, '-d', '0']) + saslauthd_conf_path = make_saslauthd_conf(port, instance_path) + test_env = { + "SEASTAR_LDAP_PORT" : str(port), + "SASLAUTHD_MUX_PATH" : os.path.join(saslauthd_socket_path.name, "mux") + } + + saslauthd_proc = subprocess.Popen( + ['saslauthd', '-d', '-n', '1', '-a', 'ldap', '-O', saslauthd_conf_path, '-m', saslauthd_socket_path.name], + stdout=subprocess.DEVNULL, stderr=subprocess.STDOUT) + def finalize(): + slapd_proc.terminate() + slapd_proc.wait() # Wait for slapd to remove slapd.pid, so it doesn't race with rmtree below. + saslauthd_proc.kill() # Somehow, invoking terminate() here also terminates toxiproxy-server. o_O + shutil.rmtree(instance_path) + saslauthd_socket_path.cleanup() + subprocess.check_output(['toxiproxy-cli', 'd', proxy_name]) + try: + if not try_something_backoff(can_connect_to_slapd): + raise Exception('Unable to connect to slapd') + if not try_something_backoff(can_connect_to_saslauthd): + raise Exception('Unable to connect to saslauthd') + except: + finalize() + raise + return finalize, '--byte-limit={}'.format(byte_limit), test_env + + class CQLApprovalTest(Test): """Run a sequence of CQL commands against a standalone Scylla""" @@ -1308,17 +1508,31 @@ def print_progress(self, test: Test) -> None: msg += " {:.2f}s".format(test.time_end - test.time_start) print(msg) +toxiproxy_id_gen = 0 async def run_test(test: Test, options: argparse.Namespace, gentle_kill=False, env=dict()) -> bool: """Run test program, return True if success else False""" with test.log_filename.open("wb") as log: - - def report_error(error): + global toxiproxy_id_gen + toxiproxy_id = toxiproxy_id_gen + toxiproxy_id_gen += 1 + ldap_port = 5000 + (toxiproxy_id * 3) % 55000 + cleanup_fn = None + finject_desc = None + def report_error(error, failure_injection_desc = None): msg = "=== TEST.PY SUMMARY START ===\n" msg += "{}\n".format(error) msg += "=== TEST.PY SUMMARY END ===\n" + if failure_injection_desc is not None: + msg += 'failure injection: {}'.format(failure_injection_desc) log.write(msg.encode(encoding="UTF-8")) + + try: + cleanup_fn, finject_desc, test_env = await test.setup(ldap_port, options) + except Exception as e: + report_error("Test setup failed ({})\n{}".format(str(e), traceback.format_exc())) + return False process = None stdout = None logging.info("Starting test %s: %s %s", test.uname, test.path, " ".join(test.args)) @@ -1334,6 +1548,19 @@ def report_error(error): "detect_stack_use_after_return=1", os.getenv("ASAN_OPTIONS"), ] + ldap_instance_path = os.path.join( + os.path.abspath(os.path.join(options.tmpdir, test.mode, 'ldap_instances')), + str(ldap_port)) + saslauthd_mux_path = os.path.join(ldap_instance_path, 'mux') + if options.manual_execution: + print('Please run the following shell command, then press :') + test_env_string = " ".join([f"{k}={v}" for k,v in test_env.items()]) + print('{} {}'.format( + test_env_string, ' '.join([shlex.quote(e) for e in [test.path, *test.args]]))) + input('-- press to continue --') + if cleanup_fn is not None: + cleanup_fn() + return True try: resource_gather = get_resource_gather(options.gather_metrics, test, options.tmpdir) resource_gather.make_cgroup() @@ -1342,6 +1569,8 @@ def report_error(error): ":".join(filter(None, UBSAN_OPTIONS))).encode(encoding="UTF-8")) log.write("export ASAN_OPTIONS='{}'\n".format( ":".join(filter(None, ASAN_OPTIONS))).encode(encoding="UTF-8")) + for k,v in test_env.items(): + log.write(f"export {k}={v}\n".encode(encoding="UTF-8")) log.write("{} {}\n".format(test.path, " ".join(test.args)).encode(encoding="UTF-8")) log.write("=== TEST.PY TEST {} OUTPUT ===\n".format(test.uname).encode(encoding="UTF-8")) log.flush() @@ -1357,11 +1586,8 @@ def report_error(error): test_running_event = asyncio.Event() test_resource_watcher = resource_gather.cgroup_monitor(test_event=test_running_event) - process = await asyncio.create_subprocess_exec( - path, *args, - stderr=log, - stdout=log, - env=dict(os.environ, + test_env.update( + dict(os.environ, UBSAN_OPTIONS=":".join(filter(None, UBSAN_OPTIONS)), ASAN_OPTIONS=":".join(filter(None, ASAN_OPTIONS)), # TMPDIR env variable is used by any seastar/scylla @@ -1369,7 +1595,13 @@ def report_error(error): TMPDIR=os.path.join(options.tmpdir, test.mode), SCYLLA_TEST_ENV='yes', **env, - ), + ) + ) + process = await asyncio.create_subprocess_exec( + path, *args, + stderr=log, + stdout=log, + env=test_env, preexec_fn=resource_gather.put_process_to_cgroup, ) stdout, _ = await asyncio.wait_for(process.communicate(), options.timeout) @@ -1413,6 +1645,9 @@ def report_error(error): report_error("Test was cancelled: the parent process is exiting") except Exception as e: report_error("Failed to run the test:\n{e}".format(e=e)) + finally: + if cleanup_fn is not None: + cleanup_fn() return False @@ -1511,6 +1746,10 @@ def parse_cmd_line() -> argparse.Namespace: parser.add_argument("--cluster-pool-size", action="store", default=None, type=int, help="Set the pool_size for PythonTest and its descendants. Alternatively environment variable " "CLUSTER_POOL_SIZE can be used to achieve the same") + parser.add_argument('--manual-execution', action='store_true', default=False, + help='Let me manually run the test executable at the moment this script would run it') + parser.add_argument('--byte-limit', action="store", default=None, type=int, + help="Specific byte limit for failure injection (random by default)") scylla_additional_options = parser.add_argument_group('Additional options for Scylla tests') scylla_additional_options.add_argument('--x-log2-compaction-groups', action="store", default="0", type=int, help="Controls number of compaction groups to be used by Scylla tests. Value of 3 implies 8 groups.") @@ -1958,20 +2197,39 @@ async def main() -> int: for t in TestSuite.all_tests()])) return 0 + if options.manual_execution and TestSuite.test_count() > 1: + print('--manual-execution only supports running a single test, but multiple selected: {}'.format( + [t.path for t in TestSuite.tests()][:3])) # Print whole t.path; same shortname may be in different dirs. + return 1 + signaled = asyncio.Event() stop_event = asyncio.Event() resource_watcher = run_resource_watcher(options.gather_metrics, signaled, stop_event, options.tmpdir) setup_signal_handlers(asyncio.get_running_loop(), signaled) + tp_server = None try: - await run_all_tests(signaled, options) - stop_event.set() - async with asyncio.timeout(5): - await resource_watcher - except Exception as e: - print(palette.fail(e)) - raise + if [t for t in TestSuite.all_tests() if isinstance(t, LdapTest)]: + tp_server = subprocess.Popen('toxiproxy-server', stderr=subprocess.DEVNULL) + def can_connect_to_toxiproxy(): + return can_connect(('127.0.0.1', 8474)) + if not try_something_backoff(can_connect_to_toxiproxy): + raise Exception('Could not connect to toxiproxy') + + try: + logging.info('running all tests') + await run_all_tests(signaled, options) + logging.info('after running all tests') + stop_event.set() + async with asyncio.timeout(5): + await resource_watcher + except Exception as e: + print(palette.fail(e)) + raise + finally: + if tp_server is not None: + tp_server.terminate() if signaled.is_set(): return -signaled.signo # type: ignore diff --git a/test/CMakeLists.txt b/test/CMakeLists.txt index 10ffdec4b3b5..54fd244dbd30 100644 --- a/test/CMakeLists.txt +++ b/test/CMakeLists.txt @@ -97,6 +97,7 @@ if(BUILD_TESTING) add_dependencies(tests scylla) add_subdirectory(boost) + add_subdirectory(ldap) add_subdirectory(manual) add_subdirectory(unit) add_subdirectory(raft) diff --git a/test/ldap/CMakeLists.txt b/test/ldap/CMakeLists.txt new file mode 100644 index 000000000000..bbf43695c32d --- /dev/null +++ b/test/ldap/CMakeLists.txt @@ -0,0 +1,11 @@ +add_scylla_test(ldap_connection_test + KIND SEASTAR) +add_scylla_test(ldap_role_manager_test + KIND SEASTAR + SOURCES role_manager_test.cc) +add_scylla_test(saslauthd_authenticator_test + KIND SEASTAR) + +if(EXISTS "${CMAKE_CURRENT_SOURCE_DIR}/suite.yaml") + set(scylla_tests "${scylla_tests}" PARENT_SCOPE) +endif() \ No newline at end of file diff --git a/test/ldap/ldap_common.hh b/test/ldap/ldap_common.hh new file mode 100644 index 000000000000..bca337d8e480 --- /dev/null +++ b/test/ldap/ldap_common.hh @@ -0,0 +1,25 @@ + +/* + * Copyright (C) 2015 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include + +// Common values used in multiple LDAP tests. +namespace { + +constexpr auto base_dn = "dc=example,dc=com"; +constexpr auto manager_dn = "cn=root,dc=example,dc=com"; +constexpr auto manager_password = "secret"; +const auto ldap_envport = std::getenv("SEASTAR_LDAP_PORT"); +const std::string ldap_port(ldap_envport ? ldap_envport : "389"); +const seastar::socket_address local_ldap_address(seastar::ipv4_addr("127.0.0.1", std::stoi(ldap_port))); +const seastar::socket_address local_fail_inject_address(seastar::ipv4_addr("127.0.0.1", std::stoi(ldap_port) + 2)); + +} // anonymous namespace diff --git a/test/ldap/ldap_connection_test.cc b/test/ldap/ldap_connection_test.cc new file mode 100644 index 000000000000..b8b2d502b123 --- /dev/null +++ b/test/ldap/ldap_connection_test.cc @@ -0,0 +1,293 @@ +/* + * Copyright (C) 2019 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#define LDAP_DEPRECATED 1 + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "ent/ldap/ldap_connection.hh" +#include "test/lib/exception_utils.hh" +#include "ldap_common.hh" +#include "seastarx.hh" + +extern "C" { +#include +} + +namespace { + +logger mylog{"ldap_connection_test"}; // `log` is taken by math. + +void set_defbase() { + ldap_set_option(nullptr, LDAP_OPT_DEFBASE, base_dn); +} + +const std::set results_expected_from_search_base_dn{ + "dc=example,dc=com", + "cn=root,dc=example,dc=com", + "ou=People,dc=example,dc=com", + "uid=cassandra,ou=People,dc=example,dc=com", + "uid=jsmith,ou=People,dc=example,dc=com", + "uid=jdoe,ou=People,dc=example,dc=com", + "cn=role1,dc=example,dc=com", + "cn=role2,dc=example,dc=com", + "cn=role3,dc=example,dc=com", +}; + +/// Ignores exceptions from LDAP operations. Useful for failure-injection tests, which must +/// tolerate aborted communication. +ldap_msg_ptr ignore(std::exception_ptr) { + return ldap_msg_ptr(); +} + +/// Extracts entries from an ldap_search result. +std::set entries(LDAP* ld, LDAPMessage* res) { + BOOST_REQUIRE_EQUAL(LDAP_RES_SEARCH_ENTRY, ldap_msgtype(res)); + std::set entry_set; + for (auto e = ldap_first_entry(ld, res); e; e = ldap_next_entry(ld, e)) { + char* dn = ldap_get_dn(ld, e); + entry_set.insert(dn); + ldap_memfree(dn); + } + return entry_set; +} + +future search(ldap_connection& conn, const char* term) { + return conn.search( + const_cast(term), + LDAP_SCOPE_SUBTREE, + /*filter=*/nullptr, + /*attrs=*/nullptr, + /*attrsonly=*/0, + /*serverctrls=*/nullptr, + /*clientctrls=*/nullptr, + /*timeout=*/nullptr, + /*sizelimit=*/0); +} + +future bind(ldap_connection& conn) { + return conn.simple_bind(manager_dn, manager_password); +} + + +/// Creates an ldap_connection, invokes a function on it, then waits for its closing. Must be +/// invoked from Seastar thread. +void with_ldap_connection(seastar::connected_socket&& socket, std::function f) { + mylog.trace("with_ldap_connection"); + ldap_connection c(std::move(socket)); + auto do_close = defer([&] { c.close().get(); }); + mylog.trace("with_ldap_connection: invoking f"); + f(c); + mylog.trace("with_ldap_connection done"); +} + +/// Connects to an address, then invokes with_ldap_connection on the resulting socket. Must be +/// invoked from Seastar thread. +void with_ldap_connection(const seastar::socket_address& a, std::function f) { + with_ldap_connection(connect(a).get(), f); +} + +} // anonymous namespace + +// Tests default (non-custom) libber networking. Failure here indicates a likely bug in test.py's +// LDAP setup. +SEASTAR_THREAD_TEST_CASE(bind_with_default_io) { + set_defbase(); + const auto server_uri = "ldap://localhost:" + ldap_port; + LDAP *manager_client_state{nullptr}; + BOOST_REQUIRE_EQUAL(LDAP_SUCCESS, ldap_initialize(&manager_client_state, server_uri.c_str())); + static constexpr int v3 = LDAP_VERSION3; + BOOST_REQUIRE_EQUAL(LDAP_OPT_SUCCESS, ldap_set_option(manager_client_state, LDAP_OPT_PROTOCOL_VERSION, &v3)); + // Retry on EINTR, rather than return error: + BOOST_REQUIRE_EQUAL(LDAP_OPT_SUCCESS, ldap_set_option(manager_client_state, LDAP_OPT_RESTART, LDAP_OPT_ON)); + BOOST_REQUIRE_EQUAL(LDAP_SUCCESS, ldap_simple_bind_s(manager_client_state, manager_dn, manager_password)); + BOOST_REQUIRE_EQUAL(LDAP_SUCCESS, ldap_unbind(manager_client_state)); +} + +SEASTAR_THREAD_TEST_CASE(bind_with_custom_sockbuf_io) { + set_defbase(); + mylog.trace("bind_with_custom_sockbuf_io"); + with_ldap_connection(local_ldap_address, [] (ldap_connection& c) { + mylog.trace("bind_with_custom_sockbuf_io invoking bind"); + const auto res = bind(c).get(); + BOOST_REQUIRE_EQUAL(LDAP_RES_BIND, ldap_msgtype(res.get())); + }); + mylog.trace("bind_with_custom_sockbuf_io done"); +} + +SEASTAR_THREAD_TEST_CASE(search_with_custom_sockbuf_io) { + set_defbase(); + mylog.trace("search_with_custom_sockbuf_io"); + with_ldap_connection(local_ldap_address, [] (ldap_connection& c) { + mylog.trace("search_with_custom_sockbuf_io: invoking search"); + const auto res = search(c, base_dn).get(); + mylog.trace("search_with_custom_sockbuf_io: got result"); + const auto actual = entries(c.get_ldap(), res.get()); + const std::set& expected = results_expected_from_search_base_dn; + BOOST_REQUIRE_EQUAL_COLLECTIONS(actual.cbegin(), actual.cend(), expected.cbegin(), expected.cend()); + }); + mylog.trace("search_with_custom_sockbuf_io done"); +} + +SEASTAR_THREAD_TEST_CASE(multiple_outstanding_operations) { + set_defbase(); + mylog.trace("multiple_outstanding_operations"); + with_ldap_connection(local_ldap_address, [] (ldap_connection& c) { + mylog.trace("multiple_outstanding_operations: bind"); + BOOST_REQUIRE_EQUAL(LDAP_RES_BIND, ldap_msgtype(bind(c).get().get())); + + std::vector> results_base; + for (size_t i = 0; i < 30; ++i) { + mylog.trace("multiple_outstanding_operations: invoking search base #{}", i); + results_base.push_back(search(c, base_dn)); + mylog.trace("multiple_outstanding_operations: search base #{} got future {}", i, static_cast(&results_base.back())); + } + + std::vector> results_jsmith; + for (size_t i = 0; i < 30; ++i) { + mylog.trace("multiple_outstanding_operations: invoking search jsmith #{}", i); + results_jsmith.push_back(search(c, "uid=jsmith,ou=People,dc=example,dc=com")); + mylog.trace("multiple_outstanding_operations: search jsmith #{} got future {}", i, static_cast(&results_jsmith.back())); + } + + using boost::test_tools::per_element; + mylog.trace("multiple_outstanding_operations: check base results"); + future<> base_result_fut = parallel_for_each(results_base, [&c] (future& res) { + const auto actual_base = entries(c.get_ldap(), res.get().get()); + BOOST_TEST_INFO("result for " << &res); + BOOST_TEST(actual_base == results_expected_from_search_base_dn, per_element()); + return make_ready_future(); + }); + + mylog.trace("multiple_outstanding_operations: check jsmith result"); + static const std::set expected_jsmith{"uid=jsmith,ou=People,dc=example,dc=com"}; + future<> jsmith_result_fut = parallel_for_each(results_jsmith, [&c] (future& res) { + const auto actual_jsmith = entries(c.get_ldap(), res.get().get()); + BOOST_TEST_INFO("result for " << &res); + BOOST_TEST(actual_jsmith == expected_jsmith, per_element()); + return make_ready_future(); + }); + + when_all(std::move(base_result_fut), std::move(jsmith_result_fut)).get(); + }); + mylog.trace("multiple_outstanding_operations done"); +} + +SEASTAR_THREAD_TEST_CASE(early_shutdown) { + set_defbase(); + mylog.trace("early_shutdown: noop"); + with_ldap_connection(local_ldap_address, [] (ldap_connection&) {}); + mylog.trace("early_shutdown: bind"); + with_ldap_connection(local_ldap_address, [] (ldap_connection& c) { bind(c).handle_exception(&ignore).get(); }); + mylog.trace("early_shutdown: search"); + with_ldap_connection( + local_ldap_address, [] (ldap_connection& c) { search(c, base_dn).handle_exception(&ignore).get(); }); +} + +SEASTAR_THREAD_TEST_CASE(bind_after_fail) { + set_defbase(); + mylog.trace("bind_after_fail: wonky connection"); + with_ldap_connection(local_fail_inject_address, [] (ldap_connection& wonky_conn) { + bind(wonky_conn).handle_exception(&ignore).get(); + }); + mylog.trace("bind_after_fail: solid connection"); + with_ldap_connection(local_ldap_address, [] (ldap_connection& c) { + const auto res = bind(c).get(); + BOOST_REQUIRE_EQUAL(LDAP_RES_BIND, ldap_msgtype(res.get())); + }); + mylog.trace("bind_after_fail done"); +} + +SEASTAR_THREAD_TEST_CASE(search_after_fail) { + set_defbase(); + mylog.trace("search_after_fail: wonky connection"); + with_ldap_connection(local_fail_inject_address, [] (ldap_connection& wonky_conn) { + search(wonky_conn, base_dn).handle_exception(&ignore).get(); + }); + mylog.trace("search_after_fail: solid connection"); + with_ldap_connection(local_ldap_address, [] (ldap_connection& c) { + const auto res = search(c, base_dn).get(); + mylog.trace("search_after_fail: got search result"); + const auto actual = entries(c.get_ldap(), res.get()); + BOOST_REQUIRE_EQUAL_COLLECTIONS( + actual.cbegin(), actual.cend(), + results_expected_from_search_base_dn.cbegin(), results_expected_from_search_base_dn.cend()); + }); + mylog.trace("search_after_fail done"); +} + +SEASTAR_THREAD_TEST_CASE(multiple_outstanding_operations_on_failing_connection) { + set_defbase(); + mylog.trace("multiple_outstanding_operations_on_failing_connection"); + with_ldap_connection(local_fail_inject_address, [] (ldap_connection& c) { + mylog.trace("multiple_outstanding_operations_on_failing_connection: invoking bind"); + bind(c).handle_exception(&ignore).get();; + + std::vector> results_base; + for (size_t i = 0; i < 10; ++i) { + mylog.trace("multiple_outstanding_operations_on_failing_connection: invoking search base"); + results_base.push_back(search(c, base_dn).handle_exception(&ignore)); + } + + std::vector> results_jsmith; + for (size_t i = 0; i < 10; ++i) { + mylog.trace("multiple_outstanding_operations_on_failing_connection: invoking search jsmith"); + results_jsmith.push_back(search(c, "uid=jsmith,ou=People,dc=example,dc=com").handle_exception(&ignore)); + } + + mylog.trace("multiple_outstanding_operations_on_failing_connection: getting base results"); + when_all_succeed(results_base.begin(), results_base.end()).get(); + mylog.trace("multiple_outstanding_operations_on_failing_connection: getting jsmith results"); + when_all_succeed(results_jsmith.begin(), results_jsmith.end()).get(); + }); + mylog.trace("multiple_outstanding_operations_on_failing_connection done"); +} + +using exception_predicate::message_contains; + +SEASTAR_THREAD_TEST_CASE(bind_after_close) { + set_defbase(); + ldap_connection c(connect(local_ldap_address).get()); + c.close().get(); + BOOST_REQUIRE_EXCEPTION(bind(c).get(), std::runtime_error, message_contains("ldap_connection")); +} + +SEASTAR_THREAD_TEST_CASE(search_after_close) { + set_defbase(); + ldap_connection c(connect(local_ldap_address).get()); + c.close().get(); + BOOST_REQUIRE_EXCEPTION(search(c, base_dn).get(), std::runtime_error, message_contains("ldap_connection")); +} + +SEASTAR_THREAD_TEST_CASE(close_after_close) { + set_defbase(); + ldap_connection c(connect(local_ldap_address).get()); + c.close().get(); + BOOST_REQUIRE_EXCEPTION(c.close().get(), std::runtime_error, message_contains("ldap_connection")); +} + +SEASTAR_THREAD_TEST_CASE(severed_connection_yields_exceptional_future) { + set_defbase(); + with_ldap_connection(local_fail_inject_address, [] (ldap_connection& c) { + int up = 1; + while (up) { + search(c, base_dn) + .handle_exception_type([&] (std::runtime_error&) { up = 0; return ldap_msg_ptr(); }) + .get(); + } + }); +} diff --git a/test/ldap/role_manager_test.cc b/test/ldap/role_manager_test.cc new file mode 100644 index 000000000000..e1f5f8a1ec3e --- /dev/null +++ b/test/ldap/role_manager_test.cc @@ -0,0 +1,537 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "auth/standard_role_manager.hh" +#include "auth/ldap_role_manager.hh" +#include "auth/password_authenticator.hh" +#include "db/config.hh" + +#include +#include +#include "utils/to_string.hh" +#include + +#include "test/lib/exception_utils.hh" +#include "test/lib/test_utils.hh" +#include "ldap_common.hh" +#include "service/migration_manager.hh" +#include "test/lib/cql_test_env.hh" + +auto make_manager(cql_test_env& env) { + auto stop_role_manager = [] (auth::standard_role_manager* m) { + m->stop().get(); + std::default_delete()(m); + }; + return std::unique_ptr( + new auth::standard_role_manager(env.local_qp(), env.get_raft_group0_client(), env.migration_manager().local()), + std::move(stop_role_manager)); +} + +SEASTAR_TEST_CASE(create_role) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_manager(env); + m->start().get(); + + const auto anon = auth::authenticated_user(); + + // + // Create a role, and verify its properties. + // + + auth::role_config c; + c.is_superuser = true; + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->create("admin", c, b).get(); + }); + BOOST_REQUIRE_EQUAL(m->exists("admin").get(), true); + BOOST_REQUIRE_EQUAL(m->can_login("admin").get(), false); + BOOST_REQUIRE_EQUAL(m->is_superuser("admin").get(), true); + + BOOST_REQUIRE_EQUAL( + m->query_granted("admin", auth::recursive_role_query::yes).get(), + std::unordered_set{"admin"}); + + // + // Creating a role that already exists is an error. + // + + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->create("admin", c, b).get(), auth::role_already_exists); + }); + }); +} + +SEASTAR_TEST_CASE(drop_role) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_manager(env); + m->start().get(); + + const auto anon = auth::authenticated_user(); + + // + // Create a role, then drop it, then verify it's gone. + // + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->create("lord", auth::role_config(), b).get(); + }); + do_with_mc(env, [&] (::service::group0_batch& b) { + m->drop("lord", b).get(); + }); + BOOST_REQUIRE_EQUAL(m->exists("lord").get(), false); + + // + // Dropping a role revokes it from other roles and revokes other roles from it. + // + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->create("peasant", auth::role_config(), b).get(); + m->create("lord", auth::role_config(), b).get(); + m->create("king", auth::role_config(), b).get(); + }); + + auth::role_config tim_config; + tim_config.is_superuser = false; + tim_config.can_login = true; + do_with_mc(env, [&] (::service::group0_batch& b) { + m->create("tim", tim_config, b).get(); + }); + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->grant("lord", "peasant", b).get(); + m->grant("king", "lord", b).get(); + m->grant("tim", "lord", b).get(); + }); + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->drop("lord", b).get(); + }); + + BOOST_REQUIRE_EQUAL( + m->query_granted("tim", auth::recursive_role_query::yes).get(), + std::unordered_set{"tim"}); + + BOOST_REQUIRE_EQUAL( + m->query_granted("king", auth::recursive_role_query::yes).get(), + std::unordered_set{"king"}); + + // + // Dropping a role that does not exist is an error. + // + + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->drop("emperor", b).get(), auth::nonexistant_role); + }); + }); +} + +SEASTAR_TEST_CASE(grant_role) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_manager(env); + m->start().get(); + + const auto anon = auth::authenticated_user(); + + auth::role_config jsnow_config; + jsnow_config.is_superuser = false; + jsnow_config.can_login = true; + do_with_mc(env, [&] (::service::group0_batch& b) { + m->create("jsnow", jsnow_config, b).get(); + + m->create("lord", auth::role_config(), b).get(); + m->create("king", auth::role_config(), b).get(); + }); + + // + // All kings have the rights of lords, and 'jsnow' is a king. + // + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->grant("king", "lord", b).get(); + m->grant("jsnow", "king", b).get(); + }); + + BOOST_REQUIRE_EQUAL( + m->query_granted("king", auth::recursive_role_query::yes).get(), + (std::unordered_set{"king", "lord"})); + + BOOST_REQUIRE_EQUAL( + m->query_granted("jsnow", auth::recursive_role_query::no).get(), + (std::unordered_set{"jsnow", "king"})); + + BOOST_REQUIRE_EQUAL( + m->query_granted("jsnow", auth::recursive_role_query::yes).get(), + (std::unordered_set{"jsnow", "king", "lord"})); + + // A non-existing role cannot be granted. + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->grant("jsnow", "doctor", b).get(), auth::nonexistant_role); + }); + + // A role cannot be granted to a non-existing role. + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->grant("hpotter", "lord", b).get(), auth::nonexistant_role); + }); + }); +} + +SEASTAR_TEST_CASE(revoke_role) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_manager(env); + m->start().get(); + + const auto anon = auth::authenticated_user(); + + auth::role_config rrat_config; + rrat_config.is_superuser = false; + rrat_config.can_login = true; + do_with_mc(env, [&] (::service::group0_batch& b) { + m->create("rrat", rrat_config, b).get(); + + m->create("chef", auth::role_config(), b).get(); + m->create("sous_chef", auth::role_config(), b).get(); + }); + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->grant("chef", "sous_chef", b).get(); + m->grant("rrat", "chef", b).get(); + }); + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->revoke("chef", "sous_chef", b).get(); + }); + + BOOST_REQUIRE_EQUAL( + m->query_granted("rrat", auth::recursive_role_query::yes).get(), + (std::unordered_set{"chef", "rrat"})); + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->revoke("rrat", "chef", b).get(); + }); + BOOST_REQUIRE_EQUAL( + m->query_granted("rrat", auth::recursive_role_query::yes).get(), + std::unordered_set{"rrat"}); + + // A non-existing role cannot be revoked. + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->revoke("rrat", "taster", b).get(), auth::nonexistant_role); + }); + + // A role cannot be revoked from a non-existing role. + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->revoke("ccasper", "chef", b).get(), auth::nonexistant_role); + }); + + // Revoking a role not granted is an error. + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->revoke("rrat", "sous_chef", b).get(), auth::revoke_ungranted_role); + }); + }); +} + +SEASTAR_TEST_CASE(alter_role) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_manager(env); + m->start().get(); + + const auto anon = auth::authenticated_user(); + + auth::role_config tsmith_config; + tsmith_config.is_superuser = true; + tsmith_config.can_login = true; + do_with_mc(env, [&] (::service::group0_batch& b) { + m->create("tsmith", tsmith_config, b).get(); + }); + + auth::role_config_update u; + u.can_login = false; + + do_with_mc(env, [&] (::service::group0_batch& b) { + m->alter("tsmith", u, b).get(); + }); + + BOOST_REQUIRE_EQUAL(m->is_superuser("tsmith").get(), true); + BOOST_REQUIRE_EQUAL(m->can_login("tsmith").get(), false); + + // Altering a non-existing role is an error. + do_with_mc(env, [&] (::service::group0_batch& b) { + BOOST_REQUIRE_THROW(m->alter("hjones", u, b).get(), auth::nonexistant_role); + }); + }); +} + +namespace { + +const auto default_query_template = fmt::format( + "ldap://localhost:{}/{}?cn?sub?(uniqueMember=uid={{USER}},ou=People,dc=example,dc=com)", + ldap_port, base_dn); + +const auto flaky_server_query_template = fmt::format( + "ldap://localhost:{}/{}?cn?sub?(uniqueMember=uid={{USER}},ou=People,dc=example,dc=com)", + std::stoi(ldap_port) + 2, base_dn); + +auto make_ldap_manager(cql_test_env& env, sstring query_template = default_query_template) { + auto stop_role_manager = [] (auth::ldap_role_manager* m) { + m->stop().get(); + std::default_delete()(m); + }; + return std::unique_ptr( + new auth::ldap_role_manager(query_template, /*target_attr=*/"cn", manager_dn, manager_password, + env.local_qp(), env.get_raft_group0_client(), env.migration_manager().local()), + std::move(stop_role_manager)); +} + +void create_ldap_roles(cql_test_env& env, auth::role_manager& rmgr) { + do_with_mc(env, [&] (::service::group0_batch& b) { + rmgr.create("jsmith", auth::role_config(), b).get(); + rmgr.create("role1", auth::role_config(), b).get(); + rmgr.create("role2", auth::role_config(), b).get(); + }); +} + +} // anonymous namespace + +using auth::role_set; + +SEASTAR_TEST_CASE(ldap_single_role) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + create_ldap_roles(env, *m); + const role_set expected{"jsmith", "role1"}; + BOOST_REQUIRE_EQUAL(expected, m->query_granted("jsmith", auth::recursive_role_query::no).get()); + }); +} + +SEASTAR_TEST_CASE(ldap_two_roles) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + create_ldap_roles(env, *m); + const role_set expected{"cassandra", "role1","role2"}; + BOOST_REQUIRE_EQUAL(expected, m->query_granted("cassandra", auth::recursive_role_query::no).get()); + }); +} + +SEASTAR_TEST_CASE(ldap_no_roles) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + create_ldap_roles(env, *m); + BOOST_REQUIRE_EQUAL(role_set{"dontexist"}, + m->query_granted("dontexist", auth::recursive_role_query::no).get()); + }); +} + +SEASTAR_TEST_CASE(ldap_wrong_role) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + create_ldap_roles(env, *m); + BOOST_REQUIRE_EQUAL(role_set{"jdoe"}, m->query_granted("jdoe", auth::recursive_role_query::no).get()); + }); +} + +SEASTAR_TEST_CASE(ldap_reconnect) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env, flaky_server_query_template); + m->start().get(); + create_ldap_roles(env, *m); + std::vector> queries; + constexpr int noq = 1000; + queries.reserve(noq); + for (int i = 0; i < noq; ++i) { + queries.push_back(m->query_granted("jsmith", auth::recursive_role_query::no) + .handle_exception([] (std::exception_ptr) { return role_set{}; })); + } + when_all(queries.begin(), queries.end()).get(); + }); +} + +using exception_predicate::message_contains; + +SEASTAR_TEST_CASE(ldap_wrong_url) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env, "wrong:/UR?L"); + BOOST_REQUIRE_EXCEPTION(m->start().get(), std::runtime_error, message_contains("server address")); + }); +} + +SEASTAR_TEST_CASE(ldap_wrong_server_name) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env, "ldap://server.that.will.never.exist.scylladb.com"); + m->start().get(); + BOOST_REQUIRE_EXCEPTION(m->query_granted("jdoe", auth::recursive_role_query::no).get(), + std::runtime_error, message_contains("reconnect fail")); + }); +} + +SEASTAR_TEST_CASE(ldap_wrong_port) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env, "ldap://localhost:2"); + m->start().get(); + BOOST_REQUIRE_EXCEPTION(m->query_granted("jdoe", auth::recursive_role_query::no).get(), + std::runtime_error, message_contains("reconnect fail")); + }); +} + +SEASTAR_TEST_CASE(ldap_qualified_name) { + return do_with_cql_env_thread([](cql_test_env& env) { + const sstring name(make_ldap_manager(env)->qualified_java_name()); + static const sstring suffix = "LDAPRoleManager"; + BOOST_REQUIRE_EQUAL(name.find(suffix), name.size() - suffix.size()); + }); +} + +SEASTAR_TEST_CASE(ldap_delegates_drop) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + create_ldap_roles(env, *m); + BOOST_REQUIRE(m->exists("role1").get()); + do_with_mc(env, [&] (service::group0_batch& b) { + m->drop("role1", b).get(); + }); + BOOST_REQUIRE(!m->exists("role1").get()); + }); +} + +SEASTAR_TEST_CASE(ldap_delegates_query_all) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + create_ldap_roles(env, *m); + const auto roles = m->query_all().get(); + BOOST_REQUIRE_EQUAL(1, roles.count("role1")); + BOOST_REQUIRE_EQUAL(1, roles.count("role2")); + BOOST_REQUIRE_EQUAL(1, roles.count("jsmith")); + }); +} + +SEASTAR_TEST_CASE(ldap_delegates_config) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + do_with_mc(env, [&] (service::group0_batch& b) { + m->create("super", auth::role_config{/*is_superuser=*/true, /*can_login=*/false}, b).get(); + }); + BOOST_REQUIRE(m->is_superuser("super").get()); + BOOST_REQUIRE(!m->can_login("super").get()); + do_with_mc(env, [&] (service::group0_batch& b) { + m->create("user", auth::role_config{/*is_superuser=*/false, /*can_login=*/true}, b).get(); + }); + BOOST_REQUIRE(!m->is_superuser("user").get()); + BOOST_REQUIRE(m->can_login("user").get()); + do_with_mc(env, [&] (service::group0_batch& b) { + m->alter("super", auth::role_config_update{/*is_superuser=*/true, /*can_login=*/true}, b).get(); + }); + BOOST_REQUIRE(m->can_login("super").get()); + }); +} + +SEASTAR_TEST_CASE(ldap_delegates_attributes) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + do_with_mc(env, [&] (service::group0_batch& b) { + m->create("r", auth::role_config{}, b).get(); + }); + BOOST_REQUIRE(!m->get_attribute("r", "a").get()); + do_with_mc(env, [&] (service::group0_batch& b) { + m->set_attribute("r", "a", "3", b).get(); + }); + // TODO: uncomment when failure is fixed. + //BOOST_REQUIRE_EQUAL("3", *m->get_attribute("r", "a").get()); + do_with_mc(env, [&] (service::group0_batch& b) { + m->remove_attribute("r", "a", b).get(); + }); + BOOST_REQUIRE(!m->get_attribute("r", "a").get()); + }); +} + +using exceptions::invalid_request_exception; + +SEASTAR_TEST_CASE(ldap_forbids_grant) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + do_with_mc(env, [&] (service::group0_batch& b) { + BOOST_REQUIRE_EXCEPTION(m->grant("a", "b", b).get(), invalid_request_exception, + message_contains("with LDAPRoleManager.")); + }); + }); +} + +SEASTAR_TEST_CASE(ldap_forbids_revoke) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + do_with_mc(env, [&] (service::group0_batch& b) { + BOOST_REQUIRE_EXCEPTION(m->revoke("a", "b", b).get(), invalid_request_exception, + message_contains("with LDAPRoleManager.")); + }); + }); +} + +SEASTAR_TEST_CASE(ldap_autocreate_user) { + return do_with_cql_env_thread([](cql_test_env& env) { + auto m = make_ldap_manager(env); + m->start().get(); + bool jsmith_exists = m->exists("jsmith").get(); + // JSmith cannot be auto-created - he's not assigned any existing roles in the system. + // He does belong to role1, but role1 was not explicitly created in Scylla yet. + BOOST_REQUIRE(!jsmith_exists); + do_with_mc(env, [&] (service::group0_batch& b) { + m->create("role1", auth::role_config(), b).get(); + }); + // JSmith is now assigned an existing role - role1 - so he can be auto-created. + jsmith_exists = m->exists("jsmith").get(); + BOOST_REQUIRE(jsmith_exists); + do_with_mc(env, [&] (service::group0_batch& b) { + m->drop("jsmith", b).get(); + m->drop("role1", b).get(); + }); + // JSmith was revoked role1 (simulated by dropping it from Scylla, which is easier), + // and his account was deleted. No auto-creation for you, Joe, move on. + jsmith_exists = m->exists("jsmith").get(); + BOOST_REQUIRE(!jsmith_exists); + // JDeer does not exist at all and is not assigned any roles, even in LDAP, + // which translates to no auto-creation. + bool jdeer_exists = m->exists("jdeer").get(); + BOOST_REQUIRE(!jdeer_exists); + }); +} + +namespace { + +shared_ptr make_ldap_config() { + auto p = make_shared(); + p->role_manager("com.scylladb.auth.LDAPRoleManager"); + p->authenticator(sstring(auth::password_authenticator_name)); + p->ldap_url_template(default_query_template); + p->ldap_attr_role("cn"); + p->ldap_bind_dn(manager_dn); + p->ldap_bind_passwd(manager_password); + return p; +} + +} // anonymous namespace + +SEASTAR_TEST_CASE(ldap_config) { + return do_with_cql_env_thread([](cql_test_env& env) { + const auto& svc = env.local_auth_service(); + BOOST_REQUIRE_EQUAL(role_set{"cassandra"}, svc.get_roles("cassandra").get()); + do_with_mc(env, [&] (service::group0_batch& b) { + auth::create_role(svc, "role1", auth::role_config{}, auth::authentication_options{}, b).get(); + }); + const role_set expected{"cassandra", "role1"}; + BOOST_REQUIRE_EQUAL(expected, svc.get_roles("cassandra").get()); + }, + make_ldap_config()); +} diff --git a/test/ldap/saslauthd_authenticator_test.cc b/test/ldap/saslauthd_authenticator_test.cc new file mode 100644 index 000000000000..a5935176a191 --- /dev/null +++ b/test/ldap/saslauthd_authenticator_test.cc @@ -0,0 +1,204 @@ +/* + * Copyright (C) 2020 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include +#include +#include +#include + +#include +#include "utils/to_string.hh" + +#include "auth/saslauthd_authenticator.hh" +#include "db/config.hh" +#include "test/ldap/ldap_common.hh" +#include "test/lib/cql_test_env.hh" +#include "test/lib/exception_utils.hh" +#include "test/lib/test_utils.hh" +#include "seastarx.hh" + +const auto sockpath = std::getenv("SASLAUTHD_MUX_PATH"); + +using exceptions::authentication_exception; +using exception_predicate::message_contains; + +SEASTAR_THREAD_TEST_CASE(simple_password_checking) { + BOOST_REQUIRE(!auth::authenticate_with_saslauthd(sockpath, {"jdoe", "xxxxxxxx", "", ""}).get()); + BOOST_REQUIRE(auth::authenticate_with_saslauthd(sockpath, {"jdoe", "pa55w0rd", "", ""}).get()); + BOOST_REQUIRE(!auth::authenticate_with_saslauthd(sockpath, {"", "", "", ""}).get()); + BOOST_REQUIRE(!auth::authenticate_with_saslauthd(sockpath, {"", "", ".", "."}).get()); + BOOST_REQUIRE_EXCEPTION( + auth::authenticate_with_saslauthd("/a/nonexistent/path", {"jdoe", "pa55w0rd", "", ""}).get(), + authentication_exception, message_contains("socket connection error")); +} + +namespace { + +void fail_test(std::exception_ptr ex) { + BOOST_FAIL(format("{}", ex)); +} + +/// Creates a network response that saslauthd would send to convey this payload. If lie_size is provided, +/// force-write it into the response's first two bytes, even if that results in an invalid response. +temporary_buffer make_saslauthd_response(std::string_view payload, std::optional lie_size = std::nullopt) { + const uint16_t sz = payload.size(); + temporary_buffer resp(sz + 2); + auto p = resp.get_write(); + produce_be(p, lie_size.value_or(sz)); + std::copy_n(payload.begin(), sz, p); + return resp; +} + +/// Invokes authenticate_with_saslauthd against a mock saslauthd instance that sends this response through this +/// domain socket. +/// +/// authenticate_with_saslauthd is invoked with correct credentials, and its result is returned. +/// +/// Must be invoked inside a Seastar thread. +bool authorize_against_this_response(temporary_buffer resp, sstring socket_path) { + auto socket = seastar::listen(socket_address(unix_domain_addr(socket_path))); + auto [result, closing] = when_all( + auth::authenticate_with_saslauthd(socket_path, {"jdoe", "pa55w0rd", "", ""}), + socket.accept().then([resp = std::move(resp), socket_path] (accept_result ar) mutable { + return do_with( + ar.connection.input(), ar.connection.output(), socket_path, + [resp = std::move(resp)] (input_stream& in, output_stream& out, sstring& socket_path) mutable { + return in.read().then( + [&out, resp=std::move(resp)] (temporary_buffer) mutable { + return out.write(std::move(resp)).finally([&out] { return out.close(); }); + }).handle_exception(fail_test).finally([&] { + return in.close().finally([&] { return remove_file(socket_path); }); + }); + }); + })).get(); + return result.get(); +} + +/// Temp file name unique to this test run and this suffix. +sstring tmpfile(const sstring& suffix) { + return seastar::format("saslauthd_authenticator_test.tmpfile.{}.{}", ldap_port, suffix); +} + +shared_ptr make_config() { + auto p = make_shared(); + p->authenticator("com.scylladb.auth.SaslauthdAuthenticator"); + p->saslauthd_socket_path(sockpath); + return p; +} + +auth::authenticator& authenticator(cql_test_env& env) { + return env.local_auth_service().underlying_authenticator(); +} + +/// Creates a cql_test_env with saslauthd_authenticator in a Seastar thread, then invokes func with the env's +/// authenticator. +future<> do_with_authenticator_thread(std::function func) { + return do_with_cql_env_thread([func = std::move(func)] (cql_test_env& env) { + return do_with_mc(env, [&] (service::group0_batch& b) { + return func(authenticator(env), b); + }); + }, make_config()); +} + +} // anonymous namespace + +SEASTAR_THREAD_TEST_CASE(empty_response) { + BOOST_REQUIRE_EXCEPTION(authorize_against_this_response(temporary_buffer(0), tmpfile("0")), + authentication_exception, message_contains("closed connection")); +} + +SEASTAR_THREAD_TEST_CASE(single_byte_response) { + BOOST_REQUIRE_EXCEPTION( + authorize_against_this_response(temporary_buffer(1), tmpfile("1")), + authentication_exception, message_contains("closed connection")); +} + +SEASTAR_THREAD_TEST_CASE(two_byte_response) { + BOOST_REQUIRE(!authorize_against_this_response(make_saslauthd_response(""), tmpfile("2"))); + BOOST_REQUIRE_EXCEPTION( + authorize_against_this_response(make_saslauthd_response("", 1), tmpfile("2")), + authentication_exception, message_contains("response length different")); + BOOST_REQUIRE_EXCEPTION( + authorize_against_this_response(make_saslauthd_response("", 100), tmpfile("2")), + authentication_exception, message_contains("response length different")); +} + +SEASTAR_THREAD_TEST_CASE(three_byte_response) { + BOOST_REQUIRE(!authorize_against_this_response(make_saslauthd_response("O"), tmpfile("3"))); + // If advertised size is 0, the payload isn't read even if sent. No exception is expected: + BOOST_REQUIRE(!authorize_against_this_response(make_saslauthd_response("O", 0), tmpfile("3"))); + BOOST_REQUIRE_EXCEPTION( + authorize_against_this_response(make_saslauthd_response("O", 100), tmpfile("3")), + authentication_exception, message_contains("response length different")); +} + +SEASTAR_THREAD_TEST_CASE(ok_response_wrong_length) { + BOOST_REQUIRE_EXCEPTION( + authorize_against_this_response(make_saslauthd_response("OK", 100), tmpfile("3")), + authentication_exception, message_contains("response length different")); + // Extra payload beyond advertised size is not read. No exception is expected: + BOOST_REQUIRE(!authorize_against_this_response(make_saslauthd_response("OK", 1), tmpfile("3"))); +} + +SEASTAR_TEST_CASE(require_authentication) { + return do_with_authenticator_thread([] (auth::authenticator& authr, service::group0_batch& b) { + BOOST_REQUIRE(authr.require_authentication()); + }); +} + +SEASTAR_TEST_CASE(authenticate) { + return do_with_authenticator_thread([] (auth::authenticator& authr, service::group0_batch& b) { + const auto user = auth::authenticator::USERNAME_KEY, pwd = auth::authenticator::PASSWORD_KEY; + BOOST_REQUIRE_EQUAL(authr.authenticate({{user, "jdoe"}, {pwd, "pa55w0rd"}}).get().name, "jdoe"); + BOOST_REQUIRE_EXCEPTION( + authr.authenticate({{user, "jdoe"}, {pwd, ""}}).get(), + authentication_exception, message_contains("Incorrect credentials")); + BOOST_REQUIRE_EXCEPTION( + authr.authenticate({{user, "jdoe"}}).get(), + authentication_exception, message_contains("password' is missing")); + BOOST_REQUIRE_EXCEPTION( + authr.authenticate({{pwd, "pwd"}}).get(), + authentication_exception, message_contains("username' is missing")); + }); +} + +SEASTAR_TEST_CASE(create) { + return do_with_authenticator_thread([] (auth::authenticator& authr, service::group0_batch& b) { + BOOST_REQUIRE_EXCEPTION( + authr.create("new-role", {auth::password_option{"password"}}, b).get(), + authentication_exception, message_contains("Cannot create")); + }); +} + +SEASTAR_TEST_CASE(alter) { + return do_with_authenticator_thread([] (auth::authenticator& authr, service::group0_batch& b) { + BOOST_REQUIRE_EXCEPTION( + authr.alter("jdoe", {auth::password_option{"password"}}, b).get(), + authentication_exception, message_contains("Cannot modify")); + }); +} + +SEASTAR_TEST_CASE(drop) { + return do_with_authenticator_thread([] (auth::authenticator& authr, service::group0_batch& b) { + BOOST_REQUIRE_EXCEPTION(authr.drop("jdoe", b).get(), authentication_exception, message_contains("Cannot delete")); + }); +} + +SEASTAR_TEST_CASE(sasl_challenge) { + return do_with_authenticator_thread([] (auth::authenticator& authr, service::group0_batch& b) { + constexpr char creds[] = "\0jdoe\0pa55w0rd"; + const auto ch = authr.new_sasl_challenge(); + BOOST_REQUIRE(ch->evaluate_response(bytes(creds, creds + 14)).empty()); + BOOST_REQUIRE_EQUAL("jdoe", ch->get_authenticated_user().get().name); + BOOST_REQUIRE(ch->evaluate_response(bytes(creds, creds + 13)).empty()); + BOOST_REQUIRE_EXCEPTION( + ch->get_authenticated_user().get(), + authentication_exception, message_contains("Incorrect credentials")); + }); +} diff --git a/test/ldap/suite.yaml b/test/ldap/suite.yaml new file mode 100644 index 000000000000..0e9afe06bc83 --- /dev/null +++ b/test/ldap/suite.yaml @@ -0,0 +1,3 @@ +type: ldap +# Custom command line arguments for some of the tests +custom_args: {} diff --git a/test/resource/slapd.conf b/test/resource/slapd.conf new file mode 100644 index 000000000000..1d13610367c4 --- /dev/null +++ b/test/resource/slapd.conf @@ -0,0 +1,16 @@ +# before the first database definition +database config +# NOTE: the suffix is hardcoded as cn=config and +# MUST not have a suffix directive +# normal rules apply - rootdn can be anything you want +# but MUST be under cn=config +rootdn "cn=admin,cn=config" + +pidfile ./pidfile.pid +include /etc/openldap/schema/core.schema + +database mdb +suffix "dc=example,dc=com" +rootdn "cn=root,dc=example,dc=com" +rootpw secret +index objectClass eq diff --git a/utils/sequential_producer.hh b/utils/sequential_producer.hh new file mode 100644 index 000000000000..a07578151079 --- /dev/null +++ b/utils/sequential_producer.hh @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2021 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include + +/// Invokes a factory to produce an object, but sequentially: only one fiber at a time may be executing the +/// factory. Any other fiber requesting the object will wait for the existing factory invocation to finish, then +/// copy the result. +/// +/// TODO: Move to Seastar. +template +class sequential_producer { + public: + using factory_t = std::function()>; + + private: + factory_t _factory; + seastar::shared_future _churning; ///< Resolves when the previous _factory call completes. + + public: + sequential_producer(factory_t&& f) : _factory(std::move(f)) + { + clear(); + } + + seastar::future operator()() { + if (_churning.available()) { + _churning = _factory(); + } + return _churning.get_future(); + } + + void clear() { + _churning = seastar::make_exception_future( + std::logic_error("initial future used in sequential_producer")); + } +}; From 8d2ff8a9157cd7d29e94684a02faf64edcdfb142 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Sun, 10 Nov 2024 15:35:45 +0200 Subject: [PATCH 257/397] utils: add disk_space_monitor Instantiated only on shard 0. Currently, only subscribe from unit test Manual unit test using loop mount was added. Note that the test requires sudo access and root access to /dev/loop, so it cannot run in rootless podman instance, and it'd fail with Permission denied. Signed-off-by: Benny Halevy Closes scylladb/scylladb#21523 --- configure.py | 1 + db/config.cc | 3 ++ db/config.hh | 4 ++ main.cc | 26 ++++++++++- test/lib/cql_test_env.cc | 12 +++++ utils/CMakeLists.txt | 1 + utils/disk_space_monitor.cc | 87 +++++++++++++++++++++++++++++++++++++ utils/disk_space_monitor.hh | 86 ++++++++++++++++++++++++++++++++++++ 8 files changed, 218 insertions(+), 2 deletions(-) create mode 100644 utils/disk_space_monitor.cc create mode 100644 utils/disk_space_monitor.hh diff --git a/configure.py b/configure.py index 225ea0e0b3d2..b162a1c1fb19 100755 --- a/configure.py +++ b/configure.py @@ -1173,6 +1173,7 @@ def find_ninja(): 'node_ops/node_ops_ctl.cc', 'node_ops/task_manager_module.cc', 'reader_concurrency_semaphore_group.cc', + 'utils/disk_space_monitor.cc', ] + [Antlr3Grammar('cql3/Cql.g')] \ + scylla_raft_core ) diff --git a/db/config.cc b/db/config.cc index 46cb9dc732ed..ccde52961518 100644 --- a/db/config.cc +++ b/db/config.cc @@ -1268,6 +1268,9 @@ db::config::config(std::shared_ptr exts) "to try to slow down the client and prevent buildup of unfinished view updates. " "To be effective, this maximal delay should be larger than the typical latencies. " "Setting view_flow_control_delay_limit_in_ms to 0 disables view-update flow control.") + , disk_space_monitor_normal_polling_interval_in_seconds(this, "disk_space_monitor_normal_polling_interval_in_seconds", value_status::Used, 10, "Disk-space polling interval while below polling threshold") + , disk_space_monitor_high_polling_interval_in_seconds(this, "disk_space_monitor_high_polling_interval_in_seconds", value_status::Used, 1, "Disk-space polling interval at or above polling threshold") + , disk_space_monitor_polling_interval_threshold(this, "disk_space_monitor_polling_interval_threshold", value_status::Used, 0.9, "Disk-space polling threshold. Polling interval is increased when disk utilization is greater than or equal to this threshold") , default_log_level(this, "default_log_level", value_status::Used, seastar::log_level::info, "Default log level for log messages") , logger_log_level(this, "logger_log_level", value_status::Used, {}, "Map of logger name to log level. Valid log levels are 'error', 'warn', 'info', 'debug' and 'trace'") , log_to_stdout(this, "log_to_stdout", value_status::Used, true, "Send log output to stdout") diff --git a/db/config.hh b/db/config.hh index 0a018250e64b..630754d1b534 100644 --- a/db/config.hh +++ b/db/config.hh @@ -521,6 +521,10 @@ public: named_value enable_tablets; named_value view_flow_control_delay_limit_in_ms; + named_value disk_space_monitor_normal_polling_interval_in_seconds; + named_value disk_space_monitor_high_polling_interval_in_seconds; + named_value disk_space_monitor_polling_interval_threshold; + static const sstring default_tls_priority; private: template diff --git a/main.cc b/main.cc index 8b73496bfcfd..3597085032ed 100644 --- a/main.cc +++ b/main.cc @@ -117,6 +117,7 @@ #include "utils/advanced_rpc_compressor.hh" #include "utils/shared_dict.hh" #include "message/dictionary_service.hh" +#include "utils/disk_space_monitor.hh" seastar::metrics::metric_groups app_metrics; @@ -751,6 +752,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl sharded erm_factory; sharded mm_notifier; sharded lifecycle_notifier; + std::optional disk_space_monitor_shard0; sharded cm; sharded sstm; distributed db; @@ -813,7 +815,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl tcp_syncookies_sanity(); tcp_timestamps_sanity(); - return seastar::async([&app, cfg, ext, &cm, &sstm, &db, &qp, &bm, &proxy, &mapreduce_service, &mm, &mm_notifier, &ctx, &opts, &dirs, + return seastar::async([&app, cfg, ext, &disk_space_monitor_shard0, &cm, &sstm, &db, &qp, &bm, &proxy, &mapreduce_service, &mm, &mm_notifier, &ctx, &opts, &dirs, &prometheus_server, &cf_cache_hitrate_calculator, &load_meter, &feature_service, &gossiper, &snitch, &token_metadata, &erm_factory, &snapshot_ctl, &messaging, &sst_dir_semaphore, &raft_gr, &service_memory_limiter, &repair, &sst_loader, &ss, &lifecycle_notifier, &stream_manager, &task_manager, &rpc_dict_training_worker] { @@ -1130,7 +1132,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl utils::directories::set data_dir_set; data_dir_set.add(cfg->data_file_directories()); dirs->create_and_verify(data_dir_set, utils::directories::recursive::no).get(); - utils::directories::verify_owner_and_mode_of_data_dir(std::move(data_dir_set)).get(); + utils::directories::verify_owner_and_mode_of_data_dir(data_dir_set).get(); auto hints_dir_initializer = db::hints::directory_initializer::make(*dirs, cfg->hints_directory()).get(); auto view_hints_dir_initializer = db::hints::directory_initializer::make(*dirs, cfg->view_hints_directory()).get(); @@ -1185,6 +1187,26 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl scheduling_group_key_config cql_sg_stats_cfg = make_scheduling_group_key_config(maintenance_socket_enabled::no); auto cql_sg_stats_key = scheduling_group_key_create(cql_sg_stats_cfg).get(); + supervisor::notify("starting disk space monitor"); + auto dsm_cfg = utils::disk_space_monitor::config{ + .sched_group = dbcfg.streaming_scheduling_group, + .normal_polling_interval = cfg->disk_space_monitor_normal_polling_interval_in_seconds, + .high_polling_interval = cfg->disk_space_monitor_high_polling_interval_in_seconds, + .polling_interval_threshold = cfg->disk_space_monitor_polling_interval_threshold, + }; + if (data_dir_set.get_paths().empty()) { + throw std::runtime_error("data_dir_set must be non-empty"); + } + fs::path data_dir = *data_dir_set.get_paths().begin(); + if (data_dir_set.get_paths().size() > 1) { + startlog.warn("Multiple data directories aren't supported. Will monitor only {}", data_dir); + } + disk_space_monitor_shard0.emplace(stop_signal.as_local_abort_source(), data_dir, dsm_cfg); + disk_space_monitor_shard0->start().get(); + auto stop_dsm = defer_verbose_shutdown("disk space monitor", [&disk_space_monitor_shard0] { + disk_space_monitor_shard0->stop().get(); + }); + supervisor::notify("starting compaction_manager"); // get_cm_cfg is called on each shard when starting a sharded // we need the getter since updateable_value is not shard-safe (#7316) diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index c1c906859ec4..5bba469e43cf 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -68,6 +68,7 @@ #include "sstables/sstables_manager.hh" #include "init.hh" #include "lang/manager.hh" +#include "utils/disk_space_monitor.hh" #include #include @@ -154,6 +155,7 @@ class single_node_cql_env : public cql_test_env { sharded _token_metadata; sharded _erm_factory; sharded _sst_dir_semaphore; + std::optional _disk_space_monitor_shard0; sharded _lang_manager; sharded _cql_config; sharded _elc_notif; @@ -575,6 +577,16 @@ class single_node_cql_env : public cql_test_env { _task_manager.stop().get(); }); + utils::disk_space_monitor::config dsm_cfg = { + .sched_group = scheduling_groups.streaming_scheduling_group, + .normal_polling_interval = cfg->disk_space_monitor_normal_polling_interval_in_seconds, + .high_polling_interval = cfg->disk_space_monitor_high_polling_interval_in_seconds, + .polling_interval_threshold = cfg->disk_space_monitor_polling_interval_threshold, + }; + _disk_space_monitor_shard0.emplace(abort_sources.local(), data_dir_path, dsm_cfg); + _disk_space_monitor_shard0->start().get(); + auto stop_dsm = defer([this] { _disk_space_monitor_shard0->stop().get(); }); + // get_cm_cfg is called on each shard when starting a sharded // we need the getter since updateable_value is not shard-safe (#7316) auto get_cm_cfg = sharded_parameter([&] { diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 77caf61615c7..1dde955f2e9b 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -21,6 +21,7 @@ target_sources(utils dict_trainer.cc directories.cc disk-error-handler.cc + disk_space_monitor.cc dynamic_bitset.cc error_injection.cc exceptions.cc diff --git a/utils/disk_space_monitor.cc b/utils/disk_space_monitor.cc new file mode 100644 index 000000000000..5d9374a073cb --- /dev/null +++ b/utils/disk_space_monitor.cc @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2024-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include + +#include +#include +#include + +#include "utils/disk_space_monitor.hh" +#include "utils/assert.hh" +#include "utils/log.hh" + +using namespace std::chrono_literals; + +namespace utils { + +seastar::logger dsmlog("disk_space_monitor"); + +disk_space_monitor::disk_space_monitor(abort_source& as, std::filesystem::path data_dir, config cfg) + : _as_sub(as.subscribe([this] () noexcept { _as.request_abort(); })) + , _data_dir(std::move(data_dir)) + , _cfg(std::move(cfg)) +{} + +disk_space_monitor::~disk_space_monitor() { + SCYLLA_ASSERT(_poller_fut.available()); +} + +future<> disk_space_monitor::start() { + _space_info = co_await get_filesystem_space(); + _poller_fut = poll(); +} + +future<> disk_space_monitor::stop() noexcept { + _as.request_abort(); + co_await _signal_barrier.advance_and_await(); + co_await std::exchange(_poller_fut, make_ready_future()); +} + +disk_space_monitor::signal_connection_type disk_space_monitor::listen(signal_callback_type callback) { + return _signal_source.connect([this, callback = std::move(callback)] () mutable -> future<> { + auto op = _signal_barrier.start(); + co_await callback(*this); + }); +} + +future<> disk_space_monitor::poll() { + try { + while (!_as.abort_requested()) { + auto now = clock_type::now(); + _space_info = co_await get_filesystem_space(); + + if (_as.abort_requested()) { + co_return; + } + co_await _signal_barrier.advance_and_await(); + _signal_source(); + + auto passed = clock_type::now() - now; + auto interval = get_polling_interval(); + if (interval > passed) { + co_await sleep_abortable(interval - passed, _as); + } + } + } catch (const sleep_aborted&) { + } catch (const abort_requested_exception&) { + } catch (...) { + dsmlog.error("poll loop exited with error: {}", std::current_exception()); + } +} + +future disk_space_monitor::get_filesystem_space() { + return engine().file_system_space(_data_dir.native()); +} + +disk_space_monitor::clock_type::duration disk_space_monitor::get_polling_interval() const noexcept { + auto du = disk_utilization(); + return std::chrono::seconds(du < _cfg.polling_interval_threshold.get() ? _cfg.normal_polling_interval.get() : _cfg.high_polling_interval.get()); +} + +} // namespace utils diff --git a/utils/disk_space_monitor.hh b/utils/disk_space_monitor.hh new file mode 100644 index 000000000000..afcc7165784b --- /dev/null +++ b/utils/disk_space_monitor.hh @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2024-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "seastarx.hh" +#include "utils/updateable_value.hh" +#include "utils/phased_barrier.hh" + +namespace utils { + +// Instantiated only on shard 0 +class disk_space_monitor { +public: + using clock_type = lowres_clock; + using signal_type = boost::signals2::signal_type>::type; + using signal_callback_type = std::function(const disk_space_monitor&)>; + using signal_connection_type = boost::signals2::scoped_connection; + + struct config { + scheduling_group sched_group; + updateable_value normal_polling_interval; + updateable_value high_polling_interval; + // Use high_polling_interval above this threshold + updateable_value polling_interval_threshold; + }; + +private: + abort_source _as; + optimized_optional _as_sub; + future<> _poller_fut = make_ready_future(); + utils::phased_barrier _signal_barrier; + signal_type _signal_source; + std::filesystem::space_info _space_info; + std::filesystem::path _data_dir; + config _cfg; + +public: + disk_space_monitor(abort_source& as, std::filesystem::path data_dir, config cfg); + ~disk_space_monitor(); + + future<> start(); + + future<> stop() noexcept; + + const std::filesystem::path& data_dir() const noexcept { + return _data_dir; + } + + std::filesystem::space_info space() const noexcept { + return _space_info; + } + + float disk_utilization() const noexcept { + return _space_info.capacity ? (float)(_space_info.capacity - _space_info.available) / _space_info.capacity : -1; + } + + signal_connection_type listen(signal_callback_type callback); + +private: + future<> poll(); + + future get_filesystem_space(); + + clock_type::duration get_polling_interval() const noexcept; +}; + +} // namespace utils From b4ca9489c46a01415d1a812f58cfcdd1dfd07734 Mon Sep 17 00:00:00 2001 From: Jenkins Promoter Date: Sun, 12 Jan 2025 15:05:23 +0200 Subject: [PATCH 258/397] Update pgo profiles - x86_64 --- pgo/profiles/x86_64/profile.profdata.xz | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 pgo/profiles/x86_64/profile.profdata.xz diff --git a/pgo/profiles/x86_64/profile.profdata.xz b/pgo/profiles/x86_64/profile.profdata.xz new file mode 100644 index 000000000000..76d83cd40bdc --- /dev/null +++ b/pgo/profiles/x86_64/profile.profdata.xz @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:792d30bc07d3db08d947dcc05f1a2829f9ac4f9d0786402bb8f9acc63670ace3 +size 5638004 From a7d8d21e8631a64994cff315f2946d2a9596518c Mon Sep 17 00:00:00 2001 From: Jenkins Promoter Date: Sun, 12 Jan 2025 15:27:45 +0200 Subject: [PATCH 259/397] Update pgo profiles - aarch64 --- pgo/profiles/aarch64/profile.profdata.xz | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 pgo/profiles/aarch64/profile.profdata.xz diff --git a/pgo/profiles/aarch64/profile.profdata.xz b/pgo/profiles/aarch64/profile.profdata.xz new file mode 100644 index 000000000000..c4dfaa9a1e20 --- /dev/null +++ b/pgo/profiles/aarch64/profile.profdata.xz @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:87fea54e41a383693359d063fae5a97b365fc7dda6b264a6e6a41bc12ea67bb5 +size 5634076 From 6f30d26f2afa7ff2759d1b82d81d61ee6c2918cf Mon Sep 17 00:00:00 2001 From: Yaron Kaikov Date: Thu, 9 Jan 2025 16:34:04 +0200 Subject: [PATCH 260/397] Update tools/cqlsh submodule * tools/cqlsh b09bc793...52c61306 (3): > cleanup: remove un-used Dockerfiles > .github/workflows/build-push.yml: update to newer macos images > cython: fix the usage of cython Closes scylladb/scylladb#22250 --- tools/cqlsh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/cqlsh b/tools/cqlsh index b09bc7936152..52c6130665a0 160000 --- a/tools/cqlsh +++ b/tools/cqlsh @@ -1 +1 @@ -Subproject commit b09bc7936152ede61cd0e654d8eb877844101886 +Subproject commit 52c6130665a0a89c48423e68284f8e91d33ef588 From e71ac354266482f0f4e822dcc1d3dc8f56ce71cd Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Thu, 9 Jan 2025 18:11:54 +0800 Subject: [PATCH 261/397] mutation_writer,redis: do not include unused headers the changes porting enterprise features to oss brought some used include to the tree. so let's remove them. these unused includes were identified by clang-include-cleaner. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22246 --- mutation_writer/partition_based_splitting_writer.hh | 1 - mutation_writer/shard_based_splitting_writer.hh | 1 - redis/abstract_command.hh | 10 +--------- redis/exceptions.hh | 4 +--- redis/lolwut.hh | 2 +- redis/mutation_utils.hh | 2 +- redis/query_processor.hh | 2 -- redis/query_utils.hh | 2 +- redis/reply.hh | 6 +++--- redis/request.hh | 2 +- redis/version.hh | 2 +- 11 files changed, 10 insertions(+), 24 deletions(-) diff --git a/mutation_writer/partition_based_splitting_writer.hh b/mutation_writer/partition_based_splitting_writer.hh index 6d8ffcc39386..9aba5e8e58a1 100644 --- a/mutation_writer/partition_based_splitting_writer.hh +++ b/mutation_writer/partition_based_splitting_writer.hh @@ -8,7 +8,6 @@ #pragma once -#include #include "readers/mutation_reader.hh" namespace mutation_writer { diff --git a/mutation_writer/shard_based_splitting_writer.hh b/mutation_writer/shard_based_splitting_writer.hh index 586be88ba72e..6a3cd9c1cf10 100644 --- a/mutation_writer/shard_based_splitting_writer.hh +++ b/mutation_writer/shard_based_splitting_writer.hh @@ -8,7 +8,6 @@ #pragma once -#include #include "readers/mutation_reader.hh" namespace mutation_writer { diff --git a/redis/abstract_command.hh b/redis/abstract_command.hh index 55252e68b15c..fe7717d20979 100644 --- a/redis/abstract_command.hh +++ b/redis/abstract_command.hh @@ -7,17 +7,9 @@ */ #pragma once -#include "bytes.hh" +#include "bytes_fwd.hh" #include -#include -#include "redis/request.hh" #include "redis/reply.hh" -#include "db/consistency_level_type.hh" -#include "db/timeout_clock.hh" -#include "db/system_keyspace.hh" -#include "keys.hh" -#include "timestamp.hh" -#include class service_permit; diff --git a/redis/exceptions.hh b/redis/exceptions.hh index 1d24b105aaa8..99d9276313d1 100644 --- a/redis/exceptions.hh +++ b/redis/exceptions.hh @@ -8,9 +8,7 @@ #pragma once -#include - -#include "bytes.hh" +#include "bytes_fwd.hh" #include "seastarx.hh" diff --git a/redis/lolwut.hh b/redis/lolwut.hh index 379c35226b35..82facaaa7e8b 100644 --- a/redis/lolwut.hh +++ b/redis/lolwut.hh @@ -10,7 +10,7 @@ #include -#include "bytes.hh" +#include "bytes_fwd.hh" #include "seastarx.hh" namespace redis { diff --git a/redis/mutation_utils.hh b/redis/mutation_utils.hh index 4910168045e8..b4e31a11ce18 100644 --- a/redis/mutation_utils.hh +++ b/redis/mutation_utils.hh @@ -9,7 +9,7 @@ #pragma once #include #include -#include "bytes.hh" +#include "bytes_fwd.hh" #include "seastarx.hh" class service_permit; diff --git a/redis/query_processor.hh b/redis/query_processor.hh index c498f204b5ab..f46faf200166 100644 --- a/redis/query_processor.hh +++ b/redis/query_processor.hh @@ -8,8 +8,6 @@ #pragma once -#include -#include #include #include diff --git a/redis/query_utils.hh b/redis/query_utils.hh index e584ac08e82b..8e36ca1a95a9 100644 --- a/redis/query_utils.hh +++ b/redis/query_utils.hh @@ -10,7 +10,7 @@ #include #include -#include "bytes.hh" +#include "bytes_fwd.hh" #include "gc_clock.hh" #include "query-request.hh" diff --git a/redis/reply.hh b/redis/reply.hh index a0a6cfbef021..52d0b0dc1081 100644 --- a/redis/reply.hh +++ b/redis/reply.hh @@ -8,10 +8,10 @@ #pragma once -#include "bytes.hh" -#include +#include "bytes_fwd.hh" +#include +#include #include -#include #include #include "redis/exceptions.hh" diff --git a/redis/request.hh b/redis/request.hh index b787caa9ca3c..8f1d53cdd336 100644 --- a/redis/request.hh +++ b/redis/request.hh @@ -9,7 +9,7 @@ #pragma once #include -#include "bytes.hh" +#include "bytes_fwd.hh" namespace redis { diff --git a/redis/version.hh b/redis/version.hh index cb254d4154d1..e74412802f98 100644 --- a/redis/version.hh +++ b/redis/version.hh @@ -7,7 +7,7 @@ */ #pragma once -#include "bytes.hh" +#include "bytes_fwd.hh" namespace redis { From a5715086a4dd8f0bb3c1566b05131e3b1b9fb9ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dawid=20M=C4=99drek?= Date: Thu, 12 Dec 2024 17:25:28 +0100 Subject: [PATCH 262/397] Revert "main,cql_test_env: start group0_service before view_builder" The patch solved a problem related to an initialization order (scylladb/scylladb#20772), but we ran into another one: scylladb/scylladb#21534. After moving the initialization of group0_service, it ended up being destroyed AFTER the CDC generation service would. Since CDC generations are accessed in `storage_service::topology_state_load()`: ``` for (const auto& gen_id : _topology_state_machine._topology.committed_cdc_generations) { rtlogger.trace("topology_state_load: process committed cdc generation {}", gen_id); co_await _cdc_gens.local().handle_cdc_generation(gen_id); ``` we started getting the following failure: ``` Service &seastar::sharded::local() [Service = cdc::generation_service]: Assertion `local_is_initialized()' failed. ``` We're reverting the patch to go back to a more stable version of Scylla and in the following commit, we'll solve the original issue in a more systematic way. This reverts commit 7bad8378c723a700e531aa55dae87db035f5b64d. --- main.cc | 24 ++++++++++++------------ test/lib/cql_test_env.cc | 14 +++++++------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/main.cc b/main.cc index 8b73496bfcfd..7f0b2401447c 100644 --- a/main.cc +++ b/main.cc @@ -1787,18 +1787,6 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl sys_dist_ks.invoke_on_all(&db::system_distributed_keyspace::stop).get(); }); - group0_service.start().get(); - auto stop_group0_service = defer_verbose_shutdown("group 0 service", [&group0_service] { - sl_controller.local().abort_group0_operations(); - group0_service.abort().get(); - }); - - utils::get_local_injector().inject("stop_after_starting_group0_service", - [] { std::raise(SIGSTOP); }); - - // Set up group0 service earlier since it is needed by group0 setup just below - ss.local().set_group0(group0_service); - supervisor::notify("starting view update generator"); view_update_generator.start(std::ref(db), std::ref(proxy), std::ref(stop_signal.as_sharded_abort_source())).get(); auto stop_view_update_generator = defer_verbose_shutdown("view update generator", [] { @@ -2070,6 +2058,18 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl */ db.local().enable_autocompaction_toggle(); + group0_service.start().get(); + auto stop_group0_service = defer_verbose_shutdown("group 0 service", [&group0_service] { + sl_controller.local().abort_group0_operations(); + group0_service.abort().get(); + }); + + utils::get_local_injector().inject("stop_after_starting_group0_service", + [] { std::raise(SIGSTOP); }); + + // Set up group0 service earlier since it is needed by group0 setup just below + ss.local().set_group0(group0_service); + // Load address_map from system.peers and subscribe to gossiper events to keep it updated. ss.local().init_address_map(gossip_address_map.local()).get(); auto cancel_address_map_subscription = defer_verbose_shutdown("storage service uninit address map", [&ss] { diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index c1c906859ec4..8611f93f19af 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -926,13 +926,6 @@ class single_node_cql_env : public cql_test_env { _group0_registry.invoke_on_all(&service::raft_group_registry::drain_on_shutdown).get(); }); - group0_service.start().get(); - auto stop_group0_service = defer([&group0_service] { - group0_service.abort().get(); - }); - - _ss.local().set_group0(group0_service); - _view_update_generator.start(std::ref(_db), std::ref(_proxy), std::ref(abort_sources)).get(); _view_update_generator.invoke_on_all(&db::view::view_update_generator::start).get(); auto stop_view_update_generator = defer([this] { @@ -982,6 +975,13 @@ class single_node_cql_env : public cql_test_env { _cdc.stop().get(); }); + group0_service.start().get(); + auto stop_group0_service = defer([&group0_service] { + group0_service.abort().get(); + }); + + _ss.local().set_group0(group0_service); + // Load address_map from system.peers and subscribe to gossiper events to keep it updated. _ss.local().init_address_map(_gossip_address_map.local()).get(); auto cancel_address_map_subscription = defer([this] { From 06ce9763707522f39735f8acc14186309ca7e545 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dawid=20M=C4=99drek?= Date: Thu, 12 Dec 2024 17:30:32 +0100 Subject: [PATCH 263/397] main, view: Pair view builder drain with its start In these changes, we pair draining the view builder with its start. To better understand what was done and why, let's first look at the situation before this commit and the context of it: (a) The following things happened in order: 1. The view builder would be constructed. 2. Right after that, a deferred lambda would be created to stop the view builder during shutdown. 3. group0_service would be started. 4. A deferred lambda stopping group0_service would be created right after that. 5. The view builder would be started. (b) Because the view builder depends on group0_client, it couldn't be started before starting group0_service. On the other hand, other services depend on the view builder, e.g. the stream manager. That makes changing the order of initialization a difficult problem, so we want to avoid doing that unless we're sure it's the right choice. (c) Since the view builder uses group0_client, there was a possibility of running into a segmentation fault issue in the following scenario: 1. A call to `view_builder::mark_view_build_success()` is issued. 2. We stop group0_service. 3. `view_builder::mark_view_build_success()` calls `announce_with_raft()`, which leads to a use-after-free because group0_service has already been destroyed. This very scenario took place in scylladb/scylladb#20772. Initially, we decided to solve the issue by initializing group0_service a bit earlier (scylladb/scylladb@7bad8378c723a700e531aa55dae87db035f5b64d). Unfortunately, it led to other issues described in scylladb/scylladb#21534. We reverted that change in the previous commit. These changes are the second attempt to the problem where we want to solve it in a safer manner. The solution we came up with is to pair the start of the view builder with a deferred lambda that deinitializes it by calling `view_builder::drain()`. No other component of the system should be able to use the view builder anymore, so it's safe to do that. Furthermore, that pairing makes the analysis of initialization/deinitialization order much easier. We also solve the aformentioned use-after-free issue because the view builder itself will no longer attempt to use group0_client. Note that we still pair a deferred lambda calling `view_builder::stop()` with the construction of the view builder; that function will also call `view_builder::drain()`. Another notable thing is `view_builder::drain()` may be called earlier by `storage_service::do_drain()`. In other words, these changes cover the situation when Scylla runs into a problem when starting up. Fixes scylladb/scylladb#20772 --- main.cc | 3 +++ test/lib/cql_test_env.cc | 3 +++ 2 files changed, 6 insertions(+) diff --git a/main.cc b/main.cc index 7f0b2401447c..5f040ad85ba8 100644 --- a/main.cc +++ b/main.cc @@ -2249,6 +2249,9 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl if (cfg->view_building()) { view_builder.invoke_on_all(&db::view::view_builder::start, std::ref(mm), utils::cross_shard_barrier()).get(); } + auto drain_view_builder = defer_verbose_shutdown("draining view builders", [&] { + view_builder.invoke_on_all(&db::view::view_builder::drain).get(); + }); api::set_server_view_builder(ctx, view_builder).get(); auto stop_vb_api = defer_verbose_shutdown("view builder API", [&ctx] { diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index 8611f93f19af..51339c47bd71 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -1045,6 +1045,9 @@ class single_node_cql_env : public cql_test_env { _view_builder.invoke_on_all([this] (db::view::view_builder& vb) { return vb.start(_mm.local()); }).get(); + auto drain_view_builder = defer([this] { + _view_builder.invoke_on_all(&db::view::view_builder::drain).get(); + }); // Create the testing user. try { From d1f960eee257bcf0c59be8f6f1dfc0ac615544a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dawid=20M=C4=99drek?= Date: Mon, 13 Jan 2025 00:30:03 +0100 Subject: [PATCH 264/397] test/topology_custom: Add test for Scylla with disabled view building Before this commit, there doesn't seem to have been a test verifying that starting and shutting down Scylla behave correctly when the configuration option `view_building` is set to false. In these changes, we add one. --- test/topology_custom/test_mv_building.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/test/topology_custom/test_mv_building.py b/test/topology_custom/test_mv_building.py index 9ee3bc2e31c0..65c41dd866bc 100644 --- a/test/topology_custom/test_mv_building.py +++ b/test/topology_custom/test_mv_building.py @@ -42,3 +42,15 @@ async def test_view_building_scheduling_group(manager: ManagerClient): ratio = ms_statement / ms_streaming print(f"ms_streaming: {ms_streaming}, ms_statement: {ms_statement}, ratio: {ratio}") assert ratio < 0.1 + +# A sanity check test ensures that starting and shutting down Scylla when view building is +# disabled is conducted properly and we don't run into any issues. +@pytest.mark.asyncio +async def test_start_scylla_with_view_building_disabled(manager: ManagerClient): + server = await manager.server_add(config={"view_building": "false"}) + await manager.server_stop_gracefully(server_id=server.server_id) + + # Make sure there have been no errors. + log = await manager.server_open_log(server.server_id) + res = await log.grep(r"ERROR.*\[shard [0-9]+:[a-z]+\]") + assert len(res) == 0 From d815d7013cb7c74dfb265f3fe920a3c5b2226646 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 9 Dec 2024 18:08:37 +0800 Subject: [PATCH 265/397] sstables_loader: report progress with the unit of batch We restore a snapshot of table by streaming the sstables of the given snapshot of the table using `sstable_streamer::stream_sstable_mutations()` in batches. This function reads mutations from a set of sstables, and streams them to the target nodes. Due to the limit of this function, we are not able to track the progress in bytes. Previously, progress tracking used individual sstables as units, which caused inaccuracies with tablet-distributed tables, where: - An sstable spanning multiple tablets could be counted multiple times - Progress reporting could become misleading (e.g., showing "40" progress for a table with 10 sstables) This change introduces a more robust progress tracking method: - Use "batch" as the unit of progress instead of individual sstables. Each batch represents a tablet when restoring a table snapshot if the tablet being restored is distributed with tablets. When it comes to tables distributed with vnode, each batch represents an sstable. - Stream sstables for each tablet separately, handling both partially and fully contained sstables - Calculate progress based on the total number of sstables being streamed - Skip tablet IDs with no owned tokens For vnode-distributed tables, the number of "batches" directly corresponds to the number of sstables, ensuring: - Consistent progress reporting across different table distribution models - Simplified implementation - Accurate representation of restore progress The new approach provides a more reliable and uniform method of tracking restoration progress across different table distribution strategies. Also, Corrected the use of `_sstables.size()` in `sstable_streamer::stream_sstables()`. It addressed a review comment from Pavel that was inadvertently overlooked during previous rebasing the commit of 5ab4932f34. Fixes scylladb/scylladb#21816 Signed-off-by: Kefu Chai Closes scylladb/scylladb#21841 --- sstables_loader.cc | 141 ++++++++++++++++++++++++------- sstables_loader.hh | 20 ++++- test/object_store/test_backup.py | 6 +- 3 files changed, 134 insertions(+), 33 deletions(-) diff --git a/sstables_loader.cc b/sstables_loader.cc index 23761ca3c9b5..7c7aac6072ee 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -8,6 +8,7 @@ #include #include +#include #include #include #include @@ -162,12 +163,12 @@ class sstable_streamer { virtual ~sstable_streamer() {} - virtual future<> stream(std::function on_streamed); + virtual future<> stream(shared_ptr progress); host_id_vector_replica_set get_endpoints(const dht::token& token) const; future<> stream_sstable_mutations(streaming::plan_id, const dht::partition_range&, std::vector); protected: virtual host_id_vector_replica_set get_primary_endpoints(const dht::token& token) const; - future<> stream_sstables(const dht::partition_range&, std::vector, std::function on_streamed); + future<> stream_sstables(const dht::partition_range&, std::vector, shared_ptr progress); private: host_id_vector_replica_set get_all_endpoints(const dht::token& token) const; }; @@ -180,7 +181,7 @@ class tablet_sstable_streamer : public sstable_streamer { , _tablet_map(_erm->get_token_metadata().tablets().get_tablet_map(table_id)) { } - virtual future<> stream(std::function on_streamed) override; + virtual future<> stream(shared_ptr on_streamed) override; virtual host_id_vector_replica_set get_primary_endpoints(const dht::token& token) const override; private: @@ -193,9 +194,9 @@ class tablet_sstable_streamer : public sstable_streamer { return result; } - future<> stream_fully_contained_sstables(const dht::partition_range& pr, std::vector sstables, std::function on_streamed) { + future<> stream_fully_contained_sstables(const dht::partition_range& pr, std::vector sstables, shared_ptr progress) { // FIXME: fully contained sstables can be optimized. - return stream_sstables(pr, std::move(sstables), std::move(on_streamed)); + return stream_sstables(pr, std::move(sstables), std::move(progress)); } bool tablet_in_scope(locator::tablet_id) const; @@ -238,10 +239,13 @@ host_id_vector_replica_set tablet_sstable_streamer::get_primary_endpoints(const return to_replica_set(replicas); } -future<> sstable_streamer::stream(std::function on_streamed) { +future<> sstable_streamer::stream(shared_ptr progress) { + if (progress) { + progress->start(_sstables.size()); + } const auto full_partition_range = dht::partition_range::make_open_ended_both_sides(); - co_await stream_sstables(full_partition_range, std::move(_sstables), std::move(on_streamed)); + co_await stream_sstables(full_partition_range, std::move(_sstables), std::move(progress)); } bool tablet_sstable_streamer::tablet_in_scope(locator::tablet_id tid) const { @@ -274,7 +278,64 @@ bool tablet_sstable_streamer::tablet_in_scope(locator::tablet_id tid) const { return false; } -future<> tablet_sstable_streamer::stream(std::function on_streamed) { +// The tablet_sstable_streamer implements a hierarchical streaming strategy: +// +// 1. Top Level (Per-Tablet Streaming): +// - Unlike vnode streaming, this streams sstables on a tablet-by-tablet basis +// - For a table with M tablets, each tablet[i] maps to its own set of SSTable files +// stored in tablet_to_sstables[i] +// - If tablet_to_sstables[i] is empty, that tablet's streaming is considered complete +// - Progress tracking advances by 1.0 unit when an entire tablet completes streaming +// +// 2. Inner Level (Per-SSTable Streaming): +// - Within each tablet's batch, individual SSTables are streamed in smaller sub-batches +// - The per_tablet_stream_progress class tracks streaming progress at this level: +// - Updates when a set of SSTables completes streaming +// - For n completed SSTables, advances by (n / total_sstables_in_current_tablet) +// - Provides granular tracking for the inner level streaming operations +// - Helps estimate completion time for the current tablet's batch +// +// Progress Tracking: +// The streaming progress is monitored at two granularity levels: +// - Tablet level: Overall progress where each tablet contributes 1.0 units +// - SSTable level: Progress of individual SSTable transfers within a tablet, +// managed by the per_tablet_stream_progress class +// +// Note: For simplicity, we assume uniform streaming time across tablets, even though +// tablets may vary significantly in their SSTable count or size. This assumption +// helps in progress estimation without requiring prior knowledge of SSTable +// distribution across tablets. +struct per_tablet_stream_progress : public stream_progress { +private: + shared_ptr _per_table_progress; + const size_t _num_sstables_mapped; +public: + per_tablet_stream_progress(shared_ptr per_table_progress, + size_t num_sstables_mapped) + : _per_table_progress(std::move(per_table_progress)) + , _num_sstables_mapped(num_sstables_mapped) { + if (_per_table_progress && _num_sstables_mapped == 0) { + // consider this tablet completed if nothing to stream + _per_table_progress->advance(1.0); + } + } + void advance(float num_sstable_streamed) override { + // we should not move backward + assert(num_sstable_streamed >= 0.); + // we should call advance() only if the current tablet maps to at least + // one sstable. + assert(_num_sstables_mapped > 0); + if (_per_table_progress) { + _per_table_progress->advance(num_sstable_streamed / _num_sstables_mapped); + } + } +}; + +future<> tablet_sstable_streamer::stream(shared_ptr progress) { + if (progress) { + progress->start(_tablet_map.tablet_count()); + } + // sstables are sorted by first key in reverse order. auto sstable_it = _sstables.rbegin(); @@ -313,14 +374,17 @@ future<> tablet_sstable_streamer::stream(std::function on_stream co_await coroutine::maybe_yield(); } + auto per_tablet_progress = make_shared( + progress, + sstables_fully_contained.size() + sstables_partially_contained.size()); auto tablet_pr = dht::to_partition_range(tablet_range); - co_await stream_sstables(tablet_pr, std::move(sstables_partially_contained), on_streamed); - co_await stream_fully_contained_sstables(tablet_pr, std::move(sstables_fully_contained), on_streamed); + co_await stream_sstables(tablet_pr, std::move(sstables_partially_contained), per_tablet_progress); + co_await stream_fully_contained_sstables(tablet_pr, std::move(sstables_fully_contained), per_tablet_progress); } } -future<> sstable_streamer::stream_sstables(const dht::partition_range& pr, std::vector sstables, std::function on_streamed) { - size_t nr_sst_total = _sstables.size(); +future<> sstable_streamer::stream_sstables(const dht::partition_range& pr, std::vector sstables, shared_ptr progress) { + size_t nr_sst_total = sstables.size(); size_t nr_sst_current = 0; while (!sstables.empty()) { @@ -337,8 +401,8 @@ future<> sstable_streamer::stream_sstables(const dht::partition_range& pr, std:: fmt::join(sst_processed | boost::adaptors::transformed([] (auto sst) { return sst->get_filename(); }), ", ")); nr_sst_current += sst_processed.size(); co_await stream_sstable_mutations(ops_uuid, pr, std::move(sst_processed)); - if (on_streamed) { - std::invoke(on_streamed, batch_sst_nr); + if (progress) { + progress->advance(batch_sst_nr); } } } @@ -450,14 +514,14 @@ static std::unique_ptr make_sstable_streamer(bool uses_tablets future<> sstables_loader::load_and_stream(sstring ks_name, sstring cf_name, ::table_id table_id, std::vector sstables, bool primary, bool unlink, stream_scope scope, - std::function on_streamed) { + shared_ptr progress) { // streamer guarantees topology stability, for correctness, by holding effective_replication_map // throughout its lifetime. auto streamer = make_sstable_streamer(_db.local().find_column_family(table_id).uses_tablets(), _messaging, _db.local(), table_id, std::move(sstables), primary_replica_only(primary), unlink_sstables(unlink), scope); - co_await streamer->stream(on_streamed); + co_await streamer->stream(progress); } // For more details, see distributed_loader::process_upload_dir(). @@ -519,7 +583,7 @@ class sstables_loader::download_task_impl : public tasks::task_manager::task::im sstring _prefix; sstables_loader::stream_scope _scope; std::vector _sstables; - std::vector _num_sstables_processed; + std::vector> _progress_per_shard; protected: virtual future<> run() override; @@ -537,9 +601,9 @@ class sstables_loader::download_task_impl : public tasks::task_manager::task::im , _prefix(std::move(prefix)) , _scope(scope) , _sstables(std::move(sstables)) - , _num_sstables_processed(smp::count) + , _progress_per_shard(smp::count) { - _status.progress_units = "sstables"; + _status.progress_units = "batches"; } virtual std::string type() const override { @@ -559,13 +623,32 @@ class sstables_loader::download_task_impl : public tasks::task_manager::task::im } virtual future get_progress() const override { - llog.debug("get_progress: {}", _num_sstables_processed); - unsigned processed = co_await _loader.map_reduce(adder(), [this] (auto&) { - return _num_sstables_processed[this_shard_id()]; - }); + struct adder { + stream_progress result; + future<> operator()(stream_progress p) { + llog.debug("get_progress: {} / {}", p.completed, p.total); + result.completed += p.completed; + result.total += p.total; + return make_ready_future<>(); + } + stream_progress get() const { + return result; + } + }; + auto p = co_await _loader.map_reduce( + adder{}, + [this] (auto&) -> stream_progress { + auto p = _progress_per_shard[this_shard_id()]; + if (p) { + return *p; + } else { + // the task was aborted + return {}; + } + }); co_return tasks::task_manager::task::progress { - .completed = processed, - .total = _sstables.size(), + .completed = p.completed, + .total = p.total, }; } }; @@ -597,11 +680,11 @@ future<> sstables_loader::download_task_impl::run() { } catch (...) { } }); - co_await _loader.invoke_on_all([this, &sstables_on_shards, table_id] (sstables_loader& loader) mutable -> future<> { - co_await loader.load_and_stream(_ks, _cf, table_id, std::move(sstables_on_shards[this_shard_id()]), false, false, _scope, [this] (unsigned num_streamed) { - _num_sstables_processed[this_shard_id()] += num_streamed; - }); + auto progress = make_shared(); + _progress_per_shard[this_shard_id()] = progress; + co_await loader.load_and_stream(_ks, _cf, table_id, std::move(sstables_on_shards[this_shard_id()]), false, false, _scope, + progress); }); } catch (...) { ex = std::current_exception(); diff --git a/sstables_loader.hh b/sstables_loader.hh index b11fde2f1ad3..7475b09d2e6c 100644 --- a/sstables_loader.hh +++ b/sstables_loader.hh @@ -29,6 +29,24 @@ class view_builder; } } +struct stream_progress { + float total = 0.; + float completed = 0.; + + virtual ~stream_progress() = default; + + void start(float amount) { + assert(amount >= 0); + total = amount; + } + virtual void advance(float amount) { + // we should not move backward + assert(amount >= 0); + completed += amount; + assert(completed <= total); + } +}; + // The handler of the 'storage_service/load_new_ss_tables' endpoint which, in // turn, is the target of the 'nodetool refresh' command. // Gets sstables from the upload directory and makes them available in the @@ -60,7 +78,7 @@ private: future<> load_and_stream(sstring ks_name, sstring cf_name, table_id, std::vector sstables, bool primary_replica_only, bool unlink_sstables, stream_scope scope, - std::function on_streamed); + shared_ptr progress); public: sstables_loader(sharded& db, diff --git a/test/object_store/test_backup.py b/test/object_store/test_backup.py index 6dcd91a6113b..57547f5d1bdd 100644 --- a/test/object_store/test_backup.py +++ b/test/object_store/test_backup.py @@ -285,9 +285,9 @@ def list_sstables(): if not do_abort: assert status is not None assert status['state'] == 'done' - assert status['progress_units'] == "sstables" - assert status['progress_completed'] == len(toc_names) - assert status['progress_total'] == len(toc_names) + assert status['progress_units'] == 'batches' + assert status['progress_completed'] == status['progress_total'] + assert status['progress_completed'] > 0 print('Check that sstables came back') files = list_sstables() From fbca0a08f78daf601e8ae135e6a9e8e066974825 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Sat, 11 Jan 2025 17:02:31 +0800 Subject: [PATCH 266/397] build: cmake: do not add absl::headers as a link directory In 0b0e661a85, we brought abseil back as a submodule, and we added absl::headers as an interface library for importing abseil headers' include directory. And: ```console $ patchelf --print-rpath build/RelWithDebInfo/scylla /home/kefu/dev/scylla/idl/absl::headers ``` In this change, we remove `absl::headers` from `target_link_directories()` as it's an interface library that only provides header files, not linkable libraries. This fixes the incorrect inclusion of absl::headers in the rpath of the scylla executable. Additionally, remove abseil library dependencies from the idl target since none of the idl source files directly include abseil headers. After this change, ```console $ patchelf --print-rpath build/RelWithDebInfo/scylla ``` the output of `pathelf` is now empty. Fixes #22265 Signed-off-by: Kefu Chai Closes scylladb/scylladb#22266 --- idl/CMakeLists.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/idl/CMakeLists.txt b/idl/CMakeLists.txt index 91672c5abf84..3caffe6ca012 100644 --- a/idl/CMakeLists.txt +++ b/idl/CMakeLists.txt @@ -77,9 +77,6 @@ add_custom_target(idl-sources DEPENDS ${idl_sources}) add_library(idl INTERFACE) add_dependencies(idl idl-sources) -target_link_directories(idl - INTERFACE - absl::headers) target_include_directories(idl INTERFACE ${scylla_gen_build_dir}) From 41de3a17e195e61975501d9ff855c083c7b923bf Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Fri, 20 Dec 2024 14:25:04 +0800 Subject: [PATCH 267/397] api: move histogram data into future to avoid deep copying Previously, we created a vector and returned it by copying into a future. Since histogram is a JSON representation of ihistogram, it can be heavyweight, making the vector copy overhead significant. Now we move the vector into the returned future instead of copying it, eliminating the deep copy overhead. The APIs backed by this function are marked deprecated, so this performance improvement is not that important. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22004 --- api/column_family.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/column_family.cc b/api/column_family.cc index 5ffccd3f04d7..8f86ec23e13f 100644 --- a/api/column_family.cc +++ b/api/column_family.cc @@ -202,7 +202,7 @@ static future get_cf_histogram(http_context& ctx, utils: return ctx.db.map(fun).then([](const std::vector &res) { std::vector r; std::ranges::copy(res | std::views::transform(to_json), std::back_inserter(r)); - return make_ready_future(r); + return make_ready_future(std::move(r)); }); } From 752e6561fb100db0039566708f65329e02cd21a6 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 16 Dec 2024 16:25:08 +0800 Subject: [PATCH 268/397] test/pylib: log if scylla exits with non-zero status code When destroying a test cluster, ScyllaCluster.stop() calls ScyllaServer.stop() for each running server. Previously, non-zero exit status codes from scylla servers were silently ignored during test teardown. This change modifies the logging behavior to print the exit status code when a scylla server exits with a non-zero status. This helps developers quickly identify potential issues or unexpected terminations during test runs. Differences in handling: - Before: Non-zero exit codes were not logged - After: Non-zero exit codes are printed, providing visibility into server termination errors This improvement aids in diagnosing intermittent test failures or unexpected server shutdowns during test execution. Refs #21742 Signed-off-by: Kefu Chai Closes scylladb/scylladb#21934 --- test/pylib/scylla_cluster.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 5e8d718e9e58..c8da0256f1de 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -708,9 +708,21 @@ async def stop(self) -> None: # this step to fail unconditionally. pass await self.shutdown_control_connection() + + if self.cmd.returncode is not None: + # process has already exited + if self.cmd.returncode != 0: + self.logger.error("%s exited with non-zero status code: %d", self, self.cmd.returncode) + self.logger.info("stopped %s in %s", self, self.workdir.name) + self.cmd = None + return + try: self.cmd.kill() except ProcessLookupError: + # the process *might* exit after checking for self.cmd.returncode + # and before self.cmd.kill() call. this is unlikely, but should not + # be considered as a failure. pass else: await self.cmd.wait() From a21ecc32532dff7fd2bfdd43d5f4d018f8d06afe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Wed, 8 Jan 2025 02:33:53 -0500 Subject: [PATCH 269/397] tools/scylla-sstable: also try reading scylla.yaml from /etc/scylla scylla-sstable tries to read scylla.yaml via the following sequence: 1) Use user-provided location is provided (--scylla-yaml-file parameter) 2) Use the environment variables SCYLLA_HOME and/or SCYLLA_CONF if set 3) Use the default location ./conf/scylla.yaml Step 3 is fine on dev machines, where the binaries are usually invoked from scylla.git, which does have conf/scylla.yaml, but it doesn't work on production machines, where the default location for scylla.yaml is /etc/scylla/scylla.yaml. To reduce friction when used on production machines, add another fallback in case (3) fails, which tries to read scylla.yaml from /etc/scylla/scylla.yaml location. Fixes: scylladb/scylladb#22202 Closes scylladb/scylladb#22241 --- tools/scylla-sstable.cc | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/tools/scylla-sstable.cc b/tools/scylla-sstable.cc index 0595cc7b54f1..518f9119fca4 100644 --- a/tools/scylla-sstable.cc +++ b/tools/scylla-sstable.cc @@ -3171,9 +3171,19 @@ Validate the specified sstables: if (app_config.count("scylla-yaml-file")) { scylla_yaml_path = app_config["scylla-yaml-file"].as(); scylla_yaml_path_source = "user provided"; + } else if (std::getenv("SCYLLA_CONF") || std::getenv("SCYLLA_HOME")) { + scylla_yaml_path = db::config::get_conf_sub("scylla.yaml").string(); + scylla_yaml_path_source = "environment provided"; } else { scylla_yaml_path = db::config::get_conf_sub("scylla.yaml").string(); - scylla_yaml_path_source = "default"; + scylla_yaml_path_source = "dev default"; + + // On production machines, the default of ./conf/scylla.yaml will not + // work, try /etc/scylla/scylla.yaml instead. + if (!file_exists(scylla_yaml_path).get()) { + scylla_yaml_path = "/etc/scylla/scylla.yaml"; + scylla_yaml_path_source = "prod default"; + } } if (file_exists(scylla_yaml_path).get()) { From f899f0e411fa9df38f8f9ff7ffd73d906d0a6432 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Wed, 8 Jan 2025 03:17:15 -0500 Subject: [PATCH 270/397] tools/scylla-sstable: dump-statistics: fix handling of {min,max}_column_names MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Said fields in statistics are of type `disk_array>` and currently are handled as array of regular strings. However these fields store exploded clustering keys, so the elements store binary data and converting to string can yield invalid UTF-8 characters that certain JSON parsers (jq, or python's json) can choke on. Fix this by treating them as binary and using `to_hex()` to convert them to string. This requires some massaging of the json_dumper: passing field offset to all visit() methods and using a caller-provided disk-string to sstring converter to convert disk strings to sstring, so in the case of statistics, these fields can be intercepted and properly handled. While at it, the type of these fields is also fixed in the documentation. Before: "min_column_names": [ "��Z���\u0011�\u0012ŷ4^��<", "�2y\u0000�}\u007f" ], "max_column_names": [ "��Z���\u0011�\u0012ŷ4^��<", "}��B\u0019l%^" ], After: "min_column_names": [ "9dd55a92bc8811ef12c5b7345eadf73c", "80327900e2827d7f" ], "max_column_names": [ "9dd55a92bc8811ef12c5b7345eadf73c", "7df79242196c255e" ], Fixes: #22078 Closes scylladb/scylladb#22225 --- .../admin-tools/scylla-sstable.rst | 4 +- tools/scylla-sstable.cc | 89 +++++++++++-------- 2 files changed, 55 insertions(+), 38 deletions(-) diff --git a/docs/operating-scylla/admin-tools/scylla-sstable.rst b/docs/operating-scylla/admin-tools/scylla-sstable.rst index e60c5d27fce8..4f1b6858429a 100644 --- a/docs/operating-scylla/admin-tools/scylla-sstable.rst +++ b/docs/operating-scylla/admin-tools/scylla-sstable.rst @@ -450,8 +450,8 @@ The content is dumped in JSON, using the following schema: "estimated_tombstone_drop_time": $STREAMING_HISTOGRAM, "sstable_level": Uint, "repaired_at": Uint64, - "min_column_names": [Uint, ...], - "max_column_names": [Uint, ...], + "min_column_names": [String, ...], + "max_column_names": [String, ...], "has_legacy_counter_shards": Bool, "columns_count": Int64, // >= MC only "rows_count": Int64, // >= MC only diff --git a/tools/scylla-sstable.cc b/tools/scylla-sstable.cc index 518f9119fca4..1b8d498b2be0 100644 --- a/tools/scylla-sstable.cc +++ b/tools/scylla-sstable.cc @@ -1186,15 +1186,21 @@ class json_dumper { json_writer& _writer; sstables::sstable_version_types _version; std::function _name_resolver; + std::function _disk_string_converter; + +public: + static sstring default_disk_string_converter(const void* const, bytes_view value) { + return sstring(value.begin(), value.end()); + } private: - void visit(int8_t val) { _writer.Int(val); } - void visit(uint8_t val) { _writer.Uint(val); } - void visit(int val) { _writer.Int(val); } - void visit(unsigned val) { _writer.Uint(val); } - void visit(int64_t val) { _writer.Int64(val); } - void visit(uint64_t val) { _writer.Uint64(val); } - void visit(double val) { + void visit(const void* const field, int8_t val) { _writer.Int(val); } + void visit(const void* const field, uint8_t val) { _writer.Uint(val); } + void visit(const void* const field, int val) { _writer.Int(val); } + void visit(const void* const field, unsigned val) { _writer.Uint(val); } + void visit(const void* const field, int64_t val) { _writer.Int64(val); } + void visit(const void* const field, uint64_t val) { _writer.Uint64(val); } + void visit(const void* const field, double val) { if (std::isnan(val)) { _writer.String("NaN"); } else { @@ -1203,42 +1209,42 @@ class json_dumper { } template - void visit(const sstables::disk_string& val) { - _writer.String(disk_string_to_string(val)); + void visit(const void* const field, const sstables::disk_string& val) { + _writer.String(_disk_string_converter(field, val.value)); } template - void visit(const std::optional& val) { + void visit(const void* const field, const std::optional& val) { if (bool(val)) { - visit(*val); + visit(field, *val); } else { _writer.Null(); } } template - void visit(const sstables::disk_array& val) { + void visit(const void* const field, const sstables::disk_array& val) { _writer.StartArray(); for (const auto& elem : val.elements) { - visit(elem); + visit(field, elem); } _writer.EndArray(); } - void visit(const sstables::disk_string_vint_size& val) { - _writer.String(sstring(val.value.begin(), val.value.end())); + void visit(const void* const field, const sstables::disk_string_vint_size& val) { + _writer.String(_disk_string_converter(field, val.value)); } template - void visit(const sstables::disk_array_vint_size& val) { + void visit(const void* const field, const sstables::disk_array_vint_size& val) { _writer.StartArray(); for (const auto& elem : val.elements) { - visit(elem); + visit(field, elem); } _writer.EndArray(); } - void visit(const utils::estimated_histogram& val) { + void visit(const void* const field, const utils::estimated_histogram& val) { _writer.StartArray(); for (size_t i = 0; i < val.buckets.size(); i++) { _writer.StartObject(); @@ -1251,7 +1257,7 @@ class json_dumper { _writer.EndArray(); } - void visit(const utils::streaming_histogram& val) { + void visit(const void* const field, const utils::streaming_histogram& val) { _writer.StartObject(); for (const auto& [k, v] : val.bin) { _writer.Key(format("{}", k)); @@ -1260,7 +1266,7 @@ class json_dumper { _writer.EndObject(); } - void visit(const db::replay_position& val) { + void visit(const void* const field, const db::replay_position& val) { _writer.StartObject(); _writer.Key("id"); _writer.Uint64(val.id); @@ -1269,30 +1275,30 @@ class json_dumper { _writer.EndObject(); } - void visit(const sstables::commitlog_interval& val) { + void visit(const void* const field, const sstables::commitlog_interval& val) { _writer.StartObject(); _writer.Key("start"); - visit(val.start); + visit(field, val.start); _writer.Key("end"); - visit(val.end); + visit(field, val.end); _writer.EndObject(); } - void visit(const utils::UUID& uuid) { + void visit(const void* const field, const utils::UUID& uuid) { _writer.String(fmt::to_string(uuid)); } template - void visit(const utils::tagged_uuid& id) { - visit(id.uuid()); + void visit(const void* const field, const utils::tagged_uuid& id) { + visit(field, id.uuid()); } template - void visit(const sstables::vint& val) { - visit(val.value); + void visit(const void* const field, const sstables::vint& val) { + visit(field, val.value); } - void visit(const sstables::serialization_header::column_desc& val) { + void visit(const void* const field, const sstables::serialization_header::column_desc& val) { auto prev_name_resolver = std::exchange(_name_resolver, [&val] (const void* const field) { if (field == &val.name) { return "name"; } else if (field == &val.type_name) { return "type_name"; } @@ -1306,28 +1312,34 @@ class json_dumper { _name_resolver = std::move(prev_name_resolver); } - json_dumper(json_writer& writer, sstables::sstable_version_types version, std::function name_resolver) - : _writer(writer), _version(version), _name_resolver(std::move(name_resolver)) { + json_dumper(json_writer& writer, sstables::sstable_version_types version, std::function name_resolver, + std::function disk_string_converter) + : _writer(writer), _version(version), _name_resolver(std::move(name_resolver)), _disk_string_converter(std::move(disk_string_converter)) { } public: template void operator()(Arg1& arg1) { _writer.Key(_name_resolver(&arg1)); - visit(arg1); + visit(&arg1, arg1); } template void operator()(Arg1& arg1, Arg&... arg) { _writer.Key(_name_resolver(&arg1)); - visit(arg1); + visit(&arg1, arg1); (*this)(arg...); } template - static void dump(json_writer& writer, sstables::sstable_version_types version, const T& obj, std::string_view name, - std::function name_resolver) { - json_dumper dumper(writer, version, std::move(name_resolver)); + static void dump( + json_writer& writer, + sstables::sstable_version_types version, + const T& obj, + std::string_view name, + std::function name_resolver, + std::function disk_string_converter = &json_dumper::default_disk_string_converter) { + json_dumper dumper(writer, version, std::move(name_resolver), std::move(disk_string_converter)); writer.Key(name); writer.StartObject(); const_cast(obj).describe_type(version, std::ref(dumper)); @@ -1375,6 +1387,11 @@ void dump_stats_metadata(json_writer& writer, sstables::sstable_version_types ve else if (field == &metadata.commitlog_intervals) { return "commitlog_intervals"; } else if (field == &metadata.originating_host_id) { return "originating_host_id"; } else { throw std::invalid_argument("invalid field offset"); } + }, [&metadata] (const void* const field, bytes_view value) { + if (field == &metadata.min_column_names || field == &metadata.max_column_names) { + return to_hex(value); + } + return json_dumper::default_disk_string_converter(field, value); }); } From 2aea2610e0e7c067230fc16b0afcaa6068d838d6 Mon Sep 17 00:00:00 2001 From: Andrei Chekun Date: Thu, 9 Jan 2025 15:42:14 +0100 Subject: [PATCH 271/397] test.py: Wait for tasks finish before going further Developers using asyncio.gather() often assume that it waits for all futures (awaitables) givens. But this isn't true when the return_exceptions parameter is False, which is the default. In that case, as soon as one future completes with an exception, the gather() call will return this exception immediately, and some of the finished tasks may continue to run in the background. This is bad for applications that use gather() to ensure that a list of background tasks has all completed. So such applications must use asyncio.gather() with return_exceptions=True, to wait for all given futures to complete either successfully or unsuccessfully. Closes scylladb/scylladb#22252 --- test/pylib/scylla_cluster.py | 31 ++++++++++++++++++++++++------- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index c8da0256f1de..67690a4285b0 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -20,7 +20,8 @@ import tempfile import time import traceback -from typing import Any, Optional, Dict, List, Set, Tuple, Callable, AsyncIterator, NamedTuple, Union, NoReturn +from typing import Any, Optional, Dict, List, Set, Tuple, Callable, AsyncIterator, NamedTuple, Union, NoReturn, \ + Awaitable import uuid from io import BufferedWriter from test.pylib.host_registry import Host, HostRegistry @@ -50,7 +51,6 @@ from cassandra.policies import WhiteListRoundRobinPolicy # type: ignore from cassandra.connection import UnixSocketEndPoint - io_executor = concurrent.futures.ThreadPoolExecutor(max_workers=20) async def async_rmtree(directory, *args, **kwargs): @@ -873,8 +873,8 @@ async def uninstall(self) -> None: self.is_dirty = True self.logger.info("Uninstalling cluster %s", self) await self.stop() - await asyncio.gather(*(srv.uninstall() for srv in self.stopped.values())) - await asyncio.gather(*(self.host_registry.release_host(Host(ip)) + await gather_safely(*(srv.uninstall() for srv in self.stopped.values())) + await gather_safely(*(self.host_registry.release_host(Host(ip)) for ip in self.leased_ips)) async def release_ips(self) -> None: @@ -897,7 +897,7 @@ async def stop(self) -> None: self.logger.info("Cluster %s stopping", self) self.is_dirty = True # If self.running is empty, no-op - await asyncio.gather(*(server.stop() for server in self.running.values())) + await gather_safely(*(server.stop() for server in self.running.values())) self.stopped.update(self.running) self.running.clear() @@ -908,7 +908,7 @@ async def stop_gracefully(self) -> None: self.logger.info("Cluster %s stopping gracefully", self) self.is_dirty = True # If self.running is empty, no-op - await asyncio.gather(*(server.stop_gracefully() for server in self.running.values())) + await gather_safely(*(server.stop_gracefully() for server in self.running.values())) self.stopped.update(self.running) self.running.clear() @@ -1024,7 +1024,7 @@ async def add_servers(self, servers_num: int = 1, """Add multiple servers to the cluster concurrently""" assert servers_num > 0, f"add_servers: cannot add {servers_num} servers" - return await asyncio.gather(*(self.add_server(None, cmdline, config, property_file, start, seeds, server_encryption, expected_error) + return await gather_safely(*(self.add_server(None, cmdline, config, property_file, start, seeds, server_encryption, expected_error) for _ in range(servers_num))) def endpoint(self) -> str: @@ -1780,3 +1780,20 @@ async def get_cluster_manager(test_uname: str, clusters: Pool[ScyllaCluster], te yield manager finally: await manager.stop() + + +async def gather_safely(*awaitables: Awaitable): + """ + Developers using asyncio.gather() often assume that it waits for all futures (awaitables) givens. + But this isn't true when the return_exceptions parameter is False, which is the default. + In that case, as soon as one future completes with an exception, the gather() call will return this exception + immediately, and some of the finished tasks may continue to run in the background. + This is bad for applications that use gather() to ensure that a list of background tasks has all completed. + So such applications must use asyncio.gather() with return_exceptions=True, to wait for all given futures to + complete either successfully or unsuccessfully. + """ + results = await asyncio.gather(*awaitables, return_exceptions=True) + for result in results: + if isinstance(result, BaseException): + raise result from None + return results From 6e3ecc70a6d5305f232e5fd98b878d7719dc51de Mon Sep 17 00:00:00 2001 From: Lakshmi Narayanan Sreethar Date: Fri, 10 Jan 2025 19:10:45 +0530 Subject: [PATCH 272/397] sstable_directory: update `load_sstable()` definition Updated `sstable_directory::load_sstable()` to directly accept `data_dictionary::storage_options` instead of a function that returns the same. This is required to ensure `process_descriptor()` loads the sstable only once in the right shard. Signed-off-by: Lakshmi Narayanan Sreethar --- sstables/sstable_directory.cc | 11 ++++------- sstables/sstable_directory.hh | 3 +-- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/sstables/sstable_directory.cc b/sstables/sstable_directory.cc index 5915d77f7571..1eac1606d748 100644 --- a/sstables/sstable_directory.cc +++ b/sstables/sstable_directory.cc @@ -225,11 +225,8 @@ void sstable_directory::validate(sstables::shared_sstable sst, process_flags fla } future sstable_directory::load_sstable(sstables::entry_descriptor desc, - noncopyable_function&& get_storage_options, - sstables::sstable_open_config cfg) const { - shared_sstable sst; - auto storage_opts = std::invoke(get_storage_options); - sst = _manager.make_sstable(_schema, storage_opts, desc.generation, _state, desc.version, desc.format, gc_clock::now(), _error_handler_gen); + const data_dictionary::storage_options& storage_opts, sstables::sstable_open_config cfg) const { + shared_sstable sst = _manager.make_sstable(_schema, storage_opts, desc.generation, _state, desc.version, desc.format, gc_clock::now(), _error_handler_gen); co_await sst->load(_sharder, cfg); co_return sst; } @@ -242,7 +239,7 @@ sstable_directory::process_descriptor(sstables::entry_descriptor desc, _max_version_seen = desc.version; } - auto sst = co_await load_sstable(desc, std::move(get_storage_options), flags.sstable_open_config); + auto sst = co_await load_sstable(desc, get_storage_options(), flags.sstable_open_config); validate(sst, flags); if (flags.need_mutate_level) { @@ -514,7 +511,7 @@ future sstable_directory::load_foreign_sstable(foreign_sstable_o future<> sstable_directory::load_foreign_sstables(sstable_entry_descriptor_vector info_vec) { co_await _manager.dir_semaphore().parallel_for_each(info_vec, [this] (const sstables::entry_descriptor& info) { - return load_sstable(info, [this] { return *_storage_opts; }).then([this] (auto sst) { + return load_sstable(info, *_storage_opts).then([this] (auto sst) { _unshared_local_sstables.push_back(sst); return make_ready_future<>(); }); diff --git a/sstables/sstable_directory.hh b/sstables/sstable_directory.hh index 08496c6e0401..fbc3561429fa 100644 --- a/sstables/sstable_directory.hh +++ b/sstables/sstable_directory.hh @@ -197,8 +197,7 @@ private: noncopyable_function&& get_storage_options); void validate(sstables::shared_sstable sst, process_flags flags) const; future load_sstable(sstables::entry_descriptor desc, - noncopyable_function&& get_storage_options, - sstables::sstable_open_config cfg = {}) const; + const data_dictionary::storage_options& storage_opts, sstables::sstable_open_config cfg = {}) const; future<> load_foreign_sstables(sstable_entry_descriptor_vector info_vec); From 63100b34daff230a59274c30c4a1f57de14af4d8 Mon Sep 17 00:00:00 2001 From: Lakshmi Narayanan Sreethar Date: Wed, 8 Jan 2025 23:21:21 +0530 Subject: [PATCH 273/397] sstable_directory: do not load remote sstables in process_descriptor The sstable loader relied on the generation id to provide an efficient hint about the shard that owns an sstable. But, this hint was rendered ineffective with the introduction of UUID generation, as the shard id was no longer embedded in the generation id. This also became suboptimal with the introduction of tablets. Commit 0c77f77 addressed this issue by reading the minimum from disk to determine sstable ownership but this improvement was lost with commit 63f1969, which optimistically assumed that hints would work most of the time, which isn't true. This commit restores that change - shard id of a table is deduced by reading minially from disk and then the sstable is fully loaded only if it belongs to the local shard. This patch also adds a testcase to verify that the sstable are loaded only in their respective shards. Fixes #21015 Signed-off-by: Lakshmi Narayanan Sreethar --- sstables/sstable_directory.cc | 29 ++++++++++++++-------------- test/boost/sstable_directory_test.cc | 29 ++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 14 deletions(-) diff --git a/sstables/sstable_directory.cc b/sstables/sstable_directory.cc index 1eac1606d748..0d465ca47a1c 100644 --- a/sstables/sstable_directory.cc +++ b/sstables/sstable_directory.cc @@ -239,7 +239,15 @@ sstable_directory::process_descriptor(sstables::entry_descriptor desc, _max_version_seen = desc.version; } - auto sst = co_await load_sstable(desc, get_storage_options(), flags.sstable_open_config); + auto storage_opts = get_storage_options(); + auto shards = co_await get_shards_for_this_sstable(desc, storage_opts, flags); + if (flags.sort_sstables_according_to_owner && shards.size() == 1 && shards[0] != this_shard_id()) { + // identified a remote unshared sstable + _unshared_remote_sstables[shards[0]].push_back(std::move(desc)); + co_return; + } + + auto sst = co_await load_sstable(desc, storage_opts, flags.sstable_open_config); validate(sst, flags); if (flags.need_mutate_level) { @@ -247,24 +255,17 @@ sstable_directory::process_descriptor(sstables::entry_descriptor desc, co_await sst->mutate_sstable_level(0); } - if (!flags.sort_sstables_according_to_owner) { - dirlog.debug("Added {} to unsorted sstables list", sst->get_filename()); - _unsorted_sstables.push_back(std::move(sst)); - co_return; - } - - auto shards = sst->get_shards_for_this_sstable(); - if (shards.size() == 1) { - if (shards[0] == this_shard_id()) { + if (flags.sort_sstables_according_to_owner) { + if (shards.size() == 1) { dirlog.trace("{} identified as a local unshared SSTable", sst->get_filename()); _unshared_local_sstables.push_back(std::move(sst)); } else { - dirlog.trace("{} identified as a remote unshared SSTable, shard={}", sst->get_filename(), shards[0]); - _unshared_remote_sstables[shards[0]].push_back(std::move(desc)); + dirlog.trace("{} identified as a shared SSTable, shards={}", sst->get_filename(), shards); + _shared_sstable_info.push_back(co_await sst->get_open_info()); } } else { - dirlog.trace("{} identified as a shared SSTable, shards={}", sst->get_filename(), shards); - _shared_sstable_info.push_back(co_await sst->get_open_info()); + dirlog.debug("Added {} to unsorted sstables list", sst->get_filename()); + _unsorted_sstables.push_back(std::move(sst)); } } diff --git a/test/boost/sstable_directory_test.cc b/test/boost/sstable_directory_test.cc index 84881b07c5c7..39a6d946c4a5 100644 --- a/test/boost/sstable_directory_test.cc +++ b/test/boost/sstable_directory_test.cc @@ -389,6 +389,35 @@ SEASTAR_THREAD_TEST_CASE(sstable_directory_unshared_sstables_sanity_unmatched_ge }).get(); } +SEASTAR_THREAD_TEST_CASE(sstable_directory_foreign_sstable_should_not_load_locally) { + if (smp::count == 1) { + fmt::print("Skipping sstable_directory_shared_sstables_reshard_correctly, smp == 1\n"); + return; + } + sstables::test_env::do_with_sharded_async([] (sharded& env) { + auto sstables_opened_for_reading = [&env] () { + return env.map_reduce0([] (sstables::test_env& env) { + return sstables_stats::get_shard_stats().open_for_reading; + }, 0, [] (auto res, auto gen) {return res + gen;}).get(); + }; + for (shard_id i = 0; i < smp::count; ++i) { + env.invoke_on(i, [] (sstables::test_env& env) -> future<> { + co_return co_await seastar::async([&env] { + make_sstable_for_this_shard(std::bind(new_sstable, std::ref(env), generation_type((this_shard_id() + 1) % smp::count))); + }); + }).get(); + } + + auto sstables_open_before_process = sstables_opened_for_reading(); + with_sstable_directory(env, [](sharded& sstdir) { + distributed_loader_for_tests::process_sstable_dir(sstdir, { .throw_on_missing_toc = true }).get(); + }); + + // verify that all the sstables were loaded only once + BOOST_REQUIRE_EQUAL(sstables_opened_for_reading(), sstables_open_before_process + smp::count); + }).get(); +} + // Test that the sstable_dir object can keep the table alive against a drop SEASTAR_TEST_CASE(sstable_directory_test_table_lock_works) { return do_with_cql_env_thread([] (cql_test_env& e) { From c8ca2bd2125a401856f896f53bf7389a26c5a4d7 Mon Sep 17 00:00:00 2001 From: Geoff Montee Date: Mon, 6 Jan 2025 23:19:52 -0800 Subject: [PATCH 274/397] docs: operating-scylla/admin-tools/virtual-tables.rst: fix link to virtual tables Closes scylladb/scylladb#22198 --- docs/operating-scylla/admin-tools/virtual-tables.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/operating-scylla/admin-tools/virtual-tables.rst b/docs/operating-scylla/admin-tools/virtual-tables.rst index af7740b7c351..32e0647e6755 100644 --- a/docs/operating-scylla/admin-tools/virtual-tables.rst +++ b/docs/operating-scylla/admin-tools/virtual-tables.rst @@ -11,5 +11,5 @@ ScyllaDB supports: * Virtual tables for retrieving system-level information, such as the cluster status, version-related information, etc. The range of information they can expose partially overlaps with the information you can obtain via :doc:`nodetool ` (unlike ``nodetool``, virtual tables permit remote access over CQL). * The virtual table for querying and updating configuration over CQL (the ``system.config`` table). -See `Virtual tables in the system keyspace `_ for the list of available virtual tables. +See `Virtual tables in the system keyspace `_ for the list of available virtual tables. From cd96fb5a7812fa6980d62aabfffece0b05990606 Mon Sep 17 00:00:00 2001 From: Asias He Date: Thu, 19 Dec 2024 21:26:12 +0800 Subject: [PATCH 275/397] repair: Add repair_hosts_filter and repair_dcs_filter They will be useful for hosts and DCs selection for the repair scheduler. It is not implemented yet. Adding it earlier, so we do not need to change the system tabler later. Closes scylladb/scylladb#21985 --- locator/tablets.hh | 2 ++ replica/tablets.cc | 10 +++++++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/locator/tablets.hh b/locator/tablets.hh index f4f6fbeef08b..71bcb8f132a1 100644 --- a/locator/tablets.hh +++ b/locator/tablets.hh @@ -160,6 +160,8 @@ struct tablet_task_info { db_clock::time_point request_time; int64_t sched_nr = 0; db_clock::time_point sched_time; + sstring repair_hosts_filter; + sstring repair_dcs_filter; bool operator==(const tablet_task_info&) const = default; bool is_valid() const; bool is_user_repair_request() const; diff --git a/replica/tablets.cc b/replica/tablets.cc index b7d75451fbe2..31bf48c925af 100644 --- a/replica/tablets.cc +++ b/replica/tablets.cc @@ -33,8 +33,8 @@ static thread_local auto repair_scheduler_config_type = user_type_impl::get_inst "system", "repair_scheduler_config", {"auto_repair_enabled", "auto_repair_threshold"}, {boolean_type, long_type}, false); static thread_local auto tablet_task_info_type = user_type_impl::get_instance( - "system", "tablet_task_info", {"request_type", "tablet_task_id", "request_time", "sched_nr", "sched_time"}, - {utf8_type, uuid_type, timestamp_type, long_type, timestamp_type}, false); + "system", "tablet_task_info", {"request_type", "tablet_task_id", "request_time", "sched_nr", "sched_time", "repair_hosts_filter", "repair_dcs_filter"}, + {utf8_type, uuid_type, timestamp_type, long_type, timestamp_type, utf8_type, utf8_type}, false); static thread_local auto replica_type = tuple_type_impl::get_instance({uuid_type, int32_type}); static thread_local auto replica_set_type = list_type_impl::get_instance(replica_type, false); static thread_local auto tablet_info_type = tuple_type_impl::get_instance({long_type, long_type, replica_set_type}); @@ -97,7 +97,9 @@ data_value tablet_task_info_to_data_value(const locator::tablet_task_info& info) data_value(info.tablet_task_id.uuid()), data_value(info.request_time), data_value(info.sched_nr), - data_value(info.sched_time) + data_value(info.sched_time), + data_value(info.repair_hosts_filter), + data_value(info.repair_dcs_filter), }); return result; }; @@ -297,6 +299,8 @@ locator::tablet_task_info tablet_task_info_from_cell(const data_value& v) { value_cast(dv[2]), value_cast(dv[3]), value_cast(dv[4]), + value_cast(dv[5]), + value_cast(dv[6]), }; return result; } From b1be2d3c41b2ee9c3aaccf0b09ffd0a7bf4fc7cb Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Tue, 14 Jan 2025 11:42:31 +0200 Subject: [PATCH 276/397] view_builder: write status to tables before starting to build When adding a new view for building, first write the status to the system tables and then add the view building step that will start building it. Otherwise, if we start building it before the status is written to the table, it may happen that we complete building the view, write the SUCCESS status, and then overwrite it with the STARTED status. The view_build_status table will remain in incorrect state indicating the view building is not complete. Fixes scylladb/scylladb#20638 --- db/view/view.cc | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/db/view/view.cc b/db/view/view.cc index 03e654d2dfb6..5cec26ace083 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -2448,11 +2448,11 @@ view_builder::view_build_statuses(sstring keyspace, sstring view_name) const { future<> view_builder::add_new_view(view_ptr view, build_step& step) { vlogger.info0("Building view {}.{}, starting at token {}", view->ks_name(), view->cf_name(), step.current_token()); + if (this_shard_id() == 0) { + co_await mark_view_build_started(view->ks_name(), view->cf_name()); + } + co_await _sys_ks.register_view_for_building(view->ks_name(), view->cf_name(), step.current_token()); step.build_status.emplace(step.build_status.begin(), view_build_status{view, step.current_token(), std::nullopt}); - auto f = this_shard_id() == 0 ? mark_view_build_started(view->ks_name(), view->cf_name()) : make_ready_future<>(); - return when_all_succeed( - std::move(f), - _sys_ks.register_view_for_building(view->ks_name(), view->cf_name(), step.current_token())).discard_result(); } static future<> flush_base(lw_shared_ptr base, abort_source& as) { From 1104411f83c7a8fbeeeaffe1caf070e1cf4204fd Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Tue, 14 Jan 2025 11:08:06 +0200 Subject: [PATCH 277/397] view_builder: pass view name by value to write_view_build_status The function write_view_build_status takes two lambda functions and chooses which of them to run depending on the upgrade state. It might run both of them. The parameters ks_name and view_name should be passed by value instead of by reference because they are moved inside each lambda function. Otherwise, if both lambdas are run, the second call operates on invalid values that were moved. --- db/view/view.cc | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/db/view/view.cc b/db/view/view.cc index 5cec26ace083..4ccded2c6244 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -2356,7 +2356,7 @@ static future<> announce_with_raft( future<> view_builder::mark_view_build_started(sstring ks_name, sstring view_name) { co_await write_view_build_status( - [&] () -> future<> { + [this, ks_name, view_name] () -> future<> { co_await utils::get_local_injector().inject("view_builder_pause_add_new_view", utils::wait_for_message(5min)); const sstring query_string = format("INSERT INTO {}.{} (keyspace_name, view_name, host_id, status) VALUES (?, ?, ?, ?)", db::system_keyspace::NAME, db::system_keyspace::VIEW_BUILD_STATUS_V2); @@ -2365,7 +2365,7 @@ future<> view_builder::mark_view_build_started(sstring ks_name, sstring view_nam {std::move(ks_name), std::move(view_name), host_id.uuid(), "STARTED"}, "view builder: mark view build STARTED"); }, - [&] () -> future<> { + [this, ks_name, view_name] () -> future<> { co_await utils::get_local_injector().inject("view_builder_pause_add_new_view", utils::wait_for_message(5min)); co_await _sys_dist_ks.start_view_build(std::move(ks_name), std::move(view_name)); } @@ -2374,7 +2374,7 @@ future<> view_builder::mark_view_build_started(sstring ks_name, sstring view_nam future<> view_builder::mark_view_build_success(sstring ks_name, sstring view_name) { co_await write_view_build_status( - [&] () -> future<> { + [this, ks_name, view_name] () -> future<> { co_await utils::get_local_injector().inject("view_builder_pause_mark_success", utils::wait_for_message(5min)); const sstring query_string = format("UPDATE {}.{} SET status = ? WHERE keyspace_name = ? AND view_name = ? AND host_id = ?", db::system_keyspace::NAME, db::system_keyspace::VIEW_BUILD_STATUS_V2); @@ -2383,7 +2383,7 @@ future<> view_builder::mark_view_build_success(sstring ks_name, sstring view_nam {"SUCCESS", std::move(ks_name), std::move(view_name), host_id.uuid()}, "view builder: mark view build SUCCESS"); }, - [&] () -> future<> { + [this, ks_name, view_name] () -> future<> { co_await utils::get_local_injector().inject("view_builder_pause_mark_success", utils::wait_for_message(5min)); co_await _sys_dist_ks.finish_view_build(std::move(ks_name), std::move(view_name)); } @@ -2392,14 +2392,14 @@ future<> view_builder::mark_view_build_success(sstring ks_name, sstring view_nam future<> view_builder::remove_view_build_status(sstring ks_name, sstring view_name) { co_await write_view_build_status( - [&] () -> future<> { + [this, ks_name, view_name] () -> future<> { const sstring query_string = format("DELETE FROM {}.{} WHERE keyspace_name = ? AND view_name = ?", db::system_keyspace::NAME, db::system_keyspace::VIEW_BUILD_STATUS_V2); co_await announce_with_raft(_qp, _group0_client, _as, std::move(query_string), {std::move(ks_name), std::move(view_name)}, "view builder: delete view build status"); }, - [&] () -> future<> { + [this, ks_name, view_name] () -> future<> { co_await _sys_dist_ks.remove_view(std::move(ks_name), std::move(view_name)); } ); From 7a6aec1a6ccf75aec58a117b4980629bd9ebed81 Mon Sep 17 00:00:00 2001 From: Michael Litvak Date: Tue, 14 Jan 2025 11:09:06 +0200 Subject: [PATCH 278/397] view_builder: hold semaphore during entire startup Guard the whole view builder startup routine by holding the semaphore until it's done instead of releasing it early, so that it's not intercepted by migration notifications. --- db/view/view.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/db/view/view.cc b/db/view/view.cc index 4ccded2c6244..2d43024c0ae6 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -2050,7 +2050,6 @@ future<> view_builder::start_in_background(service::migration_manager& mm, utils // the view build information. fail.cancel(); co_await barrier.arrive_and_wait(); - units.return_all(); _mnotifier.register_listener(this); co_await calculate_shard_build_step(vbi); From 94f4871352dd82a5bb7a3f54f717ad7818241cf3 Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Fri, 20 Dec 2024 15:18:40 +0100 Subject: [PATCH 279/397] test: start waiting for task before it gets aborted Ensure that the repair task was aborted after wait API acknowledged its existence. Fixes: #22011. Closes scylladb/scylladb#22012 --- test/topology_tasks/test_tablet_tasks.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/test/topology_tasks/test_tablet_tasks.py b/test/topology_tasks/test_tablet_tasks.py index dfd88a495c23..c682a6211ada 100644 --- a/test/topology_tasks/test_tablet_tasks.py +++ b/test/topology_tasks/test_tablet_tasks.py @@ -43,7 +43,7 @@ def check_task_status(status: TaskStatus, states: list[str], type: str, scope: s assert not status.children_ids assert status.state in states -async def check_and_abort_repair_task(tm: TaskManagerClient, servers: list[ServerInfo], module_name: str): +async def check_and_abort_repair_task(manager: ManagerClient, tm: TaskManagerClient, servers: list[ServerInfo], module_name: str): # Wait until user repair task is created. repair_tasks = await wait_tasks_created(tm, servers[0], module_name, 1, "user_repair") @@ -57,11 +57,15 @@ async def check_and_abort_repair_task(tm: TaskManagerClient, servers: list[Serve check_task_status(status, ["created", "running"], "user_repair", "table", True) + log = await manager.server_open_log(servers[0].server_id) + mark = await log.mark() + async def wait_for_task(): status_wait = await tm.wait_for_task(servers[0].ip_addr, task.task_id) check_task_status(status_wait, ["done"], "user_repair", "table", True) async def abort_task(): + await log.wait_for('tablet_virtual_task: wait until tablet operation is finished', from_mark=mark) await tm.abort_task(servers[0].ip_addr, task.task_id) await asyncio.gather(wait_for_task(), abort_task()) @@ -81,7 +85,7 @@ async def repair_task(): await inject_error_on(manager, "repair_tablet_fail_on_rpc_call", servers) await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token) - await asyncio.gather(repair_task(), check_and_abort_repair_task(tm, servers, module_name)) + await asyncio.gather(repair_task(), check_and_abort_repair_task(manager, tm, servers, module_name)) async def check_repair_task_list(tm: TaskManagerClient, servers: list[ServerInfo], module_name: str): def get_task_with_id(repair_tasks, task_id): From 25e8478051e98b9da381fe8010cd132b186a0821 Mon Sep 17 00:00:00 2001 From: Geoff Montee Date: Mon, 25 Nov 2024 11:54:16 -0800 Subject: [PATCH 280/397] docs: rest.rst: use latest docker tag to view Swagger UI for REST API Closes scylladb/scylladb#21681 --- docs/operating-scylla/rest.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/operating-scylla/rest.rst b/docs/operating-scylla/rest.rst index 99b40f230fe0..c588bae59156 100644 --- a/docs/operating-scylla/rest.rst +++ b/docs/operating-scylla/rest.rst @@ -32,7 +32,7 @@ The following example shows using the Swagger UI with Docker. .. code:: sh - docker run --name some-scylla -p 10000:10000 -d scylladb/scylla:5.0.4 --api-address 0.0.0.0 + docker run --name some-scylla -p 10000:10000 -d scylladb/scylla:latest --api-address 0.0.0.0 #. Go to http://localhost:10000/ui/. From 32ab58cdea5a6b820fe6629c5cf30ae03e21c2aa Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 2 Jan 2025 11:39:00 +0100 Subject: [PATCH 281/397] repair: add repair_service gate In main.cc storage_service is started before and stopped after repair_service. storage_service keeps a reference to sharded repair_service and calls its methods, but nothing ensures that repair_service's local instance would be alive for the whole execution of the method. Add a gate to repair_service and enter it in storage_service before executing methods on local instances of repair_service. Fixes: #21964. Closes scylladb/scylladb#22145 --- repair/row_level.cc | 1 + repair/row_level.hh | 3 +++ service/storage_service.cc | 48 ++++++++++++++++++++++++++++++-------- 3 files changed, 42 insertions(+), 10 deletions(-) diff --git a/repair/row_level.cc b/repair/row_level.cc index cb8843faaa84..ac05787d91e0 100644 --- a/repair/row_level.cc +++ b/repair/row_level.cc @@ -3282,6 +3282,7 @@ future<> repair_service::stop() { rlogger.debug("Unregistering gossiper helper"); co_await _gossiper.local().unregister_(_gossip_helper); } + co_await async_gate().close(); _stopped = true; rlogger.info("Stopped repair_service"); } catch (...) { diff --git a/repair/row_level.hh b/repair/row_level.hh index 8421287527e3..c89fff0a5829 100644 --- a/repair/row_level.hh +++ b/repair/row_level.hh @@ -107,6 +107,8 @@ class repair_service : public seastar::peering_sharded_service { shared_ptr _gossip_helper; bool _stopped = false; + gate _gate; + size_t _max_repair_memory; seastar::semaphore _memory_sem; seastar::named_semaphore _load_parallelism_semaphore = {16, named_semaphore_exception_factory{"Load repair history parallelism"}}; @@ -195,6 +197,7 @@ public: size_t max_repair_memory() const { return _max_repair_memory; } seastar::semaphore& memory_sem() { return _memory_sem; } locator::host_id my_host_id() const noexcept; + gate& async_gate() noexcept { return _gate; } repair::task_manager_module& get_repair_module() noexcept { return *_repair_module; diff --git a/service/storage_service.cc b/service/storage_service.cc index 8551a083bc2f..ac0095f39714 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -155,6 +155,14 @@ namespace { static constexpr std::chrono::seconds wait_for_live_nodes_timeout{30}; +static future<> do_with_repair_service(sharded& repair, std::function(repair_service&)> f) { + if (!repair.local_is_initialized()) { + throw seastar::abort_requested_exception(); + } + gate::holder holder = repair.local().async_gate().hold(); + co_await f(repair.local()); +} + storage_service::storage_service(abort_source& abort_source, distributed& db, gms::gossiper& gossiper, sharded& sys_ks, @@ -3944,7 +3952,9 @@ void storage_service::run_bootstrap_ops(std::unordered_set& bootstrap_tok utils::get_local_injector().inject("delay_bootstrap_120s", std::chrono::seconds(120)).get(); // Step 5: Sync data for bootstrap - _repair.local().bootstrap_with_repair(get_token_metadata_ptr(), bootstrap_tokens).get(); + do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.bootstrap_with_repair(get_token_metadata_ptr(), bootstrap_tokens); + }).get(); on_streaming_finished(); // Step 6: Finish @@ -4000,7 +4010,9 @@ void storage_service::run_replace_ops(std::unordered_set& bootstrap_token auto ignore_nodes = replace_info.ignore_nodes | std::views::transform([] (const auto& x) { return x.first; }) | std::ranges::to>(); - _repair.local().replace_with_repair(std::move(ks_erms), std::move(tmptr), bootstrap_tokens, std::move(ignore_nodes), replace_info.host_id).get(); + do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.replace_with_repair(std::move(ks_erms), std::move(tmptr), bootstrap_tokens, std::move(ignore_nodes), replace_info.host_id); + }).get(); } else { slogger.info("replace[{}]: Using streaming based node ops to sync data", uuid); dht::boot_strapper bs(_db, _stream_manager, _abort_source, get_token_metadata_ptr()->get_my_id(), _snitch.local()->get_location(), bootstrap_tokens, get_token_metadata_ptr()); @@ -4423,7 +4435,9 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad for (auto& node : req.leaving_nodes) { if (is_repair_based_node_ops_enabled(streaming::stream_reason::removenode)) { slogger.info("removenode[{}]: Started to sync data for removing node={} using repair, coordinator={}", req.ops_uuid, node, coordinator); - _repair.local().removenode_with_repair(get_token_metadata_ptr(), _gossiper.get_host_id(node), ops).get(); + do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.removenode_with_repair(get_token_metadata_ptr(), _gossiper.get_host_id(node), ops); + }).get(); } else { slogger.info("removenode[{}]: Started to sync data for removing node={} using stream, coordinator={}", req.ops_uuid, node, coordinator); removenode_with_stream(get_token_metadata().get_host_id(node), topo_guard, as).get(); @@ -4864,7 +4878,9 @@ future<> storage_service::rebuild(utils::optional_param source_dc) { auto tmptr = ss.get_token_metadata_ptr(); auto ks_erms = ss._db.local().get_non_local_strategy_keyspaces_erms(); if (ss.is_repair_based_node_ops_enabled(streaming::stream_reason::rebuild)) { - co_await ss._repair.local().rebuild_with_repair(std::move(ks_erms), tmptr, std::move(source_dc)); + co_await do_with_repair_service(ss._repair, [&] (repair_service& local_repair) { + return local_repair.rebuild_with_repair(std::move(ks_erms), tmptr, std::move(source_dc)); + }); } else { auto streamer = make_lw_shared(ss._db, ss._stream_manager, tmptr, ss._abort_source, tmptr->get_my_id(), ss._snitch.local()->get_location(), "Rebuild", streaming::stream_reason::rebuild, null_topology_guard); @@ -4986,7 +5002,9 @@ future<> storage_service::unbootstrap() { slogger.info("Finished batchlog replay for decommission"); if (is_repair_based_node_ops_enabled(streaming::stream_reason::decommission)) { - co_await _repair.local().decommission_with_repair(get_token_metadata_ptr()); + co_await do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.decommission_with_repair(get_token_metadata_ptr()); + }); } else { std::unordered_map> ranges_to_stream; @@ -5658,7 +5676,9 @@ future storage_service::raft_topology_cmd_handler(raft if (is_repair_based_node_ops_enabled(streaming::stream_reason::bootstrap)) { co_await utils::get_local_injector().inject("delay_bootstrap_120s", std::chrono::seconds(120)); - co_await _repair.local().bootstrap_with_repair(get_token_metadata_ptr(), rs.ring.value().tokens); + co_await do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.bootstrap_with_repair(get_token_metadata_ptr(), rs.ring.value().tokens); + }); } else { dht::boot_strapper bs(_db, _stream_manager, _abort_source, get_token_metadata_ptr()->get_my_id(), locator::endpoint_dc_rack{rs.datacenter, rs.rack}, rs.ring.value().tokens, get_token_metadata_ptr()); @@ -5684,7 +5704,9 @@ future storage_service::raft_topology_cmd_handler(raft auto ks_erms = _db.local().get_non_local_strategy_keyspaces_erms(); auto tmptr = get_token_metadata_ptr(); auto replaced_node = locator::host_id(replaced_id.uuid()); - co_await _repair.local().replace_with_repair(std::move(ks_erms), std::move(tmptr), rs.ring.value().tokens, std::move(ignored_nodes), replaced_node); + co_await do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.replace_with_repair(std::move(ks_erms), std::move(tmptr), rs.ring.value().tokens, std::move(ignored_nodes), replaced_node); + }); } else { dht::boot_strapper bs(_db, _stream_manager, _abort_source, get_token_metadata_ptr()->get_my_id(), locator::endpoint_dc_rack{rs.datacenter, rs.rack}, rs.ring.value().tokens, get_token_metadata_ptr()); @@ -5742,7 +5764,9 @@ future storage_service::raft_topology_cmd_handler(raft ignored_ips.push_back(*ip); } auto ops = seastar::make_shared(node_ops_id::create_random_id(), as, std::move(ignored_ips)); - return _repair.local().removenode_with_repair(get_token_metadata_ptr(), id, ops); + return do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.removenode_with_repair(get_token_metadata_ptr(), id, ops); + }); } else { return removenode_with_stream(id, _topology_state_machine._topology.session, as); } @@ -5768,7 +5792,9 @@ future storage_service::raft_topology_cmd_handler(raft if (!source_dc.empty()) { sdc_param.emplace(source_dc).set_user_provided().set_force(force); } - co_await _repair.local().rebuild_with_repair(std::move(ks_erms), tmptr, std::move(sdc_param)); + co_await do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.rebuild_with_repair(std::move(ks_erms), tmptr, std::move(sdc_param)); + }); } else { auto streamer = make_lw_shared(_db, _stream_manager, tmptr, _abort_source, tmptr->get_my_id(), _snitch.local()->get_location(), "Rebuild", streaming::stream_reason::rebuild, _topology_state_machine._topology.session); @@ -5957,7 +5983,9 @@ future<> storage_service::repair_tablet(locator::global_tablet_id tablet) { utils::get_local_injector().inject("repair_tablet_fail_on_rpc_call", [] { throw std::runtime_error("repair_tablet failed due to error injection"); }); - co_await _repair.local().repair_tablet(_address_map, guard, tablet); + co_await do_with_repair_service(_repair, [&] (repair_service& local_repair) { + return local_repair.repair_tablet(_address_map, guard, tablet); + }); co_return; }); } From 592512fd0f80b7069aea6dcddf53595d786b4b5b Mon Sep 17 00:00:00 2001 From: Aleksandra Martyniuk Date: Thu, 9 Jan 2025 17:55:04 +0100 Subject: [PATCH 282/397] test: fix memtable_flush_period test memtable_flush_period test sets the flush period to 200ms and checks whether the data is flushed after 500ms. When flush period is set, the timer is armed with the given value. On expiration, memtables are flushed and then the timer is rearmed. There is no certainty that during 500ms the flush finishes, though. Check if after 500ms flush has started. Wait until there is an sstable. Fixes: #21965. Closes scylladb/scylladb#22162 --- test/boost/memtable_test.cc | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/test/boost/memtable_test.cc b/test/boost/memtable_test.cc index daa7d85c51d1..09c85d273a99 100644 --- a/test/boost/memtable_test.cc +++ b/test/boost/memtable_test.cc @@ -1035,9 +1035,11 @@ SEASTAR_TEST_CASE(memtable_flush_period) { schema_ptr s2 = b.build(); t.set_schema(s2); - sleep(500ms).get(); // wait until memtable will be flushed at least once - - BOOST_REQUIRE_EQUAL(t.sstables_count(), 1); // check sstable exists after flush + sleep(500ms).get(); // wait until memtable flush starts at least once + BOOST_REQUIRE(t.sstables_count() == 1 || t.get_stats().pending_flushes > 0); // flush started + BOOST_REQUIRE(eventually_true([&] { // wait until memtable will be flushed at least once + return t.sstables_count() == 1; + })); // Check mutation presents in the table mutation_source ms = t.as_mutation_source(); From f8885a4afd30c0c5043b2610b39c761b347edaa3 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 14 Jan 2025 09:58:46 +0800 Subject: [PATCH 283/397] dist/docker,docs: replace "--experimental" with "--experimental-features" The "--experimental" option was removed in commit f6cca741ea. Using this deprecated option now causes Scylla to fail with the error: ``` error: the argument ('on') for option '--experimental-features' is invalid ``` So, in this change, let's update the docker entry point script to use `--experimental-features` command line option instead. The related document is updated accordingly. Fixes scylladb/scylladb#22207 Signed-off-by: Kefu Chai Closes scylladb/scylladb#22283 --- dist/docker/commandlineparser.py | 2 +- dist/docker/scyllasetup.py | 7 +-- docs/dev/docker-hub.md | 12 +++--- docs/faq.rst | 43 +++++++++++++++---- .../tips/best-practices-scylla-on-docker.rst | 12 +++--- 5 files changed, 51 insertions(+), 25 deletions(-) diff --git a/dist/docker/commandlineparser.py b/dist/docker/commandlineparser.py index 0dedb71924f2..cb168f5206c7 100644 --- a/dist/docker/commandlineparser.py +++ b/dist/docker/commandlineparser.py @@ -4,7 +4,7 @@ def parse(): parser = argparse.ArgumentParser() parser.add_argument('--developer-mode', default='1', choices=['0', '1'], dest='developerMode') - parser.add_argument('--experimental', default=0, choices=['0', '1']) + parser.add_argument('--experimental-features', action='append', help='Unlock experimental features. Can be repeated') parser.add_argument('--seeds', default=None, help="specify seeds - if left empty will use container's own IP") parser.add_argument('--cpuset', default=None, help="e.g. --cpuset 0-3 for the first four CPUs") parser.add_argument('--smp', default=None, help="e.g --smp 2 to use two CPUs") diff --git a/dist/docker/scyllasetup.py b/dist/docker/scyllasetup.py index e47fba8677f9..f1b8d774631b 100644 --- a/dist/docker/scyllasetup.py +++ b/dist/docker/scyllasetup.py @@ -23,7 +23,7 @@ def __init__(self, arguments, extra_arguments): self._reserveMemory = arguments.reserveMemory self._overprovisioned = arguments.overprovisioned self._housekeeping = not arguments.disable_housekeeping - self._experimental = arguments.experimental + self._experimental_features = arguments.experimental_features self._authenticator = arguments.authenticator self._authorizer = arguments.authorizer self._clusterName = arguments.clusterName @@ -146,8 +146,9 @@ def arguments(self): if self._authorizer is not None: args += ["--authorizer %s" % self._authorizer] - if self._experimental == "1": - args += ["--experimental=on"] + if self._experimental_features is not None: + for feature in self._experimental_features: + args += [f"--experimental-features {feature}"] if self._clusterName is not None: args += ["--cluster-name %s" % self._clusterName] diff --git a/docs/dev/docker-hub.md b/docs/dev/docker-hub.md index a0fa11b307f7..7be1bfba6839 100644 --- a/docs/dev/docker-hub.md +++ b/docs/dev/docker-hub.md @@ -326,16 +326,16 @@ For example, to disable developer mode: $ docker run --name some-scylla -d scylladb/scylla --developer-mode 0 ``` -#### `--experimental ENABLE` +#### `--experimental-features FEATURE` -The `--experimental` command line option enables ScyllaDB's experimental mode -If no `--experimental` command line option is defined, ScyllaDB defaults to running with experimental mode *disabled*. -It is highly recommended to disable experimental mode for production deployments. +The `--experimental-features` command line option enables ScyllaDB's experimental feature individually. If no feature flags are specified, ScyllaDB runs with only *stable* features enabled. -For example, to enable experimental mode: +Running experimental features in production environments is not recommended. + +For example, to enable the User Defined Functions (UDF) feature: ```console -$ docker run --name some-scylla -d scylladb/scylla --experimental 1 +$ docker run --name some-scylla -d scylladb/scylla --experimental-feature=udf ``` **Since: 2.0** diff --git a/docs/faq.rst b/docs/faq.rst index 771a076c1aff..58a6b1be72b7 100644 --- a/docs/faq.rst +++ b/docs/faq.rst @@ -104,14 +104,35 @@ You can clean snapshots by using :doc:`nodetool clearsnapshot && docker start ` - Alternately, starting from ScyllaDB 2.0, you can start ScyllaDB for Docker with the :code:`experimental` flag as follows: +Alternately, starting from ScyllaDB 3.3, you can enable features directly via command line flags the :code:`--experimental-features` flag as follows. This command line options can be repeated multiple times. For example, to enable UDF and Alternator Streams: + +.. code-block:: console - :code:`$ docker run --name -d scylladb/scylla --experimental 1` + $ docker run --name -d scylladb/scylla \ + --experimental-features=udf \ + --experimental-features=alternator-streams -You should now be able to use the experimental features available in your version of ScyllaDB. +You should now be able to use the specified experimental features. How do I check the current version of ScyllaDB that I am running? ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/docs/operating-scylla/procedures/tips/best-practices-scylla-on-docker.rst b/docs/operating-scylla/procedures/tips/best-practices-scylla-on-docker.rst index ddcdedf2f60f..22281b6744fc 100644 --- a/docs/operating-scylla/procedures/tips/best-practices-scylla-on-docker.rst +++ b/docs/operating-scylla/procedures/tips/best-practices-scylla-on-docker.rst @@ -289,17 +289,17 @@ To disable developer mode: docker run --name some-scylla -d scylladb/scylla --developer-mode 0 ---experimental ENABLE ---------------------- -The ``--experimental`` command line option enables ScyllaDB's experimental mode. If no ``--experimental`` command line option is defined, ScyllaDB defaults to running with experimental mode disabled. +--experimental-features FEATURE +------------------------------- +The ``--experimental-features`` command line option enables ScyllaDB's experimental feature individually. If no feature flags are specified, ScyllaDB runs with only stable features enabled. -**It is highly recommended to disable experimental mode for production deployments.** +**Running experimental features in production environments is not recommended.** -For example, to enable experimental mode: +For example, to enable the User Defined Functions (UDF) feature: .. code-block:: console - docker run --name some-scylla -d scylladb/scylla --experimental 1 + docker run --name some-scylla -d scylladb/scylla --experimental-features=udf Other Useful Tips and Tricks ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ From e6b05cb9ea8ca316bd1c8cd5915a52085ff1482e Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 13 Jan 2025 18:33:20 +0800 Subject: [PATCH 284/397] .github: use the toolchain specified by tools/toolchain/image Previously, we hardwire the container to a previous frozen toolchain image. but at the time of writing, the tree does not compile in the specified toolchain image anymore, after the required building environment is updated, and toolchain was updated accordingly. in order to improve the maintability, let's reuse `read-toolchain.yaml` job which reads `tools/toolchain/image`, so we don't have to hardwire the container used for building the tree with the latest seastar. this should address the build failure surfaced recently. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22287 --- .github/workflows/seastar.yaml | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/.github/workflows/seastar.yaml b/.github/workflows/seastar.yaml index 75f1f1be75dc..e810b61346a7 100644 --- a/.github/workflows/seastar.yaml +++ b/.github/workflows/seastar.yaml @@ -15,10 +15,13 @@ env: BUILD_DIR: build jobs: + read-toolchain: + uses: ./.github/workflows/read-toolchain.yaml build-with-the-latest-seastar: + needs: + - read-toolchain runs-on: ubuntu-latest - # be consistent with tools/toolchain/image - container: scylladb/scylla-toolchain:fedora-40-20240621 + container: ${{ needs.read-toolchain.outputs.image }} strategy: matrix: build_type: From 7215d4bfe9b0bd881ea0d98c0c469991f526a0bd Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 7 Jan 2025 15:00:24 +0800 Subject: [PATCH 285/397] utils: do not include unused headers these unused includes were identifier by clang-include-cleaner. after auditing these source files, all of the reports have been confirmed. please note, because quite a few source files relied on `utils/to_string.hh` to pull in the specialization of `fmt::formatter>`, after removing `#include ` from `utils/to_string.hh`, we have to include `fmt/std.h` directly. Signed-off-by: Kefu Chai --- alternator/controller.cc | 2 ++ alternator/executor.cc | 1 + cache_mutation_reader.hh | 1 + compound_compat.hh | 1 + cql3/statements/list_users_statement.cc | 1 + cql3/statements/role-management-statements.cc | 2 ++ db/config.cc | 1 + db/paxos_grace_seconds_extension.hh | 2 ++ dht/i_partitioner.cc | 1 + dht/token.cc | 1 + gms/endpoint_state.cc | 1 + gms/versioned_value.hh | 1 + locator/topology.cc | 1 + message/dictionary_service.hh | 1 + mutation/mutation_cleaner.hh | 2 ++ mutation/mutation_partition.cc | 1 + mutation_writer/partition_based_splitting_writer.cc | 1 + redis/controller.cc | 2 ++ replica/database.hh | 1 + replica/dirty_memory_manager.hh | 2 ++ replica/table.cc | 1 + service/qos/service_level_controller.hh | 2 ++ service/raft/raft_group0.cc | 1 + service/storage_proxy.cc | 1 + service/topology_coordinator.cc | 2 ++ service/topology_state_machine.hh | 1 + sstables/checksum_utils.hh | 1 + sstables/index_reader.hh | 1 + sstables/partition_index_cache.hh | 1 + sstables/storage.cc | 1 + test/boost/dirty_memory_manager_test.cc | 1 + test/boost/flush_queue_test.cc | 2 ++ test/boost/intrusive_array_test.cc | 1 + test/boost/lister_test.cc | 1 + test/boost/logalloc_test.cc | 1 + test/boost/mutation_fragment_test.cc | 1 + test/boost/partitioner_test.cc | 1 + test/boost/token_metadata_test.cc | 2 +- test/boost/virtual_table_mutation_source_test.cc | 1 + test/boost/wrapping_interval_test.cc | 3 ++- test/raft/many_test.cc | 2 +- test/raft/raft_server_test.cc | 1 + test/raft/replication.cc | 1 + test/raft/replication_test.cc | 2 +- test/unit/row_cache_stress_test.cc | 1 + transport/controller.cc | 1 + transport/messages/result_message.cc | 2 +- types/user.hh | 1 + utils/UUID.hh | 4 +--- utils/allocation_strategy.hh | 3 ++- utils/ascii.hh | 2 +- utils/atomic_vector.hh | 1 - utils/aws_sigv4.cc | 1 + utils/aws_sigv4.hh | 1 - utils/base64.hh | 2 +- utils/big_decimal.hh | 5 +++-- utils/cached_file.hh | 1 - utils/class_registrator.hh | 1 - utils/collection-concepts.hh | 1 - utils/config_file.hh | 5 ++--- utils/config_file_impl.hh | 9 ++------- utils/contiguous_shared_buffer.hh | 2 +- utils/coroutine.hh | 2 +- utils/data_input.hh | 2 +- utils/dict_trainer.hh | 3 +-- utils/digester.hh | 1 + utils/directories.hh | 2 +- utils/disk-error-handler.hh | 2 +- utils/double-decker.hh | 1 - utils/dynamic_bitset.hh | 2 +- utils/error_injection.hh | 1 - utils/exception_container.hh | 4 ---- utils/exceptions.hh | 5 ++--- utils/exponential_backoff_retry.hh | 2 +- utils/file_lock.hh | 1 - utils/flush_queue.hh | 1 - utils/gz/crc_combine.hh | 1 - utils/hashers.hh | 3 +-- utils/histogram.hh | 1 - utils/http.hh | 1 - utils/i_filter.hh | 3 +-- utils/immutable-collection.hh | 2 -- utils/intrusive-array.hh | 7 ++++--- utils/large_bitset.hh | 3 --- utils/like_matcher.hh | 2 +- utils/lister.cc | 1 + utils/lister.hh | 4 ---- utils/loading_cache.hh | 5 +---- utils/loading_shared_values.hh | 1 - utils/logalloc.cc | 3 ++- utils/logalloc.hh | 6 +----- utils/lsa/chunked_managed_vector.hh | 1 - utils/multiprecision_int.hh | 2 +- utils/murmur_hash.hh | 2 +- utils/on_internal_error.hh | 3 +-- utils/rate_limiter.hh | 1 - utils/result.hh | 5 +++-- utils/result_loop.hh | 2 +- utils/result_try.hh | 1 - utils/rjson.cc | 1 + utils/rjson.hh | 2 +- utils/s3/client.cc | 1 + utils/s3/client.hh | 2 +- utils/stall_free.hh | 1 - utils/to_string.hh | 4 ++-- utils/top_k.hh | 1 - utils/user_provided_param.hh | 1 - utils/utf8.hh | 2 +- utils/uuid.cc | 1 + utils/xx_hasher.hh | 2 +- 110 files changed, 110 insertions(+), 98 deletions(-) diff --git a/alternator/controller.cc b/alternator/controller.cc index b791ea8de2d1..3fc98d23a87a 100644 --- a/alternator/controller.cc +++ b/alternator/controller.cc @@ -6,7 +6,9 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ +#include #include + #include "controller.hh" #include "server.hh" #include "executor.hh" diff --git a/alternator/executor.cc b/alternator/executor.cc index 96e7b30a26c7..7dcd4bc0d60c 100644 --- a/alternator/executor.cc +++ b/alternator/executor.cc @@ -45,6 +45,7 @@ #include "alternator/rmw_operation.hh" #include #include +#include #include #include #include diff --git a/cache_mutation_reader.hh b/cache_mutation_reader.hh index 79bd5443d7d0..f04e55c45268 100644 --- a/cache_mutation_reader.hh +++ b/cache_mutation_reader.hh @@ -10,6 +10,7 @@ #include "utils/assert.hh" #include +#include #include "row_cache.hh" #include "mutation/mutation_fragment.hh" #include "query-request.hh" diff --git a/compound_compat.hh b/compound_compat.hh index 8ef1d4fb6c29..e78c472b8480 100644 --- a/compound_compat.hh +++ b/compound_compat.hh @@ -10,6 +10,7 @@ #include #include +#include #include "compound.hh" #include "schema/schema.hh" #include "sstables/version.hh" diff --git a/cql3/statements/list_users_statement.cc b/cql3/statements/list_users_statement.cc index 819542442076..0f833f55771b 100644 --- a/cql3/statements/list_users_statement.cc +++ b/cql3/statements/list_users_statement.cc @@ -8,6 +8,7 @@ * SPDX-License-Identifier: (LicenseRef-ScyllaDB-Source-Available-1.0 and Apache-2.0) */ +#include #include "list_users_statement.hh" #include "cql3/query_processor.hh" #include "cql3/query_options.hh" diff --git a/cql3/statements/role-management-statements.cc b/cql3/statements/role-management-statements.cc index 864e8de42602..e60c315853a5 100644 --- a/cql3/statements/role-management-statements.cc +++ b/cql3/statements/role-management-statements.cc @@ -10,6 +10,8 @@ #include +#include + #include "types/map.hh" #include "auth/authentication_options.hh" #include "auth/service.hh" diff --git a/db/config.cc b/db/config.cc index 47311a304fae..da76b4d79529 100644 --- a/db/config.cc +++ b/db/config.cc @@ -20,6 +20,7 @@ #include #include +#include #include #include #include diff --git a/db/paxos_grace_seconds_extension.hh b/db/paxos_grace_seconds_extension.hh index d1a3bba595c3..a6a3b3959791 100644 --- a/db/paxos_grace_seconds_extension.hh +++ b/db/paxos_grace_seconds_extension.hh @@ -7,6 +7,8 @@ #pragma once +#include + #include "serializer.hh" #include "schema/schema.hh" #include "utils/log.hh" diff --git a/dht/i_partitioner.cc b/dht/i_partitioner.cc index 7029e5f736a1..f2e8a7a15aba 100644 --- a/dht/i_partitioner.cc +++ b/dht/i_partitioner.cc @@ -9,6 +9,7 @@ #include "i_partitioner.hh" #include "sharder.hh" #include "auto_refreshing_sharder.hh" +#include #include #include #include "dht/ring_position.hh" diff --git a/dht/token.cc b/dht/token.cc index c6b4420aee8a..eec7f6cf3189 100644 --- a/dht/token.cc +++ b/dht/token.cc @@ -9,6 +9,7 @@ #include #include #include +#include #include #include diff --git a/gms/endpoint_state.cc b/gms/endpoint_state.cc index 5cd16f64648a..5c9ef4b90ed0 100644 --- a/gms/endpoint_state.cc +++ b/gms/endpoint_state.cc @@ -10,6 +10,7 @@ #include "gms/endpoint_state.hh" #include "gms/i_endpoint_state_change_subscriber.hh" +#include #include #include "utils/log.hh" diff --git a/gms/versioned_value.hh b/gms/versioned_value.hh index 85b9cb4a3091..f60c17f0249d 100644 --- a/gms/versioned_value.hh +++ b/gms/versioned_value.hh @@ -19,6 +19,7 @@ #include "service/state_id.hh" #include "version.hh" #include "cdc/generation_id.hh" +#include #include namespace gms { diff --git a/locator/topology.cc b/locator/topology.cc index 7371c10df0a6..179b1ae87fab 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -9,6 +9,7 @@ #include #include #include +#include #include #include diff --git a/message/dictionary_service.hh b/message/dictionary_service.hh index 19435a81c6a1..b838088d6dda 100644 --- a/message/dictionary_service.hh +++ b/message/dictionary_service.hh @@ -1,4 +1,5 @@ #include "locator/host_id.hh" +#include "utils/enum_option.hh" #include "utils/updateable_value.hh" #include "utils/dict_trainer.hh" #include diff --git a/mutation/mutation_cleaner.hh b/mutation/mutation_cleaner.hh index 7b7111a543ce..d33bfd34bfec 100644 --- a/mutation/mutation_cleaner.hh +++ b/mutation/mutation_cleaner.hh @@ -8,6 +8,8 @@ #pragma once +#include + #include "partition_version.hh" #include "partition_version_list.hh" diff --git a/mutation/mutation_partition.cc b/mutation/mutation_partition.cc index af1e0c19f6be..22f6d3ae16ad 100644 --- a/mutation/mutation_partition.cc +++ b/mutation/mutation_partition.cc @@ -7,6 +7,7 @@ */ #include +#include #include #include "mutation_partition.hh" diff --git a/mutation_writer/partition_based_splitting_writer.cc b/mutation_writer/partition_based_splitting_writer.cc index 366f8da83797..11f39e26c1ee 100644 --- a/mutation_writer/partition_based_splitting_writer.cc +++ b/mutation_writer/partition_based_splitting_writer.cc @@ -12,6 +12,7 @@ #include "replica/memtable.hh" #include +#include namespace mutation_writer { diff --git a/redis/controller.cc b/redis/controller.cc index db6917e8f0f0..8f3307b092f0 100644 --- a/redis/controller.cc +++ b/redis/controller.cc @@ -7,6 +7,8 @@ */ #include +#include + #include "timeout_config.hh" #include "redis/controller.hh" #include "redis/keyspace_utils.hh" diff --git a/replica/database.hh b/replica/database.hh index 94fc42e372e5..e08f00edd2e3 100644 --- a/replica/database.hh +++ b/replica/database.hh @@ -14,6 +14,7 @@ #include #include #include +#include #include "utils/assert.hh" #include "utils/hash.hh" #include "db_clock.hh" diff --git a/replica/dirty_memory_manager.hh b/replica/dirty_memory_manager.hh index 0c72fffa4407..cc69816c6ed7 100644 --- a/replica/dirty_memory_manager.hh +++ b/replica/dirty_memory_manager.hh @@ -11,9 +11,11 @@ #include #include #include +#include #include #include #include +#include "db/timeout_clock.hh" #include "replica/database_fwd.hh" #include "utils/assert.hh" #include "utils/logalloc.hh" diff --git a/replica/table.cc b/replica/table.cc index 270aa5be6197..72481e7ebb63 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -8,6 +8,7 @@ #include #include +#include #include #include #include diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 9004a1625cd9..5ddf4734606e 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -14,6 +14,8 @@ #include #include #include +#include + #include "seastarx.hh" #include "auth/role_manager.hh" #include "auth/service.hh" diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index 1b41c62db073..78db1adfe088 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index efe7a4a300e6..da28f1059dd9 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -14,6 +14,7 @@ #include #include +#include #include #include "gms/inet_address.hh" #include "inet_address_vectors.hh" diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index f2b3a5a68a11..4e8ca60ef2cd 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -16,7 +16,9 @@ #include #include #include +#include #include + #include #include "auth/service.hh" diff --git a/service/topology_state_machine.hh b/service/topology_state_machine.hh index 0d29d66c7f0d..23db2af0a4a8 100644 --- a/service/topology_state_machine.hh +++ b/service/topology_state_machine.hh @@ -10,6 +10,7 @@ #pragma once #include +#include #include #include #include diff --git a/sstables/checksum_utils.hh b/sstables/checksum_utils.hh index b227f0daac49..9418a5538b50 100644 --- a/sstables/checksum_utils.hh +++ b/sstables/checksum_utils.hh @@ -8,6 +8,7 @@ #pragma once +#include #include #include #include "utils/gz/crc_combine.hh" diff --git a/sstables/index_reader.hh b/sstables/index_reader.hh index b6a5ac6569e9..0693f75f87e7 100644 --- a/sstables/index_reader.hh +++ b/sstables/index_reader.hh @@ -14,6 +14,7 @@ #include "exceptions.hh" #include "sstables/partition_index_cache.hh" #include +#include #include "tracing/traced_file.hh" #include "sstables/scanning_clustered_index_cursor.hh" #include "sstables/mx/bsearch_clustered_cursor.hh" diff --git a/sstables/partition_index_cache.hh b/sstables/partition_index_cache.hh index a6b87cfd1876..85b2a043beb4 100644 --- a/sstables/partition_index_cache.hh +++ b/sstables/partition_index_cache.hh @@ -12,6 +12,7 @@ #include #include #include +#include #include #include "utils/assert.hh" #include "utils/bptree.hh" diff --git a/sstables/storage.cc b/sstables/storage.cc index 3eb754df3e15..095e9af70f61 100644 --- a/sstables/storage.cc +++ b/sstables/storage.cc @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include diff --git a/test/boost/dirty_memory_manager_test.cc b/test/boost/dirty_memory_manager_test.cc index 83ce99b0b983..c8c315121fd1 100644 --- a/test/boost/dirty_memory_manager_test.cc +++ b/test/boost/dirty_memory_manager_test.cc @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include diff --git a/test/boost/flush_queue_test.cc b/test/boost/flush_queue_test.cc index d113aa575754..bed182a1d0b6 100644 --- a/test/boost/flush_queue_test.cc +++ b/test/boost/flush_queue_test.cc @@ -12,9 +12,11 @@ #include #include #include +#include #include #include #include +#include #include #include "seastarx.hh" diff --git a/test/boost/intrusive_array_test.cc b/test/boost/intrusive_array_test.cc index ebee1464051d..6da546a993cf 100644 --- a/test/boost/intrusive_array_test.cc +++ b/test/boost/intrusive_array_test.cc @@ -10,6 +10,7 @@ #include "test/lib/scylla_test_case.hh" #include +#include "utils/allocation_strategy.hh" #include "utils/intrusive-array.hh" class element { diff --git a/test/boost/lister_test.cc b/test/boost/lister_test.cc index 6a2594839e36..8125a3541656 100644 --- a/test/boost/lister_test.cc +++ b/test/boost/lister_test.cc @@ -12,6 +12,7 @@ #include #include #include +#include #include "test/lib/scylla_test_case.hh" #include diff --git a/test/boost/logalloc_test.cc b/test/boost/logalloc_test.cc index 1458151c8ab6..38a2dd54fc5e 100644 --- a/test/boost/logalloc_test.cc +++ b/test/boost/logalloc_test.cc @@ -10,6 +10,7 @@ #include #include #include +#include #include #include diff --git a/test/boost/mutation_fragment_test.cc b/test/boost/mutation_fragment_test.cc index 859ebefa44f1..8f865f9c38ab 100644 --- a/test/boost/mutation_fragment_test.cc +++ b/test/boost/mutation_fragment_test.cc @@ -12,6 +12,7 @@ #include "test/lib/scylla_test_case.hh" #include #include +#include #include "test/lib/mutation_source_test.hh" #include "mutation/mutation_fragment.hh" diff --git a/test/boost/partitioner_test.cc b/test/boost/partitioner_test.cc index f51ec0b0205d..f77df3f75fa5 100644 --- a/test/boost/partitioner_test.cc +++ b/test/boost/partitioner_test.cc @@ -11,6 +11,7 @@ #include #include #include +#include #include "test/lib/scylla_test_case.hh" #include "dht/token.hh" diff --git a/test/boost/token_metadata_test.cc b/test/boost/token_metadata_test.cc index 6f62ad4a1575..fe9edeb13203 100644 --- a/test/boost/token_metadata_test.cc +++ b/test/boost/token_metadata_test.cc @@ -13,7 +13,7 @@ #include "locator/token_metadata.hh" #include "locator/simple_strategy.hh" #include "locator/everywhere_replication_strategy.hh" -#include "utils/to_string.hh" +#include using namespace locator; diff --git a/test/boost/virtual_table_mutation_source_test.cc b/test/boost/virtual_table_mutation_source_test.cc index 142b37bc7223..8d738dd044a8 100644 --- a/test/boost/virtual_table_mutation_source_test.cc +++ b/test/boost/virtual_table_mutation_source_test.cc @@ -6,6 +6,7 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ +#include #include #include "db/virtual_table.hh" #include "schema/schema.hh" diff --git a/test/boost/wrapping_interval_test.cc b/test/boost/wrapping_interval_test.cc index b6b120dc7e9f..3d6c44f96233 100644 --- a/test/boost/wrapping_interval_test.cc +++ b/test/boost/wrapping_interval_test.cc @@ -11,7 +11,8 @@ #include #include "boost/icl/interval_map.hpp" #include -#include +#include +#include #include "schema/schema_builder.hh" #include "locator/token_metadata.hh" diff --git a/test/raft/many_test.cc b/test/raft/many_test.cc index d1696e4514ce..f0b2a97196cb 100644 --- a/test/raft/many_test.cc +++ b/test/raft/many_test.cc @@ -10,8 +10,8 @@ // // Using slower but precise clock +#include #include "replication.hh" -#include "utils/to_string.hh" #ifdef SEASTAR_DEBUG // Increase tick time to allow debug to process messages diff --git a/test/raft/raft_server_test.cc b/test/raft/raft_server_test.cc index ebf27d7cfb60..147997b08363 100644 --- a/test/raft/raft_server_test.cc +++ b/test/raft/raft_server_test.cc @@ -1,3 +1,4 @@ +#include #include "replication.hh" #include "utils/error_injection.hh" #include diff --git a/test/raft/replication.cc b/test/raft/replication.cc index 4db83a17c2a0..db889ab022e5 100644 --- a/test/raft/replication.cc +++ b/test/raft/replication.cc @@ -7,6 +7,7 @@ */ #undef SEASTAR_TESTING_MAIN +#include #include "utils/to_string.hh" #include "replication.hh" diff --git a/test/raft/replication_test.cc b/test/raft/replication_test.cc index 86459367c232..8edaaafaaaeb 100644 --- a/test/raft/replication_test.cc +++ b/test/raft/replication_test.cc @@ -6,8 +6,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ +#include #include "replication.hh" -#include "utils/to_string.hh" // Test Raft library with declarative test definitions diff --git a/test/unit/row_cache_stress_test.cc b/test/unit/row_cache_stress_test.cc index 2f292a8dcfd7..6d9eb10dbcab 100644 --- a/test/unit/row_cache_stress_test.cc +++ b/test/unit/row_cache_stress_test.cc @@ -21,6 +21,7 @@ #include "test/lib/memtable_snapshot_source.hh" #include #include +#include static thread_local bool cancelled = false; diff --git a/transport/controller.cc b/transport/controller.cc index 458ba7beeb63..4bc8c51a2f7c 100644 --- a/transport/controller.cc +++ b/transport/controller.cc @@ -13,6 +13,7 @@ #include #include #include +#include #include "transport/server.hh" #include "service/memory_limiter.hh" #include "db/config.hh" diff --git a/transport/messages/result_message.cc b/transport/messages/result_message.cc index 228751923653..bdba1e93d3aa 100644 --- a/transport/messages/result_message.cc +++ b/transport/messages/result_message.cc @@ -9,8 +9,8 @@ #include "result_message.hh" #include "cql3/cql_statement.hh" -#include "utils/to_string.hh" #include +#include namespace cql_transport::messages { diff --git a/types/user.hh b/types/user.hh index 5c1856d61715..e853cf72a618 100644 --- a/types/user.hh +++ b/types/user.hh @@ -13,6 +13,7 @@ #include "types/tuple.hh" #include +#include class user_type_impl : public tuple_type_impl { using intern = type_interning_helper, std::vector, bool>; diff --git a/utils/UUID.hh b/utils/UUID.hh index f03e0e4efd30..8d907b952bca 100644 --- a/utils/UUID.hh +++ b/utils/UUID.hh @@ -17,9 +17,7 @@ #include #include -#include -#include -#include "bytes.hh" +#include "bytes_fwd.hh" #include "utils/assert.hh" #include "utils/hashing.hh" #include "utils/serialization.hh" diff --git a/utils/allocation_strategy.hh b/utils/allocation_strategy.hh index e7f588eadc98..f041837133ad 100644 --- a/utils/allocation_strategy.hh +++ b/utils/allocation_strategy.hh @@ -9,8 +9,9 @@ #pragma once #include +#include #include -#include +#include #include #include diff --git a/utils/ascii.hh b/utils/ascii.hh index eded0ccabb22..19bdacc2177b 100644 --- a/utils/ascii.hh +++ b/utils/ascii.hh @@ -11,7 +11,7 @@ #pragma once #include -#include "bytes.hh" +#include "bytes_fwd.hh" namespace utils { diff --git a/utils/atomic_vector.hh b/utils/atomic_vector.hh index 1cfb98ea2b63..9a01b4f513a4 100644 --- a/utils/atomic_vector.hh +++ b/utils/atomic_vector.hh @@ -11,7 +11,6 @@ #include #include #include -#include #include diff --git a/utils/aws_sigv4.cc b/utils/aws_sigv4.cc index e3797b5e7367..5fa074b8c9f3 100644 --- a/utils/aws_sigv4.cc +++ b/utils/aws_sigv4.cc @@ -9,6 +9,7 @@ #include #include "utils/aws_sigv4.hh" #include "utils/hashers.hh" +#include "bytes.hh" #include "db_clock.hh" using namespace std::chrono_literals; diff --git a/utils/aws_sigv4.hh b/utils/aws_sigv4.hh index c55d1eb73b5f..023e341f6597 100644 --- a/utils/aws_sigv4.hh +++ b/utils/aws_sigv4.hh @@ -8,7 +8,6 @@ #pragma once -#include #include "db_clock.hh" // The declared below get_signature() method makes the Signature string for AWS diff --git a/utils/base64.hh b/utils/base64.hh index 3b3ffc825d9d..a28c0833deb1 100644 --- a/utils/base64.hh +++ b/utils/base64.hh @@ -9,7 +9,7 @@ #pragma once #include -#include "bytes.hh" +#include "bytes_fwd.hh" std::string base64_encode(bytes_view); diff --git a/utils/big_decimal.hh b/utils/big_decimal.hh index e063de33f957..ae6a39f04e56 100644 --- a/utils/big_decimal.hh +++ b/utils/big_decimal.hh @@ -9,11 +9,12 @@ #pragma once #include "multiprecision_int.hh" -#include +#include +#include #include #include -#include "bytes.hh" +using seastar::sstring; uint64_t from_varint_to_integer(const utils::multiprecision_int& varint); diff --git a/utils/cached_file.hh b/utils/cached_file.hh index 334d86f83a73..3b320a10ab63 100644 --- a/utils/cached_file.hh +++ b/utils/cached_file.hh @@ -19,7 +19,6 @@ #include "utils/cached_file_stats.hh" #include -#include #include using namespace seastar; diff --git a/utils/class_registrator.hh b/utils/class_registrator.hh index 36884c453a59..388193000264 100644 --- a/utils/class_registrator.hh +++ b/utils/class_registrator.hh @@ -9,7 +9,6 @@ #pragma once #include -#include #include #include diff --git a/utils/collection-concepts.hh b/utils/collection-concepts.hh index 6f3747a1ea86..6ab56a32a460 100644 --- a/utils/collection-concepts.hh +++ b/utils/collection-concepts.hh @@ -8,7 +8,6 @@ #pragma once #include -#include #include template diff --git a/utils/config_file.hh b/utils/config_file.hh index b0da10949197..273d9df5a636 100644 --- a/utils/config_file.hh +++ b/utils/config_file.hh @@ -10,14 +10,13 @@ #pragma once #include -#include #include +#include #include -#include #include -#include +#include #include #include diff --git a/utils/config_file_impl.hh b/utils/config_file_impl.hh index 2f4c1024248a..18909cfe0fa9 100644 --- a/utils/config_file_impl.hh +++ b/utils/config_file_impl.hh @@ -9,19 +9,14 @@ #pragma once -#include -#include - -#include #include +#include +#include -#include #include #include "config_file.hh" -#include - namespace utils { template diff --git a/utils/contiguous_shared_buffer.hh b/utils/contiguous_shared_buffer.hh index b5d47297d7e9..1183398ed0d6 100644 --- a/utils/contiguous_shared_buffer.hh +++ b/utils/contiguous_shared_buffer.hh @@ -9,7 +9,7 @@ #pragma once #include -#include +#include // A contiguous buffer of char objects which can be trimmed and // supports zero-copy sharing of its underlying memory. diff --git a/utils/coroutine.hh b/utils/coroutine.hh index c220038a8bb7..545b13a11717 100644 --- a/utils/coroutine.hh +++ b/utils/coroutine.hh @@ -8,7 +8,7 @@ #pragma once -#include +#include #include #include "seastarx.hh" diff --git a/utils/data_input.hh b/utils/data_input.hh index 8dbd80f3ed66..adae8c8990ce 100644 --- a/utils/data_input.hh +++ b/utils/data_input.hh @@ -9,7 +9,7 @@ #ifndef UTILS_DATA_INPUT_HH_ #define UTILS_DATA_INPUT_HH_ -#include "bytes.hh" +#include "bytes_fwd.hh" #include #include diff --git a/utils/dict_trainer.hh b/utils/dict_trainer.hh index d2175b9ea01f..a6e487b7d786 100644 --- a/utils/dict_trainer.hh +++ b/utils/dict_trainer.hh @@ -8,13 +8,12 @@ #pragma once -#include "seastar/core/sharded.hh" -#include "utils/enum_option.hh" #include "utils/reservoir_sampling.hh" #include "utils/updateable_value.hh" #include #include #include +#include namespace utils { diff --git a/utils/digester.hh b/utils/digester.hh index 0d2185388e10..3d4077b47293 100644 --- a/utils/digester.hh +++ b/utils/digester.hh @@ -9,6 +9,7 @@ #pragma once #include "utils/digest_algorithm.hh" +#include "utils/hashing.hh" #include "utils/xx_hasher.hh" #include diff --git a/utils/directories.hh b/utils/directories.hh index 8e717d2b024b..34b9f2c06b8b 100644 --- a/utils/directories.hh +++ b/utils/directories.hh @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include "utils/file_lock.hh" using namespace seastar; diff --git a/utils/disk-error-handler.hh b/utils/disk-error-handler.hh index 7e661d70fc02..dd782d957a1e 100644 --- a/utils/disk-error-handler.hh +++ b/utils/disk-error-handler.hh @@ -7,8 +7,8 @@ #pragma once -#include #include +#include #include #include diff --git a/utils/double-decker.hh b/utils/double-decker.hh index 712019c2112c..43410f4a6ed8 100644 --- a/utils/double-decker.hh +++ b/utils/double-decker.hh @@ -9,7 +9,6 @@ #pragma once #include -#include #include "utils/assert.hh" #include "utils/bptree.hh" #include "utils/intrusive-array.hh" diff --git a/utils/dynamic_bitset.hh b/utils/dynamic_bitset.hh index 2493239e6399..223210276516 100644 --- a/utils/dynamic_bitset.hh +++ b/utils/dynamic_bitset.hh @@ -8,10 +8,10 @@ #pragma once +#include #include #include -#include #include namespace utils { diff --git a/utils/error_injection.hh b/utils/error_injection.hh index cee42fe05c73..0ac9eea91924 100644 --- a/utils/error_injection.hh +++ b/utils/error_injection.hh @@ -12,7 +12,6 @@ #include "utils/from_chars_exactly.hh" #include #include -#include #include #include #include diff --git a/utils/exception_container.hh b/utils/exception_container.hh index a994f66b76cf..13642227df7a 100644 --- a/utils/exception_container.hh +++ b/utils/exception_container.hh @@ -9,14 +9,10 @@ #pragma once #include -#include #include #include -#include #include #include -#include -#include #include "utils/variant_element.hh" namespace utils { diff --git a/utils/exceptions.hh b/utils/exceptions.hh index 950e54d11e32..3ad6facf6802 100644 --- a/utils/exceptions.hh +++ b/utils/exceptions.hh @@ -8,7 +8,8 @@ #pragma once -#include +// for checking __GLIBCXX__ +#include #if defined(__GLIBCXX__) && (defined(__x86_64__) || defined(__aarch64__)) #define OPTIMIZED_EXCEPTION_HANDLING_AVAILABLE @@ -22,8 +23,6 @@ #endif #endif -#include -#include #include #include diff --git a/utils/exponential_backoff_retry.hh b/utils/exponential_backoff_retry.hh index 9f26080b21ca..f94c959a2eea 100644 --- a/utils/exponential_backoff_retry.hh +++ b/utils/exponential_backoff_retry.hh @@ -10,9 +10,9 @@ #pragma once #include +#include #include #include -#include #include "seastarx.hh" #include diff --git a/utils/file_lock.hh b/utils/file_lock.hh index 95e3548dc04a..f5baedb3553a 100644 --- a/utils/file_lock.hh +++ b/utils/file_lock.hh @@ -9,7 +9,6 @@ #pragma once #include -#include #include #include #include diff --git a/utils/flush_queue.hh b/utils/flush_queue.hh index 5256541827c8..80635f605c3c 100644 --- a/utils/flush_queue.hh +++ b/utils/flush_queue.hh @@ -11,7 +11,6 @@ #include "utils/assert.hh" #include #include -#include #include #include diff --git a/utils/gz/crc_combine.hh b/utils/gz/crc_combine.hh index 1eba3aa1e05b..f424f8b9c690 100644 --- a/utils/gz/crc_combine.hh +++ b/utils/gz/crc_combine.hh @@ -11,7 +11,6 @@ #include #include -#include /* * Computes CRC32 (gzip format, RFC 1952) of a compound bitstream M composed by diff --git a/utils/hashers.hh b/utils/hashers.hh index 91ce2f1c955d..f3f2363a4dba 100644 --- a/utils/hashers.hh +++ b/utils/hashers.hh @@ -9,9 +9,8 @@ #pragma once #include -#include "bytes.hh" +#include "bytes_fwd.hh" #include "utils/hashing.hh" -#include template concept HasherReturningBytes = HasherReturning; diff --git a/utils/histogram.hh b/utils/histogram.hh index bddaf8cd8de7..b70e640f8740 100644 --- a/utils/histogram.hh +++ b/utils/histogram.hh @@ -12,7 +12,6 @@ #include "latency.hh" #include #include -#include #include "seastarx.hh" #include "estimated_histogram.hh" diff --git a/utils/http.hh b/utils/http.hh index bf12e007ec4c..7bf4c6e6d93d 100644 --- a/utils/http.hh +++ b/utils/http.hh @@ -8,7 +8,6 @@ #pragma once -#include #include #include #include diff --git a/utils/i_filter.hh b/utils/i_filter.hh index 2a39b912fda6..083661751509 100644 --- a/utils/i_filter.hh +++ b/utils/i_filter.hh @@ -10,8 +10,7 @@ #pragma once #include -#include "bytes.hh" -#include +#include "bytes_fwd.hh" namespace utils { diff --git a/utils/immutable-collection.hh b/utils/immutable-collection.hh index 11e733fc1627..1134116462ad 100644 --- a/utils/immutable-collection.hh +++ b/utils/immutable-collection.hh @@ -7,8 +7,6 @@ */ #pragma once -#include -#include #include namespace utils { diff --git a/utils/intrusive-array.hh b/utils/intrusive-array.hh index 5ea84e33cded..88f302466f75 100644 --- a/utils/intrusive-array.hh +++ b/utils/intrusive-array.hh @@ -8,12 +8,13 @@ #pragma once -#include #include -#include +#include +#include +#include +#include #include "utils/assert.hh" -#include "utils/allocation_strategy.hh" #include "utils/collection-concepts.hh" template diff --git a/utils/large_bitset.hh b/utils/large_bitset.hh index 453ba1ecf33f..e0090064b8fc 100644 --- a/utils/large_bitset.hh +++ b/utils/large_bitset.hh @@ -12,11 +12,8 @@ #pragma once #include -#include #include "utils/chunked_vector.hh" -using namespace seastar; - class large_bitset { using int_type = uint64_t; static constexpr size_t bits_per_int() { diff --git a/utils/like_matcher.hh b/utils/like_matcher.hh index 32ceb435272a..b2db21eb04ea 100644 --- a/utils/like_matcher.hh +++ b/utils/like_matcher.hh @@ -8,7 +8,7 @@ #pragma once -#include "bytes.hh" +#include "bytes_fwd.hh" #include /// Implements text LIKE pattern. diff --git a/utils/lister.cc b/utils/lister.cc index 37464fe32fb5..e63ead550b94 100644 --- a/utils/lister.cc +++ b/utils/lister.cc @@ -1,5 +1,6 @@ #include #include +#include #include #include #include diff --git a/utils/lister.hh b/utils/lister.hh index 35fcca9b6c34..83750a1c872c 100644 --- a/utils/lister.hh +++ b/utils/lister.hh @@ -8,13 +8,9 @@ #pragma once -#include #include -#include #include -#include #include -#include #include #include "enum_set.hh" #include "seastarx.hh" diff --git a/utils/loading_cache.hh b/utils/loading_cache.hh index e5af274a0560..8f8533679f99 100644 --- a/utils/loading_cache.hh +++ b/utils/loading_cache.hh @@ -9,18 +9,15 @@ #pragma once #include -#include #include #include #include #include #include -#include #include -#include -#include +#include #include #include diff --git a/utils/loading_shared_values.hh b/utils/loading_shared_values.hh index 1ab5a41cc55a..a1334c5726c3 100644 --- a/utils/loading_shared_values.hh +++ b/utils/loading_shared_values.hh @@ -10,7 +10,6 @@ #include "utils/assert.hh" #include -#include #include #include #include diff --git a/utils/logalloc.cc b/utils/logalloc.cc index e84552b0d1f5..f57d6396475d 100644 --- a/utils/logalloc.cc +++ b/utils/logalloc.cc @@ -18,8 +18,9 @@ #include #include #include -#include +#include #include +#include #include #include #include diff --git a/utils/logalloc.hh b/utils/logalloc.hh index 52937d5f20f0..1685adff4b9f 100644 --- a/utils/logalloc.hh +++ b/utils/logalloc.hh @@ -10,14 +10,10 @@ #include #include -#include -#include +#include #include -#include -#include #include "allocation_strategy.hh" #include "seastarx.hh" -#include "db/timeout_clock.hh" #include "utils/assert.hh" #include "utils/entangled.hh" #include "utils/memory_limit_reached.hh" diff --git a/utils/lsa/chunked_managed_vector.hh b/utils/lsa/chunked_managed_vector.hh index e1f06a4b5119..2e8a5640f378 100644 --- a/utils/lsa/chunked_managed_vector.hh +++ b/utils/lsa/chunked_managed_vector.hh @@ -21,7 +21,6 @@ #include #include #include -#include namespace lsa { diff --git a/utils/multiprecision_int.hh b/utils/multiprecision_int.hh index 6fc46aa4e70a..e5bb7038d239 100644 --- a/utils/multiprecision_int.hh +++ b/utils/multiprecision_int.hh @@ -9,8 +9,8 @@ #pragma once #include -#include #include +#include namespace utils { diff --git a/utils/murmur_hash.hh b/utils/murmur_hash.hh index 0ebb2029e559..3e668bc39191 100644 --- a/utils/murmur_hash.hh +++ b/utils/murmur_hash.hh @@ -13,7 +13,7 @@ #include #include -#include "bytes.hh" +#include "bytes_fwd.hh" /** * This is a very fast, non-cryptographic hash suitable for general hash-based diff --git a/utils/on_internal_error.hh b/utils/on_internal_error.hh index 1c10254f1afa..1e0af4f107d3 100644 --- a/utils/on_internal_error.hh +++ b/utils/on_internal_error.hh @@ -19,7 +19,6 @@ #pragma once -#include "utils/assert.hh" #include namespace utils { @@ -32,4 +31,4 @@ namespace utils { /// current backtrace. [[noreturn]] void on_internal_error(std::string_view reason); -} \ No newline at end of file +} diff --git a/utils/rate_limiter.hh b/utils/rate_limiter.hh index b54797f18890..c2906104c38d 100644 --- a/utils/rate_limiter.hh +++ b/utils/rate_limiter.hh @@ -10,7 +10,6 @@ #include #include -#include #include "seastarx.hh" namespace utils { diff --git a/utils/result.hh b/utils/result.hh index ba73bcf8d513..444a23e38e00 100644 --- a/utils/result.hh +++ b/utils/result.hh @@ -11,8 +11,9 @@ // Basic utilities which allow to start working with boost::outcome::result // in conjunction with our exception_container. -#include -#include +#include +#include +#include #include #include #include "utils/exception_container.hh" diff --git a/utils/result_loop.hh b/utils/result_loop.hh index 2cbebf33622a..2c1e2bd1b6a1 100644 --- a/utils/result_loop.hh +++ b/utils/result_loop.hh @@ -10,9 +10,9 @@ #include #include +#include #include #include -#include #include #include "utils/result.hh" diff --git a/utils/result_try.hh b/utils/result_try.hh index c0813aa16e7c..bf2b81e7aa2a 100644 --- a/utils/result_try.hh +++ b/utils/result_try.hh @@ -10,7 +10,6 @@ #include #include -#include "utils/assert.hh" #include "utils/result.hh" namespace utils { diff --git a/utils/rjson.cc b/utils/rjson.cc index ce34920ecf58..18bab3ef088a 100644 --- a/utils/rjson.cc +++ b/utils/rjson.cc @@ -16,6 +16,7 @@ #endif #include +#include namespace rjson { diff --git a/utils/rjson.hh b/utils/rjson.hh index 1a08f6292e4a..3f47a5d4f4a9 100644 --- a/utils/rjson.hh +++ b/utils/rjson.hh @@ -27,6 +27,7 @@ */ #include +#include #include #include #include @@ -65,7 +66,6 @@ public: #include #include #include -#include #include #include #include diff --git a/utils/s3/client.cc b/utils/s3/client.cc index 0789821107d4..2bf4e461148f 100644 --- a/utils/s3/client.cc +++ b/utils/s3/client.cc @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include diff --git a/utils/s3/client.hh b/utils/s3/client.hh index cf56852a4d69..60b2df4529ac 100644 --- a/utils/s3/client.hh +++ b/utils/s3/client.hh @@ -9,9 +9,9 @@ #pragma once #include +#include #include #include -#include #include #include #include diff --git a/utils/stall_free.hh b/utils/stall_free.hh index 5194cb6b7b48..f106100d4863 100644 --- a/utils/stall_free.hh +++ b/utils/stall_free.hh @@ -12,7 +12,6 @@ #include #include #include -#include #include #include #include diff --git a/utils/to_string.hh b/utils/to_string.hh index 39b8da0f04e2..37f042463ed8 100644 --- a/utils/to_string.hh +++ b/utils/to_string.hh @@ -8,8 +8,8 @@ #pragma once -#include -#include +#include +#include template <> struct fmt::formatter : fmt::formatter { auto format(std::strong_ordering, fmt::format_context& ctx) const -> decltype(ctx.out()); diff --git a/utils/top_k.hh b/utils/top_k.hh index dd0fe97ee0be..43aafa8d61ed 100644 --- a/utils/top_k.hh +++ b/utils/top_k.hh @@ -51,7 +51,6 @@ #include #include -#include #include "utils/assert.hh" #include "utils/chunked_vector.hh" diff --git a/utils/user_provided_param.hh b/utils/user_provided_param.hh index 08fdac8da358..501a18e2a4a7 100644 --- a/utils/user_provided_param.hh +++ b/utils/user_provided_param.hh @@ -10,7 +10,6 @@ #include -#include #include #include "enum_set.hh" diff --git a/utils/utf8.hh b/utils/utf8.hh index 1a6b86b574fa..de78a7135a67 100644 --- a/utils/utf8.hh +++ b/utils/utf8.hh @@ -12,7 +12,7 @@ #pragma once #include -#include "bytes.hh" +#include "bytes_fwd.hh" #include "fragment_range.hh" namespace utils { diff --git a/utils/uuid.cc b/utils/uuid.cc index f5edc1ab60fb..16706f0d84ed 100644 --- a/utils/uuid.cc +++ b/utils/uuid.cc @@ -14,6 +14,7 @@ #include #include #include +#include #include #include "marshal_exception.hh" diff --git a/utils/xx_hasher.hh b/utils/xx_hasher.hh index 446bcb78e2e3..ea83b50ad1e1 100644 --- a/utils/xx_hasher.hh +++ b/utils/xx_hasher.hh @@ -8,7 +8,7 @@ #pragma once -#include "bytes.hh" +#include "bytes_fwd.hh" #include "utils/serialization.hh" #pragma GCC diagnostic push From 92c6c8a32fc290e167aa1a79a2c0dec9c01e79db Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 14 Jan 2025 09:15:52 +0800 Subject: [PATCH 286/397] ent,main: fix misspellings these misspellings are identified by codespell. they are either in comment or logging messages. let's fix them. Signed-off-by: Kefu Chai --- ent/encryption/encrypted_file_impl.cc | 2 +- ent/encryption/encryption.cc | 2 +- ent/encryption/encryption_config.cc | 2 +- ent/encryption/gcp_host.cc | 6 +++--- ent/encryption/kmip_host.cc | 4 ++-- main.cc | 2 +- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/ent/encryption/encrypted_file_impl.cc b/ent/encryption/encrypted_file_impl.cc index 7c3cf86d49e1..db6853fb3582 100644 --- a/ent/encryption/encrypted_file_impl.cc +++ b/ent/encryption/encrypted_file_impl.cc @@ -45,7 +45,7 @@ static inline bool is_aligned(size_t n, size_t a) { * * Since all writes are assumed to be done by us, and must be aligned, * we can assume in turn that any resizing should be made by our truncate - * method. If we attept to truncate to a size not a multiple of our + * method. If we attempt to truncate to a size not a multiple of our * _key_ block size (typically 16), we add the same size to the actual * truncation size. * On read we then check the file size. If we're reading from a file diff --git a/ent/encryption/encryption.cc b/ent/encryption/encryption.cc index 0031b068433c..2ca120af336d 100644 --- a/ent/encryption/encryption.cc +++ b/ent/encryption/encryption.cc @@ -966,7 +966,7 @@ future> register_extensions(const db::co // modify schemas for tables holding sensitive data to use encryption w. key described // by the opts. - // since schemas are duplicated across shards, we must call to each shard and augument + // since schemas are duplicated across shards, we must call to each shard and augment // them all. // Since we are in pre-init phase, this should be safe. f = f.then([opts = *opts, &exts] { diff --git a/ent/encryption/encryption_config.cc b/ent/encryption/encryption_config.cc index 46e819025d2f..3940ffa73190 100644 --- a/ent/encryption/encryption_config.cc +++ b/ent/encryption/encryption_config.cc @@ -83,7 +83,7 @@ Actual connection can be either an explicit endpoint (:), or selecte If aws_use_ec2_region is true, regions is instead queried from EC2 metadata. -Authentication can be explicit with aws_access_key_id and aws_secret_access_key. Either secret or both can be ommitted +Authentication can be explicit with aws_access_key_id and aws_secret_access_key. Either secret or both can be omitted in which case the provider will try to read them from AWS credentials in ~/.aws/credentials If aws_use_ec2_credentials is true, authentication is instead queried from EC2 metadata. diff --git a/ent/encryption/gcp_host.cc b/ent/encryption/gcp_host.cc index 23b2c3964471..1faca24e7072 100644 --- a/ent/encryption/gcp_host.cc +++ b/ent/encryption/gcp_host.cc @@ -581,7 +581,7 @@ future encryption::gcp_host::impl::send_request(std::string_view u rjson::value v; co_await send_request(uri, std::move(body), content_type, [&](const http::reply& rep, std::string_view s) { if (rep._status != http::reply::status_type::ok) { - gcp_log.trace("Got unexpected reponse ({})", rep._status); + gcp_log.trace("Got unexpected response ({})", rep._status); for (auto& [k, v] : rep._headers) { gcp_log.trace("{}: {}", k, v); } @@ -660,7 +660,7 @@ future<> encryption::gcp_host::impl::send_request(std::string_view uri, std::str auto&lh = handler; auto lin = std::move(in); auto result = co_await util::read_entire_stream_contiguous(lin); - gcp_log.trace("Got reponse {}: {}", int(rep._status), result); + gcp_log.trace("Got response {}: {}", int(rep._status), result); lh(rep, result); }); @@ -874,7 +874,7 @@ future encryption::gcp_host::impl:: * I.e. something like: * mykeyring:mykey:e56sadfafa3324ff=/wfsdfwssdf * - * The actual data key can be retreived by doing a KMS "Decrypt" of the data blob part + * The actual data key can be retrieved by doing a KMS "Decrypt" of the data blob part * using the KMS key referenced by the key ID. This gives back actual key data that can * be used to create a symmetric_key with algo, length etc as specified by metadata. * diff --git a/ent/encryption/kmip_host.cc b/ent/encryption/kmip_host.cc index 1e2bf5dd0b00..bb467d88829d 100644 --- a/ent/encryption/kmip_host.cc +++ b/ent/encryption/kmip_host.cc @@ -334,7 +334,7 @@ future<> kmip_host::impl::connection::connect() { return seastar::net::dns::resolve_name(name).then([this, cred, port](seastar::net::inet_address addr) { return seastar::tls::connect(cred, seastar::ipv4_addr{addr, uint16_t(port)}).then([this](seastar::connected_socket s) { kmip_log.debug("Successfully connected {}", _host); - // #998 Set keepalive to try avoiding connection going stale inbetween commands. + // #998 Set keepalive to try avoiding connection going stale in between commands. s.set_keepalive_parameters(net::tcp_keepalive_params{60s, 60s, 10}); s.set_keepalive(true); _input = s.input(); @@ -564,7 +564,7 @@ future kmip_host::impl::do_cmd(KMIP_CMD* cmd, con_ptr cp, Func& f, bool ret release(cmd, cp, retain_connection_after_command); return make_ready_future(res); default: - // error. connection is dicarded. close it. + // error. connection is discarded. close it. return cp->close().then_wrapped([cp, res](auto f) { // ignore any exception thrown from the close. // ensure we provide the kmip error instead. diff --git a/main.cc b/main.cc index fbc7ac992eea..ebad2fa7f340 100644 --- a/main.cc +++ b/main.cc @@ -2480,7 +2480,7 @@ int main(int ac, char** av) { } // We have to override p11-kit config path before p11-kit initialization. - // And the initialization will invoke on seastar initalization, so it has to + // And the initialization will invoke on seastar initialization, so it has to // be before app.run() // #3583 - need to potentially ensure this for tools as well, since at least // sstable* might need crypto libraries. From 3b7a991f7407c966b698e67d826a02f818704c92 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 14 Jan 2025 10:25:24 +0800 Subject: [PATCH 287/397] ent/encryption: rename "sie" to "get_opt" "sie" is the short for "system info encryption". it is a wrapper around a `opts` map so we can get the individual option by providing a default value via an `optional<>` return value. but "sie" could be difficult to understand without more context. and it is used like a function -- we get the individual option using its operator(). so, in order to improve the readability, in this change, we rename it to "get_opt". Signed-off-by: Kefu Chai --- ent/encryption/encryption.cc | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/ent/encryption/encryption.cc b/ent/encryption/encryption.cc index 2ca120af336d..c295473880f0 100644 --- a/ent/encryption/encryption.cc +++ b/ent/encryption/encryption.cc @@ -936,17 +936,17 @@ future> register_extensions(const db::co auto maybe_get_options = [&](const utils::config_file::string_map& map, const sstring& what) -> std::optional { options opts(map.begin(), map.end()); - opt_wrapper sie(opts); - if (!::strcasecmp(sie("enabled").value_or("false").c_str(), "false")) { + opt_wrapper get_opt(opts); + if (!::strcasecmp(get_opt("enabled").value_or("false").c_str(), "false")) { return std::nullopt; } // commitlog/system table encryption/global user encryption should not use replicated keys, // We default to local keys, but KMIP/KMS is ok as well (better in fact). - opts[KEY_PROVIDER] = sie(KEY_PROVIDER).value_or(LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY); - if (opts[KEY_PROVIDER] == LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY && !sie(SECRET_KEY_FILE)) { + opts[KEY_PROVIDER] = get_opt(KEY_PROVIDER).value_or(LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY); + if (opts[KEY_PROVIDER] == LOCAL_FILE_SYSTEM_KEY_PROVIDER_FACTORY && !get_opt(SECRET_KEY_FILE)) { // system encryption uses different key folder than user tables. // explicitly set the key file path - opts[SECRET_KEY_FILE] = (bfs::path(cfg.system_key_directory()) / bfs::path("system") / bfs::path(sie("key_name").value_or("system_table_keytab"))).string(); + opts[SECRET_KEY_FILE] = (bfs::path(cfg.system_key_directory()) / bfs::path("system") / bfs::path(get_opt("key_name").value_or("system_table_keytab"))).string(); } // forbid replicated. we cannot guarantee being able to open sstables on populate if (opts[KEY_PROVIDER] == REPLICATED_KEY_PROVIDER_FACTORY) { From 775411ac565e17b894172f0877de436b2a693f07 Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Thu, 9 Jan 2025 17:41:21 +0100 Subject: [PATCH 288/397] raft: Add run_op_with_retry in raft_group0. Since when calling `modify_config` it's quite often we need to do retries, to avoid code duplication, a function wrapper that allows a function to be called with automatic retries in case of failures was added. --- service/raft/raft_group0.cc | 38 +++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index 1b41c62db073..c724f3a2ec4e 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -95,6 +95,44 @@ namespace service { static logging::logger group0_log("raft_group0"); static logging::logger upgrade_log("raft_group0_upgrade"); +namespace { + +constexpr std::chrono::milliseconds default_retry_period{10}; // 10 milliseconds +constexpr std::chrono::seconds default_max_retry_period{1}; // 1 second +constexpr std::chrono::seconds default_max_total_timeout{300}; // 5 minutes + +enum class operation_result : uint8_t { success, failure }; + +future<> run_op_with_retry(abort_source& as, auto&& op, const sstring op_name, + const std::optional max_total_timeout = default_max_total_timeout, std::chrono::milliseconds retry_period = default_retry_period, + const std::chrono::seconds max_retry_period = default_max_retry_period) { + const auto start = lowres_clock::now(); + while (true) { + as.check(); + const operation_result result = co_await op(); + if (result == operation_result::success) { + co_return; + } + + if (max_total_timeout) { + const auto elapsed = lowres_clock::now() - start; + if (elapsed > *max_total_timeout) { + on_internal_error(group0_log, + format("{} timed out after retrying for {} seconds", op_name, std::chrono::duration_cast(elapsed).count())); + } + } + + retry_period *= 2; + if (retry_period > max_retry_period) { + retry_period = max_retry_period; + } + co_await sleep_abortable(retry_period, as); + } + std::unreachable(); +} + +} // namespace + // TODO: change the links from master to stable/5.2 after 5.2 is released const char* const raft_upgrade_doc = "https://docs.scylladb.com/master/architecture/raft.html#verifying-that-the-internal-raft-upgrade-procedure-finished-successfully"; static const auto raft_manual_recovery_doc = "https://docs.scylladb.com/master/architecture/raft.html#raft-manual-recovery-procedure"; From 16053a86f054f898ede64dce514bdb2b50d6c794 Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Fri, 10 Jan 2025 17:49:42 +0100 Subject: [PATCH 289/397] test: Add test to check that a node does not fail on unknown commit status error when starting up. Test that a node is starting successfully if while joining a cluster and becoming a voter, it receives an unknown commit status error. Test for scylladb/scylladb#20814 --- raft/server.cc | 4 ++ .../test_error_becoming_voter.py | 43 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 test/topology_custom/test_error_becoming_voter.py diff --git a/raft/server.cc b/raft/server.cc index 2a745d3828a4..e85f8862a443 100644 --- a/raft/server.cc +++ b/raft/server.cc @@ -857,6 +857,10 @@ future server_impl::execute_modify_config(server_id from, } future<> server_impl::modify_config(std::vector add, std::vector del, seastar::abort_source* as) { + utils::get_local_injector().inject("raft/throw_commit_status_unknown_in_modify_config", [] { + throw raft::commit_status_unknown(); + }); + if (!_config.enable_forwarding) { const auto leader = _fsm->current_leader(); if (leader != _id) { diff --git a/test/topology_custom/test_error_becoming_voter.py b/test/topology_custom/test_error_becoming_voter.py new file mode 100644 index 000000000000..745ba68de8e8 --- /dev/null +++ b/test/topology_custom/test_error_becoming_voter.py @@ -0,0 +1,43 @@ +# +# Copyright (C) 2025-present ScyllaDB +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# +import logging +import pytest +import asyncio +import time + +from cassandra import ConsistencyLevel # type: ignore +from cassandra.query import SimpleStatement # type: ignore +from test.pylib.manager_client import ManagerClient +from test.pylib.util import wait_for_cql_and_get_hosts + + +logger = logging.getLogger(__name__) + + +@pytest.mark.xfail(reason="issue #20814") +@pytest.mark.asyncio +async def test_error_while_becoming_voter(request: pytest.FixtureRequest, manager: ManagerClient) -> None: + """ + Test that a node is starting successfully if while joining a cluster and becoming a voter, it + receives an unknown commit status error. + Issue https://github.com/scylladb/scylladb/issues/20814 + + 1. Create a new cluster, start 2 nodes normally. + 2. Run one node with error injection for throwing an exception commit_status_unknown in modify_config, + so that after bootstrapping the node would get a commit_status_unknown error. + 3. Make sure the node was started successfully. In case the error with the handling of commit_status_unknown is + not handled properly, the node will fail to start. + + """ + logger.info("Creating a new cluster") + await manager.servers_add(2) + + srv = await manager.server_add(config={ + "error_injections_at_startup": [ + {"name": "raft/throw_commit_status_unknown_in_modify_config", "one_shot": True} + ] + }) + await manager.server_start(srv.server_id) From f021e16d0cdf5cb7fa886aa2796e75c17b4e1a61 Mon Sep 17 00:00:00 2001 From: Jenkins Promoter Date: Wed, 15 Jan 2025 04:26:42 +0200 Subject: [PATCH 290/397] Update pgo profiles - x86_64 --- pgo/profiles/x86_64/profile.profdata.xz | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pgo/profiles/x86_64/profile.profdata.xz b/pgo/profiles/x86_64/profile.profdata.xz index 76d83cd40bdc..6516f6bf648c 100644 --- a/pgo/profiles/x86_64/profile.profdata.xz +++ b/pgo/profiles/x86_64/profile.profdata.xz @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:792d30bc07d3db08d947dcc05f1a2829f9ac4f9d0786402bb8f9acc63670ace3 -size 5638004 +oid sha256:daaeb5d5740d25ef0dcc79034959382042d007d569389940d0a7baf51fa1da59 +size 5795820 From f5f15c6d07fca89059a7ba20cad7b36a7bf8b661 Mon Sep 17 00:00:00 2001 From: Jenkins Promoter Date: Wed, 15 Jan 2025 04:49:41 +0200 Subject: [PATCH 291/397] Update pgo profiles - aarch64 --- pgo/profiles/aarch64/profile.profdata.xz | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pgo/profiles/aarch64/profile.profdata.xz b/pgo/profiles/aarch64/profile.profdata.xz index c4dfaa9a1e20..1edf1d64fb95 100644 --- a/pgo/profiles/aarch64/profile.profdata.xz +++ b/pgo/profiles/aarch64/profile.profdata.xz @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:87fea54e41a383693359d063fae5a97b365fc7dda6b264a6e6a41bc12ea67bb5 -size 5634076 +oid sha256:7cc51cd5c699c529239d1fe6fc3a7ec5dfceb3389236257388d07415f1870340 +size 5800128 From 0d399702c7ed93ecb3bc3aeeaab6921b48a8b8e7 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 14 Jan 2025 16:36:21 +0800 Subject: [PATCH 292/397] api: include used header when building the tree on fedora 41, we could have following build failure: ``` FAILED: api/CMakeFiles/api.dir/Debug/system.cc.o /usr/bin/clang++ -DDEBUG -DDEBUG_LSA_SANITIZER -DSANITIZE -DSCYLLA_BUILD_MODE=debug -DSCYLLA_ENABLE_ERROR_INJECTION -DXXH_PRIVATE_API -DCMAKE_INTDIR=\"Debug\" -I/home/kefu/dev/scylladb -I/home/kefu/dev/scylladb/build/gen -isystem /home/kefu/dev/scylladb/seastar/include -isystem /home/kefu/dev/scylladb/build/Debug/seastar/gen/include -isystem /home/kefu/dev/scylladb/build/rust -isystem /home/kefu/dev/scylladb/abseil -I/usr/include/p11-kit-1 -g -Og -g -gz -std=gnu++23 -fvisibility=hidden -Wall -Werror -Wextra -Wno-error=deprecated-declarations -Wimplicit-fallthrough -Wno-c++11-narrowing -Wno-deprecated-copy -Wno-mismatched-tags -Wno-missing-field-initializers -Wno-overloaded-virtual -Wno-unsupported-friend -Wno-enum-constexpr-conversion -Wno-explicit-specialization-storage-class -Wno-unused-parameter -ffile-prefix-map=/home/kefu/dev/scylladb/build=. -march=westmere -Xclang -fexperimental-assignment-tracking=disabled -std=gnu++23 -Werror=unused-result -fstack-clash-protection -fsanitize=address -fsanitize=undefined -DSEASTAR_API_LEVEL=7 -DSEASTAR_BUILD_SHARED_LIBS -DSEASTAR_SSTRING -DSEASTAR_LOGGER_COMPILE_TIME_FMT -DSEASTAR_SCHEDULING_GROUPS_COUNT=19 -DSEASTAR_DEBUG -DSEASTAR_DEFAULT_ALLOCATOR -DSEASTAR_SHUFFLE_TASK_QUEUE -DSEASTAR_DEBUG_SHARED_PTR -DSEASTAR_DEBUG_PROMISE -DSEASTAR_LOGGER_TYPE_STDOUT -DSEASTAR_TYPE_ERASE_MORE -DBOOST_PROGRAM_OPTIONS_NO_LIB -DBOOST_PROGRAM_OPTIONS_DYN_LINK -DBOOST_THREAD_NO_LIB -DBOOST_THREAD_DYN_LINK -DFMT_SHARED -DWITH_GZFILEOP -MD -MT api/CMakeFiles/api.dir/Debug/system.cc.o -MF api/CMakeFiles/api.dir/Debug/system.cc.o.d -o api/CMakeFiles/api.dir/Debug/system.cc.o -c /home/kefu/dev/scylladb/api/system.cc /home/kefu/dev/scylladb/api/system.cc:116:47: error: no member named 'lexical_cast' in namespace 'boost' 116 | logging::log_level level = boost::lexical_cast(std::string(req.get_query_param("level"))); | ~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:116:78: error: expected '(' for function-style cast or type construction 116 | logging::log_level level = boost::lexical_cast(std::string(req.get_query_param("level"))); | ~~~~~~~~~~~~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:118:25: error: no type named 'bad_lexical_cast' in namespace 'boost' 118 | } catch (boost::bad_lexical_cast& e) { | ~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:136:47: error: no member named 'lexical_cast' in namespace 'boost' 136 | logging::log_level level = boost::lexical_cast(std::string(req.get_query_param("level"))); | ~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:136:78: error: expected '(' for function-style cast or type construction 136 | logging::log_level level = boost::lexical_cast(std::string(req.get_query_param("level"))); | ~~~~~~~~~~~~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:140:25: error: no type named 'bad_lexical_cast' in namespace 'boost' 140 | } catch (boost::bad_lexical_cast& e) { | ~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:148:47: error: no member named 'lexical_cast' in namespace 'boost' 148 | logging::log_level level = boost::lexical_cast(std::string(req.get_query_param("level"))); | ~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:148:78: error: expected '(' for function-style cast or type construction 148 | logging::log_level level = boost::lexical_cast(std::string(req.get_query_param("level"))); | ~~~~~~~~~~~~~~~~~~^ /home/kefu/dev/scylladb/api/system.cc:150:25: error: no type named 'bad_lexical_cast' in namespace 'boost' 150 | } catch (boost::bad_lexical_cast& e) { | ~~~~~~~^ ``` in this change, we include the used header to address the build failure. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22303 --- api/system.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/api/system.cc b/api/system.cc index 735216c917f0..1b055c468ac8 100644 --- a/api/system.cc +++ b/api/system.cc @@ -13,6 +13,7 @@ #include "db/sstables-format-selector.hh" #include +#include #include #include #include From f2a53d6a2c0f9d4aeaa95072e0bf772f6ea93539 Mon Sep 17 00:00:00 2001 From: Takuya ASADA Date: Mon, 12 Jun 2023 18:27:18 +0900 Subject: [PATCH 293/397] dist: make p11-kit-trust.so able to work in relocatable package Currently, our relocatable package doesn't contains p11-kit-trust.so since it dynamically loaded, not showing on "ldd" results (Relocatable packaging script finds dependent libraries by "ldd"). So we need to add it on create-relocatable-pacakge.py. Also, we have two more problems: 1. p11 module load path is defined as "/usr/lib64/pkcs11", not referencing to /opt/scylladb/libreloc (and also RedHat variants uses different path than Debian variants) 2. ca-trust-source path is configured on build time (on Fedora), it compatible with RedHat variants but not compatible with Debian variants To solve these problems, we need to override default p11-kit configuration. To do so, we need to add an configuration file to /opt/scylladb/share/pkcs11/modules/p11-kit-trust.module. Also, ofcause p11-kit doesn't reference /opt/scylladb by default, we need to override load path by p11_kit_override_system_files(). On the configuration file, we can specify module load path by "modules: ", and also we can specify ca-trust-source path by "x-init-reservied: paths=". Fixes scylladb/scylladb#13904 Closes scylladb/scylladb#22302 --- dist/debian/debian/rules | 2 +- dist/debian/debian/scylla-server.install | 1 + dist/redhat/scylla.spec | 3 +- install.sh | 63 +++++++++++++++++++++++- main.cc | 10 ++++ scripts/create-relocatable-package.py | 6 +++ 6 files changed, 81 insertions(+), 4 deletions(-) diff --git a/dist/debian/debian/rules b/dist/debian/debian/rules index 20a81b38f26e..dd4a850cadce 100755 --- a/dist/debian/debian/rules +++ b/dist/debian/debian/rules @@ -19,7 +19,7 @@ override_dh_auto_clean: override_dh_auto_install: dh_auto_install - ./scylla/install.sh --packaging --root "$(CURDIR)/debian/tmp" $(install_arg) --sysconfdir "/etc/default" + ./scylla/install.sh --packaging --root "$(CURDIR)/debian/tmp" $(install_arg) --sysconfdir "/etc/default" --p11-trust-paths /etc/ssl/certs/ca-certificates.crt # don't use default sysconfig file, use Debian version cp scylla/dist/debian/sysconfig/scylla-housekeeping $(CURDIR)/debian/tmp/etc/default/ diff --git a/dist/debian/debian/scylla-server.install b/dist/debian/debian/scylla-server.install index f76f849c0c02..d9cf58e8d012 100644 --- a/dist/debian/debian/scylla-server.install +++ b/dist/debian/debian/scylla-server.install @@ -2,6 +2,7 @@ etc/default/scylla-server etc/default/scylla-housekeeping etc/scylla.d/*.conf etc/bash_completion.d/nodetool-completion +opt/scylladb/share/p11-kit/modules/* opt/scylladb/share/doc/scylla/* opt/scylladb/share/doc/scylla/licenses/ usr/lib/systemd/system/*.timer diff --git a/dist/redhat/scylla.spec b/dist/redhat/scylla.spec index 8010a328dc53..67f8925aff6c 100644 --- a/dist/redhat/scylla.spec +++ b/dist/redhat/scylla.spec @@ -60,7 +60,7 @@ This package installs all required packages for ScyllaDB, including %if 0%{housekeeping} install_arg="--housekeeping" %endif -./install.sh --packaging --root "$RPM_BUILD_ROOT" $install_arg +./install.sh --packaging --root "$RPM_BUILD_ROOT" --p11-trust-paths /etc/pki/ca-trust/source:/usr/share/pki/ca-trust-source $install_arg %clean rm -rf $RPM_BUILD_ROOT @@ -111,6 +111,7 @@ ln -sfT /etc/scylla /var/lib/scylla/conf %config(noreplace) %{_sysconfdir}/sysconfig/scylla-housekeeping %attr(0755,root,root) %dir %{_sysconfdir}/scylla.d %config(noreplace) %{_sysconfdir}/scylla.d/*.conf +/opt/scylladb/share/p11-kit/modules/* /opt/scylladb/share/doc/scylla/* %{_unitdir}/scylla-fstrim.service %{_unitdir}/scylla-housekeeping-daily.service diff --git a/install.sh b/install.sh index fca2e747d05e..a68b31f5f3ec 100755 --- a/install.sh +++ b/install.sh @@ -30,6 +30,7 @@ Options: --supervisor enable supervisor to manage scylla processes --supervisor-log-to-stdout logging to stdout on supervisor --without-systemd skip installing systemd units + --p11-trust-paths specify trust path for p11-kit --help this helpful message EOF exit 1 @@ -71,6 +72,7 @@ supervisor=false supervisor_log_to_stdout=false without_systemd=false skip_systemd_check=false +p11_trust_paths= while [ $# -gt 0 ]; do case "$1" in @@ -121,6 +123,10 @@ while [ $# -gt 0 ]; do skip_systemd_check=true shift 1 ;; + "--p11-trust-paths") + p11_trust_paths="$2" + shift 2 + ;; "--help") shift 1 print_usage @@ -232,6 +238,17 @@ check_usermode_support() { . /etc/os-release +is_redhat_variant() { + is_redhat=0 + for i in $ID $ID_LIKE; do + if [ "$i" = "rhel" -o "$i" = "fedora" -o "$i" = "centos" ]; then + is_redhat=1 + break + fi + done + [ $is_redhat -eq 1 ] +} + is_debian_variant() { [ "$ID_LIKE" = "debian" -o "$ID" = "debian" ] } @@ -271,6 +288,30 @@ cd "$(dirname "$0")" product="$(cat ./SCYLLA-PRODUCT-FILE)" +if [ -z "$p11_trust_paths" ]; then + # our package builder is cross-distro, so we cannot detect distro by os-release + if $packaging; then + echo "Please specify --p11-trust-paths." + echo "The path can be get by following command:" + echo " pkg-config --variable p11_trust_paths p11-kit-1" + echo + print_usage + else + # for offline installer users we provide default p11-trust-paths + if is_redhat_variant; then + p11_trust_paths=/etc/pki/ca-trust/source:/usr/share/pki/ca-trust-source + elif is_debian_variant; then + p11_trust_paths=/etc/ssl/certs/ca-certificates.crt + else + echo "Please specify --p11-trust-paths." + echo "The path can be get by following command:" + echo " pkg-config --variable p11_trust_paths p11-kit-1" + echo + print_usage + fi + fi +fi + if [ -z "$prefix" ]; then if $nonroot; then prefix=~/scylladb @@ -303,6 +344,7 @@ if ! $nonroot; then rsysconfdir=$(realpath -m "$root/$sysconfdir") rusr=$(realpath -m "$root/usr") rsystemd=$(realpath -m "$rusr/lib/systemd/system") + rshare="$rprefix/share" rdoc="$rprefix/share/doc" rdata=$(realpath -m "$root/var/lib/scylla") rhkdata=$(realpath -m "$root/var/lib/scylla-housekeeping") @@ -310,6 +352,7 @@ else retc="$rprefix/etc" rsysconfdir="$rprefix/$sysconfdir" rsystemd="$HOME/.config/systemd/user" + rshare="$rprefix/share" rdoc="$rprefix/share/doc" rdata="$rprefix" fi @@ -401,7 +444,7 @@ for file in dist/common/scylla.d/*.conf; do installconfig 644 "$file" "$retc"/scylla.d done -install -d -m755 "$retc"/scylla "$rprefix/bin" "$rprefix/libexec" "$rprefix/libreloc" "$rprefix/scripts" "$rprefix/bin" +install -d -m755 "$retc"/scylla "$rprefix/bin" "$rprefix/libexec" "$rprefix/libreloc" "$rprefix/libreloc/pkcs11" "$rprefix/scripts" "$rprefix/bin" if ! $without_systemd; then install -m644 dist/common/systemd/scylla-fstrim.service -Dt "$rsystemd" install -m644 dist/common/systemd/scylla-housekeeping-daily.service -Dt "$rsystemd" @@ -412,10 +455,16 @@ if ! $without_systemd; then fi install -m755 seastar/scripts/seastar-cpu-map.sh -Dt "$rprefix"/scripts install -m755 seastar/dpdk/usertools/dpdk-devbind.py -Dt "$rprefix"/scripts -install -m755 libreloc/* -Dt "$rprefix/libreloc" +for i in $(find libreloc/ -maxdepth 1 -type f); do + install -m755 "$i" -Dt "$rprefix/libreloc" +done for lib in libreloc/*; do remove_rpath "$rprefix/$lib" done +for i in $(find libreloc/pkcs11/ -maxdepth 1 -type f); do + install -m755 "$i" -Dt "$rprefix/libreloc/pkcs11" +done + # some files in libexec are symlinks, which "install" dereferences # use cp -P for the symlinks instead. install -m755 libexec/* -Dt "$rprefix/libexec" @@ -464,6 +513,16 @@ PRODUCT="$product" EOS chmod 644 "$rprefix"/scripts/scylla_product.py +install -d -m755 "$rshare"/p11-kit/modules +cat << EOS > "$rshare"/p11-kit/modules/p11-kit-trust.module +module: $prefix/libreloc/pkcs11/p11-kit-trust.so +priority: 1 +trust-policy: yes +x-trust-lookup: pkcs11:library-description=PKCS%2311%20Kit%20Trust%20Module +disable-in: p11-kit-proxy +x-init-reserved: paths=$p11_trust_paths +EOS + if ! $nonroot && ! $without_systemd; then install -d -m755 "$retc"/systemd/system/scylla-server.service.d install -m644 dist/common/systemd/scylla-server.service.d/dependencies.conf -Dt "$retc"/systemd/system/scylla-server.service.d diff --git a/main.cc b/main.cc index fbc7ac992eea..4c3e63bd00f2 100644 --- a/main.cc +++ b/main.cc @@ -125,6 +125,8 @@ extern "C" { #include } +namespace fs = std::filesystem; + seastar::metrics::metric_groups app_metrics; using namespace std::chrono_literals; @@ -754,6 +756,14 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl print_starting_message(ac, av, parsed_opts); } + // We have to override p11-kit config path before p11-kit initialization. + // And the initialization will invoke on seastar initalization, so it has to + // be before app.run() + auto scylla_path = fs::read_symlink(fs::path("/proc/self/exe")); + auto p11_modules = scylla_path.parent_path().parent_path().append("share/p11-kit/modules"); + auto p11_modules_str = p11_modules.string(); + ::p11_kit_override_system_files(NULL, NULL, p11_modules_str.c_str(), NULL, NULL); + sharded token_metadata; sharded erm_factory; sharded mm_notifier; diff --git a/scripts/create-relocatable-package.py b/scripts/create-relocatable-package.py index 81a292bbcbca..b8e464aadd43 100755 --- a/scripts/create-relocatable-package.py +++ b/scripts/create-relocatable-package.py @@ -110,6 +110,8 @@ def reloc_add(ar, name, arcname=None): # manually add libthread_db for debugging thread libs.update({'libthread_db.so.1': os.path.realpath('/lib64/libthread_db.so')}) +# manually add p11-kit-trust.so since it will dynamically load +libs.update({'pkcs11/p11-kit-trust.so': '/lib64/pkcs11/p11-kit-trust.so'}) ld_so = libs['ld.so'] @@ -131,6 +133,10 @@ def reloc_add(ar, name, arcname=None): version_file.flush() ar.add(version_file.name, arcname='.relocatable_package_version') +with tempfile.TemporaryDirectory() as tmpdir: + os.symlink('./pkcs11/p11-kit-trust.so', f'{tmpdir}/libnssckbi.so') + ar.reloc_add(f'{tmpdir}/libnssckbi.so', arcname='libreloc/libnssckbi.so') + for exe in executables_scylla: basename = os.path.basename(exe) if not args.stripped: From 8c48f7ad62af06e2129cdf56abe058952e81e504 Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Thu, 9 Jan 2025 17:51:38 +0100 Subject: [PATCH 294/397] raft: Handle non-critical config update errors in when changing status to voter. When a node is bootstrapped and joins a cluster as a non-voter, errors can occur while committing a new Raft record, for instance, if the Raft leader changes during this time. These errors are not critical and should not cause a node crash, as the action can be retried. Fixes scylladb/scylladb#20814 --- service/raft/raft_group0.cc | 10 +++++++++- test/topology_custom/test_error_becoming_voter.py | 1 - 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index c724f3a2ec4e..389db449ff46 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -780,7 +780,15 @@ future<> raft_group0::finish_setup_after_join(service::storage_service& ss, cql3 // Just bootstrapped and joined as non-voter. Become a voter. auto pause_shutdown = _shutdown_gate.hold(); raft::server_address my_addr{my_id, {}}; - co_await _raft_gr.group0().modify_config({{my_addr, true}}, {}, &_abort_source); + co_await run_op_with_retry(_abort_source, [this, my_addr]() -> future { + try { + co_await _raft_gr.group0().modify_config({{my_addr, true}}, {}, &_abort_source); + } catch (const raft::commit_status_unknown& e) { + group0_log.info("finish_setup_after_join({}): modify_config returned \"{}\", retrying", my_addr, e); + co_return operation_result::failure; + } + co_return operation_result::success; + }, "finish_setup_after_join->modify_config", {}); group0_log.info("finish_setup_after_join: became a group 0 voter."); // No need to run `upgrade_to_group0()` since we must have bootstrapped with Raft diff --git a/test/topology_custom/test_error_becoming_voter.py b/test/topology_custom/test_error_becoming_voter.py index 745ba68de8e8..62783a341acb 100644 --- a/test/topology_custom/test_error_becoming_voter.py +++ b/test/topology_custom/test_error_becoming_voter.py @@ -17,7 +17,6 @@ logger = logging.getLogger(__name__) -@pytest.mark.xfail(reason="issue #20814") @pytest.mark.asyncio async def test_error_while_becoming_voter(request: pytest.FixtureRequest, manager: ManagerClient) -> None: """ From 3da4848810e0458604e7026f7bd0b6e8fd4a8600 Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Fri, 10 Jan 2025 15:18:58 +0100 Subject: [PATCH 295/397] raft: Refactor functions using `modify_config` to use a common wrapper for retrying. There are several places in `raft_group0` where almost identical code is used for retrying `modify_config` in case of `commit_status_unknown` error. To avoid code duplication all these places were changed to use a new wrapper `run_op_with_retry`. --- service/raft/raft_group0.cc | 36 +++++++++++------------------------- 1 file changed, 11 insertions(+), 25 deletions(-) diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index 389db449ff46..c12003403cd3 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -954,50 +954,36 @@ future raft_group0::wait_for_raft() { future<> raft_group0::modify_raft_voter_status( const std::unordered_set& ids, can_vote can_vote, abort_source& as, std::optional timeout) { - static constexpr auto max_retry_period = std::chrono::seconds{1}; - auto retry_period = std::chrono::milliseconds{10}; - - while (true) { - as.check(); - + co_await run_op_with_retry(as, [this, &ids, timeout, can_vote, &as]() -> future { std::vector add; add.reserve(ids.size()); - std::transform(ids.begin(), ids.end(), std::back_inserter(add), [can_vote] (raft::server_id id) { + std::transform(ids.begin(), ids.end(), std::back_inserter(add), [can_vote](raft::server_id id) { return raft::config_member{{id, {}}, static_cast(can_vote)}; }); try { co_await _raft_gr.group0_with_timeouts().modify_config(std::move(add), {}, &as, timeout); - co_return; } catch (const raft::commit_status_unknown& e) { group0_log.info("modify_raft_voter_config({}): modify_config returned \"{}\", retrying", ids, e); + co_return operation_result::failure; } - retry_period *= 2; - if (retry_period > max_retry_period) { - retry_period = max_retry_period; - } - co_await sleep_abortable(retry_period, as); - } + co_return operation_result::success; + }, "modify_raft_voter_status->modify_config"); + co_return; } future<> raft_group0::remove_from_raft_config(raft::server_id id) { - static constexpr auto max_retry_period = std::chrono::seconds{1}; - auto retry_period = std::chrono::milliseconds{10}; - // TODO: add a timeout mechanism? This could get stuck (and _abort_source is only called on shutdown). - while (true) { + co_await run_op_with_retry(_abort_source, [this, id]() -> future { try { co_await _raft_gr.group0().modify_config({}, {id}, &_abort_source); - break; } catch (const raft::commit_status_unknown& e) { group0_log.info("remove_from_raft_config({}): modify_config returned \"{}\", retrying", id, e); + co_return operation_result::failure; } - retry_period *= 2; - if (retry_period > max_retry_period) { - retry_period = max_retry_period; - } - co_await sleep_abortable(retry_period, _abort_source); - } + co_return operation_result::success; + }, "remove_from_raft_config->modify_config"); + co_return; } bool raft_group0::joined_group0() const { From 228a66d030016287b6a23bbfac7d4216b658b69c Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Mon, 13 Jan 2025 09:15:32 +0100 Subject: [PATCH 296/397] raft: refactor `remove_from_raft_config` to use a timed `modify_config` call. To avoid potential hangs during the `remove_from_raft_config` operation, use a timed `modify_config` call. This ensures the operation doesn't get stuck indefinitely. --- service/raft/raft_group0.cc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/service/raft/raft_group0.cc b/service/raft/raft_group0.cc index c12003403cd3..35ac551773d6 100644 --- a/service/raft/raft_group0.cc +++ b/service/raft/raft_group0.cc @@ -973,10 +973,9 @@ future<> raft_group0::modify_raft_voter_status( } future<> raft_group0::remove_from_raft_config(raft::server_id id) { - // TODO: add a timeout mechanism? This could get stuck (and _abort_source is only called on shutdown). co_await run_op_with_retry(_abort_source, [this, id]() -> future { try { - co_await _raft_gr.group0().modify_config({}, {id}, &_abort_source); + co_await _raft_gr.group0_with_timeouts().modify_config({}, {id}, &_abort_source, raft_timeout{}); } catch (const raft::commit_status_unknown& e) { group0_log.info("remove_from_raft_config({}): modify_config returned \"{}\", retrying", id, e); co_return operation_result::failure; From 384641194a464ae7d87a8ea5a15d3e09edd91279 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Zakrzewski?= Date: Wed, 11 Dec 2024 12:12:42 +0100 Subject: [PATCH 297/397] audit: Add the audit subsystem This change introduces a new audit subsystem that allows tracking and logging of database operations for security and compliance purposes. Key features include: - Configurable audit logging to either syslog or a dedicated system table (audit.audit_log) - Selective auditing based on: - Operation categories (QUERY, DML, DDL, DCL, AUTH, ADMIN) - Specific keyspaces - Specific tables - New configuration options: - audit: Controls audit destination (none/syslog/table) - audit_categories: Comma-separated list of operation categories to audit - audit_tables: Specific tables to audit - audit_keyspaces: Specific keyspaces to audit - audit_unix_socket_path: Path for syslog socket - audit_syslog_write_buffer_size: Buffer size for syslog writes The audit logs capture details including: - Operation timestamp - Node and client IP addresses - Operation category and query - Username - Success/failure status - Affected keyspace and table names --- CMakeLists.txt | 2 + audit/CMakeLists.txt | 19 ++ audit/audit.cc | 263 ++++++++++++++++++ audit/audit.hh | 141 ++++++++++ audit/audit_cf_storage_helper.cc | 202 ++++++++++++++ audit/audit_cf_storage_helper.hh | 67 +++++ audit/audit_syslog_storage_helper.cc | 134 +++++++++ audit/audit_syslog_storage_helper.hh | 44 +++ audit/storage_helper.hh | 34 +++ conf/scylla.yaml | 22 ++ configure.py | 3 + cql3/cql_statement.hh | 10 +- cql3/statements/batch_statement.hh | 2 + cql3/statements/raw/modification_statement.hh | 3 + data_dictionary/data_dictionary.cc | 5 +- data_dictionary/keyspace_metadata.hh | 3 +- db/config.cc | 13 + db/config.hh | 7 + table_helper.cc | 13 +- table_helper.hh | 3 +- test/lib/CMakeLists.txt | 1 + tracing/trace_keyspace_helper.cc | 2 +- 22 files changed, 986 insertions(+), 7 deletions(-) create mode 100644 audit/CMakeLists.txt create mode 100644 audit/audit.cc create mode 100644 audit/audit.hh create mode 100644 audit/audit_cf_storage_helper.cc create mode 100644 audit/audit_cf_storage_helper.hh create mode 100644 audit/audit_syslog_storage_helper.cc create mode 100644 audit/audit_syslog_storage_helper.hh create mode 100644 audit/storage_helper.hh diff --git a/CMakeLists.txt b/CMakeLists.txt index 6c56fc01a566..444cfdc346de 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -261,6 +261,7 @@ add_custom_target(compiler-training) add_subdirectory(api) add_subdirectory(alternator) +add_subdirectory(audit) add_subdirectory(db) add_subdirectory(auth) add_subdirectory(cdc) @@ -300,6 +301,7 @@ add_version_library(scylla_version add_executable(scylla main.cc) set(scylla_libs + audit scylla-main api auth diff --git a/audit/CMakeLists.txt b/audit/CMakeLists.txt new file mode 100644 index 000000000000..2d606754e6c6 --- /dev/null +++ b/audit/CMakeLists.txt @@ -0,0 +1,19 @@ +include(add_whole_archive) + +add_library(scylla_audit STATIC) +target_sources(scylla_audit + PRIVATE + audit.cc + audit_cf_storage_helper.cc + audit_syslog_storage_helper.cc) +target_include_directories(scylla_audit + PUBLIC + ${CMAKE_SOURCE_DIR}) +target_link_libraries(scylla_audit + PUBLIC + Seastar::seastar + xxHash::xxhash + PRIVATE + cql3) + +add_whole_archive(audit scylla_audit) diff --git a/audit/audit.cc b/audit/audit.cc new file mode 100644 index 000000000000..7fa206e46e67 --- /dev/null +++ b/audit/audit.cc @@ -0,0 +1,263 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include +#include "audit/audit.hh" +#include "db/config.hh" +#include "cql3/cql_statement.hh" +#include "cql3/statements/batch_statement.hh" +#include "cql3/statements/modification_statement.hh" +#include "storage_helper.hh" +#include "audit.hh" +#include "../db/config.hh" +#include "utils/class_registrator.hh" + +#include +#include +#include + + +namespace audit { + +logging::logger logger("audit"); + +sstring audit_info::category_string() const { + switch (_category) { + case statement_category::QUERY: return "QUERY"; + case statement_category::DML: return "DML"; + case statement_category::DDL: return "DDL"; + case statement_category::DCL: return "DCL"; + case statement_category::AUTH: return "AUTH"; + case statement_category::ADMIN: return "ADMIN"; + } + return ""; +} + +audit::audit(locator::shared_token_metadata& token_metadata, + sstring&& storage_helper_name, + std::set&& audited_keyspaces, + std::map>&& audited_tables, + category_set&& audited_categories) + : _token_metadata(token_metadata) + , _audited_keyspaces(std::move(audited_keyspaces)) + , _audited_tables(std::move(audited_tables)) + , _audited_categories(std::move(audited_categories)) + , _storage_helper_class_name(std::move(storage_helper_name)) +{ } + +audit::~audit() = default; + +static category_set parse_audit_categories(const sstring& data) { + category_set result; + if (!data.empty()) { + std::vector tokens; + boost::split(tokens, data, boost::is_any_of(",")); + for (sstring& category : tokens) { + boost::trim(category); + if (category == "QUERY") { + result.set(statement_category::QUERY); + } else if (category == "DML") { + result.set(statement_category::DML); + } else if (category == "DDL") { + result.set(statement_category::DDL); + } else if (category == "DCL") { + result.set(statement_category::DCL); + } else if (category == "AUTH") { + result.set(statement_category::AUTH); + } else if (category == "ADMIN") { + result.set(statement_category::ADMIN); + } else { + throw audit_exception(fmt::format("Bad configuration: invalid 'audit_categories': {}", data)); + } + } + } + return result; +} + +static std::map> parse_audit_tables(const sstring& data) { + std::map> result; + if (!data.empty()) { + std::vector tokens; + boost::split(tokens, data, boost::is_any_of(",")); + for (sstring& token : tokens) { + std::vector parts; + boost::split(parts, token, boost::is_any_of(".")); + if (parts.size() != 2) { + throw audit_exception(fmt::format("Bad configuration: invalid 'audit_tables': {}", data)); + } + boost::trim(parts[0]); + boost::trim(parts[1]); + result[parts[0]].insert(std::move(parts[1])); + } + } + return result; +} + +static std::set parse_audit_keyspaces(const sstring& data) { + std::set result; + if (!data.empty()) { + std::vector tokens; + boost::split(tokens, data, boost::is_any_of(",")); + for (sstring& token : tokens) { + boost::trim(token); + result.insert(std::move(token)); + } + } + return result; +} + +future<> audit::create_audit(const db::config& cfg, sharded& stm) { + sstring storage_helper_name; + if (cfg.audit() == "table") { + storage_helper_name = "audit_cf_storage_helper"; + } else if (cfg.audit() == "syslog") { + storage_helper_name = "audit_syslog_storage_helper"; + } else if (cfg.audit() == "none") { + // Audit is off + logger.info("Audit is disabled"); + + return make_ready_future<>(); + } else { + throw audit_exception(fmt::format("Bad configuration: invalid 'audit': {}", cfg.audit())); + } + category_set audited_categories = parse_audit_categories(cfg.audit_categories()); + if (!audited_categories) { + return make_ready_future<>(); + } + std::map> audited_tables = parse_audit_tables(cfg.audit_tables()); + std::set audited_keyspaces = parse_audit_keyspaces(cfg.audit_keyspaces()); + if (audited_tables.empty() + && audited_keyspaces.empty() + && !audited_categories.contains(statement_category::AUTH) + && !audited_categories.contains(statement_category::ADMIN) + && !audited_categories.contains(statement_category::DCL)) { + return make_ready_future<>(); + } + logger.info("Audit is enabled. Auditing to: \"{}\", with the following categories: \"{}\", keyspaces: \"{}\", and tables: \"{}\"", + cfg.audit(), cfg.audit_categories(), cfg.audit_keyspaces(), cfg.audit_tables()); + + return audit_instance().start(std::ref(stm), + std::move(storage_helper_name), + std::move(audited_keyspaces), + std::move(audited_tables), + std::move(audited_categories)); +} + +future<> audit::start_audit(const db::config& cfg, sharded& qp, sharded& mm) { + if (!audit_instance().local_is_initialized()) { + return make_ready_future<>(); + } + return audit_instance().invoke_on_all([&cfg, &qp, &mm] (audit& local_audit) { + return local_audit.start(cfg, qp.local(), mm.local()); + }); +} + +future<> audit::stop_audit() { + if (!audit_instance().local_is_initialized()) { + return make_ready_future<>(); + } + return audit::audit::audit_instance().invoke_on_all([] (auto& local_audit) { + return local_audit.shutdown(); + }).then([] { + return audit::audit::audit_instance().stop(); + }); +} + +audit_info_ptr audit::create_audit_info(statement_category cat, const sstring& keyspace, const sstring& table) { + if (!audit_instance().local_is_initialized()) { + return nullptr; + } + return std::make_unique(cat, keyspace, table); +} + +audit_info_ptr audit::create_no_audit_info() { + return audit_info_ptr(); +} + +future<> audit::start(const db::config& cfg, cql3::query_processor& qp, service::migration_manager& mm) { + try { + _storage_helper_ptr = create_object(_storage_helper_class_name, qp, mm); + } catch (no_such_class& e) { + logger.error("Can't create audit storage helper {}: not supported", _storage_helper_class_name); + throw; + } catch (...) { + throw; + } + return _storage_helper_ptr->start(cfg); +} + +future<> audit::stop() { + return _storage_helper_ptr->stop(); +} + +future<> audit::shutdown() { + return make_ready_future<>(); +} + +future<> audit::log(const audit_info* audit_info, service::query_state& query_state, const cql3::query_options& options, bool error) { + const service::client_state& client_state = query_state.get_client_state(); + socket_address node_ip = _token_metadata.get()->get_topology().my_address().addr(); + db::consistency_level cl = options.get_consistency(); + thread_local static sstring no_username("undefined"); + static const sstring anonymous_username("anonymous"); + const sstring& username = client_state.user() ? client_state.user()->name.value_or(anonymous_username) : no_username; + socket_address client_ip = client_state.get_client_address().addr(); + return futurize_invoke(std::mem_fn(&storage_helper::write), _storage_helper_ptr, audit_info, node_ip, client_ip, cl, username, error) + .handle_exception([audit_info, node_ip, client_ip, cl, username, error] (auto ep) { + logger.error("Unexpected exception when writing log with: node_ip {} category {} cl {} error {} keyspace {} query '{}' client_ip {} table {} username {} exception {}", + node_ip, audit_info->category_string(), cl, error, audit_info->keyspace(), + audit_info->query(), client_ip, audit_info->table(),username, ep); + }); +} + +future<> audit::log_login(const sstring& username, socket_address client_ip, bool error) noexcept { + socket_address node_ip = _token_metadata.get()->get_topology().my_address().addr(); + return futurize_invoke(std::mem_fn(&storage_helper::write_login), _storage_helper_ptr, username, node_ip, client_ip, error) + .handle_exception([username, node_ip, client_ip, error] (auto ep) { + logger.error("Unexpected exception when writing login log with: node_ip {} client_ip {} username {} error {} exception {}", + node_ip, client_ip, username, error, ep); + }); +} + +future<> inspect(shared_ptr statement, service::query_state& query_state, const cql3::query_options& options, bool error) { + cql3::statements::batch_statement* batch = dynamic_cast(statement.get()); + if (batch != nullptr) { + return do_for_each(batch->statements().begin(), batch->statements().end(), [&query_state, &options, error] (auto&& m) { + return inspect(m.statement, query_state, options, error); + }); + } else { + auto audit_info = statement->get_audit_info(); + if (bool(audit_info) && audit::local_audit_instance().should_log(audit_info)) { + return audit::local_audit_instance().log(audit_info, query_state, options, error); + } + } + return make_ready_future<>(); +} + +future<> inspect_login(const sstring& username, socket_address client_ip, bool error) { + if (!audit::audit_instance().local_is_initialized() || !audit::local_audit_instance().should_log_login()) { + return make_ready_future<>(); + } + return audit::local_audit_instance().log_login(username, client_ip, error); +} + +bool audit::should_log_table(const sstring& keyspace, const sstring& name) const { + auto keyspace_it = _audited_tables.find(keyspace); + return keyspace_it != _audited_tables.cend() && keyspace_it->second.find(name) != keyspace_it->second.cend(); +} + +bool audit::should_log(const audit_info* audit_info) const { + return _audited_categories.contains(audit_info->category()) + && (_audited_keyspaces.find(audit_info->keyspace()) != _audited_keyspaces.cend() + || should_log_table(audit_info->keyspace(), audit_info->table()) + || audit_info->category() == statement_category::AUTH + || audit_info->category() == statement_category::ADMIN + || audit_info->category() == statement_category::DCL); +} + +} diff --git a/audit/audit.hh b/audit/audit.hh new file mode 100644 index 000000000000..0e434fbeaa3e --- /dev/null +++ b/audit/audit.hh @@ -0,0 +1,141 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#pragma once + +#include "seastarx.hh" +#include "utils/log.hh" +#include "db/consistency_level.hh" +#include "locator/token_metadata_fwd.hh" +#include +#include + +#include "enum_set.hh" + +#include + +namespace db { + +class config; + +} + +namespace cql3 { + +class cql_statement; +class query_processor; +class query_options; + +} + +namespace service { + +class migration_manager; +class query_state; + +} + +namespace locator { + +class shared_token_metadata; + +} + +namespace audit { + +extern logging::logger logger; + +class audit_exception : public std::exception { + sstring _what; +public: + explicit audit_exception(sstring&& what) : _what(std::move(what)) { } + const char* what() const noexcept override { + return _what.c_str(); + } +}; + +enum class statement_category { + QUERY, DML, DDL, DCL, AUTH, ADMIN +}; + +using category_set = enum_set>; + +class audit_info final { + statement_category _category; + sstring _keyspace; + sstring _table; + sstring _query; +public: + audit_info(statement_category cat, sstring keyspace, sstring table) + : _category(cat) + , _keyspace(std::move(keyspace)) + , _table(std::move(table)) + { } + void set_query_string(const std::string_view& query_string) { + _query = sstring(query_string); + } + const sstring& keyspace() const { return _keyspace; } + const sstring& table() const { return _table; } + const sstring& query() const { return _query; } + sstring category_string() const; + statement_category category() const { return _category; } +}; + +using audit_info_ptr = std::unique_ptr; + +class storage_helper; + +class audit final : public seastar::async_sharded_service { + locator::shared_token_metadata& _token_metadata; + const std::set _audited_keyspaces; + // Maps keyspace name to set of table names in that keyspace + const std::map> _audited_tables; + const category_set _audited_categories; + sstring _storage_helper_class_name; + std::unique_ptr _storage_helper_ptr; + + bool should_log_table(const sstring& keyspace, const sstring& name) const; +public: + static seastar::sharded& audit_instance() { + // FIXME: leaked intentionally to avoid shutdown problems, see #293 + static seastar::sharded* audit_inst = new seastar::sharded(); + + return *audit_inst; + } + + static audit& local_audit_instance() { + return audit_instance().local(); + } + static future<> create_audit(const db::config& cfg, sharded& stm); + static future<> start_audit(const db::config& cfg, sharded& qp, sharded& mm); + static future<> stop_audit(); + static audit_info_ptr create_audit_info(statement_category cat, const sstring& keyspace, const sstring& table); + static audit_info_ptr create_no_audit_info(); + audit(locator::shared_token_metadata& stm, sstring&& storage_helper_name, + std::set&& audited_keyspaces, + std::map>&& audited_tables, + category_set&& audited_categories); + ~audit(); + future<> start(const db::config& cfg, cql3::query_processor& qp, service::migration_manager& mm); + future<> stop(); + future<> shutdown(); + bool should_log(const audit_info* audit_info) const; + bool should_log_login() const { return _audited_categories.contains(statement_category::AUTH); } + future<> log(const audit_info* audit_info, service::query_state& query_state, const cql3::query_options& options, bool error); + future<> log_login(const sstring& username, socket_address client_ip, bool error) noexcept; +}; + +future<> inspect(shared_ptr statement, service::query_state& query_state, const cql3::query_options& options, bool error); + +future<> inspect_login(const sstring& username, socket_address client_ip, bool error); + +} diff --git a/audit/audit_cf_storage_helper.cc b/audit/audit_cf_storage_helper.cc new file mode 100644 index 000000000000..b2657b46c0c2 --- /dev/null +++ b/audit/audit_cf_storage_helper.cc @@ -0,0 +1,202 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "audit/audit_cf_storage_helper.hh" + +#include "cql3/query_processor.hh" +#include "data_dictionary/keyspace_metadata.hh" +#include "utils/UUID_gen.hh" +#include "utils/class_registrator.hh" +#include "cql3/query_options.hh" +#include "cql3/statements/ks_prop_defs.hh" +#include "service/migration_manager.hh" +#include "service/storage_proxy.hh" + +namespace audit { + +const sstring audit_cf_storage_helper::KEYSPACE_NAME("audit"); +const sstring audit_cf_storage_helper::TABLE_NAME("audit_log"); + +audit_cf_storage_helper::audit_cf_storage_helper(cql3::query_processor& qp, service::migration_manager& mm) + : _qp(qp) + , _mm(mm) + , _table(KEYSPACE_NAME, TABLE_NAME, + fmt::format("CREATE TABLE IF NOT EXISTS {}.{} (" + "date timestamp, " + "node inet, " + "event_time timeuuid, " + "category text, " + "consistency text, " + "table_name text, " + "keyspace_name text, " + "operation text, " + "source inet, " + "username text, " + "error boolean, " + "PRIMARY KEY ((date, node), event_time))", + KEYSPACE_NAME, TABLE_NAME), + fmt::format("INSERT INTO {}.{} (" + "date," + "node," + "event_time," + "category," + "consistency," + "table_name," + "keyspace_name," + "operation," + "source," + "username," + "error) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", + KEYSPACE_NAME, TABLE_NAME)) + , _dummy_query_state(service::client_state::for_internal_calls(), empty_service_permit()) +{ +} + +future<> audit_cf_storage_helper::migrate_audit_table(service::group0_guard group0_guard) { + while (true) { + auto const ks = _qp.db().try_find_keyspace(KEYSPACE_NAME); + if (ks && ks->metadata()->strategy_name() == "org.apache.cassandra.locator.SimpleStrategy") { + data_dictionary::database db = _qp.db(); + cql3::statements::ks_prop_defs old_ks_prop_defs; + auto old_ks_metadata = old_ks_prop_defs.as_ks_metadata_update( + ks->metadata(), *_qp.proxy().get_token_metadata_ptr(), db.features()); + std::map strategy_opts; + for (const auto &dc: _qp.proxy().get_token_metadata_ptr()->get_topology().get_datacenters()) + strategy_opts[dc] = "3"; + + auto new_ks_metadata = keyspace_metadata::new_keyspace(KEYSPACE_NAME, + "org.apache.cassandra.locator.NetworkTopologyStrategy", + strategy_opts, + std::nullopt, // initial_tablets + old_ks_metadata->durable_writes(), + old_ks_metadata->get_storage_options(), + old_ks_metadata->tables()); + auto ts = group0_guard.write_timestamp(); + try { + co_await _mm.announce( + service::prepare_keyspace_update_announcement(db.real_database(), new_ks_metadata, ts), + std::move(group0_guard), format("audit: Alter {} keyspace", KEYSPACE_NAME)); + break; + } catch (::service::group0_concurrent_modification &) { + logger.info("Concurrent operation is detected while altering {} keyspace, retrying.", KEYSPACE_NAME); + } + group0_guard = co_await _mm.start_group0_operation(); + } else { + co_return; + } + } +} + +future<> audit_cf_storage_helper::start(const db::config &cfg) { + if (this_shard_id() != 0) { + co_return; + } + + if (auto ks = _qp.db().try_find_keyspace(KEYSPACE_NAME); + !ks || + ks->metadata()->strategy_name() == "org.apache.cassandra.locator.SimpleStrategy") { + + auto group0_guard = co_await _mm.start_group0_operation(); + if (ks = _qp.db().try_find_keyspace(KEYSPACE_NAME); !ks) { + // releasing, because table_helper::setup_keyspace creates a raft guard of its own + service::release_guard(std::move(group0_guard)); + co_return co_await table_helper::setup_keyspace(_qp, _mm, KEYSPACE_NAME, + "org.apache.cassandra.locator.NetworkTopologyStrategy", + "3", _dummy_query_state, {&_table}); + } else if (ks->metadata()->strategy_name() == "org.apache.cassandra.locator.SimpleStrategy") { + // We want to migrate the old (pre-Scylla 6.0) SimpleStrategy to a newer one. + // The migrate_audit_table() function will do nothing if it races with another strategy change: + // - either by another node doing the same thing in parallel, + // - or a user manually changing the strategy of the same table. + // Note we only check the strategy, not the replication factor. + co_return co_await migrate_audit_table(std::move(group0_guard)); + } else { + co_return; + } + } +} + +future<> audit_cf_storage_helper::stop() { + return make_ready_future<>(); +} + +future<> audit_cf_storage_helper::write(const audit_info* audit_info, + socket_address node_ip, + socket_address client_ip, + db::consistency_level cl, + const sstring& username, + bool error) { + return _table.insert(_qp, _mm, _dummy_query_state, make_data, audit_info, node_ip, client_ip, cl, username, error); +} + +future<> audit_cf_storage_helper::write_login(const sstring& username, + socket_address node_ip, + socket_address client_ip, + bool error) { + return _table.insert(_qp, _mm, _dummy_query_state, make_login_data, node_ip, client_ip, username, error); +} + +cql3::query_options audit_cf_storage_helper::make_data(const audit_info* audit_info, + socket_address node_ip, + socket_address client_ip, + db::consistency_level cl, + const sstring& username, + bool error) { + auto time = std::chrono::system_clock::now(); + auto millis_since_epoch = std::chrono::duration_cast(time.time_since_epoch()).count(); + auto ticks_per_day = std::chrono::duration_cast(std::chrono::hours(24)).count(); + auto date = millis_since_epoch / ticks_per_day * ticks_per_day; + thread_local static int64_t last_nanos = 0; + auto time_id = utils::UUID_gen::get_time_UUID(table_helper::make_monotonic_UUID_tp(last_nanos, time)); + auto consistency_level = fmt::format("{}", cl); + std::vector values { + cql3::raw_value::make_value(timestamp_type->decompose(date)), + cql3::raw_value::make_value(inet_addr_type->decompose(node_ip.addr())), + cql3::raw_value::make_value(uuid_type->decompose(time_id)), + cql3::raw_value::make_value(utf8_type->decompose(audit_info->category_string())), + cql3::raw_value::make_value(utf8_type->decompose(sstring(consistency_level))), + cql3::raw_value::make_value(utf8_type->decompose(audit_info->table())), + cql3::raw_value::make_value(utf8_type->decompose(audit_info->keyspace())), + cql3::raw_value::make_value(utf8_type->decompose(audit_info->query())), + cql3::raw_value::make_value(inet_addr_type->decompose(client_ip.addr())), + cql3::raw_value::make_value(utf8_type->decompose(username)), + cql3::raw_value::make_value(boolean_type->decompose(error)), + }; + return cql3::query_options(cql3::default_cql_config, db::consistency_level::ONE, std::nullopt, std::move(values), false, cql3::query_options::specific_options::DEFAULT); +} + +cql3::query_options audit_cf_storage_helper::make_login_data(socket_address node_ip, + socket_address client_ip, + const sstring& username, + bool error) { + auto time = std::chrono::system_clock::now(); + auto millis_since_epoch = std::chrono::duration_cast(time.time_since_epoch()).count(); + auto ticks_per_day = std::chrono::duration_cast(std::chrono::hours(24)).count(); + auto date = millis_since_epoch / ticks_per_day * ticks_per_day; + thread_local static int64_t last_nanos = 0; + auto time_id = utils::UUID_gen::get_time_UUID(table_helper::make_monotonic_UUID_tp(last_nanos, time)); + std::vector values { + cql3::raw_value::make_value(timestamp_type->decompose(date)), + cql3::raw_value::make_value(inet_addr_type->decompose(node_ip.addr())), + cql3::raw_value::make_value(uuid_type->decompose(time_id)), + cql3::raw_value::make_value(utf8_type->decompose(sstring("AUTH"))), + cql3::raw_value::make_value(utf8_type->decompose(sstring(""))), + cql3::raw_value::make_value(utf8_type->decompose(sstring(""))), + cql3::raw_value::make_value(utf8_type->decompose(sstring(""))), + cql3::raw_value::make_value(utf8_type->decompose(sstring("LOGIN"))), + cql3::raw_value::make_value(inet_addr_type->decompose(client_ip.addr())), + cql3::raw_value::make_value(utf8_type->decompose(username)), + cql3::raw_value::make_value(boolean_type->decompose(error)), + }; + return cql3::query_options(cql3::default_cql_config, db::consistency_level::ONE, std::nullopt, std::move(values), false, cql3::query_options::specific_options::DEFAULT); +} + +using registry = class_registrator; +static registry registrator1("audit_cf_storage_helper"); + +} diff --git a/audit/audit_cf_storage_helper.hh b/audit/audit_cf_storage_helper.hh new file mode 100644 index 000000000000..376616786beb --- /dev/null +++ b/audit/audit_cf_storage_helper.hh @@ -0,0 +1,67 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#pragma once + +#include "audit/audit.hh" +#include "table_helper.hh" +#include "storage_helper.hh" +#include "db/config.hh" +#include "service/raft/raft_group0_client.hh" + +namespace cql3 { + +class query_processor; + +} + +namespace service { + +class migration_manager; + +} + +namespace audit { + +class audit_cf_storage_helper : public storage_helper { + static const sstring KEYSPACE_NAME; + static const sstring TABLE_NAME; + cql3::query_processor& _qp; + service::migration_manager& _mm; + table_helper _table; + service::query_state _dummy_query_state; + static cql3::query_options make_data(const audit_info* audit_info, + socket_address node_ip, + socket_address client_ip, + db::consistency_level cl, + const sstring& username, + bool error); + static cql3::query_options make_login_data(socket_address node_ip, + socket_address client_ip, + const sstring& username, + bool error); + + future<> migrate_audit_table(service::group0_guard guard); + +public: + explicit audit_cf_storage_helper(cql3::query_processor& qp, service::migration_manager& mm); + virtual ~audit_cf_storage_helper() {} + virtual future<> start(const db::config& cfg) override; + virtual future<> stop() override; + virtual future<> write(const audit_info* audit_info, + socket_address node_ip, + socket_address client_ip, + db::consistency_level cl, + const sstring& username, + bool error) override; + virtual future<> write_login(const sstring& username, + socket_address node_ip, + socket_address client_ip, + bool error) override; +}; + +} diff --git a/audit/audit_syslog_storage_helper.cc b/audit/audit_syslog_storage_helper.cc new file mode 100644 index 000000000000..dd13a8be3938 --- /dev/null +++ b/audit/audit_syslog_storage_helper.cc @@ -0,0 +1,134 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "audit/audit_syslog_storage_helper.hh" + +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + +#include "cql3/query_processor.hh" +#include "utils/class_registrator.hh" + +namespace cql3 { + +class query_processor; + +} + +namespace audit { + +namespace { + +future<> syslog_send_helper(net::datagram_channel& sender, + const socket_address& address, + const sstring& msg) { + return sender.send(address, net::packet{msg.data(), msg.size()}).handle_exception([address](auto&& exception_ptr) { + auto error_msg = seastar::format( + "Syslog audit backend failed (sending a message to {} resulted in {}).", + address, + exception_ptr + ); + logger.error("{}", error_msg); + throw audit_exception(std::move(error_msg)); + }); +} + +static auto syslog_address_helper(const db::config& cfg) +{ + return cfg.audit_unix_socket_path.is_set() + ? unix_domain_addr(cfg.audit_unix_socket_path()) + : unix_domain_addr(_PATH_LOG); +} + +} + +audit_syslog_storage_helper::audit_syslog_storage_helper(cql3::query_processor& qp, service::migration_manager&) : + _syslog_address(syslog_address_helper(qp.db().get_config())), + _sender(make_unbound_datagram_channel(AF_UNIX)) { +} + +audit_syslog_storage_helper::~audit_syslog_storage_helper() { +} + +/* + * We don't use openlog and syslog directly because it's already used by logger. + * Audit needs to use different ident so than logger but syslog.h uses a global ident + * and it's not possible to use more than one in a program. + * + * To work around it we directly communicate with the socket. + */ +future<> audit_syslog_storage_helper::start(const db::config& cfg) { + if (this_shard_id() != 0) { + return make_ready_future(); + } + + return syslog_send_helper(_sender, _syslog_address, "Initializing syslog audit backend."); +} + +future<> audit_syslog_storage_helper::stop() { + _sender.shutdown_output(); + co_return; +} + +future<> audit_syslog_storage_helper::write(const audit_info* audit_info, + socket_address node_ip, + socket_address client_ip, + db::consistency_level cl, + const sstring& username, + bool error) { + auto now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + tm time; + localtime_r(&now, &time); + sstring msg = seastar::format("<{}>{:%h %e %T} scylla-audit: \"{}\", \"{}\", \"{}\", \"{}\", \"{}\", \"{}\", \"{}\", \"{}\", \"{}\"", + LOG_NOTICE | LOG_USER, + time, + node_ip, + audit_info->category_string(), + cl, + (error ? "true" : "false"), + audit_info->keyspace(), + audit_info->query(), + client_ip, + audit_info->table(), + username); + + return syslog_send_helper(_sender, _syslog_address, msg); +} + +future<> audit_syslog_storage_helper::write_login(const sstring& username, + socket_address node_ip, + socket_address client_ip, + bool error) { + + auto now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + tm time; + localtime_r(&now, &time); + sstring msg = seastar::format("<{}>{:%h %e %T} scylla-audit: \"{}\", \"AUTH\", \"\", \"\", \"\", \"\", \"{}\", \"{}\", \"{}\"", + LOG_NOTICE | LOG_USER, + time, + node_ip, + client_ip, + username, + (error ? "true" : "false")); + + co_await syslog_send_helper(_sender, _syslog_address, msg.c_str()); +} + +using registry = class_registrator; +static registry registrator1("audit_syslog_storage_helper"); + +} diff --git a/audit/audit_syslog_storage_helper.hh b/audit/audit_syslog_storage_helper.hh new file mode 100644 index 000000000000..11f14c1ae5c3 --- /dev/null +++ b/audit/audit_syslog_storage_helper.hh @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#pragma once + +#include + +#include "audit/audit.hh" +#include "storage_helper.hh" +#include "db/config.hh" + +namespace service { + +class migration_manager; + +}; + +namespace audit { + +class audit_syslog_storage_helper : public storage_helper { + socket_address _syslog_address; + net::datagram_channel _sender; +public: + explicit audit_syslog_storage_helper(cql3::query_processor&, service::migration_manager&); + virtual ~audit_syslog_storage_helper(); + virtual future<> start(const db::config& cfg) override; + virtual future<> stop() override; + virtual future<> write(const audit_info* audit_info, + socket_address node_ip, + socket_address client_ip, + db::consistency_level cl, + const sstring& username, + bool error) override; + virtual future<> write_login(const sstring& username, + socket_address node_ip, + socket_address client_ip, + bool error) override; +}; + +} diff --git a/audit/storage_helper.hh b/audit/storage_helper.hh new file mode 100644 index 000000000000..be11a2b0b5da --- /dev/null +++ b/audit/storage_helper.hh @@ -0,0 +1,34 @@ +/* + * Copyright (C) 2017 ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ +#pragma once + +#include "audit/audit.hh" +#include + +namespace audit { + +class storage_helper { +public: + using ptr_type = std::unique_ptr; + storage_helper() {} + virtual ~storage_helper() {} + virtual future<> start(const db::config& cfg) = 0; + virtual future<> stop() = 0; + virtual future<> write(const audit_info* audit_info, + socket_address node_ip, + socket_address client_ip, + db::consistency_level cl, + const sstring& username, + bool error) = 0; + virtual future<> write_login(const sstring& username, + socket_address node_ip, + socket_address client_ip, + bool error) = 0; +}; + +} diff --git a/conf/scylla.yaml b/conf/scylla.yaml index 898581778afd..58234e5ada4f 100644 --- a/conf/scylla.yaml +++ b/conf/scylla.yaml @@ -564,6 +564,28 @@ commitlog_total_space_in_mb: -1 # it to 0.0.0.0 to listen on all interfaces. # prometheus_address: 1.2.3.4 +# audit settings +# By default, Scylla does not audit anything. +# 'audit' config option controls if and where to output audited events: +# - "none": auditing is disabled (default) +# - "table": save audited events in audit.audit_log column family +# - "syslog": send audited events via syslog (depends on OS, but usually to /dev/log) +# audit: "none" +# +# List of statement categories that should be audited. +# audit_categories: "DCL,DDL,AUTH" +# +# List of tables that should be audited. +# audit_tables: ".,." +# +# List of keyspaces that should be fully audited. +# All tables in those keyspaces will be audited +# audit_keyspaces: "," +# +# Overrides the Unix socket path used to connect to syslog. If left unset, it'll +# use the default on the build system, which is usually "/dev/log" +# audit_unix_socket_path: "/dev/log" + # Distribution of data among cores (shards) within a node # # Scylla distributes data within a node among shards, using a round-robin diff --git a/configure.py b/configure.py index 46100309705d..f5d1cf1a85b9 100755 --- a/configure.py +++ b/configure.py @@ -1124,6 +1124,9 @@ def find_ninja(): 'tracing/trace_state.cc', 'tracing/traced_file.cc', 'table_helper.cc', + 'audit/audit.cc', + 'audit/audit_cf_storage_helper.cc', + 'audit/audit_syslog_storage_helper.cc', 'tombstone_gc_options.cc', 'tombstone_gc.cc', 'utils/disk-error-handler.cc', diff --git a/cql3/cql_statement.hh b/cql3/cql_statement.hh index 951eaa2dfd4b..b6531a728a0b 100644 --- a/cql3/cql_statement.hh +++ b/cql3/cql_statement.hh @@ -12,6 +12,7 @@ #include "timeout_config.hh" #include "service/raft/raft_group0_client.hh" +#include "audit/audit.hh" namespace service { @@ -45,6 +46,7 @@ using cql_warnings_vec = std::vector; class cql_statement { timeout_config_selector _timeout_config_selector; + audit::audit_info_ptr _audit_info; public: // CQL statement text seastar::sstring raw_cql_statement; @@ -55,7 +57,8 @@ public: } explicit cql_statement(timeout_config_selector timeout_selector) : _timeout_config_selector(timeout_selector) {} - + cql_statement(cql_statement&& o) = default; + cql_statement(const cql_statement& o) : _timeout_config_selector(o._timeout_config_selector), _audit_info(o._audit_info ? std::make_unique(*o._audit_info) : nullptr) { } virtual ~cql_statement() { } @@ -111,6 +114,11 @@ public: virtual bool is_conditional() const { return false; } + + audit::audit_info* get_audit_info() { return _audit_info.get(); } + void set_audit_info(audit::audit_info_ptr&& info) { _audit_info = std::move(info); } + + virtual void sanitize_audit_info() {} }; class cql_statement_no_metadata : public cql_statement { diff --git a/cql3/statements/batch_statement.hh b/cql3/statements/batch_statement.hh index 6b7aa6bb9f16..9feb8971ce61 100644 --- a/cql3/statements/batch_statement.hh +++ b/cql3/statements/batch_statement.hh @@ -87,6 +87,8 @@ public: std::unique_ptr attrs, cql_stats& stats); + const std::vector& statements() const { return _statements; } + virtual bool depends_on(std::string_view ks_name, std::optional cf_name) const override; virtual uint32_t get_bound_terms() const override; diff --git a/cql3/statements/raw/modification_statement.hh b/cql3/statements/raw/modification_statement.hh index f96b1db51b59..9719b40d6bf4 100644 --- a/cql3/statements/raw/modification_statement.hh +++ b/cql3/statements/raw/modification_statement.hh @@ -28,6 +28,7 @@ class modification_statement; namespace raw { class modification_statement : public cf_statement { + sstring _raw_cql; protected: const std::unique_ptr _attrs; const std::optional _conditions; @@ -41,6 +42,8 @@ public: virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) override; ::shared_ptr prepare_statement(data_dictionary::database db, prepare_context& ctx, cql_stats& stats); ::shared_ptr prepare(data_dictionary::database db, prepare_context& ctx, cql_stats& stats) const; + void add_raw(sstring&& raw) { _raw_cql = std::move(raw); } + const sstring& get_raw_cql() const { return _raw_cql; } protected: virtual ::shared_ptr prepare_internal(data_dictionary::database db, schema_ptr schema, prepare_context& ctx, std::unique_ptr attrs, cql_stats& stats) const = 0; diff --git a/data_dictionary/data_dictionary.cc b/data_dictionary/data_dictionary.cc index 00cf400ab264..1245a3857ab6 100644 --- a/data_dictionary/data_dictionary.cc +++ b/data_dictionary/data_dictionary.cc @@ -237,9 +237,10 @@ keyspace_metadata::new_keyspace(std::string_view name, locator::replication_strategy_config_options options, std::optional initial_tablets, bool durables_writes, - storage_options storage_opts) + storage_options storage_opts, + std::vector cf_defs) { - return ::make_lw_shared(name, strategy_name, options, initial_tablets, durables_writes, std::vector{}, user_types_metadata{}, storage_opts); + return ::make_lw_shared(name, strategy_name, options, initial_tablets, durables_writes, cf_defs, user_types_metadata{}, storage_opts); } lw_shared_ptr diff --git a/data_dictionary/keyspace_metadata.hh b/data_dictionary/keyspace_metadata.hh index d1a66193fdf5..9c75a10f2ab5 100644 --- a/data_dictionary/keyspace_metadata.hh +++ b/data_dictionary/keyspace_metadata.hh @@ -48,7 +48,8 @@ public: locator::replication_strategy_config_options options, std::optional initial_tablets, bool durables_writes = true, - storage_options storage_opts = {}); + storage_options storage_opts = {}, + std::vector cf_defs = {}); static lw_shared_ptr new_keyspace(const keyspace_metadata& ksm); void validate(const gms::feature_service&, const locator::topology&) const; diff --git a/db/config.cc b/db/config.cc index da76b4d79529..73c712dfba6e 100644 --- a/db/config.cc +++ b/db/config.cc @@ -1259,11 +1259,24 @@ db::config::config(std::shared_ptr exts) , replication_strategy_warn_list(this, "replication_strategy_warn_list", liveness::LiveUpdate, value_status::Used, {locator::replication_strategy_type::simple}, "Controls which replication strategies to warn about when creating/altering a keyspace. Doesn't affect the pre-existing keyspaces.") , replication_strategy_fail_list(this, "replication_strategy_fail_list", liveness::LiveUpdate, value_status::Used, {}, "Controls which replication strategies are disallowed to be used when creating/altering a keyspace. Doesn't affect the pre-existing keyspaces.") , service_levels_interval(this, "service_levels_interval_ms", liveness::LiveUpdate, value_status::Used, 10000, "Controls how often service levels module polls configuration table") + + , audit(this, "audit", value_status::Used, "none", + "Controls the audit feature:\n" + "\n" + "\tnone : No auditing enabled.\n" + "\tsyslog : Audit messages sent to Syslog.\n" + "\ttable : Audit messages written to column family named audit.audit_log.\n") + , audit_categories(this, "audit_categories", value_status::Used, "DCL,DDL,AUTH", "Comma separated list of operation categories that should be audited.") + , audit_tables(this, "audit_tables", value_status::Used, "", "Comma separated list of table names (.
) that will be audited.") + , audit_keyspaces(this, "audit_keyspaces", value_status::Used, "", "Comma separated list of keyspaces that will be audited. All tables in those keyspaces will be audited") + , audit_unix_socket_path(this, "audit_unix_socket_path", value_status::Used, "/dev/log", "The path to the unix socket used for writting to syslog. Only applicable when audit is set to syslog.") + , audit_syslog_write_buffer_size(this, "audit_syslog_write_buffer_size", value_status::Used, 1048576, "The size (in bytes) of a write buffer used when writting to syslog socket.") , ldap_url_template(this, "ldap_url_template", value_status::Used, "", "LDAP URL template used by LDAPRoleManager for crafting queries.") , ldap_attr_role(this, "ldap_attr_role", value_status::Used, "", "LDAP attribute containing Scylla role.") , ldap_bind_dn(this, "ldap_bind_dn", value_status::Used, "", "Distinguished name used by LDAPRoleManager for binding to LDAP server.") , ldap_bind_passwd(this, "ldap_bind_passwd", value_status::Used, "", "Password used by LDAPRoleManager for binding to LDAP server.") , saslauthd_socket_path(this, "saslauthd_socket_path", value_status::Used, "", "UNIX domain socket on which saslauthd is listening.") + , error_injections_at_startup(this, "error_injections_at_startup", error_injection_value_status, {}, "List of error injections that should be enabled on startup.") , topology_barrier_stall_detector_threshold_seconds(this, "topology_barrier_stall_detector_threshold_seconds", value_status::Used, 2, "Report sites blocking topology barrier if it takes longer than this.") , enable_tablets(this, "enable_tablets", value_status::Used, false, "Enable tablets for newly created keyspaces.") diff --git a/db/config.hh b/db/config.hh index 630754d1b534..eb337126837a 100644 --- a/db/config.hh +++ b/db/config.hh @@ -504,6 +504,13 @@ public: named_value service_levels_interval; + named_value audit; + named_value audit_categories; + named_value audit_tables; + named_value audit_keyspaces; + named_value audit_unix_socket_path; + named_value audit_syslog_write_buffer_size; + named_value ldap_url_template; named_value ldap_attr_role; named_value ldap_bind_dn; diff --git a/table_helper.cc b/table_helper.cc index 39733e9db231..7a7bd3a3eba4 100644 --- a/table_helper.cc +++ b/table_helper.cc @@ -16,6 +16,7 @@ #include "cql3/statements/modification_statement.hh" #include "replica/database.hh" #include "service/migration_manager.hh" +#include "service/storage_proxy.hh" static logging::logger tlogger("table_helper"); @@ -139,7 +140,8 @@ future<> table_helper::insert(cql3::query_processor& qp, service::migration_mana co_await _insert_stmt->execute(qp, qs, opts, std::nullopt); } -future<> table_helper::setup_keyspace(cql3::query_processor& qp, service::migration_manager& mm, std::string_view keyspace_name, sstring replication_factor, service::query_state& qs, std::vector tables) { +future<> table_helper::setup_keyspace(cql3::query_processor& qp, service::migration_manager& mm, std::string_view keyspace_name, sstring replication_strategy_name, + sstring replication_factor, service::query_state& qs, std::vector tables) { if (this_shard_id() != 0) { co_return; } @@ -165,6 +167,15 @@ future<> table_helper::setup_keyspace(cql3::query_processor& qp, service::migrat auto ts = group0_guard.write_timestamp(); if (!db.has_keyspace(keyspace_name)) { + std::map opts; + if (replication_strategy_name == "org.apache.cassandra.locator.NetworkTopologyStrategy") { + for (const auto &dc: qp.proxy().get_token_metadata_ptr()->get_topology().get_datacenters()) + opts[dc] = replication_factor; + } + else { + opts["replication_factor"] = replication_factor; + } + auto ksm = keyspace_metadata::new_keyspace(keyspace_name, replication_strategy_name, std::move(opts), true); try { co_await mm.announce(service::prepare_new_keyspace_announcement(db.real_database(), ksm, ts), std::move(group0_guard), seastar::format("table_helper: create {} keyspace", keyspace_name)); diff --git a/table_helper.hh b/table_helper.hh index e3c8ac932efc..d064066cdc5e 100644 --- a/table_helper.hh +++ b/table_helper.hh @@ -99,7 +99,8 @@ public: future<> insert(cql3::query_processor& qp, service::migration_manager& mm, service::query_state& qs, noncopyable_function opt_maker); - static future<> setup_keyspace(cql3::query_processor& qp, service::migration_manager& mm, std::string_view keyspace_name, sstring replication_factor, service::query_state& qs, std::vector tables); + static future<> setup_keyspace(cql3::query_processor& qp, service::migration_manager& mm, std::string_view keyspace_name, sstring replication_strategy_name, + sstring replication_factor, service::query_state& qs, std::vector tables); /** * Makes a monotonically increasing value in 100ns ("nanos") based on the given time diff --git a/test/lib/CMakeLists.txt b/test/lib/CMakeLists.txt index 0dd73c7ff2c1..cf74cbb0615d 100644 --- a/test/lib/CMakeLists.txt +++ b/test/lib/CMakeLists.txt @@ -28,6 +28,7 @@ target_link_libraries(test-lib Seastar::seastar xxHash::xxhash PRIVATE + audit auth cdc compaction diff --git a/tracing/trace_keyspace_helper.cc b/tracing/trace_keyspace_helper.cc index c6067e253f98..b93a7bc0c470 100644 --- a/tracing/trace_keyspace_helper.cc +++ b/tracing/trace_keyspace_helper.cc @@ -212,7 +212,7 @@ trace_keyspace_helper::trace_keyspace_helper(tracing& tr) future<> trace_keyspace_helper::start(cql3::query_processor& qp, service::migration_manager& mm) { _qp_anchor = &qp; _mm_anchor = &mm; - return table_helper::setup_keyspace(qp, mm, KEYSPACE_NAME, "2", _dummy_query_state, { &_sessions, &_sessions_time_idx, &_events, &_slow_query_log, &_slow_query_log_time_idx }); + return table_helper::setup_keyspace(qp, mm, KEYSPACE_NAME, "org.apache.cassandra.locator.SimpleStrategy", "2", _dummy_query_state, { &_sessions, &_sessions_time_idx, &_events, &_slow_query_log, &_slow_query_log_time_idx }); } gms::inet_address trace_keyspace_helper::my_address() const noexcept { From 702e727e33dee7909ab5a2ab71077fe0eec4c11b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Zakrzewski?= Date: Mon, 13 Jan 2025 18:02:20 +0100 Subject: [PATCH 298/397] audit: Add documentation for the audit subsystem Adds detailed documentation covering the new audit subsystem: - Add new audit.md design document explaining: - Core concepts and design decisions - CQL extensions for audit management - Implementation details and trigger evaluation - Prior art references from other databases - Add user-facing documentation: - New auditing.rst guide with configuration and usage details - Integration with security documentation index - Updates to cluster management procedures - Updates to security checklist The documentation covers all aspects of the audit system including: - Configuration options and storage backends (syslog/table) - Audit categories (DCL/DDL/AUTH/DML/QUERY/ADMIN) - Permission model and security considerations - Failure handling and logging - Example configurations and output formats This ensures users have complete guidance for setting up and using the new audit capabilities. --- docs/dev/audit.md | 113 +++++++++ .../add-dc-to-existing-dc.rst | 1 + .../security/_common/security-index.rst | 1 + docs/operating-scylla/security/auditing.rst | 227 ++++++++++++++++++ docs/operating-scylla/security/index.rst | 2 + .../security/security-checklist.rst | 5 +- 6 files changed, 345 insertions(+), 4 deletions(-) create mode 100644 docs/dev/audit.md create mode 100644 docs/operating-scylla/security/auditing.rst diff --git a/docs/dev/audit.md b/docs/dev/audit.md new file mode 100644 index 000000000000..7304558d9497 --- /dev/null +++ b/docs/dev/audit.md @@ -0,0 +1,113 @@ +# Introduction + +Similar to the approach described in CASSANDRA-14471, we add the +concept of an audit specification. An audit has a target (syslog or a +table) and a set of events/actions that it wants recorded. We +introduce new CQL syntax for Scylla users to describe and manipulate +audit specifications. + +Prior art: +- Microsoft SQL Server [audit + description](https://docs.microsoft.com/en-us/sql/relational-databases/security/auditing/sql-server-audit-database-engine?view=sql-server-ver15) +- pgAudit [docs](https://github.com/pgaudit/pgaudit/blob/master/README.md) +- MySQL audit_log docs in + [MySQL](https://dev.mysql.com/doc/refman/8.0/en/audit-log.html) and + [Azure](https://docs.microsoft.com/en-us/azure/mysql/concepts-audit-logs) +- DynamoDB can [use CloudTrail](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/logging-using-cloudtrail.html) to log all events + +# CQL extensions + +## Create an audit + +```cql +CREATE AUDIT [IF NOT EXISTS] audit-name WITH TARGET { SYSLOG | table-name } +[ AND TRIGGER KEYSPACE IN (ks1, ks2, ks3) ] +[ AND TRIGGER TABLE IN (tbl1, tbl2, tbl3) ] +[ AND TRIGGER ROLE IN (usr1, usr2, usr3) ] +[ AND TRIGGER CATEGORY IN (cat1, cat2, cat3) ] +; +``` + +From this point on, every database event that matches all present +triggers will be recorded in the target. When the target is a table, +it behaves like the [current +design](https://docs.scylladb.com/operating-scylla/security/auditing/#table-storage). + +The audit name must be different from all other audits, unless IF NOT +EXISTS precedes it, in which case the existing audit must be identical +to the new definition. Case sensitivity and length limit are the same +as for table names. + +A trigger kind (ie, `KEYSPACE`, `TABLE`, `ROLE`, or `CATEGORY`) can be +specified at most once. + +## Show an audit + +```cql +DESCRIBE AUDIT [audit-name ...]; +``` + +Prints definitions of all audits named herein. If no names are +provided, prints all audits. + +## Delete an audit + +```cql +DROP AUDIT audit-name; +``` + +Stops logging events specified by this audit. Doesn't impact the +already logged events. If the target is a table, it remains as it is. + +## Alter an audit + +```cql +ALTER AUDIT audit-name WITH {same syntax as CREATE} +``` + +Any trigger provided will be updated (or newly created, if previously +absent). To drop a trigger, use `IN *`. + +## Permissions + +Only superusers can modify audits or turn them on and off. + +Only superusers can read tables that are audit targets; no user can +modify them. Only superusers can drop tables that are audit targets, +after the audit itself is dropped. If a superuser doesn't drop a +target table, it remains in existence indefinitely. + +# Implementation + +## Efficient trigger evaluation + +```c++ +namespace audit { + +/// Stores triggers from an AUDIT statement. +class triggers { + // Use trie structures for speedy string lookup. + optional _ks_trigger, _tbl_trigger, _usr_trigger; + + // A logical-AND filter. + optional _cat_trigger; + +public: + /// True iff every non-null trigger matches the corresponding ainf element. + bool should_audit(const audit_info& ainf); +}; + +} // namespace audit +``` + +To prevent modification of target tables, `audit::inspect()` will +check the statement and throw if it is disallowed, similar to what +`check_access()` currently does. + +## Persisting audit definitions + +Obviously, an audit definition must survive a server restart and stay +consistent among all nodes in a cluster. We'll accomplish both by +storing audits in a system table. They will be cached in memory the +same way `permissions_cache` caches table contents in `permission_set` +objects resident in memory. diff --git a/docs/operating-scylla/procedures/cluster-management/add-dc-to-existing-dc.rst b/docs/operating-scylla/procedures/cluster-management/add-dc-to-existing-dc.rst index 9fc676cc519a..5d73b437d4e0 100644 --- a/docs/operating-scylla/procedures/cluster-management/add-dc-to-existing-dc.rst +++ b/docs/operating-scylla/procedures/cluster-management/add-dc-to-existing-dc.rst @@ -161,6 +161,7 @@ Add New DC * Keyspace created by the user (which needed to replicate to the new DC). * System: ``system_distributed``, ``system_traces``, for example, replicate the data to three nodes in the new DC. + * ``audit`` - if enabled - replicate the data to three nodes in the new DC. For example: diff --git a/docs/operating-scylla/security/_common/security-index.rst b/docs/operating-scylla/security/_common/security-index.rst index fae922e75a87..cad2824d41de 100644 --- a/docs/operating-scylla/security/_common/security-index.rst +++ b/docs/operating-scylla/security/_common/security-index.rst @@ -5,6 +5,7 @@ * :doc:`Enable Authorization` * :doc:`Grant Authorization CQL Reference ` * :doc:`Role Based Access Control (RBAC) ` +* :doc:`Scylla Auditing Guide ` * :doc:`Encryption: Data in Transit Client to Node ` * :doc:`Encryption: Data in Transit Node to Node ` * :doc:`Generating a self-signed Certificate Chain Using openssl ` diff --git a/docs/operating-scylla/security/auditing.rst b/docs/operating-scylla/security/auditing.rst new file mode 100644 index 000000000000..d5f472aa6f09 --- /dev/null +++ b/docs/operating-scylla/security/auditing.rst @@ -0,0 +1,227 @@ +======================== +ScyllaDB Auditing Guide +======================== + +:label-tip:`ScyllaDB Enterprise` + + +Auditing allows the administrator to monitor activities on a Scylla cluster, including queries and data changes. +The information is stored in a Syslog or a Scylla table. + +Prerequisite +------------ + +Enable ScyllaDB :doc:`Authentication ` and :doc:`Authorization `. + + +Enabling Audit +--------------- + +By default, auditing is **disabled**. Enabling auditing is controlled by the ``audit:`` parameter in the ``scylla.yaml`` file. +You can set the following options: + +* ``none`` - Audit is disabled (default). +* ``table`` - Audit is enabled, and messages are stored in a Scylla table. +* ``syslog`` - Audit is enabled, and messages are sent to Syslog. + +Configuring any other value results in an error at Scylla startup. + +Configuring Audit +----------------- + +The audit can be tuned using the following flags or ``scylla.yaml`` entries: + +================== ================================== ======================================================================================================================== +Flag Default Value Description +================== ================================== ======================================================================================================================== +audit_categories "DCL,DDL,AUTH,ADMIN" Comma-separated list of statement categories that should be audited +------------------ ---------------------------------- ------------------------------------------------------------------------------------------------------------------------ +audit_tables “” Comma-separated list of table names that should be audited, in the format of . +------------------ ---------------------------------- ------------------------------------------------------------------------------------------------------------------------ +audit_keyspaces “” Comma-separated list of keyspaces that should be audited. You must specify at least one keyspace. + If you leave this option empty, no keyspace will be audited. +================== ================================== ======================================================================================================================== + +To audit all the tables in a keyspace, set the ``audit_keyspaces`` with the keyspace you want to audit and leave ``audit_tables`` empty. + +You can use DCL, AUTH, and ADMIN audit categories without including any keyspace or table. + +audit_categories parameter description +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +========= ========================================================================================= +Parameter Logs Description +========= ========================================================================================= +AUTH Logs login events +--------- ----------------------------------------------------------------------------------------- +DML Logs insert, update, delete, and other data manipulation language (DML) events +--------- ----------------------------------------------------------------------------------------- +DDL Logs object and role create, alter, drop, and other data definition language (DDL) events +--------- ----------------------------------------------------------------------------------------- +DCL Logs grant, revoke, create role, drop role, and list roles events +--------- ----------------------------------------------------------------------------------------- +QUERY Logs all queries +--------- ----------------------------------------------------------------------------------------- +ADMIN Logs service level operations: create, alter, drop, attach, detach, list. + For :ref:`service level ` + auditing, this parameter is available in Scylla Enterprise 2019.1 and later. +========= ========================================================================================= + +Note that audit for every DML or QUERY might impact performance and consume a lot of storage. + +Configuring Audit Storage +--------------------------- + +Auditing messages can be sent to :ref:`Syslog ` or stored in a Scylla :ref:`table `. +Currently, auditing messages can only be saved to one location at a time. You cannot log into both a table and the Syslog. + +.. _auditing-syslog-storage: + +Storing Audit Messages in Syslog +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +**Procedure** + +#. Set the ``audit`` parameter in the ``scylla.yaml`` file to ``syslog``. + + For example: + + .. code-block:: shell + + # audit setting + # by default, Scylla does not audit anything. + # It is possible to enable auditing to the following places: + # - audit.audit_log column family by setting the flag to "table" + audit: "syslog" + # + # List of statement categories that should be audited. + audit_categories: "DCL,DDL,AUTH" + # + # List of tables that should be audited. + audit_tables: "mykespace.mytable" + # + # List of keyspaces that should be fully audited. + # All tables in those keyspaces will be audited + audit_keyspaces: "mykespace" + +#. Restart the Scylla node. + +.. include:: /rst_include/scylla-commands-restart-index.rst + +By default, audit messages are written to the same destination as Scylla :doc:`logging `, with ``scylla-audit`` as the process name. + +Logging output example (drop table): + +.. code-block:: shell + + Mar 18 09:53:52 ip-10-143-2-108 scylla-audit[28387]: "10.143.2.108", "DDL", "ONE", "team_roster", "nba", "DROP TABLE nba.team_roster ;", "127.0.0.1", "anonymous", "false" + +To redirect the Syslog output to a file, follow the steps below (available only for CentOS) : + +#. Install rsyslog sudo ``dnf install rsyslog``. +#. Edit ``/etc/rsyslog.conf`` and append the following to the file: ``if $programname contains 'scylla-audit' then /var/log/scylla-audit.log``. +#. Start rsyslog ``systemctl start rsyslog``. +#. Enable rsyslog ``systemctl enable rsyslog``. + +.. _auditing-table-storage: + +Storing Audit Messages in a Table +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Messages are stored in a Scylla table named ``audit.audit_log``. + +For example: + +.. code-block:: shell + + CREATE TABLE IF NOT EXISTS audit.audit_log ( + date timestamp, + node inet, + event_time timeuuid, + category text, + consistency text, + table_name text, + keyspace_name text, + operation text, + source inet, + username text, + error boolean, + PRIMARY KEY ((date, node), event_time)); + +.. note:: The schema of ``audit.audit_log`` has been migrated in the 2024.2 version from ``SimpleStrategy RF=1`` to ``NetworkTopologyStrategy RF=3``: + + * By default every DC will contain 3 audit replicas. If a new DC is added, in order for it to also contain audit replicas, audit's schema has to be manually altered. + * CL for writes is still equal to ``1``, which implies that reading audit rows with CL=Quorum may fail, which is especially true for clusters with less than 3 nodes. + +**Procedure** + +#. Set the ``audit`` parameter in the ``scylla.yaml`` file to ``table``. + + For example: + + .. code-block:: shell + + # audit setting + # by default, Scylla does not audit anything. + # It is possible to enable auditing to the following places: + # - audit.audit_log column family by setting the flag to "table" + audit: "table" + # + # List of statement categories that should be audited. + audit_categories: "DCL,DDL,AUTH" + # + # List of tables that should be audited. + audit_tables: "mykespace.mytable" + # + # List of keyspaces that should be fully audited. + # All tables in those keyspaces will be audited + audit_keyspaces: "mykespace" + +#. Restart Scylla node. + + .. include:: /rst_include/scylla-commands-restart-index.rst + + Table output example (drop table): + + .. code-block:: shell + + SELECT * FROM audit.audit_log ; + + returns: + + .. code-block:: none + + date | node | event_time | category | consistency | error | keyspace_name | operation | source | table_name | username | + -------------------------+--------------+--------------------------------------+----------+-------------+-------+---------------+------------------------------+-----------------+-------------+----------+ + 2018-03-18 00:00:00+0000 | 10.143.2.108 | 3429b1a5-2a94-11e8-8f4e-000000000001 | DDL | ONE | False | nba | DROP TABLE nba.team_roster ; | 127.0.0.1 | team_roster | Scylla | + (1 row) + +Handling Audit Failures +--------------------------- + +In some cases, auditing may not be possible, for example, when: + +* A table is used as the audit’s backend, and the audit partition where the audit row is saved is not available because the node that holds this partition is down. +* Syslog is used as the audit’s backend, and the Syslog sink (a regular unix socket) is unresponsive/unavailable. + +If the audit fails and audit messages are not stored in the configured audit’s backend, you can still review the audit log in the regular ScyllaDB logs. + +The following example shows audit information in the regular ScyllaDB logs in the case when the Syslog backend is broken (for example, because the socket was closed) and you tried to connect to a node with incorrect credentials: + + .. code-block:: shell + + ERROR 2024-01-15 14:09:41,516 [shard 0:sl:d] audit - Unexpected exception when writing login log with: node_ip client_ip username error true exception audit::audit_exception (Starting syslog audit backend failed (sending a message to resulted in sendto: No such file or directory).) + +Additional Resources +----------------------------------- + +* :doc:`Authorization` + +* :doc:`Authentication` + + + + + + + diff --git a/docs/operating-scylla/security/index.rst b/docs/operating-scylla/security/index.rst index 4ae8e9c5193c..f1bf14b5246f 100644 --- a/docs/operating-scylla/security/index.rst +++ b/docs/operating-scylla/security/index.rst @@ -14,6 +14,7 @@ Security authorization certificate-authentication rbac-usecase + auditing client-node-encryption node-node-encryption generate-certificate @@ -28,6 +29,7 @@ Security :class: my-panel * :doc:`ScyllaDB Security Checklist ` + * :doc:`ScyllaDB Auditing Guide ` .. panel-box:: :title: Authentication and Authorization diff --git a/docs/operating-scylla/security/security-checklist.rst b/docs/operating-scylla/security/security-checklist.rst index 737b07a82613..3180d17ecdcd 100644 --- a/docs/operating-scylla/security/security-checklist.rst +++ b/docs/operating-scylla/security/security-checklist.rst @@ -71,10 +71,7 @@ The ScyllaDB ports are detailed in the table below. For ScyllaDB Manager ports, Audit System Activity ~~~~~~~~~~~~~~~~~~~~~ - -Auditing is available in `ScyllaDB Enterprise `_. - -Using the `auditing feature `_ allows the administrator to know “who did / looked at / changed what and when.” It also allows logging some or all the activities a user performs on the ScyllaDB cluster. +Using the :doc:`auditing feature` allows the administrator to know “who did / looked at / changed what and when.” It also allows logging some or all the activities a user performs on the ScyllaDB cluster. General Recommendations ~~~~~~~~~~~~~~~~~~~~~~~ From 1810e2e424cd25da9b12fb84d4ca916bec6c4973 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Zakrzewski?= Date: Mon, 16 Dec 2024 12:05:55 +0100 Subject: [PATCH 299/397] audit: Integrate audit subsystem into Scylla main process Adds core integration of the audit subsystem into Scylla's main process flow. Changes include: - Import audit subsystem header - Initialize audit system during server startup using configuration and token metadata - Start audit system after API server initialization with query processor and memory manager - Add proper shutdown sequence for audit system using RAII pattern - Add error handling for audit system initialization failures The audit system is now properly integrated into Scylla's lifecycle, ensuring: - Correct initialization order relative to other subsystems - Proper resource cleanup during shutdown - Graceful error handling for initialization failures --- main.cc | 9 +++++++++ test/perf/perf_simple_query.cc | 25 +++++++++++++++++++++++++ test/rest_api/test_storage_service.py | 2 +- 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/main.cc b/main.cc index fbc7ac992eea..0caef341086f 100644 --- a/main.cc +++ b/main.cc @@ -67,6 +67,7 @@ #include #include #include "tracing/tracing.hh" +#include "audit/audit.hh" #include #include "message/messaging_service.hh" #include "db/sstables-format-selector.hh" @@ -1369,6 +1370,9 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl auto destroy_tracing = defer_verbose_shutdown("tracing instance", [&tracing] { tracing.stop().get(); }); + audit::audit::create_audit(*cfg, token_metadata).handle_exception([&] (auto&& e) { + startlog.error("audit creation failed: {}", e); + }).get(); with_scheduling_group(maintenance_scheduling_group, [&] { return ctx.http_server.listen(socket_address{api_addr, cfg->api_port()}); @@ -2308,6 +2312,11 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl seastar::set_abort_on_ebadf(cfg->abort_on_ebadf()); api::set_server_done(ctx).get(); + audit::audit::start_audit(*cfg, qp, mm).get(); + auto audit_stop = defer([] { + audit::audit::stop_audit().get(); + }); + // Create controllers before drain_on_shutdown() below, so that it destructs // after drain stops them in stop_transport() // Register controllers after drain_on_shutdown() below, so that even on start diff --git a/test/perf/perf_simple_query.cc b/test/perf/perf_simple_query.cc index 18ce46336b30..48ebb0403e0f 100644 --- a/test/perf/perf_simple_query.cc +++ b/test/perf/perf_simple_query.cc @@ -19,6 +19,7 @@ #include #include #include "test/lib/random_utils.hh" +#include "db/config.hh" #include "db/config.hh" #include "schema/schema_builder.hh" @@ -521,6 +522,15 @@ void write_json_result(std::string result_file, const test_config& cfg, const ag out << results; } +/// If app configuration contains the named parameter, store its value into \p store. +static void set_from_cli(const char* name, app_template& app, utils::config_file::named_value& store) { + const auto& cfg = app.configuration(); + auto found = cfg.find(name); + if (found != cfg.end()) { + store(found->second.as()); + } +} + namespace perf { int scylla_simple_query_main(int argc, char** argv) { @@ -546,6 +556,10 @@ int scylla_simple_query_main(int argc, char** argv) { ("stop-on-error", bpo::value()->default_value(true), "stop after encountering the first error") ("timeout", bpo::value()->default_value(""), "use timeout") ("bypass-cache", "use bypass cache when querying") + ("audit", bpo::value(), "value for audit config entry") + ("audit-keyspaces", bpo::value(), "value for audit_keyspaces config entry") + ("audit-tables", bpo::value(), "value for audit_tables config entry") + ("audit-categories", bpo::value(), "value for audit_categories config entry") ; set_abort_on_internal_error(true); @@ -569,6 +583,10 @@ int scylla_simple_query_main(int argc, char** argv) { cfg.db_config->enable_tablets.set(true); cfg.initial_tablets = app.configuration()["initial-tablets"].as(); } + set_from_cli("audit", app, cfg.db_config->audit); + set_from_cli("audit-keyspaces", app, cfg.db_config->audit_keyspaces); + set_from_cli("audit-tables", app, cfg.db_config->audit_tables); + set_from_cli("audit-categories", app, cfg.db_config->audit_categories); return do_with_cql_env_thread([&app] (auto&& env) { auto cfg = test_config(); cfg.partitions = app.configuration()["partitions"].as(); @@ -601,6 +619,13 @@ int scylla_simple_query_main(int argc, char** argv) { cfg.stop_on_error = app.configuration()["stop-on-error"].as(); cfg.timeout = app.configuration()["timeout"].as(); cfg.bypass_cache = app.configuration().contains("bypass-cache"); + audit::audit::create_audit(env.local_db().get_config(), env.get_shared_token_metadata()).handle_exception([&] (auto&& e) { + fmt::print("audit creation failed: {}", e); + }).get(); + audit::audit::start_audit(env.local_db().get_config(), env.qp(), env.migration_manager()).get(); + auto audit_stop = defer([] { + audit::audit::stop_audit().get(); + }); auto results = cfg.frontend == test_config::frontend_type::cql ? do_cql_test(env, cfg) : do_alternator_test(app.configuration()["alternator"].as(), diff --git a/test/rest_api/test_storage_service.py b/test/rest_api/test_storage_service.py index 711df2bdcb6e..72c4d9e92fdc 100644 --- a/test/rest_api/test_storage_service.py +++ b/test/rest_api/test_storage_service.py @@ -332,7 +332,7 @@ def verify_snapshot_details(rest_api, expected): assert not found found = True sort_key = lambda v: f"{v['ks']}-{v['cf']}" - value = sorted([v for v in data['value'] if not v['ks'].startswith('system')], key=sort_key) + value = sorted([v for v in data['value'] if not v['ks'].startswith('system') and not v['ks'] == 'audit'], key=sort_key) expected_value = sorted(expected['value'], key=sort_key) assert len(value) == len(expected_value), f"length mismatch: expected {expected_value} but got {value}" for i in range(len(value)): From 98f5e49ea832e91017e024e5c442ed5af0fa119e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Zakrzewski?= Date: Mon, 16 Dec 2024 10:37:52 +0100 Subject: [PATCH 300/397] audit: Add support to CQL statements Integrates audit functionality into CQL statement processing to enable tracking of database operations. Key changes: - Add audit_info and statement_category to all CQL statements - Implement audit categories for different statement types: - DDL: Schema altering statements (CREATE/ALTER/DROP) - DML: Data manipulation (INSERT/UPDATE/DELETE/TRUNCATE/USE) - DCL: Access control (GRANT/REVOKE/CREATE ROLE) - QUERY: SELECT statements - ADMIN: Service level operations - Add audit inspection points in query processing: - Before statement execution - After access checks - After statement completion - On execution failures - Add password sanitization for role management statements - Mask plaintext passwords in audit logs - Handle both direct password parameters and options maps - Preserve query structure while hiding sensitive data - Modify prepared statement lifecycle to carry audit context - Pass audit info during statement preparation - Track audit info through statement execution - Support batch statement auditing This change enables comprehensive auditing of CQL operations while ensuring sensitive data is properly masked in audit logs. --- cql3/query_processor.cc | 39 ++++++-- cql3/statements/alter_keyspace_statement.cc | 2 +- cql3/statements/alter_role_statement.hh | 2 + .../alter_service_level_statement.cc | 2 +- cql3/statements/alter_table_statement.cc | 2 +- cql3/statements/alter_table_statement.hh | 2 + cql3/statements/alter_type_statement.cc | 4 +- cql3/statements/alter_view_statement.cc | 2 +- .../attach_service_level_statement.cc | 2 +- cql3/statements/authentication_statement.cc | 4 + cql3/statements/authentication_statement.hh | 5 ++ cql3/statements/authorization_statement.cc | 5 ++ cql3/statements/authorization_statement.hh | 4 + cql3/statements/batch_statement.cc | 10 ++- cql3/statements/create_aggregate_statement.cc | 13 ++- cql3/statements/create_aggregate_statement.hh | 3 + cql3/statements/create_function_statement.cc | 12 ++- cql3/statements/create_function_statement.hh | 4 + cql3/statements/create_index_statement.cc | 2 +- cql3/statements/create_keyspace_statement.cc | 2 +- cql3/statements/create_role_statement.hh | 2 + .../create_service_level_statement.cc | 2 +- cql3/statements/create_table_statement.cc | 6 +- cql3/statements/create_table_statement.hh | 2 + cql3/statements/create_type_statement.cc | 2 +- cql3/statements/create_view_statement.cc | 2 +- cql3/statements/delete_statement.cc | 8 +- cql3/statements/delete_statement.hh | 2 +- cql3/statements/describe_statement.cc | 12 ++- .../detach_service_level_statement.cc | 2 +- cql3/statements/drop_aggregate_statement.cc | 11 ++- cql3/statements/drop_aggregate_statement.hh | 3 + cql3/statements/drop_function_statement.cc | 12 ++- cql3/statements/drop_function_statement.hh | 4 + cql3/statements/drop_index_statement.cc | 2 +- cql3/statements/drop_keyspace_statement.cc | 2 +- .../drop_service_level_statement.cc | 2 +- cql3/statements/drop_table_statement.cc | 2 +- cql3/statements/drop_type_statement.cc | 2 +- cql3/statements/drop_view_statement.cc | 2 +- cql3/statements/grant_statement.cc | 2 +- .../list_effective_service_level_statement.cc | 2 +- cql3/statements/list_permissions_statement.cc | 2 +- ...ist_service_level_attachments_statement.cc | 2 +- .../list_service_level_statement.cc | 2 +- cql3/statements/list_users_statement.cc | 2 +- cql3/statements/modification_statement.cc | 6 +- cql3/statements/prepared_statement.hh | 10 ++- cql3/statements/raw/batch_statement.hh | 6 ++ cql3/statements/raw/cf_statement.hh | 4 + cql3/statements/raw/describe_statement.hh | 2 + cql3/statements/raw/modification_statement.hh | 2 + cql3/statements/raw/parsed_statement.cc | 18 ++-- cql3/statements/raw/parsed_statement.hh | 5 ++ cql3/statements/raw/select_statement.hh | 2 + cql3/statements/raw/truncate_statement.hh | 2 + cql3/statements/raw/use_statement.hh | 5 ++ cql3/statements/revoke_statement.cc | 2 +- cql3/statements/role-management-statements.cc | 89 +++++++++++++++++-- cql3/statements/schema_altering_statement.cc | 4 + cql3/statements/schema_altering_statement.hh | 2 + cql3/statements/select_statement.cc | 7 +- cql3/statements/service_level_statement.cc | 10 ++- cql3/statements/service_level_statement.hh | 4 +- cql3/statements/truncate_statement.cc | 6 +- cql3/statements/update_statement.cc | 11 ++- cql3/statements/update_statement.hh | 4 +- cql3/statements/use_statement.cc | 7 +- 68 files changed, 351 insertions(+), 72 deletions(-) diff --git a/cql3/query_processor.cc b/cql3/query_processor.cc index d5c2301fdfa4..621fe8065152 100644 --- a/cql3/query_processor.cc +++ b/cql3/query_processor.cc @@ -13,6 +13,7 @@ #include #include #include +#include #include "service/storage_proxy.hh" #include "service/migration_manager.hh" @@ -595,11 +596,27 @@ query_processor::do_execute_direct( const query_options& options, std::optional guard, cql3::cql_warnings_vec warnings) { - co_await statement->check_access(*this, query_state.get_client_state()); - auto m = co_await process_authorized_statement(statement, query_state, options, std::move(guard)); - for (const auto& w : warnings) { - m->add_warning(w); + auto access_future = co_await coroutine::as_future(statement->check_access(*this, query_state.get_client_state())); + if (access_future.failed()) { + co_await audit::inspect(statement, query_state, options, true); + std::rethrow_exception(access_future.get_exception()); } + auto mfut = co_await coroutine::as_future(process_authorized_statement(statement, query_state, options, std::move(guard))); + ::shared_ptr m; + if (!mfut.failed()) { + m = mfut.get(); + } else { + co_await audit::inspect(statement, query_state, options, true); + std::rethrow_exception(mfut.get_exception()); + } + bool is_error = true; + if (m.get()) { + is_error = m->is_exception(); + for (const auto& w : warnings) { + m->add_warning(w); + } + } + co_await audit::inspect(statement, query_state, options, is_error); co_return std::move(m); } @@ -631,6 +648,8 @@ query_processor::do_execute_prepared( log.error("failed to cache the entry: {}", std::current_exception()); } } + + co_await audit::inspect(statement, query_state, options, false); co_return co_await process_authorized_statement(std::move(statement), query_state, options, std::move(guard)); } @@ -694,6 +713,11 @@ query_processor::get_statement(const std::string_view& query, const service::cli ++_stats.prepare_invocations; auto p = statement->prepare(_db, _cql_stats); p->statement->raw_cql_statement = sstring(query); + auto audit_info = p->statement->get_audit_info(); + if (audit_info) { + audit_info->set_query_string(query); + p->statement->sanitize_audit_info(); + } return p; } @@ -964,7 +988,7 @@ query_processor::execute_batch_without_checking_exception_message( service::query_state& query_state, query_options& options, std::unordered_map pending_authorization_entries) { - co_await batch->check_access(*this, query_state.get_client_state()); + auto access_future = co_await coroutine::as_future(batch->check_access(*this, query_state.get_client_state())); co_await coroutine::parallel_for_each(pending_authorization_entries, [this, &query_state] (auto& e) -> future<> { try { co_await _authorized_prepared_cache.insert(*query_state.get_client_state().user(), e.first, std::move(e.second)); @@ -972,6 +996,11 @@ query_processor::execute_batch_without_checking_exception_message( log.error("failed to cache the entry: {}", std::current_exception()); } }); + bool failed = access_future.failed(); + co_await audit::inspect(batch, query_state, options, failed); + if (access_future.failed()) { + std::rethrow_exception(access_future.get_exception()); + } batch->validate(); batch->validate(*this, query_state.get_client_state()); _stats.queries_by_cl[size_t(options.get_consistency())] += batch->get_statements().size(); diff --git a/cql3/statements/alter_keyspace_statement.cc b/cql3/statements/alter_keyspace_statement.cc index 42cbeca93223..e2b3ec79fbd5 100644 --- a/cql3/statements/alter_keyspace_statement.cc +++ b/cql3/statements/alter_keyspace_statement.cc @@ -259,7 +259,7 @@ cql3::statements::alter_keyspace_statement::prepare_schema_mutations(query_proce std::unique_ptr cql3::statements::alter_keyspace_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } diff --git a/cql3/statements/alter_role_statement.hh b/cql3/statements/alter_role_statement.hh index 5648c08e44ef..ae3d5ddbc793 100644 --- a/cql3/statements/alter_role_statement.hh +++ b/cql3/statements/alter_role_statement.hh @@ -39,6 +39,8 @@ public: virtual future<::shared_ptr> execute(query_processor&, service::query_state&, const query_options&, std::optional guard) const override; + + virtual void sanitize_audit_info() override; }; } diff --git a/cql3/statements/alter_service_level_statement.cc b/cql3/statements/alter_service_level_statement.cc index b47b4460f1d0..c01f73a92675 100644 --- a/cql3/statements/alter_service_level_statement.cc +++ b/cql3/statements/alter_service_level_statement.cc @@ -26,7 +26,7 @@ alter_service_level_statement::alter_service_level_statement(sstring service_lev std::unique_ptr cql3::statements::alter_service_level_statement::prepare( data_dictionary::database db, cql_stats &stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> alter_service_level_statement::check_access(query_processor& qp, const service::client_state &state) const { diff --git a/cql3/statements/alter_table_statement.cc b/cql3/statements/alter_table_statement.cc index 3b73282e4642..7ae3783d0957 100644 --- a/cql3/statements/alter_table_statement.cc +++ b/cql3/statements/alter_table_statement.cc @@ -455,7 +455,7 @@ alter_table_statement::raw_statement::prepare(data_dictionary::database db, cql_ auto prepared_attrs = _attrs->prepare(db, keyspace(), column_family()); prepared_attrs->fill_prepare_context(ctx); - return std::make_unique(::make_shared( + return std::make_unique(audit_info(), ::make_shared( ctx.bound_variables_size(), *_cf_name, _type, diff --git a/cql3/statements/alter_table_statement.hh b/cql3/statements/alter_table_statement.hh index 28df9c8ef287..27c84d3a52d4 100644 --- a/cql3/statements/alter_table_statement.hh +++ b/cql3/statements/alter_table_statement.hh @@ -88,6 +88,8 @@ public: std::unique_ptr attrs); virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) override; + + virtual audit::statement_category category() const override { return audit::statement_category::DDL; } }; } diff --git a/cql3/statements/alter_type_statement.cc b/cql3/statements/alter_type_statement.cc index eddb963050a8..b753059505fd 100644 --- a/cql3/statements/alter_type_statement.cc +++ b/cql3/statements/alter_type_statement.cc @@ -211,12 +211,12 @@ user_type alter_type_statement::renames::make_updated_type(data_dictionary::data std::unique_ptr alter_type_statement::add_or_alter::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } std::unique_ptr alter_type_statement::renames::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } } diff --git a/cql3/statements/alter_view_statement.cc b/cql3/statements/alter_view_statement.cc index 127da0980278..93f9641a267b 100644 --- a/cql3/statements/alter_view_statement.cc +++ b/cql3/statements/alter_view_statement.cc @@ -90,7 +90,7 @@ future, std::vector std::unique_ptr alter_view_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } } diff --git a/cql3/statements/attach_service_level_statement.cc b/cql3/statements/attach_service_level_statement.cc index bbb0cd5d4dfc..2dd0ed44099a 100644 --- a/cql3/statements/attach_service_level_statement.cc +++ b/cql3/statements/attach_service_level_statement.cc @@ -31,7 +31,7 @@ bool attach_service_level_statement::needs_guard(query_processor& qp, service::q std::unique_ptr cql3::statements::attach_service_level_statement::prepare( data_dictionary::database db, cql_stats &stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> attach_service_level_statement::check_access(query_processor& qp, const service::client_state &state) const { diff --git a/cql3/statements/authentication_statement.cc b/cql3/statements/authentication_statement.cc index 4e3caa2a046e..7b5ca1d0d82a 100644 --- a/cql3/statements/authentication_statement.cc +++ b/cql3/statements/authentication_statement.cc @@ -28,3 +28,7 @@ future<> cql3::statements::authentication_statement::check_access(query_processo bool cql3::statements::authentication_altering_statement::needs_guard(query_processor& qp, service::query_state&) const { return !auth::legacy_mode(qp); } + +audit::statement_category cql3::statements::authentication_statement::category() const { + return audit::statement_category::DCL; +} diff --git a/cql3/statements/authentication_statement.hh b/cql3/statements/authentication_statement.hh index 3e4f51008d3f..43b8d5182906 100644 --- a/cql3/statements/authentication_statement.hh +++ b/cql3/statements/authentication_statement.hh @@ -26,6 +26,11 @@ public: bool depends_on(std::string_view ks_name, std::optional cf_name) const override; future<> check_access(query_processor& qp, const service::client_state& state) const override; +protected: + virtual audit::statement_category category() const override; + virtual audit::audit_info_ptr audit_info() const override { + return audit::audit::create_audit_info(category(), sstring(), sstring()); + } }; class authentication_altering_statement : public authentication_statement { diff --git a/cql3/statements/authorization_statement.cc b/cql3/statements/authorization_statement.cc index ea3ca4712f68..e29441a00b1f 100644 --- a/cql3/statements/authorization_statement.cc +++ b/cql3/statements/authorization_statement.cc @@ -76,3 +76,8 @@ bool cql3::statements::authorization_altering_statement::needs_guard( query_processor& qp, service::query_state&) const { return !auth::legacy_mode(qp); }; + +audit::statement_category cql3::statements::authorization_statement::category() const { + return audit::statement_category::DCL; +} + diff --git a/cql3/statements/authorization_statement.hh b/cql3/statements/authorization_statement.hh index 79343cf14f79..2c130d56da61 100644 --- a/cql3/statements/authorization_statement.hh +++ b/cql3/statements/authorization_statement.hh @@ -33,6 +33,10 @@ public: protected: static void maybe_correct_resource(auth::resource&, const service::client_state&, query_processor&); + virtual audit::statement_category category() const override; + virtual audit::audit_info_ptr audit_info() const override { + return audit::audit::create_audit_info(category(), sstring(), sstring()); + } }; class authorization_altering_statement : public authorization_statement { diff --git a/cql3/statements/batch_statement.cc b/cql3/statements/batch_statement.cc index b6977d2dfc4b..eb43f0486523 100644 --- a/cql3/statements/batch_statement.cc +++ b/cql3/statements/batch_statement.cc @@ -434,6 +434,10 @@ batch_statement::prepare(data_dictionary::database db, cql_stats& stats) { have_multiple_cfs = first_ks.value() != parsed->keyspace() || first_cf.value() != parsed->column_family(); } statements.emplace_back(parsed->prepare(db, meta, stats)); + auto audit_info = statements.back().statement->get_audit_info(); + if (audit_info) { + audit_info->set_query_string(parsed->get_raw_cql()); + } } auto&& prep_attrs = _attrs->prepare(db, "[batch]", "[batch]"); @@ -445,11 +449,15 @@ batch_statement::prepare(data_dictionary::database db, cql_stats& stats) { if (!have_multiple_cfs && batch_statement_.get_statements().size() > 0) { partition_key_bind_indices = meta.get_partition_key_bind_indexes(*batch_statement_.get_statements()[0].statement->s); } - return std::make_unique(make_shared(std::move(batch_statement_)), + return std::make_unique(audit_info(), make_shared(std::move(batch_statement_)), meta.get_variable_specifications(), std::move(partition_key_bind_indices)); } +audit::statement_category batch_statement::category() const { + return audit::statement_category::DML; +} + } diff --git a/cql3/statements/create_aggregate_statement.cc b/cql3/statements/create_aggregate_statement.cc index 8c133f3f8b8d..a29316cb44d4 100644 --- a/cql3/statements/create_aggregate_statement.cc +++ b/cql3/statements/create_aggregate_statement.cc @@ -79,7 +79,7 @@ seastar::future> create_aggregate_statement: } std::unique_ptr create_aggregate_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } future, std::vector, cql3::cql_warnings_vec>> @@ -121,5 +121,16 @@ create_aggregate_statement::create_aggregate_statement(functions::function_name , _ffunc(std::move(ffunc)) , _ival(std::move(ival)) {} + +audit::statement_category create_aggregate_statement::category() const { + return audit::statement_category::DDL; +} + +audit::audit_info_ptr +create_aggregate_statement::audit_info() const { + return audit::audit::create_audit_info(category(), sstring(), sstring()); } + +} + } diff --git a/cql3/statements/create_aggregate_statement.hh b/cql3/statements/create_aggregate_statement.hh index b595d94cb0f8..1c8e4255d15c 100644 --- a/cql3/statements/create_aggregate_statement.hh +++ b/cql3/statements/create_aggregate_statement.hh @@ -36,6 +36,9 @@ class create_aggregate_statement final : public create_function_statement_base { std::optional _ffunc; std::optional _ival; +protected: + virtual audit::audit_info_ptr audit_info() const override; + virtual audit::statement_category category() const override; public: create_aggregate_statement(functions::function_name name, std::vector> arg_types, sstring sfunc, shared_ptr stype, std::optional rfunc, std::optional ffunc, std::optional ival, bool or_replace, bool if_not_exists); diff --git a/cql3/statements/create_function_statement.cc b/cql3/statements/create_function_statement.cc index a723aaf992e9..ab54c0ee5247 100644 --- a/cql3/statements/create_function_statement.cc +++ b/cql3/statements/create_function_statement.cc @@ -44,8 +44,18 @@ seastar::future> create_function_statement::crea std::move(return_type), _called_on_null_input, std::move(*ctx)); } +audit::statement_category +create_function_statement::category() const { + return audit::statement_category::DDL; +} + +audit::audit_info_ptr +create_function_statement::audit_info() const { + return audit::audit::create_audit_info(category(), sstring(), sstring()); +} + std::unique_ptr create_function_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } future, std::vector, cql3::cql_warnings_vec>> diff --git a/cql3/statements/create_function_statement.hh b/cql3/statements/create_function_statement.hh index 3735ba1b8441..9a517751214a 100644 --- a/cql3/statements/create_function_statement.hh +++ b/cql3/statements/create_function_statement.hh @@ -9,6 +9,7 @@ #pragma once #include "cql3/statements/function_statement.hh" +#include "audit/audit.hh" #include "cql3/cql3_type.hh" namespace cql3 { @@ -32,6 +33,9 @@ class create_function_statement final : public create_function_statement_base { shared_ptr _return_type; bool _called_on_null_input; +protected: + virtual audit::statement_category category() const override; + virtual audit::audit_info_ptr audit_info() const override; public: create_function_statement(functions::function_name name, sstring language, sstring body, std::vector> arg_names, std::vector> arg_types, diff --git a/cql3/statements/create_index_statement.cc b/cql3/statements/create_index_statement.cc index 7718b2252d92..8167faaefc41 100644 --- a/cql3/statements/create_index_statement.cc +++ b/cql3/statements/create_index_statement.cc @@ -395,7 +395,7 @@ create_index_statement::prepare_schema_mutations(query_processor& qp, const quer std::unique_ptr create_index_statement::prepare(data_dictionary::database db, cql_stats& stats) { _cql_stats = &stats; - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } index_metadata create_index_statement::make_index_metadata(const std::vector<::shared_ptr>& targets, diff --git a/cql3/statements/create_keyspace_statement.cc b/cql3/statements/create_keyspace_statement.cc index 2960ba827438..e8fb3a719df2 100644 --- a/cql3/statements/create_keyspace_statement.cc +++ b/cql3/statements/create_keyspace_statement.cc @@ -135,7 +135,7 @@ future, std::vector std::unique_ptr cql3::statements::create_keyspace_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } future<> cql3::statements::create_keyspace_statement::grant_permissions_to_creator(const service::client_state& cs, service::group0_batch& mc) const { diff --git a/cql3/statements/create_role_statement.hh b/cql3/statements/create_role_statement.hh index e3713bd8f30a..e208891940e5 100644 --- a/cql3/statements/create_role_statement.hh +++ b/cql3/statements/create_role_statement.hh @@ -44,6 +44,8 @@ public: virtual future<::shared_ptr> execute(query_processor&, service::query_state&, const query_options&, std::optional guard) const override; + virtual void sanitize_audit_info() override; + private: future<> grant_permissions_to_creator(const service::client_state&, ::service::group0_batch&) const; }; diff --git a/cql3/statements/create_service_level_statement.cc b/cql3/statements/create_service_level_statement.cc index 5f67b0bcec5b..15080920519e 100644 --- a/cql3/statements/create_service_level_statement.cc +++ b/cql3/statements/create_service_level_statement.cc @@ -29,7 +29,7 @@ create_service_level_statement::create_service_level_statement(sstring service_l std::unique_ptr cql3::statements::create_service_level_statement::prepare( data_dictionary::database db, cql_stats &stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> create_service_level_statement::check_access(query_processor& qp, const service::client_state &state) const { diff --git a/cql3/statements/create_table_statement.cc b/cql3/statements/create_table_statement.cc index 42e64494ff59..4341ae1d4ecd 100644 --- a/cql3/statements/create_table_statement.cc +++ b/cql3/statements/create_table_statement.cc @@ -390,7 +390,7 @@ std::unique_ptr create_table_statement::raw_statement::prepa } } - return std::make_unique(stmt); + return std::make_unique(audit_info(), stmt); } data_type create_table_statement::raw_statement::get_type_and_remove(column_map_type& columns, ::shared_ptr t) @@ -515,6 +515,10 @@ ::shared_ptr create_table_statement::created column_family()); } +audit::statement_category create_table_statement::raw_statement::category() const { + return audit::statement_category::DDL; +} + } } diff --git a/cql3/statements/create_table_statement.hh b/cql3/statements/create_table_statement.hh index d80acf484afc..9e02f58a0108 100644 --- a/cql3/statements/create_table_statement.hh +++ b/cql3/statements/create_table_statement.hh @@ -121,6 +121,8 @@ public: void add_key_aliases(const std::vector<::shared_ptr> aliases); void add_column_alias(::shared_ptr alias); +protected: + virtual audit::statement_category category() const override; }; std::optional check_restricted_table_properties( diff --git a/cql3/statements/create_type_statement.cc b/cql3/statements/create_type_statement.cc index f6586d52f969..3c36786ea029 100644 --- a/cql3/statements/create_type_statement.cc +++ b/cql3/statements/create_type_statement.cc @@ -146,7 +146,7 @@ future, std::vector std::unique_ptr create_type_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } ::shared_ptr create_type_statement::created_event() const { diff --git a/cql3/statements/create_view_statement.cc b/cql3/statements/create_view_statement.cc index 592a6da434c6..f934c3034612 100644 --- a/cql3/statements/create_view_statement.cc +++ b/cql3/statements/create_view_statement.cc @@ -397,7 +397,7 @@ create_view_statement::prepare(data_dictionary::database db, cql_stats& stats) { if (!_prepare_ctx.get_variable_specifications().empty()) { throw exceptions::invalid_request_exception(format("Cannot use query parameters in CREATE MATERIALIZED VIEW statements")); } - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } ::shared_ptr create_view_statement::created_event() const { diff --git a/cql3/statements/delete_statement.cc b/cql3/statements/delete_statement.cc index 1735ebec7001..9ca4857177f4 100644 --- a/cql3/statements/delete_statement.cc +++ b/cql3/statements/delete_statement.cc @@ -21,9 +21,11 @@ namespace cql3 { namespace statements { -delete_statement::delete_statement(statement_type type, uint32_t bound_terms, schema_ptr s, std::unique_ptr attrs, cql_stats& stats) +delete_statement::delete_statement(audit::audit_info_ptr&& audit_info, statement_type type, uint32_t bound_terms, schema_ptr s, std::unique_ptr attrs, cql_stats& stats) : modification_statement{type, bound_terms, std::move(s), std::move(attrs), stats} -{ } +{ + set_audit_info(std::move(audit_info)); +} bool delete_statement::require_full_clustering_key() const { return false; @@ -59,7 +61,7 @@ namespace raw { ::shared_ptr delete_statement::prepare_internal(data_dictionary::database db, schema_ptr schema, prepare_context& ctx, std::unique_ptr attrs, cql_stats& stats) const { - auto stmt = ::make_shared(statement_type::DELETE, ctx.bound_variables_size(), schema, std::move(attrs), stats); + auto stmt = ::make_shared(audit_info(), statement_type::DELETE, ctx.bound_variables_size(), schema, std::move(attrs), stats); for (auto&& deletion : _deletions) { auto&& id = deletion->affected_column().prepare_column_identifier(*schema); diff --git a/cql3/statements/delete_statement.hh b/cql3/statements/delete_statement.hh index 22ec9975c04b..9c8bf9b297c7 100644 --- a/cql3/statements/delete_statement.hh +++ b/cql3/statements/delete_statement.hh @@ -24,7 +24,7 @@ namespace statements { */ class delete_statement : public modification_statement { public: - delete_statement(statement_type type, uint32_t bound_terms, schema_ptr s, std::unique_ptr attrs, cql_stats& stats); + delete_statement(audit::audit_info_ptr&& audit_info, statement_type type, uint32_t bound_terms, schema_ptr s, std::unique_ptr attrs, cql_stats& stats); virtual bool require_full_clustering_key() const override; diff --git a/cql3/statements/describe_statement.cc b/cql3/statements/describe_statement.cc index 9a257ee90561..4fd72d6ddf76 100644 --- a/cql3/statements/describe_statement.cc +++ b/cql3/statements/describe_statement.cc @@ -810,6 +810,16 @@ void describe_statement::with_internals_details(bool with_hashed_passwords) { } } +audit::statement_category +describe_statement::category() const { + return audit::statement_category::QUERY; +} + +audit::audit_info_ptr +describe_statement::audit_info() const { + return audit::audit::create_audit_info(category(), "system", ""); +} + std::unique_ptr describe_statement::prepare(data_dictionary::database db, cql_stats &stats) { bool internals = bool(_with_internals); auto desc_stmt = std::visit(overloaded_functor{ @@ -832,7 +842,7 @@ std::unique_ptr describe_statement::prepare(data_dictionary: return ::make_shared(std::move(cfg.keyspace), std::move(cfg.name), internals); } }, _config); - return std::make_unique(desc_stmt); + return std::make_unique(audit_info(), desc_stmt); } std::unique_ptr describe_statement::cluster() { diff --git a/cql3/statements/detach_service_level_statement.cc b/cql3/statements/detach_service_level_statement.cc index e1b8481abc90..31c5749e99ca 100644 --- a/cql3/statements/detach_service_level_statement.cc +++ b/cql3/statements/detach_service_level_statement.cc @@ -29,7 +29,7 @@ bool detach_service_level_statement::needs_guard(query_processor& qp, service::q std::unique_ptr cql3::statements::detach_service_level_statement::prepare( data_dictionary::database db, cql_stats &stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> detach_service_level_statement::check_access(query_processor& qp, const service::client_state &state) const { diff --git a/cql3/statements/drop_aggregate_statement.cc b/cql3/statements/drop_aggregate_statement.cc index e41eed42d575..75be5dc7e7ab 100644 --- a/cql3/statements/drop_aggregate_statement.cc +++ b/cql3/statements/drop_aggregate_statement.cc @@ -20,7 +20,7 @@ namespace cql3 { namespace statements { std::unique_ptr drop_aggregate_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } future, cql3::cql_warnings_vec>> drop_aggregate_statement::prepare_schema_mutations(query_processor& qp, service::query_state& state, const query_options& options, service::group0_batch& mc) const { @@ -48,5 +48,14 @@ drop_aggregate_statement::drop_aggregate_statement(functions::function_name name std::vector> arg_types, bool args_present, bool if_exists) : drop_function_statement_base(std::move(name), std::move(arg_types), args_present, if_exists) {} +audit::statement_category drop_aggregate_statement::category() const { + return audit::statement_category::DDL; +} + +audit::audit_info_ptr +drop_aggregate_statement::audit_info() const { + return audit::audit::create_audit_info(category(), sstring(), sstring()); +} + } } diff --git a/cql3/statements/drop_aggregate_statement.hh b/cql3/statements/drop_aggregate_statement.hh index 10a219a1a7cb..47313f4f5894 100644 --- a/cql3/statements/drop_aggregate_statement.hh +++ b/cql3/statements/drop_aggregate_statement.hh @@ -17,6 +17,9 @@ class drop_aggregate_statement final : public drop_function_statement_base { virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) override; future, cql3::cql_warnings_vec>> prepare_schema_mutations(query_processor& qp, service::query_state& state, const query_options& options, service::group0_batch& mc) const override; +protected: + virtual audit::audit_info_ptr audit_info() const override; + virtual audit::statement_category category() const override; public: drop_aggregate_statement(functions::function_name name, std::vector> arg_types, bool args_present, bool if_exists); diff --git a/cql3/statements/drop_function_statement.cc b/cql3/statements/drop_function_statement.cc index f541b618125b..23550d99d793 100644 --- a/cql3/statements/drop_function_statement.cc +++ b/cql3/statements/drop_function_statement.cc @@ -20,8 +20,18 @@ namespace cql3 { namespace statements { +audit::statement_category +drop_function_statement::category() const { + return audit::statement_category::DDL; +} + +audit::audit_info_ptr +drop_function_statement::audit_info() const { + return audit::audit::create_audit_info(category(), sstring(), sstring()); +} + std::unique_ptr drop_function_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } future, cql3::cql_warnings_vec>> drop_function_statement::prepare_schema_mutations(query_processor& qp, service::query_state& state, const query_options& options, service::group0_batch& mc) const { diff --git a/cql3/statements/drop_function_statement.hh b/cql3/statements/drop_function_statement.hh index 0853e7718ba1..1c8105c927b9 100644 --- a/cql3/statements/drop_function_statement.hh +++ b/cql3/statements/drop_function_statement.hh @@ -9,6 +9,7 @@ #pragma once #include "cql3/statements/function_statement.hh" +#include "audit/audit.hh" namespace cql3 { class query_processor; @@ -16,6 +17,9 @@ namespace statements { class drop_function_statement final : public drop_function_statement_base { virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) override; future, cql3::cql_warnings_vec>> prepare_schema_mutations(query_processor& qp, service::query_state& state, const query_options& options, service::group0_batch& mc) const override; +protected: + virtual audit::statement_category category() const override; + virtual audit::audit_info_ptr audit_info() const override; public: drop_function_statement(functions::function_name name, std::vector> arg_types, diff --git a/cql3/statements/drop_index_statement.cc b/cql3/statements/drop_index_statement.cc index a6cd3ec2515a..f265598f5981 100644 --- a/cql3/statements/drop_index_statement.cc +++ b/cql3/statements/drop_index_statement.cc @@ -93,7 +93,7 @@ drop_index_statement::prepare_schema_mutations(query_processor& qp, const query_ std::unique_ptr drop_index_statement::prepare(data_dictionary::database db, cql_stats& stats) { _cql_stats = &stats; - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } schema_ptr drop_index_statement::lookup_indexed_table(query_processor& qp) const diff --git a/cql3/statements/drop_keyspace_statement.cc b/cql3/statements/drop_keyspace_statement.cc index 8e336efe8c8e..9883466a5c0a 100644 --- a/cql3/statements/drop_keyspace_statement.cc +++ b/cql3/statements/drop_keyspace_statement.cc @@ -74,7 +74,7 @@ future, cql3::cql_w std::unique_ptr drop_keyspace_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } } diff --git a/cql3/statements/drop_service_level_statement.cc b/cql3/statements/drop_service_level_statement.cc index 2f5085662d00..7734f13661eb 100644 --- a/cql3/statements/drop_service_level_statement.cc +++ b/cql3/statements/drop_service_level_statement.cc @@ -22,7 +22,7 @@ drop_service_level_statement::drop_service_level_statement(sstring service_level std::unique_ptr cql3::statements::drop_service_level_statement::prepare( data_dictionary::database db, cql_stats &stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> drop_service_level_statement::check_access(query_processor& qp, const service::client_state &state) const { diff --git a/cql3/statements/drop_table_statement.cc b/cql3/statements/drop_table_statement.cc index 5333629507e0..59a320473914 100644 --- a/cql3/statements/drop_table_statement.cc +++ b/cql3/statements/drop_table_statement.cc @@ -68,7 +68,7 @@ future, cql3::cql_w std::unique_ptr drop_table_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } } diff --git a/cql3/statements/drop_type_statement.cc b/cql3/statements/drop_type_statement.cc index fb2a94d99d58..78157084d8bf 100644 --- a/cql3/statements/drop_type_statement.cc +++ b/cql3/statements/drop_type_statement.cc @@ -154,7 +154,7 @@ drop_type_statement::prepare_schema_mutations(query_processor& qp, const query_o std::unique_ptr drop_type_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } } diff --git a/cql3/statements/drop_view_statement.cc b/cql3/statements/drop_view_statement.cc index a1ca2669dc80..df144152e5b1 100644 --- a/cql3/statements/drop_view_statement.cc +++ b/cql3/statements/drop_view_statement.cc @@ -66,7 +66,7 @@ drop_view_statement::prepare_schema_mutations(query_processor& qp, const query_o std::unique_ptr drop_view_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(make_shared(*this)); + return std::make_unique(audit_info(), make_shared(*this)); } } diff --git a/cql3/statements/grant_statement.cc b/cql3/statements/grant_statement.cc index 7e681fd5469b..f503b85af423 100644 --- a/cql3/statements/grant_statement.cc +++ b/cql3/statements/grant_statement.cc @@ -16,7 +16,7 @@ std::unique_ptr cql3::statements::grant_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<::shared_ptr> diff --git a/cql3/statements/list_effective_service_level_statement.cc b/cql3/statements/list_effective_service_level_statement.cc index 9059662d3d51..d8a8e0660673 100644 --- a/cql3/statements/list_effective_service_level_statement.cc +++ b/cql3/statements/list_effective_service_level_statement.cc @@ -27,7 +27,7 @@ list_effective_service_level_statement::list_effective_service_level_statement(s std::unique_ptr list_effective_service_level_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } static auto make_column(sstring name, const shared_ptr type) { diff --git a/cql3/statements/list_permissions_statement.cc b/cql3/statements/list_permissions_statement.cc index f34bcb808d04..e0536b165051 100644 --- a/cql3/statements/list_permissions_statement.cc +++ b/cql3/statements/list_permissions_statement.cc @@ -30,7 +30,7 @@ cql3::statements::list_permissions_statement::list_permissions_statement( std::unique_ptr cql3::statements::list_permissions_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } void cql3::statements::list_permissions_statement::validate( diff --git a/cql3/statements/list_service_level_attachments_statement.cc b/cql3/statements/list_service_level_attachments_statement.cc index 2cd16a5b2659..3356dcb3973f 100644 --- a/cql3/statements/list_service_level_attachments_statement.cc +++ b/cql3/statements/list_service_level_attachments_statement.cc @@ -28,7 +28,7 @@ list_service_level_attachments_statement::list_service_level_attachments_stateme std::unique_ptr cql3::statements::list_service_level_attachments_statement::prepare( data_dictionary::database db, cql_stats &stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> list_service_level_attachments_statement::check_access(query_processor& qp, const service::client_state &state) const { diff --git a/cql3/statements/list_service_level_statement.cc b/cql3/statements/list_service_level_statement.cc index 2a7fd9428895..6ffbcff0a150 100644 --- a/cql3/statements/list_service_level_statement.cc +++ b/cql3/statements/list_service_level_statement.cc @@ -26,7 +26,7 @@ list_service_level_statement::list_service_level_statement(sstring service_level std::unique_ptr cql3::statements::list_service_level_statement::prepare( data_dictionary::database db, cql_stats &stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> list_service_level_statement::check_access(query_processor& qp, const service::client_state &state) const { diff --git a/cql3/statements/list_users_statement.cc b/cql3/statements/list_users_statement.cc index 0f833f55771b..0aea22b0e753 100644 --- a/cql3/statements/list_users_statement.cc +++ b/cql3/statements/list_users_statement.cc @@ -18,7 +18,7 @@ std::unique_ptr cql3::statements::list_users_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> cql3::statements::list_users_statement::check_access(query_processor& qp, const service::client_state& state) const { diff --git a/cql3/statements/modification_statement.cc b/cql3/statements/modification_statement.cc index 434598e68842..a49c9a68c70f 100644 --- a/cql3/statements/modification_statement.cc +++ b/cql3/statements/modification_statement.cc @@ -564,7 +564,7 @@ modification_statement::prepare(data_dictionary::database db, cql_stats& stats) auto meta = get_prepare_context(); auto statement = prepare_statement(db, meta, stats); auto partition_key_bind_indices = meta.get_partition_key_bind_indexes(*schema); - return std::make_unique(std::move(statement), meta, std::move(partition_key_bind_indices)); + return std::make_unique(audit_info(), std::move(statement), meta, std::move(partition_key_bind_indices)); } ::shared_ptr @@ -681,6 +681,10 @@ modification_statement::prepare_conditions(data_dictionary::database db, const s } } +audit::statement_category modification_statement::category() const { + return audit::statement_category::DML; +} + } // namespace raw void diff --git a/cql3/statements/prepared_statement.hh b/cql3/statements/prepared_statement.hh index 741d19135bfc..cd9860fa948a 100644 --- a/cql3/statements/prepared_statement.hh +++ b/cql3/statements/prepared_statement.hh @@ -10,6 +10,8 @@ #pragma once +#include "audit/audit.hh" + #include #include #include @@ -41,15 +43,15 @@ public: const std::vector partition_key_bind_indices; std::vector warnings; - prepared_statement(seastar::shared_ptr statement_, std::vector> bound_names_, + prepared_statement(audit::audit_info_ptr&& audit_info, seastar::shared_ptr statement_, std::vector> bound_names_, std::vector partition_key_bind_indices, std::vector warnings = {}); - prepared_statement(seastar::shared_ptr statement_, const prepare_context& ctx, const std::vector& partition_key_bind_indices, + prepared_statement(audit::audit_info_ptr&& audit_info, seastar::shared_ptr statement_, const prepare_context& ctx, const std::vector& partition_key_bind_indices, std::vector warnings = {}); - prepared_statement(seastar::shared_ptr statement_, prepare_context&& ctx, std::vector&& partition_key_bind_indices); + prepared_statement(audit::audit_info_ptr&& audit_info, seastar::shared_ptr statement_, prepare_context&& ctx, std::vector&& partition_key_bind_indices); - prepared_statement(seastar::shared_ptr&& statement_); + prepared_statement(audit::audit_info_ptr&& audit_info, seastar::shared_ptr&& statement_); checked_weak_ptr checked_weak_from_this() const { return checked_weak_ptr(this->weak_from_this()); diff --git a/cql3/statements/raw/batch_statement.hh b/cql3/statements/raw/batch_statement.hh index 4371b6dd37a0..cf2f0b83a01f 100644 --- a/cql3/statements/raw/batch_statement.hh +++ b/cql3/statements/raw/batch_statement.hh @@ -47,6 +47,12 @@ public: } virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) override; +protected: + virtual audit::statement_category category() const override; + virtual audit::audit_info_ptr audit_info() const override { + // We don't audit batch statements. Instead we audit statements that are inside the batch. + return audit::audit::create_no_audit_info(); + } }; } diff --git a/cql3/statements/raw/cf_statement.hh b/cql3/statements/raw/cf_statement.hh index e3a248c0862d..8a8fccd1bcad 100644 --- a/cql3/statements/raw/cf_statement.hh +++ b/cql3/statements/raw/cf_statement.hh @@ -43,6 +43,10 @@ public: virtual const sstring& keyspace() const; virtual const sstring& column_family() const; + + virtual audit::audit_info_ptr audit_info() const override { + return audit::audit::create_audit_info(category(), keyspace(), column_family()); + } }; } diff --git a/cql3/statements/raw/describe_statement.hh b/cql3/statements/raw/describe_statement.hh index 62c83dc89567..416b287a0b39 100644 --- a/cql3/statements/raw/describe_statement.hh +++ b/cql3/statements/raw/describe_statement.hh @@ -75,6 +75,8 @@ private: describe_config _config; internals _with_internals = internals(false); + virtual audit::audit_info_ptr audit_info() const override; + virtual audit::statement_category category() const override; public: explicit describe_statement(describe_config config); void with_internals_details(bool with_hashed_passwords); diff --git a/cql3/statements/raw/modification_statement.hh b/cql3/statements/raw/modification_statement.hh index 9719b40d6bf4..9fbcf0e9cc1e 100644 --- a/cql3/statements/raw/modification_statement.hh +++ b/cql3/statements/raw/modification_statement.hh @@ -53,6 +53,8 @@ protected: // we need to know what kinds of conditions (static, regular) the statement has. void prepare_conditions(data_dictionary::database db, const schema& schema, prepare_context& ctx, cql3::statements::modification_statement& stmt) const; + + virtual audit::statement_category category() const override; }; } diff --git a/cql3/statements/raw/parsed_statement.cc b/cql3/statements/raw/parsed_statement.cc index 1e0151ef4ade..4d01291efd9a 100644 --- a/cql3/statements/raw/parsed_statement.cc +++ b/cql3/statements/raw/parsed_statement.cc @@ -13,6 +13,8 @@ #include "cql3/statements/prepared_statement.hh" #include "cql3/column_specification.hh" +#include "cql3/cql_statement.hh" + namespace cql3 { namespace statements { @@ -38,26 +40,30 @@ void parsed_statement::set_bound_variables(const std::vector<::shared_ptr statement_, std::vector> bound_names_, std::vector partition_key_bind_indices, std::vector warnings) : statement(std::move(statement_)) , bound_names(std::move(bound_names_)) , partition_key_bind_indices(std::move(partition_key_bind_indices)) , warnings(std::move(warnings)) -{ } +{ + statement->set_audit_info(std::move(audit_info)); +} prepared_statement::prepared_statement( + audit::audit_info_ptr&& audit_info, ::shared_ptr statement_, const prepare_context& ctx, const std::vector& partition_key_bind_indices, std::vector warnings) - : prepared_statement(statement_, ctx.get_variable_specifications(), partition_key_bind_indices, std::move(warnings)) + : prepared_statement(std::move(audit_info), statement_, ctx.get_variable_specifications(), partition_key_bind_indices, std::move(warnings)) { } -prepared_statement::prepared_statement(::shared_ptr statement_, prepare_context&& ctx, std::vector&& partition_key_bind_indices) - : prepared_statement(statement_, std::move(ctx).get_variable_specifications(), std::move(partition_key_bind_indices)) +prepared_statement::prepared_statement(audit::audit_info_ptr&& audit_info, ::shared_ptr statement_, prepare_context&& ctx, std::vector&& partition_key_bind_indices) + : prepared_statement(std::move(audit_info), statement_, std::move(ctx).get_variable_specifications(), std::move(partition_key_bind_indices)) { } -prepared_statement::prepared_statement(::shared_ptr&& statement_) - : prepared_statement(statement_, std::vector>(), std::vector()) +prepared_statement::prepared_statement(audit::audit_info_ptr&& audit_info, ::shared_ptr&& statement_) + : prepared_statement(std::move(audit_info), statement_, std::vector>(), std::vector()) { } } diff --git a/cql3/statements/raw/parsed_statement.hh b/cql3/statements/raw/parsed_statement.hh index d4055553b580..f877708e563c 100644 --- a/cql3/statements/raw/parsed_statement.hh +++ b/cql3/statements/raw/parsed_statement.hh @@ -17,6 +17,7 @@ #include #include +#include "audit/audit.hh" namespace cql3 { @@ -42,6 +43,10 @@ public: void set_bound_variables(const std::vector<::shared_ptr>& bound_names); virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) = 0; + +protected: + virtual audit::statement_category category() const = 0; + virtual audit::audit_info_ptr audit_info() const = 0; }; } diff --git a/cql3/statements/raw/select_statement.hh b/cql3/statements/raw/select_statement.hh index 43686a01ad50..b3dff7ac3f85 100644 --- a/cql3/statements/raw/select_statement.hh +++ b/cql3/statements/raw/select_statement.hh @@ -76,6 +76,8 @@ public: using result_row_type = std::vector; using ordering_comparator_type = compare_fn; +protected: + virtual audit::statement_category category() const override; private: using prepared_orderings_type = std::vector>; private: diff --git a/cql3/statements/raw/truncate_statement.hh b/cql3/statements/raw/truncate_statement.hh index 99474c8e28d4..60a1ec36f1ea 100644 --- a/cql3/statements/raw/truncate_statement.hh +++ b/cql3/statements/raw/truncate_statement.hh @@ -32,6 +32,8 @@ public: truncate_statement(cf_name name, std::unique_ptr attrs); virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) override; + + virtual audit::statement_category category() const override; }; } // namespace raw diff --git a/cql3/statements/raw/use_statement.hh b/cql3/statements/raw/use_statement.hh index db7d765fa210..ee5472b06c3f 100644 --- a/cql3/statements/raw/use_statement.hh +++ b/cql3/statements/raw/use_statement.hh @@ -30,6 +30,11 @@ public: use_statement(sstring keyspace); virtual std::unique_ptr prepare(data_dictionary::database db, cql_stats& stats) override; +protected: + virtual audit::statement_category category() const override; + virtual audit::audit_info_ptr audit_info() const override { + return audit::audit::create_audit_info(category(), _keyspace, sstring()); + } }; } diff --git a/cql3/statements/revoke_statement.cc b/cql3/statements/revoke_statement.cc index 974b870c2dbd..82cd247c6320 100644 --- a/cql3/statements/revoke_statement.cc +++ b/cql3/statements/revoke_statement.cc @@ -15,7 +15,7 @@ std::unique_ptr cql3::statements::revoke_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<::shared_ptr> diff --git a/cql3/statements/role-management-statements.cc b/cql3/statements/role-management-statements.cc index e60c315853a5..08a3f3ea58f5 100644 --- a/cql3/statements/role-management-statements.cc +++ b/cql3/statements/role-management-statements.cc @@ -9,6 +9,7 @@ */ #include +#include #include @@ -31,6 +32,8 @@ #include "transport/messages/result_message.hh" #include "service/raft/raft_group0_client.hh" +#include + namespace cql3 { namespace statements { @@ -65,7 +68,7 @@ static auth::authentication_options extract_authentication_options(const cql3::r std::unique_ptr create_role_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> create_role_statement::grant_permissions_to_creator(const service::client_state& cs, ::service::group0_batch& mc) const { @@ -131,13 +134,76 @@ create_role_statement::execute(query_processor&, co_return nullptr; } +/// Prepends '\' to special characters in ECMAScript regex syntax. +static sstring escape_for_regex(const sstring& text) { + static const std::regex escape_re(R"([.^$|()\[\]{}*+?\\])", std::regex_constants::ECMAScript); + return std::regex_replace(text.c_str(), escape_re, R"(\$&)"); +} + +static bool is_password_empty(const sstring& password) { + // Empty passwords are stored as single char(-1) ( ), NOT as empty strings. + // + // see cql3/cql.g, and search "ugly hack". we use an ugly hack to return + // empty string literals using string with a single char(-1) + // + // please note, we can NOT compare password[0] with -1, as on aarch64 + // platforms, `char` is unsigned by default. so, if password is "empty", + // the value of password[0] would be 255. + return password.size() == 1 && password[0] == static_cast(-1); +} + +/// Removes single-quoted plaintext passwords. +static void sanitize_audit_info_password(audit::audit_info* ai, const sstring& raw_password) { + sstring password; + if (!is_password_empty(raw_password)) { + // Password needs escaping to be searchable literally: + password = escape_for_regex(raw_password); + } + const std::regex re(R"(((WITH|AND)\s*PASSWORD\s*=?\s*)')" + std::string(password) + "'", + std::regex_constants::ECMAScript | std::regex_constants::icase); + const auto replace_result = std::regex_replace(ai->query().c_str(), re, "$1'***'"); + ai->set_query_string(static_cast(replace_result)); +} + +/// A heuristic to mask the values of `OPTIONS' map when key is like "PASSWORD". +static void sanitize_audit_info_options(audit::audit_info* ai, const std::map& options) { + for (const auto& [k, v] : options) { + const auto key_trimmed = boost::trim_copy(k); + if (!boost::iequals(key_trimmed, "PASSWORD")) { // Case-insensitive comp. + continue; + } + sstring password; + if (!is_password_empty(v)) { + // Password needs escaping to be searchable literally: + password = escape_for_regex(v); + } + // Now find the pattern of map element, like: "' PassWord ' : '1234vcxz!@#$'", + // capture "' PassWord ' : " and match "'1234vcxz!@#$'" + const std::regex re(R"(('\s*)" + std::string(key_trimmed) + R"(\s*'\s*:\s*)')" + + std::string(password) + "'", std::regex_constants::ECMAScript); + const auto replace_result = std::regex_replace(ai->query().c_str(), re, "$1'***'"); + ai->set_query_string(static_cast(replace_result)); + } +} + +void create_role_statement::sanitize_audit_info() { + if (audit::audit_info* ai = get_audit_info(); ai != nullptr) { + if (_options.password) { + sanitize_audit_info_password(ai, *_options.password); + } + if (_options.options) { + sanitize_audit_info_options(ai, *_options.options); + } + } +} + // // `alter_role_statement` // std::unique_ptr alter_role_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> alter_role_statement::check_access(query_processor& qp, const service::client_state& state) const { @@ -206,13 +272,24 @@ alter_role_statement::execute(query_processor&, service::query_state& state, con co_return nullptr; } +void alter_role_statement::sanitize_audit_info() { + if (audit::audit_info* ai = get_audit_info(); ai != nullptr) { + if (_options.password) { + sanitize_audit_info_password(ai, *_options.password); + } + if (_options.options) { + sanitize_audit_info_options(ai, *_options.options); + } + } +} + // // `drop_role_statement` // std::unique_ptr drop_role_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } void drop_role_statement::validate(query_processor& qp, const service::client_state& state) const { @@ -267,7 +344,7 @@ drop_role_statement::execute(query_processor&, service::query_state& state, cons std::unique_ptr list_roles_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> list_roles_statement::check_access(query_processor& qp, const service::client_state& state) const { @@ -402,7 +479,7 @@ list_roles_statement::execute(query_processor& qp, service::query_state& state, std::unique_ptr grant_role_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> grant_role_statement::check_access(query_processor& qp, const service::client_state& state) const { @@ -432,7 +509,7 @@ grant_role_statement::execute(query_processor&, service::query_state& state, con std::unique_ptr revoke_role_statement::prepare( data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(*this)); + return std::make_unique(audit_info(), ::make_shared(*this)); } future<> revoke_role_statement::check_access(query_processor& qp, const service::client_state& state) const { diff --git a/cql3/statements/schema_altering_statement.cc b/cql3/statements/schema_altering_statement.cc index 0e4c3ad6b2d8..0e274ad6af6d 100644 --- a/cql3/statements/schema_altering_statement.cc +++ b/cql3/statements/schema_altering_statement.cc @@ -92,6 +92,10 @@ future, cql3::cql_wa co_return std::make_tuple(ret, cql_warnings); } +audit::statement_category schema_altering_statement::category() const { + return audit::statement_category::DDL; +} + } } diff --git a/cql3/statements/schema_altering_statement.hh b/cql3/statements/schema_altering_statement.hh index 1af3531c3db1..3d3f2d8ddc1c 100644 --- a/cql3/statements/schema_altering_statement.hh +++ b/cql3/statements/schema_altering_statement.hh @@ -53,6 +53,8 @@ protected: virtual future<::shared_ptr> execute(query_processor& qp, service::query_state& state, const query_options& options, std::optional guard) const override; + virtual audit::statement_category category() const override; + public: /** * When a new data_dictionary::database object (keyspace, table) is created, the creator needs to be granted all applicable diff --git a/cql3/statements/select_statement.cc b/cql3/statements/select_statement.cc index 25d40eaf53f4..d3505c6d7b9f 100644 --- a/cql3/statements/select_statement.cc +++ b/cql3/statements/select_statement.cc @@ -1881,6 +1881,10 @@ static void validate_attrs(const cql3::attributes::raw& attrs) { SCYLLA_ASSERT(!attrs.time_to_live.has_value()); } +audit::statement_category select_statement::category() const { + return audit::statement_category::QUERY; +} + select_statement::select_statement(cf_name cf_name, lw_shared_ptr parameters, std::vector<::shared_ptr> select_clause, @@ -2167,8 +2171,9 @@ std::unique_ptr select_statement::prepare(data_dictionary::d } auto partition_key_bind_indices = ctx.get_partition_key_bind_indexes(*schema); + stmt->_may_use_token_aware_routing = partition_key_bind_indices.size() != 0; - return make_unique(std::move(stmt), ctx, std::move(partition_key_bind_indices), std::move(warnings)); + return make_unique(audit_info(), std::move(stmt), ctx, std::move(partition_key_bind_indices), std::move(warnings)); } ::shared_ptr diff --git a/cql3/statements/service_level_statement.cc b/cql3/statements/service_level_statement.cc index a91acbeb76d9..8ea87502bf31 100644 --- a/cql3/statements/service_level_statement.cc +++ b/cql3/statements/service_level_statement.cc @@ -30,10 +30,12 @@ bool service_level_statement::needs_guard(query_processor&, service::query_state return state.get_service_level_controller().is_v2(); } -void service_level_statement::validate_shares_option(const query_processor& qp, const qos::service_level_options& slo) const { - if (!std::holds_alternative(slo.shares) && !qp.proxy().features().workload_prioritization) { - throw exceptions::invalid_request_exception("`shares` option can only be used when the cluster is fully upgraded to enterprise"); - } +audit::statement_category service_level_statement::category() const { + return audit::statement_category::ADMIN; +} + +audit::audit_info_ptr service_level_statement::audit_info() const { + return audit::audit::create_audit_info(category(), sstring(), sstring()); } } diff --git a/cql3/statements/service_level_statement.hh b/cql3/statements/service_level_statement.hh index 7aad668114c0..35e769c9299c 100644 --- a/cql3/statements/service_level_statement.hh +++ b/cql3/statements/service_level_statement.hh @@ -51,7 +51,9 @@ public: future<> check_access(query_processor& qp, const service::client_state& state) const override; protected: - void validate_shares_option(const query_processor& qp, const qos::service_level_options& slo) const; + virtual audit::statement_category category() const override; + + virtual audit::audit_info_ptr audit_info() const override; }; } diff --git a/cql3/statements/truncate_statement.cc b/cql3/statements/truncate_statement.cc index 9c24f2339afd..200617ac1e18 100644 --- a/cql3/statements/truncate_statement.cc +++ b/cql3/statements/truncate_statement.cc @@ -41,7 +41,7 @@ std::unique_ptr truncate_statement::prepare(data_dictionary: auto ctx = get_prepare_context(); prepared_attributes->fill_prepare_context(ctx); auto stmt = ::make_shared(std::move(schema), std::move(prepared_attributes)); - return std::make_unique(std::move(stmt)); + return std::make_unique(audit_info(), std::move(stmt)); } } // namespace raw @@ -101,6 +101,10 @@ truncate_statement::execute(query_processor& qp, service::query_state& state, co }); } +audit::statement_category raw::truncate_statement::category() const { + return audit::statement_category::DML; +} + db::timeout_clock::duration truncate_statement::get_timeout(const service::client_state& state, const query_options& options) const { return _attrs->is_timeout_set() ? _attrs->get_timeout(options) : state.get_timeout_config().truncate_timeout; } diff --git a/cql3/statements/update_statement.cc b/cql3/statements/update_statement.cc index b0556f45cf1f..622046b6cb53 100644 --- a/cql3/statements/update_statement.cc +++ b/cql3/statements/update_statement.cc @@ -86,13 +86,16 @@ parse(const sstring& json_string, const std::vector& expected namespace statements { update_statement::update_statement( + audit::audit_info_ptr&& audit_info, statement_type type, uint32_t bound_terms, schema_ptr s, std::unique_ptr attrs, cql_stats& stats) : modification_statement{type, bound_terms, std::move(s), std::move(attrs), stats} -{ } +{ + set_audit_info(std::move(audit_info)); +} bool update_statement::require_full_clustering_key() const { return true; @@ -379,7 +382,7 @@ ::shared_ptr insert_statement::prepare_internal(data_dictionary::database db, schema_ptr schema, prepare_context& ctx, std::unique_ptr attrs, cql_stats& stats) const { - auto stmt = ::make_shared(statement_type::INSERT, ctx.bound_variables_size(), schema, std::move(attrs), stats); + auto stmt = ::make_shared(audit_info(), statement_type::INSERT, ctx.bound_variables_size(), schema, std::move(attrs), stats); // Created from an INSERT if (stmt->is_counter()) { @@ -445,7 +448,7 @@ insert_json_statement::prepare_internal(data_dictionary::database db, schema_ptr auto prepared_json_value = prepare_expression(_json_value, db, "", nullptr, make_lw_shared("", "", json_column_placeholder, utf8_type)); expr::verify_no_aggregate_functions(prepared_json_value, "JSON clause"); expr::fill_prepare_context(prepared_json_value, ctx); - auto stmt = ::make_shared(ctx.bound_variables_size(), schema, std::move(attrs), stats, std::move(prepared_json_value), _default_unset); + auto stmt = ::make_shared(audit_info(), ctx.bound_variables_size(), schema, std::move(attrs), stats, std::move(prepared_json_value), _default_unset); prepare_conditions(db, *schema, ctx, *stmt); return stmt; } @@ -464,7 +467,7 @@ ::shared_ptr update_statement::prepare_internal(data_dictionary::database db, schema_ptr schema, prepare_context& ctx, std::unique_ptr attrs, cql_stats& stats) const { - auto stmt = ::make_shared(statement_type::UPDATE, ctx.bound_variables_size(), schema, std::move(attrs), stats); + auto stmt = ::make_shared(audit_info(), statement_type::UPDATE, ctx.bound_variables_size(), schema, std::move(attrs), stats); // FIXME: quadratic for (size_t i = 0; i < _updates.size(); ++i) { diff --git a/cql3/statements/update_statement.hh b/cql3/statements/update_statement.hh index 101ce1526ba9..c991b066e24d 100644 --- a/cql3/statements/update_statement.hh +++ b/cql3/statements/update_statement.hh @@ -29,6 +29,7 @@ public: #endif update_statement( + audit::audit_info_ptr&& audit_info, statement_type type, uint32_t bound_terms, schema_ptr s, @@ -56,12 +57,13 @@ class insert_prepared_json_statement : public update_statement { bool _default_unset; public: insert_prepared_json_statement( + audit::audit_info_ptr&& audit_info, uint32_t bound_terms, schema_ptr s, std::unique_ptr attrs, cql_stats& stats, expr::expression v, bool default_unset) - : update_statement(statement_type::INSERT, bound_terms, s, std::move(attrs), stats) + : update_statement(std::move(audit_info), statement_type::INSERT, bound_terms, s, std::move(attrs), stats) , _value(std::move(v)) , _default_unset(default_unset) { _restrictions = restrictions::statement_restrictions(s, false); diff --git a/cql3/statements/use_statement.cc b/cql3/statements/use_statement.cc index 1e58380f9839..cccb998b7af0 100644 --- a/cql3/statements/use_statement.cc +++ b/cql3/statements/use_statement.cc @@ -38,7 +38,12 @@ use_statement::use_statement(sstring keyspace) std::unique_ptr use_statement::prepare(data_dictionary::database db, cql_stats& stats) { - return std::make_unique(::make_shared(_keyspace)); + return std::make_unique(audit_info(), ::make_shared(_keyspace)); +} + +audit::statement_category use_statement::category() const { + // It's not obvious why USE is a DML but that's how Origin classifies it. + return audit::statement_category::DML; } } From 28bd699c51a47cb3f566fb8a7d9e5c1ebd33402b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Zakrzewski?= Date: Fri, 20 Dec 2024 03:17:18 +0100 Subject: [PATCH 301/397] audit: Add service level support to CQL login process This change integrates service level functionality into the CQL authentication and connection handling: - Add scheduling_group_name to client_data to track service level assignments - Extend SASL challenge interface to expose authenticated username - Modify connection processing to support tenant switching: - Add switch_tenant() method to handle scheduling group changes - Add process_until_tenant_switch() to handle request processing boundaries - Implement no_tenant() default executor - Add execute_under_tenant_type for scheduling group management - Update connection lifecycle to properly handle service level changes: - Initialize connections with default scheduling group - Support dynamic scheduling group updates when service levels change - Ensure proper cleanup of scheduling group assignments The changes enable proper scheduling group assignment and management based on authenticated users' service levels, while maintaining backward compatibility for connections without service level assignments. --- auth/sasl_challenge.cc | 7 +++++++ auth/sasl_challenge.hh | 4 ++++ auth/transitional.cc | 4 ++++ service/qos/qos_common.cc | 2 +- service/qos/qos_common.hh | 2 +- service/qos/service_level_controller.hh | 1 + table_helper.cc | 2 +- test/cqlpy/test_describe.py | 3 +++ test/lib/cql_test_env.cc | 4 ++++ test/lib/cql_test_env.hh | 3 +++ transport/server.cc | 26 ++++++++++++++----------- 11 files changed, 44 insertions(+), 14 deletions(-) diff --git a/auth/sasl_challenge.cc b/auth/sasl_challenge.cc index 27796d35e6c0..5f6d1889c3bf 100644 --- a/auth/sasl_challenge.cc +++ b/auth/sasl_challenge.cc @@ -68,4 +68,11 @@ future plain_sasl_challenge::get_authenticated_user() const return _when_complete(*_username, *_password); } +const sstring& plain_sasl_challenge::get_username() const { + if (!_username) { + throw std::logic_error("plain_sasl_challenge::get_username() called without username"); + } + return *_username; +} + } diff --git a/auth/sasl_challenge.hh b/auth/sasl_challenge.hh index ee2d943f0b75..22507992caeb 100644 --- a/auth/sasl_challenge.hh +++ b/auth/sasl_challenge.hh @@ -35,6 +35,8 @@ public: virtual bool is_complete() const = 0; virtual future get_authenticated_user() const = 0; + + virtual const sstring& get_username() const = 0; }; class plain_sasl_challenge : public sasl_challenge { @@ -50,6 +52,8 @@ public: virtual future get_authenticated_user() const override; + virtual const sstring& get_username() const override; + private: std::optional _username, _password; completion_callback _when_complete; diff --git a/auth/transitional.cc b/auth/transitional.cc index 3f7b895bff46..8e55a7467dd4 100644 --- a/auth/transitional.cc +++ b/auth/transitional.cc @@ -146,6 +146,10 @@ class transitional_authenticator : public authenticator { } }); }); + } + + const sstring& get_username() const override { + return _sasl->get_username(); } private: diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index e95733aa285c..a93dff2e3f4a 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -207,7 +207,7 @@ static service_level_options::timeout_type get_duration(const cql3::untyped_resu static qos::service_level_options::shares_type get_shares(const cql3::untyped_result_set_row& row, std::string_view col_name) { auto shares_opt = row.get_opt(col_name); if (!shares_opt) { - return qos::service_level_controller::default_shares; + return qos::service_level_options::unset_marker{}; } return *shares_opt; } diff --git a/service/qos/qos_common.hh b/service/qos/qos_common.hh index 0991bd217094..e3ba8b150705 100644 --- a/service/qos/qos_common.hh +++ b/service/qos/qos_common.hh @@ -11,7 +11,7 @@ #include "db/consistency_level_type.hh" #include "seastarx.hh" #include -#include +#include #include #include #include diff --git a/service/qos/service_level_controller.hh b/service/qos/service_level_controller.hh index 5ddf4734606e..55674e1c76a1 100644 --- a/service/qos/service_level_controller.hh +++ b/service/qos/service_level_controller.hh @@ -21,6 +21,7 @@ #include "auth/service.hh" #include "cql3/description.hh" #include +#include #include "qos_common.hh" #include "service/endpoint_lifecycle_subscriber.hh" #include "qos_configuration_change_subscriber.hh" diff --git a/table_helper.cc b/table_helper.cc index 7a7bd3a3eba4..1eb6f9f4225b 100644 --- a/table_helper.cc +++ b/table_helper.cc @@ -175,7 +175,7 @@ future<> table_helper::setup_keyspace(cql3::query_processor& qp, service::migrat else { opts["replication_factor"] = replication_factor; } - auto ksm = keyspace_metadata::new_keyspace(keyspace_name, replication_strategy_name, std::move(opts), true); + auto ksm = keyspace_metadata::new_keyspace(keyspace_name, replication_strategy_name, std::move(opts), std::nullopt, true); try { co_await mm.announce(service::prepare_new_keyspace_announcement(db.real_database(), ksm, ts), std::move(group0_guard), seastar::format("table_helper: create {} keyspace", keyspace_name)); diff --git a/test/cqlpy/test_describe.py b/test/cqlpy/test_describe.py index f79d50d3e5ea..ae59f403634f 100644 --- a/test/cqlpy/test_describe.py +++ b/test/cqlpy/test_describe.py @@ -2468,6 +2468,9 @@ def test_desc_auth_service_levels(cql): # Enterprise is limited in the number of service levels it supports sl_create_stmts = set(random.sample(list(sl_create_stmts), k=5)) + # Enterprise is limited in the number of service levels it supports + sl_create_stmts = set(random.sample(list(sl_create_stmts), k=5)) + for stmt in sl_create_stmts: cql.execute(stmt) diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index eae4ffc4556b..5ee4cc21a4c8 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -407,6 +407,10 @@ class single_node_cql_env : public cql_test_env { return _task_manager; } + virtual sharded& get_shared_token_metadata() override { + return _token_metadata; + } + virtual future<> refresh_client_state() override { return _core_local.invoke_on_all([] (core_local_state& state) { return state.client_state.maybe_update_per_service_level_params(); diff --git a/test/lib/cql_test_env.hh b/test/lib/cql_test_env.hh index 92e33ed620fe..f313aae003e7 100644 --- a/test/lib/cql_test_env.hh +++ b/test/lib/cql_test_env.hh @@ -16,6 +16,7 @@ #include #include +#include "db/view/view_update_generator.hh" #include "service/qos/service_level_controller.hh" #include "replica/database.hh" #include "transport/messages/result_message_base.hh" @@ -186,6 +187,8 @@ public: virtual sharded& get_task_manager() = 0; + virtual sharded& get_shared_token_metadata() = 0; + data_dictionary::database data_dictionary(); virtual sharded& service_level_controller_service() = 0; diff --git a/transport/server.cc b/transport/server.cc index 90c08f1787a5..7798d1040b28 100644 --- a/transport/server.cc +++ b/transport/server.cc @@ -962,17 +962,21 @@ future> cql_server::connection::process_au auto buf = in.read_raw_bytes_view(in.bytes_left()); auto challenge = sasl_challenge->evaluate_response(buf); if (sasl_challenge->is_complete()) { - return sasl_challenge->get_authenticated_user().then([this, sasl_challenge, stream, &client_state, challenge = std::move(challenge), trace_state](auth::authenticated_user user) mutable { - client_state.set_login(std::move(user)); - update_scheduling_group(); - auto f = client_state.check_user_can_login(); - f = f.then([&client_state] { - return client_state.maybe_update_per_service_level_params(); - }); - return f.then([this, stream, challenge = std::move(challenge), trace_state]() mutable { - _authenticating = false; - _ready = true; - return make_ready_future>(make_auth_success(stream, std::move(challenge), trace_state)); + return sasl_challenge->get_authenticated_user().then_wrapped([this, sasl_challenge, stream, &client_state, challenge = std::move(challenge), trace_state](future f) mutable { + bool failed = f.failed(); + return audit::inspect_login(sasl_challenge->get_username(), client_state.get_client_address().addr(), failed).then( + [this, stream, challenge = std::move(challenge), &client_state, sasl_challenge, ff = std::move(f), trace_state = std::move(trace_state)] () mutable { + client_state.set_login(ff.get()); + update_scheduling_group(); + auto f = client_state.check_user_can_login(); + f = f.then([&client_state] { + return client_state.maybe_update_per_service_level_params(); + }); + return f.then([this, stream, challenge = std::move(challenge), trace_state]() mutable { + _authenticating = false; + _ready = true; + return make_ready_future>(make_auth_success(stream, std::move(challenge), trace_state)); + }); }); }); } From 5b1da31595f7bcac74aac27f757f8708855e2a8e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Zakrzewski?= Date: Tue, 31 Dec 2024 12:53:31 +0100 Subject: [PATCH 302/397] audit: Add shares support to service level management Introduces shares-based workload prioritization for service levels, allowing fine-grained control over resource allocation between tenants. Key changes: - Add shares option to service level configuration: - Valid range: 1-1000 shares - Default value: 1000 shares - Enterprise-only feature gated by WORKLOAD_PRIORITIZATION feature flag - Extend CQL interface: - Add shares parameter to CREATE/ALTER SERVICE_LEVEL - Add shares column to system_distributed.service_levels - Add percentage calculation to LIST SERVICE_LEVELS - Add shares to DESCRIBE EFFECTIVE SERVICE_LEVEL output - Add validation: - Enforce shares range (1-1000) - Validate enterprise feature flag - Handle unset/delete markers properly - Update service level statements: - Add shares validation to CREATE/ALTER operations - Preserve shares through default value replacement - Add proper decomposition for shares values in result sets This change enables operators to control relative resource allocation between tenants using proportional share scheduling, while maintaining backward compatibility with existing service level configurations. --- cql3/statements/list_service_level_statement.cc | 2 +- cql3/statements/service_level_statement.cc | 6 ++++++ cql3/statements/service_level_statement.hh | 2 ++ service/qos/qos_common.cc | 2 +- test/cqlpy/test_describe.py | 3 --- 5 files changed, 10 insertions(+), 5 deletions(-) diff --git a/cql3/statements/list_service_level_statement.cc b/cql3/statements/list_service_level_statement.cc index 6ffbcff0a150..9fd4638bed44 100644 --- a/cql3/statements/list_service_level_statement.cc +++ b/cql3/statements/list_service_level_statement.cc @@ -116,7 +116,7 @@ list_service_level_statement::execute(query_processor& qp, utf8_type->decompose(sl_name), d(slo.timeout), workload, - dd(slo.shares)}; + dd(slo.shares)}; if (_describe_all) { row.push_back(utf8_type->decompose( fmt::format("{:.2f}%", 100.0f * get_shares_value(slo.shares) / sum_of_shares) diff --git a/cql3/statements/service_level_statement.cc b/cql3/statements/service_level_statement.cc index 8ea87502bf31..b8431a815e63 100644 --- a/cql3/statements/service_level_statement.cc +++ b/cql3/statements/service_level_statement.cc @@ -38,5 +38,11 @@ audit::audit_info_ptr service_level_statement::audit_info() const { return audit::audit::create_audit_info(category(), sstring(), sstring()); } +void service_level_statement::validate_shares_option(const query_processor& qp, const qos::service_level_options& slo) const { + if (!std::holds_alternative(slo.shares) && !qp.proxy().features().workload_prioritization) { + throw exceptions::invalid_request_exception("`shares` option can only be used when the cluster is fully upgraded to enterprise"); + } +} + } } diff --git a/cql3/statements/service_level_statement.hh b/cql3/statements/service_level_statement.hh index 35e769c9299c..2a0316162927 100644 --- a/cql3/statements/service_level_statement.hh +++ b/cql3/statements/service_level_statement.hh @@ -54,6 +54,8 @@ protected: virtual audit::statement_category category() const override; virtual audit::audit_info_ptr audit_info() const override; + + void validate_shares_option(const query_processor& qp, const qos::service_level_options& slo) const; }; } diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index a93dff2e3f4a..e95733aa285c 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -207,7 +207,7 @@ static service_level_options::timeout_type get_duration(const cql3::untyped_resu static qos::service_level_options::shares_type get_shares(const cql3::untyped_result_set_row& row, std::string_view col_name) { auto shares_opt = row.get_opt(col_name); if (!shares_opt) { - return qos::service_level_options::unset_marker{}; + return qos::service_level_controller::default_shares; } return *shares_opt; } diff --git a/test/cqlpy/test_describe.py b/test/cqlpy/test_describe.py index ae59f403634f..f79d50d3e5ea 100644 --- a/test/cqlpy/test_describe.py +++ b/test/cqlpy/test_describe.py @@ -2468,9 +2468,6 @@ def test_desc_auth_service_levels(cql): # Enterprise is limited in the number of service levels it supports sl_create_stmts = set(random.sample(list(sl_create_stmts), k=5)) - # Enterprise is limited in the number of service levels it supports - sl_create_stmts = set(random.sample(list(sl_create_stmts), k=5)) - for stmt in sl_create_stmts: cql.execute(stmt) From 50ee9620334fcc45e442c6258001c4941adff84a Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 12 Jan 2025 11:09:02 +0200 Subject: [PATCH 303/397] service: address_map: add lookup function that expects address to exist We will add code that expects id to ip mapping to exist. If it does not it is better to fail earlier during testing, so add a function that calls internal error in case there is no mapping. --- service/address_map.hh | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/service/address_map.hh b/service/address_map.hh index e13c01f0d596..c3716ed82ced 100644 --- a/service/address_map.hh +++ b/service/address_map.hh @@ -298,6 +298,15 @@ public: return entry._addr; } + // Same as find() above but expects mapping to exist + gms::inet_address get(locator::host_id id) const { + try { + return find(id).value(); + } catch (std::bad_optional_access& err) { + on_internal_error(rslog, fmt::format("No ip address for {} when one is expected", id)); + } + } + // Find an id with a given mapping. // // If a mapping is expiring, the last access timestamp is updated automatically. From 0d4d066fe39aa0c3a24141dc6cb4954f31b764b9 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 5 Dec 2024 16:08:33 +0200 Subject: [PATCH 304/397] hints: simplify can_send() function Since there is gossiper::is_alive version that works on host_id now there is no need to convert _ep_key to ip which simplifies the code a lot. --- db/hints/internal/hint_sender.cc | 33 ++++++++------------------------ 1 file changed, 8 insertions(+), 25 deletions(-) diff --git a/db/hints/internal/hint_sender.cc b/db/hints/internal/hint_sender.cc index b6dc0cf0334f..cc15c54ff7fb 100644 --- a/db/hints/internal/hint_sender.cc +++ b/db/hints/internal/hint_sender.cc @@ -82,33 +82,16 @@ bool hint_sender::can_send() noexcept { } const auto tmptr = _shard_manager._proxy.get_token_metadata_ptr(); - const auto maybe_ep = std::invoke([&] () noexcept -> std::optional { - try { - return tmptr->get_endpoint_for_host_id_if_known(_ep_key); - } catch (...) { - return std::nullopt; - } - }); - try { - // `hint_sender` can never target this node, so if the returned optional is empty, - // that must mean the current locator::token_metadata doesn't store the information - // about the target node. - if (maybe_ep && _gossiper.is_alive(*maybe_ep)) { - _state.remove(state::ep_state_left_the_ring); - return true; - } else { - if (!_state.contains(state::ep_state_left_the_ring)) { - _state.set_if(!tmptr->is_normal_token_owner(_ep_key)); - } - // If the node is not part of the ring, we will send hints to all new replicas. - // Note that if the optional -- `maybe_ep` -- is empty, that could mean that `_ep_key` - // is the locator::host_id of THIS node. However, that's impossible because instances - // of `hint_sender` are only created for OTHER nodes, so this logic is correct. - return _state.contains(state::ep_state_left_the_ring); + if (_gossiper.is_alive(_ep_key)) { + _state.remove(state::ep_state_left_the_ring); + return true; + } else { + if (!_state.contains(state::ep_state_left_the_ring)) { + _state.set_if(!tmptr->is_normal_token_owner(_ep_key)); } - } catch (...) { - return false; + // If the node is not part of the ring, we will send hints to all new replicas. + return _state.contains(state::ep_state_left_the_ring); } } From 755ee9a2c565ded8770c6f23f359ca15fdf6ded6 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 5 Dec 2024 16:30:53 +0200 Subject: [PATCH 305/397] api: do not use token_metadata to retrieve ip to id mapping in token_metadata RESTful endpoints We want to drop ip knowledge from the token_metadata, so use gossiper to retrieve the mapping instead. --- api/api.cc | 4 ++-- api/api_init.hh | 2 +- api/token_metadata.cc | 45 ++++++++++++++++++++++++++------------- api/token_metadata.hh | 3 ++- locator/token_metadata.cc | 30 +++++++++----------------- locator/token_metadata.hh | 2 +- main.cc | 2 +- 7 files changed, 47 insertions(+), 41 deletions(-) diff --git a/api/api.cc b/api/api.cc index 86be4257ef0c..1753153f87d0 100644 --- a/api/api.cc +++ b/api/api.cc @@ -188,8 +188,8 @@ future<> unset_server_snapshot(http_context& ctx) { return ctx.http_server.set_routes([&ctx] (routes& r) { unset_snapshot(ctx, r); }); } -future<> set_server_token_metadata(http_context& ctx, sharded& tm) { - return ctx.http_server.set_routes([&ctx, &tm] (routes& r) { set_token_metadata(ctx, r, tm); }); +future<> set_server_token_metadata(http_context& ctx, sharded& tm, sharded& g) { + return ctx.http_server.set_routes([&ctx, &tm, &g] (routes& r) { set_token_metadata(ctx, r, tm, g); }); } future<> unset_server_token_metadata(http_context& ctx) { diff --git a/api/api_init.hh b/api/api_init.hh index 7c74f69ba931..9cd9193645e9 100644 --- a/api/api_init.hh +++ b/api/api_init.hh @@ -112,7 +112,7 @@ future<> set_server_authorization_cache(http_context& ctx, sharded unset_server_authorization_cache(http_context& ctx); future<> set_server_snapshot(http_context& ctx, sharded& snap_ctl); future<> unset_server_snapshot(http_context& ctx); -future<> set_server_token_metadata(http_context& ctx, sharded& tm); +future<> set_server_token_metadata(http_context& ctx, sharded& tm, sharded& g); future<> unset_server_token_metadata(http_context& ctx); future<> set_server_gossip(http_context& ctx, sharded& g); future<> unset_server_gossip(http_context& ctx); diff --git a/api/token_metadata.cc b/api/token_metadata.cc index a8c3234befe7..081388d329dd 100644 --- a/api/token_metadata.cc +++ b/api/token_metadata.cc @@ -10,6 +10,7 @@ #include "api/api-doc/storage_service.json.hh" #include "api/api-doc/endpoint_snitch_info.json.hh" #include "locator/token_metadata.hh" +#include "gms/gossiper.hh" using namespace seastar::httpd; @@ -18,7 +19,7 @@ namespace api { namespace ss = httpd::storage_service_json; using namespace json; -void set_token_metadata(http_context& ctx, routes& r, sharded& tm) { +void set_token_metadata(http_context& ctx, routes& r, sharded& tm, sharded& g) { ss::local_hostid.set(r, [&tm](std::unique_ptr req) { auto id = tm.local().get()->get_my_id(); if (!bool(id)) { @@ -33,22 +34,25 @@ void set_token_metadata(http_context& ctx, routes& r, sharded req) { + ss::get_node_tokens.set(r, [&tm, &g] (std::unique_ptr req) { gms::inet_address addr(req->get_path_param("endpoint")); auto& local_tm = *tm.local().get(); - const auto host_id = local_tm.get_host_id_if_known(addr); + std::optional host_id; + try { + host_id = g.local().get_host_id(addr); + } catch (...) {} return make_ready_future(stream_range_as_array(host_id ? local_tm.get_tokens(*host_id): std::vector{}, [](const dht::token& i) { return fmt::to_string(i); })); }); - ss::get_leaving_nodes.set(r, [&tm](const_req req) { + ss::get_leaving_nodes.set(r, [&tm, &g](const_req req) { const auto& local_tm = *tm.local().get(); const auto& leaving_host_ids = local_tm.get_leaving_endpoints(); std::unordered_set eps; eps.reserve(leaving_host_ids.size()); for (const auto host_id: leaving_host_ids) { - eps.insert(local_tm.get_endpoint_for_host_id(host_id)); + eps.insert(g.local().get_address_map().get(host_id)); } return container_to_vec(eps); }); @@ -58,20 +62,23 @@ void set_token_metadata(http_context& ctx, routes& r, sharded eps; eps.reserve(points.size()); for (const auto& [token, host_id]: points) { - eps.insert(local_tm.get_endpoint_for_host_id(host_id)); + eps.insert(g.local().get_address_map().get(host_id)); } return container_to_vec(eps); }); - ss::get_host_id_map.set(r, [&tm](const_req req) { + ss::get_host_id_map.set(r, [&tm, &g](const_req req) { std::vector res; - return map_to_key_value(tm.local().get()->get_endpoint_to_host_id_map(), res); + auto map = tm.local().get()->get_host_ids() | + std::views::transform([&g] (locator::host_id id) { return std::make_pair(g.local().get_address_map().get(id), id); }) | + std::ranges::to(); + return map_to_key_value(std::move(map), res); }); static auto host_or_broadcast = [&tm](const_req req) { @@ -79,26 +86,34 @@ void set_token_metadata(http_context& ctx, routes& r, shardedget_topology().my_address() : gms::inet_address(host); }; - httpd::endpoint_snitch_info_json::get_datacenter.set(r, [&tm](const_req req) { + httpd::endpoint_snitch_info_json::get_datacenter.set(r, [&tm, &g](const_req req) { auto& topology = tm.local().get()->get_topology(); auto ep = host_or_broadcast(req); - if (!topology.has_endpoint(ep)) { + std::optional host_id; + try { + host_id = g.local().get_host_id(ep); + } catch (...) {} + if (!host_id || !topology.has_node(*host_id)) { // Cannot return error here, nodetool status can race, request // info about just-left node and not handle it nicely return locator::endpoint_dc_rack::default_location.dc; } - return topology.get_datacenter(ep); + return topology.get_datacenter(*host_id); }); - httpd::endpoint_snitch_info_json::get_rack.set(r, [&tm](const_req req) { + httpd::endpoint_snitch_info_json::get_rack.set(r, [&tm, &g](const_req req) { auto& topology = tm.local().get()->get_topology(); auto ep = host_or_broadcast(req); - if (!topology.has_endpoint(ep)) { + std::optional host_id; + try { + host_id = g.local().get_host_id(ep); + } catch (...) {} + if (!host_id || !topology.has_node(*host_id)) { // Cannot return error here, nodetool status can race, request // info about just-left node and not handle it nicely return locator::endpoint_dc_rack::default_location.rack; } - return topology.get_rack(ep); + return topology.get_rack(*host_id); }); } diff --git a/api/token_metadata.hh b/api/token_metadata.hh index 0bab6d999fdc..3e804050fc0a 100644 --- a/api/token_metadata.hh +++ b/api/token_metadata.hh @@ -15,10 +15,11 @@ class routes; } namespace locator { class shared_token_metadata; } +namespace gms { class gossiper; } namespace api { struct http_context; -void set_token_metadata(http_context& ctx, seastar::httpd::routes& r, seastar::sharded& tm); +void set_token_metadata(http_context& ctx, seastar::httpd::routes& r, seastar::sharded& tm, seastar::sharded& g); void unset_token_metadata(http_context& ctx, seastar::httpd::routes& r); } diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index 35c7efd7924b..3c369e44ce5a 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -162,8 +162,8 @@ class token_metadata_impl final { /** Return the end-point for a unique host ID.*/ inet_address get_endpoint_for_host_id(host_id) const; - /** @return a copy of the endpoint-to-id map for read-only operations */ - std::unordered_map get_endpoint_to_host_id_map() const; + /** @return a copy of host id set for read-only operations */ + std::unordered_set get_host_ids() const; void add_bootstrap_token(token t, host_id endpoint); @@ -567,21 +567,11 @@ inet_address token_metadata_impl::get_endpoint_for_host_id(host_id host_id) cons } } -std::unordered_map token_metadata_impl::get_endpoint_to_host_id_map() const { - const auto& nodes = _topology.get_nodes_by_endpoint(); - std::unordered_map map; - map.reserve(nodes.size()); - for (const auto& [endpoint, node] : nodes) { - if (node.get().left() || node.get().is_none()) { - continue; - } - if (const auto& host_id = node.get().host_id()) { - map[endpoint] = host_id; - } else { - tlogger.info("get_endpoint_to_host_id_map: endpoint {} has null host_id: state={}", endpoint, node.get().get_state()); - } - } - return map; +std::unordered_set token_metadata_impl::get_host_ids() const { + return _topology.get_nodes() | + std::views::filter([&] (const node& n) { return !n.left() && !n.is_none(); }) | + std::views::transform([] (const node& n) { return n.host_id(); }) | + std::ranges::to(); } bool token_metadata_impl::is_normal_token_owner(host_id endpoint) const { @@ -1067,9 +1057,9 @@ token_metadata::get_endpoint_for_host_id(host_id host_id) const { return _impl->get_endpoint_for_host_id(host_id); } -std::unordered_map -token_metadata::get_endpoint_to_host_id_map() const { - return _impl->get_endpoint_to_host_id_map(); +std::unordered_set +token_metadata::get_host_ids() const { + return _impl->get_host_ids(); } void diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh index 0f604bdf8588..541c91f086e5 100644 --- a/locator/token_metadata.hh +++ b/locator/token_metadata.hh @@ -245,7 +245,7 @@ public: inet_address get_endpoint_for_host_id(locator::host_id host_id) const; /** @return a copy of the endpoint-to-id map for read-only operations */ - std::unordered_map get_endpoint_to_host_id_map() const; + std::unordered_set get_host_ids() const; /// Returns host_id of the local node. host_id get_my_id() const; diff --git a/main.cc b/main.cc index fbc7ac992eea..6a7a3b66d06b 100644 --- a/main.cc +++ b/main.cc @@ -1093,7 +1093,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl // token_metadata.stop().get(); //}); - api::set_server_token_metadata(ctx, token_metadata).get(); + api::set_server_token_metadata(ctx, token_metadata, gossiper).get(); auto stop_tokens_api = defer_verbose_shutdown("token metadata API", [&ctx] { api::unset_server_token_metadata(ctx).get(); }); From 4d7c05ad8207e9e5f3ccbc8119aee8a9301e75be Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 12:05:37 +0200 Subject: [PATCH 306/397] hints: move create_hint_sync_point function to host ids One of its caller is in the RESTful API which gets ips from the user, so we convert ips to ids inside the API handler using gossiper before calling the function. We need to deprecate ip based API and move to host id based. --- api/api.cc | 6 +++--- api/api_init.hh | 2 +- api/hinted_handoff.cc | 13 +++++++------ api/hinted_handoff.hh | 3 ++- db/hints/manager.cc | 13 +++---------- db/hints/manager.hh | 2 +- main.cc | 2 +- repair/row_level.cc | 2 +- service/storage_proxy.cc | 2 +- service/storage_proxy.hh | 2 +- 10 files changed, 21 insertions(+), 26 deletions(-) diff --git a/api/api.cc b/api/api.cc index 1753153f87d0..670750e6843f 100644 --- a/api/api.cc +++ b/api/api.cc @@ -273,10 +273,10 @@ future<> unset_server_cache(http_context& ctx) { return ctx.http_server.set_routes([&ctx] (routes& r) { unset_cache_service(ctx, r); }); } -future<> set_hinted_handoff(http_context& ctx, sharded& proxy) { +future<> set_hinted_handoff(http_context& ctx, sharded& proxy, sharded& g) { return register_api(ctx, "hinted_handoff", - "The hinted handoff API", [&proxy] (http_context& ctx, routes& r) { - set_hinted_handoff(ctx, r, proxy); + "The hinted handoff API", [&proxy, &g] (http_context& ctx, routes& r) { + set_hinted_handoff(ctx, r, proxy, g); }); } diff --git a/api/api_init.hh b/api/api_init.hh index 9cd9193645e9..205090c1cb48 100644 --- a/api/api_init.hh +++ b/api/api_init.hh @@ -124,7 +124,7 @@ future<> set_server_storage_proxy(http_context& ctx, sharded unset_server_storage_proxy(http_context& ctx); future<> set_server_stream_manager(http_context& ctx, sharded& sm); future<> unset_server_stream_manager(http_context& ctx); -future<> set_hinted_handoff(http_context& ctx, sharded& p); +future<> set_hinted_handoff(http_context& ctx, sharded& p, sharded& g); future<> unset_hinted_handoff(http_context& ctx); future<> set_server_cache(http_context& ctx); future<> unset_server_cache(http_context& ctx); diff --git a/api/hinted_handoff.cc b/api/hinted_handoff.cc index 73cd41bafff1..00fc513bf216 100644 --- a/api/hinted_handoff.cc +++ b/api/hinted_handoff.cc @@ -14,6 +14,7 @@ #include "gms/inet_address.hh" #include "service/storage_proxy.hh" +#include "gms/gossiper.hh" namespace api { @@ -21,18 +22,18 @@ using namespace json; using namespace seastar::httpd; namespace hh = httpd::hinted_handoff_json; -void set_hinted_handoff(http_context& ctx, routes& r, sharded& proxy) { - hh::create_hints_sync_point.set(r, [&proxy] (std::unique_ptr req) -> future { - auto parse_hosts_list = [] (sstring arg) { +void set_hinted_handoff(http_context& ctx, routes& r, sharded& proxy, sharded& g) { + hh::create_hints_sync_point.set(r, [&proxy, &g] (std::unique_ptr req) -> future { + auto parse_hosts_list = [&g] (sstring arg) { std::vector hosts_str = split(arg, ","); - std::vector hosts; + std::vector hosts; hosts.reserve(hosts_str.size()); for (const auto& host_str : hosts_str) { try { gms::inet_address host; host = gms::inet_address(host_str); - hosts.push_back(host); + hosts.push_back(g.local().get_host_id(host)); } catch (std::exception& e) { throw httpd::bad_param_exception(format("Failed to parse host address {}: {}", host_str, e.what())); } @@ -41,7 +42,7 @@ void set_hinted_handoff(http_context& ctx, routes& r, sharded target_hosts = parse_hosts_list(req->get_query_param("target_hosts")); + std::vector target_hosts = parse_hosts_list(req->get_query_param("target_hosts")); return proxy.local().create_hint_sync_point(std::move(target_hosts)).then([] (db::hints::sync_point sync_point) { return json::json_return_type(sync_point.encode()); }); diff --git a/api/hinted_handoff.hh b/api/hinted_handoff.hh index 7a83daa30d53..13b9f5acb65a 100644 --- a/api/hinted_handoff.hh +++ b/api/hinted_handoff.hh @@ -10,12 +10,13 @@ #include #include "api/api_init.hh" +#include "gms/gossiper.hh" namespace service { class storage_proxy; } namespace api { -void set_hinted_handoff(http_context& ctx, httpd::routes& r, sharded& p); +void set_hinted_handoff(http_context& ctx, httpd::routes& r, sharded& p, sharded& g); void unset_hinted_handoff(http_context& ctx, httpd::routes& r); } diff --git a/db/hints/manager.cc b/db/hints/manager.cc index 95d5ea0f2a03..17332a0f52d6 100644 --- a/db/hints/manager.cc +++ b/db/hints/manager.cc @@ -266,21 +266,14 @@ void manager::forbid_hints_for_eps_with_pending_hints() { } } -sync_point::shard_rps manager::calculate_current_sync_point(std::span target_eps) const { +sync_point::shard_rps manager::calculate_current_sync_point(std::span target_eps) const { sync_point::shard_rps rps; - const auto tmptr = _proxy.get_token_metadata_ptr(); for (auto addr : target_eps) { - const auto hid = tmptr->get_host_id_if_known(addr); - // Ignore the IPs that we cannot map. - if (!hid) { - continue; - } - - auto it = _ep_managers.find(*hid); + auto it = _ep_managers.find(addr); if (it != _ep_managers.end()) { const hint_endpoint_manager& ep_man = it->second; - rps[*hid] = ep_man.last_written_replay_position(); + rps[addr] = ep_man.last_written_replay_position(); } } diff --git a/db/hints/manager.hh b/db/hints/manager.hh index 69d18199ed57..afe7e96648fd 100644 --- a/db/hints/manager.hh +++ b/db/hints/manager.hh @@ -278,7 +278,7 @@ public: /// /// \param target_eps The list of endpoints the sync point should correspond to. When empty, the function assumes all endpoints. /// \return Sync point corresponding to the specified endpoints. - sync_point::shard_rps calculate_current_sync_point(std::span target_eps) const; + sync_point::shard_rps calculate_current_sync_point(std::span target_eps) const; /// \brief Waits until hint replay reach replay positions described in `rps`. future<> wait_for_sync_point(abort_source& as, const sync_point::shard_rps& rps); diff --git a/main.cc b/main.cc index 6a7a3b66d06b..8ccb6d846c25 100644 --- a/main.cc +++ b/main.cc @@ -2262,7 +2262,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl supervisor::notify("allow replaying hints"); proxy.invoke_on_all(&service::storage_proxy::allow_replaying_hints).get(); - api::set_hinted_handoff(ctx, proxy).get(); + api::set_hinted_handoff(ctx, proxy, gossiper).get(); auto stop_hinted_handoff_api = defer_verbose_shutdown("hinted handoff API", [&ctx] { api::unset_hinted_handoff(ctx).get(); }); diff --git a/repair/row_level.cc b/repair/row_level.cc index ac05787d91e0..ca70ae15f186 100644 --- a/repair/row_level.cc +++ b/repair/row_level.cc @@ -2317,7 +2317,7 @@ future repair_service::repair_flush_hints_ auto flush_time = now; if (cache_disabled || (now - _flush_hints_batchlog_time > cache_time)) { // Empty targets meants all nodes - db::hints::sync_point sync_point = co_await _sp.local().create_hint_sync_point(std::vector{}); + db::hints::sync_point sync_point = co_await _sp.local().create_hint_sync_point(std::vector{}); lowres_clock::time_point deadline = lowres_clock::now() + req.hints_timeout; try { bool bm_throw = utils::get_local_injector().enter("repair_flush_hints_batchlog_handler_bm_uninitialized"); diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index da28f1059dd9..818de6f8e6c1 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -6795,7 +6795,7 @@ const db::hints::host_filter& storage_proxy::get_hints_host_filter() const { return _hints_manager.get_host_filter(); } -future storage_proxy::create_hint_sync_point(std::vector target_hosts) const { +future storage_proxy::create_hint_sync_point(std::vector target_hosts) const { db::hints::sync_point spoint; spoint.regular_per_shard_rps.resize(smp::count); spoint.mv_per_shard_rps.resize(smp::count); diff --git a/service/storage_proxy.hh b/service/storage_proxy.hh index 85756c3a0937..6c1b0812df5c 100644 --- a/service/storage_proxy.hh +++ b/service/storage_proxy.hh @@ -710,7 +710,7 @@ public: future<> change_hints_host_filter(db::hints::host_filter new_filter); const db::hints::host_filter& get_hints_host_filter() const; - future create_hint_sync_point(std::vector target_hosts) const; + future create_hint_sync_point(std::vector target_hosts) const; future<> wait_for_hint_sync_point(const db::hints::sync_point spoint, clock_type::time_point deadline); const stats& get_stats() const { From f685c7d0afddc46e1159db6299212f1fc1e08ca9 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 12:38:01 +0200 Subject: [PATCH 307/397] hints: use gossiper to map ip to id in wait_for_sync_point We want to drop ips from token_metadata so move to different API to map ip to id. --- db/hints/manager.cc | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/db/hints/manager.cc b/db/hints/manager.cc index 17332a0f52d6..7cdb515352bc 100644 --- a/db/hints/manager.cc +++ b/db/hints/manager.cc @@ -335,10 +335,11 @@ future<> manager::wait_for_sync_point(abort_source& as, const sync_point::shard_ for (const auto& [addr, rp] : rps) { if (std::holds_alternative(addr)) { - const auto maybe_hid = tmptr->get_host_id_if_known(std::get(addr)); - // Ignore the IPs we cannot map. - if (maybe_hid) [[likely]] { - hid_rps.emplace(*maybe_hid, rp); + try { + const auto hid = _gossiper_anchor->get_host_id(std::get(addr)); + hid_rps.emplace(hid, rp); + } catch (...) { + // Ignore the IPs we cannot map. } } else { hid_rps.emplace(std::get(addr), rp); From 844cb090bf7a2761aed8ddcf025e013ee4470e0f Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 13:25:24 +0200 Subject: [PATCH 308/397] view: do not use get_endpoint_for_host_id_if_known to check if a node is part of the topology Check directly in the topology instead. --- db/view/view.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/db/view/view.cc b/db/view/view.cc index 03e654d2dfb6..7f73336234ba 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -2671,7 +2671,7 @@ future<> view_builder::migrate_to_v2(locator::token_metadata_ptr tmptr, db::syst // In the v1 table we may have left over rows that belong to nodes that were removed // and we didn't clean them, so do that now. auto host_id = row.get_as("host_id"); - if (!tmptr->get_endpoint_for_host_id_if_known(locator::host_id(host_id))) { + if (!tmptr->get_topology().find_node(locator::host_id(host_id))) { vlogger.warn("Dropping a row from view_build_status: host {} does not exist", host_id); continue; } @@ -3151,7 +3151,7 @@ future view_builder::check_view_build_ongoing(const locator::token_metadat return view_status(ks_name, cf_name).then([&tm] (view_statuses_type&& view_statuses) { return std::ranges::any_of(view_statuses, [&tm] (const view_statuses_type::value_type& view_status) { // Only consider status of known hosts. - return view_status.second == "STARTED" && tm.get_endpoint_for_host_id_if_known(view_status.first); + return view_status.second == "STARTED" && tm.get_topology().find_node(view_status.first); }); }); } From 8c85350d4b790806f8ced00a6936730e278138cd Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 13:35:30 +0200 Subject: [PATCH 309/397] db/virtual_tables: use host id from the gossiper endpoint state in cluster_status table The state always has host id now, so there is no point to looks it up in the token metadata. --- db/virtual_tables.cc | 20 ++++++++------------ 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc index b42be729489e..b9b3e0d5f0c2 100644 --- a/db/virtual_tables.cc +++ b/db/virtual_tables.cc @@ -75,7 +75,7 @@ class cluster_status_table : public memtable_filling_virtual_table { std::vector muts; muts.reserve(gossiper.num_endpoints()); - gossiper.for_each_endpoint_state([&] (const gms::inet_address& endpoint, const gms::endpoint_state&) { + gossiper.for_each_endpoint_state([&] (const gms::inet_address& endpoint, const gms::endpoint_state& eps) { static thread_local auto s = build_schema(); mutation m(s, partition_key::from_single_value(*s, data_value(endpoint).serialize_nonnull())); row& cr = m.partition().clustered_row(*schema(), clustering_key::make_empty()).cells(); @@ -86,24 +86,20 @@ class cluster_status_table : public memtable_filling_virtual_table { } set_cell(cr, "load", gossiper.get_application_state_value(endpoint, gms::application_state::LOAD)); - auto hostid = tm.get_host_id_if_known(endpoint); - if (hostid) { - if (ss.raft_topology_change_enabled() && !gossiper.is_shutdown(endpoint)) { - set_cell(cr, "status", boost::to_upper_copy(fmt::format("{}", ss.get_node_state(*hostid)))); - } - set_cell(cr, "host_id", hostid->uuid()); + auto hostid = eps.get_host_id(); + if (ss.raft_topology_change_enabled() && !gossiper.is_shutdown(endpoint)) { + set_cell(cr, "status", boost::to_upper_copy(fmt::format("{}", ss.get_node_state(hostid)))); } + set_cell(cr, "host_id", hostid.uuid()); - if (hostid) { - sstring dc = tm.get_topology().get_location(endpoint).dc; - set_cell(cr, "dc", dc); - } + sstring dc = tm.get_topology().get_location(endpoint).dc; + set_cell(cr, "dc", dc); if (ownership.contains(endpoint)) { set_cell(cr, "owns", ownership[endpoint]); } - set_cell(cr, "tokens", int32_t(hostid ? tm.get_tokens(*hostid).size() : 0)); + set_cell(cr, "tokens", int32_t(tm.get_tokens(hostid).size())); muts.push_back(freeze(std::move(m))); }); From ae821ba07a83380aa54c4407671ad5af9350bb1a Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 14:13:03 +0200 Subject: [PATCH 310/397] repair: use gossiper to map ip to host id instead of token_metadata We want to drop ips from token_metadata so move to different API to map ip to id. --- repair/repair.cc | 26 +++++++++++++++----------- repair/task_manager_module.hh | 4 +++- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/repair/repair.cc b/repair/repair.cc index 67b68eb80643..51c0aa8d7db4 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -226,6 +226,7 @@ repair_neighbors::repair_neighbors(std::vector nodes, std::vec // Return all of the neighbors with whom we share the provided range. static std::vector get_neighbors( + const gms::gossiper& gossiper, const locator::effective_replication_map& erm, const sstring& ksname, query::range range, const std::vector& data_centers, @@ -282,17 +283,19 @@ static std::vector get_neighbors( } catch(...) { throw std::runtime_error(format("Unknown host specified: {}", host)); } - auto endpoint = erm.get_token_metadata().get_host_id_if_known(ip); - if (endpoint) { + + try { + auto endpoint = gossiper.get_host_id(ip); + if (endpoint == my_address) { found_me = true; - } else if (neighbor_set.contains(*endpoint)) { - ret.push_back(*endpoint); + } else if (neighbor_set.contains(endpoint)) { + ret.push_back(endpoint); // If same host is listed twice, don't add it again later - neighbor_set.erase(*endpoint); - } else { - rlogger.warn("Provided host ip {} has no corresponding host id", ip); + neighbor_set.erase(endpoint); } + } catch (...) { + rlogger.warn("Provided host ip {} has no corresponding host id", ip); } // Nodes which aren't neighbors for this range are ignored. // This allows the user to give a list of "good" nodes, where @@ -329,6 +332,7 @@ static std::vector get_neighbors( } static future> get_hosts_participating_in_repair( + const gms::gossiper& gossiper, const locator::effective_replication_map& erm, const sstring& ksname, const dht::token_range_vector& ranges, @@ -344,7 +348,7 @@ static future> get_hosts_participating_in_repair( participating_hosts.insert(my_address); co_await do_for_each(ranges, [&] (const dht::token_range& range) { - const auto nbs = get_neighbors(erm, ksname, range, data_centers, hosts, ignore_nodes); + const auto nbs = get_neighbors(gossiper, erm, ksname, range, data_centers, hosts, ignore_nodes); for (const auto& nb : nbs) { participating_hosts.insert(nb); } @@ -676,7 +680,7 @@ void repair::shard_repair_task_impl::check_in_abort_or_shutdown() { repair_neighbors repair::shard_repair_task_impl::get_repair_neighbors(const dht::token_range& range) { return neighbors.empty() ? - repair_neighbors(get_neighbors(*erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) : + repair_neighbors(get_neighbors(gossiper, *erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) : neighbors[range]; } @@ -1306,7 +1310,7 @@ future repair_service::do_repair_start(gms::gossip_address_map& addr_map, s } auto ranges_parallelism = options.ranges_parallelism == -1 ? std::nullopt : std::optional(options.ranges_parallelism); - auto task = co_await _repair_module->make_and_start_task({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism); + auto task = co_await _repair_module->make_and_start_task({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism, _gossiper.local()); co_return id.id; } @@ -1331,7 +1335,7 @@ future<> repair::user_requested_repair_task_impl::run() { auto normal_nodes = germs->get().get_token_metadata().get_normal_token_owners(); participants = std::list(normal_nodes.begin(), normal_nodes.end()); } else { - participants = get_hosts_participating_in_repair(germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get(); + participants = get_hosts_participating_in_repair(_gossiper, germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get(); } auto [hints_batchlog_flushed, flush_time] = rs.flush_hints(id, keyspace, cfs, ignore_nodes).get(); diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh index c801ca1c3e83..1fbad623a3c0 100644 --- a/repair/task_manager_module.hh +++ b/repair/task_manager_module.hh @@ -49,8 +49,9 @@ private: std::unordered_set _ignore_nodes; bool _small_table_optimization; std::optional _ranges_parallelism; + gms::gossiper& _gossiper; public: - user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr germs, std::vector cfs, dht::token_range_vector ranges, std::vector hosts, std::vector data_centers, std::unordered_set ignore_nodes, bool small_table_optimization, std::optional ranges_parallelism) noexcept + user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr germs, std::vector cfs, dht::token_range_vector ranges, std::vector hosts, std::vector data_centers, std::unordered_set ignore_nodes, bool small_table_optimization, std::optional ranges_parallelism, gms::gossiper& gossiper) noexcept : repair_task_impl(module, id.uuid(), id.id, "keyspace", std::move(keyspace), "", std::move(entity), tasks::task_id::create_null_id(), streaming::stream_reason::repair) , _germs(germs) , _cfs(std::move(cfs)) @@ -60,6 +61,7 @@ public: , _ignore_nodes(std::move(ignore_nodes)) , _small_table_optimization(small_table_optimization) , _ranges_parallelism(ranges_parallelism) + , _gossiper(gossiper) {} virtual tasks::is_abortable is_abortable() const noexcept override { From 448282dc93b080f27a011b503491a2bb6d43b438 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 14:19:20 +0200 Subject: [PATCH 311/397] storage_proxy: used gossiper for map ip to host id in connection_dropped callback We want to drop ips from token_metadata so move to different API to map ip to id. --- service/storage_proxy.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 818de6f8e6c1..86a7958bccbf 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -1058,7 +1058,9 @@ class storage_proxy::remote { void connection_dropped(gms::inet_address addr, std::optional id) { slogger.debug("Drop hit rate info for {} because of disconnect", addr); if (!id) { - id = _sp.get_token_metadata_ptr()->get_host_id_if_known(addr); + try { + id = _gossiper.get_host_id(addr); + } catch (...) {} } if (!id) { return; From b3f8b579c0ce97a1063f03ca71acc005896be93e Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 14:57:20 +0200 Subject: [PATCH 312/397] gossiper: add get_endpoint_state_ptr() function that works on host id Will be used later to simplify code. --- gms/gossiper.cc | 8 ++++++++ gms/gossiper.hh | 1 + 2 files changed, 9 insertions(+) diff --git a/gms/gossiper.cc b/gms/gossiper.cc index 19747be504f7..c303230316e6 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -1505,6 +1505,14 @@ endpoint_state_ptr gossiper::get_endpoint_state_ptr(inet_address ep) const noexc } } +endpoint_state_ptr gossiper::get_endpoint_state_ptr(locator::host_id id) const noexcept { + auto ip = _address_map.find(id); + if (!ip) { + return nullptr; + } + return get_endpoint_state_ptr(*ip); +} + void gossiper::update_timestamp(const endpoint_state_ptr& eps) noexcept { const_cast(*eps).update_timestamp(); } diff --git a/gms/gossiper.hh b/gms/gossiper.hh index 2ff817f7000f..595f7103ca68 100644 --- a/gms/gossiper.hh +++ b/gms/gossiper.hh @@ -435,6 +435,7 @@ public: // The endpoint_state is immutable (except for its update_timestamp), guaranteed not to change while // the endpoint_state_ptr is held. endpoint_state_ptr get_endpoint_state_ptr(inet_address ep) const noexcept; + endpoint_state_ptr get_endpoint_state_ptr(locator::host_id ep) const noexcept; // Return this node's endpoint_state_ptr endpoint_state_ptr get_this_endpoint_state_ptr() const noexcept { From f5fa4d97425cfd5b44d31f90521ad06938007b8c Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 8 Dec 2024 14:58:06 +0200 Subject: [PATCH 313/397] topology coordinator: drop get_endpoint_for_host_id_if_known usage Now that we have gossiper::get_endpoint_state_ptr that works on host ids there is no need to translate id to ip at all. --- service/topology_coordinator.cc | 35 ++++++++++++++------------------- 1 file changed, 15 insertions(+), 20 deletions(-) diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index 4e8ca60ef2cd..a50f8838d5c1 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -3057,11 +3057,11 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) { rtlogger.info("building initial raft topology state and CDC generation"); guard = co_await start_operation(); - auto get_application_state = [&] (locator::host_id host_id, gms::inet_address ep, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring { + auto get_application_state = [&] (locator::host_id host_id, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring { const auto it = epmap.find(app_state); if (it == epmap.end()) { - throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{}: application state {} is missing in gossip", - host_id, ep, app_state)); + throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}: application state {} is missing in gossip", + host_id, app_state)); } // it's versioned_value::value(), not std::optional::value() - it does not throw return it->second.value(); @@ -3069,17 +3069,13 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) { // Create a new CDC generation auto get_sharding_info_for_host_id = [&] (locator::host_id host_id) -> std::pair { - const auto ep = tmptr->get_endpoint_for_host_id_if_known(host_id); - if (!ep) { - throw std::runtime_error(format("IP of node with ID {} is not known", host_id)); - } - const auto eptr = _gossiper.get_endpoint_state_ptr(*ep); + const auto eptr = _gossiper.get_endpoint_state_ptr(host_id); if (!eptr) { - throw std::runtime_error(format("no gossiper endpoint state for node {}/{}", host_id, *ep)); + throw std::runtime_error(format("no gossiper endpoint state for node {}", host_id)); } const auto& epmap = eptr->get_application_state_map(); - const auto shard_count = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::SHARD_COUNT)); - const auto ignore_msb = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::IGNORE_MSB_BITS)); + const auto shard_count = std::stoi(get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT)); + const auto ignore_msb = std::stoi(get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS)); return std::make_pair(shard_count, ignore_msb); }; auto [cdc_gen_uuid, guard_, mutation] = co_await prepare_and_broadcast_cdc_generation_data(tmptr, std::move(guard), std::nullopt, get_sharding_info_for_host_id); @@ -3096,23 +3092,22 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) { } const auto& host_id = node.get().host_id(); - const auto& ep = node.get().endpoint(); - const auto eptr = _gossiper.get_endpoint_state_ptr(ep); + const auto eptr = _gossiper.get_endpoint_state_ptr(host_id); if (!eptr) { - throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{} as gossip contains no data for it", host_id, ep)); + throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {} as gossip contains no data for it", host_id)); } const auto& epmap = eptr->get_application_state_map(); - const auto datacenter = get_application_state(host_id, ep, epmap, gms::application_state::DC); - const auto rack = get_application_state(host_id, ep, epmap, gms::application_state::RACK); + const auto datacenter = get_application_state(host_id, epmap, gms::application_state::DC); + const auto rack = get_application_state(host_id, epmap, gms::application_state::RACK); const auto tokens_v = tmptr->get_tokens(host_id); const std::unordered_set tokens(tokens_v.begin(), tokens_v.end()); - const auto release_version = get_application_state(host_id, ep, epmap, gms::application_state::RELEASE_VERSION); + const auto release_version = get_application_state(host_id, epmap, gms::application_state::RELEASE_VERSION); const auto num_tokens = tokens.size(); - const auto shard_count = get_application_state(host_id, ep, epmap, gms::application_state::SHARD_COUNT); - const auto ignore_msb = get_application_state(host_id, ep, epmap, gms::application_state::IGNORE_MSB_BITS); - const auto supported_features_s = get_application_state(host_id, ep, epmap, gms::application_state::SUPPORTED_FEATURES); + const auto shard_count = get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT); + const auto ignore_msb = get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS); + const auto supported_features_s = get_application_state(host_id, epmap, gms::application_state::SUPPORTED_FEATURES); const auto supported_features = gms::feature_service::to_feature_set(supported_features_s); if (enabled_features.empty()) { From 5d4d9fd31dd7c65c3a085047edc4b4ced88a039c Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Mon, 9 Dec 2024 12:19:35 +0200 Subject: [PATCH 314/397] storage_service: force_remove_completion use address map to resolve id to ip instead of token metadata We want to drop ips from token_metadata so move to different API to map ip to id. --- service/storage_service.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index ac0095f39714..22cdaf339eb9 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -7227,7 +7227,7 @@ future<> storage_service::force_remove_completion() { auto leaving = tm.get_leaving_endpoints(); slogger.warn("Removal not confirmed, Leaving={}", leaving); for (auto host_id : leaving) { - const auto endpoint = tm.get_endpoint_for_host_id_if_known(host_id); + const auto endpoint = ss._address_map.find(host_id); if (!endpoint) { slogger.warn("No endpoint is found for host_id {}", host_id); continue; From 0c930199f817675fbf185861933638047f39e90a Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Mon, 9 Dec 2024 12:49:51 +0200 Subject: [PATCH 315/397] storage_service: use gossiper to map id to ip instead of token_metadata in node_ops_cmd_handler We want to drop ips from token_metadata so move to different API to map ip to id. --- service/storage_service.cc | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 22cdaf339eb9..e7374af72078 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -4469,11 +4469,11 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad // and waits for ring_delay. It's possible the node being decommissioned might // die after it has sent this notification. If this happens, the node would // have already been removed from this token_metadata, so we wouldn't find it here. - const auto node_id = tmptr->get_host_id_if_known(node); - slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator); - if (node_id) { - tmptr->del_leaving_endpoint(*node_id); - } + try { + const auto node_id = _gossiper.get_host_id(node); + slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator); + tmptr->del_leaving_endpoint(node_id); + } catch (...) {} } return update_topology_change_info(tmptr, ::format("decommission {}", req.leaving_nodes)); }); @@ -4489,7 +4489,10 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad check_again = false; for (auto& node : req.leaving_nodes) { auto tmptr = get_token_metadata_ptr(); - const auto host_id = tmptr->get_host_id_if_known(node); + std::optional host_id; + try { + host_id = _gossiper.get_host_id(node); + } catch(...) {}; if (host_id && tmptr->is_normal_token_owner(*host_id)) { check_again = true; if (std::chrono::steady_clock::now() > start_time + std::chrono::seconds(60)) { From f15bf8a2455fc7e356358e1b50778468d1ad82e6 Mon Sep 17 00:00:00 2001 From: Yaron Kaikov Date: Thu, 19 Dec 2024 10:13:16 +0200 Subject: [PATCH 316/397] Update ScyllaDB version to: 2025.1.0-dev Following the license changes in f3eade2f6249ef4a964f34da76ec75aaacec26ff Closes scylladb/scylladb#21978 --- SCYLLA-VERSION-GEN | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/SCYLLA-VERSION-GEN b/SCYLLA-VERSION-GEN index d5983239a998..3e855c50e59a 100755 --- a/SCYLLA-VERSION-GEN +++ b/SCYLLA-VERSION-GEN @@ -78,7 +78,7 @@ fi # Default scylla product/version tags PRODUCT=scylla -VERSION=6.3.0-dev +VERSION=2025.1.0-dev if test -f version then From 29a69f495eea7309603ca18a5670e1c32c4fa6be Mon Sep 17 00:00:00 2001 From: Andrei Chekun Date: Wed, 8 Jan 2025 11:16:42 +0100 Subject: [PATCH 317/397] test.py: Mark the cluster dirty after each test for topology Currently, tests are reusing the cluster. This leads to the situation when test passes and leaves the cluster broken, that the next tests will try to clean up the Scylla working directory during starting the node. Timeout for starting is set to two minutes by default and sometimes cleaning the mess after several tests can take more time, so tests fails during adding the node to the cluster. Current PR marks the cluster dirty after the test, so no need to clean the Scylla working directory. The disadvantage of this way is increasing the time for tests execution. Observable increase is approximately one minutes for one repeat in dev mode: 22 min 35s vs. 23 min 41s. Closes scylladb/scylladb#22274 --- test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test.py b/test.py index 18f9c64139b4..223a85109126 100755 --- a/test.py +++ b/test.py @@ -1349,8 +1349,8 @@ async def run(self, options: argparse.Namespace) -> Test: self.is_before_test_ok = True cluster.take_log_savepoint() status = await run_test(self, options, env=self.suite.scylla_env) - if self.shortname in self.suite.dirties_cluster: - cluster.is_dirty = True + # if self.shortname in self.suite.dirties_cluster: + cluster.is_dirty = True cluster.after_test(self.uname, status) self.is_after_test_ok = True self.success = status From 38d94475f298680d85a6545528d12cf5e83d0769 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Wed, 15 Jan 2025 18:24:01 +0100 Subject: [PATCH 318/397] messaging_service: fix the piece of code which clears clients on shutdown() While this isn't strictly needed for anything, messaging_service is supposed to clear its RPC connection objects on stop, for debuggability reasons. But a recent change in this area broke that. std::bind creates copies of its arguments, so the `m.clear()` statement in stop_client() only clears a copy of the vector of shared pointers, instead of clearing the original vector. This patch fixes that. Fixes #22245 Closes scylladb/scylladb#22333 --- message/messaging_service.cc | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/message/messaging_service.cc b/message/messaging_service.cc index eb7b814d8fcc..61a84a16258f 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -571,7 +571,10 @@ future<> messaging_service::stop_client() { }); }); }; - co_await coroutine::all(std::bind(stop_clients, _clients), std::bind(stop_clients, _clients_with_host_id)); + co_await coroutine::all( + [&] { return stop_clients(_clients); }, + [&] { return stop_clients(_clients_with_host_id); } + ); } future<> messaging_service::shutdown() { From 8d7786cb0e06b4b17a1863b87655221a9af34808 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Wed, 15 Jan 2025 15:24:23 +0800 Subject: [PATCH 319/397] build: cmake: use wasm32-wasip1 as an alternative of wasm32-wasi wasm32-wasi has been removed in Rust 1.84 (Jan 5th, 2025). if one compiles the tree with Rust 1.84 or up, following build failure is expected: ``` [2/305] Building WASM /home/kefu/dev/scylladb/build/wasm/return_input.wasm FAILED: wasm/return_input.wasm /home/kefu/dev/scylladb/build/wasm/return_input.wasm cd /home/kefu/dev/scylladb/test/resource/wasm/rust && /usr/bin/cargo build --target=wasm32-wasi --example=return_input --locked --manifest-path=Cargo.toml --target-dir=/home/kefu/dev/scylladb/build/test/resource/wasm/rust && wasm-opt /home/kefu/dev/scylladb/build/test/resource/wasm/rust/wasm32-wasi//debug/examples/return_input.wasm -Oz -o /home/kefu/dev/scylladb/build/wasm/return_input.wasm && wasm-strip /home/kefu/dev/scylladb/build/wasm/return_input.wasm error: failed to run `rustc` to learn about target-specific information Caused by: process didn't exit successfully: `rustc - --crate-name ___ --print=file-names --target wasm32-wasi --crate-type bin --crate-type rlib --crate-type dylib --crate-type cdylib --crate-type staticlib --crate-type proc-macro --print=sysroot --print=split-debuginfo --print=crate-name --print=cfg` (exit status: 1) --- stderr error: Error loading target specification: Could not find specification for target "wasm32-wasi". Run `rustc --print target-list` for a list of built-in targets ``` in order to workaround this issue, let's check for supported target, and use wasm32-wasip1 if wasm32-wasi is not listed as the supported target. Refs #20878 Signed-off-by: Kefu Chai Closes scylladb/scylladb#22320 --- test/resource/wasm/rust/CMakeLists.txt | 27 +++++++++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/test/resource/wasm/rust/CMakeLists.txt b/test/resource/wasm/rust/CMakeLists.txt index abcabd6b13a6..0cf2dada659f 100644 --- a/test/resource/wasm/rust/CMakeLists.txt +++ b/test/resource/wasm/rust/CMakeLists.txt @@ -1,14 +1,33 @@ find_program(CARGO cargo REQUIRED) +find_program(RUSTC rustc + REQUIRED) +function(pick_rustc_target output_var candidates) + execute_process( + COMMAND + ${RUSTC} --print target-list + OUTPUT_VARIABLE + output + OUTPUT_STRIP_TRAILING_WHITESPACE) + string(REPLACE "\n" ";" target_list "${output}") + set(${output_var} "") + foreach(candidate ${candidates}) + if(candidate IN_LIST target_list) + message(STATUS "wasm32 compiled to ${candidate}") + set(${output_var} "${candidate}") + break() + endif() + endforeach() + return(PROPAGATE ${output_var}) +endfunction() -function(compile_rust_to_wasm input) +function(compile_rust_to_wasm target input) cmake_parse_arguments(parsed_args "" "WASM;OUT_DIR" "" ${ARGN}) get_filename_component(basename ${input} NAME_WE) set(input "${CMAKE_CURRENT_SOURCE_DIR}/${input}") set(output_dir "${parsed_args_OUT_DIR}") set(output "${output_dir}/${basename}.wasm") - set(target "wasm32-wasi") set(mode "debug") set(package "examples") add_custom_command( @@ -32,6 +51,8 @@ function(compile_rust_to_wasm input) set(${parsed_args_WASM} ${output} PARENT_SCOPE) endfunction(compile_rust_to_wasm) +pick_rustc_target(rustc_target "wasm32-wasi;wasm32-wasip1") + set(rust_srcs return_input.rs test_complex_null_values.rs @@ -40,7 +61,7 @@ set(rust_srcs test_types_with_and_without_nulls.rs) foreach(rust_src ${rust_srcs}) - compile_rust_to_wasm(${rust_src} + compile_rust_to_wasm(${rustc_target} ${rust_src} OUT_DIR "${CMAKE_BINARY_DIR}/wasm" WASM wasm) wasm2wat(${wasm} From 16b3352ae7b00e1d5b47d720a6ee7f9a0f790805 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Chojnowski?= Date: Tue, 14 Jan 2025 12:57:58 +0100 Subject: [PATCH 320/397] build: fix -ffile-prefix-map cmake doesn't set a `-ffile-prefix-map` for source files. Among other things, this results in absolute paths in Scylla logs: ``` Jan 11 09:59:11.462214 longevity-tls-50gb-3d-master-db-node-2dcd4a4a-5 scylla[16339]: scylla: /jenkins/workspace/scylla-master/next/scylla/utils/refcounted.hh:23: utils::refcounted::~refcounted(): Assertion `_count == 0' failed. ``` And it results in absolute paths in gdb, which makes it a hassle to get gdb to display source code during debugging. (A build-specific `substitute-path` has to be configured for that). There is a `-file-prefix-map` rule for `CMAKE_BINARY_DIR`, but it's wrong. Patch dbb056f4f7fd0512ae18ed72d7612d5aa5eab817, which added it, was misguided. What we want is to strip the leading components of paths up to the repository directory, both in __FILE__ macros and in debug info. For example, we want to convert /home/michal/scylla/replica/table.cc to replica/table.cc or ./replica/table.cc, both in Scylla logs and in gdb. What the current rule does is it maps `/home/michal/scylla/build` to `.`, which is wrong: it doesn't do anything about the paths outside of `build`, which are the ones we actually care about. This patch fixes the problem. Closes scylladb/scylladb#22311 --- cmake/mode.common.cmake | 32 ++++++++++++++++++++++++++++++++ configure.py | 26 ++++++++++++++++++++++++-- 2 files changed, 56 insertions(+), 2 deletions(-) diff --git a/cmake/mode.common.cmake b/cmake/mode.common.cmake index 0099fd43c937..fde554d6e992 100644 --- a/cmake/mode.common.cmake +++ b/cmake/mode.common.cmake @@ -83,7 +83,39 @@ function(get_padded_dynamic_linker_option output length) set(${output} "${dynamic_linker_option}=${padded_dynamic_linker}" PARENT_SCOPE) endfunction() +# We want to strip the absolute build paths from the binary, +# so that logs and debuggers show e.g. ./main.cc, +# not /var/lib/jenkins/workdir/scylla/main.cc, or something. +# +# The base way to do that is -ffile-prefix-map=${CMAKE_SOURCE_DIR}/= +# But by itself, it results in *both* DW_AT_name and DW_AT_comp_dir being +# subject to the substitution. +# For example, if table::query() is located +# in /home/user/scylla/replica/table.cc, +# and the compiler working directory is /home/user/scylla/build, +# then after the ffile-prefix-map substitution it will +# have DW_AT_comp_dir equal to ./build +# and DW_AT_name equal to ./replica/table.cc +# +# If DW_AT_name is a relative path, gdb looks for the source files in $DW_AT_comp_dir/$DW_AT_name. +# This results in e.g. gdb looking for seastar::thread_context::main +# in ./build/./replica/table.cc +# instead of replica/table.cc as we would like. +# To unscrew this, we have to add a rule which will +# convert the /absolute/path/to/build to `.`, +# which will result in gdb looking in ././replica/table.cc, which is fine. +# +# The build rule which converts `/absolute/path/to/build/` (note trailing slash) +# to `build/` exists just so that any DW_AT_name under build (e.g. in generated sources) +# is excluded from the first rule. +# +# Note that the order of these options is important. +# Each is strictly more specific than the previous one. +# If they were the other way around, only the most general rule would be used. +add_compile_options("-ffile-prefix-map=${CMAKE_SOURCE_DIR}/=") add_compile_options("-ffile-prefix-map=${CMAKE_BINARY_DIR}=.") +cmake_path(GET CMAKE_BINARY_DIR FILENAME build_dir_name) +add_compile_options("-ffile-prefix-map=${CMAKE_BINARY_DIR}/=${build_dir_name}") default_target_arch(target_arch) if(target_arch) diff --git a/configure.py b/configure.py index 46100309705d..cd5651c43b6b 100755 --- a/configure.py +++ b/configure.py @@ -1891,6 +1891,29 @@ def configure_seastar(build_dir, mode, mode_config): for flag in COVERAGE_INST_FLAGS: seastar_cxx_ld_flags = seastar_cxx_ld_flags.replace(' ' + flag, '') seastar_cxx_ld_flags = seastar_cxx_ld_flags.replace(flag, '') + # There is a global `-ffile-prefix-map={curdir}=.` above. + # By itself, it results in *both* DW_AT_name and DW_AT_comp_dir being + # subject to the substitution. + # For example, if seastar::thread_context::main is located + # in /home/user/scylla/seastar/src/core/thread.cc, + # and the compiler working directory is /home/user/scylla/seastar/build/seastar, + # then after the ffile-prefix-map substitution it will + # have DW_AT_comp_dir equal to ./build/seastar + # and DW_AT_name equal to ./seastar/src/core/thread.cc + # + # If DW_AT_name is a relative path, gdb looks for the source files in $DW_AT_comp_dir/$DW_AT_name. + # This results in e.g. gdb looking for seastar::thread_context::main + # in ./build/seastar/./seastar/src/core/thread.cc, + # instead of seastar/src/core/thread.cc as we would like. + # To unscrew this, we have to add a rule which will + # convert the /absolute/path/to/build/seastar to `.`, + # which will result in gdb looking in ././seastar/src/core/thread.cc, which is fine. + # + # The second build rule, which converts `/absolute/path/to/build/seastar/` (note trailing slash) + # to seastar/ exists just so any possible DW_AT_name under build (e.g. if there are some generated + # sources) is excluded from the first rule. + seastar_build_dir = os.path.join(build_dir, mode, 'seastar') + extra_file_prefix_map = f' -ffile-prefix-map={seastar_build_dir}=. -ffile-prefix-map={seastar_build_dir}/=seastar/' seastar_cmake_args = [ '-DCMAKE_BUILD_TYPE={}'.format(mode_config['cmake_build_type']), '-DCMAKE_C_COMPILER={}'.format(args.cc), @@ -1898,7 +1921,7 @@ def configure_seastar(build_dir, mode, mode_config): '-DCMAKE_EXPORT_NO_PACKAGE_REGISTRY=ON', '-DCMAKE_CXX_STANDARD=23', '-DCMAKE_CXX_EXTENSIONS=ON', - '-DSeastar_CXX_FLAGS=SHELL:{}'.format(mode_config['lib_cflags']), + '-DSeastar_CXX_FLAGS=SHELL:{}'.format(mode_config['lib_cflags'] + extra_file_prefix_map), '-DSeastar_LD_FLAGS={}'.format(semicolon_separated(mode_config['lib_ldflags'], seastar_cxx_ld_flags)), '-DSeastar_API_LEVEL=7', '-DSeastar_DEPRECATED_OSTREAM_FORMATTERS=OFF', @@ -1927,7 +1950,6 @@ def configure_seastar(build_dir, mode, mode_config): seastar_cmake_args += ['-DBUILD_SHARED_LIBS=ON'] cmake_args = seastar_cmake_args[:] - seastar_build_dir = os.path.join(build_dir, mode, 'seastar') seastar_cmd = ['cmake', '-G', 'Ninja', real_relpath(args.seastar_path, seastar_build_dir)] + cmake_args cmake_dir = seastar_build_dir if dpdk: From 58f8395bc2e64f52e43d24f0861ff8d6b16d7d1a Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 16 Jan 2025 16:35:13 +0200 Subject: [PATCH 321/397] storage_service: use gossiper instead of token_metadata to map ip to id in gossiper notifications We want to drop ips from token_metadata so move to different API to map ip to id. --- service/storage_service.cc | 32 +++++++++++++++++++++----------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index e7374af72078..b71e55d8bd64 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2637,19 +2637,18 @@ future<> storage_service::on_join(gms::inet_address endpoint, gms::endpoint_stat future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_state_ptr state, gms::permit_id pid) { const auto& tm = get_token_metadata(); - const auto tm_host_id_opt = tm.get_host_id_if_known(endpoint); - slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, tm_host_id_opt, pid); + const auto host_id = state->get_host_id(); + slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, host_id, pid); const auto* node = tm.get_topology().find_node(endpoint); if (node && node->is_member()) { co_await notify_up(endpoint); } else if (raft_topology_change_enabled()) { slogger.debug("ignore on_alive since topology changes are using raft and " - "endpoint {}/{} is not a topology member", endpoint, tm_host_id_opt); + "endpoint {}/{} is not a topology member", endpoint, host_id); } else { auto tmlock = co_await get_token_metadata_lock(); auto tmptr = co_await get_mutable_token_metadata_ptr(); const auto dc_rack = get_dc_rack_for(endpoint); - const auto host_id = _gossiper.get_host_id(endpoint); tmptr->update_host_id(host_id, endpoint); tmptr->update_topology(host_id, dc_rack); co_await replicate_to_all_cores(std::move(tmptr)); @@ -2740,15 +2739,26 @@ future<> storage_service::on_remove(gms::inet_address endpoint, gms::permit_id p co_return; } + locator::host_id host_id; + + try { + // It seems gossiper does not check for endpoint existence before calling the callback + // so the lookup may fail, but there is nothing to do in this case. + host_id = _gossiper.get_host_id(endpoint); + } catch (...) { + co_return; + } + + // We should handle the case when the host id is mapped to a different address. + // This could happen when an address for the host id changes and the callback here is called + // with the old ip. We should just skip the remove in that case. + if (_address_map.get(host_id) != endpoint) { + co_return; + } + auto tmlock = co_await get_token_metadata_lock(); auto tmptr = co_await get_mutable_token_metadata_ptr(); - // We should handle the case when we aren't able to find endpoint -> ip mapping in token_metadata. - // This could happen e.g. when the new endpoint has bigger generation in handle_state_normal - the code - // in handle_state_normal will remap host_id to the new IP and we won't find - // old IP here. We should just skip the remove in that case. - if (const auto host_id = tmptr->get_host_id_if_known(endpoint); host_id) { - tmptr->remove_endpoint(*host_id); - } + tmptr->remove_endpoint(host_id); co_await update_topology_change_info(tmptr, ::format("on_remove {}", endpoint)); co_await replicate_to_all_cores(std::move(tmptr)); } From 6e6b2cfa6369dae83dd0546655dee92883bc46a9 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Tue, 10 Dec 2024 18:40:43 +0200 Subject: [PATCH 322/397] storage_service: use existing util function instead of re-iplementing it locator/util.hh already has get_range_to_address_map which is exactly like the one in the storage_service. So remove the later one and use the former instead. --- locator/util.cc | 2 +- locator/util.hh | 5 +++++ service/storage_service.cc | 24 +----------------------- service/storage_service.hh | 11 ----------- 4 files changed, 7 insertions(+), 35 deletions(-) diff --git a/locator/util.cc b/locator/util.cc index fc1cba53942f..d7f8768cdae3 100644 --- a/locator/util.cc +++ b/locator/util.cc @@ -47,7 +47,7 @@ get_all_ranges(const std::vector& sorted_tokens) { } // Caller is responsible to hold token_metadata valid until the returned future is resolved -static future> +future> get_range_to_address_map(locator::effective_replication_map_ptr erm, const std::vector& sorted_tokens) { co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens)); diff --git a/locator/util.hh b/locator/util.hh index e85c04048ea7..3a3360d3d94c 100644 --- a/locator/util.hh +++ b/locator/util.hh @@ -9,6 +9,9 @@ #pragma once #include "dht/token_range_endpoints.hh" +#include "dht/i_partitioner_fwd.hh" +#include "inet_address_vectors.hh" +#include "locator/abstract_replication_strategy.hh" namespace replica { class database; @@ -20,4 +23,6 @@ namespace gms { namespace locator { future> describe_ring(const replica::database& db, const gms::gossiper& gossiper, const sstring& keyspace, bool include_only_local_dc = false); + future> get_range_to_address_map( + locator::effective_replication_map_ptr erm, const std::vector& sorted_tokens); } \ No newline at end of file diff --git a/service/storage_service.cc b/service/storage_service.cc index b71e55d8bd64..b99260d8b478 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2303,14 +2303,7 @@ future<> storage_service::bootstrap(std::unordered_set& bootstrap_tokens, future> storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm) const { - return get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens()); -} - -// Caller is responsible to hold token_metadata valid until the returned future is resolved -future> -storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm, - const std::vector& sorted_tokens) const { - co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens)); + return locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens()); } future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::permit_id pid) { @@ -5282,21 +5275,6 @@ storage_service::describe_ring_for_table(const sstring& keyspace_name, const sst co_return ranges; } -future> -storage_service::construct_range_to_endpoint_map( - locator::effective_replication_map_ptr erm, - const dht::token_range_vector& ranges) const { - std::unordered_map res; - res.reserve(ranges.size()); - for (auto r : ranges) { - res[r] = erm->get_natural_endpoints( - r.end() ? r.end()->value() : dht::maximum_token()); - co_await coroutine::maybe_yield(); - } - co_return res; -} - - std::map storage_service::get_token_to_endpoint_map() { const auto& tm = get_token_metadata(); std::map result; diff --git a/service/storage_service.hh b/service/storage_service.hh index 549f67c2f6a1..40da8c869c0b 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -428,8 +428,6 @@ private: public: future> get_range_to_address_map(locator::effective_replication_map_ptr erm) const; - future> get_range_to_address_map(locator::effective_replication_map_ptr erm, - const std::vector& sorted_tokens) const; /** * The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility @@ -466,15 +464,6 @@ public: */ future> get_tablet_to_endpoint_map(table_id table); - /** - * Construct the range to endpoint mapping based on the true view - * of the world. - * @param ranges - * @return mapping of ranges to the replicas responsible for them. - */ - future> construct_range_to_endpoint_map( - locator::effective_replication_map_ptr erm, - const dht::token_range_vector& ranges) const; public: virtual future<> on_join(gms::inet_address endpoint, gms::endpoint_state_ptr ep_state, gms::permit_id) override; /* From f03a575f3d73a44547443e5b185ec5afa17c6ec7 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 15 Dec 2024 12:46:08 +0200 Subject: [PATCH 323/397] storage_service: move storage_service::get_natural_endpoints to use host ids internally and translate to ips before returning The function is called by RESful API so has to return ips for backwards compatibility, but internally we can use host ids as long as possible and convert to ips just before returning. This also drops usage of ip based erm function which we want to get rid of. --- service/storage_service.cc | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index b99260d8b478..afac37dabfda 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -7342,10 +7342,13 @@ storage_service::get_natural_endpoints(const sstring& keyspace, partition_key pk = partition_key::from_nodetool_style_string(schema, key); dht::token token = schema->get_partitioner().get_token(*schema, pk.view()); const auto& ks = _db.local().find_keyspace(keyspace); + host_id_vector_replica_set replicas; if (ks.uses_tablets()) { - return table.get_effective_replication_map()->get_natural_endpoints(token); + replicas = table.get_effective_replication_map()->get_natural_replicas(token); + } else { + replicas = ks.get_vnode_effective_replication_map()->get_natural_replicas(token); } - return ks.get_vnode_effective_replication_map()->get_natural_endpoints(token); + return replicas | std::views::transform([&] (locator::host_id id) { return _address_map.get(id); }) | std::ranges::to(); } future<> endpoint_lifecycle_notifier::notify_down(gms::inet_address endpoint) { From 9ea53a86562e9a0311f5eab87678e35f0588fb7a Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 15 Dec 2024 13:40:33 +0200 Subject: [PATCH 324/397] storage_service: move describe ring and get_range_to_endpoint_map to use host ids inside and translate to ips at the last moment The functions are called from RESful API so has to return ips for backwards compatibility, but internally we can use host ids as long as possible and convert to ips just before returning. This also drops usage of ip based erm function which we want to get rid of. --- locator/util.cc | 20 ++++++++++---------- locator/util.hh | 2 +- service/storage_service.cc | 6 +++++- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/locator/util.cc b/locator/util.cc index d7f8768cdae3..55c0e0d49118 100644 --- a/locator/util.cc +++ b/locator/util.cc @@ -12,14 +12,14 @@ namespace locator { -static future> +static future> construct_range_to_endpoint_map( locator::effective_replication_map_ptr erm, const dht::token_range_vector& ranges) { - std::unordered_map res; + std::unordered_map res; res.reserve(ranges.size()); for (auto r : ranges) { - res[r] = erm->get_natural_endpoints( + res[r] = erm->get_natural_replicas( r.end() ? r.end()->value() : dht::maximum_token()); co_await coroutine::maybe_yield(); } @@ -47,7 +47,7 @@ get_all_ranges(const std::vector& sorted_tokens) { } // Caller is responsible to hold token_metadata valid until the returned future is resolved -future> +future> get_range_to_address_map(locator::effective_replication_map_ptr erm, const std::vector& sorted_tokens) { co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens)); @@ -67,12 +67,12 @@ get_tokens_in_local_dc(const locator::token_metadata& tm) { co_return filtered_tokens; } -static future> +static future> get_range_to_address_map_in_local_dc( locator::effective_replication_map_ptr erm) { auto tmptr = erm->get_token_metadata_ptr(); auto orig_map = co_await get_range_to_address_map(erm, co_await get_tokens_in_local_dc(*tmptr)); - std::unordered_map filtered_map; + std::unordered_map filtered_map; filtered_map.reserve(orig_map.size()); auto local_dc_filter = tmptr->get_topology().get_local_dc_filter(); for (auto entry : orig_map) { @@ -90,7 +90,7 @@ get_range_to_address_map_in_local_dc( // return get_range_to_address_map(db.find_keyspace(keyspace).get_vnode_effective_replication_map()); // } -static future> +future> get_range_to_address_map(locator::effective_replication_map_ptr erm) { return get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens()); } @@ -100,7 +100,7 @@ describe_ring(const replica::database& db, const gms::gossiper& gossiper, const std::vector ranges; auto erm = db.find_keyspace(keyspace).get_vnode_effective_replication_map(); - std::unordered_map range_to_address_map = co_await ( + std::unordered_map range_to_address_map = co_await ( include_only_local_dc ? get_range_to_address_map_in_local_dc(erm) : get_range_to_address_map(erm) @@ -119,10 +119,10 @@ describe_ring(const replica::database& db, const gms::gossiper& gossiper, const } for (auto endpoint : addresses) { dht::endpoint_details details; - details._host = endpoint; + details._host = gossiper.get_address_map().get(endpoint); details._datacenter = topology.get_datacenter(endpoint); details._rack = topology.get_rack(endpoint); - tr._rpc_endpoints.push_back(gossiper.get_rpc_address(endpoint)); + tr._rpc_endpoints.push_back(gossiper.get_rpc_address(details._host)); tr._endpoints.push_back(fmt::to_string(details._host)); tr._endpoint_details.push_back(details); } diff --git a/locator/util.hh b/locator/util.hh index 3a3360d3d94c..c66e400680a9 100644 --- a/locator/util.hh +++ b/locator/util.hh @@ -23,6 +23,6 @@ namespace gms { namespace locator { future> describe_ring(const replica::database& db, const gms::gossiper& gossiper, const sstring& keyspace, bool include_only_local_dc = false); - future> get_range_to_address_map( + future> get_range_to_address_map( locator::effective_replication_map_ptr erm, const std::vector& sorted_tokens); } \ No newline at end of file diff --git a/service/storage_service.cc b/service/storage_service.cc index afac37dabfda..c19e593c9459 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -13,6 +13,7 @@ #include "compaction/task_manager_module.hh" #include "gc_clock.hh" #include "raft/raft.hh" +#include #include #include "service/qos/raft_service_level_distributed_data_accessor.hh" #include "service/qos/service_level_controller.hh" @@ -2303,7 +2304,10 @@ future<> storage_service::bootstrap(std::unordered_set& bootstrap_tokens, future> storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm) const { - return locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens()); + co_return (co_await locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens())) | + std::views::transform([&] (auto tid) { return std::make_pair(tid.first, + tid.second | std::views::transform([&] (auto id) { return _address_map.get(id); }) | std::ranges::to()); }) | + std::ranges::to(); } future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::permit_id pid) { From 542360e825d9087ef98b0c089bd639e6383f9c07 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Mon, 16 Dec 2024 13:22:20 +0200 Subject: [PATCH 325/397] test: drop inet_address usage from network_topology_strategy_test Move the test to work on host ids. IPs will be dropped eventually. --- test/boost/network_topology_strategy_test.cc | 32 ++++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index 58cbb556d739..a4369457c649 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -50,7 +50,7 @@ struct ring_point { host_id id = host_id::create_random_id(); }; -void print_natural_endpoints(double point, const inet_address_vector_replica_set v) { +void print_natural_endpoints(double point, const host_id_vector_replica_set v) { testlog.debug("Natural endpoints for a token {}:", point); std::string str; std::ostringstream strm(str); @@ -104,11 +104,11 @@ void strategy_sanity_check( void endpoints_check( replication_strategy_ptr ars_ptr, const token_metadata_ptr& tm, - const inet_address_vector_replica_set& endpoints, + const host_id_vector_replica_set& endpoints, const locator::topology& topo, bool strict_dc_rf = false) { - auto&& nodes_per_dc = tm->get_datacenter_token_owners_ips(); + auto&& nodes_per_dc = tm->get_datacenter_token_owners(); const network_topology_strategy* nts_ptr = dynamic_cast(ars_ptr.get()); @@ -123,7 +123,7 @@ void endpoints_check( BOOST_CHECK_LE(total_rf, ars_ptr->get_replication_factor(*tm)); // Check the uniqueness - std::unordered_set ep_set(endpoints.begin(), endpoints.end()); + std::unordered_set ep_set(endpoints.begin(), endpoints.end()); BOOST_CHECK_EQUAL(endpoints.size(), ep_set.size()); // Check the per-DC RF @@ -166,7 +166,7 @@ void full_ring_check(const std::vector& ring_points, for (auto& rp : ring_points) { double cur_point1 = rp.point - 0.5; token t1(tests::d2t(cur_point1 / ring_points.size())); - auto endpoints1 = erm->get_natural_endpoints(t1); + auto endpoints1 = erm->get_natural_replicas(t1); endpoints_check(ars_ptr, tmptr, endpoints1, topo); @@ -179,7 +179,7 @@ void full_ring_check(const std::vector& ring_points, // double cur_point2 = rp.point - 0.2; token t2(tests::d2t(cur_point2 / ring_points.size())); - auto endpoints2 = erm->get_natural_endpoints(t2); + auto endpoints2 = erm->get_natural_replicas(t2); endpoints_check(ars_ptr, tmptr, endpoints2, topo); check_ranges_are_sorted(erm, rp.id).get(); @@ -193,17 +193,17 @@ void full_ring_check(const tablet_map& tmap, auto& tm = *tmptr; const auto& topo = tm.get_topology(); - auto to_endpoint_set = [&] (const tablet_replica_set& replicas) { - inet_address_vector_replica_set result; + auto to_replica_set = [&] (const tablet_replica_set& replicas) { + host_id_vector_replica_set result; result.reserve(replicas.size()); for (auto&& replica : replicas) { - result.emplace_back(tm.get_endpoint_for_host_id(replica.host)); + result.emplace_back(replica.host); } return result; }; for (tablet_id tb : tmap.tablet_ids()) { - endpoints_check(rs_ptr, tmptr, to_endpoint_set(tmap.get_tablet_info(tb).replicas), topo, true); + endpoints_check(rs_ptr, tmptr, to_replica_set(tmap.get_tablet_info(tb).replicas), topo, true); } } @@ -672,7 +672,7 @@ static size_t get_replication_factor(const sstring& dc, static bool has_sufficient_replicas(const sstring& dc, const std::unordered_map>& dc_replicas, - const std::unordered_map>& all_endpoints, + const std::unordered_map>& all_endpoints, const std::unordered_map& datacenters) noexcept { auto dc_replicas_it = dc_replicas.find(dc); if (dc_replicas_it == dc_replicas.end()) { @@ -690,7 +690,7 @@ static bool has_sufficient_replicas(const sstring& dc, static bool has_sufficient_replicas( const std::unordered_map>& dc_replicas, - const std::unordered_map>& all_endpoints, + const std::unordered_map>& all_endpoints, const std::unordered_map& datacenters) noexcept { for (auto& dc : datacenters | std::views::keys) { @@ -741,16 +741,16 @@ static locator::host_id_set calculate_natural_endpoints( // the token-owning members of a DC // const std::unordered_map> - all_endpoints = tm.get_datacenter_token_owners_ips(); + std::unordered_set> + all_endpoints = tm.get_datacenter_token_owners(); // // all racks (with non-token owners filtered out) in a DC so we can check // when we have exhausted all racks in a DC // const std::unordered_map>> - racks = tm.get_datacenter_racks_token_owners_ips(); + std::unordered_set>> + racks = tm.get_datacenter_racks_token_owners(); // not aware of any cluster members SCYLLA_ASSERT(!all_endpoints.empty() && !racks.empty()); From 5262bbafffe575000c2ab5f09433500d0cb9923f Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 15 Dec 2024 13:56:12 +0200 Subject: [PATCH 326/397] locator: drop no longer used ip based functions --- locator/abstract_replication_strategy.cc | 23 ----------------------- locator/abstract_replication_strategy.hh | 5 ----- locator/tablets.cc | 4 ---- 3 files changed, 32 deletions(-) diff --git a/locator/abstract_replication_strategy.cc b/locator/abstract_replication_strategy.cc index 468426e8ffa1..c48a0a80acd5 100644 --- a/locator/abstract_replication_strategy.cc +++ b/locator/abstract_replication_strategy.cc @@ -56,19 +56,6 @@ auto& ring_mapping::operator*() const { return std::as_const(_impl->map); } -template -static ResultSet resolve_endpoints(const SourceSet& host_ids, const token_metadata& tm) { - ResultSet result{}; - result.reserve(host_ids.size()); - for (const auto& host_id: host_ids) { - // Empty host_id is used as a marker for local address. - // The reason for this hack is that we need local_strategy to - // work before the local host_id is loaded from the system.local table. - result.push_back(host_id ? tm.get_endpoint_for_host_id(host_id) : tm.get_topology().my_address()); - } - return result; -} - logging::logger rslogger("replication_strategy"); abstract_replication_strategy::abstract_replication_strategy( @@ -507,20 +494,10 @@ host_id_vector_replica_set vnode_effective_replication_map::do_get_replicas(cons return it->second; } -inet_address_vector_replica_set vnode_effective_replication_map::do_get_natural_endpoints(const token& tok, - bool is_vnode) const -{ - return resolve_endpoints(do_get_replicas(tok, is_vnode), *_tmptr); -} - host_id_vector_replica_set vnode_effective_replication_map::get_replicas(const token& tok) const { return do_get_replicas(tok, false); } -inet_address_vector_replica_set vnode_effective_replication_map::get_natural_endpoints(const token& search_token) const { - return do_get_natural_endpoints(search_token, false); -} - host_id_vector_replica_set vnode_effective_replication_map::get_natural_replicas(const token& search_token) const { return get_replicas(search_token); } diff --git a/locator/abstract_replication_strategy.hh b/locator/abstract_replication_strategy.hh index 34ed725f2ce2..df1316819574 100644 --- a/locator/abstract_replication_strategy.hh +++ b/locator/abstract_replication_strategy.hh @@ -231,9 +231,6 @@ public: /// new replica. /// /// The returned addresses are present in the topology object associated with this instance. - virtual inet_address_vector_replica_set get_natural_endpoints(const token& search_token) const = 0; - - /// Same as above but returns host ids instead of addresses virtual host_id_vector_replica_set get_natural_replicas(const token& search_token) const = 0; /// Same as above but returns host ids instead of addresses @@ -333,7 +330,6 @@ private: friend class abstract_replication_strategy; friend class effective_replication_map_factory; public: // effective_replication_map - inet_address_vector_replica_set get_natural_endpoints(const token& search_token) const override; host_id_vector_replica_set get_natural_replicas(const token& search_token) const override; host_id_vector_topology_change get_pending_replicas(const token& search_token) const override; host_id_vector_replica_set get_replicas_for_reading(const token& token) const override; @@ -397,7 +393,6 @@ public: private: future do_get_ranges(noncopyable_function consider_range_for_endpoint) const; - inet_address_vector_replica_set do_get_natural_endpoints(const token& tok, bool is_vnode) const; host_id_vector_replica_set do_get_replicas(const token& tok, bool is_vnode) const; stop_iteration for_each_natural_endpoint_until(const token& vnode_tok, const noncopyable_function& func) const; diff --git a/locator/tablets.cc b/locator/tablets.cc index cafe0fe22aa8..935a171f88fd 100644 --- a/locator/tablets.cc +++ b/locator/tablets.cc @@ -879,10 +879,6 @@ class tablet_effective_replication_map : public effective_replication_map { return to_host_set(get_replicas_for_write(search_token)); } - virtual inet_address_vector_replica_set get_natural_endpoints(const token& search_token) const override { - return to_replica_set(get_replicas_for_write(search_token)); - } - virtual host_id_vector_replica_set get_natural_replicas(const token& search_token) const override { return to_host_set(get_replicas_for_write(search_token)); } From 8cc09f435806f67023bc59af7e8113135f4707d4 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 15 Dec 2024 18:31:37 +0200 Subject: [PATCH 327/397] storage_service: do not use ip addresses from token_metadata in handling of a normal state Instead use gossiper and peers table to retrieve same information. Token_metadata is created from the mix of those two anyway. The goal is to drop ips from token_metadata entirely. --- service/storage_service.cc | 28 ++++++++++++++++++++++------ 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index c19e593c9459..ad9cc6ddee60 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2370,11 +2370,26 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit if (tmptr->is_normal_token_owner(host_id)) { slogger.info("handle_state_normal: node {}/{} was already a normal token owner", endpoint, host_id); } - auto existing = tmptr->get_endpoint_for_host_id_if_known(host_id); // Old node in replace-with-same-IP scenario. std::optional replaced_id; + auto ips = _gossiper.get_nodes_with_host_id(host_id); + + std::optional existing; + + if (tmptr->get_topology().find_node(host_id)) { + // If node is not in the topology there is no existsing address + // If there are two addresses for the same id the "other" one is existing + // If there is only one it is existing + if (ips.size() == 2) { + if (ips.erase(endpoint) == 0) { + on_internal_error(slogger, fmt::format("Gossiper has two ips {} for host id {} but none of them is {}", ips, endpoint, host_id)); + } + } + existing = *ips.begin(); + } + if (existing && *existing != endpoint) { // This branch in taken when a node changes its IP address. @@ -2420,11 +2435,12 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit // We do this after update_normal_tokens, allowing for tokens to be properly // migrated to the new host_id. - slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint); - if (const auto old_host_id = tmptr->get_host_id_if_known(endpoint); old_host_id && *old_host_id != host_id) { - // Replace with same IP scenario - slogger.info("The IP {} previously owned host ID {}", endpoint, *old_host_id); - replaced_id = *old_host_id; + auto peers = co_await _sys_ks.local().load_host_ids(); + if (peers.contains(endpoint) && peers[endpoint] != host_id) { + replaced_id = peers[endpoint]; + slogger.info("The IP {} previously owned host ID {}", endpoint, *replaced_id); + } else { + slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint); } } else { // This branch is taken if this node wasn't involved in node_ops From 7a3237c6879b5eba391b8d2cd8a3c39a10209cb0 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 18 Dec 2024 17:05:40 +0200 Subject: [PATCH 328/397] messaging_service: drop get_raw_version and knows_version The are unused. The version is always fixed. --- api/messaging_service.cc | 2 +- message/messaging_service.cc | 10 ---------- message/messaging_service.hh | 4 ---- service/migration_manager.cc | 4 +--- 4 files changed, 2 insertions(+), 18 deletions(-) diff --git a/api/messaging_service.cc b/api/messaging_service.cc index b1665c8c5c20..3fc38b127c01 100644 --- a/api/messaging_service.cc +++ b/api/messaging_service.cc @@ -114,7 +114,7 @@ void set_messaging_service(http_context& ctx, routes& r, sharded req) { diff --git a/message/messaging_service.cc b/message/messaging_service.cc index eb7b814d8fcc..22b6e35eabdd 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -257,16 +257,6 @@ const uint64_t* messaging_service::get_dropped_messages() const { return _dropped_messages; } -int32_t messaging_service::get_raw_version(const gms::inet_address& endpoint) const { - // FIXME: messaging service versioning - return current_version; -} - -bool messaging_service::knows_version(const gms::inet_address& endpoint) const { - // FIXME: messaging service versioning - return true; -} - future<> messaging_service::unregister_handler(messaging_verb verb) { return _rpc->unregister_handler(verb); } diff --git a/message/messaging_service.hh b/message/messaging_service.hh index eb5c52613ca6..873aa83c903a 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -272,10 +272,6 @@ public: const uint64_t* get_dropped_messages() const; - int32_t get_raw_version(const gms::inet_address& endpoint) const; - - bool knows_version(const gms::inet_address& endpoint) const; - enum class encrypt_what { none, rack, diff --git a/service/migration_manager.cc b/service/migration_manager.cc index 0a1acd0f12f8..6f4d6d5d6cb2 100644 --- a/service/migration_manager.cc +++ b/service/migration_manager.cc @@ -934,9 +934,7 @@ future<> migration_manager::announce_without_raft(std::vector schema, auto all_live = _gossiper.get_live_members(); auto live_members = all_live | std::views::filter([this, my_address = _messaging.broadcast_address()] (const gms::inet_address& endpoint) { // only push schema to nodes with known and equal versions - return endpoint != my_address && - _messaging.knows_version(endpoint) && - _messaging.get_raw_version(endpoint) == netw::messaging_service::current_version; + return endpoint != my_address; }); // FIXME: gossiper should return host id set auto live_host_ids = live_members | std::views::transform([&] (const gms::inet_address& ip) { return _gossiper.get_host_id(ip); }); From 36ccc897e809d867325452c7372d72d018c0a618 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 18 Dec 2024 17:06:21 +0200 Subject: [PATCH 329/397] gossiper: change get_live_members and all its users to work on host ids --- gms/gossiper.cc | 8 ++++++-- gms/gossiper.hh | 3 ++- service/migration_manager.cc | 6 ++---- service/storage_service.cc | 10 +++++----- 4 files changed, 15 insertions(+), 12 deletions(-) diff --git a/gms/gossiper.cc b/gms/gossiper.cc index c303230316e6..a8ec00dc8160 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -924,7 +924,7 @@ future> gossiper::get_live_members_synchronized() { return container().invoke_on(0, [] (gms::gossiper& g) -> future> { // Make sure the value we return is synchronized on all shards auto lock = co_await g.lock_endpoint_update_semaphore(); - co_return g.get_live_members(); + co_return g.get_live_members_helper(); }); } @@ -1175,7 +1175,7 @@ future<> gossiper::unregister_(shared_ptr su return _subscribers.remove(subscriber); } -std::set gossiper::get_live_members() const { +std::set gossiper::get_live_members_helper() const { std::set live_members(_live_endpoints.begin(), _live_endpoints.end()); auto myip = get_broadcast_address(); logger.debug("live_members before={}", live_members); @@ -1186,6 +1186,10 @@ std::set gossiper::get_live_members() const { return live_members; } +std::set gossiper::get_live_members() const { + return get_live_members_helper() | std::views::transform([this] (inet_address ip) { return get_host_id(ip); }) | std::ranges::to(); +} + std::set gossiper::get_live_token_owners() const { std::set token_owners; auto normal_token_owners = get_token_metadata_ptr()->get_normal_token_owners(); diff --git a/gms/gossiper.hh b/gms/gossiper.hh index 595f7103ca68..1b647370ff3e 100644 --- a/gms/gossiper.hh +++ b/gms/gossiper.hh @@ -296,7 +296,7 @@ public: */ future<> unregister_(shared_ptr subscriber); - std::set get_live_members() const; + std::set get_live_members() const; std::set get_live_token_owners() const; @@ -528,6 +528,7 @@ public: future<> wait_alive(std::vector nodes, std::chrono::milliseconds timeout); future<> wait_alive(std::vector nodes, std::chrono::milliseconds timeout); future<> wait_alive(noncopyable_function()> get_nodes, std::chrono::milliseconds timeout); + std::set get_live_members_helper() const; // Wait for `n` live nodes to show up in gossip (including ourself). future<> wait_for_live_nodes_to_show_up(size_t n); diff --git a/service/migration_manager.cc b/service/migration_manager.cc index 6f4d6d5d6cb2..ef3ffa98cf4e 100644 --- a/service/migration_manager.cc +++ b/service/migration_manager.cc @@ -932,13 +932,11 @@ future<> migration_manager::announce_without_raft(std::vector schema, try { using namespace std::placeholders; auto all_live = _gossiper.get_live_members(); - auto live_members = all_live | std::views::filter([this, my_address = _messaging.broadcast_address()] (const gms::inet_address& endpoint) { + auto live_members = all_live | std::views::filter([my_address = _gossiper.my_host_id()] (const locator::host_id& endpoint) { // only push schema to nodes with known and equal versions return endpoint != my_address; }); - // FIXME: gossiper should return host id set - auto live_host_ids = live_members | std::views::transform([&] (const gms::inet_address& ip) { return _gossiper.get_host_id(ip); }); - co_await coroutine::parallel_for_each(live_host_ids, + co_await coroutine::parallel_for_each(live_members, std::bind(std::mem_fn(&migration_manager::push_schema_mutation), this, std::placeholders::_1, schema)); } catch (...) { mlogger.error("failed to announce migration to all nodes: {}", std::current_exception()); diff --git a/service/storage_service.cc b/service/storage_service.cc index ad9cc6ddee60..8a446813d3a0 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -3609,17 +3609,17 @@ future>> storage_service::descr std::unordered_map> results; netw::messaging_service& ms = _messaging.local(); return map_reduce(std::move(live_hosts), [&ms, as = abort_source()] (auto host) mutable { - auto f0 = ser::migration_manager_rpc_verbs::send_schema_check(&ms, netw::msg_addr{ host, 0 }, as); + auto f0 = ser::migration_manager_rpc_verbs::send_schema_check(&ms, host, as); return std::move(f0).then_wrapped([host] (auto f) { if (f.failed()) { f.ignore_ready_future(); - return std::pair>(host, std::nullopt); + return std::pair>(host, std::nullopt); } - return std::pair>(host, f.get()); + return std::pair>(host, f.get()); }); - }, std::move(results), [] (auto results, auto host_and_version) { + }, std::move(results), [this] (auto results, auto host_and_version) { auto version = host_and_version.second ? host_and_version.second->to_sstring() : UNREACHABLE; - results.try_emplace(version).first->second.emplace_back(fmt::to_string(host_and_version.first)); + results.try_emplace(version).first->second.emplace_back(fmt::to_string(_address_map.get(host_and_version.first))); return results; }).then([this] (auto results) { // we're done: the results map is ready to return to the client. the rest is just debug logging: From da9b7b26262340875e67d533e3cd3e3417ea44c4 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 15 Jan 2025 11:55:26 +0200 Subject: [PATCH 330/397] storage_service: drop ip based topology::get_datacenter() usage We want to drop ips from the topology eventually. --- service/storage_service.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 8a446813d3a0..74c30d6d5c15 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2735,7 +2735,7 @@ future<> storage_service::maybe_reconnect_to_preferred_ip(inet_address ep, inet_ } const auto& topo = get_token_metadata().get_topology(); - if (topo.get_datacenter() == topo.get_datacenter(ep) && _messaging.local().get_preferred_ip(ep) != local_ip) { + if (topo.get_datacenter() == topo.get_datacenter(_gossiper.get_host_id(ep)) && _messaging.local().get_preferred_ip(ep) != local_ip) { slogger.debug("Initiated reconnect to an Internal IP {} for the {}", local_ip, ep); co_await _messaging.invoke_on_all([ep, local_ip] (auto& local_ms) { local_ms.cache_preferred_ip(ep, local_ip); From 1b6e1456e59ba75d499f45dddf31dc0fc1f5ee60 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 18 Dec 2024 18:07:49 +0200 Subject: [PATCH 331/397] messaging_service: drop the usage of ip based token_metadata APIs We want to drop ips from token_metadata so move to use host id based counterparts. Messaging service gets a function that maps from ips to id when is starts listening. --- main.cc | 13 ++++++++++++- message/messaging_service.cc | 21 +++++++++++++-------- message/messaging_service.hh | 6 ++++-- test/lib/cql_test_env.cc | 2 +- test/manual/message.cc | 2 +- 5 files changed, 31 insertions(+), 13 deletions(-) diff --git a/main.cc b/main.cc index 8ccb6d846c25..700cb6d83fb6 100644 --- a/main.cc +++ b/main.cc @@ -2118,7 +2118,18 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl group0_service.setup_group0_if_exist(sys_ks.local(), ss.local(), qp.local(), mm.local()).get(); with_scheduling_group(maintenance_scheduling_group, [&] { - return messaging.invoke_on_all(&netw::messaging_service::start_listen, std::ref(token_metadata)); + return messaging.invoke_on_all([&] (auto& ms) { + return ms.start_listen(token_metadata.local(), [&gossiper] (gms::inet_address ip) { + if (ip == gossiper.local().get_broadcast_address()) { + return gossiper.local().my_host_id(); + } + try { + return gossiper.local().get_host_id(ip); + } catch (...) { + return locator::host_id{}; + } + }); + }); }).get(); const auto generation_number = gms::generation_type(sys_ks.local().increment_and_get_generation().get()); diff --git a/message/messaging_service.cc b/message/messaging_service.cc index 22b6e35eabdd..8dc7ce6e9e0d 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -299,8 +299,9 @@ future<> messaging_service::start() { return make_ready_future<>(); } -future<> messaging_service::start_listen(locator::shared_token_metadata& stm) { +future<> messaging_service::start_listen(locator::shared_token_metadata& stm, std::function address_to_host_id_mapper) { _token_metadata = &stm; + _address_to_host_id_mapper = std::move(address_to_host_id_mapper); do_start_listen(); return make_ready_future<>(); } @@ -308,20 +309,21 @@ future<> messaging_service::start_listen(locator::shared_token_metadata& stm) { bool messaging_service::topology_known_for(inet_address addr) const { // The token metadata pointer is nullptr before // the service is start_listen()-ed and after it's being shutdown()-ed. + const locator::node* node; return _token_metadata - && _token_metadata->get()->get_topology().has_endpoint(addr); + && (node = _token_metadata->get()->get_topology().find_node(_address_to_host_id_mapper(addr))) && !node->is_none(); } // Precondition: `topology_known_for(addr)`. bool messaging_service::is_same_dc(inet_address addr) const { const auto& topo = _token_metadata->get()->get_topology(); - return topo.get_datacenter(addr) == topo.get_datacenter(); + return topo.get_datacenter(_address_to_host_id_mapper(addr)) == topo.get_datacenter(); } // Precondition: `topology_known_for(addr)`. bool messaging_service::is_same_rack(inet_address addr) const { const auto& topo = _token_metadata->get()->get_topology(); - return topo.get_rack(addr) == topo.get_rack(); + return topo.get_rack(_address_to_host_id_mapper(addr)) == topo.get_rack(); } // The socket metrics domain defines the way RPC metrics are grouped @@ -334,12 +336,15 @@ bool messaging_service::is_same_rack(inet_address addr) const { // that the isolation cookie suits very well, because these cookies // are different for different indices and are more informative than // plain numbers -sstring messaging_service::client_metrics_domain(unsigned idx, inet_address addr) const { +sstring messaging_service::client_metrics_domain(unsigned idx, inet_address addr, std::optional id) const { sstring ret = _scheduling_info_for_connection_index[idx].isolation_cookie; + if (!id) { + id = _address_to_host_id_mapper(addr); + } if (_token_metadata) { const auto& topo = _token_metadata->get()->get_topology(); - if (topo.has_endpoint(addr)) { - ret += ":" + topo.get_datacenter(addr); + if (topo.has_node(*id)) { + ret += ":" + topo.get_datacenter(*id); } } return ret; @@ -1106,7 +1111,7 @@ shared_ptr messaging_service::ge opts.tcp_nodelay = must_tcp_nodelay; opts.reuseaddr = true; opts.isolation_cookie = _scheduling_info_for_connection_index[idx].isolation_cookie; - opts.metrics_domain = client_metrics_domain(idx, id.addr); // not just `addr` as the latter may be internal IP + opts.metrics_domain = client_metrics_domain(idx, id.addr, host_id); // not just `addr` as the latter may be internal IP SCYLLA_ASSERT(!must_encrypt || _credentials); diff --git a/message/messaging_service.hh b/message/messaging_service.hh index 873aa83c903a..083caee6eab3 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -337,6 +337,8 @@ private: private: config _cfg; locator::shared_token_metadata* _token_metadata = nullptr; + // a function that maps from ip to host id if known (returns default constructable host_id if there is no mapping) + std::function _address_to_host_id_mapper; // map: Node broadcast address -> Node internal IP, and the reversed mapping, for communication within the same data center std::unordered_map _preferred_ip_cache, _preferred_to_endpoint; std::unique_ptr _rpc; @@ -378,7 +380,7 @@ public: ~messaging_service(); future<> start(); - future<> start_listen(locator::shared_token_metadata& stm); + future<> start_listen(locator::shared_token_metadata& stm, std::function address_to_host_id_mapper); uint16_t port() const noexcept { return _cfg.port; } @@ -455,7 +457,7 @@ private: bool is_host_banned(locator::host_id); - sstring client_metrics_domain(unsigned idx, inet_address addr) const; + sstring client_metrics_domain(unsigned idx, inet_address addr, std::optional id) const; public: // Return rpc::protocol::client for a shard which is a ip + cpuid pair. diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index eae4ffc4556b..82f597134087 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -755,7 +755,7 @@ class single_node_cql_env : public cql_test_env { // Once the seastar issue is fixed, we can just keep the tmp socket aliva across // the listen invoke below. tmp = {}; - _ms.invoke_on_all(&netw::messaging_service::start_listen, std::ref(_token_metadata)).get(); + _ms.invoke_on_all(&netw::messaging_service::start_listen, std::ref(_token_metadata), [host_id] (gms::inet_address ip) {return host_id; }).get(); } } catch (std::system_error& e) { // if we still hit a used port (quick other process), just shut down ms and try again. diff --git a/test/manual/message.cc b/test/manual/message.cc index 7e153edd3e09..d32a9891a8dc 100644 --- a/test/manual/message.cc +++ b/test/manual/message.cc @@ -224,7 +224,7 @@ int main(int ac, char ** av) { auto deinit_testers = deferred_action([&testers] { testers.invoke_on_all(&tester::deinit_handler).get(); }); - messaging.invoke_on_all(&netw::messaging_service::start_listen, std::ref(token_metadata)).get(); + messaging.invoke_on_all(&netw::messaging_service::start_listen, std::ref(token_metadata), [] (gms::inet_address ip){ return locator::host_id{}; }).get(); if (config.contains("server")) { auto ip = config["server"].as(); auto cpuid = config["cpuid"].as(); From 315db647dd82b27809091fc0a6587ccff8265865 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 19 Dec 2024 13:03:38 +0200 Subject: [PATCH 332/397] consistency_level: drop templates since the same types of ranges are used by all the callers --- db/consistency_level.cc | 19 ++++++------------- db/consistency_level.hh | 10 ++-------- 2 files changed, 8 insertions(+), 21 deletions(-) diff --git a/db/consistency_level.cc b/db/consistency_level.cc index 1ecefa837eb2..7bdaaaa046eb 100644 --- a/db/consistency_level.cc +++ b/db/consistency_level.cc @@ -113,11 +113,10 @@ bool is_datacenter_local(consistency_level l) { return l == consistency_level::LOCAL_ONE || l == consistency_level::LOCAL_QUORUM; } -template > std::unordered_map count_per_dc_endpoints( const locator::effective_replication_map& erm, - const Range& live_endpoints, - const PendingRange& pending_endpoints = std::array()) { + const host_id_vector_replica_set& live_endpoints, + const host_id_vector_topology_change& pending_endpoints = host_id_vector_topology_change()) { using namespace locator; auto& rs = erm.get_replication_strategy(); @@ -148,12 +147,11 @@ std::unordered_map count_per_dc_endpoints( return dc_endpoints; } -template bool assure_sufficient_live_nodes_each_quorum( consistency_level cl, const locator::effective_replication_map& erm, - const Range& live_endpoints, - const PendingRange& pending_endpoints) { + const host_id_vector_replica_set& live_endpoints, + const host_id_vector_topology_change& pending_endpoints) { using namespace locator; auto& rs = erm.get_replication_strategy(); @@ -175,12 +173,11 @@ bool assure_sufficient_live_nodes_each_quorum( return false; } -template void assure_sufficient_live_nodes( consistency_level cl, const locator::effective_replication_map& erm, - const Range& live_endpoints, - const PendingRange& pending_endpoints) { + const host_id_vector_replica_set& live_endpoints, + const host_id_vector_topology_change& pending_endpoints) { size_t need = block_for(erm, cl); auto adjust_live_for_error = [] (size_t live, size_t pending) { @@ -228,10 +225,6 @@ void assure_sufficient_live_nodes( } } -template void assure_sufficient_live_nodes(consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const std::array&); -template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const utils::small_vector&); -template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const host_id_vector_replica_set&, const host_id_vector_topology_change&); - host_id_vector_replica_set filter_for_query(consistency_level cl, const locator::effective_replication_map& erm, diff --git a/db/consistency_level.hh b/db/consistency_level.hh index 787702b1273a..f9619a97c4e9 100644 --- a/db/consistency_level.hh +++ b/db/consistency_level.hh @@ -61,15 +61,9 @@ is_sufficient_live_nodes(consistency_level cl, const locator::effective_replication_map& erm, const host_id_vector_replica_set& live_endpoints); -template> void assure_sufficient_live_nodes( consistency_level cl, const locator::effective_replication_map& erm, - const Range& live_endpoints, - const PendingRange& pending_endpoints = std::array()); - -extern template void assure_sufficient_live_nodes(consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const std::array&); -extern template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const utils::small_vector&); -extern template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const host_id_vector_replica_set&, const host_id_vector_topology_change&); - + const host_id_vector_replica_set& live_endpoints, + const host_id_vector_topology_change& pending_endpoints = host_id_vector_topology_change()); } From 25eb98ecbcd763f76035152c3acda55dc82cb7a8 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 19 Dec 2024 14:46:19 +0200 Subject: [PATCH 333/397] locator: topology: drop no longer used ip based overloads --- locator/topology.hh | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/locator/topology.hh b/locator/topology.hh index 95b5c5eeed65..2dec983987ef 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -323,11 +323,6 @@ public: const sstring& get_datacenter(host_id id) const { return get_location(id).dc; } - // Get datacenter of a node identified by endpoint - // The specified node must exist. - const sstring& get_datacenter(inet_address ep) const { - return get_location(ep).dc; - } // Get rack of this node const sstring& get_rack() const noexcept { @@ -338,11 +333,6 @@ public: const sstring& get_rack(host_id id) const { return get_location(id).rack; } - // Get rack of a node identified by endpoint - // The specified node must exist. - const sstring& get_rack(inet_address ep) const { - return get_location(ep).rack; - } auto get_local_dc_filter() const noexcept { return [ this, local_dc = get_datacenter() ] (auto ep) { From 8433947932cd1397730e0d0ef09168c8b9f97ce1 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 19 Dec 2024 14:52:49 +0200 Subject: [PATCH 334/397] locator: topology: remove get_location overload that works on ip and its last users --- db/virtual_tables.cc | 7 +++++-- locator/topology.cc | 18 ------------------ locator/topology.hh | 3 --- test/boost/locator_topology_test.cc | 12 +++--------- 4 files changed, 8 insertions(+), 32 deletions(-) diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc index b9b3e0d5f0c2..95cb7523da2c 100644 --- a/db/virtual_tables.cc +++ b/db/virtual_tables.cc @@ -92,8 +92,11 @@ class cluster_status_table : public memtable_filling_virtual_table { } set_cell(cr, "host_id", hostid.uuid()); - sstring dc = tm.get_topology().get_location(endpoint).dc; - set_cell(cr, "dc", dc); + if (tm.get_topology().has_node(hostid)) { + // Not all entries in gossiper are present in the topology + sstring dc = tm.get_topology().get_location(hostid).dc; + set_cell(cr, "dc", dc); + } if (ownership.contains(endpoint)) { set_cell(cr, "owns", ownership[endpoint]); diff --git a/locator/topology.cc b/locator/topology.cc index 179b1ae87fab..12f3ae6c23b1 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -555,24 +555,6 @@ bool topology::has_endpoint(inet_address ep) const return has_node(ep); } -const endpoint_dc_rack& topology::get_location(const inet_address& ep) const { - if (auto node = find_node(ep)) { - return node->dc_rack(); - } - // We should do the following check after lookup in nodes. - // In tests, there may be no config for local node, so fall back to get_location() - // only if no mapping is found. Otherwise, get_location() will return empty location - // from config or random node, neither of which is correct. - if (ep == _cfg.this_endpoint) { - return get_location(); - } - // FIXME -- this shouldn't happen. After topology is stable and is - // correctly populated with endpoints, this should be replaced with - // on_internal_error() - tlogger.warn("Requested location for node {} not in topology. backtrace {}", ep, lazy_backtrace()); - return endpoint_dc_rack::default_location; -} - void topology::sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const { if (can_sort_by_proximity()) { do_sort_by_proximity(address, addresses); diff --git a/locator/topology.hh b/locator/topology.hh index 2dec983987ef..6dfe80869b77 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -310,9 +310,6 @@ public: const endpoint_dc_rack& get_location(host_id id) const { return find_node(id)->dc_rack(); } - // Get dc/rack location of a node identified by endpoint - // The specified node must exist. - const endpoint_dc_rack& get_location(const inet_address& ep) const; // Get datacenter of this node const sstring& get_datacenter() const noexcept { diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc index c41ded826c89..99e815e84717 100644 --- a/test/boost/locator_topology_test.cc +++ b/test/boost/locator_topology_test.cc @@ -122,7 +122,7 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) { auto node = const_cast(topo.this_node()); topo.update_node(*node, std::nullopt, ep1, std::nullopt, std::nullopt); - + BOOST_REQUIRE_EQUAL(topo.find_node(id1), node); BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt, std::nullopt), std::runtime_error); @@ -138,31 +138,27 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) { topo.update_node(*node, std::nullopt, std::nullopt, dc_rack1, std::nullopt); BOOST_REQUIRE(topo.get_location(id1) == dc_rack1); - BOOST_REQUIRE(topo.get_location(ep1) == dc_rack1); auto dc_rack2 = endpoint_dc_rack{"DC2", "RACK2"}; topo.update_node(*node, std::nullopt, std::nullopt, dc_rack2, std::nullopt); BOOST_REQUIRE(topo.get_location(id1) == dc_rack2); - BOOST_REQUIRE(topo.get_location(ep1) == dc_rack2); BOOST_REQUIRE_NE(node->get_state(), locator::node::state::being_decommissioned); topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::being_decommissioned); - + BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned); auto dc_rack3 = endpoint_dc_rack{"DC3", "RACK3"}; // Note: engage state option, but keep node::state value the same // to reproduce #13502 topo.update_node(*node, std::nullopt, ep3, dc_rack3, locator::node::state::being_decommissioned); - + BOOST_REQUIRE_EQUAL(topo.find_node(id1), node); BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr); BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr); BOOST_REQUIRE_EQUAL(topo.find_node(ep3), node); BOOST_REQUIRE(topo.get_location(id1) == dc_rack3); - BOOST_REQUIRE(topo.get_location(ep2) == endpoint_dc_rack::default_location); - BOOST_REQUIRE(topo.get_location(ep3) == dc_rack3); BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned); // In state::left the node will remain indexed only by its host_id @@ -172,8 +168,6 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) { BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr); BOOST_REQUIRE_EQUAL(topo.find_node(ep3), nullptr); BOOST_REQUIRE(topo.get_location(id1) == dc_rack3); - BOOST_REQUIRE(topo.get_location(ep2) == endpoint_dc_rack::default_location); - BOOST_REQUIRE(topo.get_location(ep3) == endpoint_dc_rack::default_location); BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::left); } From 2ea8df2cf5b948bee21fe0025c0ae77f4e9b180c Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 19 Dec 2024 15:31:30 +0200 Subject: [PATCH 335/397] storage_proxy: drop is_alive that works on ip since it is not used any more --- service/storage_proxy.cc | 22 +++++++--------------- service/storage_proxy.hh | 3 +-- 2 files changed, 8 insertions(+), 17 deletions(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 86a7958bccbf..ecb059bef999 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -186,10 +186,6 @@ locator::host_id storage_proxy::my_host_id(const locator::effective_replication_ return erm.get_topology().my_host_id(); } -bool storage_proxy::is_me(gms::inet_address addr) const noexcept { - return local_db().get_token_metadata().get_topology().is_me(addr); -} - bool storage_proxy::is_me(const locator::effective_replication_map& erm, locator::host_id id) const noexcept { return erm.get_topology().is_me(id); } @@ -3392,7 +3388,7 @@ storage_proxy::create_write_response_handler_helper(schema_ptr s, const dht::tok live_endpoints.reserve(all.size()); dead_endpoints.reserve(all.size()); std::partition_copy(all.begin(), all.end(), std::back_inserter(live_endpoints), - std::back_inserter(dead_endpoints), std::bind_front(&storage_proxy::is_alive_id, this, std::cref(*erm))); + std::back_inserter(dead_endpoints), std::bind_front(&storage_proxy::is_alive, this, std::cref(*erm))); db::per_partition_rate_limit::info rate_limit_info; if (allow_limit && _db.local().can_apply_per_partition_rate_limit(*s, db::operation_type::write)) { @@ -3769,7 +3765,7 @@ storage_proxy::get_paxos_participants(const sstring& ks_name, const locator::eff auto all_as_spans = std::array{std::span(natural_endpoints), std::span(pending_endpoints)}; std::ranges::copy(all_as_spans | std::views::join | - std::views::filter(std::bind_front(&storage_proxy::is_alive_id, this, std::cref(erm))), std::back_inserter(live_endpoints)); + std::views::filter(std::bind_front(&storage_proxy::is_alive, this, std::cref(erm))), std::back_inserter(live_endpoints)); if (live_endpoints.size() < required_participants) { throw exceptions::unavailable_exception(cl_for_paxos, required_participants, live_endpoints.size()); @@ -4019,7 +4015,7 @@ storage_proxy::mutate_atomically_result(std::vector mutations, db::con std::ranges::to>()); } auto local_rack = topology.get_rack(); - auto chosen_endpoints = endpoint_filter(std::bind_front(&storage_proxy::is_alive_id, &_p, std::cref(*_ermp)), local_addr, + auto chosen_endpoints = endpoint_filter(std::bind_front(&storage_proxy::is_alive, &_p, std::cref(*_ermp)), local_addr, local_rack, local_token_owners); if (chosen_endpoints.empty()) { @@ -4173,7 +4169,7 @@ future<> storage_proxy::send_to_endpoint( std::array{std::span(pending_endpoints), std::span(target.begin(), target.end())} | std::views::join, std::inserter(targets, targets.begin()), std::back_inserter(dead_endpoints), - std::bind_front(&storage_proxy::is_alive_id, this, std::cref(*erm))); + std::bind_front(&storage_proxy::is_alive, this, std::cref(*erm))); slogger.trace("Creating write handler with live: {}; dead: {}", targets, dead_endpoints); db::assure_sufficient_live_nodes(cl, *erm, targets, pending_endpoints); return create_write_response_handler( @@ -6600,7 +6596,7 @@ future storage_proxy::cas(schema_ptr schema, shared_ptr reque host_id_vector_replica_set storage_proxy::get_live_endpoints(const locator::effective_replication_map& erm, const dht::token& token) const { host_id_vector_replica_set eps = erm.get_natural_replicas(token); - auto itend = std::ranges::remove_if(eps, std::not_fn(std::bind_front(&storage_proxy::is_alive_id, this, std::cref(erm)))).begin(); + auto itend = std::ranges::remove_if(eps, std::not_fn(std::bind_front(&storage_proxy::is_alive, this, std::cref(erm)))).begin(); eps.erase(itend, eps.end()); return eps; } @@ -6625,7 +6621,7 @@ void storage_proxy::sort_endpoints_by_proximity(const locator::effective_replica host_id_vector_replica_set storage_proxy::get_endpoints_for_reading(const sstring& ks_name, const locator::effective_replication_map& erm, const dht::token& token) const { auto endpoints = erm.get_replicas_for_reading(token); validate_read_replicas(erm, endpoints); - auto it = std::ranges::remove_if(endpoints, std::not_fn(std::bind_front(&storage_proxy::is_alive_id, this, std::cref(erm)))).begin(); + auto it = std::ranges::remove_if(endpoints, std::not_fn(std::bind_front(&storage_proxy::is_alive, this, std::cref(erm)))).begin(); endpoints.erase(it, endpoints.end()); sort_endpoints_by_proximity(erm, endpoints); return endpoints; @@ -6662,11 +6658,7 @@ storage_proxy::filter_replicas_for_read( return filter_replicas_for_read(cl, erm, live_endpoints, preferred_endpoints, db::read_repair_decision::NONE, nullptr, cf); } -bool storage_proxy::is_alive(const gms::inet_address& ep) const { - return _remote ? _remote->is_alive(ep) : is_me(ep); -} - -bool storage_proxy::is_alive_id(const locator::effective_replication_map& erm, const locator::host_id& ep) const { +bool storage_proxy::is_alive(const locator::effective_replication_map& erm, const locator::host_id& ep) const { return is_me(erm, ep) || (_remote ? _remote->is_alive(ep) : false); } diff --git a/service/storage_proxy.hh b/service/storage_proxy.hh index 6c1b0812df5c..a307a4ffb04a 100644 --- a/service/storage_proxy.hh +++ b/service/storage_proxy.hh @@ -362,8 +362,7 @@ private: host_id_vector_replica_set filter_replicas_for_read(db::consistency_level, const locator::effective_replication_map&, host_id_vector_replica_set live_endpoints, const host_id_vector_replica_set& preferred_endpoints, db::read_repair_decision, std::optional* extra, replica::column_family*) const; // As above with read_repair_decision=NONE, extra=nullptr. host_id_vector_replica_set filter_replicas_for_read(db::consistency_level, const locator::effective_replication_map&, const host_id_vector_replica_set& live_endpoints, const host_id_vector_replica_set& preferred_endpoints, replica::column_family*) const; - bool is_alive(const gms::inet_address&) const; - bool is_alive_id(const locator::effective_replication_map& erm, const locator::host_id&) const; + bool is_alive(const locator::effective_replication_map& erm, const locator::host_id&) const; result<::shared_ptr> get_read_executor(lw_shared_ptr cmd, locator::effective_replication_map_ptr ermp, schema_ptr schema, From 8a0fea5fef19755211769bc8910b3e6eaa51cfd5 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Thu, 19 Dec 2024 15:32:30 +0200 Subject: [PATCH 336/397] locator: topology: drop is_me ip overload along with remaning users --- db/system_keyspace.cc | 2 +- locator/topology.hh | 4 ---- service/qos/service_level_controller.cc | 2 +- service/storage_service.hh | 2 +- 4 files changed, 3 insertions(+), 7 deletions(-) diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 419e201d769c..01edce0b7ebb 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -2080,7 +2080,7 @@ future<> system_keyspace::update_peer_info(gms::inet_address ep, locator::host_i if (!hid) { on_internal_error(slogger, format("update_peer_info called with empty host_id, ep {}", ep)); } - if (_db.get_token_metadata().get_topology().is_me(ep)) { + if (_db.get_token_metadata().get_topology().is_me(hid)) { on_internal_error(slogger, format("update_peer_info called for this node: {}", ep)); } diff --git a/locator/topology.hh b/locator/topology.hh index 6dfe80869b77..01941b63e4f2 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -393,10 +393,6 @@ public: return id == my_host_id(); } - bool is_me(const inet_address& addr) const noexcept { - return addr == my_address(); - } - private: using random_engine_type = std::mt19937_64; diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index 9c1ebeb6c6d7..bdcbaac2d58d 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -894,7 +894,7 @@ future<> service_level_controller::do_remove_service_level(sstring name, bool re void service_level_controller::on_join_cluster(const gms::inet_address& endpoint) { } void service_level_controller::on_leave_cluster(const gms::inet_address& endpoint, const locator::host_id& hid) { - if (this_shard_id() == global_controller && _token_metadata.get()->get_topology().is_me(endpoint)) { + if (this_shard_id() == global_controller && _token_metadata.get()->get_topology().is_me(hid)) { _global_controller_db->dist_data_update_aborter.request_abort(); _global_controller_db->group0_aborter.request_abort(); } diff --git a/service/storage_service.hh b/service/storage_service.hh index 40da8c869c0b..b9e012d1c12a 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -327,7 +327,7 @@ private: return get_token_metadata_ptr()->get_topology().my_host_id(); } bool is_me(inet_address addr) const noexcept { - return get_token_metadata_ptr()->get_topology().is_me(addr); + return addr == get_broadcast_address(); } bool is_me(locator::host_id id) const noexcept { return get_token_metadata_ptr()->get_topology().is_me(id); From 415e8de36e879c556480bf32585a0a16a2ff281f Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Mon, 23 Dec 2024 13:43:43 +0200 Subject: [PATCH 337/397] locator: topology: change get_datacenter_endpoints and get_datacenter_racks to return host ids and amend users --- alternator/server.cc | 5 +++-- locator/topology.cc | 4 ++-- locator/topology.hh | 8 ++++---- test/boost/locator_topology_test.cc | 18 +++++++++--------- 4 files changed, 18 insertions(+), 17 deletions(-) diff --git a/alternator/server.cc b/alternator/server.cc index f20f52786c63..8a9a7d6884b0 100644 --- a/alternator/server.cc +++ b/alternator/server.cc @@ -217,7 +217,7 @@ class local_nodelist_handler : public gated_handler { // If the DC does not exist, we return an empty list - not an error. sstring query_dc = req->get_query_param("dc"); sstring local_dc = query_dc.empty() ? topology.get_datacenter() : query_dc; - std::unordered_set local_dc_nodes; + std::unordered_set local_dc_nodes; const auto& endpoints = topology.get_datacenter_endpoints(); auto dc_it = endpoints.find(local_dc); if (dc_it != endpoints.end()) { @@ -227,7 +227,8 @@ class local_nodelist_handler : public gated_handler { // DC, unless a single rack is selected by the "rack" query option. // If the rack does not exist, we return an empty list - not an error. sstring query_rack = req->get_query_param("rack"); - for (auto& ip : local_dc_nodes) { + for (auto& id : local_dc_nodes) { + auto ip = _gossiper.get_address_map().get(id); if (!query_rack.empty()) { auto rack = _gossiper.get_application_state_value(ip, gms::application_state::RACK); if (rack != query_rack) { diff --git a/locator/topology.cc b/locator/topology.cc index 12f3ae6c23b1..6c1417ca6eb1 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -394,7 +394,7 @@ void topology::index_node(const node& node) { if (!node.left() && !node.is_none()) { const auto& dc = node.dc_rack().dc; const auto& rack = node.dc_rack().rack; - const auto& endpoint = node.endpoint(); + const auto& endpoint = node.host_id(); _dc_nodes[dc].emplace(std::cref(node)); _dc_rack_nodes[dc][rack].emplace(std::cref(node)); _dc_endpoints[dc].insert(endpoint); @@ -416,7 +416,7 @@ void topology::unindex_node(const node& node) { bool found = _dc_nodes.at(dc).erase(std::cref(node)); if (found) { if (auto dit = _dc_endpoints.find(dc); dit != _dc_endpoints.end()) { - const auto& ep = node.endpoint(); + const auto& ep = node.host_id(); auto& eps = dit->second; eps.erase(ep); if (eps.empty()) { diff --git a/locator/topology.hh b/locator/topology.hh index 01941b63e4f2..8163c0e7064f 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -271,7 +271,7 @@ public: bool has_endpoint(inet_address) const; const std::unordered_map>& + std::unordered_set>& get_datacenter_endpoints() const { return _dc_endpoints; } @@ -292,7 +292,7 @@ public: const std::unordered_map>>& + std::unordered_set>>& get_datacenter_racks() const { return _dc_racks; } @@ -424,13 +424,13 @@ private: /** multi-map: DC -> endpoints in that DC */ std::unordered_map> + std::unordered_set> _dc_endpoints; /** map: DC -> (multi-map: rack -> endpoints in that rack) */ std::unordered_map>> + std::unordered_set>> _dc_racks; bool _sort_by_proximity = true; diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc index 99e815e84717..ccb321a11d7e 100644 --- a/test/boost/locator_topology_test.cc +++ b/test/boost/locator_topology_test.cc @@ -207,8 +207,8 @@ SEASTAR_THREAD_TEST_CASE(test_add_or_update_by_host_id) { } SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) { - using dc_endpoints_t = std::unordered_map>; - using dc_racks_t = std::unordered_map>>; + using dc_endpoints_t = std::unordered_map>; + using dc_racks_t = std::unordered_map>>; using dcs_t = std::unordered_set; const auto id1 = host_id::create_random_id(); @@ -235,19 +235,19 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) { topo.add_or_update_endpoint(id1, ep1, dc_rack1, node::state::normal); topo.add_node(id2, ep2, dc_rack2, node::state::normal); - BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {ep1, ep2}}})); - BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {ep1}}, {"rack2", {ep2}}}}})); + BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1, id2}}})); + BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}, {"rack2", {id2}}}}})); BOOST_REQUIRE_EQUAL(topo.get_datacenters(), (dcs_t{"dc1"})); topo.remove_endpoint(id2); - BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {ep1}}})); - BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {ep1}}}}})); + BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1}}})); + BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}}}})); BOOST_REQUIRE_EQUAL(topo.get_datacenters(), (dcs_t{"dc1"})); // Local endpoint cannot be removed topo.remove_endpoint(id1); - BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {ep1}}})); - BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {ep1}}}}})); + BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1}}})); + BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}}}})); BOOST_REQUIRE_EQUAL(topo.get_datacenters(), (dcs_t{"dc1"})); } @@ -411,7 +411,7 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) { } // left nodes are not members. - BOOST_REQUIRE(!topo.get_datacenter_endpoints().at(dc_rack1.dc).contains(ep3)); + BOOST_REQUIRE(!topo.get_datacenter_endpoints().at(dc_rack1.dc).contains(id3)); BOOST_REQUIRE(topo.get_datacenter(id3) == dc_rack1.dc); BOOST_REQUIRE(topo.get_rack(id3) == dc_rack1.rack); From c7d08fe1fec2eac557903e470a8bc757e9806fd3 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 25 Dec 2024 17:17:03 +0200 Subject: [PATCH 338/397] storage_service: change get_dc_rack_for() to work on host ids --- service/storage_service.cc | 20 ++++++++++---------- service/storage_service.hh | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 74c30d6d5c15..a44ed865bc9b 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2334,7 +2334,7 @@ future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::per } tmptr->remove_endpoint(host_id); } - tmptr->update_topology(host_id, get_dc_rack_for(endpoint), locator::node::state::bootstrapping); + tmptr->update_topology(host_id, get_dc_rack_for(host_id), locator::node::state::bootstrapping); tmptr->add_bootstrap_tokens(tokens, host_id); tmptr->update_host_id(host_id, endpoint); @@ -2544,7 +2544,7 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit do_notify_joined = true; } - const auto dc_rack = get_dc_rack_for(endpoint); + const auto dc_rack = get_dc_rack_for(host_id); tmptr->update_topology(host_id, dc_rack, locator::node::state::normal); co_await tmptr->update_normal_tokens(owned_tokens, host_id); if (replaced_id) { @@ -2661,7 +2661,7 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta } else { auto tmlock = co_await get_token_metadata_lock(); auto tmptr = co_await get_mutable_token_metadata_ptr(); - const auto dc_rack = get_dc_rack_for(endpoint); + const auto dc_rack = get_dc_rack_for(host_id); tmptr->update_host_id(host_id, endpoint); tmptr->update_topology(host_id, dc_rack); co_await replicate_to_all_cores(std::move(tmptr)); @@ -2881,7 +2881,7 @@ std::optional storage_service::get_dc_rack_for(const }; } -std::optional storage_service::get_dc_rack_for(inet_address endpoint) { +std::optional storage_service::get_dc_rack_for(locator::host_id endpoint) { auto eps = _gossiper.get_endpoint_state_ptr(endpoint); if (!eps) { return std::nullopt; @@ -3410,12 +3410,12 @@ storage_service::prepare_replacement_info(std::unordered_set } } - auto dc_rack = get_dc_rack_for(replace_address).value_or(locator::endpoint_dc_rack::default_location); - if (!replace_host_id) { replace_host_id = _gossiper.get_host_id(replace_address); } + auto dc_rack = get_dc_rack_for(replace_host_id).value_or(locator::endpoint_dc_rack::default_location); + auto ri = replacement_info { .tokens = std::move(tokens), .dc_rack = std::move(dc_rack), @@ -4568,7 +4568,7 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad // excluded from normal_endpoints (maybe_remove_node_being_replaced function). // In handle_state_normal we'll remap the IP to the new host_id. tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced); - tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node), locator::node::state::replacing); + tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing); tmptr->update_host_id(replacing_node_id, replacing_node); tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id); } @@ -4586,7 +4586,7 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id); tmptr->del_replacing_endpoint(existing_node_id); - const auto dc_rack = get_dc_rack_for(replacing_node); + const auto dc_rack = get_dc_rack_for(replacing_node_id); tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal); tmptr->remove_endpoint(replacing_node_id); } @@ -4632,7 +4632,7 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad auto& endpoint = x.first; auto tokens = std::unordered_set(x.second.begin(), x.second.end()); const auto host_id = *coordinator_host_id; - const auto dc_rack = get_dc_rack_for(endpoint); + const auto dc_rack = get_dc_rack_for(host_id); slogger.info("bootstrap[{}]: Added node={}/{} as bootstrap, coordinator={}/{}", req.ops_uuid, endpoint, host_id, coordinator, *coordinator_host_id); tmptr->update_host_id(host_id, endpoint); @@ -5368,7 +5368,7 @@ future<> storage_service::update_topology_change_info(mutable_token_metadata_ptr return std::nullopt; } - return get_dc_rack_for(tm.get_endpoint_for_host_id(host_id)); + return get_dc_rack_for(host_id); }); co_await tmptr->update_topology_change_info(get_dc_rack_by_host_id); } catch (...) { diff --git a/service/storage_service.hh b/service/storage_service.hh index b9e012d1c12a..9700235bc808 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -530,7 +530,7 @@ private: std::unordered_set get_tokens_for(inet_address endpoint); std::optional get_dc_rack_for(const gms::endpoint_state& ep_state); - std::optional get_dc_rack_for(inet_address endpoint); + std::optional get_dc_rack_for(locator::host_id endpoint); private: // Should be serialized under token_metadata_lock. future<> replicate_to_all_cores(mutable_token_metadata_ptr tmptr) noexcept; From ae8dc595e1df2375c9037a33ca2a414f202033f9 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 25 Dec 2024 17:24:20 +0200 Subject: [PATCH 339/397] hints: move id to ip translation into store_hint() function Also use gossiper to translate instead of token_metadata since we want to get rid of ip base APIs there. --- db/hints/manager.cc | 3 ++- db/hints/manager.hh | 2 +- service/storage_proxy.cc | 6 ++---- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/db/hints/manager.cc b/db/hints/manager.cc index 7cdb515352bc..015d492b788e 100644 --- a/db/hints/manager.cc +++ b/db/hints/manager.cc @@ -430,9 +430,10 @@ bool manager::have_ep_manager(const std::variant(ep)); } -bool manager::store_hint(endpoint_id host_id, gms::inet_address ip, schema_ptr s, lw_shared_ptr fm, +bool manager::store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr fm, tracing::trace_state_ptr tr_state) noexcept { + auto ip = _gossiper_anchor->get_address_map().get(host_id); if (utils::get_local_injector().enter("reject_incoming_hints")) { manager_logger.debug("Rejecting a hint to {} / {} due to an error injection", host_id, ip); ++_stats.dropped; diff --git a/db/hints/manager.hh b/db/hints/manager.hh index afe7e96648fd..7f075714f096 100644 --- a/db/hints/manager.hh +++ b/db/hints/manager.hh @@ -171,7 +171,7 @@ public: void register_metrics(const sstring& group_name); future<> start(shared_ptr gossiper_ptr); future<> stop(); - bool store_hint(endpoint_id host_id, gms::inet_address ip, schema_ptr s, lw_shared_ptr fm, + bool store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr fm, tracing::trace_state_ptr tr_state) noexcept; /// \brief Changes the host_filter currently used, stopping and starting endpoint_managers relevant to the new host_filter. diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index ecb059bef999..43b4da3806f3 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -1244,8 +1244,7 @@ class per_destination_mutation : public mutation_holder { tracing::trace_state_ptr tr_state) override { auto m = _mutations[hid]; if (m) { - const auto ep = ermptr->get_token_metadata().get_endpoint_for_host_id(hid); - return hm.store_hint(hid, ep, _schema, std::move(m), tr_state); + return hm.store_hint(hid, _schema, std::move(m), tr_state); } else { return false; } @@ -1303,8 +1302,7 @@ class shared_mutation : public mutation_holder { } virtual bool store_hint(db::hints::manager& hm, locator::host_id hid, locator::effective_replication_map_ptr ermptr, tracing::trace_state_ptr tr_state) override { - const auto ep = ermptr->get_token_metadata().get_endpoint_for_host_id(hid); - return hm.store_hint(hid, ep, _schema, _mutation, tr_state); + return hm.store_hint(hid, _schema, _mutation, tr_state); } virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout, tracing::trace_state_ptr tr_state, db::per_partition_rate_limit::info rate_limit_info, From 593308a05189f13a797b7e4ae0daec65afbf2577 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 25 Dec 2024 17:24:38 +0200 Subject: [PATCH 340/397] node_ops, cdc: drop remaining token_metadata::get_endpoint_for_host_id() usage Use address map to translate id to ip instead. We want to drop ips from token_metadata. --- cdc/generation.cc | 2 +- node_ops/node_ops_ctl.cc | 2 +- service/storage_service.cc | 29 +++++++++++------------------ 3 files changed, 13 insertions(+), 20 deletions(-) diff --git a/cdc/generation.cc b/cdc/generation.cc index 8ace3cea7e28..22645a7c27cd 100644 --- a/cdc/generation.cc +++ b/cdc/generation.cc @@ -402,7 +402,7 @@ future generation_service::legacy_make_new_generation(const throw std::runtime_error( format("Can't find endpoint for token {}", end)); } - const auto ep = tmptr->get_endpoint_for_host_id(*endpoint); + const auto ep = _gossiper.get_address_map().get(*endpoint); auto sc = get_shard_count(ep, _gossiper); return {sc > 0 ? sc : 1, get_sharding_ignore_msb(ep, _gossiper)}; } diff --git a/node_ops/node_ops_ctl.cc b/node_ops/node_ops_ctl.cc index 1eaaf2e333e2..64a1ab36186d 100644 --- a/node_ops/node_ops_ctl.cc +++ b/node_ops/node_ops_ctl.cc @@ -143,7 +143,7 @@ future<> node_ops_ctl::abort_on_error(node_ops_cmd cmd, std::exception_ptr ex) n future<> node_ops_ctl::send_to_all(node_ops_cmd cmd) { req.cmd = cmd; req.ignore_nodes = ignore_nodes | - std::views::transform([&] (locator::host_id id) { return tmptr->get_endpoint_for_host_id(id); }) | + std::views::transform([&] (locator::host_id id) { return ss.gossiper().get_address_map().get(id); }) | std::ranges::to(); sstring op_desc = ::format("{}", cmd); nlogger.info("{}[{}]: Started {}", desc, uuid(), req); diff --git a/service/storage_service.cc b/service/storage_service.cc index a44ed865bc9b..da07059ccf9f 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -356,16 +356,6 @@ static future<> set_gossip_tokens(gms::gossiper& g, ); } -static std::unordered_map get_token_to_endpoint(const locator::token_metadata& tm) { - const auto& map = tm.get_token_to_endpoint(); - std::unordered_map result; - result.reserve(map.size()); - for (const auto [t, id]: map) { - result.insert({t, tm.get_endpoint_for_host_id(id)}); - } - return result; -} - /* * The helper waits for two things * 1) for schema agreement @@ -1964,7 +1954,7 @@ future<> storage_service::join_topology(sharded for (auto token : bootstrap_tokens) { auto existing = tmptr->get_endpoint(token); if (existing) { - auto eps = _gossiper.get_endpoint_state_ptr(tmptr->get_endpoint_for_host_id(*existing)); + auto eps = _gossiper.get_endpoint_state_ptr(*existing); if (eps && eps->get_update_timestamp() > gms::gossiper::clk::now() - delay) { throw std::runtime_error("Cannot replace a live node..."); } @@ -2468,7 +2458,10 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit // token_to_endpoint_map is used to track the current token owners for the purpose of removing replaced endpoints. // when any token is replaced by a new owner, we track the existing owner in `candidates_for_removal` // and eventually, if any candidate for removal ends up owning no tokens, it is removed from token_metadata. - std::unordered_map token_to_endpoint_map = get_token_to_endpoint(get_token_metadata()); + std::unordered_map token_to_endpoint_map = get_token_metadata().get_token_to_endpoint() | + std::views::transform([this] (auto& e) { + return std::make_pair(e.first, _address_map.get(e.second)); + }) | std::ranges::to(); std::unordered_set candidates_for_removal; // Here we convert endpoint tokens from gossiper to owned_tokens, which will be assigned as a new @@ -2587,7 +2580,7 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit const auto& tm = get_token_metadata(); auto ver = tm.get_ring_version(); for (auto& x : tm.get_token_to_endpoint()) { - slogger.debug("handle_state_normal: token_metadata.ring_version={}, token={} -> endpoint={}/{}", ver, x.first, tm.get_endpoint_for_host_id(x.second), x.second); + slogger.debug("handle_state_normal: token_metadata.ring_version={}, token={} -> endpoint={}/{}", ver, x.first, _address_map.get(x.second), x.second); } } _normal_state_handled_on_boot.insert(endpoint); @@ -3472,7 +3465,7 @@ storage_service::prepare_replacement_info(std::unordered_set } future> storage_service::get_ownership() { - return run_with_no_api_lock([] (storage_service& ss) { + return run_with_no_api_lock([this] (storage_service& ss) { const auto& tm = ss.get_token_metadata(); auto token_map = dht::token::describe_ownership(tm.sorted_tokens()); // describeOwnership returns tokens in an unspecified order, let's re-order them @@ -3480,7 +3473,7 @@ future> storage_service::get_ownership() { for (auto entry : token_map) { locator::host_id id = tm.get_endpoint(entry.first).value(); auto token_ownership = entry.second; - ownership[tm.get_endpoint_for_host_id(id)] += token_ownership; + ownership[_address_map.get(id)] += token_ownership; } return ownership; }); @@ -5299,10 +5292,10 @@ std::map storage_service::get_token_to_endpoint_map() { const auto& tm = get_token_metadata(); std::map result; for (const auto [t, id]: tm.get_token_to_endpoint()) { - result.insert({t, tm.get_endpoint_for_host_id(id)}); + result.insert({t, _address_map.get(id)}); } for (const auto [t, id]: tm.get_bootstrap_tokens()) { - result.insert({t, tm.get_endpoint_for_host_id(id)}); + result.insert({t, _address_map.get(id)}); } return result; } @@ -5312,7 +5305,7 @@ future> storage_service::get_tablet_to_endpoint_ma const auto& tmap = tm.tablets().get_tablet_map(table); std::map result; for (std::optional tid = tmap.first_tablet(); tid; tid = tmap.next_tablet(*tid)) { - result.emplace(tmap.get_last_token(*tid), tm.get_endpoint_for_host_id(tmap.get_primary_replica(*tid).host)); + result.emplace(tmap.get_last_token(*tid), _address_map.get(tmap.get_primary_replica(*tid).host)); co_await coroutine::maybe_yield(); } co_return result; From 7556e3d0453efb69e6f406fe5313727672618d12 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 1 Jan 2025 15:59:19 +0200 Subject: [PATCH 341/397] topology coordinator: remove gossiper entry only if host id matches provided one Currently the entry is removed only if ip is not used by any normal or transitioning node. This is done to not remove a wrong entry that just happen to use the same ip, but the same can be achieved by checking host id in the entry. --- service/storage_service.cc | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index da07059ccf9f..e45682258891 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -428,20 +428,6 @@ future storage_service::sync_raft_t } }; - auto get_used_ips = [&, used_ips = std::optional>{}]() mutable - -> const std::unordered_set& - { - if (!used_ips) { - used_ips.emplace(); - for (const auto& [sid, rs]: boost::range::join(t.normal_nodes, t.transition_nodes)) { - if (const auto used_ip = am.find(locator::host_id{sid.uuid()})) { - used_ips->insert(*used_ip); - } - } - } - return *used_ips; - }; - using host_id_to_ip_map_t = std::unordered_map; auto get_host_id_to_ip_map = [&, map = std::optional{}]() mutable -> future { if (!map.has_value()) { @@ -464,7 +450,7 @@ future storage_service::sync_raft_t auto remove_ip = [&](inet_address ip, locator::host_id host_id, bool notify) -> future<> { sys_ks_futures.push_back(_sys_ks.local().remove_endpoint(ip)); - if (_gossiper.get_endpoint_state_ptr(ip) && !get_used_ips().contains(ip)) { + if (const auto ep = _gossiper.get_endpoint_state_ptr(ip); ep && ep->get_host_id() == host_id) { co_await _gossiper.force_remove_endpoint(ip, gms::null_permit_id); if (notify) { nodes_to_notify.left.push_back({ip, host_id}); From 8e55cc6c78fb371c09b0a7215d128a38767b7048 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 10:56:10 +0200 Subject: [PATCH 342/397] storage_service: fix logging When logger outputs a range it already does join, so no other join is needed. --- service/storage_service.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index e45682258891..904f9ca56ff4 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -7457,7 +7457,7 @@ future<> storage_service::wait_for_normal_state_handled_on_boot() { static auto fmt_nodes_with_statuses = [this] (const auto& eps) { return eps | std::views::transform([this] (const auto& ep) { return ::format("({}, status={})", ep, _gossiper.get_gossip_status(ep)); - }) | std::views::join_with(','); + }); }; slogger.info("Started waiting for normal state handlers to finish"); From 70cc014307b10f76b13eff98f723c1dc8eca9b23 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 12:03:45 +0200 Subject: [PATCH 343/397] storage_service: ip_address_updater: check peers table instead of token_metadata whether ip was changed As part of changing IP address peers table is updated. If it has a new address the update can be skipped. --- service/storage_service.cc | 11 ++++++++++- service/storage_service.hh | 3 +++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 904f9ca56ff4..e5957608b76e 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -909,7 +909,8 @@ class storage_service::ip_address_updater: public gms::i_endpoint_state_change_s locator::host_id id(utils::UUID(app_state_ptr->value())); rslog.debug("ip_address_updater::on_endpoint_change({}) {} {}", ev, endpoint, id); - auto prev_ip = _ss.get_token_metadata().get_endpoint_for_host_id_if_known(id); + // If id maps to different ip in peers table it needs to be updated which is done by sync_raft_topology_nodes below + std::optional prev_ip = co_await _ss.get_ip_from_peers_table(id); if (prev_ip == endpoint) { co_return; } @@ -2647,6 +2648,14 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta } } +future> storage_service::get_ip_from_peers_table(locator::host_id id) { + auto peers = co_await _sys_ks.local().load_host_ids(); + if (auto it = std::ranges::find_if(peers, [&id] (const auto& e) { return e.second == id; }); it != peers.end()) { + co_return it->first; + } + co_return std::nullopt; +} + future<> storage_service::on_change(gms::inet_address endpoint, const gms::application_state_map& states_, gms::permit_id pid) { // copy the states map locally since the coroutine may yield auto states = states_; diff --git a/service/storage_service.hh b/service/storage_service.hh index 9700235bc808..79820afcb5a9 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -604,6 +604,9 @@ private: future> get_changed_ranges_for_leaving(locator::vnode_effective_replication_map_ptr erm, locator::host_id endpoint); future<> maybe_reconnect_to_preferred_ip(inet_address ep, inet_address local_ip); + + // Return ip of the peers table entry with given host id + future> get_ip_from_peers_table(locator::host_id id); public: sstring get_release_version(); From 7c4c485651c998284531cfbe8671b4bf6fa459ec Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 13:34:43 +0200 Subject: [PATCH 344/397] host_id_or_endpoint: use gossiper to resolve ip to id and back mappings host_id_or_endpoint is a helper class that hold either id or ip and translate one into another on demand. Use gossiper to do a translation there instead of token_metadata since we want to drop ip based APIs from the later. --- db/hints/manager.cc | 8 ++++---- locator/token_metadata.cc | 14 +++++++------- locator/token_metadata.hh | 8 ++++++-- service/storage_service.cc | 2 +- 4 files changed, 18 insertions(+), 14 deletions(-) diff --git a/db/hints/manager.cc b/db/hints/manager.cc index 015d492b788e..1f996d193fce 100644 --- a/db/hints/manager.cc +++ b/db/hints/manager.cc @@ -594,9 +594,9 @@ future<> manager::change_host_filter(host_filter filter) { // been created by mistake and they're invalid. The same for pre-host-ID hinted handoff // -- hint directories representing host IDs are NOT valid. if (hid_or_ep.has_host_id() && _uses_host_id) { - return std::make_optional(pair_type{hid_or_ep.id(), hid_or_ep.resolve_endpoint(*tmptr)}); + return std::make_optional(pair_type{hid_or_ep.id(), hid_or_ep.resolve_endpoint(*_gossiper_anchor)}); } else if (hid_or_ep.has_endpoint() && !_uses_host_id) { - return std::make_optional(pair_type{hid_or_ep.resolve_id(*tmptr), hid_or_ep.endpoint()}); + return std::make_optional(pair_type{hid_or_ep.resolve_id(*_gossiper_anchor), hid_or_ep.endpoint()}); } else { return std::nullopt; } @@ -817,7 +817,7 @@ future<> manager::initialize_endpoint_managers() { const auto maybe_host_id = std::invoke([&] () -> std::optional { try { - return maybe_host_id_or_ep->resolve_id(*tmptr); + return maybe_host_id_or_ep->resolve_id(*_gossiper_anchor); } catch (...) { return std::nullopt; } @@ -869,7 +869,7 @@ future<> manager::migrate_ip_directories() { continue; } - const locator::host_id host_id = hid_or_ep.resolve_id(*tmptr); + const locator::host_id host_id = hid_or_ep.resolve_id(*_gossiper_anchor); dirs_to_rename.push_back({.current_name = std::move(directory), .new_name = host_id.to_sstring()}); } catch (...) { // We cannot map the IP to the corresponding host ID either because diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index 3c369e44ce5a..75603c58e6a5 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -22,7 +22,7 @@ #include #include "utils/assert.hh" #include "utils/stall_free.hh" - +#include "gms/gossiper.hh" namespace locator { static logging::logger tlogger("token_metadata"); @@ -1391,22 +1391,22 @@ host_id_or_endpoint::host_id_or_endpoint(const sstring& s, param_type restrict) } } -host_id host_id_or_endpoint::resolve_id(const token_metadata& tm) const { +host_id host_id_or_endpoint::resolve_id(const gms::gossiper& g) const { if (has_host_id()) { return id(); } - auto opt_id = tm.get_host_id_if_known(endpoint()); - if (!opt_id) { + try { + return g.get_host_id(endpoint()); + } catch (...) { throw std::runtime_error(format("Host inet address {} not found in the cluster", endpoint())); } - return *opt_id; } -gms::inet_address host_id_or_endpoint::resolve_endpoint(const token_metadata& tm) const { +gms::inet_address host_id_or_endpoint::resolve_endpoint(const gms::gossiper& g) const { if (has_endpoint()) { return endpoint(); } - auto endpoint_opt = tm.get_endpoint_for_host_id_if_known(id()); + auto endpoint_opt = g.get_address_map().find(id()); if (!endpoint_opt) { throw std::runtime_error(format("Host ID {} not found in the cluster", id())); } diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh index 541c91f086e5..b8420761242c 100644 --- a/locator/token_metadata.hh +++ b/locator/token_metadata.hh @@ -37,6 +37,10 @@ namespace replica { class keyspace; } +namespace gms { +class gossiper; +} + namespace locator { class abstract_replication_strategy; @@ -75,8 +79,8 @@ struct host_id_or_endpoint { // Map the host_id to endpoint or vice verse, using the token_metadata. // Throws runtime error if failed to resolve. - host_id resolve_id(const token_metadata&) const; - gms::inet_address resolve_endpoint(const token_metadata&) const; + host_id resolve_id(const gms::gossiper&) const; + gms::inet_address resolve_endpoint(const gms::gossiper&) const; }; using host_id_or_endpoint_list = std::vector; diff --git a/service/storage_service.cc b/service/storage_service.cc index e5957608b76e..f611727e20dc 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -4189,7 +4189,7 @@ future<> storage_service::removenode(locator::host_id host_id, locator::host_id_ } for (auto& hoep : ignore_nodes_params) { - ctl.ignore_nodes.insert(hoep.resolve_id(*tmptr)); + ctl.ignore_nodes.insert(hoep.resolve_id(ss._gossiper)); } if (!removed_from_token_ring) { From fcfd0050234e5da11259e54bcd99fb0f52776ed7 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 13:48:24 +0200 Subject: [PATCH 345/397] token_metadata: drop no longer used functions --- locator/token_metadata.cc | 32 -------------------------------- locator/token_metadata.hh | 7 ------- 2 files changed, 39 deletions(-) diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index 75603c58e6a5..3320e37ed141 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -153,12 +153,6 @@ class token_metadata_impl final { /** Return the unique host ID for an end-point. */ host_id get_host_id(inet_address endpoint) const; - /// Return the unique host ID for an end-point or nullopt if not found. - std::optional get_host_id_if_known(inet_address endpoint) const; - - /** Return the end-point for a unique host ID or nullopt if not found.*/ - std::optional get_endpoint_for_host_id_if_known(host_id) const; - /** Return the end-point for a unique host ID.*/ inet_address get_endpoint_for_host_id(host_id) const; @@ -543,22 +537,6 @@ host_id token_metadata_impl::get_host_id(inet_address endpoint) const { } } -std::optional token_metadata_impl::get_host_id_if_known(inet_address endpoint) const { - if (const auto* node = _topology.find_node(endpoint)) [[likely]] { - return node->host_id(); - } else { - return std::nullopt; - } -} - -std::optional token_metadata_impl::get_endpoint_for_host_id_if_known(host_id host_id) const { - if (const auto* node = _topology.find_node(host_id)) [[likely]] { - return node->endpoint(); - } else { - return std::nullopt; - } -} - inet_address token_metadata_impl::get_endpoint_for_host_id(host_id host_id) const { if (const auto* node = _topology.find_node(host_id)) [[likely]] { return node->endpoint(); @@ -1042,16 +1020,6 @@ token_metadata::get_host_id(inet_address endpoint) const { return _impl->get_host_id(endpoint); } -std::optional -token_metadata::get_host_id_if_known(inet_address endpoint) const { - return _impl->get_host_id_if_known(endpoint); -} - -std::optional -token_metadata::get_endpoint_for_host_id_if_known(host_id host_id) const { - return _impl->get_endpoint_for_host_id_if_known(host_id); -} - token_metadata::inet_address token_metadata::get_endpoint_for_host_id(host_id host_id) const { return _impl->get_endpoint_for_host_id(host_id); diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh index b8420761242c..4ac6a2143b41 100644 --- a/locator/token_metadata.hh +++ b/locator/token_metadata.hh @@ -239,13 +239,6 @@ public: /** Return the unique host ID for an end-point. */ host_id get_host_id(inet_address endpoint) const; - /// Return the unique host ID for an end-point or nullopt if not found. - std::optional get_host_id_if_known(inet_address endpoint) const; - - /** Return the end-point for a unique host ID or nullopt if not found. */ - std::optional get_endpoint_for_host_id_if_known(locator::host_id host_id) const; - - /** Return the end-point for a unique host ID */ inet_address get_endpoint_for_host_id(locator::host_id host_id) const; /** @return a copy of the endpoint-to-id map for read-only operations */ From 9197b88e48e583dcb77b191168b80cccb69ed9ae Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 8 Jan 2025 14:03:48 +0200 Subject: [PATCH 346/397] storage_service: drop loops from node ops replace_prepare handling since there can be only one replacing node The call already throw an error if there are more than one. Throw is there are zero as well and drop the loops. --- service/storage_service.cc | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index f611727e20dc..880faa173a3a 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -4534,13 +4534,17 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad slogger.warn("{}", msg); throw std::runtime_error(msg); } + if (req.replace_nodes.size() == 0) { + auto msg = ::format("replace[{}]: Replacing node was not specified", req.ops_uuid); + slogger.warn("{}", msg); + throw std::runtime_error(msg); + } if (!coordinator_host_id) { throw std::runtime_error("Coordinator host_id not found"); } - mutate_token_metadata([coordinator, coordinator_host_id, &req, this] (mutable_token_metadata_ptr tmptr) mutable { - for (auto& x: req.replace_nodes) { - auto existing_node = x.first; - auto replacing_node = x.second; + auto existing_node = req.replace_nodes.begin()->first; + auto replacing_node = req.replace_nodes.begin()->second; + mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, &req, this] (mutable_token_metadata_ptr tmptr) mutable { const auto existing_node_id = tmptr->get_host_id(existing_node); const auto replacing_node_id = *coordinator_host_id; slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", @@ -4559,15 +4563,11 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing); tmptr->update_host_id(replacing_node_id, replacing_node); tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id); - } return make_ready_future<>(); }).get(); auto ignore_nodes = std::move(req.ignore_nodes); - node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, req = std::move(req)] () mutable { - return mutate_token_metadata([this, coordinator, coordinator_host_id, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable { - for (auto& x: req.replace_nodes) { - auto existing_node = x.first; - auto replacing_node = x.second; + node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] () mutable { + return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable { const auto existing_node_id = tmptr->get_host_id(existing_node); const auto replacing_node_id = *coordinator_host_id; slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", @@ -4577,7 +4577,6 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad const auto dc_rack = get_dc_rack_for(replacing_node_id); tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal); tmptr->remove_endpoint(replacing_node_id); - } return update_topology_change_info(tmptr, ::format("replace {}", req.replace_nodes)); }); }); From 0db6136fa50c94c13aca0dba7b07e8ac0442d79c Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 8 Jan 2025 14:04:35 +0200 Subject: [PATCH 347/397] storage_service: fix indentation after the last patch --- service/storage_service.cc | 54 +++++++++++++++++++------------------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 880faa173a3a..626433cd9fc6 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -4545,38 +4545,38 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad auto existing_node = req.replace_nodes.begin()->first; auto replacing_node = req.replace_nodes.begin()->second; mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, &req, this] (mutable_token_metadata_ptr tmptr) mutable { - const auto existing_node_id = tmptr->get_host_id(existing_node); - const auto replacing_node_id = *coordinator_host_id; - slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", - req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id); - - // In case of replace-with-same-ip we need to map both host_id-s - // to the same IP. The locator::topology allows this specifically in case - // where one node is being_replaced and another is replacing, - // so here we adjust the state of the original node accordingly. - // The host_id -> IP map works as usual, and IP -> host_id will map - // IP to the being_replaced node - this is what is implied by the - // current code. The IP will be placed in pending_endpoints and - // excluded from normal_endpoints (maybe_remove_node_being_replaced function). - // In handle_state_normal we'll remap the IP to the new host_id. - tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced); - tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing); - tmptr->update_host_id(replacing_node_id, replacing_node); - tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id); + const auto existing_node_id = tmptr->get_host_id(existing_node); + const auto replacing_node_id = *coordinator_host_id; + slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", + req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id); + + // In case of replace-with-same-ip we need to map both host_id-s + // to the same IP. The locator::topology allows this specifically in case + // where one node is being_replaced and another is replacing, + // so here we adjust the state of the original node accordingly. + // The host_id -> IP map works as usual, and IP -> host_id will map + // IP to the being_replaced node - this is what is implied by the + // current code. The IP will be placed in pending_endpoints and + // excluded from normal_endpoints (maybe_remove_node_being_replaced function). + // In handle_state_normal we'll remap the IP to the new host_id. + tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced); + tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing); + tmptr->update_host_id(replacing_node_id, replacing_node); + tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id); return make_ready_future<>(); }).get(); auto ignore_nodes = std::move(req.ignore_nodes); node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] () mutable { return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable { - const auto existing_node_id = tmptr->get_host_id(existing_node); - const auto replacing_node_id = *coordinator_host_id; - slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", - req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id); - - tmptr->del_replacing_endpoint(existing_node_id); - const auto dc_rack = get_dc_rack_for(replacing_node_id); - tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal); - tmptr->remove_endpoint(replacing_node_id); + const auto existing_node_id = tmptr->get_host_id(existing_node); + const auto replacing_node_id = *coordinator_host_id; + slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", + req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id); + + tmptr->del_replacing_endpoint(existing_node_id); + const auto dc_rack = get_dc_rack_for(replacing_node_id); + tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal); + tmptr->remove_endpoint(replacing_node_id); return update_topology_change_info(tmptr, ::format("replace {}", req.replace_nodes)); }); }); From a7a7cdcf429d3d78ee4ee01755f3282570f26639 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 14:05:51 +0200 Subject: [PATCH 348/397] storage_service: use gossiper to map ip to id in node_ops operations Replace operation is special though. In case of replacing with the same IP the gossiper will not have the mapping, and node_ops RPC unfortunately does not send host id of a replaced node. For replace we consult peers table instead to find the old owner of the IP. A node that is replacing (the coordinator of the replace) will not have it though, but luckily it is not needed since it updates metadata during join_topology() anyway. The only thing that is missing there is add_replacing_endpoint() call which the patch adds. --- service/storage_service.cc | 27 ++++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 626433cd9fc6..7e331106bb0a 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -1552,6 +1552,7 @@ future<> storage_service::join_topology(sharded tmptr->update_topology(ri->host_id, std::move(ri->dc_rack), locator::node::state::being_replaced); co_await tmptr->update_normal_tokens(bootstrap_tokens, ri->host_id); tmptr->update_host_id(ri->host_id, *replace_address); + tmptr->add_replacing_endpoint(ri->host_id, tmptr->get_my_id()); replaced_host_id = ri->host_id; @@ -4417,7 +4418,7 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad mutate_token_metadata([coordinator, &req, this] (mutable_token_metadata_ptr tmptr) mutable { for (auto& node : req.leaving_nodes) { slogger.info("removenode[{}]: Added node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator); - tmptr->add_leaving_endpoint(tmptr->get_host_id(node)); + tmptr->add_leaving_endpoint(_gossiper.get_host_id(node)); } return update_topology_change_info(tmptr, ::format("removenode {}", req.leaving_nodes)); }).get(); @@ -4425,7 +4426,7 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad return mutate_token_metadata([this, coordinator, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable { for (auto& node : req.leaving_nodes) { slogger.info("removenode[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator); - tmptr->del_leaving_endpoint(tmptr->get_host_id(node)); + tmptr->del_leaving_endpoint(_gossiper.get_host_id(node)); } return update_topology_change_info(tmptr, ::format("removenode {}", req.leaving_nodes)); }); @@ -4451,7 +4452,7 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad }).get(); } else { slogger.info("removenode[{}]: Started to sync data for removing node={} using stream, coordinator={}", req.ops_uuid, node, coordinator); - removenode_with_stream(get_token_metadata().get_host_id(node), topo_guard, as).get(); + removenode_with_stream(_gossiper.get_host_id(node), topo_guard, as).get(); } } } else if (req.cmd == node_ops_cmd::removenode_abort) { @@ -4467,7 +4468,7 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad mutate_token_metadata([coordinator, &req, this] (mutable_token_metadata_ptr tmptr) mutable { for (auto& node : req.leaving_nodes) { slogger.info("decommission[{}]: Added node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator); - tmptr->add_leaving_endpoint(tmptr->get_host_id(node)); + tmptr->add_leaving_endpoint(_gossiper.get_host_id(node)); } return update_topology_change_info(tmptr, ::format("decommission {}", req.leaving_nodes)); }).get(); @@ -4544,8 +4545,13 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad } auto existing_node = req.replace_nodes.begin()->first; auto replacing_node = req.replace_nodes.begin()->second; - mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, &req, this] (mutable_token_metadata_ptr tmptr) mutable { - const auto existing_node_id = tmptr->get_host_id(existing_node); + auto existing_node_id = _sys_ks.local().load_host_ids().get()[existing_node]; + mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, existing_node_id, &req, this] (mutable_token_metadata_ptr tmptr) mutable { + if (is_me(*coordinator_host_id)) { + // A coordinor already updated token metadata in join_topology() + return make_ready_future<>(); + } + const auto replacing_node_id = *coordinator_host_id; slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id); @@ -4566,9 +4572,12 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad return make_ready_future<>(); }).get(); auto ignore_nodes = std::move(req.ignore_nodes); - node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] () mutable { - return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable { - const auto existing_node_id = tmptr->get_host_id(existing_node); + node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, existing_node_id, req = std::move(req)] () mutable { + return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, existing_node_id, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable { + if (is_me(*coordinator_host_id)) { + // No need to cancel replace operation on a node replacing node since it will be killed anyway + return make_ready_future<>(); + } const auto replacing_node_id = *coordinator_host_id; slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}", req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id); From 0ec9f7de64d47ba5bcaee563914ac5913963fb97 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 15:04:16 +0200 Subject: [PATCH 349/397] gossiper: drop get_unreachable_token_owners functions It is used by truncate code only and even there it only check if the returned set is not empty. Check for dead token owners in the truncation code directly. --- gms/gossiper.cc | 11 ----------- gms/gossiper.hh | 5 ----- service/storage_proxy.cc | 4 +++- 3 files changed, 3 insertions(+), 17 deletions(-) diff --git a/gms/gossiper.cc b/gms/gossiper.cc index a8ec00dc8160..0521080f8c45 100644 --- a/gms/gossiper.cc +++ b/gms/gossiper.cc @@ -1201,17 +1201,6 @@ std::set gossiper::get_live_token_owners() const { return token_owners; } -std::set gossiper::get_unreachable_token_owners() const { - std::set token_owners; - auto normal_token_owners = get_token_metadata_ptr()->get_normal_token_owners_ips(); - for (auto& node: normal_token_owners) { - if (!is_alive(node)) { - token_owners.insert(node); - } - } - return token_owners; -} - std::set gossiper::get_unreachable_nodes() const { std::set unreachable_nodes; auto nodes = get_token_metadata_ptr()->get_topology().get_all_host_ids(); diff --git a/gms/gossiper.hh b/gms/gossiper.hh index 1b647370ff3e..f3d5dd631cb6 100644 --- a/gms/gossiper.hh +++ b/gms/gossiper.hh @@ -306,11 +306,6 @@ public: std::set get_unreachable_members() const; std::set get_unreachable_host_ids() const; - /** - * @return a list of unreachable token owners - */ - std::set get_unreachable_token_owners() const; - /** * @return a list of unreachable nodes */ diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 43b4da3806f3..31d041137059 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -471,7 +471,9 @@ class storage_proxy::remote { } future<> send_truncate_blocking(sstring keyspace, sstring cfname, std::chrono::milliseconds timeout_in_ms) { - if (!_gossiper.get_unreachable_token_owners().empty()) { + auto s = _sp.local_db().find_schema(keyspace, cfname); + auto erm_ptr = s->table().get_effective_replication_map(); + if (!std::ranges::all_of(erm_ptr->get_token_metadata().get_normal_token_owners(), std::bind_front(&storage_proxy::is_alive, &_sp, std::cref(*erm_ptr)))) { slogger.info("Cannot perform truncate, some hosts are down"); // Since the truncate operation is so aggressive and is typically only // invoked by an admin, for simplicity we require that all nodes are up From 3068e38baabf0e1545d1ff4d04fbe6f13102ae0e Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 15:23:00 +0200 Subject: [PATCH 350/397] locator: network_topology_strategy: use host_id based function to check number of endpoints in dcs --- locator/network_topology_strategy.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/locator/network_topology_strategy.cc b/locator/network_topology_strategy.cc index e59581a32ee2..776778b81cc4 100644 --- a/locator/network_topology_strategy.cc +++ b/locator/network_topology_strategy.cc @@ -235,7 +235,7 @@ class natural_endpoints_tracker { } static void check_enough_endpoints(const token_metadata& tm, const std::unordered_map& dc_rf) { - auto dc_endpoints = tm.get_datacenter_token_owners_ips(); + auto dc_endpoints = tm.get_datacenter_token_owners(); auto endpoints_in = [&dc_endpoints](sstring dc) { auto i = dc_endpoints.find(dc); return i != dc_endpoints.end() ? i->second.size() : size_t(0); @@ -309,7 +309,7 @@ effective_replication_map_ptr network_topology_strategy::make_replication_map(ta static unsigned calculate_initial_tablets_from_topology(const schema& s, token_metadata_ptr tm, const std::unordered_map& rf) { unsigned initial_tablets = std::numeric_limits::min(); - for (const auto& dc : tm->get_datacenter_token_owners_ips()) { + for (const auto& dc : tm->get_datacenter_token_owners()) { unsigned shards_in_dc = 0; unsigned rf_in_dc = 1; From 97f95f1dbdaea51714371235493172326363f309 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 15:51:11 +0200 Subject: [PATCH 351/397] locator: token_metadata: remove unused ip based functions --- locator/token_metadata.cc | 77 ------------------------------- locator/token_metadata.hh | 11 +---- test/boost/token_metadata_test.cc | 1 - 3 files changed, 2 insertions(+), 87 deletions(-) diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index 3320e37ed141..27b8e9912859 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -150,12 +150,6 @@ class token_metadata_impl final { */ void update_host_id(const host_id& host_id, inet_address endpoint); - /** Return the unique host ID for an end-point. */ - host_id get_host_id(inet_address endpoint) const; - - /** Return the end-point for a unique host ID.*/ - inet_address get_endpoint_for_host_id(host_id) const; - /** @return a copy of host id set for read-only operations */ std::unordered_set get_host_ids() const; @@ -254,11 +248,8 @@ class token_metadata_impl final { * Bootstrapping tokens are not taken into account. */ size_t count_normal_token_owners() const; - std::unordered_map> get_datacenter_token_owners_ips() const; std::unordered_map> get_datacenter_token_owners() const; - std::unordered_map>> - get_datacenter_racks_token_owners_ips() const; std::unordered_map>> get_datacenter_racks_token_owners() const; @@ -529,22 +520,6 @@ void token_metadata_impl::update_host_id(const host_id& host_id, inet_address en _topology.add_or_update_endpoint(host_id, endpoint); } -host_id token_metadata_impl::get_host_id(inet_address endpoint) const { - if (const auto* node = _topology.find_node(endpoint)) [[likely]] { - return node->host_id(); - } else { - on_internal_error(tlogger, format("host_id for endpoint {} is not found", endpoint)); - } -} - -inet_address token_metadata_impl::get_endpoint_for_host_id(host_id host_id) const { - if (const auto* node = _topology.find_node(host_id)) [[likely]] { - return node->endpoint(); - } else { - on_internal_error(tlogger, format("endpoint for host_id {} is not found", host_id)); - } -} - std::unordered_set token_metadata_impl::get_host_ids() const { return _topology.get_nodes() | std::views::filter([&] (const node& n) { return !n.left() && !n.is_none(); }) | @@ -773,17 +748,6 @@ void token_metadata_impl::for_each_token_owner(std::function }); } -std::unordered_map> -token_metadata_impl::get_datacenter_token_owners_ips() const { - std::unordered_map> datacenter_token_owners; - _topology.for_each_node([&] (const node& n) { - if (is_normal_token_owner(n.host_id())) { - datacenter_token_owners[n.dc_rack().dc].insert(n.endpoint()); - } - }); - return datacenter_token_owners; -} - std::unordered_map> token_metadata_impl::get_datacenter_token_owners() const { std::unordered_map> datacenter_token_owners; @@ -795,18 +759,6 @@ token_metadata_impl::get_datacenter_token_owners() const { return datacenter_token_owners; } -std::unordered_map>> -token_metadata_impl::get_datacenter_racks_token_owners_ips() const { - std::unordered_map>> dc_racks_token_owners; - _topology.for_each_node([&] (const node& n) { - const auto& dc_rack = n.dc_rack(); - if (is_normal_token_owner(n.host_id())) { - dc_racks_token_owners[dc_rack.dc][dc_rack.rack].insert(n.endpoint()); - } - }); - return dc_racks_token_owners; -} - std::unordered_map>> token_metadata_impl::get_datacenter_racks_token_owners() const { std::unordered_map>> dc_racks_token_owners; @@ -1015,16 +967,6 @@ token_metadata::update_host_id(const host_id& host_id, inet_address endpoint) { _impl->update_host_id(host_id, endpoint); } -host_id -token_metadata::get_host_id(inet_address endpoint) const { - return _impl->get_host_id(endpoint); -} - -token_metadata::inet_address -token_metadata::get_endpoint_for_host_id(host_id host_id) const { - return _impl->get_endpoint_for_host_id(host_id); -} - std::unordered_set token_metadata::get_host_ids() const { return _impl->get_host_ids(); @@ -1147,16 +1089,6 @@ token_metadata::get_normal_token_owners() const { return _impl->get_normal_token_owners(); } -std::unordered_set token_metadata::get_normal_token_owners_ips() const { - const auto& host_ids = _impl->get_normal_token_owners(); - std::unordered_set result; - result.reserve(host_ids.size()); - for (const auto& id: host_ids) { - result.insert(_impl->get_endpoint_for_host_id(id)); - } - return result; -} - void token_metadata::for_each_token_owner(std::function func) const { return _impl->for_each_token_owner(func); } @@ -1166,19 +1098,10 @@ token_metadata::count_normal_token_owners() const { return _impl->count_normal_token_owners(); } -std::unordered_map> token_metadata::get_datacenter_token_owners_ips() const { - return _impl->get_datacenter_token_owners_ips(); -} - std::unordered_map> token_metadata::get_datacenter_token_owners() const { return _impl->get_datacenter_token_owners(); } -std::unordered_map>> -token_metadata::get_datacenter_racks_token_owners_ips() const { - return _impl->get_datacenter_racks_token_owners_ips(); -} - std::unordered_map>> token_metadata::get_datacenter_racks_token_owners() const { return _impl->get_datacenter_racks_token_owners(); diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh index 4ac6a2143b41..f2c27fcb14d5 100644 --- a/locator/token_metadata.hh +++ b/locator/token_metadata.hh @@ -239,8 +239,6 @@ public: /** Return the unique host ID for an end-point. */ host_id get_host_id(inet_address endpoint) const; - inet_address get_endpoint_for_host_id(locator::host_id host_id) const; - /** @return a copy of the endpoint-to-id map for read-only operations */ std::unordered_set get_host_ids() const; @@ -324,23 +322,18 @@ public: const std::unordered_set& get_normal_token_owners() const; - std::unordered_set get_normal_token_owners_ips() const; - void for_each_token_owner(std::function func) const; /* Returns the number of different endpoints that own tokens in the ring. * Bootstrapping tokens are not taken into account. */ size_t count_normal_token_owners() const; - // Returns the map: DC -> addresses of token owners in that DC. + // Returns the map: DC -> host_id of token owners in that DC. // If there are no token owners in a DC, it is not present in the result. - std::unordered_map> get_datacenter_token_owners_ips() const; std::unordered_map> get_datacenter_token_owners() const; - // Returns the map: DC -> (map: rack -> addresses of token owners in that rack). + // Returns the map: DC -> (map: rack -> host_id of token owners in that rack). // If there are no token owners in a DC/rack, it is not present in the result. - std::unordered_map>> - get_datacenter_racks_token_owners_ips() const; std::unordered_map>> get_datacenter_racks_token_owners() const; diff --git a/test/boost/token_metadata_test.cc b/test/boost/token_metadata_test.cc index fe9edeb13203..96aa2f9b0fd4 100644 --- a/test/boost/token_metadata_test.cc +++ b/test/boost/token_metadata_test.cc @@ -300,7 +300,6 @@ SEASTAR_THREAD_TEST_CASE(test_replace_node_with_same_endpoint) { token_metadata->add_replacing_endpoint(e1_id1, e1_id2); auto erm = create_erm(token_metadata, {{"replication_factor", "2"}}); - BOOST_REQUIRE_EQUAL(token_metadata->get_host_id(e1), e1_id1); BOOST_REQUIRE_EQUAL(erm->get_pending_replicas(dht::token::from_int64(1)), host_id_vector_topology_change{e1_id2}); BOOST_REQUIRE_EQUAL(erm->get_natural_replicas(dht::token::from_int64(1)), From 122d58b4ad8741016eaaa00302fb9282d4980bc6 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 16:21:31 +0200 Subject: [PATCH 352/397] api: view_build_statuses: do not use IP from the topology, but translate id to ip using address map instead --- api/api.cc | 4 ++-- api/api_init.hh | 2 +- api/storage_service.cc | 6 +++--- api/storage_service.hh | 2 +- db/view/view.cc | 4 ++-- db/view/view_builder.hh | 2 +- main.cc | 2 +- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/api/api.cc b/api/api.cc index 670750e6843f..e2880dad6e91 100644 --- a/api/api.cc +++ b/api/api.cc @@ -153,8 +153,8 @@ future<> unset_server_sstables_loader(http_context& ctx) { return ctx.http_server.set_routes([&ctx] (routes& r) { unset_sstables_loader(ctx, r); }); } -future<> set_server_view_builder(http_context& ctx, sharded& vb) { - return ctx.http_server.set_routes([&ctx, &vb] (routes& r) { set_view_builder(ctx, r, vb); }); +future<> set_server_view_builder(http_context& ctx, sharded& vb, sharded& g) { + return ctx.http_server.set_routes([&ctx, &vb, &g] (routes& r) { set_view_builder(ctx, r, vb, g); }); } future<> unset_server_view_builder(http_context& ctx) { diff --git a/api/api_init.hh b/api/api_init.hh index 205090c1cb48..b09948a8dab1 100644 --- a/api/api_init.hh +++ b/api/api_init.hh @@ -100,7 +100,7 @@ future<> set_server_storage_service(http_context& ctx, sharded unset_server_storage_service(http_context& ctx); future<> set_server_sstables_loader(http_context& ctx, sharded& sst_loader); future<> unset_server_sstables_loader(http_context& ctx); -future<> set_server_view_builder(http_context& ctx, sharded& vb); +future<> set_server_view_builder(http_context& ctx, sharded& vb, sharded& g); future<> unset_server_view_builder(http_context& ctx); future<> set_server_repair(http_context& ctx, sharded& repair, sharded& am); future<> unset_server_repair(http_context& ctx); diff --git a/api/storage_service.cc b/api/storage_service.cc index 6e5cdbfbbffc..b91d099027f4 100644 --- a/api/storage_service.cc +++ b/api/storage_service.cc @@ -533,11 +533,11 @@ void unset_sstables_loader(http_context& ctx, routes& r) { ss::start_restore.unset(r); } -void set_view_builder(http_context& ctx, routes& r, sharded& vb) { - ss::view_build_statuses.set(r, [&ctx, &vb] (std::unique_ptr req) { +void set_view_builder(http_context& ctx, routes& r, sharded& vb, sharded& g) { + ss::view_build_statuses.set(r, [&ctx, &vb, &g] (std::unique_ptr req) { auto keyspace = validate_keyspace(ctx, req); auto view = req->get_path_param("view"); - return vb.local().view_build_statuses(std::move(keyspace), std::move(view)).then([] (std::unordered_map status) { + return vb.local().view_build_statuses(std::move(keyspace), std::move(view), g.local()).then([] (std::unordered_map status) { std::vector res; return make_ready_future(map_to_key_value(std::move(status), res)); }); diff --git a/api/storage_service.hh b/api/storage_service.hh index 339bf8f37a6f..b26795a21bb9 100644 --- a/api/storage_service.hh +++ b/api/storage_service.hh @@ -75,7 +75,7 @@ void set_storage_service(http_context& ctx, httpd::routes& r, sharded& sst_loader); void unset_sstables_loader(http_context& ctx, httpd::routes& r); -void set_view_builder(http_context& ctx, httpd::routes& r, sharded& vb); +void set_view_builder(http_context& ctx, httpd::routes& r, sharded& vb, sharded& g); void unset_view_builder(http_context& ctx, httpd::routes& r); void set_repair(http_context& ctx, httpd::routes& r, sharded& repair, sharded& am); void unset_repair(http_context& ctx, httpd::routes& r); diff --git a/db/view/view.cc b/db/view/view.cc index 7f73336234ba..74b72dd1c27e 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -2434,14 +2434,14 @@ future> view_builder::view_status( } future> -view_builder::view_build_statuses(sstring keyspace, sstring view_name) const { +view_builder::view_build_statuses(sstring keyspace, sstring view_name, const gms::gossiper& gossiper) const { std::unordered_map status = co_await view_status(std::move(keyspace), std::move(view_name)); std::unordered_map status_map; const auto& topo = _db.get_token_metadata().get_topology(); topo.for_each_node([&] (const locator::node& node) { auto it = status.find(node.host_id()); auto s = it != status.end() ? std::move(it->second) : "UNKNOWN"; - status_map.emplace(fmt::to_string(node.endpoint()), std::move(s)); + status_map.emplace(fmt::to_string(gossiper.get_address_map().get(node.host_id())), std::move(s)); }); co_return status_map; } diff --git a/db/view/view_builder.hh b/db/view/view_builder.hh index 89dfddc070f7..6b26a6eaa4ae 100644 --- a/db/view/view_builder.hh +++ b/db/view/view_builder.hh @@ -235,7 +235,7 @@ public: // For tests future<> wait_until_built(const sstring& ks_name, const sstring& view_name); - future> view_build_statuses(sstring keyspace, sstring view_name) const; + future> view_build_statuses(sstring keyspace, sstring view_name, const gms::gossiper& g) const; // Can only be called on shard-0 future<> mark_existing_views_as_built(); diff --git a/main.cc b/main.cc index 700cb6d83fb6..36c2694f73ac 100644 --- a/main.cc +++ b/main.cc @@ -2289,7 +2289,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl view_builder.invoke_on_all(&db::view::view_builder::start, std::ref(mm), utils::cross_shard_barrier()).get(); } - api::set_server_view_builder(ctx, view_builder).get(); + api::set_server_view_builder(ctx, view_builder, gossiper).get(); auto stop_vb_api = defer_verbose_shutdown("view builder API", [&ctx] { api::unset_server_view_builder(ctx).get(); }); From 5cd3627baafb2051ee6528ed9fa1178fdf3909d9 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 16:23:47 +0200 Subject: [PATCH 353/397] locator: drop unused function from tablet_effective_replication_map --- locator/tablets.cc | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/locator/tablets.cc b/locator/tablets.cc index 935a171f88fd..bdddae1ae7d7 100644 --- a/locator/tablets.cc +++ b/locator/tablets.cc @@ -765,19 +765,6 @@ class tablet_effective_replication_map : public effective_replication_map { tablet_sharder _sharder; mutable const tablet_map* _tmap = nullptr; private: - inet_address_vector_replica_set to_replica_set(const tablet_replica_set& replicas) const { - inet_address_vector_replica_set result; - result.reserve(replicas.size()); - auto& topo = _tmptr->get_topology(); - for (auto&& replica : replicas) { - auto* node = topo.find_node(replica.host); - if (node && !node->left()) { - result.emplace_back(node->endpoint()); - } - } - return result; - } - host_id_vector_replica_set to_host_set(const tablet_replica_set& replicas) const { host_id_vector_replica_set result; result.reserve(replicas.size()); From 83d15b8e32ffe911cf486e26d80212ce78d887e1 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 16:34:15 +0200 Subject: [PATCH 354/397] cql3: report host id instead of ip in error during SELECT FROM MUTATION_FRAGMENTS query We want to drop ip from the topology::node. --- cql3/statements/select_statement.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cql3/statements/select_statement.cc b/cql3/statements/select_statement.cc index 25d40eaf53f4..662bdeaf70ab 100644 --- a/cql3/statements/select_statement.cc +++ b/cql3/statements/select_statement.cc @@ -1820,7 +1820,7 @@ mutation_fragments_select_statement::do_execute(query_processor& qp, service::qu throw exceptions::invalid_request_exception(seastar::format( "Moving between coordinators is not allowed in SELECT FROM MUTATION_FRAGMENTS() statements, last page's coordinator was {}{}", last_host, - last_node ? fmt::format("({})", last_node->endpoint()) : "")); + last_node ? fmt::format("({})", last_node->host_id()) : "")); } } } From 49fa1130ef591c9eebd21be2fbe4bab303ed321d Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 16:35:13 +0200 Subject: [PATCH 355/397] topology coordinator: change connection dropping code to work on host ids Do not use ip from topology::node, but look it up in address map instead. We want to drop ip from the topology::node. --- service/storage_service.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 7e331106bb0a..8dc6a1783f28 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -780,9 +780,9 @@ future<> storage_service::topology_state_load(state_change_hint hint) { { std::vector> futures; get_token_metadata_ptr()->get_topology().for_each_node([&](const locator::node& n) { - const auto ep = n.endpoint(); - if (ep != inet_address{} && !saved_tmpr->get_topology().has_endpoint(ep)) { - futures.push_back(remove_rpc_client_with_ignored_topology(ep, n.host_id())); + const auto ep = n.host_id(); + if (auto ip_opt = _address_map.find(ep); ip_opt && !saved_tmpr->get_topology().has_node(ep)) { + futures.push_back(remove_rpc_client_with_ignored_topology(*ip_opt, n.host_id())); } }); co_await when_all_succeed(futures.begin(), futures.end()).discard_result(); From 163099678e0bb464d519689df5c12ad8fd0a5649 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 16:39:39 +0200 Subject: [PATCH 356/397] storage_proxy: translate id to ip using address map in tablets's describe_ring code instead of taking one from the topology We want to drop ip from the locator::node. --- service/storage_service.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index 8dc6a1783f28..c5b5fe3b9357 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -5278,10 +5278,11 @@ storage_service::describe_ring_for_table(const sstring& keyspace_name, const sst for (auto& r : replicas) { dht::endpoint_details details; const auto& node = topology.get_node(r.host); + const auto ip = _address_map.get(r.host); details._datacenter = node.dc_rack().dc; details._rack = node.dc_rack().rack; - details._host = node.endpoint(); - tr._rpc_endpoints.push_back(_gossiper.get_rpc_address(node.endpoint())); + details._host = ip; + tr._rpc_endpoints.push_back(_gossiper.get_rpc_address(ip)); tr._endpoints.push_back(fmt::to_string(details._host)); tr._endpoint_details.push_back(std::move(details)); } From fb28ff5176cff527ca1948a6ba40f5d6d68cf002 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 17:06:21 +0200 Subject: [PATCH 357/397] storage_service: check for outdated ip in on_change notification in the peers table The code checks that it does not run for an ip address that is no longer in use (after ip address change). To check that we can use peers table and see if the host id is mapped to the address. If yes, this is the latest address for this host id otherwise this is an outdated entry. --- service/storage_service.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index c5b5fe3b9357..de306755a53e 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2694,14 +2694,14 @@ future<> storage_service::on_change(gms::inet_address endpoint, const gms::appli const auto host_id = _gossiper.get_host_id(endpoint); const auto& tm = get_token_metadata(); const auto* node = tm.get_topology().find_node(host_id); - // The check node->endpoint() == endpoint is needed when a node changes + // The check peers[host_id] == endpoint is needed when a node changes // its IP - on_change can be called by the gossiper for old IP as part // of its removal, after handle_state_normal has already been called for // the new one. Without the check, the do_update_system_peers_table call // overwrites the IP back to its old value. // In essence, the code under the 'if' should fire if the given IP belongs // to a cluster member. - if (node && node->is_member() && node->endpoint() == endpoint) { + if (node && node->is_member() && (co_await get_ip_from_peers_table(host_id)) == endpoint) { if (!is_me(endpoint)) { slogger.debug("endpoint={}/{} on_change: updating system.peers table", endpoint, host_id); if (auto info = get_peer_info_for_update(endpoint, states)) { From db73758655ecd45dbcbc3f6c4cdc2ae3406af73b Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 17:18:43 +0200 Subject: [PATCH 358/397] locator: topology: remove unused functions --- locator/topology.cc | 11 ----------- locator/topology.hh | 6 ------ 2 files changed, 17 deletions(-) diff --git a/locator/topology.cc b/locator/topology.cc index 6c1417ca6eb1..b4c9b971ca90 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -544,17 +544,6 @@ bool topology::has_node(host_id id) const noexcept { return bool(node); } -bool topology::has_node(inet_address ep) const noexcept { - auto node = find_node(ep); - tlogger.trace("topology[{}]: has_node: endpoint={}: node={}", fmt::ptr(this), ep, node_printer(node)); - return bool(node); -} - -bool topology::has_endpoint(inet_address ep) const -{ - return has_node(ep); -} - void topology::sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const { if (can_sort_by_proximity()) { do_sort_by_proximity(address, addresses); diff --git a/locator/topology.hh b/locator/topology.hh index 8163c0e7064f..27aaa6992645 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -251,7 +251,6 @@ public: // Returns true if a node with given host_id is found bool has_node(host_id id) const noexcept; - bool has_node(inet_address id) const noexcept; /** * Stores current DC/rack assignment for ep @@ -265,11 +264,6 @@ public: bool remove_endpoint(locator::host_id ep); - /** - * Returns true iff contains given endpoint. - */ - bool has_endpoint(inet_address) const; - const std::unordered_map>& get_datacenter_endpoints() const { From d45ce6fa1240bb744acd9ab4c08315c7d1cbe063 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 17:35:33 +0200 Subject: [PATCH 359/397] storage_proxy: translate ips to ids in forward array using gossiper We already use it to translate reply_to, so do it for consistency and to drop ip based API usage. --- service/storage_proxy.cc | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc index 31d041137059..5cb726b54d89 100644 --- a/service/storage_proxy.cc +++ b/service/storage_proxy.cc @@ -105,13 +105,13 @@ namespace { template -utils::small_vector addr_vector_to_id(const locator::topology& topo, const utils::small_vector& set) { +utils::small_vector addr_vector_to_id(const gms::gossiper& g, const utils::small_vector& set) { return set | std::views::transform([&] (gms::inet_address ip) { - auto* node = topo.find_node(ip); - if (!node) { + try { + return g.get_host_id(ip); + } catch (...) { on_internal_error(slogger, fmt::format("addr_vector_to_id cannot map {} to host id", ip)); } - return node->host_id(); }) | std::ranges::to>(); } @@ -587,7 +587,7 @@ class storage_proxy::remote { } auto reply_to_host_id = reply_to_id ? *reply_to_id : _gossiper.get_host_id(reply_to); - auto forward_host_id = forward_id ? std::move(*forward_id) : addr_vector_to_id(_sp._shared_token_metadata.get()->get_topology(), forward); + auto forward_host_id = forward_id ? std::move(*forward_id) : addr_vector_to_id(_gossiper, forward); if (reply_to_id) { _gossiper.get_mutable_address_map().opt_add_entry(reply_to_host_id, reply_to); From 12da203cae18807b266e760cdcaba515ec538b65 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 17:36:36 +0200 Subject: [PATCH 360/397] storage_service: use host_id to look for a node in on_alive handler --- service/storage_service.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/service/storage_service.cc b/service/storage_service.cc index de306755a53e..c65fb5f26578 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -2633,7 +2633,7 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta const auto& tm = get_token_metadata(); const auto host_id = state->get_host_id(); slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, host_id, pid); - const auto* node = tm.get_topology().find_node(endpoint); + const auto* node = tm.get_topology().find_node(host_id); if (node && node->is_member()) { co_await notify_up(endpoint); } else if (raft_topology_change_enabled()) { From f9df092fd1c4b8c54416b252f81f45146faf8451 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 18:31:44 +0200 Subject: [PATCH 361/397] repair: drop unneeded code There is a code that creates a map from id to ip and then creates a vector from the keys of the map. Create a vector directly instead. --- repair/repair.cc | 6 +++--- repair/repair.hh | 3 --- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/repair/repair.cc b/repair/repair.cc index 51c0aa8d7db4..bd54cb6d9246 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -2150,11 +2150,11 @@ future<> repair_service::do_rebuild_replace_with_repair(std::unordered_map(); + return n.host_id(); + }) | std::ranges::to(); rlogger.debug("{}: keyspace={}, range={}, natural_enpoints={}, neighbors={}", op, keyspace_name, r, natural_eps, neighbors); if (!neighbors.empty()) { - range_sources[r] = repair_neighbors(neighbors); + range_sources[r] = repair_neighbors(std::move(neighbors)); ++it; } else { // Skip the range with zero neighbors diff --git a/repair/repair.hh b/repair/repair.hh index a65c4acc9299..942ecc435b00 100644 --- a/repair/repair.hh +++ b/repair/repair.hh @@ -143,9 +143,6 @@ public: explicit repair_neighbors(std::vector a) : all(std::move(a)) { } - explicit repair_neighbors(const std::unordered_map& a) - : all(a | std::views::keys | std::ranges::to>()) { - } repair_neighbors(std::vector a, std::vector m) : all(std::move(a)) , mandatory(std::move(m)) { From 50fb22c8f9fda42f16f1e94fc6e9566cf38361f4 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Sun, 5 Jan 2025 18:33:00 +0200 Subject: [PATCH 362/397] locator: topology: drop indexing by ips Do not track id to ip mapping in the topology class any longer. There are no remaining users. --- locator/token_metadata.cc | 17 +--- locator/topology.cc | 97 +++----------------- locator/topology.hh | 27 +----- main.cc | 3 +- test/boost/locator_topology_test.cc | 97 ++++---------------- test/boost/network_topology_strategy_test.cc | 30 ++---- test/boost/tablets_test.cc | 6 +- test/lib/cql_test_env.cc | 3 +- test/perf/perf_sort_by_proximity.cc | 1 - 9 files changed, 52 insertions(+), 229 deletions(-) diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index 27b8e9912859..519a66475492 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -116,7 +116,7 @@ class token_metadata_impl final { } void update_topology(host_id id, std::optional opt_dr, std::optional opt_st, std::optional shard_count = std::nullopt) { - _topology.add_or_update_endpoint(id, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count)); + _topology.add_or_update_endpoint(id, std::move(opt_dr), std::move(opt_st), std::move(shard_count)); } /** @@ -141,15 +141,6 @@ class token_metadata_impl final { void debug_show() const; - /** - * Store an end-point to host ID mapping. Each ID must be unique, and - * cannot be changed after the fact. - * - * @param hostId - * @param endpoint - */ - void update_host_id(const host_id& host_id, inet_address endpoint); - /** @return a copy of host id set for read-only operations */ std::unordered_set get_host_ids() const; @@ -516,10 +507,6 @@ void token_metadata_impl::debug_show() const { reporter->arm_periodic(std::chrono::seconds(1)); } -void token_metadata_impl::update_host_id(const host_id& host_id, inet_address endpoint) { - _topology.add_or_update_endpoint(host_id, endpoint); -} - std::unordered_set token_metadata_impl::get_host_ids() const { return _topology.get_nodes() | std::views::filter([&] (const node& n) { return !n.left() && !n.is_none(); }) | @@ -964,7 +951,7 @@ token_metadata::debug_show() const { void token_metadata::update_host_id(const host_id& host_id, inet_address endpoint) { - _impl->update_host_id(host_id, endpoint); + // Do nothing for now. Remove later. } std::unordered_set diff --git a/locator/topology.cc b/locator/topology.cc index b4c9b971ca90..caa4f1d2e08b 100644 --- a/locator/topology.cc +++ b/locator/topology.cc @@ -55,10 +55,9 @@ thread_local const endpoint_dc_rack endpoint_dc_rack::default_location = { .rack = locator::production_snitch_base::default_rack, }; -node::node(const locator::topology* topology, locator::host_id id, inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count, this_node is_this_node, node::idx_type idx) +node::node(const locator::topology* topology, locator::host_id id, endpoint_dc_rack dc_rack, state state, shard_id shard_count, this_node is_this_node, node::idx_type idx) : _topology(topology) , _host_id(id) - , _endpoint(endpoint) , _dc_rack(std::move(dc_rack)) , _state(state) , _shard_count(std::move(shard_count)) @@ -66,12 +65,12 @@ node::node(const locator::topology* topology, locator::host_id id, inet_address , _idx(idx) {} -node_holder node::make(const locator::topology* topology, locator::host_id id, inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count, node::this_node is_this_node, node::idx_type idx) { - return std::make_unique(topology, std::move(id), std::move(endpoint), std::move(dc_rack), std::move(state), shard_count, is_this_node, idx); +node_holder node::make(const locator::topology* topology, locator::host_id id, endpoint_dc_rack dc_rack, state state, shard_id shard_count, node::this_node is_this_node, node::idx_type idx) { + return std::make_unique(topology, std::move(id), std::move(dc_rack), std::move(state), shard_count, is_this_node, idx); } node_holder node::clone() const { - return make(nullptr, host_id(), endpoint(), dc_rack(), get_state(), get_shard_count(), is_this_node()); + return make(nullptr, host_id(), dc_rack(), get_state(), get_shard_count(), is_this_node()); } std::string node::to_string(node::state s) { @@ -95,7 +94,6 @@ future<> topology::clear_gently() noexcept { _datacenters.clear(); _dc_rack_nodes.clear(); _dc_nodes.clear(); - _nodes_by_endpoint.clear(); _nodes_by_host_id.clear(); co_await utils::clear_gently(_nodes); } @@ -116,7 +114,7 @@ topology::topology(config cfg) { tlogger.trace("topology[{}]: constructing using config: endpoint={} id={} dc={} rack={}", fmt::ptr(this), cfg.this_endpoint, cfg.this_host_id, cfg.local_dc_rack.dc, cfg.local_dc_rack.rack); - add_node(cfg.this_host_id, cfg.this_endpoint, cfg.local_dc_rack, node::state::none); + add_node(cfg.this_host_id, cfg.local_dc_rack, node::state::none); } topology::topology(topology&& o) noexcept @@ -125,7 +123,6 @@ topology::topology(topology&& o) noexcept , _this_node(std::exchange(o._this_node, nullptr)) , _nodes(std::move(o._nodes)) , _nodes_by_host_id(std::move(o._nodes_by_host_id)) - , _nodes_by_endpoint(std::move(o._nodes_by_endpoint)) , _dc_nodes(std::move(o._dc_nodes)) , _dc_rack_nodes(std::move(o._dc_rack_nodes)) , _dc_endpoints(std::move(o._dc_endpoints)) @@ -171,7 +168,7 @@ void topology::set_host_id_cfg(host_id this_host_id) { tlogger.trace("topology[{}]: set host id to {}", fmt::ptr(this), this_host_id); _cfg.this_host_id = this_host_id; - add_or_update_endpoint(this_host_id, _cfg.this_endpoint); + add_or_update_endpoint(this_host_id); } future topology::clone_gently() const { @@ -188,21 +185,15 @@ future topology::clone_gently() const { co_return ret; } -const node& topology::add_node(host_id id, const inet_address& ep, const endpoint_dc_rack& dr, node::state state, shard_id shard_count) { +const node& topology::add_node(host_id id, const endpoint_dc_rack& dr, node::state state, shard_id shard_count) { if (dr.dc.empty() || dr.rack.empty()) { on_internal_error(tlogger, "Node must have valid dc and rack"); } - return add_node(node::make(this, id, ep, dr, state, shard_count)); + return add_node(node::make(this, id, dr, state, shard_count)); } bool topology::is_configured_this_node(const node& n) const { - if (_cfg.this_host_id && n.host_id()) { // Selection by host_id - return _cfg.this_host_id == n.host_id(); - } - if (_cfg.this_endpoint != inet_address()) { // Selection by endpoint - return _cfg.this_endpoint == n.endpoint(); - } - return false; // No selection; + return _cfg.this_host_id == n.host_id(); } const node& topology::add_node(node_holder nptr) { @@ -245,10 +236,9 @@ const node& topology::add_node(node_holder nptr) { return *node; } -void topology::update_node(node& node, std::optional opt_id, std::optional opt_ep, std::optional opt_dr, std::optional opt_st, std::optional opt_shard_count) { - tlogger.debug("topology[{}]: update_node: {}: to: host_id={} endpoint={} dc={} rack={} state={} shard_count={}, at {}", fmt::ptr(this), node_printer(&node), +void topology::update_node(node& node, std::optional opt_id, std::optional opt_dr, std::optional opt_st, std::optional opt_shard_count) { + tlogger.debug("topology[{}]: update_node: {}: to: host_id={} dc={} rack={} state={} shard_count={}, at {}", fmt::ptr(this), node_printer(&node), opt_id ? format("{}", *opt_id) : "unchanged", - opt_ep ? format("{}", *opt_ep) : "unchanged", opt_dr ? format("{}", opt_dr->dc) : "unchanged", opt_dr ? format("{}", opt_dr->rack) : "unchanged", opt_st ? format("{}", *opt_st) : "unchanged", @@ -272,16 +262,6 @@ void topology::update_node(node& node, std::optional opt_id, std::optio opt_id.reset(); } } - if (opt_ep) { - if (*opt_ep != node.endpoint()) { - if (*opt_ep == inet_address{}) { - on_internal_error(tlogger, seastar::format("Updating node endpoint to null is disallowed: {}: new endpoint={}", node_printer(&node), *opt_ep)); - } - changed = true; - } else { - opt_ep.reset(); - } - } if (opt_dr) { if (opt_dr->dc.empty() || opt_dr->dc == production_snitch_base::default_dc) { opt_dr->dc = node.dc_rack().dc; @@ -312,9 +292,6 @@ void topology::update_node(node& node, std::optional opt_id, std::optio if (opt_id) { node._host_id = *opt_id; } - if (opt_ep) { - node._endpoint = *opt_ep; - } if (opt_dr) { node._dc_rack = std::move(*opt_dr); } @@ -360,32 +337,6 @@ void topology::index_node(const node& node) { if (!inserted_host_id) { on_internal_error(tlogger, seastar::format("topology[{}]: {}: node already exists", fmt::ptr(this), node_printer(&node))); } - if (node.endpoint() != inet_address{}) { - auto eit = _nodes_by_endpoint.find(node.endpoint()); - if (eit != _nodes_by_endpoint.end()) { - if (eit->second.get().get_state() == node::state::none && eit->second.get().is_this_node()) { - // eit->second is default entry created for local node and it is replaced by existing node with the same ip - // it means this node is going to replace the existing node with the same ip, but it does not know it yet - // map ip to the old node - _nodes_by_endpoint.erase(node.endpoint()); - } else if (eit->second.get().get_state() == node::state::replacing && node.get_state() == node::state::being_replaced) { - // replace-with-same-ip, map ip to the old node - _nodes_by_endpoint.erase(node.endpoint()); - } else if (eit->second.get().get_state() == node::state::being_replaced && node.get_state() == node::state::replacing) { - // replace-with-same-ip, map ip to the old node, do nothing if it's already the case - } else if (eit->second.get().is_leaving() || eit->second.get().left()) { - _nodes_by_endpoint.erase(node.endpoint()); - } else if (!node.is_leaving() && !node.left()) { - if (node.host_id()) { - _nodes_by_host_id.erase(node.host_id()); - } - on_internal_error(tlogger, seastar::format("topology[{}]: {}: node endpoint already mapped to {}", fmt::ptr(this), node_printer(&node), node_printer(&eit->second.get()))); - } - } - if (!node.left() && !node.is_none()) { - _nodes_by_endpoint.try_emplace(node.endpoint(), std::cref(node)); - } - } // We keep location of left nodes because they may still appear in tablet replica sets // and algorithms expect to know which dc they belonged to. View replica pairing needs stable @@ -442,10 +393,6 @@ void topology::unindex_node(const node& node) { if (host_it != _nodes_by_host_id.end() && host_it->second == node) { _nodes_by_host_id.erase(host_it); } - auto ep_it = _nodes_by_endpoint.find(node.endpoint()); - if (ep_it != _nodes_by_endpoint.end() && ep_it->second.get() == node) { - _nodes_by_endpoint.erase(ep_it); - } if (_this_node == &node) { _this_node = nullptr; } @@ -484,16 +431,6 @@ node* topology::find_node(host_id id) noexcept { return make_mutable(const_cast(this)->find_node(id)); } -// Finds a node by its endpoint -// Returns nullptr if not found -const node* topology::find_node(const inet_address& ep) const noexcept { - auto it = _nodes_by_endpoint.find(ep); - if (it != _nodes_by_endpoint.end()) { - return &it->second.get(); - } - return nullptr; -} - // Finds a node by its index // Returns nullptr if not found const node* topology::find_node(node::idx_type idx) const noexcept { @@ -503,23 +440,19 @@ const node* topology::find_node(node::idx_type idx) const noexcept { return _nodes.at(idx).get(); } -const node& topology::add_or_update_endpoint(host_id id, std::optional opt_ep, std::optional opt_dr, std::optional opt_st, std::optional shard_count) +const node& topology::add_or_update_endpoint(host_id id, std::optional opt_dr, std::optional opt_st, std::optional shard_count) { - tlogger.trace("topology[{}]: add_or_update_endpoint: host_id={} ep={} dc={} rack={} state={} shards={}, at {}", fmt::ptr(this), - id, opt_ep, opt_dr.value_or(endpoint_dc_rack{}).dc, opt_dr.value_or(endpoint_dc_rack{}).rack, opt_st.value_or(node::state::none), shard_count, + tlogger.trace("topology[{}]: add_or_update_endpoint: host_id={} dc={} rack={} state={} shards={}, at {}", fmt::ptr(this), + id, opt_dr.value_or(endpoint_dc_rack{}).dc, opt_dr.value_or(endpoint_dc_rack{}).rack, opt_st.value_or(node::state::none), shard_count, lazy_backtrace()); auto* n = find_node(id); if (n) { - update_node(*n, std::nullopt, opt_ep, std::move(opt_dr), std::move(opt_st), std::move(shard_count)); - return *n; - } else if (opt_ep && (n = make_mutable(find_node(*opt_ep)))) { - update_node(*n, id, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count)); + update_node(*n, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count)); return *n; } return add_node(id, - opt_ep.value_or(inet_address{}), opt_dr.value_or(endpoint_dc_rack::default_location), opt_st.value_or(node::state::none), shard_count.value_or(0)); diff --git a/locator/topology.hh b/locator/topology.hh index 27aaa6992645..d1c141b506cc 100644 --- a/locator/topology.hh +++ b/locator/topology.hh @@ -59,7 +59,6 @@ public: private: const locator::topology* _topology; locator::host_id _host_id; - inet_address _endpoint; endpoint_dc_rack _dc_rack; state _state; shard_id _shard_count = 0; @@ -72,7 +71,6 @@ private: public: node(const locator::topology* topology, locator::host_id id, - inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count = 0, @@ -94,10 +92,6 @@ public: return _host_id; } - const inet_address& endpoint() const noexcept { - return _endpoint; - } - const endpoint_dc_rack& dc_rack() const noexcept { return _dc_rack; } @@ -164,7 +158,6 @@ public: private: static node_holder make(const locator::topology* topology, locator::host_id id, - inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count = 0, @@ -211,7 +204,7 @@ public: } // Adds a node with given host_id, endpoint, and DC/rack. - const node& add_node(host_id id, const inet_address& ep, const endpoint_dc_rack& dr, node::state state, + const node& add_node(host_id id, const endpoint_dc_rack& dr, node::state state, shard_id shard_count = 0); // Optionally updates node's current host_id, endpoint, or DC/rack. @@ -219,7 +212,6 @@ public: // or a peer node host_id may be updated when the node is replaced with another node using the same ip address. void update_node(node& node, std::optional opt_id, - std::optional opt_ep, std::optional opt_dr, std::optional opt_st, std::optional opt_shard_count = std::nullopt); @@ -241,10 +233,6 @@ public: return *n; }; - // Looks up a node by its inet_address. - // Returns a pointer to the node if found, or nullptr otherwise. - const node* find_node(const inet_address& ep) const noexcept; - // Finds a node by its index // Returns a pointer to the node if found, or nullptr otherwise. const node* find_node(node::idx_type idx) const noexcept; @@ -257,8 +245,7 @@ public: * * Adds or updates a node with given endpoint */ - const node& add_or_update_endpoint(host_id id, std::optional opt_ep, - std::optional opt_dr = std::nullopt, + const node& add_or_update_endpoint(host_id id, std::optional opt_dr = std::nullopt, std::optional opt_st = std::nullopt, std::optional shard_count = std::nullopt); @@ -411,7 +398,6 @@ private: const node* _this_node = nullptr; std::vector _nodes; std::unordered_map> _nodes_by_host_id; - std::unordered_map> _nodes_by_endpoint; std::unordered_map>> _dc_nodes; std::unordered_map>>> _dc_rack_nodes; @@ -434,10 +420,6 @@ private: void calculate_datacenters(); - const std::unordered_map>& get_nodes_by_endpoint() const noexcept { - return _nodes_by_endpoint; - }; - mutable random_engine_type _random_engine; friend class token_metadata_impl; @@ -491,12 +473,11 @@ struct fmt::formatter : fmt::formatter { template auto format(const locator::node& node, FormatContext& ctx) const { if (!verbose) { - return fmt::format_to(ctx.out(), "{}/{}", node.host_id(), node.endpoint()); + return fmt::format_to(ctx.out(), "{}", node.host_id()); } else { - return fmt::format_to(ctx.out(), " idx={} host_id={} endpoint={} dc={} rack={} state={} shards={} this_node={}", + return fmt::format_to(ctx.out(), " idx={} host_id={} dc={} rack={} state={} shards={} this_node={}", node.idx(), node.host_id(), - node.endpoint(), node.dc_rack().dc, node.dc_rack().rack, locator::node::to_string(node.get_state()), diff --git a/main.cc b/main.cc index 36c2694f73ac..4e6b682151d9 100644 --- a/main.cc +++ b/main.cc @@ -1465,12 +1465,11 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl const auto listen_address = utils::resolve(cfg->listen_address, family).get(); const auto host_id = initialize_local_info_thread(sys_ks, snitch, listen_address, *cfg, broadcast_addr, broadcast_rpc_addr); - shared_token_metadata::mutate_on_all_shards(token_metadata, [host_id, endpoint = broadcast_addr] (locator::token_metadata& tm) { + shared_token_metadata::mutate_on_all_shards(token_metadata, [host_id] (locator::token_metadata& tm) { // Makes local host id available in topology cfg as soon as possible. // Raft topology discard the endpoint-to-id map, so the local id can // still be found in the config. tm.get_topology().set_host_id_cfg(host_id); - tm.get_topology().add_or_update_endpoint(host_id, endpoint); return make_ready_future<>(); }).get(); diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc index ccb321a11d7e..f42dd7435626 100644 --- a/test/boost/locator_topology_test.cc +++ b/test/boost/locator_topology_test.cc @@ -32,9 +32,7 @@ SEASTAR_THREAD_TEST_CASE(test_add_node) { auto id1 = host_id::create_random_id(); auto ep1 = gms::inet_address("127.0.0.1"); auto id2 = host_id::create_random_id(); - auto ep2 = gms::inet_address("127.0.0.2"); auto id3 = host_id::create_random_id(); - auto ep3 = gms::inet_address("127.0.0.3"); topology::config cfg = { .this_endpoint = ep1, @@ -51,16 +49,13 @@ SEASTAR_THREAD_TEST_CASE(test_add_node) { std::unordered_set> nodes; - nodes.insert(std::cref(topo.add_node(id2, ep2, endpoint_dc_rack::default_location, node::state::normal))); - nodes.insert(std::cref(topo.add_or_update_endpoint(id1, ep1, endpoint_dc_rack::default_location, node::state::normal))); + nodes.insert(std::cref(topo.add_node(id2, endpoint_dc_rack::default_location, node::state::normal))); + nodes.insert(std::cref(topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal))); - BOOST_REQUIRE_THROW(topo.add_node(id1, ep2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error); - BOOST_REQUIRE_THROW(topo.add_node(id2, ep1, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error); - BOOST_REQUIRE_THROW(topo.add_node(id2, ep2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error); - BOOST_REQUIRE_THROW(topo.add_node(id2, ep3, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error); - BOOST_REQUIRE_THROW(topo.add_node(id3, ep3, endpoint_dc_rack{}, node::state::normal), std::runtime_error); + BOOST_REQUIRE_THROW(topo.add_node(id2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error); + BOOST_REQUIRE_THROW(topo.add_node(id3, endpoint_dc_rack{}, node::state::normal), std::runtime_error); - nodes.insert(std::cref(topo.add_node(id3, ep3, endpoint_dc_rack::default_location, node::state::normal))); + nodes.insert(std::cref(topo.add_node(id3, endpoint_dc_rack::default_location, node::state::normal))); topo.for_each_node([&] (const locator::node& node) { BOOST_REQUIRE(nodes.erase(std::cref(node))); @@ -82,7 +77,7 @@ SEASTAR_THREAD_TEST_CASE(test_moving) { auto topo = topology(cfg); - topo.add_or_update_endpoint(id1, ep1, endpoint_dc_rack::default_location, node::state::normal); + topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal); BOOST_REQUIRE(topo.this_node()->topology() == &topo); @@ -101,8 +96,6 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) { auto id1 = host_id::create_random_id(); auto ep1 = gms::inet_address("127.0.0.1"); auto id2 = host_id::create_random_id(); - auto ep2 = gms::inet_address("127.0.0.2"); - auto ep3 = gms::inet_address("127.0.0.3"); topology::config cfg = { .this_endpoint = ep1, @@ -117,93 +110,42 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) { set_abort_on_internal_error(true); }); - topo.add_or_update_endpoint(id1, std::nullopt, endpoint_dc_rack::default_location, node::state::normal); + topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal); auto node = const_cast(topo.this_node()); - topo.update_node(*node, std::nullopt, ep1, std::nullopt, std::nullopt); + topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt); BOOST_REQUIRE_EQUAL(topo.find_node(id1), node); - BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt, std::nullopt), std::runtime_error); - BOOST_REQUIRE_THROW(topo.update_node(*node, id2, std::nullopt, std::nullopt, std::nullopt), std::runtime_error); + BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt), std::runtime_error); + BOOST_REQUIRE_THROW(topo.update_node(*node, id2, std::nullopt, std::nullopt), std::runtime_error); BOOST_REQUIRE_EQUAL(topo.find_node(id1), node); BOOST_REQUIRE_EQUAL(topo.find_node(id2), nullptr); - topo.update_node(*node, std::nullopt, ep2, std::nullopt, std::nullopt); - BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr); - BOOST_REQUIRE_EQUAL(topo.find_node(ep2), node); - auto dc_rack1 = endpoint_dc_rack{"DC1", "RACK1"}; - topo.update_node(*node, std::nullopt, std::nullopt, dc_rack1, std::nullopt); + topo.update_node(*node, std::nullopt, dc_rack1, std::nullopt); BOOST_REQUIRE(topo.get_location(id1) == dc_rack1); auto dc_rack2 = endpoint_dc_rack{"DC2", "RACK2"}; - topo.update_node(*node, std::nullopt, std::nullopt, dc_rack2, std::nullopt); + topo.update_node(*node, std::nullopt, dc_rack2, std::nullopt); BOOST_REQUIRE(topo.get_location(id1) == dc_rack2); BOOST_REQUIRE_NE(node->get_state(), locator::node::state::being_decommissioned); - topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::being_decommissioned); + topo.update_node(*node, std::nullopt, std::nullopt, locator::node::state::being_decommissioned); BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned); auto dc_rack3 = endpoint_dc_rack{"DC3", "RACK3"}; // Note: engage state option, but keep node::state value the same // to reproduce #13502 - topo.update_node(*node, std::nullopt, ep3, dc_rack3, locator::node::state::being_decommissioned); + topo.update_node(*node, std::nullopt, dc_rack3, locator::node::state::being_decommissioned); BOOST_REQUIRE_EQUAL(topo.find_node(id1), node); - BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr); - BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr); - BOOST_REQUIRE_EQUAL(topo.find_node(ep3), node); BOOST_REQUIRE(topo.get_location(id1) == dc_rack3); BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned); - - // In state::left the node will remain indexed only by its host_id - topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::left); - BOOST_REQUIRE_EQUAL(topo.find_node(id1), node); - BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr); - BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr); - BOOST_REQUIRE_EQUAL(topo.find_node(ep3), nullptr); - BOOST_REQUIRE(topo.get_location(id1) == dc_rack3); - BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::left); -} - -SEASTAR_THREAD_TEST_CASE(test_add_or_update_by_host_id) { - auto id1 = host_id::create_random_id(); - auto id2 = host_id::create_random_id(); - auto ep1 = gms::inet_address("127.0.0.1"); - - // In this test we check that add_or_update_endpoint searches by host_id first. - // We create two nodes, one matches by id, another - by ip, - // and SCYLLA_ASSERT that add_or_update_endpoint updates the first. - // We need to make the second node 'being_decommissioned', so that - // it gets removed from ip index and we don't get the non-unique IP error. - - auto topo = topology({ - .this_host_id = id1, - .local_dc_rack = endpoint_dc_rack::default_location, - }); - - topo.add_or_update_endpoint(id1, gms::inet_address{}, endpoint_dc_rack::default_location, node::state::normal); - topo.add_node(id2, ep1, endpoint_dc_rack::default_location, node::state::being_decommissioned); - - topo.add_or_update_endpoint(id1, ep1, std::nullopt, node::state::bootstrapping); - - auto* n = topo.find_node(id1); - BOOST_REQUIRE_EQUAL(n->get_state(), node::state::bootstrapping); - BOOST_REQUIRE_EQUAL(n->host_id(), id1); - BOOST_REQUIRE_EQUAL(n->endpoint(), ep1); - - auto* n2 = topo.find_node(ep1); - BOOST_REQUIRE_EQUAL(n, n2); - - auto* n3 = topo.find_node(id2); - BOOST_REQUIRE_EQUAL(n3->get_state(), node::state::being_decommissioned); - BOOST_REQUIRE_EQUAL(n3->host_id(), id2); - BOOST_REQUIRE_EQUAL(n3->endpoint(), ep1); } SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) { @@ -214,7 +156,6 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) { const auto id1 = host_id::create_random_id(); const auto ep1 = gms::inet_address("127.0.0.1"); const auto id2 = host_id::create_random_id(); - const auto ep2 = gms::inet_address("127.0.0.2"); const auto dc_rack1 = endpoint_dc_rack { .dc = "dc1", .rack = "rack1" @@ -232,8 +173,8 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) { auto topo = topology(cfg); - topo.add_or_update_endpoint(id1, ep1, dc_rack1, node::state::normal); - topo.add_node(id2, ep2, dc_rack2, node::state::normal); + topo.add_or_update_endpoint(id1, dc_rack1, node::state::normal); + topo.add_node(id2, dc_rack2, node::state::normal); BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1, id2}}})); BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}, {"rack2", {id2}}}}})); @@ -374,9 +315,7 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) { auto id1 = host_id::create_random_id(); auto ep1 = gms::inet_address("127.0.0.1"); auto id2 = host_id::create_random_id(); - auto ep2 = gms::inet_address("127.0.0.2"); auto id3 = host_id::create_random_id(); - auto ep3 = gms::inet_address("127.0.0.3"); const auto dc_rack1 = endpoint_dc_rack { .dc = "dc1", @@ -397,8 +336,8 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) { std::unordered_set> nodes; - nodes.insert(std::cref(topo.add_node(id2, ep2, dc_rack1, node::state::normal))); - nodes.insert(std::cref(topo.add_node(id3, ep3, dc_rack1, node::state::left))); + nodes.insert(std::cref(topo.add_node(id2, dc_rack1, node::state::normal))); + nodes.insert(std::cref(topo.add_node(id3, dc_rack1, node::state::left))); topo.for_each_node([&] (const locator::node& node) { BOOST_REQUIRE(node.host_id() != id3); diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index a4369457c649..a3a7333d9c95 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -304,7 +304,7 @@ void simple_test() { for (const auto& [ring_point, endpoint, id] : ring_points) { std::unordered_set tokens; tokens.insert(token{tests::d2t(ring_point / ring_points.size())}); - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal); + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal); co_await tm.update_normal_tokens(std::move(tokens), id); } }).get(); @@ -412,7 +412,7 @@ void heavy_origin_test() { stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { auto& topo = tm.get_topology(); for (const auto& [ring_point, endpoint, id] : ring_points) { - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal); + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal); co_await tm.update_normal_tokens(tokens[endpoint], id); } }).get(); @@ -484,7 +484,7 @@ SEASTAR_THREAD_TEST_CASE(NetworkTopologyStrategy_tablets_test) { for (const auto& [ring_point, endpoint, id] : ring_points) { std::unordered_set tokens; tokens.insert(token{tests::d2t(ring_point / ring_points.size())}); - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count); + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count); tm.update_host_id(id, endpoint); co_await tm.update_normal_tokens(std::move(tokens), id); } @@ -576,7 +576,7 @@ static void test_random_balancing(sharded& snitch, gms::inet_address for (const auto& [ring_point, endpoint, id] : ring_points) { std::unordered_set tokens; tokens.insert(token{tests::d2t(ring_point / ring_points.size())}); - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count); + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count); tm.update_host_id(id, endpoint); co_await tm.update_normal_tokens(std::move(tokens), id); } @@ -865,12 +865,11 @@ void generate_topology(topology& topo, const std::unordered_map out = std::fill_n(out, rf, std::cref(dc)); } - unsigned i = 0; for (auto& node : nodes) { const sstring& dc = dcs[udist(0, dcs.size() - 1)(e1)]; auto rc = racks_per_dc.at(dc); auto r = udist(0, rc)(e1); - topo.add_or_update_endpoint(node, inet_address((127u << 24) | ++i), endpoint_dc_rack{dc, to_sstring(r)}, locator::node::state::normal); + topo.add_or_update_endpoint(node, endpoint_dc_rack{dc, to_sstring(r)}, locator::node::state::normal); } } @@ -1145,21 +1144,15 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { const node* n1 = stm.get()->get_topology().find_node(host1); BOOST_REQUIRE(n1); - n1 = stm.get()->get_topology().find_node(ip1); - BOOST_REQUIRE(n1); BOOST_REQUIRE(bool(n1->is_this_node())); BOOST_REQUIRE_EQUAL(n1->host_id(), host1); - BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1); BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack); BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack); const node* n2 = stm.get()->get_topology().find_node(host2); BOOST_REQUIRE(n2); - n2 = stm.get()->get_topology().find_node(ip2); - BOOST_REQUIRE(n2); BOOST_REQUIRE(!bool(n2->is_this_node())); BOOST_REQUIRE_EQUAL(n2->host_id(), host2); - BOOST_REQUIRE_EQUAL(n2->endpoint(), ip2); BOOST_REQUIRE(n2->dc_rack() == endpoint_dc_rack::default_location); // Local node cannot be removed @@ -1172,8 +1165,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { n1 = stm.get()->get_topology().find_node(host1); BOOST_REQUIRE(n1); - n1 = stm.get()->get_topology().find_node(ip1); - BOOST_REQUIRE(n1); // Removing node with no local node @@ -1184,22 +1175,19 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { n2 = stm.get()->get_topology().find_node(host2); BOOST_REQUIRE(!n2); - n2 = stm.get()->get_topology().find_node(ip2); - BOOST_REQUIRE(!n2); // Repopulate after clear_gently() stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { co_await tm.clear_gently(); - tm.update_host_id(host2, ip2); - tm.update_host_id(host1, ip1); // this_node added last on purpose + tm.update_topology(host2, std::nullopt, std::nullopt); + tm.update_topology(host1, std::nullopt, std::nullopt); // this_node added last on purpose }).get(); n1 = stm.get()->get_topology().find_node(host1); BOOST_REQUIRE(n1); BOOST_REQUIRE(bool(n1->is_this_node())); BOOST_REQUIRE_EQUAL(n1->host_id(), host1); - BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1); BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack); BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack); @@ -1207,21 +1195,19 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { BOOST_REQUIRE(n2); BOOST_REQUIRE(!bool(n2->is_this_node())); BOOST_REQUIRE_EQUAL(n2->host_id(), host2); - BOOST_REQUIRE_EQUAL(n2->endpoint(), ip2); BOOST_REQUIRE(n2->dc_rack() == endpoint_dc_rack::default_location); // get_location() should pick up endpoint_dc_rack from node info stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { co_await tm.clear_gently(); - tm.get_topology().add_or_update_endpoint(host1, ip1, ip1_dc_rack_v2, node::state::being_decommissioned); + tm.get_topology().add_or_update_endpoint(host1, ip1_dc_rack_v2, node::state::being_decommissioned); }).get(); n1 = stm.get()->get_topology().find_node(host1); BOOST_REQUIRE(n1); BOOST_REQUIRE(bool(n1->is_this_node())); BOOST_REQUIRE_EQUAL(n1->host_id(), host1); - BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1); BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack_v2); BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack_v2); } diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 3e4c241810f7..1e8b62f8b525 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -1020,7 +1020,7 @@ SEASTAR_TEST_CASE(test_sharder) { auto table1 = table_id(utils::UUID_gen::get_time_UUID()); token_metadata tokm(token_metadata::config{ .topo_cfg{ .this_host_id = h1, .local_dc_rack = locator::endpoint_dc_rack::default_location } }); - tokm.get_topology().add_or_update_endpoint(h1, tokm.get_topology().my_address()); + tokm.get_topology().add_or_update_endpoint(h1); std::vector tablet_ids; { @@ -1235,7 +1235,7 @@ SEASTAR_TEST_CASE(test_intranode_sharding) { auto table1 = table_id(utils::UUID_gen::get_time_UUID()); token_metadata tokm(token_metadata::config{ .topo_cfg{ .this_host_id = h1, .local_dc_rack = locator::endpoint_dc_rack::default_location } }); - tokm.get_topology().add_or_update_endpoint(h1, tokm.get_topology().my_address()); + tokm.get_topology().add_or_update_endpoint(h1); auto leaving_replica = tablet_replica{h1, 5}; auto pending_replica = tablet_replica{h1, 7}; @@ -3340,7 +3340,7 @@ static void execute_tablet_for_new_rf_test(calculate_tablet_replicas_for_new_rf_ for (const auto& [ring_point, endpoint, id] : test_config.ring_points) { std::unordered_set tokens; tokens.insert(dht::token{tests::d2t(ring_point / test_config.ring_points.size())}); - topo.add_or_update_endpoint(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1); + topo.add_or_update_endpoint(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1); tm.update_host_id(id, endpoint); co_await tm.update_normal_tokens(std::move(tokens), id); } diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc index 82f597134087..07674fcfc48a 100644 --- a/test/lib/cql_test_env.cc +++ b/test/lib/cql_test_env.cc @@ -698,11 +698,10 @@ class single_node_cql_env : public cql_test_env { host_id = linfo.host_id; _sys_ks.local().save_local_info(std::move(linfo), _snitch.local()->get_location(), my_address, my_address).get(); } - locator::shared_token_metadata::mutate_on_all_shards(_token_metadata, [hostid = host_id, &cfg_in] (locator::token_metadata& tm) { + locator::shared_token_metadata::mutate_on_all_shards(_token_metadata, [hostid = host_id] (locator::token_metadata& tm) { auto& topo = tm.get_topology(); topo.set_host_id_cfg(hostid); topo.add_or_update_endpoint(hostid, - cfg_in.broadcast_address, std::nullopt, locator::node::state::normal, smp::count); diff --git a/test/perf/perf_sort_by_proximity.cc b/test/perf/perf_sort_by_proximity.cc index 4d86bcea1df6..438bf19ebd26 100644 --- a/test/perf/perf_sort_by_proximity.cc +++ b/test/perf/perf_sort_by_proximity.cc @@ -53,7 +53,6 @@ struct sort_by_proximity_topology { auto id = locator::host_id{utils::UUID(0, i)}; nodes[dc][rack].emplace_back(id); topology.add_or_update_endpoint(id, - gms::inet_address((127u << 24) | i), locator::endpoint_dc_rack{format("dc{}", dc), format("rack{}", rack)}, locator::node::state::normal); } From 1e4b2f25dc7dca2f4f769db6cd8bbf91bb767d96 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Tue, 7 Jan 2025 15:50:20 +0200 Subject: [PATCH 363/397] locator: token_metadata: drop update_host_id() function that does nothing now --- locator/token_metadata.cc | 5 -- locator/token_metadata.hh | 9 --- service/storage_service.cc | 26 ++------ test/boost/locator_topology_test.cc | 3 - test/boost/network_topology_strategy_test.cc | 6 -- test/boost/tablets_test.cc | 62 +------------------- test/boost/token_metadata_test.cc | 37 ------------ test/perf/tablet_load_balancing.cc | 1 - 8 files changed, 9 insertions(+), 140 deletions(-) diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc index 519a66475492..e48c740db3bf 100644 --- a/locator/token_metadata.cc +++ b/locator/token_metadata.cc @@ -949,11 +949,6 @@ token_metadata::debug_show() const { _impl->debug_show(); } -void -token_metadata::update_host_id(const host_id& host_id, inet_address endpoint) { - // Do nothing for now. Remove later. -} - std::unordered_set token_metadata::get_host_ids() const { return _impl->get_host_ids(); diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh index f2c27fcb14d5..988734eef2bd 100644 --- a/locator/token_metadata.hh +++ b/locator/token_metadata.hh @@ -227,15 +227,6 @@ public: const topology& get_topology() const; void debug_show() const; - /** - * Store an end-point to host ID mapping. Each ID must be unique, and - * cannot be changed after the fact. - * - * @param hostId - * @param endpoint - */ - void update_host_id(const locator::host_id& host_id, inet_address endpoint); - /** Return the unique host ID for an end-point. */ host_id get_host_id(inet_address endpoint) const; diff --git a/service/storage_service.cc b/service/storage_service.cc index c65fb5f26578..fb65b690a40b 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -420,12 +420,9 @@ future storage_service::sync_raft_t const auto& am =_address_map; const auto& t = _topology_state_machine._topology; - auto update_topology = [&] (locator::host_id id, std::optional ip, const replica_state& rs) { + auto update_topology = [&] (locator::host_id id, const replica_state& rs) { tmptr->update_topology(id, locator::endpoint_dc_rack{rs.datacenter, rs.rack}, to_topology_node_state(rs.state), rs.shard_count); - if (ip) { - tmptr->update_host_id(id, *ip); - } }; using host_id_to_ip_map_t = std::unordered_map; @@ -466,7 +463,7 @@ future storage_service::sync_raft_t } if (t.left_nodes_rs.find(id) != t.left_nodes_rs.end()) { - update_topology(host_id, std::nullopt, t.left_nodes_rs.at(id)); + update_topology(host_id, t.left_nodes_rs.at(id)); } // However if we do that, we need to also implement unbanning a node and do it if `removenode` is aborted. @@ -524,7 +521,7 @@ future storage_service::sync_raft_t co_await remove_ip(it->second, host_id, false); } } - update_topology(host_id, ip, rs); + update_topology(host_id, rs); co_await tmptr->update_normal_tokens(rs.ring.value().tokens, host_id); }; @@ -551,7 +548,7 @@ future storage_service::sync_raft_t // Save ip -> id mapping in peers table because we need it on restart, but do not save tokens until owned sys_ks_futures.push_back(_sys_ks.local().update_peer_info(*ip, host_id, {})); } - update_topology(host_id, ip, rs); + update_topology(host_id, rs); if (_topology_state_machine._topology.normal_nodes.empty()) { // This is the first node in the cluster. Insert the tokens as normal to the token ring early // so we can perform writes to regular 'distributed' tables during the bootstrap procedure @@ -579,7 +576,7 @@ future storage_service::sync_raft_t co_await process_normal_node(id, rs); break; } - update_topology(host_id, ip, rs); + update_topology(host_id, rs); co_await tmptr->update_normal_tokens(rs.ring.value().tokens, host_id); tmptr->add_leaving_endpoint(host_id); co_await update_topology_change_info(tmptr, ::format("{} {}/{}", rs.state, id, ip)); @@ -592,7 +589,7 @@ future storage_service::sync_raft_t tmptr->update_topology(replaced_host_id, std::nullopt, locator::node::state::being_replaced); tmptr->add_replacing_endpoint(replaced_host_id, host_id); if (rs.ring.has_value()) { - update_topology(host_id, ip, rs); + update_topology(host_id, rs); co_await update_topology_change_info(tmptr, ::format("replacing {}/{} by {}/{}", replaced_id, existing_ip.value_or(gms::inet_address{}), id, ip)); } } @@ -1551,7 +1548,6 @@ future<> storage_service::join_topology(sharded tmptr->update_topology(tmptr->get_my_id(), std::nullopt, locator::node::state::replacing); tmptr->update_topology(ri->host_id, std::move(ri->dc_rack), locator::node::state::being_replaced); co_await tmptr->update_normal_tokens(bootstrap_tokens, ri->host_id); - tmptr->update_host_id(ri->host_id, *replace_address); tmptr->add_replacing_endpoint(ri->host_id, tmptr->get_my_id()); replaced_host_id = ri->host_id; @@ -1561,7 +1557,6 @@ future<> storage_service::join_topology(sharded // therefore we need to "inject" their state here after we // learn about them in the shadow round initiated in `prepare_replacement_info`. for (const auto& [host_id, st] : ri->ignore_nodes) { - tmptr->update_host_id(host_id, st.endpoint); if (st.opt_dc_rack) { tmptr->update_topology(host_id, st.opt_dc_rack, locator::node::state::normal); } @@ -2314,7 +2309,6 @@ future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::per } tmptr->update_topology(host_id, get_dc_rack_for(host_id), locator::node::state::bootstrapping); tmptr->add_bootstrap_tokens(tokens, host_id); - tmptr->update_host_id(host_id, endpoint); co_await update_topology_change_info(tmptr, ::format("handle_state_bootstrap {}", endpoint)); co_await replicate_to_all_cores(std::move(tmptr)); @@ -2382,8 +2376,6 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit slogger.warn("Host ID collision for {} between {} and {}; {} is the new owner", host_id, *existing, endpoint, endpoint); do_remove_node(*existing); - slogger.info("Set host_id={} to be owned by node={}, existing={}", host_id, endpoint, *existing); - tmptr->update_host_id(host_id, endpoint); } else { // The new IP has smaller generation than the existing one, // we are going to remove it, so we add it to the endpoints_to_remove. @@ -2434,8 +2426,6 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit _normal_state_handled_on_boot.insert(endpoint); co_return; } - slogger.info("Set host_id={} to be owned by node={}", host_id, endpoint); - tmptr->update_host_id(host_id, endpoint); } // Tokens owned by the handled endpoint. @@ -2643,7 +2633,6 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta auto tmlock = co_await get_token_metadata_lock(); auto tmptr = co_await get_mutable_token_metadata_ptr(); const auto dc_rack = get_dc_rack_for(host_id); - tmptr->update_host_id(host_id, endpoint); tmptr->update_topology(host_id, dc_rack); co_await replicate_to_all_cores(std::move(tmptr)); } @@ -3047,7 +3036,6 @@ future<> storage_service::join_cluster(sharded& slogger.debug("Loaded tokens: endpoint={}/{} dc={} rack={} tokens={}", host_id, st.endpoint, dc_rack.dc, dc_rack.rack, st.tokens); tmptr->update_topology(host_id, dc_rack, locator::node::state::normal); co_await tmptr->update_normal_tokens(st.tokens, host_id); - tmptr->update_host_id(host_id, st.endpoint); // gossiping hasn't started yet // so no need to lock the endpoint co_await _gossiper.add_saved_endpoint(host_id, st, gms::null_permit_id); @@ -4567,7 +4555,6 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad // In handle_state_normal we'll remap the IP to the new host_id. tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced); tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing); - tmptr->update_host_id(replacing_node_id, replacing_node); tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id); return make_ready_future<>(); }).get(); @@ -4631,7 +4618,6 @@ future storage_service::node_ops_cmd_handler(gms::inet_ad const auto dc_rack = get_dc_rack_for(host_id); slogger.info("bootstrap[{}]: Added node={}/{} as bootstrap, coordinator={}/{}", req.ops_uuid, endpoint, host_id, coordinator, *coordinator_host_id); - tmptr->update_host_id(host_id, endpoint); tmptr->update_topology(host_id, dc_rack, locator::node::state::bootstrapping); tmptr->add_bootstrap_tokens(tokens, host_id); } diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc index f42dd7435626..1256a33a4934 100644 --- a/test/boost/locator_topology_test.cc +++ b/test/boost/locator_topology_test.cc @@ -215,9 +215,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_sketch) { }); stm.mutate_token_metadata([&] (token_metadata& tm) { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, node1_shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, node2_shard_count); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, node3_shard_count); diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index a3a7333d9c95..312f79a9546d 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -485,7 +485,6 @@ SEASTAR_THREAD_TEST_CASE(NetworkTopologyStrategy_tablets_test) { std::unordered_set tokens; tokens.insert(token{tests::d2t(ring_point / ring_points.size())}); topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count); - tm.update_host_id(id, endpoint); co_await tm.update_normal_tokens(std::move(tokens), id); } }).get(); @@ -577,7 +576,6 @@ static void test_random_balancing(sharded& snitch, gms::inet_address std::unordered_set tokens; tokens.insert(token{tests::d2t(ring_point / ring_points.size())}); topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count); - tm.update_host_id(id, endpoint); co_await tm.update_normal_tokens(std::move(tokens), id); } }).get(); @@ -1115,7 +1113,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { auto host1 = host_id(utils::make_random_uuid()); auto host2 = host_id(utils::make_random_uuid()); - auto host3 = host_id(utils::make_random_uuid()); auto ip1_dc_rack = endpoint_dc_rack{ "dc1", "rack_ip1" }; auto ip1_dc_rack_v2 = endpoint_dc_rack{ "dc1", "rack_ip1_v2" }; @@ -1134,8 +1131,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack); stm.mutate_token_metadata([&] (token_metadata& tm) { - tm.update_host_id(host2, ip2); - tm.update_host_id(host1, ip1); // this_node added last on purpose // Need to move to non left or none state in order to be indexed by ip tm.update_topology(host1, {}, locator::node::state::normal); tm.update_topology(host2, {}, locator::node::state::normal); @@ -1159,7 +1154,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) { stm.mutate_token_metadata([&] (token_metadata& tm) { tm.remove_endpoint(host1); - tm.update_host_id(host3, ip3); return make_ready_future<>(); }).get(); diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 1e8b62f8b525..13bd5a52d990 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -773,9 +773,6 @@ SEASTAR_TEST_CASE(test_get_shard) { tablet_id tid1(0); stm.mutate_token_metadata([&] (token_metadata& tm) { - tm.update_host_id(h1, ip1); - tm.update_host_id(h2, ip2); - tm.update_host_id(h3, ip3); tm.update_topology(h1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(h2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(h3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); @@ -1515,9 +1512,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_empty_node) { }); stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); @@ -1616,9 +1610,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_skiplist) { }); stm.mutate_token_metadata([&] (token_metadata& tm) { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); @@ -1707,9 +1698,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_rf_met) { stm.mutate_token_metadata([&](token_metadata& tm) -> future<> { const unsigned shard_count = 2; - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::being_decommissioned, @@ -1807,10 +1795,6 @@ SEASTAR_THREAD_TEST_CASE(test_table_creation_during_decommission) { const unsigned shard_count = 1; stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); - tm.update_host_id(host4, ip4); tm.update_topology(host1, dcrack, node::state::normal, shard_count); tm.update_topology(host2, dcrack, node::state::normal, shard_count); tm.update_topology(host3, dcrack, node::state::being_decommissioned, shard_count); @@ -1877,10 +1861,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_two_racks) { stm.mutate_token_metadata([&](token_metadata& tm) -> future<> { const unsigned shard_count = 1; - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); - tm.update_host_id(host4, ip4); tm.update_topology(host1, racks[0], node::state::normal, shard_count); tm.update_topology(host2, racks[1], node::state::normal, shard_count); tm.update_topology(host3, racks[0], node::state::normal, shard_count); @@ -1984,10 +1964,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_rack_load_failure) { stm.mutate_token_metadata([&](token_metadata& tm) -> future<> { const unsigned shard_count = 1; - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); - tm.update_host_id(host4, ip4); tm.update_topology(host1, racks[0], node::state::normal, shard_count); tm.update_topology(host2, racks[0], node::state::normal, shard_count); tm.update_topology(host3, racks[0], node::state::normal, shard_count); @@ -2063,9 +2039,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_rf_not_met) { stm.mutate_token_metadata([&](token_metadata& tm) -> future<> { const unsigned shard_count = 2; - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::being_decommissioned, @@ -2122,9 +2095,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_works_with_in_progress_transitions) }); stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, 1); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, 1); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, 2); @@ -2196,9 +2166,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancer_shuffle_mode) { }); stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, 1); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, 1); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, 2); @@ -2263,10 +2230,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_two_empty_nodes) { }); stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); - tm.update_host_id(host4, ip4); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); @@ -2328,9 +2291,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_asymmetric_node_capacity) { }); stm.mutate_token_metadata([&](token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::being_decommissioned, 8); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, 1); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, 7); @@ -2395,8 +2355,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancer_disabling) { // host1 is loaded and host2 is empty, resulting in an imbalance. // host1's shard 0 is loaded and shard 1 is empty, resulting in intra-node imbalance. stm.mutate_token_metadata([&] (auto& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1); @@ -2487,8 +2445,6 @@ SEASTAR_THREAD_TEST_CASE(test_drained_node_is_not_balanced_internally) { }); stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1); @@ -2537,7 +2493,6 @@ SEASTAR_THREAD_TEST_CASE(test_plan_fails_when_removing_last_replica) { }); stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count); co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1); @@ -2589,9 +2544,6 @@ SEASTAR_THREAD_TEST_CASE(test_skiplist_is_ignored_when_draining) { }); stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); - tm.update_host_id(host3, ip3); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, locator::node::state::normal, shard_count); tm.update_topology(host3, locator::endpoint_dc_rack::default_location, locator::node::state::normal, shard_count); @@ -2699,10 +2651,8 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_random_load) { int i = 0; for (auto h : hosts) { - auto ip = inet_address(format("192.168.0.{}", ++i)); auto shard_count = 2; - tm.update_host_id(h, ip); - auto rack = racks[i % racks.size()]; + auto rack = racks[++i % racks.size()]; tm.update_topology(h, rack, node::state::normal, shard_count); if (h != hosts[0]) { // Leave the first host empty by making it invisible to allocation algorithm. @@ -2871,13 +2821,11 @@ static void do_test_load_balancing_merge_colocation(cql_test_env& e, const int n int i = 0; for (auto h : hosts) { - auto ip = inet_address(format("192.168.0.{}", ++i)); - tm.update_host_id(h, ip); - auto rack = racks[i % racks.size()]; + auto rack = racks[++i % racks.size()]; hosts_by_rack[rack.rack].push_back(h); tm.update_topology(h, rack, node::state::normal, shard_count); co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(float(i) / hosts.size()))}, h); - testlog.debug("adding host {}, ip {}, rack {}, token {}", h, ip, rack.rack, token(tests::d2t(1. / hosts.size()))); + testlog.debug("adding host {}, rack {}, token {}", h, rack.rack, token(tests::d2t(1. / hosts.size()))); } tablet_map tmap(initial_tablets); @@ -3056,8 +3004,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_resize_requests) { }); stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> { - tm.update_host_id(host1, ip1); - tm.update_host_id(host2, ip2); tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count); co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1); @@ -3341,7 +3287,6 @@ static void execute_tablet_for_new_rf_test(calculate_tablet_replicas_for_new_rf_ std::unordered_set tokens; tokens.insert(dht::token{tests::d2t(ring_point / test_config.ring_points.size())}); topo.add_or_update_endpoint(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1); - tm.update_host_id(id, endpoint); co_await tm.update_normal_tokens(std::move(tokens), id); } }).get(); @@ -3362,7 +3307,6 @@ static void execute_tablet_for_new_rf_test(calculate_tablet_replicas_for_new_rf_ stm.mutate_token_metadata([&] (token_metadata& tm) { for (size_t i = 0; i < test_config.ring_points.size(); ++i) { auto& [ring_point, endpoint, id] = test_config.ring_points[i]; - tm.update_host_id(id, endpoint); tm.update_topology(id, make_endpoint_dc_rack(endpoint), node::state::normal, nodes_shard_count[i]); } return make_ready_future<>(); diff --git a/test/boost/token_metadata_test.cc b/test/boost/token_metadata_test.cc index 96aa2f9b0fd4..ba92fe80e8be 100644 --- a/test/boost/token_metadata_test.cc +++ b/test/boost/token_metadata_test.cc @@ -50,16 +50,12 @@ namespace { } SEASTAR_THREAD_TEST_CASE(test_pending_and_read_endpoints_for_everywhere_strategy) { - const auto e1 = inet_address("192.168.0.1"); - const auto e2 = inet_address("192.168.0.2"); const auto e1_id = gen_id(1); const auto e2_id = gen_id(2); const auto t1 = dht::token::from_int64(10); const auto t2 = dht::token::from_int64(20); auto token_metadata = create_token_metadata(e1_id); - token_metadata->update_host_id(e1_id, e1); - token_metadata->update_host_id(e2_id, e2); token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal); token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal); token_metadata->update_normal_tokens({t1}, e1_id).get(); @@ -74,16 +70,12 @@ SEASTAR_THREAD_TEST_CASE(test_pending_and_read_endpoints_for_everywhere_strategy } SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_bootstrap_second_node) { - const auto e1 = inet_address("192.168.0.1"); const auto t1 = dht::token::from_int64(1); - const auto e2 = inet_address("192.168.0.2"); const auto t2 = dht::token::from_int64(100); const auto e1_id = gen_id(1); const auto e2_id = gen_id(2); auto token_metadata = create_token_metadata(e1_id); - token_metadata->update_host_id(e1_id, e1); - token_metadata->update_host_id(e2_id, e2); token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal); token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal); token_metadata->update_normal_tokens({t1}, e1_id).get(); @@ -107,17 +99,11 @@ SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_bootstrap_with_replicas) { const auto t10 = dht::token::from_int64(10); const auto t100 = dht::token::from_int64(100); const auto t1000 = dht::token::from_int64(1000); - const auto e1 = inet_address("192.168.0.1"); - const auto e2 = inet_address("192.168.0.2"); - const auto e3 = inet_address("192.168.0.3"); const auto e1_id = gen_id(1); const auto e2_id = gen_id(2); const auto e3_id = gen_id(3); auto token_metadata = create_token_metadata(e1_id); - token_metadata->update_host_id(e1_id, e1); - token_metadata->update_host_id(e2_id, e2); - token_metadata->update_host_id(e3_id, e3); token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal); token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal); token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal); @@ -143,17 +129,11 @@ SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_leave_with_replicas) { const auto t10 = dht::token::from_int64(10); const auto t100 = dht::token::from_int64(100); const auto t1000 = dht::token::from_int64(1000); - const auto e1 = inet_address("192.168.0.1"); - const auto e2 = inet_address("192.168.0.2"); - const auto e3 = inet_address("192.168.0.3"); const auto e1_id = gen_id(1); const auto e2_id = gen_id(2); const auto e3_id = gen_id(3); auto token_metadata = create_token_metadata(e1_id); - token_metadata->update_host_id(e1_id, e1); - token_metadata->update_host_id(e2_id, e2); - token_metadata->update_host_id(e3_id, e3); token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal); token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal); token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal); @@ -180,20 +160,12 @@ SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_replace_with_replicas) { const auto t10 = dht::token::from_int64(10); const auto t100 = dht::token::from_int64(100); const auto t1000 = dht::token::from_int64(1000); - const auto e1 = inet_address("192.168.0.1"); - const auto e2 = inet_address("192.168.0.2"); - const auto e3 = inet_address("192.168.0.3"); - const auto e4 = inet_address("192.168.0.4"); const auto e1_id = gen_id(1); const auto e2_id = gen_id(2); const auto e3_id = gen_id(3); const auto e4_id = gen_id(4); auto token_metadata = create_token_metadata(e1_id); - token_metadata->update_host_id(e1_id, e1); - token_metadata->update_host_id(e2_id, e2); - token_metadata->update_host_id(e3_id, e3); - token_metadata->update_host_id(e4_id, e4); token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal); token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal); token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal); @@ -225,17 +197,11 @@ SEASTAR_THREAD_TEST_CASE(test_endpoints_for_reading_when_bootstrap_with_replicas const auto t10 = dht::token::from_int64(10); const auto t100 = dht::token::from_int64(100); const auto t1000 = dht::token::from_int64(1000); - const auto e1 = inet_address("192.168.0.1"); - const auto e2 = inet_address("192.168.0.2"); - const auto e3 = inet_address("192.168.0.3"); const auto e1_id = gen_id(1); const auto e2_id = gen_id(2); const auto e3_id = gen_id(3); auto token_metadata = create_token_metadata(e1_id); - token_metadata->update_host_id(e1_id, e1); - token_metadata->update_host_id(e2_id, e2); - token_metadata->update_host_id(e3_id, e3); token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal); token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal); token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal); @@ -285,17 +251,14 @@ SEASTAR_THREAD_TEST_CASE(test_endpoints_for_reading_when_bootstrap_with_replicas SEASTAR_THREAD_TEST_CASE(test_replace_node_with_same_endpoint) { const auto t1 = dht::token::from_int64(1); - const auto e1 = inet_address("192.168.0.1"); const auto e1_id1 = gen_id(1); const auto e1_id2 = gen_id(2); auto token_metadata = create_token_metadata(e1_id2); - token_metadata->update_host_id(e1_id1, e1); token_metadata->update_topology(e1_id1, get_dc_rack(e1_id1), node::state::being_replaced); token_metadata->update_normal_tokens({t1}, e1_id1).get(); token_metadata->update_topology(e1_id2, get_dc_rack(e1_id2), node::state::replacing); - token_metadata->update_host_id(e1_id2, e1); token_metadata->add_replacing_endpoint(e1_id1, e1_id2); diff --git a/test/perf/tablet_load_balancing.cc b/test/perf/tablet_load_balancing.cc index 9a9e5178cb9a..c0625b0f318e 100644 --- a/test/perf/tablet_load_balancing.cc +++ b/test/perf/tablet_load_balancing.cc @@ -243,7 +243,6 @@ future test_load_balancing_with_many_tables(params p, bool tablet_aware }; auto add_host_to_topology = [&] (token_metadata& tm, int i) -> future<> { - tm.update_host_id(hosts[i], ips[i]); tm.update_topology(hosts[i], rack1, node::state::normal, shard_count); co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(float(i) / hosts.size()))}, hosts[i]); }; From a40e8104426969d4a9e543015b8237ac97ddfbd0 Mon Sep 17 00:00:00 2001 From: Gleb Natapov Date: Wed, 8 Jan 2025 11:37:26 +0200 Subject: [PATCH 364/397] hint manager: do not translate ip to id in case hint manager is stopped already Since we do not stop storage proxy on shutdown this code can be called during shutdown when address map is no longer usable. --- db/hints/manager.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/db/hints/manager.cc b/db/hints/manager.cc index 1f996d193fce..77ca698b2aea 100644 --- a/db/hints/manager.cc +++ b/db/hints/manager.cc @@ -433,9 +433,8 @@ bool manager::have_ep_manager(const std::variant fm, tracing::trace_state_ptr tr_state) noexcept { - auto ip = _gossiper_anchor->get_address_map().get(host_id); if (utils::get_local_injector().enter("reject_incoming_hints")) { - manager_logger.debug("Rejecting a hint to {} / {} due to an error injection", host_id, ip); + manager_logger.debug("Rejecting a hint to {} due to an error injection", host_id); ++_stats.dropped; return false; } @@ -446,6 +445,8 @@ bool manager::store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptrget_address_map().get(host_id); + try { manager_logger.trace("Going to store a hint to {}", host_id); tracing::trace(tr_state, "Going to store a hint to {}", host_id); From 955ac1b7b794f71da528b5f4660af17a2ecd6900 Mon Sep 17 00:00:00 2001 From: Nadav Har'El Date: Thu, 16 Jan 2025 10:04:04 +0200 Subject: [PATCH 365/397] test/alternator: close boto3 client before shutting down For several years now, we have seen a strange, and very rare, flakiness in Alternator tests described in issue #17564: We see all the test pass, pytest declares them to have passed, and while Python is existing, it crashes with a signal 11 (SIGSEGV). Because this happens exclusively in test/alternator and never in the test/cqlpy, we suspect that something that the test/alternator leaves behind but test/cqlpy does not, causes some race and crashes during shutdown. The immediate suspect is the boto3 library, or rather, the urllib3 library which it uses. This is more-or-less the only thing that test/alternator does which test/cqlpy doesn't. The urllib3 library keeps around pools of reusable connections, and it's possible (although I don't actually have any proof for it) that these open connections may cause a crash during shutdown. So in this patch I add to the "dynamodb" and "dynamodbstreams" fixtures (which all Alternator tests use to connect to the server), a teardown which calls close() for the boto3 client object. This close() call percolates down to calling clear() on urllib3's PoolManager. Hopefully, this will make some difference in the chance to crash during shutdown - and if it doesn't, it won't hurt. Refs #17564 Closes scylladb/scylladb#22341 --- test/alternator/conftest.py | 12 ++++++++---- test/alternator/test_service_levels.py | 1 - 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/test/alternator/conftest.py b/test/alternator/conftest.py index 5fff5beefd54..4e71619dd20a 100644 --- a/test/alternator/conftest.py +++ b/test/alternator/conftest.py @@ -108,7 +108,7 @@ def dynamodb(request): # because boto3 checks them before we can get the server to check them. boto_config = botocore.client.Config(parameter_validation=False) if request.config.getoption('aws'): - return boto3.resource('dynamodb', config=boto_config) + res = boto3.resource('dynamodb', config=boto_config) else: # Even though we connect to the local installation, Boto3 still # requires us to specify dummy region and credential parameters, @@ -124,9 +124,11 @@ def dynamodb(request): # Disable verifying in order to be able to use self-signed TLS certificates verify = not request.config.getoption('https') user, secret = get_valid_alternator_role(local_url) - return boto3.resource('dynamodb', endpoint_url=local_url, verify=verify, + res = boto3.resource('dynamodb', endpoint_url=local_url, verify=verify, region_name='us-east-1', aws_access_key_id=user, aws_secret_access_key=secret, config=boto_config.merge(botocore.client.Config(retries={"max_attempts": 0}, read_timeout=300))) + yield res + res.meta.client.close() def new_dynamodb_session(request, dynamodb, user='cassandra', password='secret_pass'): ses = boto3.Session() @@ -148,7 +150,7 @@ def dynamodbstreams(request): # because boto3 checks them before we can get the server to check them. boto_config = botocore.client.Config(parameter_validation=False) if request.config.getoption('aws'): - return boto3.client('dynamodbstreams', config=boto_config) + res = boto3.client('dynamodbstreams', config=boto_config) else: # Even though we connect to the local installation, Boto3 still # requires us to specify dummy region and credential parameters, @@ -164,9 +166,11 @@ def dynamodbstreams(request): # Disable verifying in order to be able to use self-signed TLS certificates verify = not request.config.getoption('https') user, secret = get_valid_alternator_role(local_url) - return boto3.client('dynamodbstreams', endpoint_url=local_url, verify=verify, + res = boto3.client('dynamodbstreams', endpoint_url=local_url, verify=verify, region_name='us-east-1', aws_access_key_id=user, aws_secret_access_key=secret, config=boto_config.merge(botocore.client.Config(retries={"max_attempts": 0}, read_timeout=300))) + yield res + res.close() # A function-scoped autouse=True fixture allows us to test after every test # that the server is still alive - and if not report the test which crashed diff --git a/test/alternator/test_service_levels.py b/test/alternator/test_service_levels.py index 848c0e74207a..ea85fd0c7689 100644 --- a/test/alternator/test_service_levels.py +++ b/test/alternator/test_service_levels.py @@ -3,7 +3,6 @@ # SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 import pytest -from test.alternator.conftest import new_dynamodb_session from test.alternator.test_metrics import metrics, get_metrics, check_increases_metric from contextlib import contextmanager from cassandra.auth import PlainTextAuthProvider From aad46bd6f3b3f9025c5bf6d9ef59d75005da3fdd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Paszkowski?= Date: Thu, 16 Jan 2025 09:54:05 +0100 Subject: [PATCH 366/397] reader_concurrency_semaphore: do_wait_admission(): remove dumping diagnostics The commit b39ca29b3c introduced detection of admission-waiter anomaly and dumps permit diagnostics as soon as the semaphore did not admit readers even though it could. Later on, the commit bf3d0b3543 introduces the optimization where the admission check is moved to the fiber processing the _read_list. Since the semaphore no longer admits readers as soon as it can, dumping diagnostic errors is not necessary as the situation is not abnormal. Closes scylladb/scylladb#22344 --- reader_concurrency_semaphore.cc | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/reader_concurrency_semaphore.cc b/reader_concurrency_semaphore.cc index 2dfc438265ef..2b17979f21f6 100644 --- a/reader_concurrency_semaphore.cc +++ b/reader_concurrency_semaphore.cc @@ -1460,11 +1460,10 @@ future<> reader_concurrency_semaphore::do_wait_admission(reader_permit::impl& pe if (admit != can_admit::yes || !_wait_list.empty()) { auto fut = enqueue_waiter(permit, wait_on::admission); if (admit == can_admit::yes && !_wait_list.empty()) { - // This is a contradiction: the semaphore could admit waiters yet it has waiters. - // Normally, the semaphore should admit waiters as soon as it can. - // So at any point in time, there should either be no waiters, or it - // shouldn't be able to admit new reads. Otherwise something went wrong. - maybe_dump_reader_permit_diagnostics(*this, "semaphore could admit new reads yet there are waiters", nullptr); + // Enters the case where the semaphore can admit waiters yet it has waiters. + // Hence, wake the execution loop to process the waiters. Since readers are + // no longer admitted as soon as they can, the resource release could be delayed + // as well. maybe_wake_execution_loop(); } else if (admit == can_admit::maybe) { tracing::trace(permit.trace_state(), "[reader concurrency semaphore {}] evicting inactive reads in the background to free up resources", _name); From a00ab654911cab585fc4329c3dde34decbe20d34 Mon Sep 17 00:00:00 2001 From: Tzach Livyatan Date: Wed, 1 Jan 2025 10:09:18 +0200 Subject: [PATCH 367/397] remove BETA from metric and API reference Closes scylladb/scylladb#22092 --- docs/reference/_common/reference-toc.rst | 4 ++-- docs/reference/api-reference.rst | 2 +- docs/reference/index.rst | 4 ++-- docs/reference/metrics.rst | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/reference/_common/reference-toc.rst b/docs/reference/_common/reference-toc.rst index 607c2a30cda2..d4b5cfa49a3c 100644 --- a/docs/reference/_common/reference-toc.rst +++ b/docs/reference/_common/reference-toc.rst @@ -8,5 +8,5 @@ Configuration Parameters Glossary Limits - API Reference (BETA) - Metrics (BETA) \ No newline at end of file + API Reference + Metrics diff --git a/docs/reference/api-reference.rst b/docs/reference/api-reference.rst index be16fa79c99a..7cdaa8f92004 100644 --- a/docs/reference/api-reference.rst +++ b/docs/reference/api-reference.rst @@ -1,6 +1,6 @@ :exclude-doctools: -API Reference (BETA) +API Reference ==================== .. scylladb_swagger_inc:: diff --git a/docs/reference/index.rst b/docs/reference/index.rst index 08cbdd032617..7d3a6af0a700 100644 --- a/docs/reference/index.rst +++ b/docs/reference/index.rst @@ -13,6 +13,6 @@ Reference * :doc:`Configuration Parameters ` - ScyllaDB properties configurable in the ``scylla.yaml`` configuration file. * :doc:`Glossary ` - ScyllaDB-related terms and definitions. * :doc:`Limits ` -* :doc:`API Reference (BETA) ` -* :doc:`Metrics (BETA) ` +* :doc:`API Reference ` +* :doc:`Metrics ` * .. scylladb_include_flag:: enterprise-vs-oss-matrix-link.rst diff --git a/docs/reference/metrics.rst b/docs/reference/metrics.rst index 68245ec34d51..40b34d9e3ec6 100644 --- a/docs/reference/metrics.rst +++ b/docs/reference/metrics.rst @@ -1,5 +1,5 @@ ============== -Metrics (BETA) +Metrics ============== .. scylladb_metrics:: From 2428f22d3ebde93eb78a11bd9b7e4d07f49f815a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Fri, 17 Jan 2025 09:17:29 +0200 Subject: [PATCH 368/397] Update tools/python3 submodule * tools/python3 fbf12d02...8415caf4 (1): > dist: Support FIPS mode --- tools/python3 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/python3 b/tools/python3 index fbf12d025fb8..8415caf43bcf 160000 --- a/tools/python3 +++ b/tools/python3 @@ -1 +1 @@ -Subproject commit fbf12d025fb8458557136770ff972cb622fe8712 +Subproject commit 8415caf43bcfb574140738bacf817f7b8665888f From d6f7f873d0c89020fe4f2a09c191012afb1e7f2a Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Mon, 13 Jan 2025 21:06:36 +0200 Subject: [PATCH 369/397] utils: config_file: don't use extern fully specialized variable templates Declaring-but-not-defining a fully specialized template is a great way to cut dependencies between users and providers, but unfortunately not supported for variable templates. Clang 18 does support it, but apparently it is a misinterpretation of the standard, and was removed in clang 19. We started using this non-feature in 7ed89266b30f782c05. The fix is to use function templates. This is more verbose as each specialization needs to define a static variable to return, but is fully supported. Closes scylladb/scylladb#22299 --- db/config.cc | 122 ++++++++++++++++++++++++++++++++++--------- utils/config_file.hh | 19 +------ 2 files changed, 100 insertions(+), 41 deletions(-) diff --git a/db/config.cc b/db/config.cc index da76b4d79529..266158499a8b 100644 --- a/db/config.cc +++ b/db/config.cc @@ -116,80 +116,154 @@ config_from_string(std::string_view value) { } template <> -const config_type config_type_for = config_type("bool", value_to_json); +const config_type& config_type_for() { + static config_type ct("bool", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("integer", value_to_json); +const config_type& config_type_for() { + static config_type ct("integer", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("integer", value_to_json); +const config_type& config_type_for() { + static config_type ct("integer", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("integer", value_to_json); +const config_type& config_type_for() { + static config_type ct("integer", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("float", value_to_json); +const config_type& config_type_for() { + static config_type ct("float", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("double", value_to_json); +const config_type& config_type_for() { + static config_type ct("double", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("string", log_level_to_json); +const config_type& config_type_for() { + static config_type ct("string", log_level_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("string", value_to_json); +const config_type& config_type_for() { + static config_type ct("string", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("string", value_to_json); +const config_type& config_type_for() { + static config_type ct("string", value_to_json); + return ct; +} template <> -const config_type config_type_for> = config_type("string list", value_to_json>); +const config_type& config_type_for>() { + static config_type ct("string list", value_to_json>); + return ct; +} template <> -const config_type config_type_for>> = config_type("string map map", value_to_json>>); +const config_type& config_type_for>>() { + static config_type ct("string map map", value_to_json>>); + return ct; +} template <> -const config_type config_type_for> = config_type("string map", value_to_json>); +const config_type& config_type_for>() { + static config_type ct("string map", value_to_json>); + return ct; +} template <> -const config_type config_type_for>> = config_type("string map list", value_to_json>>); +const config_type& config_type_for>>() { + static config_type ct("string map list", value_to_json>>); + return ct; +} template <> -const config_type config_type_for> = config_type("string map", log_level_map_to_json); +const config_type& config_type_for>() { + static config_type ct("string map", log_level_map_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("integer", value_to_json); +const config_type& config_type_for() { + static config_type ct("integer", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("integer", value_to_json); +const config_type& config_type_for() { + static config_type ct("integer", value_to_json); + return ct; +} template <> -const config_type config_type_for = config_type("seed provider", seed_provider_to_json); +const config_type& config_type_for() { + static config_type ct("seed provider", seed_provider_to_json); + return ct; +} template <> -const config_type config_type_for>> = config_type( +const config_type& config_type_for>>() { + static config_type ct( "experimental features", printable_vector_to_json>); + return ct; +} + template <> -const config_type config_type_for>> = config_type( +const config_type& config_type_for>>() { + static config_type ct( "replication strategy list", printable_vector_to_json>); + return ct; +} template <> -const config_type config_type_for> = config_type( +const config_type& config_type_for>() { + static config_type ct( "restriction mode", printable_to_json>); + return ct; +} template <> -const config_type config_type_for = config_type("hinted handoff enabled", hinted_handoff_enabled_to_json); +const config_type& config_type_for() { + static config_type ct("hinted handoff enabled", hinted_handoff_enabled_to_json); + return ct; +} template <> -const config_type config_type_for> = config_type("error injection list", error_injection_list_to_json); +const config_type& config_type_for>() { + static config_type ct("error injection list", error_injection_list_to_json); + return ct; +} template <> -const config_type config_type_for> = config_type( +const config_type& config_type_for>() { + static config_type ct( "dictionary training conditions", printable_to_json>); + return ct; +} template <> -const config_type config_type_for = config_type( +const config_type& config_type_for() { + static config_type ct( "advanced rpc compressor config", printable_vector_to_json>); + return ct; +} + } namespace YAML { diff --git a/utils/config_file.hh b/utils/config_file.hh index 273d9df5a636..a6fc4eb828e6 100644 --- a/utils/config_file.hh +++ b/utils/config_file.hh @@ -52,22 +52,7 @@ public: }; template -extern const config_type config_type_for; - -template<> -extern const config_type config_type_for; - -template<> -extern const config_type config_type_for; - -template<> -extern const config_type config_type_for; - -template<> -extern const config_type config_type_for>; - -template<> -extern const config_type config_type_for>>; +extern const config_type& config_type_for(); class config_file { static thread_local unsigned s_shard_id; @@ -210,7 +195,7 @@ public: named_value(config_file* file, std::string_view name, std::string_view alias, liveness liveness_, value_status vs, const T& t = T(), std::string_view desc = {}, std::initializer_list allowed_values = {}) - : config_src(file, name, alias, &config_type_for, desc) + : config_src(file, name, alias, &config_type_for(), desc) , _value_status(vs) , _liveness(liveness_) , _allowed_values(std::move(allowed_values)) { From 614c3380c67c4f8304f0bcdf80797e1278bec7a0 Mon Sep 17 00:00:00 2001 From: Asias He Date: Tue, 10 Dec 2024 10:49:34 +0800 Subject: [PATCH 370/397] service: Add tablet_operation.hh A tablet_operation_result struct is added to track the result of a tablet operation. --- service/tablet_operation.hh | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 service/tablet_operation.hh diff --git a/service/tablet_operation.hh b/service/tablet_operation.hh new file mode 100644 index 000000000000..203833c12b06 --- /dev/null +++ b/service/tablet_operation.hh @@ -0,0 +1,21 @@ +// Copyright (C) 2024-present ScyllaDB +// SPDX-License-Identifier: AGPL-3.0-or-later + +#pragma once + +#include +#include "gc_clock.hh" + +namespace service { + +struct tablet_operation_empty_result { +}; + +struct tablet_operation_repair_result { + gc_clock::time_point repair_time; +}; + +using tablet_operation_result = std::variant; + +} + From 41a1eca072536da24cb08ac9ac2b36af64b29d01 Mon Sep 17 00:00:00 2001 From: Asias He Date: Tue, 10 Dec 2024 10:49:34 +0800 Subject: [PATCH 371/397] repair: Return repair time for repair_service::repair_tablet The repair time returned by repair_service::repair_tablet considers the hints and batchlog flush time, so it could be used for the tombstone gc purpose. --- repair/repair.cc | 27 ++++++++++++++++++++------- repair/row_level.hh | 2 +- repair/task_manager_module.hh | 4 ++++ 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/repair/repair.cc b/repair/repair.cc index 67b68eb80643..d55a71812c9b 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -2396,7 +2396,7 @@ future<> repair_service::repair_tablets(repair_uniq_id rid, sstring keyspace_nam } // It is called by the repair_tablet rpc verb to repair the given tablet -future<> repair_service::repair_tablet(gms::gossip_address_map& addr_map, locator::tablet_metadata_guard& guard, locator::global_tablet_id gid) { +future repair_service::repair_tablet(gms::gossip_address_map& addr_map, locator::tablet_metadata_guard& guard, locator::global_tablet_id gid) { auto id = _repair_module->new_repair_uniq_id(); rlogger.debug("repair[{}]: Starting tablet repair global_tablet_id={}", id.uuid(), gid); auto& db = get_db().local(); @@ -2405,7 +2405,7 @@ future<> repair_service::repair_tablet(gms::gossip_address_map& addr_map, locato auto t = db.get_tables_metadata().get_table_if_exists(table_id); if (!t) { - co_return; + co_return gc_clock::now(); } auto& tmap = guard.get_tablet_map(); auto s = t->schema(); @@ -2434,16 +2434,20 @@ future<> repair_service::repair_tablet(gms::gossip_address_map& addr_map, locato auto ranges_parallelism = std::nullopt; auto start = std::chrono::steady_clock::now(); task_metas.push_back(tablet_repair_task_meta{keyspace_name, table_name, table_id, master_shard_id, range, repair_neighbors(nodes, shards), replicas}); - auto task = co_await _repair_module->make_and_start_task({}, id, keyspace_name, table_names, streaming::stream_reason::repair, std::move(task_metas), ranges_parallelism); + auto task_impl_ptr = seastar::make_shared(_repair_module, id, keyspace_name, table_names, streaming::stream_reason::repair, std::move(task_metas), ranges_parallelism); + auto task = co_await _repair_module->make_task(task_impl_ptr, {}); + task->start(); co_await task->done(); + auto flush_time = task_impl_ptr->get_flush_time(); auto delay = utils::get_local_injector().inject_parameter("tablet_repair_add_delay_in_ms"); if (delay) { rlogger.debug("Execute tablet_repair_add_delay_in_ms={}", *delay); co_await seastar::sleep(std::chrono::milliseconds(*delay)); } auto duration = std::chrono::duration(std::chrono::steady_clock::now()- start); - rlogger.info("repair[{}]: Finished tablet repair for table={}.{} range={} duration={} replicas={} global_tablet_id={}", - id.uuid(), keyspace_name, table_name, range, duration, replicas, gid); + rlogger.info("repair[{}]: Finished tablet repair for table={}.{} range={} duration={} replicas={} global_tablet_id={} flush_time={}", + id.uuid(), keyspace_name, table_name, range, duration, replicas, gid, flush_time); + co_return flush_time; } tasks::is_user_task repair::tablet_repair_task_impl::is_user_task() const noexcept { @@ -2519,7 +2523,8 @@ future<> repair::tablet_repair_task_impl::run() { }); auto parent_shard = this_shard_id(); - rs.container().invoke_on_all([&idx, id, metas = _metas, parent_data, reason = _reason, tables = _tables, ranges_parallelism = _ranges_parallelism, parent_shard] (repair_service& rs) -> future<> { + std::vector flush_times(smp::count); + rs.container().invoke_on_all([&idx, &flush_times, id, metas = _metas, parent_data, reason = _reason, tables = _tables, ranges_parallelism = _ranges_parallelism, parent_shard] (repair_service& rs) -> future<> { std::exception_ptr error; for (auto& m : metas) { if (m.master_shard_id != this_shard_id()) { @@ -2555,7 +2560,7 @@ future<> repair::tablet_repair_task_impl::run() { m.keyspace_name, rs, erm, std::move(ranges), std::move(table_ids), id, std::move(data_centers), std::move(hosts), std::move(ignore_nodes), reason, hints_batchlog_flushed, small_table_optimization, ranges_parallelism, flush_time); task_impl_ptr->neighbors = std::move(neighbors); - auto task = co_await rs._repair_module->make_task(std::move(task_impl_ptr), parent_data); + auto task = co_await rs._repair_module->make_task(task_impl_ptr, parent_data); task->start(); auto res = co_await coroutine::as_future(task->done()); if (res.failed()) { @@ -2573,11 +2578,19 @@ future<> repair::tablet_repair_task_impl::run() { error = std::move(ep); } } + auto current = flush_times[this_shard_id()]; + auto time = task_impl_ptr->get_flush_time(); + flush_times[this_shard_id()] = current == gc_clock::time_point() ? time : std::min(current, time); } if (error) { co_await coroutine::return_exception_ptr(std::move(error)); } }).get(); + for (auto& time : flush_times) { + if (time != gc_clock::time_point()) { + _flush_time = _flush_time == gc_clock::time_point() ? time : std::min(_flush_time, time); + } + } auto duration = std::chrono::duration(std::chrono::steady_clock::now() - start_time); rlogger.info("repair[{}]: Finished user-requested repair for tablet keyspace={} tables={} repair_id={} tablets_repaired={} duration={}", id.uuid(), _keyspace, _tables, id.id, _metas.size(), duration); diff --git a/repair/row_level.hh b/repair/row_level.hh index c89fff0a5829..52bdf1904f34 100644 --- a/repair/row_level.hh +++ b/repair/row_level.hh @@ -177,7 +177,7 @@ private: public: future<> repair_tablets(repair_uniq_id id, sstring keyspace_name, std::vector table_names, bool primary_replica_only = true, dht::token_range_vector ranges_specified = {}, std::vector dcs = {}, std::unordered_set hosts = {}, std::unordered_set ignore_nodes = {}, std::optional ranges_parallelism = std::nullopt); - future<> repair_tablet(gms::gossip_address_map& addr_map, locator::tablet_metadata_guard& guard, locator::global_tablet_id gid); + future repair_tablet(gms::gossip_address_map& addr_map, locator::tablet_metadata_guard& guard, locator::global_tablet_id gid); private: future repair_update_system_table_handler( diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh index c801ca1c3e83..04b629b1a80e 100644 --- a/repair/task_manager_module.hh +++ b/repair/task_manager_module.hh @@ -111,6 +111,7 @@ private: optimized_optional _abort_subscription; std::optional _ranges_parallelism; size_t _metas_size = 0; + gc_clock::time_point _flush_time; public: tablet_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, sstring keyspace, std::vector tables, streaming::stream_reason reason, std::vector metas, std::optional ranges_parallelism) : repair_task_impl(module, id.uuid(), id.id, "keyspace", keyspace, "", "", tasks::task_id::create_null_id(), reason) @@ -125,6 +126,8 @@ public: return tasks::is_abortable(!_abort_subscription); } + gc_clock::time_point get_flush_time() const { return _flush_time; } + tasks::is_user_task is_user_task() const noexcept override; virtual void release_resources() noexcept override; private: @@ -188,6 +191,7 @@ public: void check_failed_ranges(); void check_in_abort_or_shutdown(); repair_neighbors get_repair_neighbors(const dht::token_range& range); + gc_clock::time_point get_flush_time() const { return _flush_time; } void update_statistics(const repair_stats& stats) { _stats.add(stats); } From 23afbd938cfe29b68262bdf804fe16c89d61fb0e Mon Sep 17 00:00:00 2001 From: Asias He Date: Tue, 10 Dec 2024 10:49:34 +0800 Subject: [PATCH 372/397] test: Introduce guarantee_repair_time_next_second helper The repair time granularity is seconds. This helper makes sure the repair time is different than the previous one. --- test/topology_custom/test_tablet_repair_scheduler.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/test/topology_custom/test_tablet_repair_scheduler.py b/test/topology_custom/test_tablet_repair_scheduler.py index 6079de8984b0..63c2d17fb25f 100644 --- a/test/topology_custom/test_tablet_repair_scheduler.py +++ b/test/topology_custom/test_tablet_repair_scheduler.py @@ -30,6 +30,11 @@ async def inject_error_off(manager, error_name, servers): errs = [manager.api.disable_injection(s.ip_addr, error_name) for s in servers] await asyncio.gather(*errs) +async def guarantee_repair_time_next_second(): + # The repair time granularity is seconds. This ensures the repair time is + # different than the previous one. + await asyncio.sleep(1) + @pytest.mark.asyncio async def test_tablet_manual_repair(manager: ManagerClient): servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager, fast_stats_refresh=False) @@ -41,9 +46,7 @@ async def test_tablet_manual_repair(manager: ManagerClient): map1 = await load_tablet_repair_time(cql, hosts[0:1], table_id) logging.info(f'map1={map1} duration={duration}') - # The repair time granularity is seconds. This makes sure the second repair time - # is different than the previous one. - await asyncio.sleep(1) + await guarantee_repair_time_next_second() start = time.time() await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token) @@ -63,6 +66,9 @@ async def test_tablet_manual_repair_all_tokens(manager: ManagerClient): token = "all" now = datetime.datetime.utcnow() map1 = await load_tablet_repair_time(cql, hosts[0:1], table_id) + + await guarantee_repair_time_next_second() + await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token) map2 = await load_tablet_repair_time(cql, hosts[0:1], table_id) logging.info(f'{map1=} {map2=}') From 0b2fef74bcd6268a52dcfeea9bdcc5793ea40ca2 Mon Sep 17 00:00:00 2001 From: Asias He Date: Tue, 10 Dec 2024 10:49:34 +0800 Subject: [PATCH 373/397] test: Disable flush_cache_time for two tablet repair tests The cache of the hints and batchlog flush makes the exact repair time check difficult in the test. Disabling it for two repair tests that check the exact repair time. --- test/pylib/repair.py | 4 +++- test/topology_custom/test_tablet_repair_scheduler.py | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/test/pylib/repair.py b/test/pylib/repair.py index ca77adf382a5..73f8d15b1afc 100644 --- a/test/pylib/repair.py +++ b/test/pylib/repair.py @@ -28,11 +28,13 @@ async def load_tablet_repair_time(cql, hosts, table_id): return repair_time_map -async def create_table_insert_data_for_repair(manager, rf = 3 , tablets = 8, fast_stats_refresh = True, nr_keys = 256): +async def create_table_insert_data_for_repair(manager, rf = 3 , tablets = 8, fast_stats_refresh = True, nr_keys = 256, disable_flush_cache_time = False): if fast_stats_refresh: config = {'error_injections_at_startup': ['short_tablet_stats_refresh_interval']} else: config = {} + if disable_flush_cache_time: + config.update({'repair_hints_batchlog_flush_cache_time_in_ms': 0}) servers = [await manager.server_add(config=config), await manager.server_add(config=config), await manager.server_add(config=config)] cql = manager.get_cql() await cql.run_async("CREATE KEYSPACE test WITH replication = {{'class': 'NetworkTopologyStrategy', " diff --git a/test/topology_custom/test_tablet_repair_scheduler.py b/test/topology_custom/test_tablet_repair_scheduler.py index 63c2d17fb25f..5ef98b6dfce0 100644 --- a/test/topology_custom/test_tablet_repair_scheduler.py +++ b/test/topology_custom/test_tablet_repair_scheduler.py @@ -37,7 +37,7 @@ async def guarantee_repair_time_next_second(): @pytest.mark.asyncio async def test_tablet_manual_repair(manager: ManagerClient): - servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager, fast_stats_refresh=False) + servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager, fast_stats_refresh=False, disable_flush_cache_time=True) token = -1 start = time.time() @@ -62,7 +62,7 @@ async def test_tablet_manual_repair(manager: ManagerClient): @pytest.mark.asyncio async def test_tablet_manual_repair_all_tokens(manager: ManagerClient): - servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager, fast_stats_refresh=False) + servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager, fast_stats_refresh=False, disable_flush_cache_time=True) token = "all" now = datetime.datetime.utcnow() map1 = await load_tablet_repair_time(cql, hosts[0:1], table_id) From 53e6025aa68f23687cd323495b16dab43befe421 Mon Sep 17 00:00:00 2001 From: Asias He Date: Tue, 10 Dec 2024 10:49:34 +0800 Subject: [PATCH 374/397] repair: Wire repair_time in system.tablets for tombstone gc The repair_time in system.tablets will be updated when repair runs successfully. We can now use it to update the repair time for tombstone gc, i.e, when the system.tablets.repair_time is propagated, call gc_state.update_repair_time() on the node that is the owner of the tablet. Since b3b3e880d3391 ("repair: Reduce hints and batchlog flush"), the repair time that could be used for tombstone gc might be smaller than when the repair is started, so the actual repair time for tombstone gc is returned by the repair rpc call from the repair master node. Fixes #17507 --- idl/storage_service.idl.hh | 8 ++- message/messaging_service.cc | 1 + replica/tablets.cc | 36 +++++++++---- replica/tablets.hh | 2 +- service/storage_service.cc | 54 +++++++++++-------- service/storage_service.hh | 9 ++-- service/topology_coordinator.cc | 15 ++++-- test/boost/tablets_test.cc | 2 +- test/perf/perf_tablets.cc | 2 +- .../test_tablet_repair_scheduler.py | 33 +++++++++++- tombstone_gc.cc | 26 +++++++++ tombstone_gc.hh | 17 ++++++ 12 files changed, 163 insertions(+), 42 deletions(-) diff --git a/idl/storage_service.idl.hh b/idl/storage_service.idl.hh index 48fd5a31e702..2acbdd5b2e1e 100644 --- a/idl/storage_service.idl.hh +++ b/idl/storage_service.idl.hh @@ -6,6 +6,8 @@ * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 */ +#include "service/tablet_operation.hh" + namespace locator { struct tablet_id final { @@ -59,10 +61,14 @@ struct raft_snapshot_pull_params { std::vector tables; }; +struct tablet_operation_repair_result { + gc_clock::time_point repair_time; +}; + verb raft_topology_cmd (raft::server_id dst_id, raft::term_t term, uint64_t cmd_index, service::raft_topology_cmd) -> service::raft_topology_cmd_result; verb [[cancellable]] raft_pull_snapshot (raft::server_id dst_id, service::raft_snapshot_pull_params) -> service::raft_snapshot; verb [[cancellable]] tablet_stream_data (raft::server_id dst_id, locator::global_tablet_id); verb [[cancellable]] tablet_cleanup (raft::server_id dst_id, locator::global_tablet_id); verb [[cancellable]] table_load_stats (raft::server_id dst_id) -> locator::load_stats; -verb [[cancellable]] tablet_repair(raft::server_id dst_id, locator::global_tablet_id); +verb [[cancellable]] tablet_repair(raft::server_id dst_id, locator::global_tablet_id) -> service::tablet_operation_repair_result; } diff --git a/message/messaging_service.cc b/message/messaging_service.cc index 61a84a16258f..b7b4efcea4c7 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -47,6 +47,7 @@ #include "replica/exceptions.hh" #include "serializer.hh" #include "db/per_partition_rate_limit_info.hh" +#include "service/tablet_operation.hh" #include "service/topology_state_machine.hh" #include "service/topology_guard.hh" #include "service/raft/join_node.hh" diff --git a/replica/tablets.cc b/replica/tablets.cc index 31bf48c925af..69c054ba0219 100644 --- a/replica/tablets.cc +++ b/replica/tablets.cc @@ -24,6 +24,7 @@ #include "sstables/sstable_set.hh" #include "dht/token.hh" #include "mutation/async_utils.hh" +#include "compaction/compaction_manager.hh" namespace replica { @@ -462,7 +463,7 @@ void update_tablet_metadata_change_hint(locator::tablet_metadata_change_hint& hi namespace { -tablet_id process_one_row(table_id table, tablet_map& map, tablet_id tid, const cql3::untyped_result_set_row& row) { +tablet_id process_one_row(replica::database* db, table_id table, tablet_map& map, tablet_id tid, const cql3::untyped_result_set_row& row) { tablet_replica_set tablet_replicas; if (row.has("replicas")) { tablet_replicas = deserialize_replica_set(row.get_view("replicas")); @@ -474,8 +475,10 @@ tablet_id process_one_row(table_id table, tablet_map& map, tablet_id tid, const } db_clock::time_point repair_time; + bool update_repair_time = false; if (row.has("repair_time")) { repair_time = row.get_as("repair_time"); + update_repair_time = true; } locator::tablet_task_info repair_task_info; @@ -511,6 +514,21 @@ tablet_id process_one_row(table_id table, tablet_map& map, tablet_id tid, const map.set_tablet(tid, tablet_info{std::move(tablet_replicas), repair_time, repair_task_info, migration_task_info}); + if (update_repair_time && db) { + auto myid = db->get_token_metadata().get_my_id(); + auto range = map.get_token_range(tid); + auto& info = map.get_tablet_info(tid); + for (auto r : info.replicas) { + if (r.host == myid) { + auto& gc_state = db->get_compaction_manager().get_tombstone_gc_state(); + gc_state.insert_pending_repair_time_update(table, range, to_gc_clock(repair_time), r.shard); + tablet_logger.debug("Insert pending repair time for tombstone gc: table={} tablet={} range={} repair_time={}", + table, tid, range, repair_time); + break; + } + } + } + auto persisted_last_token = dht::token::from_int64(row.get_as("last_token")); auto current_last_token = map.get_last_token(tid); if (current_last_token != persisted_last_token) { @@ -531,7 +549,7 @@ struct tablet_metadata_builder { }; std::optional current; - void process_row(const cql3::untyped_result_set_row& row) { + void process_row(const cql3::untyped_result_set_row& row, replica::database* db) { auto table = table_id(row.get_as("table_id")); if (!current || current->table != table) { @@ -557,7 +575,7 @@ struct tablet_metadata_builder { } } - current->tid = process_one_row(current->table, current->map, current->tid, row); + current->tid = process_one_row(db, current->table, current->map, current->tid, row); } void on_end_of_stream() { @@ -576,7 +594,7 @@ future read_tablet_metadata(cql3::query_processor& qp) { try { co_await qp.query_internal("select * from system.tablets", [&] (const cql3::untyped_result_set_row& row) -> future { - builder.process_row(row); + builder.process_row(row, qp.db().real_database_ptr()); return make_ready_future(stop_iteration::no); }); } catch (...) { @@ -635,7 +653,7 @@ do_update_tablet_metadata_partition(cql3::query_processor& qp, tablet_metadata& {data_value(hint.table_id.uuid())}, 1000, [&] (const cql3::untyped_result_set_row& row) -> future { - builder.process_row(row); + builder.process_row(row, qp.db().real_database_ptr()); return make_ready_future(stop_iteration::no); }); if (builder.current) { @@ -646,7 +664,7 @@ do_update_tablet_metadata_partition(cql3::query_processor& qp, tablet_metadata& } static future<> -do_update_tablet_metadata_rows(cql3::query_processor& qp, tablet_map& tmap, const tablet_metadata_change_hint::table_hint& hint) { +do_update_tablet_metadata_rows(replica::database& db, cql3::query_processor& qp, tablet_map& tmap, const tablet_metadata_change_hint::table_hint& hint) { for (const auto token : hint.tokens) { auto res = co_await qp.execute_internal( "select * from system.tablets where table_id = ? and last_token = ?", @@ -658,19 +676,19 @@ do_update_tablet_metadata_rows(cql3::query_processor& qp, tablet_map& tmap, cons throw std::runtime_error("Failed to update tablet metadata: updated row is empty"); } else { tmap.clear_tablet_transition_info(tid); - process_one_row(hint.table_id, tmap, tid, res->one()); + process_one_row(&db, hint.table_id, tmap, tid, res->one()); } } } -future<> update_tablet_metadata(cql3::query_processor& qp, tablet_metadata& tm, const locator::tablet_metadata_change_hint& hint) { +future<> update_tablet_metadata(replica::database& db, cql3::query_processor& qp, tablet_metadata& tm, const locator::tablet_metadata_change_hint& hint) { try { for (const auto& [_, table_hint] : hint.tables) { if (table_hint.tokens.empty()) { co_await do_update_tablet_metadata_partition(qp, tm, table_hint); } else { co_await tm.mutate_tablet_map_async(table_hint.table_id, [&] (tablet_map& tmap) -> future<> { - co_await do_update_tablet_metadata_rows(qp, tmap, table_hint); + co_await do_update_tablet_metadata_rows(db, qp, tmap, table_hint); }); } } diff --git a/replica/tablets.hh b/replica/tablets.hh index f0fc90f3096e..2bb8fa746d28 100644 --- a/replica/tablets.hh +++ b/replica/tablets.hh @@ -93,7 +93,7 @@ future> read_required_hosts(cql3::query_pro /// /// The hint is used to determine what has changed and only reload the changed /// parts from disk, updating the passed-in metadata in-place accordingly. -future<> update_tablet_metadata(cql3::query_processor&, locator::tablet_metadata&, const locator::tablet_metadata_change_hint&); +future<> update_tablet_metadata(replica::database& db, cql3::query_processor&, locator::tablet_metadata&, const locator::tablet_metadata_change_hint&); /// Reads tablet metadata from system.tablets in the form of mutations. future> read_tablet_mutations(seastar::sharded&); diff --git a/service/storage_service.cc b/service/storage_service.cc index ac0095f39714..c469b38afeb3 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -784,7 +784,7 @@ future<> storage_service::topology_state_load(state_change_hint hint) { // We want to update the tablet metadata incrementally, so copy it // from the current token metadata and update only the changed parts. tablets = co_await get_token_metadata().tablets().copy(); - co_await replica::update_tablet_metadata(_qp, *tablets, *hint.tablets_hint); + co_await replica::update_tablet_metadata(_db.local(), _qp, *tablets, *hint.tablets_hint); } else { tablets = co_await replica::read_tablet_metadata(_qp); } @@ -3227,6 +3227,9 @@ future<> storage_service::replicate_to_all_cores(mutable_token_metadata_ptr tmpt for (auto id : open_sessions) { session_mgr.create_session(id); } + + auto& gc_state = db.get_compaction_manager().get_tombstone_gc_state(); + co_await gc_state.flush_pending_repair_time_update(db); }); } catch (...) { // applying the changes on all shards should never fail @@ -5399,7 +5402,7 @@ void storage_service::on_update_tablet_metadata(const locator::tablet_metadata_c future<> storage_service::load_tablet_metadata(const locator::tablet_metadata_change_hint& hint) { return mutate_token_metadata([this, &hint] (mutable_token_metadata_ptr tmptr) -> future<> { if (hint) { - co_await replica::update_tablet_metadata(_qp, tmptr->tablets(), hint); + co_await replica::update_tablet_metadata(_db.local(), _qp, tmptr->tablets(), hint); } else { tmptr->set_tablets(co_await replica::read_tablet_metadata(_qp)); } @@ -5920,9 +5923,9 @@ inet_address storage_service::host2ip(locator::host_id host) const { // may receive stale triggers started in the previous stage, so that those nodes will // see tablet metadata which reflects group0 state. This will cut-off stale triggers // as soon as the coordinator moves to the next stage. -future<> storage_service::do_tablet_operation(locator::global_tablet_id tablet, +future storage_service::do_tablet_operation(locator::global_tablet_id tablet, sstring op_name, - std::function(locator::tablet_metadata_guard&)> op) { + std::function(locator::tablet_metadata_guard&)> op) { // The coordinator may not execute global token metadata barrier before triggering the operation, so we need // a barrier here to see the token metadata which is at least as recent as that of the sender. auto& raft_server = _group0->group0_server(); @@ -5930,8 +5933,8 @@ future<> storage_service::do_tablet_operation(locator::global_tablet_id tablet, if (_tablet_ops.contains(tablet)) { rtlogger.debug("{} retry joining with existing session for tablet {}", op_name, tablet); - co_await _tablet_ops[tablet].done.get_future(); - co_return; + auto result = co_await _tablet_ops[tablet].done.get_future(); + co_return result; } locator::tablet_metadata_guard guard(_db.local().find_column_family(tablet.table), tablet); @@ -5941,18 +5944,19 @@ future<> storage_service::do_tablet_operation(locator::global_tablet_id tablet, }); auto async_gate_holder = _async_gate.hold(); - promise<> p; + promise p; _tablet_ops.emplace(tablet, tablet_operation { - op_name, seastar::shared_future<>(p.get_future()) + op_name, seastar::shared_future(p.get_future()) }); auto erase_registry_entry = seastar::defer([&] { _tablet_ops.erase(tablet); }); try { - co_await op(guard); - p.set_value(); + auto result = co_await op(guard); + p.set_value(result); rtlogger.debug("{} for tablet migration of {} successful", op_name, tablet); + co_return result; } catch (...) { p.set_exception(std::current_exception()); rtlogger.warn("{} for tablet migration of {} failed: {}", op_name, tablet, std::current_exception()); @@ -5960,8 +5964,8 @@ future<> storage_service::do_tablet_operation(locator::global_tablet_id tablet, } } -future<> storage_service::repair_tablet(locator::global_tablet_id tablet) { - return do_tablet_operation(tablet, "Repair", [this, tablet] (locator::tablet_metadata_guard& guard) -> future<> { +future storage_service::repair_tablet(locator::global_tablet_id tablet) { + auto result = co_await do_tablet_operation(tablet, "Repair", [this, tablet] (locator::tablet_metadata_guard& guard) -> future { slogger.debug("Executing repair for tablet={}", tablet); auto& tmap = guard.get_tablet_map(); auto* trinfo = tmap.get_tablet_transition_info(tablet.tablet); @@ -5983,11 +5987,17 @@ future<> storage_service::repair_tablet(locator::global_tablet_id tablet) { utils::get_local_injector().inject("repair_tablet_fail_on_rpc_call", [] { throw std::runtime_error("repair_tablet failed due to error injection"); }); - co_await do_with_repair_service(_repair, [&] (repair_service& local_repair) { - return local_repair.repair_tablet(_address_map, guard, tablet); + service::tablet_operation_repair_result result; + co_await do_with_repair_service(_repair, [&] (repair_service& local_repair) -> future<> { + auto time = co_await local_repair.repair_tablet(_address_map, guard, tablet); + result = service::tablet_operation_repair_result{time}; }); - co_return; + co_return result; }); + if (std::holds_alternative(result)) { + co_return std::get(result); + } + on_internal_error(slogger, "Got wrong tablet_operation_repair_result"); } future<> storage_service::clone_locally_tablet_storage(locator::global_tablet_id tablet, locator::tablet_replica leaving, locator::tablet_replica pending) { @@ -6035,7 +6045,7 @@ future<> storage_service::clone_locally_tablet_storage(locator::global_tablet_id // Streams data to the pending tablet replica of a given tablet on this node. // The source tablet replica is determined from the current transition info of the tablet. future<> storage_service::stream_tablet(locator::global_tablet_id tablet) { - return do_tablet_operation(tablet, "Streaming", [this, tablet] (locator::tablet_metadata_guard& guard) -> future<> { + co_await do_tablet_operation(tablet, "Streaming", [this, tablet] (locator::tablet_metadata_guard& guard) -> future { auto tm = guard.get_token_metadata(); auto& tmap = guard.get_tablet_map(); auto* trinfo = tmap.get_tablet_transition_info(tablet.tablet); @@ -6142,7 +6152,7 @@ future<> storage_service::stream_tablet(locator::global_tablet_id tablet) { return table.maybe_split_compaction_group_of(tablet.tablet); }); - co_return; + co_return tablet_operation_result(); }); } @@ -6152,7 +6162,7 @@ future<> storage_service::cleanup_tablet(locator::global_tablet_id tablet) { _exit(1); }); - return do_tablet_operation(tablet, "Cleanup", [this, tablet] (locator::tablet_metadata_guard& guard) { + co_await do_tablet_operation(tablet, "Cleanup", [this, tablet] (locator::tablet_metadata_guard& guard) -> future { shard_id shard; { @@ -6188,10 +6198,11 @@ future<> storage_service::cleanup_tablet(locator::global_tablet_id tablet) { throw std::runtime_error(fmt::format("Tablet {} stage is not at cleanup/cleanup_target", tablet)); } } - return _db.invoke_on(shard, [tablet, &sys_ks = _sys_ks] (replica::database& db) { + co_await _db.invoke_on(shard, [tablet, &sys_ks = _sys_ks] (replica::database& db) { auto& table = db.find_column_family(tablet.table); return table.cleanup_tablet(db, sys_ks.local(), tablet.tablet); }); + co_return tablet_operation_result(); }); } @@ -7121,8 +7132,9 @@ void storage_service::init_messaging_service() { }); }); ser::storage_service_rpc_verbs::register_tablet_repair(&_messaging.local(), [handle_raft_rpc] (raft::server_id dst_id, locator::global_tablet_id tablet) { - return handle_raft_rpc(dst_id, [tablet] (auto& ss) { - return ss.repair_tablet(tablet); + return handle_raft_rpc(dst_id, [tablet] (auto& ss) -> future { + auto res = co_await ss.repair_tablet(tablet); + co_return res; }); }); ser::storage_service_rpc_verbs::register_tablet_cleanup(&_messaging.local(), [handle_raft_rpc] (raft::server_id dst_id, locator::global_tablet_id tablet) { diff --git a/service/storage_service.hh b/service/storage_service.hh index 549f67c2f6a1..ee02935850a0 100644 --- a/service/storage_service.hh +++ b/service/storage_service.hh @@ -47,6 +47,7 @@ #include "raft/server.hh" #include "service/topology_state_machine.hh" #include "service/tablet_allocator.hh" +#include "service/tablet_operation.hh" #include "utils/user_provided_param.hh" #include "utils/sequenced_set.hh" @@ -151,7 +152,7 @@ private: struct tablet_operation { sstring name; - shared_future<> done; + shared_future done; }; using tablet_op_registry = std::unordered_map; @@ -198,10 +199,10 @@ private: future<> node_ops_abort(node_ops_id ops_uuid); void node_ops_signal_abort(std::optional ops_uuid); future<> node_ops_abort_thread(); - future<> do_tablet_operation(locator::global_tablet_id tablet, + future do_tablet_operation(locator::global_tablet_id tablet, sstring op_name, - std::function(locator::tablet_metadata_guard&)> op); - future<> repair_tablet(locator::global_tablet_id); + std::function(locator::tablet_metadata_guard&)> op); + future repair_tablet(locator::global_tablet_id); future<> stream_tablet(locator::global_tablet_id); // Clones storage of leaving tablet into pending one. Done in the context of intra-node migration, // when both of which sit on the same node. So all the movement is local. diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc index 4e8ca60ef2cd..41c82d494db8 100644 --- a/service/topology_coordinator.cc +++ b/service/topology_coordinator.cc @@ -46,6 +46,7 @@ #include "service/raft/raft_group0.hh" #include "service/raft/raft_group0_client.hh" #include "service/tablet_allocator.hh" +#include "service/tablet_operation.hh" #include "service/topology_state_machine.hh" #include "topology_mutation.hh" #include "utils/assert.hh" @@ -1159,6 +1160,8 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { background_action_holder cleanup; background_action_holder repair; std::unordered_map barriers; + // Record the repair_time returned by the repair_tablet rpc call + db_clock::time_point repair_time; }; std::unordered_map _tablets; @@ -1571,10 +1574,13 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { auto dst = primary.host; auto tablet = gid; rtlogger.info("Initiating tablet repair host={} tablet={}", dst, gid); - co_await ser::storage_service_rpc_verbs::send_tablet_repair(&_messaging, + auto res = co_await ser::storage_service_rpc_verbs::send_tablet_repair(&_messaging, dst, _as, raft::server_id(dst.uuid()), gid); auto duration = std::chrono::duration(db_clock::now() - sched_time); - rtlogger.info("Finished tablet repair host={} tablet={} duration={}", dst, tablet, duration); + auto& tablet_state = _tablets[tablet]; + tablet_state.repair_time = db_clock::from_time_t(gc_clock::to_time_t(res.repair_time)); + rtlogger.info("Finished tablet repair host={} tablet={} duration={} repair_time={}", + dst, tablet, duration, res.repair_time); })) { auto& tinfo = tmap.get_tablet_info(gid.tablet); bool valid = tinfo.repair_task_info.is_valid(); @@ -1584,7 +1590,10 @@ class topology_coordinator : public endpoint_lifecycle_subscriber { .del_repair_task_info(last_token) .del_session(last_token); if (valid) { - auto time = tinfo.repair_task_info.sched_time; + auto sched_time = tinfo.repair_task_info.sched_time; + auto time = tablet_state.repair_time; + rtlogger.debug("Set tablet repair time sched_time={} return_time={} set_time={}", + sched_time, tablet_state.repair_time, time); update.set_repair_time(last_token, time); } updates.emplace_back(update.build()); diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index 3e4c241810f7..1dedd5258f1e 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -78,7 +78,7 @@ void verify_tablet_metadata_update(cql_test_env& env, tablet_metadata& tm, std:: update_tablet_metadata_change_hint(hint, mut); } - update_tablet_metadata(env.local_qp(), tm, hint).get(); + update_tablet_metadata(db, env.local_qp(), tm, hint).get(); auto tm_reload = read_tablet_metadata(env.local_qp()).get(); BOOST_REQUIRE_EQUAL(tm, tm_reload); diff --git a/test/perf/perf_tablets.cc b/test/perf/perf_tablets.cc index d9edff0c03d2..6050142445ae 100644 --- a/test/perf/perf_tablets.cc +++ b/test/perf/perf_tablets.cc @@ -182,7 +182,7 @@ static future<> test_basic_operations(app_template& app) { const auto full_reload_duration = std::chrono::duration(end_full_reload - start_full_reload); const auto start_partial_reload = clk::now(); - update_tablet_metadata(e.local_qp(), tm, hint).get(); + update_tablet_metadata(e.local_db(), e.local_qp(), tm, hint).get(); const auto end_partial_reload = clk::now(); const auto partial_reload_duration = std::chrono::duration(end_partial_reload - start_partial_reload); diff --git a/test/topology_custom/test_tablet_repair_scheduler.py b/test/topology_custom/test_tablet_repair_scheduler.py index 5ef98b6dfce0..00cf7621f3f7 100644 --- a/test/topology_custom/test_tablet_repair_scheduler.py +++ b/test/topology_custom/test_tablet_repair_scheduler.py @@ -7,7 +7,7 @@ from test.pylib.manager_client import ManagerClient from test.topology.conftest import skip_mode from test.pylib.repair import load_tablet_repair_time, create_table_insert_data_for_repair, get_tablet_task_id -from test.pylib.rest_client import inject_error_one_shot +from test.pylib.rest_client import inject_error_one_shot, read_barrier import pytest import asyncio @@ -60,6 +60,37 @@ async def test_tablet_manual_repair(manager: ManagerClient): assert t2 > t1 +@pytest.mark.asyncio +async def test_tombstone_gc_insert_flush(manager: ManagerClient): + servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager, fast_stats_refresh=False, disable_flush_cache_time=True) + token = "all" + logs = [] + for s in servers: + await manager.api.set_logger_level(s.ip_addr, "database", "debug") + await manager.api.set_logger_level(s.ip_addr, "tablets", "debug") + logs.append(await manager.server_open_log(s.server_id)) + + await manager.api.tablet_repair(servers[0].ip_addr, "test", "test", token) + + timeout = 600 + deadline = time.time() + timeout + while True: + done = True + for s in servers: + await read_barrier(manager.api, s.ip_addr) + for log in logs: + inserts = await log.grep(rf'.*Insert pending repair time for tombstone gc: table={table_id}.*') + flushes = await log.grep(rf'.*Flush pending repair time for tombstone gc: table={table_id}.*') + logging.info(f'{inserts=} {flushes=}'); + logging.info(f'{len(inserts)=} {len(flushes)=}'); + ok = len(inserts) == len(flushes) and len(inserts) > 0 + if not ok: + done = False + if done: + break + else: + assert time.time() < deadline + @pytest.mark.asyncio async def test_tablet_manual_repair_all_tokens(manager: ManagerClient): servers, cql, hosts, table_id = await create_table_insert_data_for_repair(manager, fast_stats_refresh=False, disable_flush_cache_time=True) diff --git a/tombstone_gc.cc b/tombstone_gc.cc index d4df8a6e95a8..64ced8102015 100644 --- a/tombstone_gc.cc +++ b/tombstone_gc.cc @@ -18,6 +18,8 @@ #include "replica/database.hh" #include "data_dictionary/data_dictionary.hh" #include "gms/feature_service.hh" +#include "compaction/compaction_manager.hh" +#include extern logging::logger dblog; @@ -217,6 +219,30 @@ void tombstone_gc_state::update_repair_time(table_id id, const dht::token_range& *m += std::make_pair(locator::token_metadata::range_to_interval(range), repair_time); } +void tombstone_gc_state::insert_pending_repair_time_update(table_id id, + const dht::token_range& range, gc_clock::time_point repair_time, shard_id shard) { + _pending_updates[id].push_back(range_repair_time{range, repair_time, shard}); +} + +future<> tombstone_gc_state::flush_pending_repair_time_update(replica::database& db) { + auto pending_updates = std::exchange(_pending_updates, {}); + + co_await db.container().invoke_on_all([&pending_updates] (replica::database &localdb) -> future<> { + auto& gc_state = localdb.get_compaction_manager().get_tombstone_gc_state(); + for (auto& x : pending_updates) { + auto& table = x.first; + for (auto& update : x.second) { + co_await coroutine::maybe_yield(); + if (update.shard == this_shard_id()) { + gc_state.update_repair_time(table, update.range, update.time); + dblog.debug("Flush pending repair time for tombstone gc: table={} range={} repair_time={}", + table, update.range, update.time); + } + } + } + }); +}; + void tombstone_gc_state::update_group0_refresh_time(gc_clock::time_point refresh_time) { auto m = get_or_create_group0_gc_time(); if (!m) { diff --git a/tombstone_gc.hh b/tombstone_gc.hh index 878d512d3ec2..9b28e9658b12 100644 --- a/tombstone_gc.hh +++ b/tombstone_gc.hh @@ -13,6 +13,11 @@ #include "dht/token.hh" #include "schema/schema_fwd.hh" #include "interval.hh" +#include "utils/chunked_vector.hh" + +namespace replica { +class database; +} namespace dht { @@ -49,6 +54,12 @@ class tombstone_gc_options; using gc_time_min_source = std::function; +struct range_repair_time { + dht::token_range range; + gc_clock::time_point time; + shard_id shard; +}; + class tombstone_gc_state { gc_time_min_source _gc_min_source; per_table_history_maps* _reconcile_history_maps; @@ -62,6 +73,9 @@ class tombstone_gc_state { [[nodiscard]] gc_clock::time_point get_gc_before_for_group0(schema_ptr s) const; +private: + std::unordered_map> _pending_updates; + public: tombstone_gc_state() = delete; explicit tombstone_gc_state(per_table_history_maps* maps) noexcept : _reconcile_history_maps(maps) {} @@ -94,6 +108,9 @@ public: // returns a tombstone_gc_state copy with the commitlog check disabled (i.e.) without _gc_min_source. [[nodiscard]] tombstone_gc_state with_commitlog_check_disabled() const { return tombstone_gc_state(_reconcile_history_maps); } + + void insert_pending_repair_time_update(table_id id, const dht::token_range& range, gc_clock::time_point repair_time, shard_id shard); + future<> flush_pending_repair_time_update(replica::database& db); }; std::map get_default_tombstonesonte_gc_mode(data_dictionary::database db, sstring ks_name); From 387b2050df9e493250e4c15b75d4609527a121f4 Mon Sep 17 00:00:00 2001 From: Asias He Date: Tue, 10 Dec 2024 10:49:35 +0800 Subject: [PATCH 375/397] repair: Stop using rpc to update repair time for repairs scheduled by scheduler If a tablet repair is scheduled by tablet repair scheduler, the repair time for tombstone gc will be updated when the system.tablet.repair_time is updated. Skip updating using rpc calls in this case. --- repair/repair.cc | 9 ++++++--- repair/row_level.cc | 9 +++++++++ repair/task_manager_module.hh | 7 ++++++- 3 files changed, 21 insertions(+), 4 deletions(-) diff --git a/repair/repair.cc b/repair/repair.cc index d55a71812c9b..375e33ec7331 100644 --- a/repair/repair.cc +++ b/repair/repair.cc @@ -615,7 +615,8 @@ repair::shard_repair_task_impl::shard_repair_task_impl(tasks::task_manager::modu bool hints_batchlog_flushed, bool small_table_optimization, std::optional ranges_parallelism, - gc_clock::time_point flush_time) + gc_clock::time_point flush_time, + bool sched_by_scheduler) : repair_task_impl(module, id, 0, "shard", keyspace, "", "", parent_id_.uuid(), reason_) , rs(repair) , db(repair.get_db()) @@ -635,6 +636,7 @@ repair::shard_repair_task_impl::shard_repair_task_impl(tasks::task_manager::modu , _small_table_optimization(small_table_optimization) , _user_ranges_parallelism(ranges_parallelism ? std::optional(semaphore(*ranges_parallelism)) : std::nullopt) , _flush_time(flush_time) + , sched_by_scheduler(sched_by_scheduler) { rlogger.debug("repair[{}]: Setting user_ranges_parallelism to {}", global_repair_id.uuid(), _user_ranges_parallelism ? std::to_string(_user_ranges_parallelism->available_units()) : "unlimited"); @@ -2435,6 +2437,7 @@ future repair_service::repair_tablet(gms::gossip_address_m auto start = std::chrono::steady_clock::now(); task_metas.push_back(tablet_repair_task_meta{keyspace_name, table_name, table_id, master_shard_id, range, repair_neighbors(nodes, shards), replicas}); auto task_impl_ptr = seastar::make_shared(_repair_module, id, keyspace_name, table_names, streaming::stream_reason::repair, std::move(task_metas), ranges_parallelism); + task_impl_ptr->sched_by_scheduler = true; auto task = co_await _repair_module->make_task(task_impl_ptr, {}); task->start(); co_await task->done(); @@ -2524,7 +2527,7 @@ future<> repair::tablet_repair_task_impl::run() { auto parent_shard = this_shard_id(); std::vector flush_times(smp::count); - rs.container().invoke_on_all([&idx, &flush_times, id, metas = _metas, parent_data, reason = _reason, tables = _tables, ranges_parallelism = _ranges_parallelism, parent_shard] (repair_service& rs) -> future<> { + rs.container().invoke_on_all([&idx, &flush_times, id, metas = _metas, parent_data, reason = _reason, tables = _tables, sched_by_scheduler = sched_by_scheduler, ranges_parallelism = _ranges_parallelism, parent_shard] (repair_service& rs) -> future<> { std::exception_ptr error; for (auto& m : metas) { if (m.master_shard_id != this_shard_id()) { @@ -2558,7 +2561,7 @@ future<> repair::tablet_repair_task_impl::run() { auto task_impl_ptr = seastar::make_shared(rs._repair_module, tasks::task_id::create_random_id(), m.keyspace_name, rs, erm, std::move(ranges), std::move(table_ids), id, std::move(data_centers), std::move(hosts), - std::move(ignore_nodes), reason, hints_batchlog_flushed, small_table_optimization, ranges_parallelism, flush_time); + std::move(ignore_nodes), reason, hints_batchlog_flushed, small_table_optimization, ranges_parallelism, flush_time, sched_by_scheduler); task_impl_ptr->neighbors = std::move(neighbors); auto task = co_await rs._repair_module->make_task(task_impl_ptr, parent_data); task->start(); diff --git a/repair/row_level.cc b/repair/row_level.cc index ac05787d91e0..3019cd60cee2 100644 --- a/repair/row_level.cc +++ b/repair/row_level.cc @@ -2992,6 +2992,15 @@ class row_level_repair { if (!_shard_task.hints_batchlog_flushed()) { co_return; } + + // The tablet repair time for tombstone gc will be updated when the + // system.tablet.repair_time is updated. + if (_is_tablet && _shard_task.sched_by_scheduler) { + rlogger.debug("repair[{}]: Skipped to update system.repair_history for tablet repair scheduled by scheduler total_rf={} repaired_replicas={} local={} peers={}", + _shard_task.global_repair_id.uuid(), _shard_task.total_rf, repaired_replicas, my_address, _all_live_peer_nodes); + co_return; + } + repair_service& rs = _shard_task.rs; std::optional repair_time_opt = co_await rs.update_history(_shard_task.global_repair_id.uuid(), _table_id, _range, _start_time, _is_tablet); if (!repair_time_opt) { diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh index 04b629b1a80e..e82759f22301 100644 --- a/repair/task_manager_module.hh +++ b/repair/task_manager_module.hh @@ -112,6 +112,8 @@ private: std::optional _ranges_parallelism; size_t _metas_size = 0; gc_clock::time_point _flush_time; +public: + bool sched_by_scheduler = false; public: tablet_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, sstring keyspace, std::vector tables, streaming::stream_reason reason, std::vector metas, std::optional ranges_parallelism) : repair_task_impl(module, id.uuid(), id.id, "keyspace", keyspace, "", "", tasks::task_id::create_null_id(), reason) @@ -171,6 +173,8 @@ private: std::optional _user_ranges_parallelism; uint64_t _ranges_complete = 0; gc_clock::time_point _flush_time; +public: + bool sched_by_scheduler = false; public: shard_repair_task_impl(tasks::task_manager::module_ptr module, tasks::task_id id, @@ -187,7 +191,8 @@ public: bool hints_batchlog_flushed, bool small_table_optimization, std::optional ranges_parallelism, - gc_clock::time_point flush_time); + gc_clock::time_point flush_time, + bool sched_by_scheduler = false); void check_failed_ranges(); void check_in_abort_or_shutdown(); repair_neighbors get_repair_neighbors(const dht::token_range& range); From adef719c43249fea37af538cca9b34ad264ed0b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Paszkowski?= Date: Fri, 10 Jan 2025 11:43:38 +0100 Subject: [PATCH 376/397] api/storage_service: Remove unimplemented truncate API The API /storage_service/truncate/{ks} returns an unimplemented error when invoked. As we already have a CQL command, `TRUNCATE TABLE ks.cf` that causes the table to be truncated on all nodes, the API can be dropped. Due to the error, it is unused. Fixes https://github.com/scylladb/scylladb/issues/10520 No backport is required. A small cleanup of not working API. Closes scylladb/scylladb#22258 --- api/api-doc/storage_service.json | 32 -------------------------------- api/storage_service.cc | 8 -------- 2 files changed, 40 deletions(-) diff --git a/api/api-doc/storage_service.json b/api/api-doc/storage_service.json index e43bb3caf243..8235c50d98a1 100644 --- a/api/api-doc/storage_service.json +++ b/api/api-doc/storage_service.json @@ -1648,38 +1648,6 @@ } ] }, - { - "path":"/storage_service/truncate/{keyspace}", - "operations":[ - { - "method":"POST", - "summary":"Truncates (deletes) the given columnFamily from the provided keyspace. Calling truncate results in actual deletion of all data in the cluster under the given columnFamily and it will fail unless all hosts are up. All data in the given column family will be deleted, but its definition will not be affected.", - "type":"void", - "nickname":"truncate", - "produces":[ - "application/json" - ], - "parameters":[ - { - "name":"keyspace", - "description":"The keyspace", - "required":true, - "allowMultiple":false, - "type":"string", - "paramType":"path" - }, - { - "name":"cf", - "description":"Column family name", - "required":false, - "allowMultiple":false, - "type":"string", - "paramType":"query" - } - ] - } - ] - }, { "path":"/storage_service/keyspaces", "operations":[ diff --git a/api/storage_service.cc b/api/storage_service.cc index 6e5cdbfbbffc..b57413611e97 100644 --- a/api/storage_service.cc +++ b/api/storage_service.cc @@ -990,13 +990,6 @@ void set_storage_service(http_context& ctx, routes& r, sharded(json_void()); }); }); - ss::truncate.set(r, [&ctx](std::unique_ptr req) { - //TBD - unimplemented(); - auto keyspace = validate_keyspace(ctx, req); - auto column_family = req->get_query_param("cf"); - return make_ready_future(json_void()); - }); ss::get_keyspaces.set(r, [&ctx](const_req req) { auto type = req.get_query_param("type"); @@ -1620,7 +1613,6 @@ void unset_storage_service(http_context& ctx, routes& r) { ss::is_starting.unset(r); ss::get_drain_progress.unset(r); ss::drain.unset(r); - ss::truncate.unset(r); ss::get_keyspaces.unset(r); ss::stop_gossiping.unset(r); ss::start_gossiping.unset(r); From 55963f8f795db2a3233f9397707e95356c347885 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Thu, 9 Jan 2025 09:55:31 -0500 Subject: [PATCH 377/397] replica: remove noexcept from token -> tablet resolution path The methods to resolve a key/token/range to a table are all noexcept. Yet the method below all of these, `storage_group_for_id()` can throw. This means that if due to any mistake a tablet without local replica is attempted to be looked up, it will result in a crash, as the exception bubbles up into the noexcept methods. There is no value in pretending that looking up the tablet replica is noexcept, remove the noexcept specifiers so that any bad lookup only fails the operation at hand and doesn't crash the node. This is especially relevant to replace, which still has a window where writes can arrive for tablets that don't (yet) have a local replica. Currently, this results in a crash. After this patch, this will only fail the writes and the replace can move on. Fixes: #21480 Closes scylladb/scylladb#22251 --- replica/compaction_group.hh | 8 ++++---- replica/database.hh | 10 +++++----- replica/table.cc | 32 ++++++++++++++++---------------- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/replica/compaction_group.hh b/replica/compaction_group.hh index 839ea13fccaa..8d2c30979130 100644 --- a/replica/compaction_group.hh +++ b/replica/compaction_group.hh @@ -358,13 +358,13 @@ public: // new tablet replica is allocated). virtual void update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) = 0; - virtual compaction_group& compaction_group_for_token(dht::token token) const noexcept = 0; + virtual compaction_group& compaction_group_for_token(dht::token token) const = 0; virtual utils::chunked_vector compaction_groups_for_token_range(dht::token_range tr) const = 0; - virtual compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const noexcept = 0; - virtual compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const noexcept = 0; + virtual compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const = 0; + virtual compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const = 0; virtual size_t log2_storage_groups() const = 0; - virtual storage_group& storage_group_for_token(dht::token) const noexcept = 0; + virtual storage_group& storage_group_for_token(dht::token) const = 0; virtual locator::table_load_stats table_load_stats(std::function tablet_filter) const noexcept = 0; virtual bool all_storage_groups_split() = 0; diff --git a/replica/database.hh b/replica/database.hh index e08f00edd2e3..2b3f1e92cbbe 100644 --- a/replica/database.hh +++ b/replica/database.hh @@ -608,19 +608,19 @@ private: future<> handle_tablet_split_completion(size_t old_tablet_count, const locator::tablet_map& new_tmap); // Select a storage group from a given token. - storage_group& storage_group_for_token(dht::token token) const noexcept; + storage_group& storage_group_for_token(dht::token token) const; storage_group& storage_group_for_id(size_t i) const; std::unique_ptr make_storage_group_manager(); - compaction_group* get_compaction_group(size_t id) const noexcept; + compaction_group* get_compaction_group(size_t id) const; // Select a compaction group from a given token. - compaction_group& compaction_group_for_token(dht::token token) const noexcept; + compaction_group& compaction_group_for_token(dht::token token) const; // Return compaction groups, present in this shard, that own a particular token range. utils::chunked_vector compaction_groups_for_token_range(dht::token_range tr) const; // Select a compaction group from a given key. - compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const noexcept; + compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const; // Select a compaction group from a given sstable based on its token range. - compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const noexcept; + compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const; // Safely iterate through compaction groups, while performing async operations on them. future<> parallel_foreach_compaction_group(std::function(compaction_group&)> action); void for_each_compaction_group(std::function action); diff --git a/replica/table.cc b/replica/table.cc index 72481e7ebb63..69e119fe5a7c 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -696,7 +696,7 @@ class single_storage_group_manager final : public storage_group_manager { void update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) override {} - compaction_group& compaction_group_for_token(dht::token token) const noexcept override { + compaction_group& compaction_group_for_token(dht::token token) const override { return get_compaction_group(); } utils::chunked_vector compaction_groups_for_token_range(dht::token_range tr) const override { @@ -704,16 +704,16 @@ class single_storage_group_manager final : public storage_group_manager { ret.push_back(&get_compaction_group()); return ret; } - compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const noexcept override { + compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const override { return get_compaction_group(); } - compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const noexcept override { + compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const override { return get_compaction_group(); } size_t log2_storage_groups() const override { return 0; } - storage_group& storage_group_for_token(dht::token token) const noexcept override { + storage_group& storage_group_for_token(dht::token token) const override { return *_single_sg; } @@ -843,15 +843,15 @@ class tablet_storage_group_manager final : public storage_group_manager { void update_effective_replication_map(const locator::effective_replication_map& erm, noncopyable_function refresh_mutation_source) override; - compaction_group& compaction_group_for_token(dht::token token) const noexcept override; + compaction_group& compaction_group_for_token(dht::token token) const override; utils::chunked_vector compaction_groups_for_token_range(dht::token_range tr) const override; - compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const noexcept override; - compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const noexcept override; + compaction_group& compaction_group_for_key(partition_key_view key, const schema_ptr& s) const override; + compaction_group& compaction_group_for_sstable(const sstables::shared_sstable& sst) const override; size_t log2_storage_groups() const override { return log2ceil(tablet_map().tablet_count()); } - storage_group& storage_group_for_token(dht::token token) const noexcept override { + storage_group& storage_group_for_token(dht::token token) const override { return storage_group_for_id(storage_group_of(token).first); } @@ -1117,11 +1117,11 @@ std::unique_ptr table::make_storage_group_manager() { return ret; } -compaction_group* table::get_compaction_group(size_t id) const noexcept { +compaction_group* table::get_compaction_group(size_t id) const { return storage_group_for_id(id).main_compaction_group().get(); } -storage_group& table::storage_group_for_token(dht::token token) const noexcept { +storage_group& table::storage_group_for_token(dht::token token) const { return _sg_manager->storage_group_for_token(token); } @@ -1129,13 +1129,13 @@ storage_group& table::storage_group_for_id(size_t i) const { return _sg_manager->storage_group_for_id(_schema, i); } -compaction_group& tablet_storage_group_manager::compaction_group_for_token(dht::token token) const noexcept { +compaction_group& tablet_storage_group_manager::compaction_group_for_token(dht::token token) const { auto [idx, range_side] = storage_group_of(token); auto& sg = storage_group_for_id(idx); return *sg.select_compaction_group(range_side); } -compaction_group& table::compaction_group_for_token(dht::token token) const noexcept { +compaction_group& table::compaction_group_for_token(dht::token token) const { return _sg_manager->compaction_group_for_token(token); } @@ -1166,15 +1166,15 @@ utils::chunked_vector table::compaction_groups_for_token_rang return _sg_manager->compaction_groups_for_token_range(tr); } -compaction_group& tablet_storage_group_manager::compaction_group_for_key(partition_key_view key, const schema_ptr& s) const noexcept { +compaction_group& tablet_storage_group_manager::compaction_group_for_key(partition_key_view key, const schema_ptr& s) const { return compaction_group_for_token(dht::get_token(*s, key)); } -compaction_group& table::compaction_group_for_key(partition_key_view key, const schema_ptr& s) const noexcept { +compaction_group& table::compaction_group_for_key(partition_key_view key, const schema_ptr& s) const { return _sg_manager->compaction_group_for_key(key, s); } -compaction_group& tablet_storage_group_manager::compaction_group_for_sstable(const sstables::shared_sstable& sst) const noexcept { +compaction_group& tablet_storage_group_manager::compaction_group_for_sstable(const sstables::shared_sstable& sst) const { auto [first_id, first_range_side] = storage_group_of(sst->get_first_decorated_key().token()); auto [last_id, last_range_side] = storage_group_of(sst->get_last_decorated_key().token()); @@ -1196,7 +1196,7 @@ compaction_group& tablet_storage_group_manager::compaction_group_for_sstable(con } } -compaction_group& table::compaction_group_for_sstable(const sstables::shared_sstable& sst) const noexcept { +compaction_group& table::compaction_group_for_sstable(const sstables::shared_sstable& sst) const { return _sg_manager->compaction_group_for_sstable(sst); } From 4a5a00347fd8e6ce373b8acb248331c8e59ff783 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 7 Jan 2025 15:00:24 +0800 Subject: [PATCH 378/397] utils: do not include unused headers these unused includes were identifier by clang-include-cleaner. after auditing these source files, all of the reports have been confirmed. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22201 --- api/config.cc | 1 + utils/config_file.hh | 2 -- utils/disk_space_monitor.hh | 2 -- 3 files changed, 1 insertion(+), 4 deletions(-) diff --git a/api/config.cc b/api/config.cc index 29fdabd647c5..950391eeb22c 100644 --- a/api/config.cc +++ b/api/config.cc @@ -14,6 +14,7 @@ #include "replica/database.hh" #include "db/config.hh" #include +#include #include #include diff --git a/utils/config_file.hh b/utils/config_file.hh index a6fc4eb828e6..cd9d5acb6c45 100644 --- a/utils/config_file.hh +++ b/utils/config_file.hh @@ -10,8 +10,6 @@ #pragma once #include -#include -#include #include #include diff --git a/utils/disk_space_monitor.hh b/utils/disk_space_monitor.hh index afcc7165784b..1aa99a2fc7e0 100644 --- a/utils/disk_space_monitor.hh +++ b/utils/disk_space_monitor.hh @@ -8,7 +8,6 @@ #pragma once -#include #include #include @@ -18,7 +17,6 @@ #include #include #include -#include #include #include "seastarx.hh" From 1ef2d9d076928ee198e97e0eb84973130ef6cbac Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Tue, 14 Jan 2025 17:46:33 +0800 Subject: [PATCH 379/397] tree: migrate from boost::adaptors::transformed to std::views::transform Replace remaining uses of boost::adaptors::transformed with std::views::transform to reduce Boost dependencies, following the migration pattern established in bab12e3a. This change addresses recently merged code that reintroduced Boost header dependencies through boost::adaptors::transformed usage. Signed-off-by: Kefu Chai Closes scylladb/scylladb#22365 --- compaction/incremental_backlog_tracker.cc | 2 +- compaction/incremental_compaction_strategy.cc | 6 +++--- reader_concurrency_semaphore_group.hh | 2 +- service/qos/qos_common.cc | 8 +++----- service/task_manager_module.cc | 1 - sstables_loader.cc | 4 +--- test/boost/incremental_compaction_test.cc | 9 +++++---- test/boost/reader_concurrency_semaphore_test.cc | 4 ++-- 8 files changed, 16 insertions(+), 20 deletions(-) diff --git a/compaction/incremental_backlog_tracker.cc b/compaction/incremental_backlog_tracker.cc index 85263d7508b8..322cda38b421 100644 --- a/compaction/incremental_backlog_tracker.cc +++ b/compaction/incremental_backlog_tracker.cc @@ -31,7 +31,7 @@ incremental_backlog_tracker::calculate_sstables_backlog_contribution(const std:: if (!all.empty()) { auto freeze = [] (const sstable_run& run) { return make_lw_shared(run); }; - for (auto& bucket : incremental_compaction_strategy::get_buckets(boost::copy_range>(all | boost::adaptors::map_values | boost::adaptors::transformed(freeze)), options)) { + for (auto& bucket : incremental_compaction_strategy::get_buckets(all | std::views::values | std::views::transform(freeze) | std::ranges::to(), options)) { if (!incremental_compaction_strategy::is_bucket_interesting(bucket, threshold)) { continue; } diff --git a/compaction/incremental_compaction_strategy.cc b/compaction/incremental_compaction_strategy.cc index f07806b1aeaf..3a4bf0d434df 100644 --- a/compaction/incremental_compaction_strategy.cc +++ b/compaction/incremental_compaction_strategy.cc @@ -372,7 +372,7 @@ incremental_compaction_strategy::get_sstables_for_compaction(table_state& t, str }; auto total_size = [] (const size_bucket_t& bucket) -> uint64_t { - return boost::accumulate(bucket | boost::adaptors::transformed(std::mem_fn(&sstable_run::data_size)), uint64_t(0)); + return std::ranges::fold_left(bucket | std::views::transform(std::mem_fn(&sstable_run::data_size)), uint64_t(0), std::plus{}); }; auto [s0, s1] = find_two_largest_tiers(std::move(buckets)); @@ -433,7 +433,7 @@ incremental_compaction_strategy::sstables_to_runs(std::vector ss (void)runs[sst->run_identifier()].insert(std::move(sst)); } auto freeze = [] (const sstable_run& run) { return make_lw_shared(run); }; - return boost::copy_range>(runs | boost::adaptors::map_values | boost::adaptors::transformed(freeze)); + return runs | std::views::values | std::views::transform(freeze) | std::ranges::to(); } void incremental_compaction_strategy::sort_run_bucket_by_first_key(size_bucket_t& bucket, size_t max_elements, const schema_ptr& schema) { @@ -457,7 +457,7 @@ incremental_compaction_strategy::get_reshaping_job(std::vector i offstrategy_threshold = max_sstables; } - auto run_count = boost::copy_range>(input | boost::adaptors::transformed(std::mem_fn(&sstable::run_identifier))).size(); + auto run_count = std::ranges::size(input | std::views::transform(std::mem_fn(&sstable::run_identifier)) | std::ranges::to()); if (run_count >= offstrategy_threshold && mode == reshape_mode::strict) { std::sort(input.begin(), input.end(), [&schema] (const shared_sstable& a, const shared_sstable& b) { return dht::ring_position(a->get_first_decorated_key()).less_compare(*schema, dht::ring_position(b->get_first_decorated_key())); diff --git a/reader_concurrency_semaphore_group.hh b/reader_concurrency_semaphore_group.hh index 6d073ef47228..349f8e79cb46 100644 --- a/reader_concurrency_semaphore_group.hh +++ b/reader_concurrency_semaphore_group.hh @@ -85,6 +85,6 @@ public: auto sum_read_concurrency_sem_var(std::invocable auto member) { using ret_type = std::invoke_result_t; - return boost::accumulate(_semaphores | boost::adaptors::map_values | boost::adaptors::transformed([=] (weighted_reader_concurrency_semaphore& wrcs) { return std::invoke(member, wrcs.sem); }), ret_type(0)); + return std::ranges::fold_left(_semaphores | std::views::values | std::views::transform([=] (weighted_reader_concurrency_semaphore& wrcs) { return std::invoke(member, wrcs.sem); }), ret_type(0), std::plus{}); } }; diff --git a/service/qos/qos_common.cc b/service/qos/qos_common.cc index e95733aa285c..bd9077778237 100644 --- a/service/qos/qos_common.cc +++ b/service/qos/qos_common.cc @@ -12,8 +12,6 @@ #include "cql3/query_processor.hh" #include "cql3/result_set.hh" #include "cql3/untyped_result_set.hh" -#include -#include #include namespace qos { @@ -212,11 +210,11 @@ static qos::service_level_options::shares_type get_shares(const cql3::untyped_re return *shares_opt; } -static sstring get_columns(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name) { +static std::string get_columns(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name) { auto schema = qp.db().find_schema(ks_name, cf_name); - return boost::algorithm::join(schema->all_columns() | boost::adaptors::transformed([] (const auto& col) { + return fmt::to_string(fmt::join(schema->all_columns() | std::views::transform([] (const auto& col) { return col.name_as_cql_string(); - }), " ,"); + }), " ,")); } future get_service_levels(cql3::query_processor& qp, std::string_view ks_name, std::string_view cf_name, db::consistency_level cl, qos::query_context ctx) { diff --git a/service/task_manager_module.cc b/service/task_manager_module.cc index 3262f8e9ec81..37b73a693325 100644 --- a/service/task_manager_module.cc +++ b/service/task_manager_module.cc @@ -13,7 +13,6 @@ #include "tasks/task_handler.hh" #include "tasks/virtual_task_hint.hh" #include -#include namespace service { diff --git a/sstables_loader.cc b/sstables_loader.cc index 7c7aac6072ee..eab2642318a9 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -28,8 +28,6 @@ #include #include -#include - static logging::logger llog("sstables_loader"); namespace { @@ -398,7 +396,7 @@ future<> sstable_streamer::stream_sstables(const dht::partition_range& pr, std:: auto ops_uuid = streaming::plan_id{utils::make_random_uuid()}; llog.info("load_and_stream: started ops_uuid={}, process [{}-{}] out of {} sstables=[{}]", ops_uuid, nr_sst_current, nr_sst_current + sst_processed.size(), nr_sst_total, - fmt::join(sst_processed | boost::adaptors::transformed([] (auto sst) { return sst->get_filename(); }), ", ")); + fmt::join(sst_processed | std::views::transform([] (auto sst) { return sst->get_filename(); }), ", ")); nr_sst_current += sst_processed.size(); co_await stream_sstable_mutations(ops_uuid, pr, std::move(sst_processed)); if (progress) { diff --git a/test/boost/incremental_compaction_test.cc b/test/boost/incremental_compaction_test.cc index 0a79d5fa1e9b..84ae733f0315 100644 --- a/test/boost/incremental_compaction_test.cc +++ b/test/boost/incremental_compaction_test.cc @@ -146,8 +146,9 @@ SEASTAR_TEST_CASE(incremental_compaction_test) { }); BOOST_REQUIRE(desc.sstables.size() == expected_input); - auto sstable_run = boost::copy_range>(desc.sstables - | boost::adaptors::transformed([] (auto& sst) { return sst->generation(); })); + auto sstable_run = desc.sstables + | std::views::transform([] (auto& sst) { return sst->generation(); }) + | std::ranges::to(); auto expected_sst = sstable_run.begin(); auto closed_sstables_tracker = sstable_run.begin(); auto replacer = [&] (compaction_completion_desc ccd) { @@ -223,7 +224,7 @@ SEASTAR_THREAD_TEST_CASE(incremental_compaction_sag_test) { double space_amplification() const { auto sstables = _cf->get_sstables(); - auto total = boost::accumulate(*sstables | boost::adaptors::transformed(std::mem_fn(&sstable::data_size)), uint64_t(0)); + auto total = std::ranges::fold_left(*sstables | std::views::transform(std::mem_fn(&sstable::data_size)), uint64_t(0), std::plus{}); return double(total) / data_set_size; } @@ -255,7 +256,7 @@ SEASTAR_THREAD_TEST_CASE(incremental_compaction_sag_test) { if (desc.sstables.empty()) { break; } - auto total = boost::accumulate(desc.sstables | boost::adaptors::transformed(std::mem_fn(&sstable::data_size)), uint64_t(0)); + auto total = std::ranges::fold_left(desc.sstables | std::views::transform(std::mem_fn(&sstable::data_size)), uint64_t(0), std::plus{}); std::vector new_ssts = { make_sstable_with_size(std::min(total, data_set_size)) }; column_family_test(_cf).rebuild_sstable_list(table_s, new_ssts, desc.sstables).get(); } diff --git a/test/boost/reader_concurrency_semaphore_test.cc b/test/boost/reader_concurrency_semaphore_test.cc index be0283916dae..ac6c4b6ff7aa 100644 --- a/test/boost/reader_concurrency_semaphore_test.cc +++ b/test/boost/reader_concurrency_semaphore_test.cc @@ -1245,8 +1245,8 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_group) { const auto max_sched_groups = 8; auto check_sem_group = [&] { - const auto total_shares = boost::accumulate(scheduling_groups - | boost::adaptors::transformed([] (const scheduling_group_with_shares& sgs) { return sgs.shares; }), size_t(0)); + const auto total_shares = std::ranges::fold_left(scheduling_groups + | std::views::transform([] (const scheduling_group_with_shares& sgs) { return sgs.shares; }), size_t(0), std::plus{}); ssize_t total_memory = 0; sem_group.foreach_semaphore([&] (scheduling_group sg, reader_concurrency_semaphore& sem) { const auto res = sem.available_resources(); From 0110eb050605b94da9a1ea8689f581b728965b53 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Wed, 13 Dec 2023 13:17:40 +0200 Subject: [PATCH 380/397] config: add enable_create_table_with_compact_storage As discussed in https://github.com/scylladb/scylladb/issues/12263#issuecomment-1853576813, compact storage tables are deprecated. Yet, there's is nothing in the code that prevents users from creating such tables. This patch adds a live-updateable config option: `enable_create_table_with_compact_storage` that require users to opt-in in order to create new tables WITH COMPACT STORAGE. The option is currently set to `true` by default in db/config to reduce the churn to tests and to `false` in scylla.yaml, for new clusters. TODO: once regressions tests that use compact storage are converted to enable the option, change the default in db/config to false. A unit test was added to test/cql-pytest that checks that the respective cql query fails as expected with the default option or when it is explicitly set to `false`, and that the query succeeds when the option is set to `true`. Note that `check_restricted_table_properties` already returns an optional warning, but it is only logged but not returned in the `prepared_statement`. Fixing that is out of the scope of this patch. See https://github.com/scylladb/scylladb/issues/20945 Signed-off-by: Benny Halevy --- conf/scylla.yaml | 3 ++ cql3/statements/alter_table_statement.cc | 2 ++ cql3/statements/create_table_statement.cc | 17 ++++++++++- cql3/statements/prepared_statement.hh | 2 +- cql3/statements/raw/parsed_statement.cc | 4 +-- db/config.cc | 1 + db/config.hh | 2 ++ test/cqlpy/test_guardrail_compact_storage.py | 30 ++++++++++++++++++++ 8 files changed, 57 insertions(+), 4 deletions(-) create mode 100644 test/cqlpy/test_guardrail_compact_storage.py diff --git a/conf/scylla.yaml b/conf/scylla.yaml index 58234e5ada4f..5c853fc0eb94 100644 --- a/conf/scylla.yaml +++ b/conf/scylla.yaml @@ -674,6 +674,9 @@ maintenance_socket: ignore # - SimpleStrategy # replication_strategy_fail_list: +# Guardrail to enable the deprecated feature of CREATE TABLE WITH COMPACT STORAGE. +# enable_create_table_with_compact_storage: false + # Enable tablets for new keyspaces. # When enabled, newly created keyspaces will have tablets enabled by default. # That can be explicitly disabled in the CREATE KEYSPACE query diff --git a/cql3/statements/alter_table_statement.cc b/cql3/statements/alter_table_statement.cc index 7ae3783d0957..f8bec2f467f3 100644 --- a/cql3/statements/alter_table_statement.cc +++ b/cql3/statements/alter_table_statement.cc @@ -448,6 +448,8 @@ alter_table_statement::raw_statement::prepare(data_dictionary::database db, cql_ std::optional s = t ? std::make_optional(t->schema()) : std::nullopt; std::optional warning = check_restricted_table_properties(db, s, keyspace(), column_family(), *_properties); if (warning) { + // FIXME: should this warning be returned to the caller? + // See https://github.com/scylladb/scylladb/issues/20945 mylogger.warn("{}", *warning); } diff --git a/cql3/statements/create_table_statement.cc b/cql3/statements/create_table_statement.cc index 4341ae1d4ecd..d7b720e5b6c5 100644 --- a/cql3/statements/create_table_statement.cc +++ b/cql3/statements/create_table_statement.cc @@ -185,8 +185,17 @@ std::unique_ptr create_table_statement::raw_statement::prepa if (_properties.properties()->get_synchronous_updates_flag()) { throw exceptions::invalid_request_exception(format("The synchronous_updates option is only applicable to materialized views, not to base tables")); } + std::vector stmt_warnings; + auto stmt_warning = [&] (sstring msg) { + if (this_shard_id() == 0) { + mylogger.warn("{}: {}", cf_name, msg); + } + stmt_warnings.emplace_back(std::move(msg)); + }; std::optional warning = check_restricted_table_properties(db, std::nullopt, keyspace(), column_family(), *_properties.properties()); if (warning) { + // FIXME: should this warning be returned to the caller? + // See https://github.com/scylladb/scylladb/issues/20945 mylogger.warn("{}", *warning); } const bool has_default_ttl = _properties.properties()->get_default_time_to_live() > 0; @@ -240,6 +249,12 @@ std::unique_ptr create_table_statement::raw_statement::prepa } stmt->_use_compact_storage = _properties.use_compact_storage(); + if (stmt->_use_compact_storage) { + if (!db.get_config().enable_create_table_with_compact_storage()) { + throw exceptions::invalid_request_exception("Support for the deprecated feature of 'CREATE TABLE WITH COMPACT STORAGE' is disabled and will eventually be removed in a future version. To enable, set the 'enable_create_table_with_compact_storage' config option to 'true'."); + } + stmt_warning("CREATE TABLE WITH COMPACT STORAGE is deprecated and will eventually be removed in a future version."); + } auto& key_aliases = _key_aliases[0]; std::vector key_types; @@ -390,7 +405,7 @@ std::unique_ptr create_table_statement::raw_statement::prepa } } - return std::make_unique(audit_info(), stmt); + return std::make_unique(audit_info(), stmt, std::move(stmt_warnings)); } data_type create_table_statement::raw_statement::get_type_and_remove(column_map_type& columns, ::shared_ptr t) diff --git a/cql3/statements/prepared_statement.hh b/cql3/statements/prepared_statement.hh index cd9860fa948a..9425144b1ce3 100644 --- a/cql3/statements/prepared_statement.hh +++ b/cql3/statements/prepared_statement.hh @@ -51,7 +51,7 @@ public: prepared_statement(audit::audit_info_ptr&& audit_info, seastar::shared_ptr statement_, prepare_context&& ctx, std::vector&& partition_key_bind_indices); - prepared_statement(audit::audit_info_ptr&& audit_info, seastar::shared_ptr&& statement_); + prepared_statement(audit::audit_info_ptr&& audit_info, seastar::shared_ptr&& statement_, std::vector warnings = {}); checked_weak_ptr checked_weak_from_this() const { return checked_weak_ptr(this->weak_from_this()); diff --git a/cql3/statements/raw/parsed_statement.cc b/cql3/statements/raw/parsed_statement.cc index 4d01291efd9a..9b58a12cec75 100644 --- a/cql3/statements/raw/parsed_statement.cc +++ b/cql3/statements/raw/parsed_statement.cc @@ -62,8 +62,8 @@ prepared_statement::prepared_statement(audit::audit_info_ptr&& audit_info, ::sha : prepared_statement(std::move(audit_info), statement_, std::move(ctx).get_variable_specifications(), std::move(partition_key_bind_indices)) { } -prepared_statement::prepared_statement(audit::audit_info_ptr&& audit_info, ::shared_ptr&& statement_) - : prepared_statement(std::move(audit_info), statement_, std::vector>(), std::vector()) +prepared_statement::prepared_statement(audit::audit_info_ptr&& audit_info, ::shared_ptr&& statement_, std::vector warnings) + : prepared_statement(std::move(audit_info), statement_, std::vector>(), std::vector(), std::move(warnings)) { } } diff --git a/db/config.cc b/db/config.cc index f73733d02c31..b00b203fc83b 100644 --- a/db/config.cc +++ b/db/config.cc @@ -1362,6 +1362,7 @@ db::config::config(std::shared_ptr exts) , disk_space_monitor_normal_polling_interval_in_seconds(this, "disk_space_monitor_normal_polling_interval_in_seconds", value_status::Used, 10, "Disk-space polling interval while below polling threshold") , disk_space_monitor_high_polling_interval_in_seconds(this, "disk_space_monitor_high_polling_interval_in_seconds", value_status::Used, 1, "Disk-space polling interval at or above polling threshold") , disk_space_monitor_polling_interval_threshold(this, "disk_space_monitor_polling_interval_threshold", value_status::Used, 0.9, "Disk-space polling threshold. Polling interval is increased when disk utilization is greater than or equal to this threshold") + , enable_create_table_with_compact_storage(this, "enable_create_table_with_compact_storage", liveness::LiveUpdate, value_status::Used, true, "Enable the deprecated feature of CREATE TABLE WITH COMPACT STORAGE. This feature will eventually be removed in a future version.") , default_log_level(this, "default_log_level", value_status::Used, seastar::log_level::info, "Default log level for log messages") , logger_log_level(this, "logger_log_level", value_status::Used, {}, "Map of logger name to log level. Valid log levels are 'error', 'warn', 'info', 'debug' and 'trace'") , log_to_stdout(this, "log_to_stdout", value_status::Used, true, "Send log output to stdout") diff --git a/db/config.hh b/db/config.hh index eb337126837a..a0e8a486e12f 100644 --- a/db/config.hh +++ b/db/config.hh @@ -532,6 +532,8 @@ public: named_value disk_space_monitor_high_polling_interval_in_seconds; named_value disk_space_monitor_polling_interval_threshold; + named_value enable_create_table_with_compact_storage; + static const sstring default_tls_priority; private: template diff --git a/test/cqlpy/test_guardrail_compact_storage.py b/test/cqlpy/test_guardrail_compact_storage.py new file mode 100644 index 000000000000..ce951bbf64b9 --- /dev/null +++ b/test/cqlpy/test_guardrail_compact_storage.py @@ -0,0 +1,30 @@ +import pytest + +from cassandra.protocol import InvalidRequest + +from .util import config_value_context, new_test_table + +# Tests for the enable_create_table_with_compact_storage guardrail. +# Because this feature does not exist in Cassandra , *all* tests in this file are +# Scylla-only. Let's mark them all scylla_only with an autouse fixture: +@pytest.fixture(scope="function", autouse=True) +def all_tests_are_scylla_only(scylla_only): + pass + +def test_create_table_with_compact_storage_default_config(cql, test_keyspace): + # enable_create_table_with_compact_storage is still enabled in db/config + with new_test_table(cql, test_keyspace, schema="p int PRIMARY KEY, v int", extra="WITH COMPACT STORAGE") as test_table: + pass + +@pytest.mark.parametrize("enable_compact_storage", [False, True]) +def test_create_table_with_compact_storage_config(cql, test_keyspace, enable_compact_storage): + with config_value_context(cql, 'enable_create_table_with_compact_storage', str(enable_compact_storage).lower()): + try: + with new_test_table(cql, test_keyspace, schema="p int PRIMARY KEY, v int", extra="WITH COMPACT STORAGE") as test_table: + if not enable_compact_storage: + pytest.fail + except InvalidRequest: + # expected to throw InvalidRequest only when + # enable_create_table_with_compact_storage=false + if enable_compact_storage: + pytest.fail From f3ab00e61c501492b9b95f67cfdafdf18fa7bed6 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Wed, 13 Dec 2023 13:17:40 +0200 Subject: [PATCH 381/397] test: enable_create_table_with_compact_storage for tests that need it Now enable_create_table_with_compact_storage can be set to `false` by default in db/config. Signed-off-by: Benny Halevy --- db/config.cc | 2 +- test/boost/cql_query_test.cc | 10 +++++++--- test/cql/suite.yaml | 3 +++ .../validation/operations/compact_storage_test.py | 4 ++++ .../validation/operations/compact_table_test.py | 4 ++++ test/cqlpy/conftest.py | 13 ++++++++++++- test/cqlpy/test_empty.py | 4 ++-- test/cqlpy/test_guardrail_compact_storage.py | 7 +++++-- 8 files changed, 38 insertions(+), 9 deletions(-) diff --git a/db/config.cc b/db/config.cc index b00b203fc83b..42dce007f3a7 100644 --- a/db/config.cc +++ b/db/config.cc @@ -1362,7 +1362,7 @@ db::config::config(std::shared_ptr exts) , disk_space_monitor_normal_polling_interval_in_seconds(this, "disk_space_monitor_normal_polling_interval_in_seconds", value_status::Used, 10, "Disk-space polling interval while below polling threshold") , disk_space_monitor_high_polling_interval_in_seconds(this, "disk_space_monitor_high_polling_interval_in_seconds", value_status::Used, 1, "Disk-space polling interval at or above polling threshold") , disk_space_monitor_polling_interval_threshold(this, "disk_space_monitor_polling_interval_threshold", value_status::Used, 0.9, "Disk-space polling threshold. Polling interval is increased when disk utilization is greater than or equal to this threshold") - , enable_create_table_with_compact_storage(this, "enable_create_table_with_compact_storage", liveness::LiveUpdate, value_status::Used, true, "Enable the deprecated feature of CREATE TABLE WITH COMPACT STORAGE. This feature will eventually be removed in a future version.") + , enable_create_table_with_compact_storage(this, "enable_create_table_with_compact_storage", liveness::LiveUpdate, value_status::Used, false, "Enable the deprecated feature of CREATE TABLE WITH COMPACT STORAGE. This feature will eventually be removed in a future version.") , default_log_level(this, "default_log_level", value_status::Used, seastar::log_level::info, "Default log level for log messages") , logger_log_level(this, "logger_log_level", value_status::Used, {}, "Map of logger name to log level. Valid log levels are 'error', 'warn', 'info', 'debug' and 'trace'") , log_to_stdout(this, "log_to_stdout", value_status::Used, true, "Send log output to stdout") diff --git a/test/boost/cql_query_test.cc b/test/boost/cql_query_test.cc index 036fe376d0d8..5f177c59c3c3 100644 --- a/test/boost/cql_query_test.cc +++ b/test/boost/cql_query_test.cc @@ -1274,6 +1274,7 @@ SEASTAR_TEST_CASE(test_range_deletion_scenarios) { SEASTAR_TEST_CASE(test_range_deletion_scenarios_with_compact_storage) { return do_with_cql_env_thread([] (cql_test_env& e) { + e.execute_cql("update system.config SET value='true' where name='enable_create_table_with_compact_storage';").get(); e.execute_cql("create table cf (p int, c int, v text, primary key (p, c)) with compact storage;").get(); for (auto i = 0; i < 10; ++i) { e.execute_cql(format("insert into cf (p, c, v) values (1, {:d}, 'abc');", i)).get(); @@ -2626,6 +2627,7 @@ SEASTAR_TEST_CASE(test_in_restriction) { SEASTAR_TEST_CASE(test_compact_storage) { return do_with_cql_env_thread([] (cql_test_env& e) { + e.execute_cql("update system.config SET value='true' where name='enable_create_table_with_compact_storage';").get(); e.execute_cql("create table tcs (p1 int, c1 int, r1 int, PRIMARY KEY (p1, c1)) with compact storage;").get(); BOOST_REQUIRE(e.local_db().has_schema("ks", "tcs")); e.execute_cql("insert into tcs (p1, c1, r1) values (1, 2, 3);").get(); @@ -2743,8 +2745,9 @@ SEASTAR_TEST_CASE(test_collections_of_collections) { SEASTAR_TEST_CASE(test_result_order) { - return do_with_cql_env([] (cql_test_env& e) { - return e.execute_cql("create table tro (p1 int, c1 text, r1 int, PRIMARY KEY (p1, c1)) with compact storage;").discard_result().then([&e] { + return do_with_cql_env_thread([] (cql_test_env& e) { + e.execute_cql("update system.config SET value='true' where name='enable_create_table_with_compact_storage';").get(); + e.execute_cql("create table tro (p1 int, c1 text, r1 int, PRIMARY KEY (p1, c1)) with compact storage;").discard_result().then([&e] { return e.execute_cql("insert into tro (p1, c1, r1) values (1, 'z', 1);").discard_result(); }).then([&e] { return e.execute_cql("insert into tro (p1, c1, r1) values (1, 'bbbb', 2);").discard_result(); @@ -2767,7 +2770,7 @@ SEASTAR_TEST_CASE(test_result_order) { { int32_type->decompose(1), utf8_type->decompose(sstring("cccc")), int32_type->decompose(6) }, { int32_type->decompose(1), utf8_type->decompose(sstring("z")), int32_type->decompose(1) }, }); - }); + }).get(); }); } @@ -3730,6 +3733,7 @@ SEASTAR_TEST_CASE(test_aggregate_and_simple_selection_together) { SEASTAR_TEST_CASE(test_alter_type_on_compact_storage_with_no_regular_columns_does_not_crash) { return do_with_cql_env_thread([] (cql_test_env& e) { + e.execute_cql("update system.config SET value='true' where name='enable_create_table_with_compact_storage';").get(); cquery_nofail(e, "CREATE TYPE my_udf (first text);"); cquery_nofail(e, "create table z (pk int, ck frozen, primary key(pk, ck)) with compact storage;"); cquery_nofail(e, "alter type my_udf add test_int int;"); diff --git a/test/cql/suite.yaml b/test/cql/suite.yaml index de1f994a90f0..34187f51061e 100644 --- a/test/cql/suite.yaml +++ b/test/cql/suite.yaml @@ -1 +1,4 @@ type: Approval +extra_scylla_cmdline_options: + # required for cdc_compact_storage_test.cql + - '--enable-create-table-with-compact-storage=true' diff --git a/test/cqlpy/cassandra_tests/validation/operations/compact_storage_test.py b/test/cqlpy/cassandra_tests/validation/operations/compact_storage_test.py index 40a1ccb39e35..ac7fe42731ac 100644 --- a/test/cqlpy/cassandra_tests/validation/operations/compact_storage_test.py +++ b/test/cqlpy/cassandra_tests/validation/operations/compact_storage_test.py @@ -12,6 +12,10 @@ compactOption = " WITH COMPACT STORAGE" +@pytest.fixture(scope="function", autouse=True) +def all_tests_in_this_file_use_compact_storage(compact_storage): + pass + # ALTER ... DROP COMPACT STORAGE was recently dropped (unless a special # flag is used) by Cassandra, and it was never implemented in Scylla, so # let's skip its test. diff --git a/test/cqlpy/cassandra_tests/validation/operations/compact_table_test.py b/test/cqlpy/cassandra_tests/validation/operations/compact_table_test.py index 15cef48b30e4..b003be0802c2 100644 --- a/test/cqlpy/cassandra_tests/validation/operations/compact_table_test.py +++ b/test/cqlpy/cassandra_tests/validation/operations/compact_table_test.py @@ -21,6 +21,10 @@ from ...porting import * +@pytest.fixture(scope="function", autouse=True) +def all_tests_in_this_file_use_compact_storage(compact_storage): + pass + # ALTER ... DROP COMPACT STORAGE was recently dropped (unless a special # flag is used) by Cassandra, and it was never implemented in Scylla, so # let's skip its test. diff --git a/test/cqlpy/conftest.py b/test/cqlpy/conftest.py index f862d157cec4..50da45a7836e 100644 --- a/test/cqlpy/conftest.py +++ b/test/cqlpy/conftest.py @@ -21,7 +21,7 @@ import time import random -from .util import unique_name, new_test_keyspace, keyspace_has_tablets, cql_session, local_process_id, is_scylla +from .util import unique_name, new_test_keyspace, keyspace_has_tablets, cql_session, local_process_id, is_scylla, config_value_context print(f"Driver name {DRIVER_NAME}, version {DRIVER_VERSION}") @@ -262,3 +262,14 @@ def has_tablets(cql, this_dc): def skip_without_tablets(scylla_only, has_tablets): if not has_tablets: pytest.skip("Test needs tablets experimental feature on") + +@pytest.fixture(scope="function") +def compact_storage(cql): + try: + with config_value_context(cql, 'enable_create_table_with_compact_storage', 'true') as ctx: + yield ctx + except: + # enable_create_table_with_compact_storage is a scylla only feature + # so the above may fail on cassandra. + # This is fine since compact storage is enabled there by default. + pass diff --git a/test/cqlpy/test_empty.py b/test/cqlpy/test_empty.py index 3cdcf42fda81..732e7948b7c9 100644 --- a/test/cqlpy/test_empty.py +++ b/test/cqlpy/test_empty.py @@ -52,7 +52,7 @@ def test_insert_empty_string_key_with_flush(cql, table1, scylla_only): # in a WITH COMPACT STORAGE table, an empty clustering key is not allowed. # As usual, an empty partition key is not allowed either. @pytest.mark.xfail(reason="issue #12749, misleading error message") -def test_insert_empty_string_key_compact(cql, test_keyspace): +def test_insert_empty_string_key_compact(compact_storage, cql, test_keyspace): schema = 'p text, c text, v text, primary key (p, c)' with new_test_table(cql, test_keyspace, schema, 'WITH COMPACT STORAGE') as table: s = unique_key_string() @@ -64,7 +64,7 @@ def test_insert_empty_string_key_compact(cql, test_keyspace): # However, in a COMPACT STORAGE table with a *compound* clustering key (more # than one clustering key column), setting one of them to empty *is* allowed. @pytest.mark.xfail(reason="issue #12749") -def test_insert_empty_string_compound_clustering_key_compact(cql, test_keyspace): +def test_insert_empty_string_compound_clustering_key_compact(compact_storage, cql, test_keyspace): schema = 'p text, c1 text, c2 text, v text, primary key (p, c1, c2)' with new_test_table(cql, test_keyspace, schema, 'WITH COMPACT STORAGE') as table: s = unique_key_string() diff --git a/test/cqlpy/test_guardrail_compact_storage.py b/test/cqlpy/test_guardrail_compact_storage.py index ce951bbf64b9..cea14fb035e4 100644 --- a/test/cqlpy/test_guardrail_compact_storage.py +++ b/test/cqlpy/test_guardrail_compact_storage.py @@ -12,8 +12,11 @@ def all_tests_are_scylla_only(scylla_only): pass def test_create_table_with_compact_storage_default_config(cql, test_keyspace): - # enable_create_table_with_compact_storage is still enabled in db/config - with new_test_table(cql, test_keyspace, schema="p int PRIMARY KEY, v int", extra="WITH COMPACT STORAGE") as test_table: + # enable_create_table_with_compact_storage is now disabled in db/config + try: + with new_test_table(cql, test_keyspace, schema="p int PRIMARY KEY, v int", extra="WITH COMPACT STORAGE") as test_table: + pytest.fail + except InvalidRequest: pass @pytest.mark.parametrize("enable_compact_storage", [False, True]) From 5c77956205ce4568b04def60fc7e9749a891b2da Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Wed, 13 Dec 2023 17:16:31 +0200 Subject: [PATCH 382/397] docs: ddl: document the deprecation of compact tables Add a paragraph documenting the decision to deprecate the COMPACT STORAGE feature, and instruct the user how to enable the feature despite that. Note that we don't have an official migration strategy for users like `DROP COMPACT STORAGE`, which is not implemented at this time (See #3882). Fixes #16375 Signed-off-by: Benny Halevy --- docs/cql/ddl.rst | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/cql/ddl.rst b/docs/cql/ddl.rst index fd53ea405e2e..81ce69fc03c5 100644 --- a/docs/cql/ddl.rst +++ b/docs/cql/ddl.rst @@ -597,6 +597,11 @@ options of a table are described in the following sections. Compact tables `````````````` +Support for compact tables has been deprecated. +By default, new tables cannot be created with the ``COMPACT STORAGE`` option, but existing tables continue to work as before. +To allow creating new COMPACT STORAGE tables, set the ``enable_create_table_with_compact_storage`` option to ``true`` in ``scylla.yaml``, +yet beware that support for compact tables is planned to be permanently removed in a future version. + .. .. caution:: Since Apache Cassandra 3.0, compact tables have the exact same layout internally than non compact ones (for the same schema obviously), and declaring a table compact **only** creates artificial limitations on the table definition From e340d6a452e91668923698cb469833cf4aae09e4 Mon Sep 17 00:00:00 2001 From: Anna Stuchlik Date: Fri, 17 Jan 2025 17:18:29 +0100 Subject: [PATCH 383/397] doc: remove Open Source references in the docs Fixes https://github.com/scylladb/scylladb/issues/22325 Closes scylladb/scylladb#22377 --- docs/alternator/compatibility.md | 4 -- .../install-scylla/install-on-linux.rst | 2 +- .../disable-housekeeping.rst | 2 +- docs/index.rst | 40 ++++++++++++------- docs/kb/decode-stack-trace.rst | 16 -------- docs/kb/perftune-modes-sync.rst | 2 +- docs/operating-scylla/admin.rst | 3 +- docs/operating-scylla/index.rst | 2 +- .../cluster-management/remove-seed.rst | 2 +- .../cluster-management/replace-seed-node.rst | 4 +- .../integrations/integration-ansible.rst | 2 +- .../integrations/integration-kafka.rst | 2 +- docs/using-scylla/mig-tool-review.rst | 2 +- 13 files changed, 36 insertions(+), 47 deletions(-) diff --git a/docs/alternator/compatibility.md b/docs/alternator/compatibility.md index 0ea3f9520727..3c1d3f0d1c5d 100644 --- a/docs/alternator/compatibility.md +++ b/docs/alternator/compatibility.md @@ -216,10 +216,6 @@ events appear in the Streams API as normal deletions - without the distinctive marker on deletions which are really expirations. See . - - -> **Note** This feature is experimental in versions earlier than ScyllaDB Open Source 5.2 and ScyllaDB Enterprise 2022.2. - ## Scan ordering In DynamoDB, scanning the _entire_ table returns the partitions sorted by diff --git a/docs/getting-started/install-scylla/install-on-linux.rst b/docs/getting-started/install-scylla/install-on-linux.rst index fd969f94086f..474e0f150ade 100644 --- a/docs/getting-started/install-scylla/install-on-linux.rst +++ b/docs/getting-started/install-scylla/install-on-linux.rst @@ -61,7 +61,7 @@ Install ScyllaDB sudo apt-get update sudo apt-get install -y scylla - Running the command installs the latest official version of ScyllaDB Open Source. + Running the command installs the latest official version of ScyllaDB. To install a specific patch version, list all the available patch versions: .. code-block:: console diff --git a/docs/getting-started/installation-common/disable-housekeeping.rst b/docs/getting-started/installation-common/disable-housekeeping.rst index 6418c171eb2c..6291d8d805df 100644 --- a/docs/getting-started/installation-common/disable-housekeeping.rst +++ b/docs/getting-started/installation-common/disable-housekeeping.rst @@ -3,7 +3,7 @@ ScyllaDB Housekeeping and how to disable it ============================================ -It is always recommended to run the latest version of ScyllaDB Open Source or ScyllaDB Enterprise. +It is always recommended to run the latest version of ScyllaDB. The latest stable release version is always available from the `Download Center `_. When you install ScyllaDB, it installs by default two services: **scylla-housekeeping-restart** and **scylla-housekeeping-daily**. These services check for the latest ScyllaDB version and prompt the user if they are using a version that is older than what is publicly available. diff --git a/docs/index.rst b/docs/index.rst index 8ea4dc2d5a51..732fe4f5bae0 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,44 +1,54 @@ :orphan: ==================================== -ScyllaDB Open Source Documentation +ScyllaDB Documentation ==================================== .. meta:: - :title: ScyllaDB Open Source Documentation - :description: ScyllaDB Open Source Documentation - :keywords: ScyllaDB Open Source, ScyllaDB Open Source, ScyllaDB docs, ScyllaDB documentation, ScyllaDB Documentation + :title: ScyllaDB Documentation + :description: ScyllaDB Documentation + :keywords: ScyllaDB docs, ScyllaDB documentation, ScyllaDB Documentation About This User Guide ----------------------- -ScyllaDB is a distributed NoSQL wide-column database for data-intensive apps that require +This user guide covers topics related to ScyllaDB - a distributed NoSQL +wide-column database for data-intensive apps that require high performance and low latency. -This user guide covers topics related to ScyllaDB Open Source - an open-source project that allows you to evaluate -experimental features, review the `source code `_, and add your contributions -to the project. +You can deploy ScyllaDB: -For topics related to other ScyllaDB flavors, see the documentation for `ScyllaDB Enterprise `_ and -`ScyllaDB Cloud `_. +* On AWS, GCP, and Azure. +* On premises. +* With `ScyllaDB Cloud `_ - a fully managed DBaaS. + +This user guide covers topics related to ScyllaDB cloud or on-premises +self-managed deployments. +For details about ScyllaDB Cloud, see the +`ScyllaDB Cloud documentation `_. Documentation Highlights -------------------------- -* :doc:`Install ScyllaDB Open Source ` -* :doc:`Configure ScyllaDB Open Source ` +* :doc:`Install ScyllaDB ` +* :doc:`Configure ScyllaDB ` * :doc:`Cluster Management Procedures ` -* :doc:`Upgrade ScyllaDB Open Source ` +* :doc:`Upgrade ScyllaDB ` * :doc:`CQL Reference ` * :doc:`ScyllaDB Drivers ` * :doc:`Features ` +ScyllaDB Support +-------------------- + +ScyllaDB customers can open or check on tickets in the +`ScyllaDB Customer Portal `_. + ScyllaDB Community -------------------------- -Join the ScyllaDB Open Source community: +Join the ScyllaDB community: -* Contribute to the ScyllaDB Open Source `project `_. * Join the `ScyllaDB Community Forum `_. * Join our `Slack Channel `_. diff --git a/docs/kb/decode-stack-trace.rst b/docs/kb/decode-stack-trace.rst index 46c0bf03d217..4f20b0afa636 100644 --- a/docs/kb/decode-stack-trace.rst +++ b/docs/kb/decode-stack-trace.rst @@ -31,28 +31,12 @@ Install the Debug binaries according to your OS distribution .. group-tab:: RPM based distributions - For ScyllaDB Enterprise: - - .. code-block:: none - - yum install scylla-enterprise-debuginfo - - For ScyllaDB Open Source: - .. code-block:: none yum install scylla-debuginfo .. group-tab:: DEB based distributions - For ScyllaDB Enterprise: - - .. code-block:: none - - apt-get install scylla-enterprise-server-dbg - - For ScyllaDB Open Source: - .. code-block:: none apt-get install scylla-server-dbg diff --git a/docs/kb/perftune-modes-sync.rst b/docs/kb/perftune-modes-sync.rst index ff1f78d2e1b2..28e708f68c8a 100644 --- a/docs/kb/perftune-modes-sync.rst +++ b/docs/kb/perftune-modes-sync.rst @@ -2,7 +2,7 @@ Updating the Mode in perftune.yaml After a ScyllaDB Upgrade ============================================================== -In versions 5.1 (ScyllaDB Open Source) and 2022.2 (ScyllaDB Enterprise), we improved ScyllaDB's performance by `removing the rx_queues_count from the mode +We improved ScyllaDB's performance by `removing the rx_queues_count from the mode condition `_. As a result, ScyllaDB operates in the ``sq_split`` mode instead of the ``mq`` mode (see :doc:`Seastar Perftune ` for information about the modes). If you upgrade from an earlier version of ScyllaDB, your cluster's existing nodes may use the ``mq`` mode, diff --git a/docs/operating-scylla/admin.rst b/docs/operating-scylla/admin.rst index b1757cc1077a..ab1b473290d5 100644 --- a/docs/operating-scylla/admin.rst +++ b/docs/operating-scylla/admin.rst @@ -252,10 +252,9 @@ discard, or trim, any blocks no longer in use by the filesystem. Experimental Features ===================== -ScyllaDB Open Source uses experimental flags to expose non-production-ready features safely. These features are not stable enough to be used in production, and their API will likely change, breaking backward or forward compatibility. +ScyllaDB uses experimental flags to expose non-production-ready features safely. These features are not stable enough to be used in production, and their API will likely change, breaking backward or forward compatibility. In recent ScyllaDB versions, these features are controlled by the ``experimental_features`` list in scylla.yaml, allowing one to choose which experimental to enable. -For example, some of the experimental features in ScyllaDB Open Source 4.5 are: ``udf``, ``alternator-streams`` and ``raft``. Use ``scylla --help`` to get the list of experimental features. ScyllaDB Enterprise and ScyllaDB Cloud do not officially support experimental Features. diff --git a/docs/operating-scylla/index.rst b/docs/operating-scylla/index.rst index ad69e16ceccf..a8a736642b46 100644 --- a/docs/operating-scylla/index.rst +++ b/docs/operating-scylla/index.rst @@ -58,7 +58,7 @@ ScyllaDB for Administrators :id: "getting-started" :class: my-panel - * :doc:`ScyllaDB Features ` - Feature list for ScyllaDB Open Source and ScyllaDB Enterprise + * :doc:`ScyllaDB Features ` - Feature list for ScyllaDB diff --git a/docs/operating-scylla/procedures/cluster-management/remove-seed.rst b/docs/operating-scylla/procedures/cluster-management/remove-seed.rst index cf26519b9af3..41ac7a26b6f6 100644 --- a/docs/operating-scylla/procedures/cluster-management/remove-seed.rst +++ b/docs/operating-scylla/procedures/cluster-management/remove-seed.rst @@ -5,7 +5,7 @@ Remove a Seed Node from Seed List This procedure describes how to remove a seed node from the seed list. .. note:: - The seed concept in gossip has been removed. Starting with ScyllaDB Open Source 4.3 and ScyllaDB Enterprise 2021.1, a seed node + The seed concept in gossip has been removed. A seed node is only used by a new node during startup to learn about the cluster topology. As a result, you only need to configure one seed node in a node's ``scylla.yaml`` file. diff --git a/docs/operating-scylla/procedures/cluster-management/replace-seed-node.rst b/docs/operating-scylla/procedures/cluster-management/replace-seed-node.rst index 1d8e9f345629..78f71e005442 100644 --- a/docs/operating-scylla/procedures/cluster-management/replace-seed-node.rst +++ b/docs/operating-scylla/procedures/cluster-management/replace-seed-node.rst @@ -4,8 +4,8 @@ Replacing a Dead Seed Node =========================== .. note:: - The seed concept in gossip has been removed. Starting with ScyllaDB Open Source 4.3 and ScyllaDB Enterprise 2021.1, - a seed node is only used by a new node during startup to learn about the cluster topology. As a result, there's no need + The seed concept in gossip has been removed. + A seed node is only used by a new node during startup to learn about the cluster topology. As a result, there's no need to replace the node configured with the ``seeds`` parameter in the ``scylla.yaml`` file. In ScyllaDB, it is not possible to bootstrap a seed node. The following steps describe how to replace a dead seed node. diff --git a/docs/using-scylla/integrations/integration-ansible.rst b/docs/using-scylla/integrations/integration-ansible.rst index 5b6e539ed7a6..0cabc9e96271 100644 --- a/docs/using-scylla/integrations/integration-ansible.rst +++ b/docs/using-scylla/integrations/integration-ansible.rst @@ -5,7 +5,7 @@ Install ScyllaDB with Ansible You can use the Ansible roles and the playbook examples that follow to deploy and maintain ScyllaDB clusters. There are roles for creating a ScyllaDB cluster, a ScyllaDB Manager, ScyllaDB Monitoring Stack, and a Loader. These roles can be used independently or together, using each role's outputs. -You can use these roles with ScyllaDB (Open Source and Enterprise), ScyllaDB Manager, and ScyllaDB Monitoring Stack. +You can use these roles with ScyllaDB, ScyllaDB Manager, and ScyllaDB Monitoring Stack. To get started, visit the `GitHub project `_. diff --git a/docs/using-scylla/integrations/integration-kafka.rst b/docs/using-scylla/integrations/integration-kafka.rst index e17e18eca1cf..0258ba2412bc 100644 --- a/docs/using-scylla/integrations/integration-kafka.rst +++ b/docs/using-scylla/integrations/integration-kafka.rst @@ -9,7 +9,7 @@ Integrate ScyllaDB with Kafka scylla-cdc-source-connector Apache Kafka is capable of delivering reliable, scalable, high-throughput data streams across a myriad of data sources and sinks. -A great number of open source users and enterprise customers use ScyllaDB and Kafka together. +A great number of customers use ScyllaDB and Kafka together. You can use ScyllaDB and Apache Kafka in integration solutions, such as creating a scalable backend for an IoT service. If you have tested your application with ScyllaDB and Kafka and want to publish the results, contact us using the `community forum `_. diff --git a/docs/using-scylla/mig-tool-review.rst b/docs/using-scylla/mig-tool-review.rst index 9d4ab13a2fdd..5d037f94822e 100644 --- a/docs/using-scylla/mig-tool-review.rst +++ b/docs/using-scylla/mig-tool-review.rst @@ -3,7 +3,7 @@ ScyllaDB Migration Tools: An Overview ======================================= The following migration tools are available for migrating to ScyllaDB from compatible databases, -such as Apache Cassandra, or from other ScyllaDB clusters (ScyllaDB Open Source or Enterprise): +such as Apache Cassandra, or from other ScyllaDB clusters: * From SSTable to SSTable - Using nodetool refresh, :ref:`Load and Stream ` option. From d2869ecb2bb088767a67b7f4bd8fb8edf69e8678 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Fri, 3 Jan 2025 17:27:46 +0200 Subject: [PATCH 384/397] partition_range_compat: drop dependency on boost ranges Unused anyway. Closes scylladb/scylladb#22359 --- partition_range_compat.hh | 1 - 1 file changed, 1 deletion(-) diff --git a/partition_range_compat.hh b/partition_range_compat.hh index 1f6ec0cb9cf8..a2351b1c3e2d 100644 --- a/partition_range_compat.hh +++ b/partition_range_compat.hh @@ -12,7 +12,6 @@ #include #include "interval.hh" #include "dht/ring_position.hh" -#include namespace compat { From 12cbdfa095315bcb5201b66a5bed29da56ec6fb1 Mon Sep 17 00:00:00 2001 From: Nadav Har'El Date: Thu, 16 Jan 2025 18:49:43 +0200 Subject: [PATCH 385/397] test/cqlpy: add regression test for tombstone_gc in "desc table" The small cqlpy test in this patch is a regression test for issue #14390, which claimed that the Scylla-only "tombstone_gc" option is missing from the output of "describe table". This test shows that this report is *not* true, at least not when the "server-side describe" is used. "test/cqlpy/run --release ..." shows that this test passes on master and also for Scylla versions all the way back to Scylla 5.2 (Scylla 5.1 did not support server-side describe, so the test fails for that reason). This suggests that the report in issue #14390 was for old-style client-side (cqlsh) describe, which we no longer support, so this issue can be closed. Fixes #14390. Signed-off-by: Nadav Har'El Closes scylladb/scylladb#22354 --- test/cqlpy/test_describe.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/test/cqlpy/test_describe.py b/test/cqlpy/test_describe.py index f79d50d3e5ea..17ab26feb868 100644 --- a/test/cqlpy/test_describe.py +++ b/test/cqlpy/test_describe.py @@ -2786,3 +2786,15 @@ def test_describe_udf_with_udt(cql, test_keyspace, scylla_only): expected_stmt = f"CREATE FUNCTION {test_keyspace}.{fn}{fn_content}" assert udf_stmt == expected_stmt + +# Test that the Scylla-only "tombstone_gc" option appears in the output of +# "desc table". Reproduces issue #14390. +# The test is marked scylla_only because Cassandra doesn't have this +# "tombstone_gc" option. +def test_desc_table_tombstone_gc(cql, test_keyspace, scylla_only): + with_clause = "tombstone_gc = {'mode': 'timeout', 'propagation_delay_in_seconds': '73'}" + with new_test_table(cql, test_keyspace, "p int PRIMARY KEY", "WITH " + with_clause) as table: + desc = cql.execute(f"DESCRIBE TABLE {table} WITH INTERNALS").one() + # ignore spaces in comparison, as different versions of Scylla + # add spaces in different places + assert with_clause.replace(' ','') in desc.create_statement.replace(' ','') From 8caea23d2a1e908e212f57979f7410e1d4088fb9 Mon Sep 17 00:00:00 2001 From: Nadav Har'El Date: Thu, 16 Jan 2025 18:24:44 +0200 Subject: [PATCH 386/397] test/cqlpy/run: fix regression in "--release" option The way that the "test/cqlpy/run --release" feature runs older Scylla releases is that it takes *today*'s command line parameters and "fixes" it to conform to what old releases took. This approach was easy to implement (and the resulting "--release" feature is super useful), but the downside is that we need to update this fixup code whenever we add new options to the Scylla command line used by test/cqlpy/run.py. Commit d04f376 made test/cqlpy/run.py use a new option "--experimental-features=views-with-tablets", so now we need to remove it when running older versions of Scylla. So this is what we do in this patch. Fixes #22349 Signed-off-by: Nadav Har'El Closes scylladb/scylladb#22350 --- test/cqlpy/run.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test/cqlpy/run.py b/test/cqlpy/run.py index 029e7bf44ddc..702112b02dd9 100755 --- a/test/cqlpy/run.py +++ b/test/cqlpy/run.py @@ -384,6 +384,8 @@ def run_precompiled_scylla_cmd(exe, pid, dir): cmd.remove('--kernel-page-cache=1') cmd.remove('--flush-schema-tables-after-modification=false') cmd.remove('--strict-allow-filtering=true') + if major <= [6,2] or (enterprise and major <= [2025,1]): + cmd.remove('--experimental-features=views-with-tablets') if major <= [4,5]: cmd.remove('--max-networking-io-control-blocks=1000') if major == [5,4] or major == [2024,1]: From 7495237a33116c37e96f1a9175d26337118d4d17 Mon Sep 17 00:00:00 2001 From: Yaniv Michael Kaul Date: Wed, 1 Jan 2025 16:56:33 +0200 Subject: [PATCH 387/397] Remove noexcept_traits.hh header file The content of the header file noexcept_traits.hh is unused throughout ScyllaDB's code base. As part of a greater effort to cleanup Scylla's code and reduce content in the root directory, this header file is simply removed. This is code cleanup - no need to backport. Fixes: https://github.com/scylladb/scylladb/issues/22117 Signed-off-by: Yaniv Kaul Closes scylladb/scylladb#22139 --- noexcept_traits.hh | 69 ---------------------------------------------- 1 file changed, 69 deletions(-) delete mode 100644 noexcept_traits.hh diff --git a/noexcept_traits.hh b/noexcept_traits.hh deleted file mode 100644 index c58f71645d38..000000000000 --- a/noexcept_traits.hh +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright 2015-present ScyllaDB - */ - -/* - * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 - */ - -#include -#include -#include - -#include "seastarx.hh" - -#pragma once - -// -// Utility for adapting types which are not nothrow move constructible into such -// by wrapping them if necessary. -// -// Example usage: -// -// T val{}; -// using traits = noexcept_movable; -// auto f = make_ready_future(traits::wrap(std::move(val))); -// T val2 = traits::unwrap(f.get()); -// - -template -struct noexcept_movable; - -template -requires std::is_nothrow_move_constructible_v -struct noexcept_movable { - using type = T; - - static type wrap(T&& v) { - return std::move(v); - } - - static future wrap(future&& v) { - return std::move(v); - } - - static T unwrap(type&& v) { - return std::move(v); - } - - static future unwrap(future&& v) { - return std::move(v); - } -}; - -template -requires (!std::is_nothrow_move_constructible_v) -struct noexcept_movable { - using type = std::unique_ptr; - - static type wrap(T&& v) { - return std::make_unique(std::move(v)); - } - - static T unwrap(type&& v) { - return std::move(*v); - } -}; - -template -using noexcept_movable_t = typename noexcept_movable::type; From 8208688178d737a7bacc72f499808419577f7b90 Mon Sep 17 00:00:00 2001 From: Asias He Date: Mon, 23 Dec 2024 10:38:59 +0800 Subject: [PATCH 388/397] Introduce file stream for tablet File based stream is a new feature that optimizes tablet movement significantly. It streams the entire SSTable files without deserializing SSTable files into mutation fragments and re-serializing them back into SSTables on receiving nodes. As a result, less data is streamed over the network, and less CPU is consumed, especially for data models that contain small cells. The following patches are imported from the scylla enterprise: *) Merge 'Introduce file stream for tablet' from Asias He This patch uses Seastar RPC stream interface to stream sstable files on network for tablet migration. It streams sstables instead of mutation fragments. The file based stream has multiple advantages over the mutation streaming. - No serialization or deserialization for mutation fragments - No need to read and process each mutation fragments - On wire data is more compact and smaller In the test below, a significant speed up is observed. Two nodes, 1 shard per node, 1 initial_tablets: - Start node 1 - Insert 10M rows of data with c-s - Bootstrap node 2 Node 1 will migration data to node2 with the file stream. Test results: 1) File stream: bytes on wire = 1132006250 bytes, bw = 836MB/s [shard 0:stre] stream_blob - stream_sstables[eadaa8e0-a4f2-4cc6-bf10-39ad1ce106b0] Finished sending sstable_nr=2 files_nr=18 files={} range=(-1,9223372036854775807] bytes_sent=1132006250 stream_bw=836MB/s [shard 0:stre] storage_service - Streaming for tablet migration of a4f68900-568a-11ee-b7b9-c2b13945eed2:1 took 1.08004s seconds 2) Mutation stream: bytes on wire = 3030004736 bytes, bw = 125410.87 KiB/s = 128MB/s [shard 0:stre] stream_session - [Stream #406dc8b0-56b5-11ee-bc2d-000bf4871058] Streaming plan for Tablet migration-ks1-index-0 succeeded, peers={127.0.0.1}, tx=0 KiB, 0.00 KiB/s, rx=2958989 KiB, 125410.87 KiB/s [shard 0:stre] storage_service - Streaming for tablet migration of a4f68900-568a-11ee-b7b9-c2b13945eed2:1 took 23.5992s seconds Test Summary: File stream v.s. Mutation stream improvements - Stream bandwidth = 836 / 128 (MB/s) = 6.53X - Stream time = 23.60 / 1.08 (Seconds) = 21.85X - Stream bytes on wire = 3030004736 / 1132006250 (Bytes)= 2.67X Closes scylladb/scylla-enterprise#3438 * github.com:scylladb/scylla-enterprise: tests: Add file_stream_test streaming: Implement file stream for tablet *) streaming: Use new take_storage_snapshot interface The new take_storage_snapshot returns a file object instead of a file name. This allows the file stream sender to read from the file even if the file is deleted by compaction. Closes scylladb/scylla-enterprise#3728 *) streaming: Protect unsupported file types for file stream Currently, we assume the file streamed over the stream_blob rpc verb is a sstable file. This patch rejects the unsupported file types on the receiver side. This allows us to stream more file types later using the current file stream infrastructure without worrying about old nodes processing the new file types in the wrong way. - The file_ops::noop is renamed to file_ops::stream_sstables to be explicit about the file types - A missing test_file_stream_error_injection is added to the idl Fixes: #3846 Tests: test_unsupported_file_ops Closes scylladb/scylla-enterprise#3847 *) idl: Add service::session_id id to idl It will be used in the next patch. Refs #3907 *) streaming: Protect file stream with topology_guard Similar to "storage_service, tablets: Use session to guard tablet streaming", this patch protects file stream with topology_guard. Fixes #3907 *) streaming: Take service topology_guard under the try block Taking the service::topology_guard could throw. Currently, it throws outside the try block, so the rpc sink will not be closed, causing the following assertion: ``` scylla: seastar/include/seastar/rpc/rpc_impl.hh:815: virtual seastar::rpc::sink_impl::~sink_impl() [Serializer = netw::serializer, Out = ]: Assertion `this->_con->get()->sink_closed()' failed. ``` To fix, move more code including the topology_guard taking code to the try block. Fixes https://github.com/scylladb/scylla-enterprise/issues/4106 Closes scylladb/scylla-enterprise#4110 *) Merge 'Preserve original SSTable state with file based tablet migration' from Raphael "Raph" Carvalho We're not preserving the SSTable state across file based migration, so staging SSTables for example are being placed into main directory, and consequently, we're mixing staging and non-staging data, losing the ability to continue from where the old replica left off. It's expected that the view update backlog is transferred from old into new replica, as migration doesn't wait for leaving replica to complete view update work (which can take long). Elasticity is preferred. So this fix guarantees that the state of the SSTable will be preserved by propagating it in form of subdirectory (each subdirectory is statically mapped with a particular state). The staging sstables aren't being registered into view update generator yet, as that's supposed to be fixed in OSS (more details can be found at https://github.com/scylladb/scylladb/issues/19149). Fixes #4265. Closes scylladb/scylla-enterprise#4267 * github.com:scylladb/scylla-enterprise: tablet: Preserve original SSTable state with file based tablet migration sstables: Add get method for sstable state *) sstable: (Re-)add shareabled_components getter *) Merge 'File streaming sstables: Use sstable source/sink to transfer snapshots' from Calle Wilund Fixes #4246 Alternative approach/better separation of concern, transport vs. sstable layer. Builds on #4472, but fancier. Ensures we transfer and pre-process scylla metadata for streamed file blobs first, then properly apply receiving nodes local config by using a source and sink layer exported from sstables, which handles things like ordering, metadata filtering (on source) as well as handling metadata and proper IO paths when writing data on receiver node (sink). This implementation maintains the statelessness of the current design, and the delegated sink side will re-read and re-write the metadata for each component processed. This is a little wasteful, but the meta is small, and it is less error prone than trying to do caching cross-shards etc. The transport is isolated from the knowledge. This is an alternative/complement to #4436 and #4472, fixing the underlying issue. Note that while the layers/API:s here allows easy fixing of other fundamental problems in the feature (such as destination location etc), these are not included in the PR, to keep it as close to the current behaviour as possible. Closes scylladb/scylla-enterprise#4646 * github.com:scylladb/scylla-enterprise: raft_tests: Copy/add a topology test with encryption file streaming: Use sstable source/sink to transfer snapshots sstables: Add source and sink objects + producers for transfering a snapshot sstable::types: Add remove accessor for extension info in metadata *) The change for error injection in merge commit 966ea5955dd8760: File streaming now has "stream_mutation_fragments" error injection points so test_table_dropped_during_streaming works with file streaming. *) doc: document file-based streaming This commit adds a description of the file-based streaming feature to the documentation. It will be displayed in the docs using the scylladb_include_flag directive after https://github.com/scylladb/scylladb/pull/20182 is merged, backported to branch-6.0, and, in turn, branch-2024.2. Refs https://github.com/scylladb/scylla-enterprise/issues/4585 Refs https://github.com/scylladb/scylla-enterprise/issues/4254 Closes scylladb/scylla-enterprise#4587 *) doc: move File-based streaming to the Tablets source file-based-streaming This commit moves the description of file-based streaming from a common include file to the regular doc source file where tablets are described. Closes scylladb/scylla-enterprise#4652 *) streaming: sstable_stream_sink_impl: abort: prevent null pointer dereference Closes scylladb/scylladb#22034 --- configure.py | 3 + db/config.cc | 1 + db/config.hh | 1 + docs/architecture/tablets.rst | 19 + gms/feature_service.hh | 1 + idl/CMakeLists.txt | 1 + idl/sstables.idl.hh | 19 + idl/streaming.idl.hh | 59 ++ idl/uuid.idl.hh | 1 + message/messaging_service.cc | 34 + message/messaging_service.hh | 11 + service/storage_service.cc | 70 +- sstables/sstables.cc | 217 ++++++ sstables/sstables.hh | 51 ++ sstables/types.hh | 3 + streaming/CMakeLists.txt | 1 + streaming/stream_blob.cc | 705 ++++++++++++++++++ streaming/stream_blob.hh | 192 +++++ streaming/stream_session.cc | 12 +- test/boost/CMakeLists.txt | 2 + test/boost/file_stream_test.cc | 260 +++++++ test/topology_custom/test_tablets2.py | 1 + .../topology_custom/test_tablets_migration.py | 101 ++- .../test_topology_ops_encrypted.py | 85 +++ 24 files changed, 1847 insertions(+), 3 deletions(-) create mode 100644 idl/sstables.idl.hh create mode 100644 streaming/stream_blob.cc create mode 100644 streaming/stream_blob.hh create mode 100644 test/boost/file_stream_test.cc create mode 100644 test/topology_custom/test_topology_ops_encrypted.py diff --git a/configure.py b/configure.py index 0458a85cd1a0..dbcb40a80f5a 100755 --- a/configure.py +++ b/configure.py @@ -522,6 +522,7 @@ def find_ninja(): 'test/boost/mutation_reader_another_test', 'test/boost/mutation_test', 'test/boost/mvcc_test', + 'test/boost/file_stream_test', 'test/boost/nonwrapping_interval_test', 'test/boost/observable_test', 'test/boost/partitioner_test', @@ -1081,6 +1082,7 @@ def find_ninja(): 'streaming/stream_request.cc', 'streaming/stream_summary.cc', 'streaming/stream_transfer_task.cc', + 'streaming/stream_blob.cc', 'streaming/stream_receive_task.cc', 'streaming/stream_plan.cc', 'streaming/progress_info.cc', @@ -1312,6 +1314,7 @@ def find_ninja(): 'idl/group0.idl.hh', 'idl/hinted_handoff.idl.hh', 'idl/storage_proxy.idl.hh', + 'idl/sstables.idl.hh', 'idl/group0_state_machine.idl.hh', 'idl/mapreduce_request.idl.hh', 'idl/replica_exception.idl.hh', diff --git a/db/config.cc b/db/config.cc index f73733d02c31..a9817eb17e55 100644 --- a/db/config.cc +++ b/db/config.cc @@ -760,6 +760,7 @@ db::config::config(std::shared_ptr exts) "Throttles streaming I/O to the specified total throughput (in MiBs/s) across the entire system. Streaming I/O includes the one performed by repair and both RBNO and legacy topology operations such as adding or removing a node. Setting the value to 0 disables stream throttling.") , stream_plan_ranges_fraction(this, "stream_plan_ranges_fraction", liveness::LiveUpdate, value_status::Used, 0.1, "Specify the fraction of ranges to stream in a single stream plan. Value is between 0 and 1.") + , enable_file_stream(this, "enable_file_stream", liveness::LiveUpdate, value_status::Used, true, "Set true to use file based stream for tablet instead of mutation based stream") , trickle_fsync(this, "trickle_fsync", value_status::Unused, false, "When doing sequential writing, enabling this option tells fsync to force the operating system to flush the dirty buffers at a set interval trickle_fsync_interval_in_kb. Enable this parameter to avoid sudden dirty buffer flushing from impacting read latencies. Recommended to use on SSDs, but not on HDDs.") , trickle_fsync_interval_in_kb(this, "trickle_fsync_interval_in_kb", value_status::Unused, 10240, diff --git a/db/config.hh b/db/config.hh index eb337126837a..7d24d0a3f2dc 100644 --- a/db/config.hh +++ b/db/config.hh @@ -243,6 +243,7 @@ public: named_value inter_dc_stream_throughput_outbound_megabits_per_sec; named_value stream_io_throughput_mb_per_sec; named_value stream_plan_ranges_fraction; + named_value enable_file_stream; named_value trickle_fsync; named_value trickle_fsync_interval_in_kb; named_value auto_bootstrap; diff --git a/docs/architecture/tablets.rst b/docs/architecture/tablets.rst index 0176adb7e82b..605b27bf16ec 100644 --- a/docs/architecture/tablets.rst +++ b/docs/architecture/tablets.rst @@ -72,6 +72,25 @@ to a new node. .. image:: images/tablets-load-balancing.png +File-based Streaming +======================== + +:label-tip:`ScyllaDB Enterprise` + +File-based streaming is a ScyllaDB Enterprise-only feature that optimizes +tablet migration. + +In ScyllaDB Open Source, migrating tablets is performed by streaming mutation +fragments, which involves deserializing SSTable files into mutation fragments +and re-serializing them back into SSTables on the other node. +In ScyllaDB Enterprise, migrating tablets is performed by streaming entire +SStables, which does not require (de)serializing or processing mutation fragments. +As a result, less data is streamed over the network, and less CPU is consumed, +especially for data models that contain small cells. + +File-based streaming is used for tablet migration in all +:ref:`keyspaces created with tablets enabled `. + .. _tablets-enable-tablets: Enabling Tablets diff --git a/gms/feature_service.hh b/gms/feature_service.hh index c52337b9f788..7ac881ed3d5b 100644 --- a/gms/feature_service.hh +++ b/gms/feature_service.hh @@ -160,6 +160,7 @@ public: gms::feature workload_prioritization { *this, "WORKLOAD_PRIORITIZATION"sv }; gms::feature compression_dicts { *this, "COMPRESSION_DICTS"sv }; + gms::feature file_stream { *this, "FILE_STREAM"sv }; public: const std::unordered_map>& registered_features() const; diff --git a/idl/CMakeLists.txt b/idl/CMakeLists.txt index 3caffe6ca012..7f5c9082ac4f 100644 --- a/idl/CMakeLists.txt +++ b/idl/CMakeLists.txt @@ -51,6 +51,7 @@ set(idl_headers raft_storage.idl.hh group0.idl.hh hinted_handoff.idl.hh + sstables.idl.hh storage_proxy.idl.hh storage_service.idl.hh group0_state_machine.idl.hh diff --git a/idl/sstables.idl.hh b/idl/sstables.idl.hh new file mode 100644 index 000000000000..a2e87c7f3fd3 --- /dev/null +++ b/idl/sstables.idl.hh @@ -0,0 +1,19 @@ +/* + * Copyright 2024-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + + +namespace sstables { + +enum class sstable_state : uint8_t { + normal, + staging, + quarantine, + upload, +}; + +} diff --git a/idl/streaming.idl.hh b/idl/streaming.idl.hh index 61539410d0c4..955320e4fba2 100644 --- a/idl/streaming.idl.hh +++ b/idl/streaming.idl.hh @@ -9,8 +9,10 @@ #include "idl/range.idl.hh" #include "idl/token.idl.hh" #include "idl/uuid.idl.hh" +#include "idl/sstables.idl.hh" #include "streaming/stream_fwd.hh" +#include "streaming/stream_blob.hh" namespace service { @@ -72,4 +74,61 @@ verb [[with_client_info]] prepare_message (streaming::prepare_message msg, strea verb [[with_client_info]] prepare_done_message (streaming::plan_id plan_id, unsigned dst_cpu_id); verb [[with_client_info]] stream_mutation_done (streaming::plan_id plan_id, dht::token_range_vector ranges, table_id cf_id, unsigned dst_cpu_id); verb [[with_client_info]] complete_message (streaming::plan_id plan_id, unsigned dst_cpu_id, bool failed [[version 2.1.0]]); + +class file_stream_id final { + utils::UUID uuid(); +}; + +enum class stream_blob_cmd : uint8_t { + ok, + error, + data, + end_of_stream, +}; + +enum class file_ops : uint16_t { + stream_sstables, + load_sstables, +}; + +class stream_blob_data { + temporary_buffer buf; +}; + +class stream_blob_cmd_data { + streaming::stream_blob_cmd cmd; + std::optional data; +}; + +class stream_blob_meta { + streaming::file_stream_id ops_id; + table_id table; + sstring filename; + seastar::shard_id dst_shard_id; + streaming::file_ops fops; + service::frozen_topology_guard topo_guard; + std::optional sstable_state; +}; + +class node_and_shard { + locator::host_id node; + seastar::shard_id shard; +}; + +class stream_files_request { + streaming::file_stream_id ops_id; + sstring keyspace_name; + sstring table_name; + table_id table; + dht::token_range range; + std::vector targets; + service::frozen_topology_guard topo_guard; +}; + +class stream_files_response { + size_t stream_bytes; +}; + +verb [[with_client_info, cancellable]] tablet_stream_files (streaming::stream_files_request req) -> streaming::stream_files_response; + } diff --git a/idl/uuid.idl.hh b/idl/uuid.idl.hh index b69946fe04b8..70a2712535cf 100644 --- a/idl/uuid.idl.hh +++ b/idl/uuid.idl.hh @@ -11,6 +11,7 @@ #include "query_id.hh" #include "locator/host_id.hh" #include "tasks/types.hh" +#include "service/session.hh" namespace utils { class UUID final { diff --git a/message/messaging_service.cc b/message/messaging_service.cc index 8fc9cdd8654c..8a0385f76da7 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -41,6 +41,7 @@ #include "repair/repair.hh" #include "streaming/stream_reason.hh" #include "streaming/stream_mutation_fragments_cmd.hh" +#include "streaming/stream_blob.hh" #include "cache_temperature.hh" #include "raft/raft.hh" #include "service/raft/group0_fwd.hh" @@ -1269,6 +1270,39 @@ future<> messaging_service::unregister_stream_mutation_fragments() { return unregister_handler(messaging_verb::STREAM_MUTATION_FRAGMENTS); } +// Wrapper for STREAM_BLOB +rpc::sink messaging_service::make_sink_for_stream_blob(rpc::source& source) { + return source.make_sink(); +} + +future, rpc::source>> +messaging_service::make_sink_and_source_for_stream_blob(streaming::stream_blob_meta meta, locator::host_id id) { + if (is_shutting_down()) { + co_await coroutine::return_exception(rpc::closed_error()); + } + auto rpc_client = get_rpc_client(messaging_verb::STREAM_BLOB, addr_for_host_id(id), id); + auto sink = co_await rpc_client->make_stream_sink(); + std::exception_ptr ex; + try { + auto rpc_handler = rpc()->make_client (streaming::stream_blob_meta, rpc::sink)>(messaging_verb::STREAM_BLOB); + auto source = co_await rpc_handler(*rpc_client, meta, sink); + co_return std::make_tuple(std::move(sink), std::move(source)); + } catch (...) { + ex = std::current_exception(); + } + // Reach here only in case of error + co_await sink.close(); + co_return coroutine::return_exception_ptr(ex); +} + +void messaging_service::register_stream_blob(std::function> (const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source)>&& func) { + register_handler(this, messaging_verb::STREAM_BLOB, std::move(func)); +} + +future<> messaging_service::unregister_stream_blob() { + return unregister_handler(messaging_verb::STREAM_BLOB); +} + template future, rpc::source>> do_make_sink_source(messaging_verb verb, uint32_t repair_meta_id, shard_id dst_shard_id, shared_ptr rpc_client, std::unique_ptr& rpc) { diff --git a/message/messaging_service.hh b/message/messaging_service.hh index 083caee6eab3..ab4182d91239 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -37,6 +37,10 @@ namespace streaming { class prepare_message; enum class stream_mutation_fragments_cmd : uint8_t; + enum class stream_blob_cmd : uint8_t; + class stream_blob_data; + class stream_blob_meta; + class stream_blob_cmd_data; } namespace gms { @@ -411,6 +415,13 @@ public: rpc::sink make_sink_for_stream_mutation_fragments(rpc::source>& source); future, rpc::source>> make_sink_and_source_for_stream_mutation_fragments(table_schema_version schema_id, streaming::plan_id plan_id, table_id cf_id, uint64_t estimated_partitions, streaming::stream_reason reason, service::session_id session, locator::host_id id); + // Wrapper for STREAM_BLOB + // The receiver of STREAM_BLOB sends streaming::stream_blob_cmd_data as status code to the sender to notify any error on the receiver side. + void register_stream_blob(std::function> (const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source)>&& func); + future<> unregister_stream_blob(); + rpc::sink make_sink_for_stream_blob(rpc::source& source); + future, rpc::source>> make_sink_and_source_for_stream_blob(streaming::stream_blob_meta meta, locator::host_id id); + // Wrapper for REPAIR_GET_ROW_DIFF_WITH_RPC_STREAM future, rpc::source>> make_sink_and_source_for_repair_get_row_diff_with_rpc_stream(uint32_t repair_meta_id, shard_id dst_cpu_id, locator::host_id id); rpc::sink make_sink_for_repair_get_row_diff_with_rpc_stream(rpc::source& source); diff --git a/service/storage_service.cc b/service/storage_service.cc index cf826ff3f968..5811190b57cf 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -63,6 +63,7 @@ #include "locator/local_strategy.hh" #include "utils/user_provided_param.hh" #include "version.hh" +#include "streaming/stream_blob.hh" #include "dht/range_streamer.hh" #include #include @@ -93,10 +94,12 @@ #include #include #include +#include "utils/pretty_printers.hh" #include "utils/stall_free.hh" #include "utils/error_injection.hh" #include "locator/util.hh" #include "idl/storage_service.dist.hh" +#include "idl/streaming.dist.hh" #include "service/storage_proxy.hh" #include "service/raft/join_node.hh" #include "idl/join_node.dist.hh" @@ -6072,6 +6075,53 @@ future<> storage_service::stream_tablet(locator::global_tablet_id tablet) { } }); + if (trinfo->transition != locator::tablet_transition_kind::intranode_migration && _feature_service.file_stream && _db.local().get_config().enable_file_stream()) { + co_await utils::get_local_injector().inject("migration_streaming_wait", [] (auto& handler) { + rtlogger.info("migration_streaming_wait: start"); + return handler.wait_for_message(db::timeout_clock::now() + std::chrono::minutes(2)); + }); + + auto dst_node = trinfo->pending_replica->host; + auto dst_shard_id = trinfo->pending_replica->shard; + auto transition = trinfo->transition; + + // Release token_metadata_ptr early so it will no block barriers for other migrations + // Don't access trinfo after this. + tm = {}; + + co_await utils::get_local_injector().inject("stream_sstable_files", [&] (auto& handler) -> future<> { + slogger.info("stream_sstable_files: waiting"); + while (!handler.poll_for_message()) { + co_await sleep_abortable(std::chrono::milliseconds(5), guard.get_abort_source()); + } + slogger.info("stream_sstable_files: released"); + }); + + for (auto src : streaming_info.read_from) { + // Use file stream for tablet to stream data + auto ops_id = streaming::file_stream_id::create_random_id(); + auto start_time = std::chrono::steady_clock::now(); + size_t stream_bytes = 0; + try { + auto& table = _db.local().find_column_family(tablet.table); + slogger.debug("stream_sstables[{}] Streaming for tablet {} of {} started table={}.{} range={} src={}", + ops_id, transition, tablet, table.schema()->ks_name(), table.schema()->cf_name(), range, src); + auto resp = co_await streaming::tablet_stream_files(ops_id, table, range, src.host, dst_node, dst_shard_id, _messaging.local(), _abort_source, topo_guard); + stream_bytes = resp.stream_bytes; + slogger.debug("stream_sstables[{}] Streaming for tablet migration of {} successful", ops_id, tablet); + auto duration = std::chrono::duration(std::chrono::steady_clock::now() - start_time); + auto bw = utils::pretty_printed_throughput(stream_bytes, duration);; + slogger.info("stream_sstables[{}] Streaming for tablet migration of {} finished table={}.{} range={} stream_bytes={} stream_time={} stream_bw={}", + ops_id, tablet, table.schema()->ks_name(), table.schema()->cf_name(), range, stream_bytes, duration, bw); + } catch (...) { + slogger.warn("stream_sstables[{}] Streaming for tablet migration of {} from {} failed: {}", ops_id, tablet, leaving_replica, std::current_exception()); + throw; + } + } + + } else { // Caution: following code is intentionally unindented to be in sync with OSS + + if (trinfo->transition == locator::tablet_transition_kind::intranode_migration) { if (!leaving_replica || leaving_replica->host != tm->get_my_id()) { throw std::runtime_error(fmt::format("Invalid leaving replica for intra-node migration, tablet: {}, leaving: {}", @@ -6123,6 +6173,8 @@ future<> storage_service::stream_tablet(locator::global_tablet_id tablet) { slogger.info("Streaming for tablet migration of {} finished table={}.{} range={}", tablet, table.schema()->ks_name(), table.schema()->cf_name(), range); } + } // Traditional streaming vs file-based streaming. + // If new pending tablet replica needs splitting, streaming waits for it to complete. // That's to provide a guarantee that once migration is over, the coordinator can finalize // splitting under the promise that compaction groups of tablets are all split, ready @@ -7031,6 +7083,21 @@ void storage_service::init_messaging_service() { return handler(ss); }); }; + ser::streaming_rpc_verbs::register_tablet_stream_files(&_messaging.local(), + [this] (const rpc::client_info& cinfo, streaming::stream_files_request req) -> future { + streaming::stream_files_response resp; + resp.stream_bytes = co_await container().map_reduce0([req] (storage_service& ss) -> future { + auto res = co_await streaming::tablet_stream_files_handler(ss._db.local(), ss._messaging.local(), req, [&ss] (locator::host_id host) -> future { + return ss.container().invoke_on(0, [host] (storage_service& ss) { + return ss.host2ip(host); + }); + }); + co_return res.stream_bytes; + }, + size_t(0), + std::plus()); + co_return resp; + }); ser::storage_service_rpc_verbs::register_raft_topology_cmd(&_messaging.local(), [handle_raft_rpc] (raft::server_id dst_id, raft::term_t term, uint64_t cmd_index, raft_topology_cmd cmd) { return handle_raft_rpc(dst_id, [cmd = std::move(cmd), term, cmd_index] (auto& ss) { return ss.raft_topology_cmd_handler(term, cmd_index, cmd); @@ -7165,7 +7232,8 @@ future<> storage_service::uninit_messaging_service() { return when_all_succeed( ser::node_ops_rpc_verbs::unregister(&_messaging.local()), ser::storage_service_rpc_verbs::unregister(&_messaging.local()), - ser::join_node_rpc_verbs::unregister(&_messaging.local()) + ser::join_node_rpc_verbs::unregister(&_messaging.local()), + ser::streaming_rpc_verbs::unregister_tablet_stream_files(&_messaging.local()) ).discard_result(); } diff --git a/sstables/sstables.cc b/sstables/sstables.cc index c5e9f434031a..083aa0a3331c 100644 --- a/sstables/sstables.cc +++ b/sstables/sstables.cc @@ -3417,6 +3417,223 @@ generation_type::from_string(const std::string& s) { } } +std::string sstable_stream_source::component_basename() const { + return _sst->component_basename(_type); +} + +sstable_stream_source::sstable_stream_source(shared_sstable sst, component_type type) + : _sst(std::move(sst)) + , _type(type) +{} + +std::vector> create_stream_sources(const sstables::sstable_files_snapshot& snapshot) { + std::vector> result; + result.reserve(snapshot.files.size()); + + class sstable_stream_source_impl : public sstable_stream_source { + file _file; + public: + sstable_stream_source_impl(shared_sstable table, component_type type, file f) + : sstable_stream_source(std::move(table), type) + , _file(std::move(f)) + {} + future> input(const file_input_stream_options& options) const override { + if (_type == component_type::Scylla) { + // Filter out any node-local info (i.e. extensions) + // and reserialize data. Load into a temp object. + // TODO/FIXME. Not all extension attributes might + // need removing. In fact, it might be wrong (in the future) + // to do so. ATM we know this is safe and correct, but really + // extensions should remove themselves if required. + scylla_metadata tmp; + uint64_t size = co_await _file.size(); + auto r = file_random_access_reader(_file, size, default_sstable_buffer_size); + co_await parse(*_sst->get_schema(), _sst->get_version(), r, tmp); + co_await r.close(); + + tmp.remove_extension_attributes(); + + std::vector> bufs; + // TODO: move to seastar. Based on memory_data_sink, but allowing us + // to actually move away the buffers later. I don't want to modify + // util classes in an enterprise patch. + class buffer_data_sink_impl : public data_sink_impl { + std::vector>& _bufs; + public: + buffer_data_sink_impl(std::vector>& bufs) + : _bufs(bufs) + {} + future<> put(net::packet data) override { + throw std::logic_error("unsupported operation"); + } + future<> put(temporary_buffer buf) override { + _bufs.emplace_back(std::move(buf)); + return make_ready_future<>(); + } + future<> flush() override { + return make_ready_future<>(); + } + future<> close() override { + return make_ready_future<>(); + } + size_t buffer_size() const noexcept override { + return 128*1024; + } + }; + + co_await seastar::async([&] { + file_writer fw(data_sink(std::make_unique(bufs))); + write(_sst->get_version(), fw, tmp); + fw.close(); + }); + // TODO: move to seastar. Based on buffer_input... in utils, but + // handles potential 1+ buffers + class buffer_data_source_impl : public data_source_impl { + private: + std::vector> _bufs; + size_t _index = 0; + public: + buffer_data_source_impl(std::vector>&& bufs) + : _bufs(std::move(bufs)) + {} + buffer_data_source_impl(buffer_data_source_impl&&) noexcept = default; + buffer_data_source_impl& operator=(buffer_data_source_impl&&) noexcept = default; + + future> get() override { + if (_index < _bufs.size()) { + return make_ready_future>(std::move(_bufs.at(_index++))); + } + return make_ready_future>(); + } + future> skip(uint64_t n) override { + while (n > 0 && _index < _bufs.size()) { + auto& buf = _bufs.at(_index); + auto min = std::min(n, buf.size()); + buf.trim_front(min); + if (buf.empty()) { + ++_index; + } + n -= min; + } + return get(); + } + }; + co_return input_stream(data_source(std::make_unique(std::move(bufs)))); + } + co_return make_file_input_stream(_file, options); + } + }; + + auto& files = snapshot.files; + + auto add = [&](component_type type, file f) { + result.emplace_back(std::make_unique(snapshot.sst, type, std::move(f))); + }; + + try { + add(component_type::TOC, files.at(component_type::TOC)); + add(component_type::Scylla, files.at(component_type::Scylla)); + } catch (std::out_of_range&) { + std::throw_with_nested(std::invalid_argument("Missing required sstable component")); + } + for (auto&& [type, f] : files) { + if (type != component_type::TOC && type != component_type::Scylla) { + add(type, std::move(f)); + } + } + + return result; +} + +class sstable_stream_sink_impl : public sstable_stream_sink { + shared_sstable _sst; + component_type _type; + bool _last_component; +public: + sstable_stream_sink_impl(shared_sstable sst, component_type type, bool last_component) + : _sst(std::move(sst)) + , _type(type) + , _last_component(last_component) + {} +private: + future<> load_metadata() const { + auto metafile = _sst->filename(sstables::component_type::Scylla); + if (!co_await file_exists(metafile)) { + // for compatibility with streaming a non-scylla table (no scylla component) + co_return; + } + if (!_sst->get_shared_components().scylla_metadata) { + sstables::scylla_metadata tmp; + co_await _sst->read_simple(tmp); + _sst->get_shared_components().scylla_metadata = std::move(tmp); + } + } + future<> save_metadata() const { + if (!_sst->get_shared_components().scylla_metadata) { + co_return; + } + file_output_stream_options options; + options.buffer_size = default_sstable_buffer_size; + co_await seastar::async([&] { + auto w = _sst->make_component_file_writer(component_type::Scylla, std::move(options), open_flags::wo | open_flags::create).get(); + write(_sst->get_version(), w, *_sst->get_shared_components().scylla_metadata); + w.close(); + }); + } +public: + future> output(const file_open_options& foptions, const file_output_stream_options& stream_options) override { + assert(_type != component_type::TOC); + // TOC and scylla components are guaranteed not to depend on metadata. Ignore these (chicken, egg) + bool load_save_meta = _type != component_type::TemporaryTOC && _type != component_type::Scylla; + + // otherwise, first load scylla metadata from disk as written so far. + if (load_save_meta) { + co_await load_metadata(); + } + // now we can open the component file. any extensions applied should write info into metadata + auto f = co_await _sst->open_file(_type, open_flags::wo | open_flags::create, foptions); + + // Save back to disk. + if (load_save_meta) { + co_await save_metadata(); + } + + co_return co_await make_file_output_stream(std::move(f), stream_options); + } + future close_and_seal() override { + if (_last_component) { + // If we are the last component in a sequence, we can seal the table. + co_await _sst->_storage->seal(*_sst); + co_return std::move(_sst); + } + _sst = {}; + co_return nullptr; + } + future<> abort() override { + if (!_sst) { + co_return; + } + auto filename = fs::path(_sst->_storage->prefix()) / std::string_view(_sst->component_basename(_type)); + // TODO: if we are the last component (or really always), should we remove all component files? + // For now, this remains the responsibility of calling code (see handle_tablet_migration etc) + co_await remove_file(filename.native()); + } +}; + +std::unique_ptr create_stream_sink(schema_ptr schema, sstables_manager& sstm, const data_dictionary::storage_options& s_opts, sstable_state state, std::string_view component_filename, bool last_component) { + auto desc = parse_path(component_filename, schema->ks_name(), schema->cf_name()); + auto sst = sstm.make_sstable(schema, s_opts, desc.generation, state, desc.version, desc.format); + + auto type = desc.component; + // Don't write actual TOC. Write temp, if successful, storage::seal will rename this to actual + // TOC (see above close_and_seal). + if (type == component_type::TOC) { + type = component_type::TemporaryTOC; + } + + return std::make_unique(std::move(sst), type, last_component); +} + } // namespace sstables namespace seastar { diff --git a/sstables/sstables.hh b/sstables/sstables.hh index 820435e5af04..3a1eea4bdb95 100644 --- a/sstables/sstables.hh +++ b/sstables/sstables.hh @@ -180,6 +180,8 @@ public: future<> commit(); }; +class sstable_stream_sink_impl; + class sstable : public enable_lw_shared_from_this { friend ::sstable_assertions; public: @@ -256,6 +258,10 @@ public: // It's up to the storage driver how to implement this. future<> change_state(sstable_state to, delayed_commit_changes* delay = nullptr); + sstable_state state() const { + return _state; + } + // Filesystem-specific call to grab an sstable from upload dir and // put it into the desired destination assigning the given generation future<> pick_up_from_upload(sstable_state to, generation_type new_generation); @@ -517,6 +523,7 @@ private: return filename(dir, _schema->ks_name(), _schema->cf_name(), _version, _generation, _format, f); } + friend class sstable_stream_sink_impl; friend class sstable_directory; friend class filesystem_storage; friend class s3_storage; @@ -1131,6 +1138,50 @@ struct sstable_files_snapshot { std::unordered_map files; }; +// A sstable_stream_source gives back +// component input streams suitable for streaming to other nodes, +// in appropriate order. Data will be decrypted and sanitized as required. +class sstable_stream_source { +protected: + shared_sstable _sst; + component_type _type; +public: + sstable_stream_source(shared_sstable, component_type); + virtual ~sstable_stream_source() = default; + + // Input stream for data appropriate for stream transfer for this component + virtual future> input(const file_input_stream_options&) const = 0; + + // source sstable + const sstable& source() const { + return *_sst; + } + // component + component_type type() const { + return _type; + } + std::string component_basename() const; +}; + +// Translates the result of gathering readable snapshot files into ordered items for streaming. +std::vector> create_stream_sources(const sstables::sstable_files_snapshot&); + +class sstable_stream_sink { +public: + virtual ~sstable_stream_sink() = default; + // Stream to the component file + virtual future> output(const file_open_options&, const file_output_stream_options&) = 0; + // closes this component. If this is the last component in a set (see "last_component" in creating method below) + // the table on disk will be sealed. + // Returns sealed sstable if last, or nullptr otherwise. + virtual future close_and_seal() = 0; + virtual future<> abort() = 0; +}; + +// Creates a sink object which can receive a component file sourced from above source object data. + +std::unique_ptr create_stream_sink(schema_ptr, sstables_manager&, const data_dictionary::storage_options&, sstable_state, std::string_view component_filename, bool last_component); + } // namespace sstables template <> struct fmt::formatter : fmt::formatter { diff --git a/sstables/types.hh b/sstables/types.hh index 51dd55868449..4e0930b8cfd3 100644 --- a/sstables/types.hh +++ b/sstables/types.hh @@ -626,6 +626,9 @@ struct scylla_metadata { const extension_attributes* get_extension_attributes() const { return data.get(); } + void remove_extension_attributes() { + data.data.erase(scylla_metadata_type::ExtensionAttributes); + } extension_attributes& get_or_create_extension_attributes() { auto* ext = data.get(); if (ext == nullptr) { diff --git a/streaming/CMakeLists.txt b/streaming/CMakeLists.txt index 5a6d3da49243..7456f6ed4078 100644 --- a/streaming/CMakeLists.txt +++ b/streaming/CMakeLists.txt @@ -14,6 +14,7 @@ target_sources(streaming stream_session_state.cc stream_summary.cc stream_task.cc + stream_blob.cc stream_transfer_task.cc) target_include_directories(streaming PUBLIC diff --git a/streaming/stream_blob.cc b/streaming/stream_blob.cc new file mode 100644 index 000000000000..4f3c3d9a6d0f --- /dev/null +++ b/streaming/stream_blob.cc @@ -0,0 +1,705 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "message/messaging_service.hh" +#include "streaming/stream_blob.hh" +#include "streaming/stream_plan.hh" +#include "gms/inet_address.hh" +#include "utils/pretty_printers.hh" +#include "utils/error_injection.hh" +#include "locator/host_id.hh" +#include "replica/database.hh" +#include "sstables/sstables.hh" +#include "sstables/sstables_manager.hh" +#include "sstables/sstable_version.hh" +#include "sstables/generation_type.hh" +#include "sstables/types.hh" +#include "idl/streaming.dist.hh" +#include "service/topology_guard.hh" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace streaming { + +static logging::logger blogger("stream_blob"); + +constexpr size_t file_stream_buffer_size = 128 * 1024; +constexpr size_t file_stream_write_behind = 10; +constexpr size_t file_stream_read_ahead = 4; + +static sstables::sstable_state sstable_state(const streaming::stream_blob_meta& meta) { + return meta.sstable_state.value_or(sstables::sstable_state::normal); +} + +static future<> load_sstable_for_tablet(const file_stream_id& ops_id, replica::database& db, table_id table, sstables::sstable_state state, sstring filename, seastar::shard_id shard) { + blogger.debug("stream_sstables[{}] Loading sstable {} on shard {}", ops_id, filename, shard); + auto s = db.find_column_family(table).schema(); + auto data_path = std::filesystem::path(filename); + auto desc = sstables::parse_path(data_path, s->ks_name(), s->cf_name()); + co_await db.container().invoke_on(shard, [id = s->id(), desc, state] (replica::database& db) -> future<> { + replica::table& t = db.find_column_family(id); + auto erm = t.get_effective_replication_map(); + auto& sstm = t.get_sstables_manager(); + auto sst = sstm.make_sstable(t.schema(), t.get_storage_options(), desc.generation, state, desc.version, desc.format); + co_await sst->load(erm->get_sharder(*t.schema())); + co_await t.add_sstable_and_update_cache(sst); + }); + blogger.info("stream_sstables[{}] Loaded sstable {} on shard {} successfully", ops_id, filename, shard); +} + +static utils::pretty_printed_throughput get_bw(size_t total_size, std::chrono::steady_clock::time_point start_time) { + auto duration = std::chrono::steady_clock::now() - start_time; + return utils::pretty_printed_throughput(total_size, duration); +} + +// For tablet stream checks +class tablet_stream_status { +public: + bool finished = false; + void check_valid_stream(); +}; + +void tablet_stream_status::check_valid_stream() { + if (finished) { + throw std::runtime_error("The stream has finished already"); + } +} + +static thread_local std::unordered_map> tablet_streams; + +future<> mark_tablet_stream_start(file_stream_id ops_id) { + return seastar::smp::invoke_on_all([ops_id] { + auto status = make_lw_shared(); + tablet_streams.emplace(ops_id, status); + }); +} + +future<> mark_tablet_stream_done(file_stream_id ops_id) { + return seastar::smp::invoke_on_all([ops_id] { + auto it = tablet_streams.find(ops_id); + if (it == tablet_streams.end()) { + return; + } + auto status = it->second; + if (status) { + status->finished = true; + } + tablet_streams.erase(ops_id); + }); +} + +lw_shared_ptr get_tablet_stream(file_stream_id ops_id) { + auto status = tablet_streams[ops_id]; + if (!status) { + auto msg = format("stream_sstables[{}] Could not find ops_id={}", ops_id, ops_id); + blogger.warn("{}", msg); + throw std::runtime_error(msg); + } + return status; +} + +static void may_inject_error(const streaming::stream_blob_meta& meta, bool may_inject, const sstring& error) { + if (may_inject) { + if (rand() % 500 == 0) { + auto msg = format("fstream[{}] Injected file stream error={} file={}", + meta.ops_id, error, meta.filename); + blogger.warn("{}", msg); + throw std::runtime_error(msg); + } + } +} + +future<> stream_blob_handler(replica::database& db, + netw::messaging_service& ms, + gms::inet_address from, + streaming::stream_blob_meta meta, + rpc::sink sink, + rpc::source source, + stream_blob_create_output_fn create_output, + bool inject_errors) { + bool fstream_closed = false; + bool sink_closed = false; + bool status_sent = false; + size_t total_size = 0; + auto start_time = std::chrono::steady_clock::now(); + std::optional> fstream; + std::exception_ptr error; + stream_blob_finish_fn finish; + + // Will log a message when streaming is done. Used to synchronize tests. + lw_shared_ptr log_done; + if (utils::get_local_injector().is_enabled("stream_mutation_fragments")) { + log_done = make_lw_shared(seastar::make_shared(seastar::defer([] { + blogger.info("stream_mutation_fragments: done (tablets)"); + }))); + } + + try { + auto status = get_tablet_stream(meta.ops_id); + auto guard = service::topology_guard(meta.topo_guard); + + // Reject any file_ops that is not support by this node + if (meta.fops != streaming::file_ops::stream_sstables && + meta.fops != streaming::file_ops::load_sstables) { + auto msg = format("fstream[{}] Unsupported file_ops={} peer={} file={}", + meta.ops_id, int(meta.fops), from, meta.filename); + blogger.warn("{}", msg); + throw std::runtime_error(msg); + } + + blogger.debug("fstream[{}] Follower started peer={} file={}", + meta.ops_id, from, meta.filename); + + auto [f, out] = co_await create_output(db, meta); + finish = std::move(f); + fstream = std::move(out); + + bool got_end_of_stream = false; + for (;;) { + try { + auto opt = co_await source(); + if (!opt) { + break; + } + + co_await utils::get_local_injector().inject("stream_mutation_fragments", [&guard] (auto& handler) -> future<> { + blogger.info("stream_mutation_fragments: waiting (tablets)"); + while (!handler.poll_for_message()) { + guard.check(); + co_await sleep(std::chrono::milliseconds(5)); + } + blogger.info("stream_mutation_fragments: released (tablets)"); + }); + + stream_blob_cmd_data& cmd_data = std::get<0>(*opt); + auto cmd = cmd_data.cmd; + if (cmd == streaming::stream_blob_cmd::error) { + blogger.warn("fstream[{}] Follower got stream_blob_cmd::error from peer={} file={}", + meta.ops_id, from, meta.filename); + throw std::runtime_error(format("Got stream_blob_cmd::error from peer={} file={}", from, meta.filename)); + } else if (cmd == streaming::stream_blob_cmd::end_of_stream) { + blogger.debug("fstream[{}] Follower got stream_blob_cmd::end_of_stream from peer={} file={}", + meta.ops_id, from, meta.filename); + got_end_of_stream = true; + } else if (cmd == streaming::stream_blob_cmd::data) { + std::optional data = std::move(cmd_data.data); + if (data) { + total_size += data->size(); + blogger.trace("fstream[{}] Follower received data from peer={} data={}", meta.ops_id, from, data->size()); + status->check_valid_stream(); + if (!data->empty()) { + co_await fstream->write((char*)data->data(), data->size()); + } + } + } + } catch (seastar::rpc::stream_closed) { + // After we get streaming::stream_blob_cmd::end_of_stream which + // is the last message from peer, it does not matter if the + // source() is closed or not. + if (got_end_of_stream) { + break; + } else { + throw; + } + } catch (...) { + throw; + } + may_inject_error(meta, inject_errors, "rx_data"); + } + + // If we reach here, streaming::stream_blob_cmd::end_of_stream should be received. Otherwise there + // must be an error, e.g., the sender closed the stream without sending streaming::stream_blob_cmd::error. + if (!got_end_of_stream) { + throw std::runtime_error(format("fstream[{}] Follower failed to get end_of_stream", meta.ops_id)); + } + + status->check_valid_stream(); + co_await fstream->flush(); + co_await fstream->close(); + fstream_closed = true; + + may_inject_error(meta, inject_errors, "flush_and_close"); + + co_await finish(store_result::ok); + + // Send status code and close the sink + co_await sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::ok)); + status_sent = true; + co_await sink.close(); + sink_closed = true; + } catch (...) { + error = std::current_exception(); + } + if (error) { + blogger.warn("fstream[{}] Follower failed peer={} file={} received_size={} bw={} error={}", + meta.ops_id, from, meta.filename, total_size, get_bw(total_size, start_time), error); + if (!fstream_closed) { + try { + if (fstream) { + // Make sure fstream is always closed + co_await fstream->close(); + } + } catch (...) { + blogger.warn("fstream[{}] Follower failed to close the file stream: {}", + meta.ops_id, std::current_exception()); + // We could do nothing but continue to cleanup more + } + } + if (!status_sent) { + try { + may_inject_error(meta, inject_errors, "no_error_code_back"); + co_await sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::error)); + } catch (...) { + // Try our best to send the status code. + // If we could not send it, we could do nothing but close the sink. + blogger.warn("fstream[{}] Follower failed to send error code: {}", + meta.ops_id, std::current_exception()); + } + } + try { + if (!sink_closed) { + // Make sure sink is always closed + co_await sink.close(); + } + } catch (...) { + blogger.warn("fstream[{}] Follower failed to close the stream sink: {}", + meta.ops_id, std::current_exception()); + } + try { + // Drain everything in source + for (;;) { + auto opt = co_await source(); + if (!opt) { + break; + } + } + } catch (...) { + blogger.warn("fstream[{}] Follower failed to drain rpc stream source: {}", + meta.ops_id, std::current_exception()); + } + + try { + // Remove the file in case of error + if (finish) { + co_await finish(store_result::failure); + blogger.info("fstream[{}] Follower removed partial file={}", meta.ops_id, meta.filename); + } + } catch (...) { + blogger.warn("fstream[{}] Follower failed to remove partial file={}: {}", + meta.ops_id, meta.filename, std::current_exception()); + } + + // Do not call rethrow_exception(error) because the caller could do nothing but log + // the error. We have already logged the error here. + } else { + // Get some statistics + blogger.debug("fstream[{}] Follower finished peer={} file={} received_size={} bw={}", + meta.ops_id, from, meta.filename, total_size, get_bw(total_size, start_time)); + } + co_return; +} + + +future<> stream_blob_handler(replica::database& db, netw::messaging_service& ms, + gms::inet_address from, + streaming::stream_blob_meta meta, + rpc::sink sink, + rpc::source source) { + + co_await stream_blob_handler(db, ms, std::move(from), meta, std::move(sink), std::move(source), [](replica::database& db, const streaming::stream_blob_meta& meta) -> future { + auto foptions = file_open_options(); + foptions.sloppy_size = true; + foptions.extent_allocation_size_hint = 32 << 20; + + auto stream_options = file_output_stream_options(); + stream_options.buffer_size = file_stream_buffer_size; + stream_options.write_behind = file_stream_write_behind; + + auto& table = db.find_column_family(meta.table); + auto& sstm = table.get_sstables_manager(); + auto sstable_sink = sstables::create_stream_sink(table.schema(), sstm, table.get_storage_options(), sstable_state(meta), meta.filename, meta.fops == file_ops::load_sstables); + auto out = co_await sstable_sink->output(foptions, stream_options); + co_return output_result{ + [sstable_sink = std::move(sstable_sink), &meta, &db](store_result res) -> future<> { + if (res != store_result::ok) { + co_await sstable_sink->abort(); + co_return; + } + auto sst = co_await sstable_sink->close_and_seal(); + if (sst) { + auto filename = sst->toc_filename(); + sst = {}; + co_await load_sstable_for_tablet(meta.ops_id, db, meta.table, sstable_state(meta), std::move(filename), meta.dst_shard_id); + } + }, + std::move(out) + }; + }); +} + +// Get a new sstable name using the new generation +// For example: +// oldname: me-3ga1_0iiv_2e5uo2flv7lgdl2j0d-big-Index.db +// newgen: 3ga1_0iiv_3vj5c2flv7lgdl2j0d +// newname: me-3ga1_0iiv_3vj5c2flv7lgdl2j0d-big-Index.db +static std::string get_sstable_name_with_generation(const file_stream_id& ops_id, const std::string& oldname, const std::string& newgen) { + std::string newname = oldname; + // The generation name starts after the first '-'. + auto it = newname.find("-"); + if (it != std::string::npos) { + newname.replace(++it, newgen.size(), newgen); + return newname; + } else { + auto msg = fmt::format("fstream[{}] Failed to get sstable name for {} with generation {}", ops_id, oldname, newgen); + blogger.warn("{}", msg); + throw std::runtime_error(msg); + } +} +} + +template<> struct fmt::formatter : fmt::ostream_formatter {}; + +namespace streaming { + +// Send files in the files list to the nodes in targets list over network +// Returns number of bytes sent over network +future +tablet_stream_files(netw::messaging_service& ms, std::list sources, std::vector targets, table_id table, file_stream_id ops_id, host2ip_t host2ip, service::frozen_topology_guard topo_guard, bool inject_errors) { + size_t ops_total_size = 0; + if (targets.empty()) { + co_return ops_total_size; + } + if (sources.empty()) { + co_return ops_total_size; + } + + blogger.debug("fstream[{}] Master started sending n={}, sources={}, targets={}", + ops_id, sources.size(), sources, targets); + + struct sink_and_source { + gms::inet_address node; + rpc::sink sink; + rpc::source source; + bool sink_closed = false; + bool status_sent = false; + }; + + auto ops_start_time = std::chrono::steady_clock::now(); + streaming::stream_blob_meta meta; + meta.ops_id = ops_id; + meta.table = table; + meta.topo_guard = topo_guard; + std::exception_ptr error; + + auto stream_options = file_input_stream_options(); + stream_options.buffer_size = file_stream_buffer_size; + stream_options.read_ahead = file_stream_read_ahead; + + for (auto& info : sources) { + auto& filename = info.filename; + std::optional> fstream; + bool fstream_closed = false; + try { + meta.fops = info.fops; + meta.filename = info.filename; + meta.sstable_state = info.sstable_state; + fstream = co_await info.source(stream_options); + } catch (...) { + blogger.warn("fstream[{}] Master failed sources={} targets={} error={}", + ops_id, sources, targets, std::current_exception()); + throw; + } + + std::vector ss; + size_t total_size = 0; + auto start_time = std::chrono::steady_clock::now(); + bool got_error_from_peer = false; + try { + for (auto& x : targets) { + const auto& node = x.node; + meta.dst_shard_id = x.shard; + auto ip = co_await host2ip(node); + blogger.debug("fstream[{}] Master creating sink and source for node={}/{}, file={}, targets={}", ops_id, node, ip, filename, targets); + auto [sink, source] = co_await ms.make_sink_and_source_for_stream_blob(meta, node); + ss.push_back(sink_and_source{ip, std::move(sink), std::move(source)}); + } + + // This fiber sends data to peer node + auto send_data_to_peer = [&] () mutable -> future<> { + std::exception_ptr error; + try { + while (!got_error_from_peer) { + may_inject_error(meta, inject_errors, "read_data"); + auto buf = co_await fstream->read_up_to(file_stream_buffer_size); + if (buf.size() == 0) { + break; + } + streaming::stream_blob_data data(std::move(buf)); + auto data_size = data.size(); + stream_blob_cmd_data cmd_data(streaming::stream_blob_cmd::data, std::move(data)); + co_await coroutine::parallel_for_each(ss, [&] (sink_and_source& s) mutable -> future<> { + total_size += data_size; + ops_total_size += data_size; + blogger.trace("fstream[{}] Master sending file={} to node={} chunk_size={}", + ops_id, filename, s.node, data_size); + may_inject_error(meta, inject_errors, "tx_data"); + co_await s.sink(cmd_data); + }); + } + } catch (...) { + error = std::current_exception(); + } + if (error) { + // We have to close the stream otherwise if the stream is + // ok, the get_status_code_from_peer fiber below might + // wait for the source() forever. + for (auto& s : ss) { + try { + co_await s.sink.close(); + s.sink_closed = true; + } catch (...) { + } + } + std::rethrow_exception(error); + } + + if (fstream) { + co_await fstream->close(); + fstream_closed = true; + } + + for (auto& s : ss) { + blogger.debug("fstream[{}] Master done sending file={} to node={}", ops_id, filename, s.node); + if (!got_error_from_peer) { + co_await s.sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::end_of_stream)); + s.status_sent = true; + } + co_await s.sink.close(); + s.sink_closed = true; + } + }; + + // This fiber gets status code from peer node + auto get_status_code_from_peer = [&] () mutable -> future<> { + co_await coroutine::parallel_for_each(ss, [&] (sink_and_source& s) mutable -> future<> { + bool got_cmd_ok = false; + while (!got_error_from_peer) { + try { + auto opt = co_await s.source(); + if (opt) { + stream_blob_cmd_data& cmd_data = std::get<0>(*opt); + if (cmd_data.cmd == streaming::stream_blob_cmd::error) { + got_error_from_peer = true; + blogger.warn("fstream[{}] Master got stream_blob_cmd::error file={} peer={}", + ops_id, filename, s.node); + throw std::runtime_error(format("Got stream_blob_cmd::error from peer {}", s.node)); + } if (cmd_data.cmd == streaming::stream_blob_cmd::ok) { + got_cmd_ok = true; + } + blogger.debug("fstream[{}] Master got stream_blob_cmd={} file={} peer={}", + ops_id, int(cmd_data.cmd), filename, s.node); + } else { + break; + } + } catch (seastar::rpc::stream_closed) { + // After we get streaming::stream_blob_cmd::ok + // which is the last message from peer, it does not + // matter if the source() is closed or not. + if (got_cmd_ok) { + break; + } else { + throw; + } + } catch (...) { + throw; + } + } + }); + }; + + co_await coroutine::all(send_data_to_peer, get_status_code_from_peer); + } catch (...) { + error = std::current_exception(); + } + if (error) { + blogger.warn("fstream[{}] Master failed sending file={} to targets={} send_size={} bw={} error={}", + ops_id, filename, targets, total_size, get_bw(total_size, start_time), error); + // Error handling for fstream and sink + if (!fstream_closed) { + try { + if (fstream) { + co_await fstream->close(); + } + } catch (...) { + // We could do nothing but continue to cleanup more + blogger.warn("fstream[{}] Master failed to close file stream: {}", + ops_id, std::current_exception()); + } + } + for (auto& s : ss) { + try { + if (!s.status_sent && !s.sink_closed) { + co_await s.sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::error)); + s.status_sent = true; + } + } catch (...) { + // We could do nothing but continue to close + blogger.warn("fstream[{}] Master failed to send error code: {}", + ops_id, std::current_exception()); + } + try { + if (!s.sink_closed) { + co_await s.sink.close(); + s.sink_closed = true; + } + } catch (...) { + // We could do nothing but continue + blogger.warn("fstream[{}] Master failed to close rpc stream sink: {}", + ops_id, std::current_exception()); + } + + try { + // Drain everything in source + for (;;) { + auto opt = co_await s.source(); + if (!opt) { + break; + } + } + } catch (...) { + blogger.warn("fstream[{}] Master failed to drain rpc stream source: {}", + ops_id, std::current_exception()); + } + } + // Stop handling remaining files + break; + } else { + blogger.debug("fstream[{}] Master done sending file={} to targets={} send_size={} bw={}", + ops_id, filename, targets, total_size, get_bw(total_size, start_time)); + } + } + if (error) { + blogger.warn("fstream[{}] Master failed sending files_nr={} files={} targets={} send_size={} bw={} error={}", + ops_id, sources.size(), sources, targets, ops_total_size, get_bw(ops_total_size, ops_start_time), error); + std::rethrow_exception(error); + } else { + blogger.debug("fstream[{}] Master finished sending files_nr={} files={} targets={} send_size={} bw={}", + ops_id, sources.size(), sources, targets, ops_total_size, get_bw(ops_total_size, ops_start_time)); + } + co_return ops_total_size; +} + + +future tablet_stream_files_handler(replica::database& db, netw::messaging_service& ms, streaming::stream_files_request req, host2ip_t host2ip) { + stream_files_response resp; + auto& table = db.find_column_family(req.table); + auto sstables = co_await table.take_storage_snapshot(req.range); + co_await utils::get_local_injector().inject("order_sstables_for_streaming", [&sstables] (auto& handler) -> future<> { + if (sstables.size() == 3) { + // make sure the sstables are ordered so that the sstable containing shadowed data is streamed last + const std::string_view shadowed_file = handler.template get("shadowed_file").value(); + for (int index: {0, 1}) { + if (sstables[index].sst->component_basename(component_type::Data) == shadowed_file) { + std::swap(sstables[index], sstables[2]); + } + } + } + return make_ready_future<>(); + }); + auto files = std::list(); + + sstables::sstable_generation_generator sst_gen(0); + + for (auto& sst_snapshot : sstables) { + auto& sst = sst_snapshot.sst; + // stable state (across files) is a must for load to work on destination + auto sst_state = sst->state(); + + auto sources = create_stream_sources(sst_snapshot); + auto newgen = fmt::to_string(sst_gen(sstables::uuid_identifiers::yes)); + + for (auto&& s : sources) { + auto oldname = s->component_basename(); + auto newname = get_sstable_name_with_generation(req.ops_id, oldname, newgen); + + blogger.debug("fstream[{}] Get name oldname={}, newname={}", req.ops_id, oldname, newname); + + auto& info = files.emplace_back(); + info.fops = file_ops::stream_sstables; + info.sstable_state = sst_state; + info.filename = std::move(newname); + info.source = [s = std::move(s)](const file_input_stream_options& options) { + return s->input(options); + }; + } + // ensure we mark the end of each component sequence. + if (!files.empty()) { + files.back().fops = file_ops::load_sstables; + } + } + if (files.empty()) { + co_return resp; + } + blogger.debug("stream_sstables[{}] Started sending sstable_nr={} files_nr={} files={} range={}", + req.ops_id, sstables.size(), files.size(), files, req.range); + auto ops_start_time = std::chrono::steady_clock::now(); + size_t stream_bytes = co_await tablet_stream_files(ms, std::move(files), req.targets, req.table, req.ops_id, std::move(host2ip), req.topo_guard); + resp.stream_bytes = stream_bytes; + auto duration = std::chrono::steady_clock::now() - ops_start_time; + blogger.info("stream_sstables[{}] Finished sending sstable_nr={} files_nr={} files={} range={} stream_bytes={} stream_time={} stream_bw={}", + req.ops_id, sstables.size(), files.size(), files, req.range, stream_bytes, duration, get_bw(stream_bytes, ops_start_time)); + co_return resp; +} + +future tablet_stream_files(const file_stream_id& ops_id, + replica::table& table, + const dht::token_range& range, + const locator::host_id& src_host, + const locator::host_id& dst_host, + seastar::shard_id dst_shard_id, + netw::messaging_service& ms, + abort_source& as, + service::frozen_topology_guard topo_guard) { + stream_files_response resp; + std::exception_ptr error; + try { + co_await mark_tablet_stream_start(ops_id); + } catch (...) { + error = std::current_exception(); + } + if (!error) { + try { + streaming::stream_files_request req; + req.ops_id = ops_id; + req.keyspace_name = table.schema()->ks_name(), + req.table_name = table.schema()->cf_name(); + req.table = table.schema()->id(); + req.range = range; + req.targets = std::vector{node_and_shard{dst_host, dst_shard_id}}; + req.topo_guard = topo_guard; + resp = co_await ser::streaming_rpc_verbs::send_tablet_stream_files(&ms, src_host, as, req); + } catch (...) { + error = std::current_exception(); + } + } + co_await mark_tablet_stream_done(ops_id); + if (error) { + std::rethrow_exception(error); + } + co_return resp; +} + +} diff --git a/streaming/stream_blob.hh b/streaming/stream_blob.hh new file mode 100644 index 000000000000..47ad861bc862 --- /dev/null +++ b/streaming/stream_blob.hh @@ -0,0 +1,192 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include "message/messaging_service_fwd.hh" +#include +#include +#include +#include +#include +#include +#include +#include "utils/UUID.hh" +#include "dht/i_partitioner.hh" +#include "bytes.hh" +#include "replica/database_fwd.hh" +#include "locator/host_id.hh" +#include "service/topology_guard.hh" +#include "sstables/open_info.hh" + +#include +#include + +namespace streaming { + +using file_stream_id = utils::tagged_uuid; + +// - The file_ops::stream_sstables is used to stream a sstable file. +// +// - The file_ops::load_sstables is used to stream a sstable file and +// ask the receiver to load the sstable into the system. +enum class file_ops : uint16_t { + stream_sstables, + load_sstables, +}; + +// For STREAM_BLOB verb +enum class stream_blob_cmd : uint8_t { + ok, + error, + data, + end_of_stream, +}; + +class stream_blob_data { +public: + temporary_buffer buf; + stream_blob_data() = default; + stream_blob_data(temporary_buffer b) : buf(std::move(b)) {} + const char* data() const { + return buf.get(); + } + size_t size() const { + return buf.size(); + } + bool empty() const { + return buf.size() == 0; + } +}; + +class stream_blob_cmd_data { +public: + stream_blob_cmd cmd; + // The optional data contains value when the cmd is stream_blob_cmd::data. + // When the cmd is set to other values, e.g., stream_blob_cmd::error, the + // data contains no value. + std::optional data; + stream_blob_cmd_data(stream_blob_cmd c) : cmd(c) {} + stream_blob_cmd_data(stream_blob_cmd c, std::optional d) + : cmd(c) + , data(std::move(d)) + {} + stream_blob_cmd_data(stream_blob_cmd c, stream_blob_data d) + : cmd(c) + , data(std::move(d)) + {} + +}; + +class stream_blob_meta { +public: + file_stream_id ops_id; + table_id table; + sstring filename; + seastar::shard_id dst_shard_id; + streaming::file_ops fops; + service::frozen_topology_guard topo_guard; + std::optional sstable_state; + // We can extend this verb to send arbitary blob of data +}; + +enum class store_result { + ok, failure, +}; + +using stream_blob_source_fn = noncopyable_function>(const file_input_stream_options&)>; +using stream_blob_finish_fn = noncopyable_function(store_result)>; +using output_result = std::tuple>; +using stream_blob_create_output_fn = noncopyable_function(replica::database&, const streaming::stream_blob_meta&)>; + +struct stream_blob_info { + sstring filename; + streaming::file_ops fops; + std::optional sstable_state; + stream_blob_source_fn source; + + friend inline std::ostream& operator<<(std::ostream& os, const stream_blob_info& x) { + return os << x.filename; + } +}; + +// The handler for the STREAM_BLOB verb. +seastar::future<> stream_blob_handler(replica::database& db, netw::messaging_service& ms, gms::inet_address from, streaming::stream_blob_meta meta, rpc::sink sink, rpc::source source); + +// Exposed mainly for testing + +future<> stream_blob_handler(replica::database& db, + netw::messaging_service& ms, + gms::inet_address from, + streaming::stream_blob_meta meta, + rpc::sink sink, + rpc::source source, + stream_blob_create_output_fn, + bool may_inject_errors = false + ); + +// For TABLET_STREAM_FILES +class node_and_shard { +public: + locator::host_id node; + seastar::shard_id shard; + friend inline std::ostream& operator<<(std::ostream& os, const node_and_shard& x) { + return os << x.node << ":" << x.shard; + } + +}; + +} + +template <> struct fmt::formatter : fmt::ostream_formatter {}; + +namespace streaming { + +class stream_files_request { +public: + file_stream_id ops_id; + sstring keyspace_name; + sstring table_name; + table_id table; + dht::token_range range; + std::vector targets; + service::frozen_topology_guard topo_guard; +}; + +class stream_files_response { +public: + size_t stream_bytes = 0; +}; + +using host2ip_t = std::function (locator::host_id)>; + +// The handler for the TABLET_STREAM_FILES verb. The receiver of this verb will +// stream sstables files specified by the stream_files_request req. +future tablet_stream_files_handler(replica::database& db, netw::messaging_service& ms, streaming::stream_files_request req, host2ip_t host2ip); + +// Ask the src node to stream sstables to dst node for table in the given token range using TABLET_STREAM_FILES verb. +future tablet_stream_files(const file_stream_id& ops_id, replica::table& table, const dht::token_range& range, const locator::host_id& src, const locator::host_id& dst, seastar::shard_id dst_shard_id, netw::messaging_service& ms, abort_source& as, service::frozen_topology_guard topo_guard); + +// Exposed for testability +future tablet_stream_files(netw::messaging_service& ms, + std::list sources, + std::vector targets, + table_id table, + file_stream_id ops_id, + host2ip_t host2ip, + service::frozen_topology_guard topo_guard, + bool may_inject_errors = false + ); + + +future<> mark_tablet_stream_start(file_stream_id); +future<> mark_tablet_stream_done(file_stream_id); + +} + +template<> struct fmt::formatter; diff --git a/streaming/stream_session.cc b/streaming/stream_session.cc index 4619f2b09242..b876b62381fb 100644 --- a/streaming/stream_session.cc +++ b/streaming/stream_session.cc @@ -19,6 +19,7 @@ #include "dht/auto_refreshing_sharder.hh" #include #include +#include "streaming/stream_blob.hh" #include "streaming/stream_session_state.hh" #include "service/migration_manager.hh" #include "mutation_writer/multishard_writer.hh" @@ -278,13 +279,22 @@ void stream_manager::init_messaging_service_handler(abort_source& as) { return make_ready_future<>(); } }); + ms.register_stream_blob([this] (const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source) { + auto from = netw::messaging_service::get_source(cinfo).addr; + auto sink = _ms.local().make_sink_for_stream_blob(source); + (void)stream_blob_handler(_db.local(), _ms.local(), from, meta, sink, source).handle_exception([ms = _ms.local().shared_from_this()] (std::exception_ptr eptr) { + sslog.warn("Failed to run stream blob handler: {}", eptr); + }); + return make_ready_future>(sink); + }); } future<> stream_manager::uninit_messaging_service_handler() { auto& ms = _ms.local(); return when_all_succeed( ser::streaming_rpc_verbs::unregister(&ms), - ms.unregister_stream_mutation_fragments()).discard_result(); + ms.unregister_stream_mutation_fragments(), + ms.unregister_stream_blob()).discard_result(); } stream_session::stream_session(stream_manager& mgr, locator::host_id peer_) diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index 1d95d935a34d..0b7f29c1c37b 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -251,6 +251,8 @@ add_scylla_test(string_format_test KIND BOOST) add_scylla_test(summary_test KIND BOOST) +add_scylla_test(file_stream_test + KIND SEASTAR) add_scylla_test(tagged_integer_test KIND SEASTAR) add_scylla_test(token_metadata_test diff --git a/test/boost/file_stream_test.cc b/test/boost/file_stream_test.cc new file mode 100644 index 000000000000..e704231837e4 --- /dev/null +++ b/test/boost/file_stream_test.cc @@ -0,0 +1,260 @@ +/* + * Copyright (C) 2023-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "test/lib/cql_test_env.hh" +#include "streaming/stream_blob.hh" +#include "message/messaging_service.hh" +#include "test/lib/log.hh" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +future generate_file_hash(sstring filename) { + auto f = co_await seastar::open_file_dma(filename, seastar::open_flags::ro); + auto in = seastar::make_file_input_stream(std::move(f)); + CryptoPP::SHA256 hash; + unsigned char digest[CryptoPP::SHA256::DIGESTSIZE]; + std::stringstream ss; + while (true) { + auto buf = co_await in.read(); + if (buf.empty()) { + break; + } + hash.Update((const unsigned char*)buf.get(), buf.size()); + } + co_await in.close(); + hash.Final(digest); + for (int i = 0; i < CryptoPP::SHA256::DIGESTSIZE; i++) { + ss << std::hex << std::setw(2) << std::setfill('0') << (int)digest[i]; + } + co_return ss.str(); +} + +sstring generate_random_filename() { + char filename[L_tmpnam]; + std::tmpnam(filename); + return filename; +} + +future<> write_random_content_to_file(const sstring& filename, size_t content_size = 1024) { + auto f = co_await open_file_dma(filename, open_flags::rw | open_flags::create); + auto ostream = co_await make_file_output_stream(std::move(f)); + srand(time(nullptr)); + for (size_t i = 0; i < content_size; ++i) { + char c = rand() % 256; + co_await ostream.write(&c, 1); + } + co_await ostream.close(); +} + +using namespace streaming; + +static future +do_test_file_stream(replica::database& db, netw::messaging_service& ms, std::vector filelist, const std::string& suffix, bool inject_error, bool unsupported_file_ops = false) { + bool ret = false; + bool verb_register = false; + auto ops_id = file_stream_id::create_random_id(); + auto& global_db = db.container(); + auto& global_ms = ms.container(); + int n_retries = 0; + + do { + try { + if (!verb_register) { + co_await smp::invoke_on_all([&] { + return global_ms.local().register_stream_blob([&](const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source) { + auto from = netw::messaging_service::get_source(cinfo).addr; + auto sink = global_ms.local().make_sink_for_stream_blob(source); + (void)stream_blob_handler(global_db.local(), global_ms.local(), from, meta, sink, source, [&suffix](auto&, const streaming::stream_blob_meta& meta) -> future { + auto path = meta.filename + suffix; + auto f = co_await open_file_dma(path, open_flags::wo|open_flags::create); + auto out = co_await make_file_output_stream(std::move(f)); + co_return output_result{ + [path = std::move(path)](store_result res) -> future<> { + if (res != store_result::ok) { + co_await remove_file(path); + } + }, + std::move(out) + }; + }, inject_error).handle_exception([sink, source, ms = global_ms.local().shared_from_this()] (std::exception_ptr eptr) { + testlog.warn("Failed to run stream blob handler: {}", eptr); + }); + return make_ready_future>(sink); + }); + }); + } + verb_register = true; + auto table = table_id::create_random_id(); + auto files = std::list(); + auto hostid = db.get_token_metadata().get_my_id(); + seastar::shard_id dst_shard_id = 0; + co_await mark_tablet_stream_start(ops_id); + auto targets = std::vector{node_and_shard{hostid, dst_shard_id}}; + for (const auto& filename : filelist) { + auto fops = file_ops::stream_sstables; + fops = unsupported_file_ops ? file_ops(0xff55) : fops; + auto file = co_await open_file_dma(filename, open_flags::ro); + auto& info = files.emplace_back(); + info.filename = filename; + info.fops = fops; + info.source = [file = std::move(file)](const file_input_stream_options& foptions) mutable -> future> { + co_return make_file_input_stream(std::move(file), foptions); + }; + } + auto host2ip = [&global_db] (locator::host_id id) -> future { + co_return global_db.local().get_token_metadata().get_topology().my_address(); + }; + size_t stream_bytes = co_await tablet_stream_files(ms, std::move(files), targets, table, ops_id, host2ip, service::null_topology_guard, inject_error); + co_await mark_tablet_stream_done(ops_id); + testlog.info("do_test_file_stream[{}] status=ok files={} stream_bytes={}", ops_id, filelist.size(), stream_bytes); + ret = true; + } catch (seastar::rpc::stream_closed&) { + testlog.warn("do_test_file_stream[{}] status=fail error={} retry={}", ops_id, std::current_exception(), n_retries++); + if (n_retries < 3) { + testlog.info("Retrying send"); + continue; + } + } catch (...) { + testlog.warn("do_test_file_stream[{}] status=fail error={}", ops_id, std::current_exception()); + } + } while (false); + + if (verb_register) { + co_await smp::invoke_on_all([&global_ms] { + return global_ms.local().unregister_stream_blob(); + }); + } + co_return ret; +} + +void do_test_file_stream(bool inject_error) { + cql_test_config cfg; + cfg.ms_listen = true; + std::vector files; + std::vector files_rx; + std::vector hash_tx; + std::vector hash_rx; + size_t nr_files = 10; + size_t file_size = 0; + static const std::string suffix = ".rx"; + + while (files.size() != nr_files) { + auto name = generate_random_filename(); + files.push_back(name); + files_rx.push_back(name + suffix); + } + + size_t base_size = 1024; + +#ifdef SEASTAR_DEBUG + base_size = 1; +#endif + + for (auto& file : files) { + if (file_size == 0) { + file_size = 1 * 1024 * base_size; + } else { + file_size = (rand() % 10) * 1024 * base_size + rand() % base_size; + } + file_size = std::max(size_t(1), file_size); + testlog.info("file_tx={} file_size={}", file, file_size); + write_random_content_to_file(file, file_size).get(); + } + + do_with_cql_env_thread([files, inject_error] (auto& e) { + do_test_file_stream(e.local_db(), e.get_messaging_service().local(), files, suffix, inject_error).get(); + }, cfg).get(); + + bool cleanup = true; + for (auto& file : files) { + auto hash = generate_file_hash(file).get(); + testlog.info("file_tx={} hash={}", file, hash); + hash_tx.push_back(hash); + if (cleanup) { + seastar::remove_file(file).get(); + } + } + for (auto& file : files_rx) { + sstring hash = "SKIP"; + try { + hash = generate_file_hash(file).get(); + if (cleanup) { + seastar::remove_file(file).get(); + } + } catch (...) { + if (!inject_error) { + throw; + } + } + hash_rx.push_back(hash); + testlog.info("file_rx={} hash={}", file, hash); + } + + BOOST_REQUIRE(hash_tx.size() == hash_rx.size()); + for (size_t i = 0; i < hash_tx.size(); i++) { + testlog.info("Check tx_hash={} rx_hash={}", hash_tx[i], hash_rx[i]); + if (inject_error) { + BOOST_REQUIRE(hash_tx[i] == hash_rx[i] || sstring("SKIP") == hash_rx[i]); + } else { + BOOST_REQUIRE(hash_tx[i] == hash_rx[i]); + } + } +} + +void do_test_unsupported_file_ops() { + bool inject_error = false; + bool unsupported_file_ops = true; + + cql_test_config cfg; + cfg.ms_listen = true; + std::vector files; + size_t nr_files = 2; + size_t file_size = 1024; + + while (files.size() != nr_files) { + auto name = generate_random_filename(); + files.push_back(name); + } + + for (auto& file : files) { + testlog.info("file_tx={} file_size={}", file, file_size); + write_random_content_to_file(file, file_size).get(); + } + + do_with_cql_env_thread([files, inject_error, unsupported_file_ops] (auto& e) { + auto ok = do_test_file_stream(e.local_db(), e.get_messaging_service().local(), files, "", inject_error, unsupported_file_ops).get(); + // Stream with a unsupported file ops should fail + BOOST_REQUIRE(ok == false); + }, cfg).get(); + + for (auto& file : files) { + seastar::remove_file(file).get(); + } +} + +SEASTAR_THREAD_TEST_CASE(test_file_stream) { + bool inject_error = false; + do_test_file_stream(inject_error); +} + +SEASTAR_THREAD_TEST_CASE(test_file_stream_inject_error) { + bool inject_error = true; + do_test_file_stream(inject_error); +} + +SEASTAR_THREAD_TEST_CASE(test_unsupported_file_ops) { + do_test_unsupported_file_ops(); +} diff --git a/test/topology_custom/test_tablets2.py b/test/topology_custom/test_tablets2.py index 441f11323428..fcfbbc20706b 100644 --- a/test/topology_custom/test_tablets2.py +++ b/test/topology_custom/test_tablets2.py @@ -251,6 +251,7 @@ async def test_streaming_is_guarded_by_topology_guard(manager: ManagerClient): cmdline = [ '--logger-log-level', 'storage_service=trace', '--logger-log-level', 'raft_topology=trace', + '--enable-file-stream', 'false', ] servers = [await manager.server_add(cmdline=cmdline)] diff --git a/test/topology_custom/test_tablets_migration.py b/test/topology_custom/test_tablets_migration.py index 486b93915563..f203679b7edb 100644 --- a/test/topology_custom/test_tablets_migration.py +++ b/test/topology_custom/test_tablets_migration.py @@ -6,13 +6,15 @@ from cassandra.query import SimpleStatement, ConsistencyLevel from test.pylib.manager_client import ManagerClient from test.pylib.rest_client import HTTPError, read_barrier -from test.pylib.tablets import get_all_tablet_replicas +from test.pylib.tablets import get_tablet_replica, get_all_tablet_replicas from test.topology.conftest import skip_mode from test.topology.util import wait_for_cql_and_get_hosts import time import pytest import logging import asyncio +import os +import glob logger = logging.getLogger(__name__) @@ -289,3 +291,100 @@ async def assert_rows(num): await assert_rows(2) await cql.run_async(f"INSERT INTO test.test (pk, c) VALUES ({3}, {3});") await assert_rows(3) + +@pytest.mark.asyncio +@skip_mode('release', 'error injections are not supported in release mode') +async def test_staging_backlog_is_preserved_with_file_based_streaming(manager: ManagerClient): + logger.info("Bootstrapping cluster") + # the error injection will halt view updates from staging, allowing migration to transfer the view update backlog. + cfg = {'enable_user_defined_functions': False, 'enable_tablets': True, + 'error_injections_at_startup': ['view_update_generator_consume_staging_sstable']} + servers = [await manager.server_add(config=cfg)] + + await manager.api.disable_tablet_balancing(servers[0].ip_addr) + + cql = manager.get_cql() + await cql.run_async("CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 1};") + await cql.run_async("CREATE TABLE test.test (pk int PRIMARY KEY, c int);") + await cql.run_async("CREATE MATERIALIZED VIEW test.mv1 AS \ + SELECT * FROM test.test WHERE pk IS NOT NULL AND c IS NOT NULL \ + PRIMARY KEY (c, pk);") + + logger.info("Populating single tablet") + keys = range(256) + await asyncio.gather(*[cql.run_async(f"INSERT INTO test.test (pk, c) VALUES ({k}, {k});") for k in keys]) + + await manager.api.flush_keyspace(servers[0].ip_addr, "test") + + # check + async def check(expected): + rows = await cql.run_async("SELECT pk from test.test") + assert len(list(rows)) == len(expected) + await check(keys) + + logger.info("Adding new server") + servers.append(await manager.server_add(config=cfg)) + + async def get_table_dir(manager, server_id): + node_workdir = await manager.server_get_workdir(server_id) + return glob.glob(os.path.join(node_workdir, "data", "test", "test-*"))[0] + + s0_table_dir = await get_table_dir(manager, servers[0].server_id) + logger.info(f"Table dir in server 0: {s0_table_dir}") + + s1_table_dir = await get_table_dir(manager, servers[1].server_id) + logger.info(f"Table dir in server 1: {s1_table_dir}") + + # Explicitly close the driver to avoid reconnections if scylla fails to update gossiper state on shutdown. + # It's a problem until https://github.com/scylladb/scylladb/issues/15356 is fixed. + manager.driver_close() + cql = None + await manager.server_stop_gracefully(servers[0].server_id) + + def move_sstables_to_staging(table_dir: str): + table_staging_dir = os.path.join(table_dir, "staging") + logger.info(f"Moving sstables to staging dir: {table_staging_dir}") + for sst in glob.glob(os.path.join(table_dir, "*-Data.db")): + for src_path in glob.glob(os.path.join(table_dir, sst.removesuffix("-Data.db") + "*")): + dst_path = os.path.join(table_staging_dir, os.path.basename(src_path)) + logger.info(f"Moving sstable file {src_path} to {dst_path}") + os.rename(src_path, dst_path) + + def sstable_count_in_staging(table_dir: str): + table_staging_dir = os.path.join(table_dir, "staging") + return len(glob.glob(os.path.join(table_staging_dir, "*-Data.db"))) + + move_sstables_to_staging(s0_table_dir) + s0_sstables_in_staging = sstable_count_in_staging(s0_table_dir) + + await manager.server_start(servers[0].server_id) + cql = manager.get_cql() + await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60) + + tablet_token = 0 # Doesn't matter since there is one tablet + replica = await get_tablet_replica(manager, servers[0], 'test', 'test', tablet_token) + s1_host_id = await manager.get_host_id(servers[1].server_id) + dst_shard = 0 + + migration_task = asyncio.create_task( + manager.api.move_tablet(servers[0].ip_addr, "test", "test", replica[0], replica[1], s1_host_id, dst_shard, tablet_token)) + + logger.info("Waiting for migration to finish") + await migration_task + logger.info("Migration done") + + # FIXME: After https://github.com/scylladb/scylladb/issues/19149 is fixed, we can check that view updates complete + # after migration and then check for base-view consistency. By the time being, we only check that backlog is + # transferred by looking at staging directory. + + s1_sstables_in_staging = sstable_count_in_staging(s1_table_dir) + logger.info(f"SSTable count in staging dir of server 1: {s1_sstables_in_staging}") + + logger.info("Allowing view update generator to progress again") + for server in servers: + manager.api.disable_injection(server.ip_addr, 'view_update_generator_consume_staging_sstable') + + assert s0_sstables_in_staging > 0 + assert s0_sstables_in_staging == s1_sstables_in_staging + + await check(keys) diff --git a/test/topology_custom/test_topology_ops_encrypted.py b/test/topology_custom/test_topology_ops_encrypted.py new file mode 100644 index 000000000000..ac42c35e7ca2 --- /dev/null +++ b/test/topology_custom/test_topology_ops_encrypted.py @@ -0,0 +1,85 @@ +# +# Copyright (C) 2023-present ScyllaDB +# +# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 +# +from test.pylib.scylla_cluster import ReplaceConfig +from test.pylib.manager_client import ManagerClient +from test.pylib.internal_types import ServerInfo +from test.pylib.util import unique_name, wait_for_cql_and_get_hosts +from test.topology.util import check_token_ring_and_group0_consistency, reconnect_driver +from test.topology_custom.test_topology_ops import check_node_log_for_failed_mutations, start_writes + +from cassandra.cluster import Session, ConsistencyLevel +from cassandra.query import SimpleStatement + +import asyncio +import time +import pytest +import logging + +logger = logging.getLogger(__name__) + +@pytest.mark.asyncio +@pytest.mark.parametrize("tablets_enabled", [True, False]) +async def test_topology_ops_encrypted(request, manager: ManagerClient, tablets_enabled: bool, tmp_path): + """Test basic topology operations using the topology coordinator. But encrypted.""" + d = tmp_path / "keys" + d.mkdir() + k = d / "system_key" + k.write_text('AES/CBC/PKCS5Padding:128:ApvJEoFpQmogvam18bb54g==') + cfg = {'enable_tablets' : tablets_enabled, + 'user_info_encryption': {'enabled': True, 'key_provider': 'LocalFileSystemKeyProviderFactory'}, + 'system_key_directory': d.as_posix()} + rf = 3 + num_nodes = rf + if tablets_enabled: + num_nodes += 1 + + logger.info("Bootstrapping first node") + servers = [await manager.server_add(config=cfg)] + + logger.info(f"Restarting node {servers[0]}") + await manager.server_stop_gracefully(servers[0].server_id) + await manager.server_start(servers[0].server_id) + + logger.info("Bootstrapping other nodes") + servers += await manager.servers_add(num_nodes, config=cfg) + + await wait_for_cql_and_get_hosts(manager.cql, servers, time.time() + 60) + cql = await reconnect_driver(manager) + finish_writes = await start_writes(cql, rf, ConsistencyLevel.ONE) + + logger.info(f"Decommissioning node {servers[0]}") + await manager.decommission_node(servers[0].server_id) + await check_token_ring_and_group0_consistency(manager) + servers = servers[1:] + + logger.info(f"Restarting node {servers[0]} when other nodes have bootstrapped") + await manager.server_stop_gracefully(servers[0].server_id) + await manager.server_start(servers[0].server_id) + + logger.info(f"Stopping node {servers[0]}") + await manager.server_stop_gracefully(servers[0].server_id) + await check_node_log_for_failed_mutations(manager, servers[0]) + + logger.info(f"Replacing node {servers[0]}") + replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = False) + servers = servers[1:] + [await manager.server_add(replace_cfg)] + await check_token_ring_and_group0_consistency(manager) + + logger.info(f"Stopping node {servers[0]}") + await manager.server_stop_gracefully(servers[0].server_id) + await check_node_log_for_failed_mutations(manager, servers[0]) + + logger.info(f"Removing node {servers[0]} using {servers[1]}") + await manager.remove_node(servers[1].server_id, servers[0].server_id) + await check_token_ring_and_group0_consistency(manager) + servers = servers[1:] + + logger.info("Checking results of the background writes") + await finish_writes() + + for server in servers: + await check_node_log_for_failed_mutations(manager, server) + From 9d6ec45730a196e79e988371525437dddc45b918 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Sat, 18 Jan 2025 19:19:01 +0800 Subject: [PATCH 389/397] build: support wasm32-wasip1 target in configure.py Update configure.py to use wasm32-wasip1 as an alternative to wasm32-wasi, matching the behavior previously implemented for CMake builds in 8d7786cb0e. This ensures consistent WASI target handling across both build systems. Refs #20878 Signed-off-by: Kefu Chai Closes scylladb/scylladb#22386 --- configure.py | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/configure.py b/configure.py index dbcb40a80f5a..b0d805299571 100755 --- a/configure.py +++ b/configure.py @@ -2154,6 +2154,15 @@ def get_release_cxxflags(scylla_product, return [f'-D{name}="\\"{value}\\""' for name, value in definitions.items()] +def pick_rustc_target(*candidates): + output = subprocess.check_output(['rustc', '--print', 'target-list'], text=True) + target_list = output.splitlines() + for candidate in candidates: + if candidate in target_list: + return candidate + raise RuntimeError("none of the specified target is supported by rustc") + + def write_build_file(f, arch, ninja, @@ -2162,6 +2171,7 @@ def write_build_file(f, scylla_release, args): warnings = get_warning_options(args.cxx) + rustc_target = pick_rustc_target('wasm32-wasi', 'wasm32-wasip1') f.write(textwrap.dedent('''\ configure_args = {configure_args} builddir = {outdir} @@ -2224,8 +2234,8 @@ def write_build_file(f, command = clang --target=wasm32 --no-standard-libraries -Wl,--export-all -Wl,--no-entry $in -o $out description = C2WASM $out rule rust2wasm - command = cargo build --target=wasm32-wasi --example=$example --locked --manifest-path=test/resource/wasm/rust/Cargo.toml --target-dir=$builddir/wasm/ $ - && wasm-opt -Oz $builddir/wasm/wasm32-wasi/debug/examples/$example.wasm -o $builddir/wasm/$example.wasm $ + command = cargo build --target={rustc_target} --example=$example --locked --manifest-path=test/resource/wasm/rust/Cargo.toml --target-dir=$builddir/wasm/ $ + && wasm-opt -Oz $builddir/wasm/{rustc_target}/debug/examples/$example.wasm -o $builddir/wasm/$example.wasm $ && wasm-strip $builddir/wasm/$example.wasm description = RUST2WASM $out rule wasm2wat @@ -2247,6 +2257,7 @@ def write_build_file(f, linker_flags=linker_flags, user_ldflags=user_ldflags, libs=libs, + rustc_target=rustc_target, link_pool_depth=link_pool_depth, seastar_path=args.seastar_path, ninja=ninja, From 88ae067ddbd1faf316d0971f641961582a793476 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Tue, 3 Dec 2024 11:26:30 +0200 Subject: [PATCH 390/397] everywhere: add skeletal support for the in_memory_tables feature Forward-ported from scylla-enterprise. Note that the feature has been deprecated and the implementation is provided only for backward compatibility with pre-existing features and schema. Tested manually after adding the following to feature_service: ``` gms::feature workload_prioritization { *this, "WORKLOAD_PRIORITIZATION"sv }; ``` Launched a single-node cluster running 2023.1.10 ``` cqlsh> create KEYSPACE ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; cqlsh> create TABLE ks.test ( pk int PRIMARY KEY, val int ) WITH compaction = {'class': 'InMemoryCompactionStrategy'}; ``` log: ``` Scylla version 2023.1.10-0.20241227.21cffccc1ccd with build-id bd65b8399cb13b713a87e57fe333cfcabfd50be7 starting ... ... INFO 2024-12-27 19:45:16,563 [shard 0] migration_manager - Create new ColumnFamily: org.apache.cassandra.config.CFMetaData@0x600000f1b400[cfId=5529c630-c47a-11ef-bd1d-4295734ce5a8,ksName=ks,cfName=test,cfType=Standard,comparator=org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type),comment=,readRepairChance=0,dcLocalReadRepairChance=0,tombstoneGcOptions={"mode":"timeout","propagation_delay_in_seconds":"3600"},gcGraceSeconds=864000,keyValidator=org.apache.cassandra.db.marshal.Int32Type,minCompactionThreshold=4,maxCompactionThreshold=32,columnMetadata=[ColumnDefinition{name=pk, type=org.apache.cassandra.db.marshal.Int32Type, kind=PARTITION_KEY, componentIndex=0, droppedAt=-9223372036854775808}, ColumnDefinition{name=val, type=org.apache.cassandra.db.marshal.Int32Type, kind=REGULAR, componentIndex=null, droppedAt=-9223372036854775808}],compactionStrategyClass=class org.apache.cassandra.db.compaction.InMemoryCompactionStrategy,compactionStrategyOptions={enabled=true},compressionParameters={sstable_compression=org.apache.cassandra.io.compress.LZ4Compressor},bloomFilterFpChance=0.01,memtableFlushPeriod=0,caching={"keys":"ALL","rows_per_partition":"ALL"},cdc={},defaultTimeToLive=0,minIndexInterval=128,maxIndexInterval=2048,speculativeRetry=99.0PERCENTILE,triggers=[],isDense=false,in_memory=false,version=5529c631-c47a-11ef-bd1d-4295734ce5a8,droppedColumns={},collections={},indices={}] INFO 2024-12-27 19:45:16,564 [shard 0] schema_tables - Creating ks.test id=5529c630-c47a-11ef-bd1d-4295734ce5a8 version=ec88d510-6aff-344a-914d-541d37081440 ``` Upgraded to this branch and started scylla. Verified that ks.test was successfuly loaded: log: ``` INFO 2024-12-27 19:48:58,115 [shard 0:main] init - Scylla version 6.3.0~dev-0.20241227.a64c6dfc153e with build-id f9496134a09cf2e55d3865b9e9ff499f672aa7da starting ... ... WARN 2024-12-27 19:53:02,948 [shard 1:main] CompactionStrategy - InMemoryCompactionStrategy is no longer supported. Defaulting to NullCompactionStrategy. ... INFO 2024-12-27 19:53:02,948 [shard 0:main] database - Keyspace ks: Reading CF test id=5529c630-c47a-11ef-bd1d-4295734ce5a8 version=ec88d510-6aff-344a-914d-541d37081440 storage=/home/bhalevy/scylladb/data/ks/test-5529c630c47a11efbd1d4295734ce5a8 ``` Then, tested: ``` cqlsh> describe KEYSPACE ks; CREATE KEYSPACE ks WITH replication = {'class': 'org.apache.cassandra.locator.SimpleStrategy', 'replication_factor': '1'} AND durable_writes = true AND tablets = {'enabled': false}; CREATE TABLE ks.test ( pk int, val int, PRIMARY KEY (pk) ) WITH bloom_filter_fp_chance = 0.01 AND caching = {'keys': 'ALL', 'rows_per_partition': 'ALL'} AND comment = '' AND compaction = {'class': 'InMemoryCompactionStrategy'} AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.LZ4Compressor'} AND crc_check_chance = 1 AND default_time_to_live = 0 AND gc_grace_seconds = 864000 AND max_index_interval = 2048 AND memtable_flush_period_in_ms = 0 AND min_index_interval = 128 AND speculative_retry = '99.0PERCENTILE'; cqlsh> alter TABLE ks.test with compaction = {'class': 'SizeTieredCompactionStrategy'}; cqlsh> describe KEYSPACE ks; CREATE KEYSPACE ks WITH replication = {'class': 'org.apache.cassandra.locator.SimpleStrategy', 'replication_factor': '1'} AND durable_writes = true AND tablets = {'enabled': false}; CREATE TABLE ks.test ( pk int, val int, PRIMARY KEY (pk) ) WITH bloom_filter_fp_chance = 0.01 AND caching = {'keys': 'ALL', 'rows_per_partition': 'ALL'} AND comment = '' AND compaction = {'class': 'SizeTieredCompactionStrategy'} AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.LZ4Compressor'} AND crc_check_chance = 1 AND default_time_to_live = 0 AND gc_grace_seconds = 864000 AND max_index_interval = 2048 AND memtable_flush_period_in_ms = 0 AND min_index_interval = 128 AND speculative_retry = '99.0PERCENTILE' AND tombstone_gc = {'mode': 'timeout', 'propagation_delay_in_seconds': '3600'}; ``` log: ``` INFO 2024-12-27 19:56:40,465 [shard 0:stmt] migration_manager - Update table 'ks.test' From org.apache.cassandra.config.CFMetaData@0x60000362d800[cfId=5529c630-c47a-11ef-bd1d-4295734ce5a8,ksName==ks,cfName=test,cfType=Standard,comparator=org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type),comment=,tombstoneGcOptions={"mode":"timeout","propagation_delay_in_seconds":"3600"},gcGraceSeconds=864000,minCompactionThreshold=4,maxCompactionThreshold=32,columnMetadata=[ColumnDefinition{name=pk, type=org.apache.cassandra.db.marshal.Int32Type, kind=PARTITION_KEY, componentIndex=0, droppedAt=-9223372036854775808}, ColumnDefinition{name=val, type=org.apache.cassandra.db.marshal.Int32Type, kind=REGULAR, componentIndex=null, droppedAt=-9223372036854775808}],compactionStrategyClass=class org.apache.cassandra.db.compaction.InMemoryCompactionStrategy,compactionStrategyOptions={enabled=true},compressionParameters={sstable_compression=org.apache.cassandra.io.compress.LZ4Compressor},bloomFilterFpChance=0.01,memtableFlushPeriod=0,caching={"keys":"ALL","rows_per_partition":"ALL"},cdc={},defaultTimeToLive=0,minIndexInterval=128,maxIndexInterval=2048,speculativeRetry=99.0PERCENTILE,triggers=[],isDense=false,version=ec88d510-6aff-344a-914d-541d37081440,droppedColumns={},collections={},indices={}] To org.apache.cassandra.config.CFMetaData@0x60000336e000[cfId=5529c630-c47a-11ef-bd1d-4295734ce5a8,ksName==ks,cfName=test,cfType=Standard,comparator=org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type),comment=,tombstoneGcOptions={"mode":"timeout","propagation_delay_in_seconds":"3600"},gcGraceSeconds=864000,minCompactionThreshold=4,maxCompactionThreshold=32,columnMetadata=[ColumnDefinition{name=pk, type=org.apache.cassandra.db.marshal.Int32Type, kind=PARTITION_KEY, componentIndex=0, droppedAt=-9223372036854775808}, ColumnDefinition{name=val, type=org.apache.cassandra.db.marshal.Int32Type, kind=REGULAR, componentIndex=null, droppedAt=-9223372036854775808}],compactionStrategyClass=class org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy,compactionStrategyOptions={enabled=true},compressionParameters={sstable_compression=org.apache.cassandra.io.compress.LZ4Compressor},bloomFilterFpChance=0.01,memtableFlushPeriod=0,caching={"keys":"ALL","rows_per_partition":"ALL"},cdc={},defaultTimeToLive=0,minIndexInterval=128,maxIndexInterval=2048,speculativeRetry=99.0PERCENTILE,triggers=[],isDense=false,version=ecccf010-c47b-11ef-b52c-622f2f0e87c4,droppedColumns={},collections={},indices={}] INFO 2024-12-27 19:56:40,466 [shard 0: gms] schema_tables - Altering ks.test id=5529c630-c47a-11ef-bd1d-4295734ce5a8 version=ecccf010-c47b-11ef-b52c-622f2f0e87c4 ``` Signed-off-by: Benny Halevy Closes scylladb/scylladb#22068 --- compaction/compaction_strategy.cc | 10 ++++++++ compaction/compaction_strategy.hh | 4 ++++ compaction/compaction_strategy_type.hh | 1 + cql3/statements/create_table_statement.cc | 3 +++ db/schema_features.hh | 6 ++++- db/schema_tables.cc | 24 +++++++++++++++++-- .../sstable2/sstable-interpretation.rst | 2 +- gms/feature_service.cc | 1 + gms/feature_service.hh | 1 + schema/schema.cc | 2 ++ schema/schema.hh | 5 ++++ schema/schema_builder.hh | 5 ++++ service/migration_manager.cc | 1 + sstables/sstables.hh | 4 ++++ test/boost/schema_change_test.cc | 2 +- 15 files changed, 66 insertions(+), 5 deletions(-) diff --git a/compaction/compaction_strategy.cc b/compaction/compaction_strategy.cc index 4312e130caf5..2ca5d46be062 100644 --- a/compaction/compaction_strategy.cc +++ b/compaction/compaction_strategy.cc @@ -180,6 +180,9 @@ void compaction_strategy_impl::validate_options_for_strategy_type(const std::map break; default: break; + case compaction_strategy_type::null: + case compaction_strategy_type::in_memory: + return; } unchecked_options.erase("class"); @@ -762,6 +765,13 @@ compaction_strategy make_compaction_strategy(compaction_strategy_type strategy, case compaction_strategy_type::time_window: impl = ::make_shared(options); break; + case compaction_strategy_type::in_memory: + compaction_strategy_logger.warn( + "{} is no longer supported. Defaulting to {}.", + compaction_strategy::name(compaction_strategy_type::in_memory), + compaction_strategy::name(compaction_strategy_type::null)); + impl = ::make_shared(); + break; case compaction_strategy_type::incremental: impl = make_shared(incremental_compaction_strategy(options)); break; diff --git a/compaction/compaction_strategy.hh b/compaction/compaction_strategy.hh index aaaaade7f4dd..a7bb8ab8363a 100644 --- a/compaction/compaction_strategy.hh +++ b/compaction/compaction_strategy.hh @@ -70,6 +70,8 @@ public: return "LeveledCompactionStrategy"; case compaction_strategy_type::time_window: return "TimeWindowCompactionStrategy"; + case compaction_strategy_type::in_memory: + return "InMemoryCompactionStrategy"; case compaction_strategy_type::incremental: return "IncrementalCompactionStrategy"; default: @@ -88,6 +90,8 @@ public: return compaction_strategy_type::leveled; } else if (short_name == "TimeWindowCompactionStrategy") { return compaction_strategy_type::time_window; + } else if (short_name == "InMemoryCompactionStrategy") { + return compaction_strategy_type::in_memory; } else if (short_name == "IncrementalCompactionStrategy") { return compaction_strategy_type::incremental; } else { diff --git a/compaction/compaction_strategy_type.hh b/compaction/compaction_strategy_type.hh index b0a85a087ec1..51837de6c646 100644 --- a/compaction/compaction_strategy_type.hh +++ b/compaction/compaction_strategy_type.hh @@ -17,6 +17,7 @@ enum class compaction_strategy_type { size_tiered, leveled, time_window, + in_memory, incremental, }; diff --git a/cql3/statements/create_table_statement.cc b/cql3/statements/create_table_statement.cc index 4341ae1d4ecd..7f00fbede427 100644 --- a/cql3/statements/create_table_statement.cc +++ b/cql3/statements/create_table_statement.cc @@ -464,6 +464,9 @@ std::optional check_restricted_table_properties( // Evaluate whether the strategy to evaluate was explicitly passed auto cs = (strategy) ? strategy : current_strategy; + if (cs == sstables::compaction_strategy_type::in_memory) { + throw exceptions::configuration_exception(format("{} has been deprecated.", sstables::compaction_strategy::name(*cs))); + } if (cs == sstables::compaction_strategy_type::time_window) { std::map options = (strategy) ? cfprops.get_compaction_type_options() : (*schema)->compaction_strategy_options(); sstables::time_window_compaction_strategy_options twcs_options(options); diff --git a/db/schema_features.hh b/db/schema_features.hh index 792a0db2dd8e..22197720079a 100644 --- a/db/schema_features.hh +++ b/db/schema_features.hh @@ -27,6 +27,9 @@ enum class schema_feature { // When enabled we'll add a new column to the `system_schema.scylla_tables` table. GROUP0_SCHEMA_VERSIONING, + + // Unused. Defined for backward compatibility only + IN_MEMORY_TABLES, }; using schema_features = enum_set>; } diff --git a/db/schema_tables.cc b/db/schema_tables.cc index fcb87626be85..644fd30709d7 100644 --- a/db/schema_tables.cc +++ b/db/schema_tables.cc @@ -302,11 +302,12 @@ schema_ptr tables() { // Holds Scylla-specific table metadata. schema_ptr scylla_tables(schema_features features) { - static thread_local schema_ptr schemas[2]{}; + static thread_local schema_ptr schemas[2][2]{}; bool has_group0_schema_versioning = features.contains(schema_feature::GROUP0_SCHEMA_VERSIONING); + bool has_in_memory = features.contains(schema_feature::IN_MEMORY_TABLES); - schema_ptr& s = schemas[has_group0_schema_versioning]; + schema_ptr& s = schemas[has_in_memory][has_group0_schema_versioning]; if (!s) { auto id = generate_legacy_id(NAME, SCYLLA_TABLES); auto sb = schema_builder(NAME, SCYLLA_TABLES, std::make_optional(id)) @@ -319,6 +320,10 @@ schema_ptr scylla_tables(schema_features features) { // PER_TABLE_PARTITIONERS sb.with_column("partitioner", utf8_type); + if (has_in_memory) { + sb.with_column("in_memory", boolean_type); + } + if (has_group0_schema_versioning) { // If true, this table's latest schema was committed by group 0. // In this case `version` column is non-null and will be used for `schema::version()` instead of calculating a hash. @@ -1728,6 +1733,9 @@ mutation make_scylla_tables_mutation(schema_ptr table, api::timestamp_type times auto& cdef = *scylla_tables()->get_column_definition("partitioner"); m.set_clustered_cell(ckey, cdef, atomic_cell::make_dead(timestamp, gc_clock::now())); } + // In-memory tables are deprecated since scylla-2024.1.0 + // FIXME: delete the column when there's no live version supporting it anymore. + // Writing it here breaks upgrade rollback to versions that do not support the in_memory schema_feature return m; } @@ -2197,6 +2205,18 @@ schema_ptr create_table_from_mutations(const schema_ctxt& ctxt, schema_mutations prepare_builder_from_table_row(ctxt, builder, table_row); + if (sm.scylla_tables()) { + table_rs = query::result_set(*sm.scylla_tables()); + if (!table_rs.empty()) { + query::result_set_row table_row = table_rs.row(0); + auto in_mem = table_row.get("in_memory"); + auto in_mem_enabled = in_mem.value_or(false); + if (in_mem_enabled) { + slogger.warn("Support for in_memory tables has been deprecated."); + } + builder.set_in_memory(in_mem_enabled); + } + } v3_columns columns(std::move(column_defs), is_dense, is_compound); columns.apply_to(builder); diff --git a/docs/architecture/sstable/sstable2/sstable-interpretation.rst b/docs/architecture/sstable/sstable2/sstable-interpretation.rst index 993d60cc7811..d71a6ea44bec 100644 --- a/docs/architecture/sstable/sstable2/sstable-interpretation.rst +++ b/docs/architecture/sstable/sstable2/sstable-interpretation.rst @@ -22,7 +22,7 @@ is just a **mutation**, a list of changed (added or deleted) columns and their new values (or "tombstone" for a deleted column), and a timestamp for each such change (this timestamp is used for reconciling conflicting mutations). The full data row needed by a request will be composed from -potentially multiple sstables and/or the in-memory table(s). +potentially multiple sstables. As we'll explain below when discussing clustering columns, the best term for what we read from one row in the SSTable isn't a "row", but rather a diff --git a/gms/feature_service.cc b/gms/feature_service.cc index 009224591fde..d687369028e1 100644 --- a/gms/feature_service.cc +++ b/gms/feature_service.cc @@ -236,6 +236,7 @@ db::schema_features feature_service::cluster_schema_features() const { f.set_if(aggregate_storage_options); f.set_if(table_digest_insensitive_to_expiry); f.set_if(group0_schema_versioning); + f.set_if(bool(in_memory_tables)); return f; } diff --git a/gms/feature_service.hh b/gms/feature_service.hh index 7ac881ed3d5b..8b1f7a78a44f 100644 --- a/gms/feature_service.hh +++ b/gms/feature_service.hh @@ -158,6 +158,7 @@ public: gms::feature test_only_feature { *this, "TEST_ONLY_FEATURE"sv }; gms::feature address_nodes_by_host_ids { *this, "ADDRESS_NODES_BY_HOST_IDS"sv }; + gms::feature in_memory_tables { *this, "IN_MEMORY_TABLES"sv }; gms::feature workload_prioritization { *this, "WORKLOAD_PRIORITIZATION"sv }; gms::feature compression_dicts { *this, "COMPRESSION_DICTS"sv }; gms::feature file_stream { *this, "FILE_STREAM"sv }; diff --git a/schema/schema.cc b/schema/schema.cc index fe5ad80c4b0f..ff56c511f2c5 100644 --- a/schema/schema.cc +++ b/schema/schema.cc @@ -581,6 +581,7 @@ bool operator==(const schema& x, const schema& y) && indirect_equal_to>()(x._view_info, y._view_info) && x._raw._indices_by_name == y._raw._indices_by_name && x._raw._is_counter == y._raw._is_counter + && x._raw._in_memory == y._raw._in_memory ; } @@ -839,6 +840,7 @@ auto fmt::formatter::format(const schema& s, fmt::format_context& ctx) c out = fmt::format_to(out, ",speculativeRetry={}", s._raw._speculative_retry.to_sstring()); out = fmt::format_to(out, ",triggers=[]"); out = fmt::format_to(out, ",isDense={}", s._raw._is_dense); + out = fmt::format_to(out, ",in_memory={}", s._raw._in_memory); out = fmt::format_to(out, ",version={}", s.version()); out = fmt::format_to(out, ",droppedColumns={{"); diff --git a/schema/schema.hh b/schema/schema.hh index ce66a9a59ac5..8ad53689edbe 100644 --- a/schema/schema.hh +++ b/schema/schema.hh @@ -566,6 +566,7 @@ private: // Sharding info is not stored in the schema mutation and does not affect // schema digest. It is also not set locally on a schema tables. std::reference_wrapper _sharder; + bool _in_memory = false; std::optional _view_info; }; raw_schema _raw; @@ -763,6 +764,10 @@ public: bool has_custom_partitioner() const; + bool is_in_memory() const { + return _raw._in_memory; + } + const column_definition* get_column_definition(const bytes& name) const; const column_definition& column_at(column_kind, column_id) const; // Find a column definition given column ordinal id in the schema diff --git a/schema/schema_builder.hh b/schema/schema_builder.hh index 782a63edff9a..22fb39cbd8cf 100644 --- a/schema/schema_builder.hh +++ b/schema/schema_builder.hh @@ -222,6 +222,11 @@ public: schema_builder& with_partitioner(sstring name); schema_builder& with_sharder(unsigned shard_count, unsigned sharding_ignore_msb_bits); + schema_builder& set_in_memory(bool in_memory) { + _raw._in_memory = in_memory; + return *this; + } + class default_names { public: default_names(const schema_builder&); diff --git a/service/migration_manager.cc b/service/migration_manager.cc index ef3ffa98cf4e..6cd5f43f5532 100644 --- a/service/migration_manager.cc +++ b/service/migration_manager.cc @@ -121,6 +121,7 @@ void migration_manager::init_messaging_service() _feature_listeners.push_back(feature.when_enabled(reload_schema_in_bg)); } } + _feature_listeners.push_back(_feat.in_memory_tables.when_enabled(reload_schema_in_bg)); } ser::migration_manager_rpc_verbs::register_definitions_update(&_messaging, [this] (const rpc::client_info& cinfo, std::vector, rpc::optional> cm) { diff --git a/sstables/sstables.hh b/sstables/sstables.hh index 3a1eea4bdb95..21301578c75d 100644 --- a/sstables/sstables.hh +++ b/sstables/sstables.hh @@ -54,6 +54,8 @@ namespace data_dictionary { class storage_options; } +class in_memory_config_type; + namespace db { class large_data_handler; } @@ -1057,6 +1059,8 @@ public: future> read_digest(); future> read_checksum(); + + friend in_memory_config_type; }; // Validate checksums diff --git a/test/boost/schema_change_test.cc b/test/boost/schema_change_test.cc index 3f7a7f7c4e00..3b8e08a8edce 100644 --- a/test/boost/schema_change_test.cc +++ b/test/boost/schema_change_test.cc @@ -1158,7 +1158,7 @@ SEASTAR_TEST_CASE(test_system_schema_version_is_stable) { // If you changed the schema of system.batchlog then this is expected to fail. // Just replace expected version with the new version. - BOOST_REQUIRE_EQUAL(s->version(), table_schema_version(utils::UUID("776f1766-8688-3d52-908b-a5228900dc00"))); + BOOST_REQUIRE_EQUAL(s->version(), table_schema_version(utils::UUID("3febbbce-8841-304a-abb9-170078ac173d"))); }); } From de153a2ba71b389292d3b85aad8504689018b481 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 20 Jan 2025 17:52:43 +0100 Subject: [PATCH 391/397] main: rename `cql_sg_stats` metrics on scheduling group rename This commit contains the part of a fix for scylladb/scylla-enterprise#4912 that was accidentally omitted when workload prioritization were ported from enterprise to scylladb.git repo. Without it, the metrics created by `cql_sg_stats` would not be updated, leading to wrong scheduling group names being used in metrics' names, and could lead to "double metric registration errors" in some unlucky circumstances where a scheduling group would be created, destroyed and then created again. Fixes: scylladb/scylladb#22404 --- main.cc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/main.cc b/main.cc index 291e088e312b..b1c74d11694d 100644 --- a/main.cc +++ b/main.cc @@ -1202,6 +1202,9 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl make_scheduling_group_key_config(maintenance_socket_enabled::yes); auto maintenance_cql_sg_stats_key = scheduling_group_key_create(maintenance_cql_sg_stats_cfg).get(); scheduling_group_key_config cql_sg_stats_cfg = make_scheduling_group_key_config(maintenance_socket_enabled::no); + cql_sg_stats_cfg.rename = [] (void* ptr) { + reinterpret_cast(ptr)->rename_metrics(); + }; auto cql_sg_stats_key = scheduling_group_key_create(cql_sg_stats_cfg).get(); supervisor::notify("starting disk space monitor"); From 780ff17ff528ff191231be987a3e70073692d23b Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 20 Jan 2025 17:26:58 +0100 Subject: [PATCH 392/397] test/auth_cluster: make test_service_level_metric_name_change useful The test test_service_level_metric_name_change was originally introduced to serve as a regression test for scylladb/scylla-enterprise#4912. Before the fix, some per-scheduling-group metrics would not get adjusted when the scheduling group gets renamed (which does happen for SL-managed scheduling groups) and it would be possible to attempt to register metrics with the same set of labels, resulting in an error. However, in scylladb/scylla-enterprise#4764, another bug was fixed which affected the test. Before a service level is created, a "test" scheduling group can be created by service level controller if it is unsure whether it is allowed to create more scheduling groups or not. If creation of the scheduling group succeeds, it is put into the pool of scheduling groups to be reused when a new service level is created. Therefore, the node handling CREATE SERVICE LEVEL would always use the scheduling group that was originally created for the sake of the test as a SG for the new service level. All of the above is intentional and was actually fixed by the aforementioned issue. However, the test scheduling groups would always get unique names and, therefore, the error would no longer reproduce. However, the faulty logic that ran previously and caused the bug still runs - when a node updates its service levels cache on group0 reload. The test previously used only one node. Fix it by starting two nodes instead of one at the beginning of the test and by serving all service level commands to the first node - were the issue not fixed, the error would get triggered on the second node. --- test/auth_cluster/test_raft_service_levels.py | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/test/auth_cluster/test_raft_service_levels.py b/test/auth_cluster/test_raft_service_levels.py index b2ebb6bfed3c..5d2177042b6e 100644 --- a/test/auth_cluster/test_raft_service_levels.py +++ b/test/auth_cluster/test_raft_service_levels.py @@ -445,21 +445,27 @@ async def test_service_levels_over_limit(manager: ManagerClient): # Reproduces issue scylla-enterprise#4912 @pytest.mark.asyncio async def test_service_level_metric_name_change(manager: ManagerClient) -> None: - s = await manager.server_add() - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + servers = await manager.servers_add(2) + s = servers[0] cql = manager.get_cql() + [h] = await wait_for_cql_and_get_hosts(cql, [s], time.time() + 60) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) sl1 = unique_name() sl2 = unique_name() + # All service level commands need to run on the first node. It is the logic + # that is exercised during service level data reload from group0 which is + # prone to name reuse and, hence, could trigger the error fixed by 4912. + # creates scheduling group `sl:sl1` - await cql.run_async(f"CREATE SERVICE LEVEL {sl1}") + await cql.run_async(f"CREATE SERVICE LEVEL {sl1}", host=h) # renames scheduling group `sl:sl1` to `sl_deleted:sl1` - await cql.run_async(f"DROP SERVICE LEVEL {sl1}") + await cql.run_async(f"DROP SERVICE LEVEL {sl1}", host=h) # renames scheduling group `sl_deleted:sl1` to `sl:sl2` - await cql.run_async(f"CREATE SERVICE LEVEL {sl2}") + await cql.run_async(f"CREATE SERVICE LEVEL {sl2}", host=h) # creates scheduling group `sl:sl1` - await cql.run_async(f"CREATE SERVICE LEVEL {sl1}") + await cql.run_async(f"CREATE SERVICE LEVEL {sl1}", host=h) # In issue #4912, service_level_controller thought there was no room # for `sl:sl1` scheduling group because create_scheduling_group() failed due to # `seastar::metrics::double_registration (registering metrics twice for metrics: transport_cql_requests_count)` @@ -467,7 +473,7 @@ async def test_service_level_metric_name_change(manager: ManagerClient) -> None: # When sl2 is dropped, service_level_controller tries to rename its # scheduling group to `sl:sl1`, triggering # `seastar::metrics::double_registration (registering metrics twice for metrics: scheduler_runtime_ms)` - await cql.run_async(f"DROP SERVICE LEVEL {sl2}") + await cql.run_async(f"DROP SERVICE LEVEL {sl2}", host=h) # Check if group0 is healthy s2 = await manager.server_add() From d5d251da9afcc6ed56ba3480953856d986e6f968 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Fri, 17 Jan 2025 11:51:04 +0800 Subject: [PATCH 393/397] utils: implement drop-in replacement for replacing boost::adaptors::uniqued Add a custom implementation of boost::adaptors::uniqued that is compatible with C++20 ranges library. This bridges the gap between Boost.Range and the C++ standard library ranges until std::views::unique becomes available in C++26. Currently, the unique view is included in [P2214](https://www.open-std.org/jtc1/sc22/wg21/docs/papers/2023/p2760r0.html) "A Plan for C++ Ranges Evolution", which targets C++26. The implementation provides: - A lazy view adaptor that presents unique consecutive elements - No modification of source range - Compatibility with C++20 range views and concepts - Lighter header dependencies compared to Boost This resolves compilation errors when piping C++20 range views to boost::adaptors::uniqued, which fails due to concept requirements mismatch. For example: ```c++ auto range = std::views::take(n) | boost::adaptors::uniqued; // fails ``` This change also offers us a lightweight solution in terms of smaller header dependency. While std::ranges::unique exists in C++23, it's an eager algorithm that modifies the source range in-place, unlike boost::adaptors::uniqued which is a lazy view. The proposed std::views::unique (P2214) targeting C++26 would provide this functionality, but is not yet available. This implementation serves as an interim solution for filtering consecutive duplicate elements using range views until std::views::unique is standardized. For more details on the differences between `std::ranges::unique` and `boost::adaptors::uniqued`: - boost::adaptors::uniqued is a view adaptor that creates a lazy view over the original range. It: * Doesn't modify the source range * Returns a view that presents unique consecutive elements * Is non-destructive and lazy-evaluated * Can be composed with other views - std::ranges::unique is an algorithm that: * Modifies the source range in-place * Removes consecutive duplicates by shifting elements * Returns an iterator to the new logical end * Cannot be used as a view or composed with other range adaptors Signed-off-by: Kefu Chai --- configure.py | 1 + test/boost/CMakeLists.txt | 2 + test/boost/unique_view_test.cc | 116 +++++++++++++++++++ utils/unique_view.hh | 201 +++++++++++++++++++++++++++++++++ 4 files changed, 320 insertions(+) create mode 100644 test/boost/unique_view_test.cc create mode 100644 utils/unique_view.hh diff --git a/configure.py b/configure.py index 0458a85cd1a0..0fb3fc3c2be0 100755 --- a/configure.py +++ b/configure.py @@ -570,6 +570,7 @@ def find_ninja(): 'test/boost/wasm_alloc_test', 'test/boost/wasm_test', 'test/boost/wrapping_interval_test', + 'test/boost/unique_view_test', 'test/manual/ec2_snitch_test', 'test/manual/enormous_table_scan_test', 'test/manual/gce_snitch_test', diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index 1d95d935a34d..26fddfec575e 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -261,6 +261,8 @@ add_scylla_test(transport_test KIND SEASTAR) add_scylla_test(types_test KIND SEASTAR) +add_scylla_test(unique_view_test + KIND BOOST) add_scylla_test(utf8_test KIND BOOST LIBRARIES utils) diff --git a/test/boost/unique_view_test.cc b/test/boost/unique_view_test.cc new file mode 100644 index 000000000000..42862bcacc76 --- /dev/null +++ b/test/boost/unique_view_test.cc @@ -0,0 +1,116 @@ +#define BOOST_TEST_MODULE test-ranges + +#include +#include +#include +#include +#include + +#include "utils/unique_view.hh" + + +BOOST_AUTO_TEST_CASE(test_empty_range) { + std::vector empty; + auto view = empty | utils::views::unique; + + BOOST_CHECK(std::ranges::empty(view)); +} + +BOOST_AUTO_TEST_CASE(test_single_element) { + std::vector single{42}; + auto view = single | utils::views::unique; + + BOOST_CHECK_EQUAL(std::ranges::distance(view), 1); + BOOST_CHECK_EQUAL(*view.begin(), 42); +} + +BOOST_AUTO_TEST_CASE(test_all_same_elements) { + std::vector same{1, 1, 1, 1, 1}; + auto view = same | utils::views::unique; + + BOOST_CHECK_EQUAL(std::ranges::distance(view), 1); + BOOST_CHECK_EQUAL(*view.begin(), 1); +} + +BOOST_AUTO_TEST_CASE(test_all_different_elements) { + std::vector different{1, 2, 3, 4, 5}; + auto result = different | utils::views::unique | std::ranges::to(); + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + different.begin(), different.end()); +} + +BOOST_AUTO_TEST_CASE(test_consecutive_duplicates) { + std::vector input{1, 1, 2, 2, 3, 3, 2, 2, 1, 1}; + auto result = input | utils::views::unique | std::ranges::to(); + + std::vector expected{1, 2, 3, 2, 1}; + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + expected.begin(), expected.end()); +} + +BOOST_AUTO_TEST_CASE(test_string_elements) { + std::vector input{"hello", "hello", "world", "world", "hello"}; + auto result = input | utils::views::unique | std::ranges::to(); + + std::vector expected{"hello", "world", "hello"}; + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + expected.begin(), expected.end()); +} + +BOOST_AUTO_TEST_CASE(test_different_container_type) { + std::list input{1, 1, 2, 2, 3, 3}; + auto result = input | utils::views::unique | std::ranges::to(); + + std::vector expected{1, 2, 3}; + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + expected.begin(), expected.end()); +} + +BOOST_AUTO_TEST_CASE(test_const_range) { + const std::vector input{1, 1, 2, 2, 3}; + auto result = input | utils::views::unique | std::ranges::to(); + + std::vector expected{1, 2, 3}; + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + expected.begin(), expected.end()); +} + +BOOST_AUTO_TEST_CASE(test_non_common_range) { + auto result = std::views::iota(1) + | utils::views::unique + | std::views::take(3) + | std::ranges::to(); + + std::vector expected{1, 2, 3}; + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + expected.begin(), expected.end()); +} + +BOOST_AUTO_TEST_CASE(test_composition_with_other_views) { + std::vector input{1, 1, 2, 2, 3, 3, 4, 4, 5, 5}; + auto result = input + | utils::views::unique + | std::views::take(3) + | std::ranges::to(); + + std::vector expected{1, 2, 3}; + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + expected.begin(), expected.end()); +} + +BOOST_AUTO_TEST_CASE(test_function_call_syntax) { + std::vector input{1, 1, 2, 2, 3}; + auto result = utils::views::unique(input) | std::ranges::to(); + + std::vector expected{1, 2, 3}; + + BOOST_CHECK_EQUAL_COLLECTIONS(result.begin(), result.end(), + expected.begin(), expected.end()); +} diff --git a/utils/unique_view.hh b/utils/unique_view.hh new file mode 100644 index 000000000000..0397edebce76 --- /dev/null +++ b/utils/unique_view.hh @@ -0,0 +1,201 @@ +/* + * Copyright (C) 2025-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#pragma once + +#include +#include +#include +#include + +/** + * @brief A lazy view adapter that yields only the first element from every consecutive + * group of equal elements in the underlying range. + * + * This view adapter provides similar functionality to boost::adaptors::uniqued but + * is compatible with C++20 ranges. + * + * @section implementations Related Implementations + * + * @subsection this_impl This implementation (unique_view) + * - Creates a non-modifying view over the source range + * - Lazily filters consecutive duplicates + * - Compatible with C++20 ranges (satisfies view/range concepts) + * - Can be composed with other range adaptors + * + * Example: + * @code + * range | unique_view{} | std::views::take(n) + * @endcode + * + * @subsection boost_uniqued boost::adaptors::uniqued + * - Functionally identical to this implementation + * - Not compatible with C++20 ranges (doesn't satisfy required concepts) + * + * Example: + * @code + * range | boost::adaptors::uniqued + * @endcode + * + * @subsection ranges_unique std::ranges::unique (C++23) + * - Eager algorithm that modifies the source range in-place + * - Returns iterator to new end after removing duplicates + * - Cannot be used as a view or composed with other adaptors + * + * Example: + * @code + * auto r = std::ranges::unique(range); // range is modified + * @endcode + * + * @subsection std_unique std::unique (pre-C++20) + * - Like std::ranges::unique but with iterator-based interface + * - Modifies source range in-place + * + * Example: + * @code + * auto it = std::unique(range.begin(), range.end()); + * @endcode + * + * @subsection boost_range_unique boost::range::unique + * - Range-based wrapper around std::unique + * - Modifies source range in-place + * + * @section future Future Standardization + * std::views::unique is proposed for C++26 in P2214 and will provide + * standardized lazy unique view functionality with expected API: + * @code + * range | std::views::unique + * @endcode + * + * @section why Why This Implementation + * 1. std::ranges::unique/std::unique modify the source range, whereas we need + * a non-destructive view + * 2. boost::adaptors::uniqued is incompatible with C++20 ranges + * 3. std::views::unique isn't standardized yet (targeting C++26) + * + * @section compat API Compatibility + * - Provides pipe operator (|) for consistency with range adaptor patterns + * - Can be used as drop-in replacement for boost::adaptors::uniqued in most cases + * - Satisfies C++20 view/range concepts for compatibility with std::ranges + * + * @section usage Usage Example + * @code + * std::vector v{1, 1, 2, 2, 3, 3}; + * auto unique = v | unique_view{}; // yields: 1, 2, 3 + * // v is unchanged, unique is a view + * @endcode + */ +namespace utils { + +template +requires std::ranges::view && std::equality_comparable> +class unique_view : public std::ranges::view_interface> { + V _base = V(); + + class iterator; + class sentinel { + std::ranges::sentinel_t _end = std::ranges::sentinel_t(); + + public: + sentinel() = default; + + constexpr explicit sentinel(unique_view* parent) + : _end(std::ranges::end(parent->_base)) {} + + friend constexpr bool operator==(const iterator& it, const sentinel& sent) { + return it._current == sent._end; + } + }; + + class iterator { + friend class sentinel; + using base_iterator = std::ranges::iterator_t; + base_iterator _current = base_iterator(); + base_iterator _next = base_iterator(); + std::ranges::sentinel_t _end = std::ranges::sentinel_t(); + public: + using iterator_concept = std::forward_iterator_tag; + using iterator_category = std::forward_iterator_tag; + using value_type = std::ranges::range_value_t; + using difference_type = std::ranges::range_difference_t; + + iterator() requires std::default_initializable = default; + + constexpr iterator(base_iterator current, std::ranges::sentinel_t end) + : _current(current) + , _next(current) + , _end(end) { + if (_current != _end) { + _next = std::next(_current); + skip_duplicates(); + } + } + + constexpr const std::ranges::range_reference_t operator*() const { + return *_current; + } + + constexpr iterator& operator++() { + _current = _next; + if (_current != _end) { + _next = std::next(_current); + skip_duplicates(); + } + return *this; + } + + constexpr iterator operator++(int) { + auto tmp = *this; + ++*this; + return tmp; + } + + friend constexpr bool operator==(const iterator& x, const iterator& y) + requires std::equality_comparable { + return x._current == y._current; + } + + private: + constexpr void skip_duplicates() { + while (_next != _end && *_current == *_next) { + ++_next; + } + } + }; + +public: + unique_view() requires std::default_initializable = default; + + constexpr explicit unique_view(V base) + : _base(std::move(base)) {} + + constexpr auto begin() { + return iterator{std::ranges::begin(_base), std::ranges::end(_base)}; + } + + constexpr auto end() { + return sentinel{this}; + } +}; + +template +unique_view(R&&) -> unique_view>; + +namespace detail { +struct unique_closure : std::ranges::range_adaptor_closure { + template + constexpr auto operator()(R&& r) const { + return unique_view(std::forward(r)); + } +}; +} +namespace views { +inline constexpr detail::unique_closure unique{}; +} + +} // namespace utils From ccb7b4e606390946e0f5c3a2c8182c81b26a49b1 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Mon, 20 Jan 2025 13:13:05 +0800 Subject: [PATCH 394/397] cql3, test: switch from boost::adaptors::uniqued to utils::views:unique In order to reduce the dependency on external libraries, and for better integration with ranges in C++ standard library. let's use the homebrew `utils::views::unique()` before unique is accepted by the C++ standard. Signed-off-by: Kefu Chai --- cql3/statements/batch_statement.cc | 16 +++++++--------- test/boost/cql_functions_test.cc | 4 ++-- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/cql3/statements/batch_statement.cc b/cql3/statements/batch_statement.cc index eb43f0486523..270de52a5465 100644 --- a/cql3/statements/batch_statement.cc +++ b/cql3/statements/batch_statement.cc @@ -18,9 +18,7 @@ #include "cql3/query_processor.hh" #include "service/storage_proxy.hh" #include "tracing/trace_state.hh" - -#include -#include +#include "utils/unique_view.hh" template using coordinator_result = exceptions::coordinator_result; @@ -128,12 +126,12 @@ void batch_statement::validate() if (_has_conditions && !_statements.empty() - && (boost::distance(_statements - | boost::adaptors::transformed([] (auto&& s) { return s.statement->keyspace(); }) - | boost::adaptors::uniqued) != 1 - || (boost::distance(_statements - | boost::adaptors::transformed([] (auto&& s) { return s.statement->column_family(); }) - | boost::adaptors::uniqued) != 1))) { + && (std::ranges::distance(_statements + | std::views::transform([] (auto&& s) { return s.statement->keyspace(); }) + | utils::views::unique) != 1 + || (std::ranges::distance(_statements + | std::views::transform([] (auto&& s) { return s.statement->column_family(); }) + | utils::views::unique) != 1))) { throw exceptions::invalid_request_exception("BATCH with conditions cannot span multiple tables"); } std::optional raw_counter; diff --git a/test/boost/cql_functions_test.cc b/test/boost/cql_functions_test.cc index def66d186b76..67215c7cbc8a 100644 --- a/test/boost/cql_functions_test.cc +++ b/test/boost/cql_functions_test.cc @@ -10,7 +10,6 @@ #include #include -#include #include #include @@ -26,6 +25,7 @@ #include "transport/messages/result_message.hh" #include "utils/assert.hh" #include "utils/big_decimal.hh" +#include "utils/unique_view.hh" #include "types/map.hh" #include "types/list.hh" #include "types/set.hh" @@ -75,7 +75,7 @@ SEASTAR_TEST_CASE(test_functions) { msg->accept(v); // No boost::adaptors::sorted std::ranges::sort(v.res); - BOOST_REQUIRE_EQUAL(boost::distance(v.res | boost::adaptors::uniqued), 3); + BOOST_REQUIRE_EQUAL(std::ranges::distance(v.res | utils::views::unique), 3); }).then([&] { return e.execute_cql("select sum(c1), count(c1) from cf where p1 = 'key1';"); }).then([] (shared_ptr msg) { From 38caabe3efc35ca1acf16a40311477bca00e0992 Mon Sep 17 00:00:00 2001 From: Sergey Zolotukhin Date: Mon, 20 Jan 2025 16:19:02 +0100 Subject: [PATCH 395/397] test: Fix inconsistent naming of the log files. The log file names created in `scylla_cluster.py` by `ScyllaClusterManager` and files to be collected in conftest.py by `manager` should be in sync. This patch fixes the issue, originally introduced in scylladb/scylladb#22192 Fixes scylladb/scylladb#22387 Backports: 6.1 and 6.2. Closes scylladb/scylladb#22415 --- test.py | 2 +- test/topology/conftest.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/test.py b/test.py index 223a85109126..441ed80c4142 100755 --- a/test.py +++ b/test.py @@ -1388,7 +1388,7 @@ async def run(self, options: argparse.Namespace) -> Test: self._prepare_pytest_params(options) test_path = os.path.join(self.suite.options.tmpdir, self.mode) - async with get_cluster_manager(self.mode + '/' + self.uname, self.suite.clusters, test_path) as manager: + async with get_cluster_manager(self.uname, self.suite.clusters, test_path) as manager: self.args.insert(0, "--tmpdir={}".format(options.tmpdir)) self.args.insert(0, "--manager-api={}".format(manager.sock_path)) if options.artifacts_dir_url: diff --git a/test/topology/conftest.py b/test/topology/conftest.py index b81a2e1fd0f8..97b69e817956 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -186,7 +186,7 @@ async def manager(request, manager_internal, record_property, build_mode): ) test_log = suite_testpy_log.parent / f"{suite_testpy_log.stem}.{test_case_name}.log" # this should be consistent with scylla_cluster.py handler name in _before_test method - test_py_log_test = suite_testpy_log.parent / f"{test_case_name}.log" + test_py_log_test = suite_testpy_log.parent / f"{suite_testpy_log.stem}_{test_case_name}_cluster.log" manager_client = manager_internal() # set up client object in fixture with scope function await manager_client.before_test(test_case_name, test_log) From 23284f038f19afd31c21eee2195deea3856eb3f7 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Mon, 20 Jan 2025 14:07:23 +0200 Subject: [PATCH 396/397] table: flush: synchronize with stop() When the table is stopped, all compaction groups are stopped, and as part of that, they are flushing their memtables. To synchronize with stop-induced flush operation, move _pending_flushes_phaser.stop() later in table::stop(), after all compaction groups are flushed and stopped. This way, in table::flush, if we see that the phaser is already closed, we know that there is nothing to flush, otherwise we start a flush operation that would be waited on by a parallel table::stop(). Fixes #22243 Signed-off-by: Benny Halevy Closes scylladb/scylladb#22339 --- replica/table.cc | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/replica/table.cc b/replica/table.cc index d6f378e506a2..c3a0adddbe19 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -1691,9 +1691,14 @@ table::stop() { co_await when_all( _pending_reads_phaser.close(), _pending_writes_phaser.close(), - _pending_streams_phaser.close(), - _pending_flushes_phaser.close()); - co_await _sg_manager->stop_storage_groups(); + _pending_streams_phaser.close()); + // Allow parallel flushes from the commitlog path + // to synchronize with table::stop + { + auto op = _pending_flushes_phaser.start(); + co_await _sg_manager->stop_storage_groups(); + } + co_await _pending_flushes_phaser.close(); co_await _sstable_deletion_gate.close(); co_await std::move(gate_closed_fut); co_await get_row_cache().invalidate(row_cache::external_updater([this] { @@ -3057,6 +3062,10 @@ future<> table::flush(std::optional pos) { if (pos && *pos < _flush_rp) { co_return; } + // There is nothing to flush if the table was stopped. + if (_pending_flushes_phaser.is_closed()) { + co_return; + } auto op = _pending_flushes_phaser.start(); auto fp = _highest_rp; co_await parallel_foreach_compaction_group(std::mem_fn(&compaction_group::flush)); From 59d3a66d1830f4ec4e1a7c09d48f475c92da98ee Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Wed, 22 Jan 2025 09:42:20 +0200 Subject: [PATCH 397/397] Revert "Introduce file stream for tablet" This reverts commit 8208688178d737a7bacc72f499808419577f7b90. It was contributed from enterprise, but is too different from the original for me to merge back. --- configure.py | 3 - db/config.cc | 1 - db/config.hh | 1 - docs/architecture/tablets.rst | 19 - gms/feature_service.hh | 1 - idl/CMakeLists.txt | 1 - idl/sstables.idl.hh | 19 - idl/streaming.idl.hh | 59 -- idl/uuid.idl.hh | 1 - message/messaging_service.cc | 34 - message/messaging_service.hh | 11 - service/storage_service.cc | 70 +- sstables/sstables.cc | 217 ------ sstables/sstables.hh | 51 -- sstables/types.hh | 3 - streaming/CMakeLists.txt | 1 - streaming/stream_blob.cc | 705 ------------------ streaming/stream_blob.hh | 192 ----- streaming/stream_session.cc | 12 +- test/boost/CMakeLists.txt | 2 - test/boost/file_stream_test.cc | 260 ------- test/topology_custom/test_tablets2.py | 1 - .../topology_custom/test_tablets_migration.py | 101 +-- .../test_topology_ops_encrypted.py | 85 --- 24 files changed, 3 insertions(+), 1847 deletions(-) delete mode 100644 idl/sstables.idl.hh delete mode 100644 streaming/stream_blob.cc delete mode 100644 streaming/stream_blob.hh delete mode 100644 test/boost/file_stream_test.cc delete mode 100644 test/topology_custom/test_topology_ops_encrypted.py diff --git a/configure.py b/configure.py index ad3f3da5a0d3..0c065856e908 100755 --- a/configure.py +++ b/configure.py @@ -522,7 +522,6 @@ def find_ninja(): 'test/boost/mutation_reader_another_test', 'test/boost/mutation_test', 'test/boost/mvcc_test', - 'test/boost/file_stream_test', 'test/boost/nonwrapping_interval_test', 'test/boost/observable_test', 'test/boost/partitioner_test', @@ -1083,7 +1082,6 @@ def find_ninja(): 'streaming/stream_request.cc', 'streaming/stream_summary.cc', 'streaming/stream_transfer_task.cc', - 'streaming/stream_blob.cc', 'streaming/stream_receive_task.cc', 'streaming/stream_plan.cc', 'streaming/progress_info.cc', @@ -1315,7 +1313,6 @@ def find_ninja(): 'idl/group0.idl.hh', 'idl/hinted_handoff.idl.hh', 'idl/storage_proxy.idl.hh', - 'idl/sstables.idl.hh', 'idl/group0_state_machine.idl.hh', 'idl/mapreduce_request.idl.hh', 'idl/replica_exception.idl.hh', diff --git a/db/config.cc b/db/config.cc index 65eca086267c..42dce007f3a7 100644 --- a/db/config.cc +++ b/db/config.cc @@ -760,7 +760,6 @@ db::config::config(std::shared_ptr exts) "Throttles streaming I/O to the specified total throughput (in MiBs/s) across the entire system. Streaming I/O includes the one performed by repair and both RBNO and legacy topology operations such as adding or removing a node. Setting the value to 0 disables stream throttling.") , stream_plan_ranges_fraction(this, "stream_plan_ranges_fraction", liveness::LiveUpdate, value_status::Used, 0.1, "Specify the fraction of ranges to stream in a single stream plan. Value is between 0 and 1.") - , enable_file_stream(this, "enable_file_stream", liveness::LiveUpdate, value_status::Used, true, "Set true to use file based stream for tablet instead of mutation based stream") , trickle_fsync(this, "trickle_fsync", value_status::Unused, false, "When doing sequential writing, enabling this option tells fsync to force the operating system to flush the dirty buffers at a set interval trickle_fsync_interval_in_kb. Enable this parameter to avoid sudden dirty buffer flushing from impacting read latencies. Recommended to use on SSDs, but not on HDDs.") , trickle_fsync_interval_in_kb(this, "trickle_fsync_interval_in_kb", value_status::Unused, 10240, diff --git a/db/config.hh b/db/config.hh index 52f2f934e772..a0e8a486e12f 100644 --- a/db/config.hh +++ b/db/config.hh @@ -243,7 +243,6 @@ public: named_value inter_dc_stream_throughput_outbound_megabits_per_sec; named_value stream_io_throughput_mb_per_sec; named_value stream_plan_ranges_fraction; - named_value enable_file_stream; named_value trickle_fsync; named_value trickle_fsync_interval_in_kb; named_value auto_bootstrap; diff --git a/docs/architecture/tablets.rst b/docs/architecture/tablets.rst index 605b27bf16ec..0176adb7e82b 100644 --- a/docs/architecture/tablets.rst +++ b/docs/architecture/tablets.rst @@ -72,25 +72,6 @@ to a new node. .. image:: images/tablets-load-balancing.png -File-based Streaming -======================== - -:label-tip:`ScyllaDB Enterprise` - -File-based streaming is a ScyllaDB Enterprise-only feature that optimizes -tablet migration. - -In ScyllaDB Open Source, migrating tablets is performed by streaming mutation -fragments, which involves deserializing SSTable files into mutation fragments -and re-serializing them back into SSTables on the other node. -In ScyllaDB Enterprise, migrating tablets is performed by streaming entire -SStables, which does not require (de)serializing or processing mutation fragments. -As a result, less data is streamed over the network, and less CPU is consumed, -especially for data models that contain small cells. - -File-based streaming is used for tablet migration in all -:ref:`keyspaces created with tablets enabled `. - .. _tablets-enable-tablets: Enabling Tablets diff --git a/gms/feature_service.hh b/gms/feature_service.hh index 8b1f7a78a44f..78b801d38901 100644 --- a/gms/feature_service.hh +++ b/gms/feature_service.hh @@ -161,7 +161,6 @@ public: gms::feature in_memory_tables { *this, "IN_MEMORY_TABLES"sv }; gms::feature workload_prioritization { *this, "WORKLOAD_PRIORITIZATION"sv }; gms::feature compression_dicts { *this, "COMPRESSION_DICTS"sv }; - gms::feature file_stream { *this, "FILE_STREAM"sv }; public: const std::unordered_map>& registered_features() const; diff --git a/idl/CMakeLists.txt b/idl/CMakeLists.txt index 7f5c9082ac4f..3caffe6ca012 100644 --- a/idl/CMakeLists.txt +++ b/idl/CMakeLists.txt @@ -51,7 +51,6 @@ set(idl_headers raft_storage.idl.hh group0.idl.hh hinted_handoff.idl.hh - sstables.idl.hh storage_proxy.idl.hh storage_service.idl.hh group0_state_machine.idl.hh diff --git a/idl/sstables.idl.hh b/idl/sstables.idl.hh deleted file mode 100644 index a2e87c7f3fd3..000000000000 --- a/idl/sstables.idl.hh +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Copyright 2024-present ScyllaDB - */ - -/* - * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 - */ - - -namespace sstables { - -enum class sstable_state : uint8_t { - normal, - staging, - quarantine, - upload, -}; - -} diff --git a/idl/streaming.idl.hh b/idl/streaming.idl.hh index 955320e4fba2..61539410d0c4 100644 --- a/idl/streaming.idl.hh +++ b/idl/streaming.idl.hh @@ -9,10 +9,8 @@ #include "idl/range.idl.hh" #include "idl/token.idl.hh" #include "idl/uuid.idl.hh" -#include "idl/sstables.idl.hh" #include "streaming/stream_fwd.hh" -#include "streaming/stream_blob.hh" namespace service { @@ -74,61 +72,4 @@ verb [[with_client_info]] prepare_message (streaming::prepare_message msg, strea verb [[with_client_info]] prepare_done_message (streaming::plan_id plan_id, unsigned dst_cpu_id); verb [[with_client_info]] stream_mutation_done (streaming::plan_id plan_id, dht::token_range_vector ranges, table_id cf_id, unsigned dst_cpu_id); verb [[with_client_info]] complete_message (streaming::plan_id plan_id, unsigned dst_cpu_id, bool failed [[version 2.1.0]]); - -class file_stream_id final { - utils::UUID uuid(); -}; - -enum class stream_blob_cmd : uint8_t { - ok, - error, - data, - end_of_stream, -}; - -enum class file_ops : uint16_t { - stream_sstables, - load_sstables, -}; - -class stream_blob_data { - temporary_buffer buf; -}; - -class stream_blob_cmd_data { - streaming::stream_blob_cmd cmd; - std::optional data; -}; - -class stream_blob_meta { - streaming::file_stream_id ops_id; - table_id table; - sstring filename; - seastar::shard_id dst_shard_id; - streaming::file_ops fops; - service::frozen_topology_guard topo_guard; - std::optional sstable_state; -}; - -class node_and_shard { - locator::host_id node; - seastar::shard_id shard; -}; - -class stream_files_request { - streaming::file_stream_id ops_id; - sstring keyspace_name; - sstring table_name; - table_id table; - dht::token_range range; - std::vector targets; - service::frozen_topology_guard topo_guard; -}; - -class stream_files_response { - size_t stream_bytes; -}; - -verb [[with_client_info, cancellable]] tablet_stream_files (streaming::stream_files_request req) -> streaming::stream_files_response; - } diff --git a/idl/uuid.idl.hh b/idl/uuid.idl.hh index 70a2712535cf..b69946fe04b8 100644 --- a/idl/uuid.idl.hh +++ b/idl/uuid.idl.hh @@ -11,7 +11,6 @@ #include "query_id.hh" #include "locator/host_id.hh" #include "tasks/types.hh" -#include "service/session.hh" namespace utils { class UUID final { diff --git a/message/messaging_service.cc b/message/messaging_service.cc index fe90dea9fdb8..2734449f71ef 100644 --- a/message/messaging_service.cc +++ b/message/messaging_service.cc @@ -41,7 +41,6 @@ #include "repair/repair.hh" #include "streaming/stream_reason.hh" #include "streaming/stream_mutation_fragments_cmd.hh" -#include "streaming/stream_blob.hh" #include "cache_temperature.hh" #include "raft/raft.hh" #include "service/raft/group0_fwd.hh" @@ -1271,39 +1270,6 @@ future<> messaging_service::unregister_stream_mutation_fragments() { return unregister_handler(messaging_verb::STREAM_MUTATION_FRAGMENTS); } -// Wrapper for STREAM_BLOB -rpc::sink messaging_service::make_sink_for_stream_blob(rpc::source& source) { - return source.make_sink(); -} - -future, rpc::source>> -messaging_service::make_sink_and_source_for_stream_blob(streaming::stream_blob_meta meta, locator::host_id id) { - if (is_shutting_down()) { - co_await coroutine::return_exception(rpc::closed_error()); - } - auto rpc_client = get_rpc_client(messaging_verb::STREAM_BLOB, addr_for_host_id(id), id); - auto sink = co_await rpc_client->make_stream_sink(); - std::exception_ptr ex; - try { - auto rpc_handler = rpc()->make_client (streaming::stream_blob_meta, rpc::sink)>(messaging_verb::STREAM_BLOB); - auto source = co_await rpc_handler(*rpc_client, meta, sink); - co_return std::make_tuple(std::move(sink), std::move(source)); - } catch (...) { - ex = std::current_exception(); - } - // Reach here only in case of error - co_await sink.close(); - co_return coroutine::return_exception_ptr(ex); -} - -void messaging_service::register_stream_blob(std::function> (const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source)>&& func) { - register_handler(this, messaging_verb::STREAM_BLOB, std::move(func)); -} - -future<> messaging_service::unregister_stream_blob() { - return unregister_handler(messaging_verb::STREAM_BLOB); -} - template future, rpc::source>> do_make_sink_source(messaging_verb verb, uint32_t repair_meta_id, shard_id dst_shard_id, shared_ptr rpc_client, std::unique_ptr& rpc) { diff --git a/message/messaging_service.hh b/message/messaging_service.hh index ab4182d91239..083caee6eab3 100644 --- a/message/messaging_service.hh +++ b/message/messaging_service.hh @@ -37,10 +37,6 @@ namespace streaming { class prepare_message; enum class stream_mutation_fragments_cmd : uint8_t; - enum class stream_blob_cmd : uint8_t; - class stream_blob_data; - class stream_blob_meta; - class stream_blob_cmd_data; } namespace gms { @@ -415,13 +411,6 @@ public: rpc::sink make_sink_for_stream_mutation_fragments(rpc::source>& source); future, rpc::source>> make_sink_and_source_for_stream_mutation_fragments(table_schema_version schema_id, streaming::plan_id plan_id, table_id cf_id, uint64_t estimated_partitions, streaming::stream_reason reason, service::session_id session, locator::host_id id); - // Wrapper for STREAM_BLOB - // The receiver of STREAM_BLOB sends streaming::stream_blob_cmd_data as status code to the sender to notify any error on the receiver side. - void register_stream_blob(std::function> (const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source)>&& func); - future<> unregister_stream_blob(); - rpc::sink make_sink_for_stream_blob(rpc::source& source); - future, rpc::source>> make_sink_and_source_for_stream_blob(streaming::stream_blob_meta meta, locator::host_id id); - // Wrapper for REPAIR_GET_ROW_DIFF_WITH_RPC_STREAM future, rpc::source>> make_sink_and_source_for_repair_get_row_diff_with_rpc_stream(uint32_t repair_meta_id, shard_id dst_cpu_id, locator::host_id id); rpc::sink make_sink_for_repair_get_row_diff_with_rpc_stream(rpc::source& source); diff --git a/service/storage_service.cc b/service/storage_service.cc index 5c738e304b4f..668bf414c741 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -63,7 +63,6 @@ #include "locator/local_strategy.hh" #include "utils/user_provided_param.hh" #include "version.hh" -#include "streaming/stream_blob.hh" #include "dht/range_streamer.hh" #include #include @@ -94,12 +93,10 @@ #include #include #include -#include "utils/pretty_printers.hh" #include "utils/stall_free.hh" #include "utils/error_injection.hh" #include "locator/util.hh" #include "idl/storage_service.dist.hh" -#include "idl/streaming.dist.hh" #include "service/storage_proxy.hh" #include "service/raft/join_node.hh" #include "idl/join_node.dist.hh" @@ -6085,53 +6082,6 @@ future<> storage_service::stream_tablet(locator::global_tablet_id tablet) { } }); - if (trinfo->transition != locator::tablet_transition_kind::intranode_migration && _feature_service.file_stream && _db.local().get_config().enable_file_stream()) { - co_await utils::get_local_injector().inject("migration_streaming_wait", [] (auto& handler) { - rtlogger.info("migration_streaming_wait: start"); - return handler.wait_for_message(db::timeout_clock::now() + std::chrono::minutes(2)); - }); - - auto dst_node = trinfo->pending_replica->host; - auto dst_shard_id = trinfo->pending_replica->shard; - auto transition = trinfo->transition; - - // Release token_metadata_ptr early so it will no block barriers for other migrations - // Don't access trinfo after this. - tm = {}; - - co_await utils::get_local_injector().inject("stream_sstable_files", [&] (auto& handler) -> future<> { - slogger.info("stream_sstable_files: waiting"); - while (!handler.poll_for_message()) { - co_await sleep_abortable(std::chrono::milliseconds(5), guard.get_abort_source()); - } - slogger.info("stream_sstable_files: released"); - }); - - for (auto src : streaming_info.read_from) { - // Use file stream for tablet to stream data - auto ops_id = streaming::file_stream_id::create_random_id(); - auto start_time = std::chrono::steady_clock::now(); - size_t stream_bytes = 0; - try { - auto& table = _db.local().find_column_family(tablet.table); - slogger.debug("stream_sstables[{}] Streaming for tablet {} of {} started table={}.{} range={} src={}", - ops_id, transition, tablet, table.schema()->ks_name(), table.schema()->cf_name(), range, src); - auto resp = co_await streaming::tablet_stream_files(ops_id, table, range, src.host, dst_node, dst_shard_id, _messaging.local(), _abort_source, topo_guard); - stream_bytes = resp.stream_bytes; - slogger.debug("stream_sstables[{}] Streaming for tablet migration of {} successful", ops_id, tablet); - auto duration = std::chrono::duration(std::chrono::steady_clock::now() - start_time); - auto bw = utils::pretty_printed_throughput(stream_bytes, duration);; - slogger.info("stream_sstables[{}] Streaming for tablet migration of {} finished table={}.{} range={} stream_bytes={} stream_time={} stream_bw={}", - ops_id, tablet, table.schema()->ks_name(), table.schema()->cf_name(), range, stream_bytes, duration, bw); - } catch (...) { - slogger.warn("stream_sstables[{}] Streaming for tablet migration of {} from {} failed: {}", ops_id, tablet, leaving_replica, std::current_exception()); - throw; - } - } - - } else { // Caution: following code is intentionally unindented to be in sync with OSS - - if (trinfo->transition == locator::tablet_transition_kind::intranode_migration) { if (!leaving_replica || leaving_replica->host != tm->get_my_id()) { throw std::runtime_error(fmt::format("Invalid leaving replica for intra-node migration, tablet: {}, leaving: {}", @@ -6183,8 +6133,6 @@ future<> storage_service::stream_tablet(locator::global_tablet_id tablet) { slogger.info("Streaming for tablet migration of {} finished table={}.{} range={}", tablet, table.schema()->ks_name(), table.schema()->cf_name(), range); } - } // Traditional streaming vs file-based streaming. - // If new pending tablet replica needs splitting, streaming waits for it to complete. // That's to provide a guarantee that once migration is over, the coordinator can finalize // splitting under the promise that compaction groups of tablets are all split, ready @@ -7094,21 +7042,6 @@ void storage_service::init_messaging_service() { return handler(ss); }); }; - ser::streaming_rpc_verbs::register_tablet_stream_files(&_messaging.local(), - [this] (const rpc::client_info& cinfo, streaming::stream_files_request req) -> future { - streaming::stream_files_response resp; - resp.stream_bytes = co_await container().map_reduce0([req] (storage_service& ss) -> future { - auto res = co_await streaming::tablet_stream_files_handler(ss._db.local(), ss._messaging.local(), req, [&ss] (locator::host_id host) -> future { - return ss.container().invoke_on(0, [host] (storage_service& ss) { - return ss.host2ip(host); - }); - }); - co_return res.stream_bytes; - }, - size_t(0), - std::plus()); - co_return resp; - }); ser::storage_service_rpc_verbs::register_raft_topology_cmd(&_messaging.local(), [handle_raft_rpc] (raft::server_id dst_id, raft::term_t term, uint64_t cmd_index, raft_topology_cmd cmd) { return handle_raft_rpc(dst_id, [cmd = std::move(cmd), term, cmd_index] (auto& ss) { return ss.raft_topology_cmd_handler(term, cmd_index, cmd); @@ -7244,8 +7177,7 @@ future<> storage_service::uninit_messaging_service() { return when_all_succeed( ser::node_ops_rpc_verbs::unregister(&_messaging.local()), ser::storage_service_rpc_verbs::unregister(&_messaging.local()), - ser::join_node_rpc_verbs::unregister(&_messaging.local()), - ser::streaming_rpc_verbs::unregister_tablet_stream_files(&_messaging.local()) + ser::join_node_rpc_verbs::unregister(&_messaging.local()) ).discard_result(); } diff --git a/sstables/sstables.cc b/sstables/sstables.cc index 083aa0a3331c..c5e9f434031a 100644 --- a/sstables/sstables.cc +++ b/sstables/sstables.cc @@ -3417,223 +3417,6 @@ generation_type::from_string(const std::string& s) { } } -std::string sstable_stream_source::component_basename() const { - return _sst->component_basename(_type); -} - -sstable_stream_source::sstable_stream_source(shared_sstable sst, component_type type) - : _sst(std::move(sst)) - , _type(type) -{} - -std::vector> create_stream_sources(const sstables::sstable_files_snapshot& snapshot) { - std::vector> result; - result.reserve(snapshot.files.size()); - - class sstable_stream_source_impl : public sstable_stream_source { - file _file; - public: - sstable_stream_source_impl(shared_sstable table, component_type type, file f) - : sstable_stream_source(std::move(table), type) - , _file(std::move(f)) - {} - future> input(const file_input_stream_options& options) const override { - if (_type == component_type::Scylla) { - // Filter out any node-local info (i.e. extensions) - // and reserialize data. Load into a temp object. - // TODO/FIXME. Not all extension attributes might - // need removing. In fact, it might be wrong (in the future) - // to do so. ATM we know this is safe and correct, but really - // extensions should remove themselves if required. - scylla_metadata tmp; - uint64_t size = co_await _file.size(); - auto r = file_random_access_reader(_file, size, default_sstable_buffer_size); - co_await parse(*_sst->get_schema(), _sst->get_version(), r, tmp); - co_await r.close(); - - tmp.remove_extension_attributes(); - - std::vector> bufs; - // TODO: move to seastar. Based on memory_data_sink, but allowing us - // to actually move away the buffers later. I don't want to modify - // util classes in an enterprise patch. - class buffer_data_sink_impl : public data_sink_impl { - std::vector>& _bufs; - public: - buffer_data_sink_impl(std::vector>& bufs) - : _bufs(bufs) - {} - future<> put(net::packet data) override { - throw std::logic_error("unsupported operation"); - } - future<> put(temporary_buffer buf) override { - _bufs.emplace_back(std::move(buf)); - return make_ready_future<>(); - } - future<> flush() override { - return make_ready_future<>(); - } - future<> close() override { - return make_ready_future<>(); - } - size_t buffer_size() const noexcept override { - return 128*1024; - } - }; - - co_await seastar::async([&] { - file_writer fw(data_sink(std::make_unique(bufs))); - write(_sst->get_version(), fw, tmp); - fw.close(); - }); - // TODO: move to seastar. Based on buffer_input... in utils, but - // handles potential 1+ buffers - class buffer_data_source_impl : public data_source_impl { - private: - std::vector> _bufs; - size_t _index = 0; - public: - buffer_data_source_impl(std::vector>&& bufs) - : _bufs(std::move(bufs)) - {} - buffer_data_source_impl(buffer_data_source_impl&&) noexcept = default; - buffer_data_source_impl& operator=(buffer_data_source_impl&&) noexcept = default; - - future> get() override { - if (_index < _bufs.size()) { - return make_ready_future>(std::move(_bufs.at(_index++))); - } - return make_ready_future>(); - } - future> skip(uint64_t n) override { - while (n > 0 && _index < _bufs.size()) { - auto& buf = _bufs.at(_index); - auto min = std::min(n, buf.size()); - buf.trim_front(min); - if (buf.empty()) { - ++_index; - } - n -= min; - } - return get(); - } - }; - co_return input_stream(data_source(std::make_unique(std::move(bufs)))); - } - co_return make_file_input_stream(_file, options); - } - }; - - auto& files = snapshot.files; - - auto add = [&](component_type type, file f) { - result.emplace_back(std::make_unique(snapshot.sst, type, std::move(f))); - }; - - try { - add(component_type::TOC, files.at(component_type::TOC)); - add(component_type::Scylla, files.at(component_type::Scylla)); - } catch (std::out_of_range&) { - std::throw_with_nested(std::invalid_argument("Missing required sstable component")); - } - for (auto&& [type, f] : files) { - if (type != component_type::TOC && type != component_type::Scylla) { - add(type, std::move(f)); - } - } - - return result; -} - -class sstable_stream_sink_impl : public sstable_stream_sink { - shared_sstable _sst; - component_type _type; - bool _last_component; -public: - sstable_stream_sink_impl(shared_sstable sst, component_type type, bool last_component) - : _sst(std::move(sst)) - , _type(type) - , _last_component(last_component) - {} -private: - future<> load_metadata() const { - auto metafile = _sst->filename(sstables::component_type::Scylla); - if (!co_await file_exists(metafile)) { - // for compatibility with streaming a non-scylla table (no scylla component) - co_return; - } - if (!_sst->get_shared_components().scylla_metadata) { - sstables::scylla_metadata tmp; - co_await _sst->read_simple(tmp); - _sst->get_shared_components().scylla_metadata = std::move(tmp); - } - } - future<> save_metadata() const { - if (!_sst->get_shared_components().scylla_metadata) { - co_return; - } - file_output_stream_options options; - options.buffer_size = default_sstable_buffer_size; - co_await seastar::async([&] { - auto w = _sst->make_component_file_writer(component_type::Scylla, std::move(options), open_flags::wo | open_flags::create).get(); - write(_sst->get_version(), w, *_sst->get_shared_components().scylla_metadata); - w.close(); - }); - } -public: - future> output(const file_open_options& foptions, const file_output_stream_options& stream_options) override { - assert(_type != component_type::TOC); - // TOC and scylla components are guaranteed not to depend on metadata. Ignore these (chicken, egg) - bool load_save_meta = _type != component_type::TemporaryTOC && _type != component_type::Scylla; - - // otherwise, first load scylla metadata from disk as written so far. - if (load_save_meta) { - co_await load_metadata(); - } - // now we can open the component file. any extensions applied should write info into metadata - auto f = co_await _sst->open_file(_type, open_flags::wo | open_flags::create, foptions); - - // Save back to disk. - if (load_save_meta) { - co_await save_metadata(); - } - - co_return co_await make_file_output_stream(std::move(f), stream_options); - } - future close_and_seal() override { - if (_last_component) { - // If we are the last component in a sequence, we can seal the table. - co_await _sst->_storage->seal(*_sst); - co_return std::move(_sst); - } - _sst = {}; - co_return nullptr; - } - future<> abort() override { - if (!_sst) { - co_return; - } - auto filename = fs::path(_sst->_storage->prefix()) / std::string_view(_sst->component_basename(_type)); - // TODO: if we are the last component (or really always), should we remove all component files? - // For now, this remains the responsibility of calling code (see handle_tablet_migration etc) - co_await remove_file(filename.native()); - } -}; - -std::unique_ptr create_stream_sink(schema_ptr schema, sstables_manager& sstm, const data_dictionary::storage_options& s_opts, sstable_state state, std::string_view component_filename, bool last_component) { - auto desc = parse_path(component_filename, schema->ks_name(), schema->cf_name()); - auto sst = sstm.make_sstable(schema, s_opts, desc.generation, state, desc.version, desc.format); - - auto type = desc.component; - // Don't write actual TOC. Write temp, if successful, storage::seal will rename this to actual - // TOC (see above close_and_seal). - if (type == component_type::TOC) { - type = component_type::TemporaryTOC; - } - - return std::make_unique(std::move(sst), type, last_component); -} - } // namespace sstables namespace seastar { diff --git a/sstables/sstables.hh b/sstables/sstables.hh index 21301578c75d..008888eff31e 100644 --- a/sstables/sstables.hh +++ b/sstables/sstables.hh @@ -182,8 +182,6 @@ public: future<> commit(); }; -class sstable_stream_sink_impl; - class sstable : public enable_lw_shared_from_this { friend ::sstable_assertions; public: @@ -260,10 +258,6 @@ public: // It's up to the storage driver how to implement this. future<> change_state(sstable_state to, delayed_commit_changes* delay = nullptr); - sstable_state state() const { - return _state; - } - // Filesystem-specific call to grab an sstable from upload dir and // put it into the desired destination assigning the given generation future<> pick_up_from_upload(sstable_state to, generation_type new_generation); @@ -525,7 +519,6 @@ private: return filename(dir, _schema->ks_name(), _schema->cf_name(), _version, _generation, _format, f); } - friend class sstable_stream_sink_impl; friend class sstable_directory; friend class filesystem_storage; friend class s3_storage; @@ -1142,50 +1135,6 @@ struct sstable_files_snapshot { std::unordered_map files; }; -// A sstable_stream_source gives back -// component input streams suitable for streaming to other nodes, -// in appropriate order. Data will be decrypted and sanitized as required. -class sstable_stream_source { -protected: - shared_sstable _sst; - component_type _type; -public: - sstable_stream_source(shared_sstable, component_type); - virtual ~sstable_stream_source() = default; - - // Input stream for data appropriate for stream transfer for this component - virtual future> input(const file_input_stream_options&) const = 0; - - // source sstable - const sstable& source() const { - return *_sst; - } - // component - component_type type() const { - return _type; - } - std::string component_basename() const; -}; - -// Translates the result of gathering readable snapshot files into ordered items for streaming. -std::vector> create_stream_sources(const sstables::sstable_files_snapshot&); - -class sstable_stream_sink { -public: - virtual ~sstable_stream_sink() = default; - // Stream to the component file - virtual future> output(const file_open_options&, const file_output_stream_options&) = 0; - // closes this component. If this is the last component in a set (see "last_component" in creating method below) - // the table on disk will be sealed. - // Returns sealed sstable if last, or nullptr otherwise. - virtual future close_and_seal() = 0; - virtual future<> abort() = 0; -}; - -// Creates a sink object which can receive a component file sourced from above source object data. - -std::unique_ptr create_stream_sink(schema_ptr, sstables_manager&, const data_dictionary::storage_options&, sstable_state, std::string_view component_filename, bool last_component); - } // namespace sstables template <> struct fmt::formatter : fmt::formatter { diff --git a/sstables/types.hh b/sstables/types.hh index 4e0930b8cfd3..51dd55868449 100644 --- a/sstables/types.hh +++ b/sstables/types.hh @@ -626,9 +626,6 @@ struct scylla_metadata { const extension_attributes* get_extension_attributes() const { return data.get(); } - void remove_extension_attributes() { - data.data.erase(scylla_metadata_type::ExtensionAttributes); - } extension_attributes& get_or_create_extension_attributes() { auto* ext = data.get(); if (ext == nullptr) { diff --git a/streaming/CMakeLists.txt b/streaming/CMakeLists.txt index 7456f6ed4078..5a6d3da49243 100644 --- a/streaming/CMakeLists.txt +++ b/streaming/CMakeLists.txt @@ -14,7 +14,6 @@ target_sources(streaming stream_session_state.cc stream_summary.cc stream_task.cc - stream_blob.cc stream_transfer_task.cc) target_include_directories(streaming PUBLIC diff --git a/streaming/stream_blob.cc b/streaming/stream_blob.cc deleted file mode 100644 index 4f3c3d9a6d0f..000000000000 --- a/streaming/stream_blob.cc +++ /dev/null @@ -1,705 +0,0 @@ -/* - * Copyright (C) 2023-present ScyllaDB - */ - -/* - * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 - */ - -#include "message/messaging_service.hh" -#include "streaming/stream_blob.hh" -#include "streaming/stream_plan.hh" -#include "gms/inet_address.hh" -#include "utils/pretty_printers.hh" -#include "utils/error_injection.hh" -#include "locator/host_id.hh" -#include "replica/database.hh" -#include "sstables/sstables.hh" -#include "sstables/sstables_manager.hh" -#include "sstables/sstable_version.hh" -#include "sstables/generation_type.hh" -#include "sstables/types.hh" -#include "idl/streaming.dist.hh" -#include "service/topology_guard.hh" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace streaming { - -static logging::logger blogger("stream_blob"); - -constexpr size_t file_stream_buffer_size = 128 * 1024; -constexpr size_t file_stream_write_behind = 10; -constexpr size_t file_stream_read_ahead = 4; - -static sstables::sstable_state sstable_state(const streaming::stream_blob_meta& meta) { - return meta.sstable_state.value_or(sstables::sstable_state::normal); -} - -static future<> load_sstable_for_tablet(const file_stream_id& ops_id, replica::database& db, table_id table, sstables::sstable_state state, sstring filename, seastar::shard_id shard) { - blogger.debug("stream_sstables[{}] Loading sstable {} on shard {}", ops_id, filename, shard); - auto s = db.find_column_family(table).schema(); - auto data_path = std::filesystem::path(filename); - auto desc = sstables::parse_path(data_path, s->ks_name(), s->cf_name()); - co_await db.container().invoke_on(shard, [id = s->id(), desc, state] (replica::database& db) -> future<> { - replica::table& t = db.find_column_family(id); - auto erm = t.get_effective_replication_map(); - auto& sstm = t.get_sstables_manager(); - auto sst = sstm.make_sstable(t.schema(), t.get_storage_options(), desc.generation, state, desc.version, desc.format); - co_await sst->load(erm->get_sharder(*t.schema())); - co_await t.add_sstable_and_update_cache(sst); - }); - blogger.info("stream_sstables[{}] Loaded sstable {} on shard {} successfully", ops_id, filename, shard); -} - -static utils::pretty_printed_throughput get_bw(size_t total_size, std::chrono::steady_clock::time_point start_time) { - auto duration = std::chrono::steady_clock::now() - start_time; - return utils::pretty_printed_throughput(total_size, duration); -} - -// For tablet stream checks -class tablet_stream_status { -public: - bool finished = false; - void check_valid_stream(); -}; - -void tablet_stream_status::check_valid_stream() { - if (finished) { - throw std::runtime_error("The stream has finished already"); - } -} - -static thread_local std::unordered_map> tablet_streams; - -future<> mark_tablet_stream_start(file_stream_id ops_id) { - return seastar::smp::invoke_on_all([ops_id] { - auto status = make_lw_shared(); - tablet_streams.emplace(ops_id, status); - }); -} - -future<> mark_tablet_stream_done(file_stream_id ops_id) { - return seastar::smp::invoke_on_all([ops_id] { - auto it = tablet_streams.find(ops_id); - if (it == tablet_streams.end()) { - return; - } - auto status = it->second; - if (status) { - status->finished = true; - } - tablet_streams.erase(ops_id); - }); -} - -lw_shared_ptr get_tablet_stream(file_stream_id ops_id) { - auto status = tablet_streams[ops_id]; - if (!status) { - auto msg = format("stream_sstables[{}] Could not find ops_id={}", ops_id, ops_id); - blogger.warn("{}", msg); - throw std::runtime_error(msg); - } - return status; -} - -static void may_inject_error(const streaming::stream_blob_meta& meta, bool may_inject, const sstring& error) { - if (may_inject) { - if (rand() % 500 == 0) { - auto msg = format("fstream[{}] Injected file stream error={} file={}", - meta.ops_id, error, meta.filename); - blogger.warn("{}", msg); - throw std::runtime_error(msg); - } - } -} - -future<> stream_blob_handler(replica::database& db, - netw::messaging_service& ms, - gms::inet_address from, - streaming::stream_blob_meta meta, - rpc::sink sink, - rpc::source source, - stream_blob_create_output_fn create_output, - bool inject_errors) { - bool fstream_closed = false; - bool sink_closed = false; - bool status_sent = false; - size_t total_size = 0; - auto start_time = std::chrono::steady_clock::now(); - std::optional> fstream; - std::exception_ptr error; - stream_blob_finish_fn finish; - - // Will log a message when streaming is done. Used to synchronize tests. - lw_shared_ptr log_done; - if (utils::get_local_injector().is_enabled("stream_mutation_fragments")) { - log_done = make_lw_shared(seastar::make_shared(seastar::defer([] { - blogger.info("stream_mutation_fragments: done (tablets)"); - }))); - } - - try { - auto status = get_tablet_stream(meta.ops_id); - auto guard = service::topology_guard(meta.topo_guard); - - // Reject any file_ops that is not support by this node - if (meta.fops != streaming::file_ops::stream_sstables && - meta.fops != streaming::file_ops::load_sstables) { - auto msg = format("fstream[{}] Unsupported file_ops={} peer={} file={}", - meta.ops_id, int(meta.fops), from, meta.filename); - blogger.warn("{}", msg); - throw std::runtime_error(msg); - } - - blogger.debug("fstream[{}] Follower started peer={} file={}", - meta.ops_id, from, meta.filename); - - auto [f, out] = co_await create_output(db, meta); - finish = std::move(f); - fstream = std::move(out); - - bool got_end_of_stream = false; - for (;;) { - try { - auto opt = co_await source(); - if (!opt) { - break; - } - - co_await utils::get_local_injector().inject("stream_mutation_fragments", [&guard] (auto& handler) -> future<> { - blogger.info("stream_mutation_fragments: waiting (tablets)"); - while (!handler.poll_for_message()) { - guard.check(); - co_await sleep(std::chrono::milliseconds(5)); - } - blogger.info("stream_mutation_fragments: released (tablets)"); - }); - - stream_blob_cmd_data& cmd_data = std::get<0>(*opt); - auto cmd = cmd_data.cmd; - if (cmd == streaming::stream_blob_cmd::error) { - blogger.warn("fstream[{}] Follower got stream_blob_cmd::error from peer={} file={}", - meta.ops_id, from, meta.filename); - throw std::runtime_error(format("Got stream_blob_cmd::error from peer={} file={}", from, meta.filename)); - } else if (cmd == streaming::stream_blob_cmd::end_of_stream) { - blogger.debug("fstream[{}] Follower got stream_blob_cmd::end_of_stream from peer={} file={}", - meta.ops_id, from, meta.filename); - got_end_of_stream = true; - } else if (cmd == streaming::stream_blob_cmd::data) { - std::optional data = std::move(cmd_data.data); - if (data) { - total_size += data->size(); - blogger.trace("fstream[{}] Follower received data from peer={} data={}", meta.ops_id, from, data->size()); - status->check_valid_stream(); - if (!data->empty()) { - co_await fstream->write((char*)data->data(), data->size()); - } - } - } - } catch (seastar::rpc::stream_closed) { - // After we get streaming::stream_blob_cmd::end_of_stream which - // is the last message from peer, it does not matter if the - // source() is closed or not. - if (got_end_of_stream) { - break; - } else { - throw; - } - } catch (...) { - throw; - } - may_inject_error(meta, inject_errors, "rx_data"); - } - - // If we reach here, streaming::stream_blob_cmd::end_of_stream should be received. Otherwise there - // must be an error, e.g., the sender closed the stream without sending streaming::stream_blob_cmd::error. - if (!got_end_of_stream) { - throw std::runtime_error(format("fstream[{}] Follower failed to get end_of_stream", meta.ops_id)); - } - - status->check_valid_stream(); - co_await fstream->flush(); - co_await fstream->close(); - fstream_closed = true; - - may_inject_error(meta, inject_errors, "flush_and_close"); - - co_await finish(store_result::ok); - - // Send status code and close the sink - co_await sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::ok)); - status_sent = true; - co_await sink.close(); - sink_closed = true; - } catch (...) { - error = std::current_exception(); - } - if (error) { - blogger.warn("fstream[{}] Follower failed peer={} file={} received_size={} bw={} error={}", - meta.ops_id, from, meta.filename, total_size, get_bw(total_size, start_time), error); - if (!fstream_closed) { - try { - if (fstream) { - // Make sure fstream is always closed - co_await fstream->close(); - } - } catch (...) { - blogger.warn("fstream[{}] Follower failed to close the file stream: {}", - meta.ops_id, std::current_exception()); - // We could do nothing but continue to cleanup more - } - } - if (!status_sent) { - try { - may_inject_error(meta, inject_errors, "no_error_code_back"); - co_await sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::error)); - } catch (...) { - // Try our best to send the status code. - // If we could not send it, we could do nothing but close the sink. - blogger.warn("fstream[{}] Follower failed to send error code: {}", - meta.ops_id, std::current_exception()); - } - } - try { - if (!sink_closed) { - // Make sure sink is always closed - co_await sink.close(); - } - } catch (...) { - blogger.warn("fstream[{}] Follower failed to close the stream sink: {}", - meta.ops_id, std::current_exception()); - } - try { - // Drain everything in source - for (;;) { - auto opt = co_await source(); - if (!opt) { - break; - } - } - } catch (...) { - blogger.warn("fstream[{}] Follower failed to drain rpc stream source: {}", - meta.ops_id, std::current_exception()); - } - - try { - // Remove the file in case of error - if (finish) { - co_await finish(store_result::failure); - blogger.info("fstream[{}] Follower removed partial file={}", meta.ops_id, meta.filename); - } - } catch (...) { - blogger.warn("fstream[{}] Follower failed to remove partial file={}: {}", - meta.ops_id, meta.filename, std::current_exception()); - } - - // Do not call rethrow_exception(error) because the caller could do nothing but log - // the error. We have already logged the error here. - } else { - // Get some statistics - blogger.debug("fstream[{}] Follower finished peer={} file={} received_size={} bw={}", - meta.ops_id, from, meta.filename, total_size, get_bw(total_size, start_time)); - } - co_return; -} - - -future<> stream_blob_handler(replica::database& db, netw::messaging_service& ms, - gms::inet_address from, - streaming::stream_blob_meta meta, - rpc::sink sink, - rpc::source source) { - - co_await stream_blob_handler(db, ms, std::move(from), meta, std::move(sink), std::move(source), [](replica::database& db, const streaming::stream_blob_meta& meta) -> future { - auto foptions = file_open_options(); - foptions.sloppy_size = true; - foptions.extent_allocation_size_hint = 32 << 20; - - auto stream_options = file_output_stream_options(); - stream_options.buffer_size = file_stream_buffer_size; - stream_options.write_behind = file_stream_write_behind; - - auto& table = db.find_column_family(meta.table); - auto& sstm = table.get_sstables_manager(); - auto sstable_sink = sstables::create_stream_sink(table.schema(), sstm, table.get_storage_options(), sstable_state(meta), meta.filename, meta.fops == file_ops::load_sstables); - auto out = co_await sstable_sink->output(foptions, stream_options); - co_return output_result{ - [sstable_sink = std::move(sstable_sink), &meta, &db](store_result res) -> future<> { - if (res != store_result::ok) { - co_await sstable_sink->abort(); - co_return; - } - auto sst = co_await sstable_sink->close_and_seal(); - if (sst) { - auto filename = sst->toc_filename(); - sst = {}; - co_await load_sstable_for_tablet(meta.ops_id, db, meta.table, sstable_state(meta), std::move(filename), meta.dst_shard_id); - } - }, - std::move(out) - }; - }); -} - -// Get a new sstable name using the new generation -// For example: -// oldname: me-3ga1_0iiv_2e5uo2flv7lgdl2j0d-big-Index.db -// newgen: 3ga1_0iiv_3vj5c2flv7lgdl2j0d -// newname: me-3ga1_0iiv_3vj5c2flv7lgdl2j0d-big-Index.db -static std::string get_sstable_name_with_generation(const file_stream_id& ops_id, const std::string& oldname, const std::string& newgen) { - std::string newname = oldname; - // The generation name starts after the first '-'. - auto it = newname.find("-"); - if (it != std::string::npos) { - newname.replace(++it, newgen.size(), newgen); - return newname; - } else { - auto msg = fmt::format("fstream[{}] Failed to get sstable name for {} with generation {}", ops_id, oldname, newgen); - blogger.warn("{}", msg); - throw std::runtime_error(msg); - } -} -} - -template<> struct fmt::formatter : fmt::ostream_formatter {}; - -namespace streaming { - -// Send files in the files list to the nodes in targets list over network -// Returns number of bytes sent over network -future -tablet_stream_files(netw::messaging_service& ms, std::list sources, std::vector targets, table_id table, file_stream_id ops_id, host2ip_t host2ip, service::frozen_topology_guard topo_guard, bool inject_errors) { - size_t ops_total_size = 0; - if (targets.empty()) { - co_return ops_total_size; - } - if (sources.empty()) { - co_return ops_total_size; - } - - blogger.debug("fstream[{}] Master started sending n={}, sources={}, targets={}", - ops_id, sources.size(), sources, targets); - - struct sink_and_source { - gms::inet_address node; - rpc::sink sink; - rpc::source source; - bool sink_closed = false; - bool status_sent = false; - }; - - auto ops_start_time = std::chrono::steady_clock::now(); - streaming::stream_blob_meta meta; - meta.ops_id = ops_id; - meta.table = table; - meta.topo_guard = topo_guard; - std::exception_ptr error; - - auto stream_options = file_input_stream_options(); - stream_options.buffer_size = file_stream_buffer_size; - stream_options.read_ahead = file_stream_read_ahead; - - for (auto& info : sources) { - auto& filename = info.filename; - std::optional> fstream; - bool fstream_closed = false; - try { - meta.fops = info.fops; - meta.filename = info.filename; - meta.sstable_state = info.sstable_state; - fstream = co_await info.source(stream_options); - } catch (...) { - blogger.warn("fstream[{}] Master failed sources={} targets={} error={}", - ops_id, sources, targets, std::current_exception()); - throw; - } - - std::vector ss; - size_t total_size = 0; - auto start_time = std::chrono::steady_clock::now(); - bool got_error_from_peer = false; - try { - for (auto& x : targets) { - const auto& node = x.node; - meta.dst_shard_id = x.shard; - auto ip = co_await host2ip(node); - blogger.debug("fstream[{}] Master creating sink and source for node={}/{}, file={}, targets={}", ops_id, node, ip, filename, targets); - auto [sink, source] = co_await ms.make_sink_and_source_for_stream_blob(meta, node); - ss.push_back(sink_and_source{ip, std::move(sink), std::move(source)}); - } - - // This fiber sends data to peer node - auto send_data_to_peer = [&] () mutable -> future<> { - std::exception_ptr error; - try { - while (!got_error_from_peer) { - may_inject_error(meta, inject_errors, "read_data"); - auto buf = co_await fstream->read_up_to(file_stream_buffer_size); - if (buf.size() == 0) { - break; - } - streaming::stream_blob_data data(std::move(buf)); - auto data_size = data.size(); - stream_blob_cmd_data cmd_data(streaming::stream_blob_cmd::data, std::move(data)); - co_await coroutine::parallel_for_each(ss, [&] (sink_and_source& s) mutable -> future<> { - total_size += data_size; - ops_total_size += data_size; - blogger.trace("fstream[{}] Master sending file={} to node={} chunk_size={}", - ops_id, filename, s.node, data_size); - may_inject_error(meta, inject_errors, "tx_data"); - co_await s.sink(cmd_data); - }); - } - } catch (...) { - error = std::current_exception(); - } - if (error) { - // We have to close the stream otherwise if the stream is - // ok, the get_status_code_from_peer fiber below might - // wait for the source() forever. - for (auto& s : ss) { - try { - co_await s.sink.close(); - s.sink_closed = true; - } catch (...) { - } - } - std::rethrow_exception(error); - } - - if (fstream) { - co_await fstream->close(); - fstream_closed = true; - } - - for (auto& s : ss) { - blogger.debug("fstream[{}] Master done sending file={} to node={}", ops_id, filename, s.node); - if (!got_error_from_peer) { - co_await s.sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::end_of_stream)); - s.status_sent = true; - } - co_await s.sink.close(); - s.sink_closed = true; - } - }; - - // This fiber gets status code from peer node - auto get_status_code_from_peer = [&] () mutable -> future<> { - co_await coroutine::parallel_for_each(ss, [&] (sink_and_source& s) mutable -> future<> { - bool got_cmd_ok = false; - while (!got_error_from_peer) { - try { - auto opt = co_await s.source(); - if (opt) { - stream_blob_cmd_data& cmd_data = std::get<0>(*opt); - if (cmd_data.cmd == streaming::stream_blob_cmd::error) { - got_error_from_peer = true; - blogger.warn("fstream[{}] Master got stream_blob_cmd::error file={} peer={}", - ops_id, filename, s.node); - throw std::runtime_error(format("Got stream_blob_cmd::error from peer {}", s.node)); - } if (cmd_data.cmd == streaming::stream_blob_cmd::ok) { - got_cmd_ok = true; - } - blogger.debug("fstream[{}] Master got stream_blob_cmd={} file={} peer={}", - ops_id, int(cmd_data.cmd), filename, s.node); - } else { - break; - } - } catch (seastar::rpc::stream_closed) { - // After we get streaming::stream_blob_cmd::ok - // which is the last message from peer, it does not - // matter if the source() is closed or not. - if (got_cmd_ok) { - break; - } else { - throw; - } - } catch (...) { - throw; - } - } - }); - }; - - co_await coroutine::all(send_data_to_peer, get_status_code_from_peer); - } catch (...) { - error = std::current_exception(); - } - if (error) { - blogger.warn("fstream[{}] Master failed sending file={} to targets={} send_size={} bw={} error={}", - ops_id, filename, targets, total_size, get_bw(total_size, start_time), error); - // Error handling for fstream and sink - if (!fstream_closed) { - try { - if (fstream) { - co_await fstream->close(); - } - } catch (...) { - // We could do nothing but continue to cleanup more - blogger.warn("fstream[{}] Master failed to close file stream: {}", - ops_id, std::current_exception()); - } - } - for (auto& s : ss) { - try { - if (!s.status_sent && !s.sink_closed) { - co_await s.sink(streaming::stream_blob_cmd_data(streaming::stream_blob_cmd::error)); - s.status_sent = true; - } - } catch (...) { - // We could do nothing but continue to close - blogger.warn("fstream[{}] Master failed to send error code: {}", - ops_id, std::current_exception()); - } - try { - if (!s.sink_closed) { - co_await s.sink.close(); - s.sink_closed = true; - } - } catch (...) { - // We could do nothing but continue - blogger.warn("fstream[{}] Master failed to close rpc stream sink: {}", - ops_id, std::current_exception()); - } - - try { - // Drain everything in source - for (;;) { - auto opt = co_await s.source(); - if (!opt) { - break; - } - } - } catch (...) { - blogger.warn("fstream[{}] Master failed to drain rpc stream source: {}", - ops_id, std::current_exception()); - } - } - // Stop handling remaining files - break; - } else { - blogger.debug("fstream[{}] Master done sending file={} to targets={} send_size={} bw={}", - ops_id, filename, targets, total_size, get_bw(total_size, start_time)); - } - } - if (error) { - blogger.warn("fstream[{}] Master failed sending files_nr={} files={} targets={} send_size={} bw={} error={}", - ops_id, sources.size(), sources, targets, ops_total_size, get_bw(ops_total_size, ops_start_time), error); - std::rethrow_exception(error); - } else { - blogger.debug("fstream[{}] Master finished sending files_nr={} files={} targets={} send_size={} bw={}", - ops_id, sources.size(), sources, targets, ops_total_size, get_bw(ops_total_size, ops_start_time)); - } - co_return ops_total_size; -} - - -future tablet_stream_files_handler(replica::database& db, netw::messaging_service& ms, streaming::stream_files_request req, host2ip_t host2ip) { - stream_files_response resp; - auto& table = db.find_column_family(req.table); - auto sstables = co_await table.take_storage_snapshot(req.range); - co_await utils::get_local_injector().inject("order_sstables_for_streaming", [&sstables] (auto& handler) -> future<> { - if (sstables.size() == 3) { - // make sure the sstables are ordered so that the sstable containing shadowed data is streamed last - const std::string_view shadowed_file = handler.template get("shadowed_file").value(); - for (int index: {0, 1}) { - if (sstables[index].sst->component_basename(component_type::Data) == shadowed_file) { - std::swap(sstables[index], sstables[2]); - } - } - } - return make_ready_future<>(); - }); - auto files = std::list(); - - sstables::sstable_generation_generator sst_gen(0); - - for (auto& sst_snapshot : sstables) { - auto& sst = sst_snapshot.sst; - // stable state (across files) is a must for load to work on destination - auto sst_state = sst->state(); - - auto sources = create_stream_sources(sst_snapshot); - auto newgen = fmt::to_string(sst_gen(sstables::uuid_identifiers::yes)); - - for (auto&& s : sources) { - auto oldname = s->component_basename(); - auto newname = get_sstable_name_with_generation(req.ops_id, oldname, newgen); - - blogger.debug("fstream[{}] Get name oldname={}, newname={}", req.ops_id, oldname, newname); - - auto& info = files.emplace_back(); - info.fops = file_ops::stream_sstables; - info.sstable_state = sst_state; - info.filename = std::move(newname); - info.source = [s = std::move(s)](const file_input_stream_options& options) { - return s->input(options); - }; - } - // ensure we mark the end of each component sequence. - if (!files.empty()) { - files.back().fops = file_ops::load_sstables; - } - } - if (files.empty()) { - co_return resp; - } - blogger.debug("stream_sstables[{}] Started sending sstable_nr={} files_nr={} files={} range={}", - req.ops_id, sstables.size(), files.size(), files, req.range); - auto ops_start_time = std::chrono::steady_clock::now(); - size_t stream_bytes = co_await tablet_stream_files(ms, std::move(files), req.targets, req.table, req.ops_id, std::move(host2ip), req.topo_guard); - resp.stream_bytes = stream_bytes; - auto duration = std::chrono::steady_clock::now() - ops_start_time; - blogger.info("stream_sstables[{}] Finished sending sstable_nr={} files_nr={} files={} range={} stream_bytes={} stream_time={} stream_bw={}", - req.ops_id, sstables.size(), files.size(), files, req.range, stream_bytes, duration, get_bw(stream_bytes, ops_start_time)); - co_return resp; -} - -future tablet_stream_files(const file_stream_id& ops_id, - replica::table& table, - const dht::token_range& range, - const locator::host_id& src_host, - const locator::host_id& dst_host, - seastar::shard_id dst_shard_id, - netw::messaging_service& ms, - abort_source& as, - service::frozen_topology_guard topo_guard) { - stream_files_response resp; - std::exception_ptr error; - try { - co_await mark_tablet_stream_start(ops_id); - } catch (...) { - error = std::current_exception(); - } - if (!error) { - try { - streaming::stream_files_request req; - req.ops_id = ops_id; - req.keyspace_name = table.schema()->ks_name(), - req.table_name = table.schema()->cf_name(); - req.table = table.schema()->id(); - req.range = range; - req.targets = std::vector{node_and_shard{dst_host, dst_shard_id}}; - req.topo_guard = topo_guard; - resp = co_await ser::streaming_rpc_verbs::send_tablet_stream_files(&ms, src_host, as, req); - } catch (...) { - error = std::current_exception(); - } - } - co_await mark_tablet_stream_done(ops_id); - if (error) { - std::rethrow_exception(error); - } - co_return resp; -} - -} diff --git a/streaming/stream_blob.hh b/streaming/stream_blob.hh deleted file mode 100644 index 47ad861bc862..000000000000 --- a/streaming/stream_blob.hh +++ /dev/null @@ -1,192 +0,0 @@ -/* - * Copyright (C) 2023-present ScyllaDB - */ - -/* - * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 - */ - -#pragma once - -#include "message/messaging_service_fwd.hh" -#include -#include -#include -#include -#include -#include -#include -#include "utils/UUID.hh" -#include "dht/i_partitioner.hh" -#include "bytes.hh" -#include "replica/database_fwd.hh" -#include "locator/host_id.hh" -#include "service/topology_guard.hh" -#include "sstables/open_info.hh" - -#include -#include - -namespace streaming { - -using file_stream_id = utils::tagged_uuid; - -// - The file_ops::stream_sstables is used to stream a sstable file. -// -// - The file_ops::load_sstables is used to stream a sstable file and -// ask the receiver to load the sstable into the system. -enum class file_ops : uint16_t { - stream_sstables, - load_sstables, -}; - -// For STREAM_BLOB verb -enum class stream_blob_cmd : uint8_t { - ok, - error, - data, - end_of_stream, -}; - -class stream_blob_data { -public: - temporary_buffer buf; - stream_blob_data() = default; - stream_blob_data(temporary_buffer b) : buf(std::move(b)) {} - const char* data() const { - return buf.get(); - } - size_t size() const { - return buf.size(); - } - bool empty() const { - return buf.size() == 0; - } -}; - -class stream_blob_cmd_data { -public: - stream_blob_cmd cmd; - // The optional data contains value when the cmd is stream_blob_cmd::data. - // When the cmd is set to other values, e.g., stream_blob_cmd::error, the - // data contains no value. - std::optional data; - stream_blob_cmd_data(stream_blob_cmd c) : cmd(c) {} - stream_blob_cmd_data(stream_blob_cmd c, std::optional d) - : cmd(c) - , data(std::move(d)) - {} - stream_blob_cmd_data(stream_blob_cmd c, stream_blob_data d) - : cmd(c) - , data(std::move(d)) - {} - -}; - -class stream_blob_meta { -public: - file_stream_id ops_id; - table_id table; - sstring filename; - seastar::shard_id dst_shard_id; - streaming::file_ops fops; - service::frozen_topology_guard topo_guard; - std::optional sstable_state; - // We can extend this verb to send arbitary blob of data -}; - -enum class store_result { - ok, failure, -}; - -using stream_blob_source_fn = noncopyable_function>(const file_input_stream_options&)>; -using stream_blob_finish_fn = noncopyable_function(store_result)>; -using output_result = std::tuple>; -using stream_blob_create_output_fn = noncopyable_function(replica::database&, const streaming::stream_blob_meta&)>; - -struct stream_blob_info { - sstring filename; - streaming::file_ops fops; - std::optional sstable_state; - stream_blob_source_fn source; - - friend inline std::ostream& operator<<(std::ostream& os, const stream_blob_info& x) { - return os << x.filename; - } -}; - -// The handler for the STREAM_BLOB verb. -seastar::future<> stream_blob_handler(replica::database& db, netw::messaging_service& ms, gms::inet_address from, streaming::stream_blob_meta meta, rpc::sink sink, rpc::source source); - -// Exposed mainly for testing - -future<> stream_blob_handler(replica::database& db, - netw::messaging_service& ms, - gms::inet_address from, - streaming::stream_blob_meta meta, - rpc::sink sink, - rpc::source source, - stream_blob_create_output_fn, - bool may_inject_errors = false - ); - -// For TABLET_STREAM_FILES -class node_and_shard { -public: - locator::host_id node; - seastar::shard_id shard; - friend inline std::ostream& operator<<(std::ostream& os, const node_and_shard& x) { - return os << x.node << ":" << x.shard; - } - -}; - -} - -template <> struct fmt::formatter : fmt::ostream_formatter {}; - -namespace streaming { - -class stream_files_request { -public: - file_stream_id ops_id; - sstring keyspace_name; - sstring table_name; - table_id table; - dht::token_range range; - std::vector targets; - service::frozen_topology_guard topo_guard; -}; - -class stream_files_response { -public: - size_t stream_bytes = 0; -}; - -using host2ip_t = std::function (locator::host_id)>; - -// The handler for the TABLET_STREAM_FILES verb. The receiver of this verb will -// stream sstables files specified by the stream_files_request req. -future tablet_stream_files_handler(replica::database& db, netw::messaging_service& ms, streaming::stream_files_request req, host2ip_t host2ip); - -// Ask the src node to stream sstables to dst node for table in the given token range using TABLET_STREAM_FILES verb. -future tablet_stream_files(const file_stream_id& ops_id, replica::table& table, const dht::token_range& range, const locator::host_id& src, const locator::host_id& dst, seastar::shard_id dst_shard_id, netw::messaging_service& ms, abort_source& as, service::frozen_topology_guard topo_guard); - -// Exposed for testability -future tablet_stream_files(netw::messaging_service& ms, - std::list sources, - std::vector targets, - table_id table, - file_stream_id ops_id, - host2ip_t host2ip, - service::frozen_topology_guard topo_guard, - bool may_inject_errors = false - ); - - -future<> mark_tablet_stream_start(file_stream_id); -future<> mark_tablet_stream_done(file_stream_id); - -} - -template<> struct fmt::formatter; diff --git a/streaming/stream_session.cc b/streaming/stream_session.cc index b876b62381fb..4619f2b09242 100644 --- a/streaming/stream_session.cc +++ b/streaming/stream_session.cc @@ -19,7 +19,6 @@ #include "dht/auto_refreshing_sharder.hh" #include #include -#include "streaming/stream_blob.hh" #include "streaming/stream_session_state.hh" #include "service/migration_manager.hh" #include "mutation_writer/multishard_writer.hh" @@ -279,22 +278,13 @@ void stream_manager::init_messaging_service_handler(abort_source& as) { return make_ready_future<>(); } }); - ms.register_stream_blob([this] (const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source) { - auto from = netw::messaging_service::get_source(cinfo).addr; - auto sink = _ms.local().make_sink_for_stream_blob(source); - (void)stream_blob_handler(_db.local(), _ms.local(), from, meta, sink, source).handle_exception([ms = _ms.local().shared_from_this()] (std::exception_ptr eptr) { - sslog.warn("Failed to run stream blob handler: {}", eptr); - }); - return make_ready_future>(sink); - }); } future<> stream_manager::uninit_messaging_service_handler() { auto& ms = _ms.local(); return when_all_succeed( ser::streaming_rpc_verbs::unregister(&ms), - ms.unregister_stream_mutation_fragments(), - ms.unregister_stream_blob()).discard_result(); + ms.unregister_stream_mutation_fragments()).discard_result(); } stream_session::stream_session(stream_manager& mgr, locator::host_id peer_) diff --git a/test/boost/CMakeLists.txt b/test/boost/CMakeLists.txt index e11f2426c709..26fddfec575e 100644 --- a/test/boost/CMakeLists.txt +++ b/test/boost/CMakeLists.txt @@ -251,8 +251,6 @@ add_scylla_test(string_format_test KIND BOOST) add_scylla_test(summary_test KIND BOOST) -add_scylla_test(file_stream_test - KIND SEASTAR) add_scylla_test(tagged_integer_test KIND SEASTAR) add_scylla_test(token_metadata_test diff --git a/test/boost/file_stream_test.cc b/test/boost/file_stream_test.cc deleted file mode 100644 index e704231837e4..000000000000 --- a/test/boost/file_stream_test.cc +++ /dev/null @@ -1,260 +0,0 @@ -/* - * Copyright (C) 2023-present ScyllaDB - */ - -/* - * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 - */ - -#include "test/lib/cql_test_env.hh" -#include "streaming/stream_blob.hh" -#include "message/messaging_service.hh" -#include "test/lib/log.hh" - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -future generate_file_hash(sstring filename) { - auto f = co_await seastar::open_file_dma(filename, seastar::open_flags::ro); - auto in = seastar::make_file_input_stream(std::move(f)); - CryptoPP::SHA256 hash; - unsigned char digest[CryptoPP::SHA256::DIGESTSIZE]; - std::stringstream ss; - while (true) { - auto buf = co_await in.read(); - if (buf.empty()) { - break; - } - hash.Update((const unsigned char*)buf.get(), buf.size()); - } - co_await in.close(); - hash.Final(digest); - for (int i = 0; i < CryptoPP::SHA256::DIGESTSIZE; i++) { - ss << std::hex << std::setw(2) << std::setfill('0') << (int)digest[i]; - } - co_return ss.str(); -} - -sstring generate_random_filename() { - char filename[L_tmpnam]; - std::tmpnam(filename); - return filename; -} - -future<> write_random_content_to_file(const sstring& filename, size_t content_size = 1024) { - auto f = co_await open_file_dma(filename, open_flags::rw | open_flags::create); - auto ostream = co_await make_file_output_stream(std::move(f)); - srand(time(nullptr)); - for (size_t i = 0; i < content_size; ++i) { - char c = rand() % 256; - co_await ostream.write(&c, 1); - } - co_await ostream.close(); -} - -using namespace streaming; - -static future -do_test_file_stream(replica::database& db, netw::messaging_service& ms, std::vector filelist, const std::string& suffix, bool inject_error, bool unsupported_file_ops = false) { - bool ret = false; - bool verb_register = false; - auto ops_id = file_stream_id::create_random_id(); - auto& global_db = db.container(); - auto& global_ms = ms.container(); - int n_retries = 0; - - do { - try { - if (!verb_register) { - co_await smp::invoke_on_all([&] { - return global_ms.local().register_stream_blob([&](const rpc::client_info& cinfo, streaming::stream_blob_meta meta, rpc::source source) { - auto from = netw::messaging_service::get_source(cinfo).addr; - auto sink = global_ms.local().make_sink_for_stream_blob(source); - (void)stream_blob_handler(global_db.local(), global_ms.local(), from, meta, sink, source, [&suffix](auto&, const streaming::stream_blob_meta& meta) -> future { - auto path = meta.filename + suffix; - auto f = co_await open_file_dma(path, open_flags::wo|open_flags::create); - auto out = co_await make_file_output_stream(std::move(f)); - co_return output_result{ - [path = std::move(path)](store_result res) -> future<> { - if (res != store_result::ok) { - co_await remove_file(path); - } - }, - std::move(out) - }; - }, inject_error).handle_exception([sink, source, ms = global_ms.local().shared_from_this()] (std::exception_ptr eptr) { - testlog.warn("Failed to run stream blob handler: {}", eptr); - }); - return make_ready_future>(sink); - }); - }); - } - verb_register = true; - auto table = table_id::create_random_id(); - auto files = std::list(); - auto hostid = db.get_token_metadata().get_my_id(); - seastar::shard_id dst_shard_id = 0; - co_await mark_tablet_stream_start(ops_id); - auto targets = std::vector{node_and_shard{hostid, dst_shard_id}}; - for (const auto& filename : filelist) { - auto fops = file_ops::stream_sstables; - fops = unsupported_file_ops ? file_ops(0xff55) : fops; - auto file = co_await open_file_dma(filename, open_flags::ro); - auto& info = files.emplace_back(); - info.filename = filename; - info.fops = fops; - info.source = [file = std::move(file)](const file_input_stream_options& foptions) mutable -> future> { - co_return make_file_input_stream(std::move(file), foptions); - }; - } - auto host2ip = [&global_db] (locator::host_id id) -> future { - co_return global_db.local().get_token_metadata().get_topology().my_address(); - }; - size_t stream_bytes = co_await tablet_stream_files(ms, std::move(files), targets, table, ops_id, host2ip, service::null_topology_guard, inject_error); - co_await mark_tablet_stream_done(ops_id); - testlog.info("do_test_file_stream[{}] status=ok files={} stream_bytes={}", ops_id, filelist.size(), stream_bytes); - ret = true; - } catch (seastar::rpc::stream_closed&) { - testlog.warn("do_test_file_stream[{}] status=fail error={} retry={}", ops_id, std::current_exception(), n_retries++); - if (n_retries < 3) { - testlog.info("Retrying send"); - continue; - } - } catch (...) { - testlog.warn("do_test_file_stream[{}] status=fail error={}", ops_id, std::current_exception()); - } - } while (false); - - if (verb_register) { - co_await smp::invoke_on_all([&global_ms] { - return global_ms.local().unregister_stream_blob(); - }); - } - co_return ret; -} - -void do_test_file_stream(bool inject_error) { - cql_test_config cfg; - cfg.ms_listen = true; - std::vector files; - std::vector files_rx; - std::vector hash_tx; - std::vector hash_rx; - size_t nr_files = 10; - size_t file_size = 0; - static const std::string suffix = ".rx"; - - while (files.size() != nr_files) { - auto name = generate_random_filename(); - files.push_back(name); - files_rx.push_back(name + suffix); - } - - size_t base_size = 1024; - -#ifdef SEASTAR_DEBUG - base_size = 1; -#endif - - for (auto& file : files) { - if (file_size == 0) { - file_size = 1 * 1024 * base_size; - } else { - file_size = (rand() % 10) * 1024 * base_size + rand() % base_size; - } - file_size = std::max(size_t(1), file_size); - testlog.info("file_tx={} file_size={}", file, file_size); - write_random_content_to_file(file, file_size).get(); - } - - do_with_cql_env_thread([files, inject_error] (auto& e) { - do_test_file_stream(e.local_db(), e.get_messaging_service().local(), files, suffix, inject_error).get(); - }, cfg).get(); - - bool cleanup = true; - for (auto& file : files) { - auto hash = generate_file_hash(file).get(); - testlog.info("file_tx={} hash={}", file, hash); - hash_tx.push_back(hash); - if (cleanup) { - seastar::remove_file(file).get(); - } - } - for (auto& file : files_rx) { - sstring hash = "SKIP"; - try { - hash = generate_file_hash(file).get(); - if (cleanup) { - seastar::remove_file(file).get(); - } - } catch (...) { - if (!inject_error) { - throw; - } - } - hash_rx.push_back(hash); - testlog.info("file_rx={} hash={}", file, hash); - } - - BOOST_REQUIRE(hash_tx.size() == hash_rx.size()); - for (size_t i = 0; i < hash_tx.size(); i++) { - testlog.info("Check tx_hash={} rx_hash={}", hash_tx[i], hash_rx[i]); - if (inject_error) { - BOOST_REQUIRE(hash_tx[i] == hash_rx[i] || sstring("SKIP") == hash_rx[i]); - } else { - BOOST_REQUIRE(hash_tx[i] == hash_rx[i]); - } - } -} - -void do_test_unsupported_file_ops() { - bool inject_error = false; - bool unsupported_file_ops = true; - - cql_test_config cfg; - cfg.ms_listen = true; - std::vector files; - size_t nr_files = 2; - size_t file_size = 1024; - - while (files.size() != nr_files) { - auto name = generate_random_filename(); - files.push_back(name); - } - - for (auto& file : files) { - testlog.info("file_tx={} file_size={}", file, file_size); - write_random_content_to_file(file, file_size).get(); - } - - do_with_cql_env_thread([files, inject_error, unsupported_file_ops] (auto& e) { - auto ok = do_test_file_stream(e.local_db(), e.get_messaging_service().local(), files, "", inject_error, unsupported_file_ops).get(); - // Stream with a unsupported file ops should fail - BOOST_REQUIRE(ok == false); - }, cfg).get(); - - for (auto& file : files) { - seastar::remove_file(file).get(); - } -} - -SEASTAR_THREAD_TEST_CASE(test_file_stream) { - bool inject_error = false; - do_test_file_stream(inject_error); -} - -SEASTAR_THREAD_TEST_CASE(test_file_stream_inject_error) { - bool inject_error = true; - do_test_file_stream(inject_error); -} - -SEASTAR_THREAD_TEST_CASE(test_unsupported_file_ops) { - do_test_unsupported_file_ops(); -} diff --git a/test/topology_custom/test_tablets2.py b/test/topology_custom/test_tablets2.py index fcfbbc20706b..441f11323428 100644 --- a/test/topology_custom/test_tablets2.py +++ b/test/topology_custom/test_tablets2.py @@ -251,7 +251,6 @@ async def test_streaming_is_guarded_by_topology_guard(manager: ManagerClient): cmdline = [ '--logger-log-level', 'storage_service=trace', '--logger-log-level', 'raft_topology=trace', - '--enable-file-stream', 'false', ] servers = [await manager.server_add(cmdline=cmdline)] diff --git a/test/topology_custom/test_tablets_migration.py b/test/topology_custom/test_tablets_migration.py index f203679b7edb..486b93915563 100644 --- a/test/topology_custom/test_tablets_migration.py +++ b/test/topology_custom/test_tablets_migration.py @@ -6,15 +6,13 @@ from cassandra.query import SimpleStatement, ConsistencyLevel from test.pylib.manager_client import ManagerClient from test.pylib.rest_client import HTTPError, read_barrier -from test.pylib.tablets import get_tablet_replica, get_all_tablet_replicas +from test.pylib.tablets import get_all_tablet_replicas from test.topology.conftest import skip_mode from test.topology.util import wait_for_cql_and_get_hosts import time import pytest import logging import asyncio -import os -import glob logger = logging.getLogger(__name__) @@ -291,100 +289,3 @@ async def assert_rows(num): await assert_rows(2) await cql.run_async(f"INSERT INTO test.test (pk, c) VALUES ({3}, {3});") await assert_rows(3) - -@pytest.mark.asyncio -@skip_mode('release', 'error injections are not supported in release mode') -async def test_staging_backlog_is_preserved_with_file_based_streaming(manager: ManagerClient): - logger.info("Bootstrapping cluster") - # the error injection will halt view updates from staging, allowing migration to transfer the view update backlog. - cfg = {'enable_user_defined_functions': False, 'enable_tablets': True, - 'error_injections_at_startup': ['view_update_generator_consume_staging_sstable']} - servers = [await manager.server_add(config=cfg)] - - await manager.api.disable_tablet_balancing(servers[0].ip_addr) - - cql = manager.get_cql() - await cql.run_async("CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 1};") - await cql.run_async("CREATE TABLE test.test (pk int PRIMARY KEY, c int);") - await cql.run_async("CREATE MATERIALIZED VIEW test.mv1 AS \ - SELECT * FROM test.test WHERE pk IS NOT NULL AND c IS NOT NULL \ - PRIMARY KEY (c, pk);") - - logger.info("Populating single tablet") - keys = range(256) - await asyncio.gather(*[cql.run_async(f"INSERT INTO test.test (pk, c) VALUES ({k}, {k});") for k in keys]) - - await manager.api.flush_keyspace(servers[0].ip_addr, "test") - - # check - async def check(expected): - rows = await cql.run_async("SELECT pk from test.test") - assert len(list(rows)) == len(expected) - await check(keys) - - logger.info("Adding new server") - servers.append(await manager.server_add(config=cfg)) - - async def get_table_dir(manager, server_id): - node_workdir = await manager.server_get_workdir(server_id) - return glob.glob(os.path.join(node_workdir, "data", "test", "test-*"))[0] - - s0_table_dir = await get_table_dir(manager, servers[0].server_id) - logger.info(f"Table dir in server 0: {s0_table_dir}") - - s1_table_dir = await get_table_dir(manager, servers[1].server_id) - logger.info(f"Table dir in server 1: {s1_table_dir}") - - # Explicitly close the driver to avoid reconnections if scylla fails to update gossiper state on shutdown. - # It's a problem until https://github.com/scylladb/scylladb/issues/15356 is fixed. - manager.driver_close() - cql = None - await manager.server_stop_gracefully(servers[0].server_id) - - def move_sstables_to_staging(table_dir: str): - table_staging_dir = os.path.join(table_dir, "staging") - logger.info(f"Moving sstables to staging dir: {table_staging_dir}") - for sst in glob.glob(os.path.join(table_dir, "*-Data.db")): - for src_path in glob.glob(os.path.join(table_dir, sst.removesuffix("-Data.db") + "*")): - dst_path = os.path.join(table_staging_dir, os.path.basename(src_path)) - logger.info(f"Moving sstable file {src_path} to {dst_path}") - os.rename(src_path, dst_path) - - def sstable_count_in_staging(table_dir: str): - table_staging_dir = os.path.join(table_dir, "staging") - return len(glob.glob(os.path.join(table_staging_dir, "*-Data.db"))) - - move_sstables_to_staging(s0_table_dir) - s0_sstables_in_staging = sstable_count_in_staging(s0_table_dir) - - await manager.server_start(servers[0].server_id) - cql = manager.get_cql() - await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60) - - tablet_token = 0 # Doesn't matter since there is one tablet - replica = await get_tablet_replica(manager, servers[0], 'test', 'test', tablet_token) - s1_host_id = await manager.get_host_id(servers[1].server_id) - dst_shard = 0 - - migration_task = asyncio.create_task( - manager.api.move_tablet(servers[0].ip_addr, "test", "test", replica[0], replica[1], s1_host_id, dst_shard, tablet_token)) - - logger.info("Waiting for migration to finish") - await migration_task - logger.info("Migration done") - - # FIXME: After https://github.com/scylladb/scylladb/issues/19149 is fixed, we can check that view updates complete - # after migration and then check for base-view consistency. By the time being, we only check that backlog is - # transferred by looking at staging directory. - - s1_sstables_in_staging = sstable_count_in_staging(s1_table_dir) - logger.info(f"SSTable count in staging dir of server 1: {s1_sstables_in_staging}") - - logger.info("Allowing view update generator to progress again") - for server in servers: - manager.api.disable_injection(server.ip_addr, 'view_update_generator_consume_staging_sstable') - - assert s0_sstables_in_staging > 0 - assert s0_sstables_in_staging == s1_sstables_in_staging - - await check(keys) diff --git a/test/topology_custom/test_topology_ops_encrypted.py b/test/topology_custom/test_topology_ops_encrypted.py deleted file mode 100644 index ac42c35e7ca2..000000000000 --- a/test/topology_custom/test_topology_ops_encrypted.py +++ /dev/null @@ -1,85 +0,0 @@ -# -# Copyright (C) 2023-present ScyllaDB -# -# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 -# -from test.pylib.scylla_cluster import ReplaceConfig -from test.pylib.manager_client import ManagerClient -from test.pylib.internal_types import ServerInfo -from test.pylib.util import unique_name, wait_for_cql_and_get_hosts -from test.topology.util import check_token_ring_and_group0_consistency, reconnect_driver -from test.topology_custom.test_topology_ops import check_node_log_for_failed_mutations, start_writes - -from cassandra.cluster import Session, ConsistencyLevel -from cassandra.query import SimpleStatement - -import asyncio -import time -import pytest -import logging - -logger = logging.getLogger(__name__) - -@pytest.mark.asyncio -@pytest.mark.parametrize("tablets_enabled", [True, False]) -async def test_topology_ops_encrypted(request, manager: ManagerClient, tablets_enabled: bool, tmp_path): - """Test basic topology operations using the topology coordinator. But encrypted.""" - d = tmp_path / "keys" - d.mkdir() - k = d / "system_key" - k.write_text('AES/CBC/PKCS5Padding:128:ApvJEoFpQmogvam18bb54g==') - cfg = {'enable_tablets' : tablets_enabled, - 'user_info_encryption': {'enabled': True, 'key_provider': 'LocalFileSystemKeyProviderFactory'}, - 'system_key_directory': d.as_posix()} - rf = 3 - num_nodes = rf - if tablets_enabled: - num_nodes += 1 - - logger.info("Bootstrapping first node") - servers = [await manager.server_add(config=cfg)] - - logger.info(f"Restarting node {servers[0]}") - await manager.server_stop_gracefully(servers[0].server_id) - await manager.server_start(servers[0].server_id) - - logger.info("Bootstrapping other nodes") - servers += await manager.servers_add(num_nodes, config=cfg) - - await wait_for_cql_and_get_hosts(manager.cql, servers, time.time() + 60) - cql = await reconnect_driver(manager) - finish_writes = await start_writes(cql, rf, ConsistencyLevel.ONE) - - logger.info(f"Decommissioning node {servers[0]}") - await manager.decommission_node(servers[0].server_id) - await check_token_ring_and_group0_consistency(manager) - servers = servers[1:] - - logger.info(f"Restarting node {servers[0]} when other nodes have bootstrapped") - await manager.server_stop_gracefully(servers[0].server_id) - await manager.server_start(servers[0].server_id) - - logger.info(f"Stopping node {servers[0]}") - await manager.server_stop_gracefully(servers[0].server_id) - await check_node_log_for_failed_mutations(manager, servers[0]) - - logger.info(f"Replacing node {servers[0]}") - replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = False) - servers = servers[1:] + [await manager.server_add(replace_cfg)] - await check_token_ring_and_group0_consistency(manager) - - logger.info(f"Stopping node {servers[0]}") - await manager.server_stop_gracefully(servers[0].server_id) - await check_node_log_for_failed_mutations(manager, servers[0]) - - logger.info(f"Removing node {servers[0]} using {servers[1]}") - await manager.remove_node(servers[1].server_id, servers[0].server_id) - await check_token_ring_and_group0_consistency(manager) - servers = servers[1:] - - logger.info("Checking results of the background writes") - await finish_writes() - - for server in servers: - await check_node_log_for_failed_mutations(manager, server) -