replica: move ::database, ::keyspace, and ::table to replica namespace
Move replica-oriented classes to the replica namespace. The main classes moved are ::database, ::keyspace, and ::table, but a few ancillary classes are also moved. There are certainly classes that should be moved but aren't (like distributed_loader) but we have to start somewhere. References are adjusted treewide. In many cases, it is obvious that a call site should not access the replica (but the data_dictionary instead), but that is left for separate work. scylla-gdb.py is adjusted to look for both the new and old names.
This commit is contained in:
@@ -195,7 +195,7 @@ schema_ptr executor::find_table(service::storage_proxy& proxy, const rjson::valu
|
||||
}
|
||||
try {
|
||||
return proxy.get_db().local().find_schema(sstring(executor::KEYSPACE_NAME_PREFIX) + sstring(*table_name), *table_name);
|
||||
} catch(no_such_column_family&) {
|
||||
} catch(replica::no_such_column_family&) {
|
||||
throw api_error::resource_not_found(
|
||||
format("Requested resource not found: Table: {} not found", *table_name));
|
||||
}
|
||||
@@ -244,7 +244,7 @@ get_table_or_view(service::storage_proxy& proxy, const rjson::value& request) {
|
||||
if (is_internal_table) {
|
||||
try {
|
||||
return { proxy.get_db().local().find_schema(sstring(internal_ks_name), sstring(internal_table_name)), type };
|
||||
} catch (no_such_column_family&) {
|
||||
} catch (replica::no_such_column_family&) {
|
||||
throw api_error::resource_not_found(
|
||||
format("Requested resource not found: Internal table: {}.{} not found", internal_ks_name, internal_table_name));
|
||||
}
|
||||
@@ -271,7 +271,7 @@ get_table_or_view(service::storage_proxy& proxy, const rjson::value& request) {
|
||||
|
||||
try {
|
||||
return { proxy.get_db().local().find_schema(keyspace_name, table_name), type };
|
||||
} catch(no_such_column_family&) {
|
||||
} catch(replica::no_such_column_family&) {
|
||||
if (index_name) {
|
||||
// DynamoDB returns a different error depending on whether the
|
||||
// base table doesn't exist (ResourceNotFoundException) or it
|
||||
@@ -420,7 +420,7 @@ future<executor::request_return_type> executor::describe_table(client_state& cli
|
||||
// Add base table's KeySchema and collect types for AttributeDefinitions:
|
||||
describe_key_schema(table_description, *schema, key_attribute_types);
|
||||
|
||||
table& t = _proxy.get_db().local().find_column_family(schema);
|
||||
replica::table& t = _proxy.get_db().local().find_column_family(schema);
|
||||
if (!t.views().empty()) {
|
||||
rjson::value gsi_array = rjson::empty_array();
|
||||
rjson::value lsi_array = rjson::empty_array();
|
||||
@@ -592,7 +592,7 @@ static schema_ptr get_table_from_arn(service::storage_proxy& proxy, std::string_
|
||||
std::string_view table_name = arn.substr(table_start + 1);
|
||||
// FIXME: remove sstring creation once find_schema gains a view-based interface
|
||||
return proxy.get_db().local().find_schema(sstring(keyspace_name), sstring(table_name));
|
||||
} catch (const no_such_column_family& e) {
|
||||
} catch (const replica::no_such_column_family& e) {
|
||||
throw api_error::access_denied("Incorrect resource identifier");
|
||||
} catch (const std::out_of_range& e) {
|
||||
throw api_error::access_denied("Incorrect resource identifier");
|
||||
@@ -1699,7 +1699,7 @@ static schema_ptr get_table_from_batch_request(const service::storage_proxy& pro
|
||||
validate_table_name(table_name);
|
||||
try {
|
||||
return proxy.get_db().local().find_schema(sstring(executor::KEYSPACE_NAME_PREFIX) + table_name, table_name);
|
||||
} catch(no_such_column_family&) {
|
||||
} catch(replica::no_such_column_family&) {
|
||||
throw api_error::resource_not_found(format("Requested resource not found: Table: {} not found", table_name));
|
||||
}
|
||||
}
|
||||
@@ -3967,10 +3967,10 @@ future<executor::request_return_type> executor::list_tables(client_state& client
|
||||
|
||||
auto table_names = _proxy.get_db().local().get_column_families()
|
||||
| boost::adaptors::map_values
|
||||
| boost::adaptors::filtered([] (const lw_shared_ptr<table>& t) {
|
||||
| boost::adaptors::filtered([] (const lw_shared_ptr<replica::table>& t) {
|
||||
return t->schema()->ks_name().find(KEYSPACE_NAME_PREFIX) == 0 && !t->schema()->is_view();
|
||||
})
|
||||
| boost::adaptors::transformed([] (const lw_shared_ptr<table>& t) {
|
||||
| boost::adaptors::transformed([] (const lw_shared_ptr<replica::table>& t) {
|
||||
return t->schema()->cf_name();
|
||||
});
|
||||
|
||||
|
||||
@@ -169,7 +169,7 @@ future<alternator::executor::request_return_type> alternator::executor::list_str
|
||||
// between queries may or may not miss info. But that should be rare,
|
||||
// and we can probably expect this to be a single call.
|
||||
if (streams_start) {
|
||||
i = std::find_if(i, e, [&](const std::pair<utils::UUID, lw_shared_ptr<column_family>>& p) {
|
||||
i = std::find_if(i, e, [&](const std::pair<utils::UUID, lw_shared_ptr<replica::column_family>>& p) {
|
||||
return p.first == streams_start
|
||||
&& cdc::get_base_table(db, *p.second->schema())
|
||||
&& is_alternator_keyspace(p.second->schema()->ks_name())
|
||||
@@ -424,7 +424,7 @@ using namespace std::string_literals;
|
||||
* This will be a partial overlap, but it is the best we can do.
|
||||
*/
|
||||
|
||||
static std::chrono::seconds confidence_interval(const database& db) {
|
||||
static std::chrono::seconds confidence_interval(const replica::database& db) {
|
||||
return std::chrono::seconds(db.get_config().alternator_streams_time_window_s());
|
||||
}
|
||||
|
||||
|
||||
@@ -178,7 +178,7 @@ future<executor::request_return_type> executor::describe_time_to_live(client_sta
|
||||
// like user deletions, will also appear on the CDC log and therefore
|
||||
// Alternator Streams if enabled (FIXME: explain how we mark the
|
||||
// deletion different from user deletes. We don't do it yet.).
|
||||
expiration_service::expiration_service(database& db, service::storage_proxy& proxy)
|
||||
expiration_service::expiration_service(replica::database& db, service::storage_proxy& proxy)
|
||||
: _db(db)
|
||||
, _proxy(proxy)
|
||||
{
|
||||
@@ -436,7 +436,7 @@ class token_ranges_owned_by_this_shard {
|
||||
size_t _end_idx;
|
||||
std::optional<dht::selective_token_range_sharder> _intersecter;
|
||||
public:
|
||||
token_ranges_owned_by_this_shard(database& db, schema_ptr s)
|
||||
token_ranges_owned_by_this_shard(replica::database& db, schema_ptr s)
|
||||
: _s(s)
|
||||
, _token_ranges(db.find_keyspace(s->ks_name()).get_effective_replication_map(),
|
||||
utils::fb_utilities::get_broadcast_address())
|
||||
@@ -642,7 +642,7 @@ static future<> scan_table_ranges(
|
||||
// reboot.
|
||||
static future<bool> scan_table(
|
||||
service::storage_proxy& proxy,
|
||||
database& db,
|
||||
replica::database& db,
|
||||
schema_ptr s,
|
||||
abort_source& abort_source,
|
||||
named_semaphore& page_sem)
|
||||
|
||||
@@ -26,7 +26,9 @@
|
||||
#include <seastar/core/abort_source.hh>
|
||||
#include <seastar/core/semaphore.hh>
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace service {
|
||||
class storage_proxy;
|
||||
@@ -38,7 +40,7 @@ namespace alternator {
|
||||
// items in all tables with per-item expiration enabled. Currently, this means
|
||||
// Alternator tables with TTL configured via a UpdateTimeToLeave request.
|
||||
class expiration_service final : public seastar::peering_sharded_service<expiration_service> {
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
service::storage_proxy& _proxy;
|
||||
// _end is set by start(), and resolves when the the background service
|
||||
// started by it ends. To ask the background service to end, _abort_source
|
||||
@@ -52,7 +54,7 @@ public:
|
||||
// sharded_service<expiration_service>::start() creates this object on
|
||||
// all shards, so calls this constructor on each shard. Later, the
|
||||
// additional start() function should be invoked on all shards.
|
||||
expiration_service(database&, service::storage_proxy&);
|
||||
expiration_service(replica::database&, service::storage_proxy&);
|
||||
future<> start();
|
||||
future<> run();
|
||||
// sharded_service<expiration_service>::stop() calls the following stop()
|
||||
|
||||
@@ -49,7 +49,7 @@ namespace api {
|
||||
static std::unique_ptr<reply> exception_reply(std::exception_ptr eptr) {
|
||||
try {
|
||||
std::rethrow_exception(eptr);
|
||||
} catch (const no_such_keyspace& ex) {
|
||||
} catch (const replica::no_such_keyspace& ex) {
|
||||
throw bad_param_exception(ex.what());
|
||||
}
|
||||
// We never going to get here
|
||||
|
||||
@@ -72,12 +72,12 @@ struct http_context {
|
||||
sstring api_dir;
|
||||
sstring api_doc;
|
||||
httpd::http_server_control http_server;
|
||||
distributed<database>& db;
|
||||
distributed<replica::database>& db;
|
||||
distributed<service::storage_proxy>& sp;
|
||||
service::load_meter& lmeter;
|
||||
const sharded<locator::shared_token_metadata>& shared_token_metadata;
|
||||
|
||||
http_context(distributed<database>& _db,
|
||||
http_context(distributed<replica::database>& _db,
|
||||
distributed<service::storage_proxy>& _sp,
|
||||
service::load_meter& _lm, const sharded<locator::shared_token_metadata>& _stm)
|
||||
: db(_db), sp(_sp), lmeter(_lm), shared_token_metadata(_stm) {
|
||||
|
||||
@@ -208,7 +208,7 @@ void set_cache_service(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cs::get_row_capacity.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return ctx.db.map_reduce0([](database& db) -> uint64_t {
|
||||
return ctx.db.map_reduce0([](replica::database& db) -> uint64_t {
|
||||
return db.row_cache_tracker().region().occupancy().used_space();
|
||||
}, uint64_t(0), std::plus<uint64_t>()).then([](const int64_t& res) {
|
||||
return make_ready_future<json::json_return_type>(res);
|
||||
@@ -216,26 +216,26 @@ void set_cache_service(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cs::get_row_hits.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [](const column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().hits.count();
|
||||
}, std::plus<uint64_t>());
|
||||
});
|
||||
|
||||
cs::get_row_requests.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [](const column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().hits.count() + cf.get_row_cache().stats().misses.count();
|
||||
}, std::plus<uint64_t>());
|
||||
});
|
||||
|
||||
cs::get_row_hit_rate.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, ratio_holder(), [](const column_family& cf) {
|
||||
return map_reduce_cf(ctx, ratio_holder(), [](const replica::column_family& cf) {
|
||||
return ratio_holder(cf.get_row_cache().stats().hits.count() + cf.get_row_cache().stats().misses.count(),
|
||||
cf.get_row_cache().stats().hits.count());
|
||||
}, std::plus<ratio_holder>());
|
||||
});
|
||||
|
||||
cs::get_row_hits_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().hits.rate();
|
||||
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
|
||||
return make_ready_future<json::json_return_type>(meter_to_json(m));
|
||||
@@ -243,7 +243,7 @@ void set_cache_service(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cs::get_row_requests_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().hits.rate() + cf.get_row_cache().stats().misses.rate();
|
||||
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
|
||||
return make_ready_future<json::json_return_type>(meter_to_json(m));
|
||||
@@ -253,7 +253,7 @@ void set_cache_service(http_context& ctx, routes& r) {
|
||||
cs::get_row_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
// In origin row size is the weighted size.
|
||||
// We currently do not support weights, so we use num entries instead
|
||||
return ctx.db.map_reduce0([](database& db) -> uint64_t {
|
||||
return ctx.db.map_reduce0([](replica::database& db) -> uint64_t {
|
||||
return db.row_cache_tracker().partitions();
|
||||
}, uint64_t(0), std::plus<uint64_t>()).then([](const int64_t& res) {
|
||||
return make_ready_future<json::json_return_type>(res);
|
||||
@@ -261,7 +261,7 @@ void set_cache_service(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cs::get_row_entries.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return ctx.db.map_reduce0([](database& db) -> uint64_t {
|
||||
return ctx.db.map_reduce0([](replica::database& db) -> uint64_t {
|
||||
return db.row_cache_tracker().partitions();
|
||||
}, uint64_t(0), std::plus<uint64_t>()).then([](const int64_t& res) {
|
||||
return make_ready_future<json::json_return_type>(res);
|
||||
|
||||
@@ -56,57 +56,57 @@ std::tuple<sstring, sstring> parse_fully_qualified_cf_name(sstring name) {
|
||||
return std::make_tuple(name.substr(0, pos), name.substr(end));
|
||||
}
|
||||
|
||||
const utils::UUID& get_uuid(const sstring& ks, const sstring& cf, const database& db) {
|
||||
const utils::UUID& get_uuid(const sstring& ks, const sstring& cf, const replica::database& db) {
|
||||
try {
|
||||
return db.find_uuid(ks, cf);
|
||||
} catch (no_such_column_family& e) {
|
||||
} catch (replica::no_such_column_family& e) {
|
||||
throw bad_param_exception(e.what());
|
||||
}
|
||||
}
|
||||
|
||||
const utils::UUID& get_uuid(const sstring& name, const database& db) {
|
||||
const utils::UUID& get_uuid(const sstring& name, const replica::database& db) {
|
||||
auto [ks, cf] = parse_fully_qualified_cf_name(name);
|
||||
return get_uuid(ks, cf, db);
|
||||
}
|
||||
|
||||
future<> foreach_column_family(http_context& ctx, const sstring& name, function<void(column_family&)> f) {
|
||||
future<> foreach_column_family(http_context& ctx, const sstring& name, function<void(replica::column_family&)> f) {
|
||||
auto uuid = get_uuid(name, ctx.db.local());
|
||||
|
||||
return ctx.db.invoke_on_all([f, uuid](database& db) {
|
||||
return ctx.db.invoke_on_all([f, uuid](replica::database& db) {
|
||||
f(db.find_column_family(uuid));
|
||||
});
|
||||
}
|
||||
|
||||
future<json::json_return_type> get_cf_stats(http_context& ctx, const sstring& name,
|
||||
int64_t column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, name, int64_t(0), [f](const column_family& cf) {
|
||||
int64_t replica::column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, name, int64_t(0), [f](const replica::column_family& cf) {
|
||||
return cf.get_stats().*f;
|
||||
}, std::plus<int64_t>());
|
||||
}
|
||||
|
||||
future<json::json_return_type> get_cf_stats(http_context& ctx,
|
||||
int64_t column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [f](const column_family& cf) {
|
||||
int64_t replica::column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [f](const replica::column_family& cf) {
|
||||
return cf.get_stats().*f;
|
||||
}, std::plus<int64_t>());
|
||||
}
|
||||
|
||||
static future<json::json_return_type> get_cf_stats_count(http_context& ctx, const sstring& name,
|
||||
utils::timed_rate_moving_average_and_histogram column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, name, int64_t(0), [f](const column_family& cf) {
|
||||
utils::timed_rate_moving_average_and_histogram replica::column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, name, int64_t(0), [f](const replica::column_family& cf) {
|
||||
return (cf.get_stats().*f).hist.count;
|
||||
}, std::plus<int64_t>());
|
||||
}
|
||||
|
||||
static future<json::json_return_type> get_cf_stats_sum(http_context& ctx, const sstring& name,
|
||||
utils::timed_rate_moving_average_and_histogram column_family_stats::*f) {
|
||||
utils::timed_rate_moving_average_and_histogram replica::column_family_stats::*f) {
|
||||
auto uuid = get_uuid(name, ctx.db.local());
|
||||
return ctx.db.map_reduce0([uuid, f](database& db) {
|
||||
return ctx.db.map_reduce0([uuid, f](replica::database& db) {
|
||||
// Histograms information is sample of the actual load
|
||||
// so to get an estimation of sum, we multiply the mean
|
||||
// with count. The information is gather in nano second,
|
||||
// but reported in micro
|
||||
column_family& cf = db.find_column_family(uuid);
|
||||
replica::column_family& cf = db.find_column_family(uuid);
|
||||
return ((cf.get_stats().*f).hist.count/1000.0) * (cf.get_stats().*f).hist.mean;
|
||||
}, 0.0, std::plus<double>()).then([](double res) {
|
||||
return make_ready_future<json::json_return_type>((int64_t)res);
|
||||
@@ -115,16 +115,16 @@ static future<json::json_return_type> get_cf_stats_sum(http_context& ctx, const
|
||||
|
||||
|
||||
static future<json::json_return_type> get_cf_stats_count(http_context& ctx,
|
||||
utils::timed_rate_moving_average_and_histogram column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [f](const column_family& cf) {
|
||||
utils::timed_rate_moving_average_and_histogram replica::column_family_stats::*f) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [f](const replica::column_family& cf) {
|
||||
return (cf.get_stats().*f).hist.count;
|
||||
}, std::plus<int64_t>());
|
||||
}
|
||||
|
||||
static future<json::json_return_type> get_cf_histogram(http_context& ctx, const sstring& name,
|
||||
utils::timed_rate_moving_average_and_histogram column_family_stats::*f) {
|
||||
utils::timed_rate_moving_average_and_histogram replica::column_family_stats::*f) {
|
||||
utils::UUID uuid = get_uuid(name, ctx.db.local());
|
||||
return ctx.db.map_reduce0([f, uuid](const database& p) {
|
||||
return ctx.db.map_reduce0([f, uuid](const replica::database& p) {
|
||||
return (p.find_column_family(uuid).get_stats().*f).hist;},
|
||||
utils::ihistogram(),
|
||||
std::plus<utils::ihistogram>())
|
||||
@@ -133,8 +133,8 @@ static future<json::json_return_type> get_cf_histogram(http_context& ctx, const
|
||||
});
|
||||
}
|
||||
|
||||
static future<json::json_return_type> get_cf_histogram(http_context& ctx, utils::timed_rate_moving_average_and_histogram column_family_stats::*f) {
|
||||
std::function<utils::ihistogram(const database&)> fun = [f] (const database& db) {
|
||||
static future<json::json_return_type> get_cf_histogram(http_context& ctx, utils::timed_rate_moving_average_and_histogram replica::column_family_stats::*f) {
|
||||
std::function<utils::ihistogram(const replica::database&)> fun = [f] (const replica::database& db) {
|
||||
utils::ihistogram res;
|
||||
for (auto i : db.get_column_families()) {
|
||||
res += (i.second->get_stats().*f).hist;
|
||||
@@ -149,9 +149,9 @@ static future<json::json_return_type> get_cf_histogram(http_context& ctx, utils:
|
||||
}
|
||||
|
||||
static future<json::json_return_type> get_cf_rate_and_histogram(http_context& ctx, const sstring& name,
|
||||
utils::timed_rate_moving_average_and_histogram column_family_stats::*f) {
|
||||
utils::timed_rate_moving_average_and_histogram replica::column_family_stats::*f) {
|
||||
utils::UUID uuid = get_uuid(name, ctx.db.local());
|
||||
return ctx.db.map_reduce0([f, uuid](const database& p) {
|
||||
return ctx.db.map_reduce0([f, uuid](const replica::database& p) {
|
||||
return (p.find_column_family(uuid).get_stats().*f).rate();},
|
||||
utils::rate_moving_average_and_histogram(),
|
||||
std::plus<utils::rate_moving_average_and_histogram>())
|
||||
@@ -160,8 +160,8 @@ static future<json::json_return_type> get_cf_rate_and_histogram(http_context& c
|
||||
});
|
||||
}
|
||||
|
||||
static future<json::json_return_type> get_cf_rate_and_histogram(http_context& ctx, utils::timed_rate_moving_average_and_histogram column_family_stats::*f) {
|
||||
std::function<utils::rate_moving_average_and_histogram(const database&)> fun = [f] (const database& db) {
|
||||
static future<json::json_return_type> get_cf_rate_and_histogram(http_context& ctx, utils::timed_rate_moving_average_and_histogram replica::column_family_stats::*f) {
|
||||
std::function<utils::rate_moving_average_and_histogram(const replica::database&)> fun = [f] (const replica::database& db) {
|
||||
utils::rate_moving_average_and_histogram res;
|
||||
for (auto i : db.get_column_families()) {
|
||||
res += (i.second->get_stats().*f).rate();
|
||||
@@ -176,12 +176,12 @@ static future<json::json_return_type> get_cf_rate_and_histogram(http_context& ct
|
||||
}
|
||||
|
||||
static future<json::json_return_type> get_cf_unleveled_sstables(http_context& ctx, const sstring& name) {
|
||||
return map_reduce_cf(ctx, name, int64_t(0), [](const column_family& cf) {
|
||||
return map_reduce_cf(ctx, name, int64_t(0), [](const replica::column_family& cf) {
|
||||
return cf.get_unleveled_sstables();
|
||||
}, std::plus<int64_t>());
|
||||
}
|
||||
|
||||
static int64_t min_partition_size(column_family& cf) {
|
||||
static int64_t min_partition_size(replica::column_family& cf) {
|
||||
int64_t res = INT64_MAX;
|
||||
for (auto sstables = cf.get_sstables(); auto& i : *sstables) {
|
||||
res = std::min(res, i->get_stats_metadata().estimated_partition_size.min());
|
||||
@@ -189,7 +189,7 @@ static int64_t min_partition_size(column_family& cf) {
|
||||
return (res == INT64_MAX) ? 0 : res;
|
||||
}
|
||||
|
||||
static int64_t max_partition_size(column_family& cf) {
|
||||
static int64_t max_partition_size(replica::column_family& cf) {
|
||||
int64_t res = 0;
|
||||
for (auto sstables = cf.get_sstables(); auto& i : *sstables) {
|
||||
res = std::max(i->get_stats_metadata().estimated_partition_size.max(), res);
|
||||
@@ -197,7 +197,7 @@ static int64_t max_partition_size(column_family& cf) {
|
||||
return res;
|
||||
}
|
||||
|
||||
static integral_ratio_holder mean_partition_size(column_family& cf) {
|
||||
static integral_ratio_holder mean_partition_size(replica::column_family& cf) {
|
||||
integral_ratio_holder res;
|
||||
for (auto sstables = cf.get_sstables(); auto& i : *sstables) {
|
||||
auto c = i->get_stats_metadata().estimated_partition_size.count();
|
||||
@@ -223,7 +223,7 @@ static json::json_return_type sum_map(const std::unordered_map<sstring, uint64_t
|
||||
|
||||
static future<json::json_return_type> sum_sstable(http_context& ctx, const sstring name, bool total) {
|
||||
auto uuid = get_uuid(name, ctx.db.local());
|
||||
return ctx.db.map_reduce0([uuid, total](database& db) {
|
||||
return ctx.db.map_reduce0([uuid, total](replica::database& db) {
|
||||
std::unordered_map<sstring, uint64_t> m;
|
||||
auto sstables = (total) ? db.find_column_family(uuid).get_sstables_including_compacted_undeleted() :
|
||||
db.find_column_family(uuid).get_sstables();
|
||||
@@ -239,7 +239,7 @@ static future<json::json_return_type> sum_sstable(http_context& ctx, const sstr
|
||||
|
||||
|
||||
static future<json::json_return_type> sum_sstable(http_context& ctx, bool total) {
|
||||
return map_reduce_cf_raw(ctx, std::unordered_map<sstring, uint64_t>(), [total](column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, std::unordered_map<sstring, uint64_t>(), [total](replica::column_family& cf) {
|
||||
std::unordered_map<sstring, uint64_t> m;
|
||||
auto sstables = (total) ? cf.get_sstables_including_compacted_undeleted() :
|
||||
cf.get_sstables();
|
||||
@@ -252,7 +252,7 @@ static future<json::json_return_type> sum_sstable(http_context& ctx, bool total)
|
||||
});
|
||||
}
|
||||
|
||||
future<json::json_return_type> map_reduce_cf_time_histogram(http_context& ctx, const sstring& name, std::function<utils::time_estimated_histogram(const column_family&)> f) {
|
||||
future<json::json_return_type> map_reduce_cf_time_histogram(http_context& ctx, const sstring& name, std::function<utils::time_estimated_histogram(const replica::column_family&)> f) {
|
||||
return map_reduce_cf_raw(ctx, name, utils::time_estimated_histogram(), f, utils::time_estimated_histogram_merge).then([](const utils::time_estimated_histogram& res) {
|
||||
return make_ready_future<json::json_return_type>(time_to_json_histogram(res));
|
||||
});
|
||||
@@ -275,7 +275,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
static double get_compression_ratio(column_family& cf) {
|
||||
static double get_compression_ratio(replica::column_family& cf) {
|
||||
sum_ratio<double> result;
|
||||
for (auto sstables = cf.get_sstables(); auto& i : *sstables) {
|
||||
auto compression_ratio = i->get_compression_ratio();
|
||||
@@ -334,13 +334,13 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_memtable_columns_count.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t{0}, [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t{0}, [](replica::column_family& cf) {
|
||||
return cf.active_memtable().partition_count();
|
||||
}, std::plus<>());
|
||||
});
|
||||
|
||||
cf::get_all_memtable_columns_count.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t{0}, [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t{0}, [](replica::column_family& cf) {
|
||||
return cf.active_memtable().partition_count();
|
||||
}, std::plus<>());
|
||||
});
|
||||
@@ -354,25 +354,25 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_memtable_off_heap_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.active_memtable().region().occupancy().total_space();
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_all_memtable_off_heap_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.active_memtable().region().occupancy().total_space();
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_memtable_live_data_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.active_memtable().region().occupancy().used_space();
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_all_memtable_live_data_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.active_memtable().region().occupancy().used_space();
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
@@ -387,14 +387,14 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
|
||||
cf::get_cf_all_memtables_off_heap_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
warn(unimplemented::cause::INDEXES);
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.occupancy().total_space();
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_all_cf_all_memtables_off_heap_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
warn(unimplemented::cause::INDEXES);
|
||||
return ctx.db.map_reduce0([](const database& db){
|
||||
return ctx.db.map_reduce0([](const replica::database& db){
|
||||
return db.dirty_memory_region_group().memory_used();
|
||||
}, int64_t(0), std::plus<int64_t>()).then([](int res) {
|
||||
return make_ready_future<json::json_return_type>(res);
|
||||
@@ -403,29 +403,29 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
|
||||
cf::get_cf_all_memtables_live_data_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
warn(unimplemented::cause::INDEXES);
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.occupancy().used_space();
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_all_cf_all_memtables_live_data_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
warn(unimplemented::cause::INDEXES);
|
||||
return map_reduce_cf(ctx, int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.active_memtable().region().occupancy().used_space();
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_memtable_switch_count.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx,req->param["name"] ,&column_family_stats::memtable_switch_count);
|
||||
return get_cf_stats(ctx,req->param["name"] ,&replica::column_family_stats::memtable_switch_count);
|
||||
});
|
||||
|
||||
cf::get_all_memtable_switch_count.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx, &column_family_stats::memtable_switch_count);
|
||||
return get_cf_stats(ctx, &replica::column_family_stats::memtable_switch_count);
|
||||
});
|
||||
|
||||
// FIXME: this refers to partitions, not rows.
|
||||
cf::get_estimated_row_size_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], utils::estimated_histogram(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], utils::estimated_histogram(0), [](replica::column_family& cf) {
|
||||
utils::estimated_histogram res(0);
|
||||
for (auto sstables = cf.get_sstables(); auto& i : *sstables) {
|
||||
res.merge(i->get_stats_metadata().estimated_partition_size);
|
||||
@@ -437,7 +437,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
|
||||
// FIXME: this refers to partitions, not rows.
|
||||
cf::get_estimated_row_count.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](replica::column_family& cf) {
|
||||
uint64_t res = 0;
|
||||
for (auto sstables = cf.get_sstables(); auto& i : *sstables) {
|
||||
res += i->get_stats_metadata().estimated_partition_size.count();
|
||||
@@ -448,7 +448,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_estimated_column_count_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], utils::estimated_histogram(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], utils::estimated_histogram(0), [](replica::column_family& cf) {
|
||||
utils::estimated_histogram res(0);
|
||||
for (auto sstables = cf.get_sstables(); auto& i : *sstables) {
|
||||
res.merge(i->get_stats_metadata().estimated_cells_count);
|
||||
@@ -465,87 +465,87 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_pending_flushes.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx,req->param["name"] ,&column_family_stats::pending_flushes);
|
||||
return get_cf_stats(ctx,req->param["name"] ,&replica::column_family_stats::pending_flushes);
|
||||
});
|
||||
|
||||
cf::get_all_pending_flushes.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx, &column_family_stats::pending_flushes);
|
||||
return get_cf_stats(ctx, &replica::column_family_stats::pending_flushes);
|
||||
});
|
||||
|
||||
cf::get_read.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats_count(ctx,req->param["name"] ,&column_family_stats::reads);
|
||||
return get_cf_stats_count(ctx,req->param["name"] ,&replica::column_family_stats::reads);
|
||||
});
|
||||
|
||||
cf::get_all_read.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats_count(ctx, &column_family_stats::reads);
|
||||
return get_cf_stats_count(ctx, &replica::column_family_stats::reads);
|
||||
});
|
||||
|
||||
cf::get_write.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats_count(ctx, req->param["name"] ,&column_family_stats::writes);
|
||||
return get_cf_stats_count(ctx, req->param["name"] ,&replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_all_write.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats_count(ctx, &column_family_stats::writes);
|
||||
return get_cf_stats_count(ctx, &replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_read_latency_histogram_depricated.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_histogram(ctx, req->param["name"], &column_family_stats::reads);
|
||||
return get_cf_histogram(ctx, req->param["name"], &replica::column_family_stats::reads);
|
||||
});
|
||||
|
||||
cf::get_read_latency_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_rate_and_histogram(ctx, req->param["name"], &column_family_stats::reads);
|
||||
return get_cf_rate_and_histogram(ctx, req->param["name"], &replica::column_family_stats::reads);
|
||||
});
|
||||
|
||||
cf::get_read_latency.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats_sum(ctx,req->param["name"] ,&column_family_stats::reads);
|
||||
return get_cf_stats_sum(ctx,req->param["name"] ,&replica::column_family_stats::reads);
|
||||
});
|
||||
|
||||
cf::get_write_latency.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats_sum(ctx, req->param["name"] ,&column_family_stats::writes);
|
||||
return get_cf_stats_sum(ctx, req->param["name"] ,&replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_all_read_latency_histogram_depricated.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_histogram(ctx, &column_family_stats::writes);
|
||||
return get_cf_histogram(ctx, &replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_all_read_latency_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_rate_and_histogram(ctx, &column_family_stats::writes);
|
||||
return get_cf_rate_and_histogram(ctx, &replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_write_latency_histogram_depricated.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_histogram(ctx, req->param["name"], &column_family_stats::writes);
|
||||
return get_cf_histogram(ctx, req->param["name"], &replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_write_latency_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_rate_and_histogram(ctx, req->param["name"], &column_family_stats::writes);
|
||||
return get_cf_rate_and_histogram(ctx, req->param["name"], &replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_all_write_latency_histogram_depricated.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_histogram(ctx, &column_family_stats::writes);
|
||||
return get_cf_histogram(ctx, &replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_all_write_latency_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_rate_and_histogram(ctx, &column_family_stats::writes);
|
||||
return get_cf_rate_and_histogram(ctx, &replica::column_family_stats::writes);
|
||||
});
|
||||
|
||||
cf::get_pending_compactions.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.get_compaction_strategy().estimated_pending_compactions(cf.as_table_state());
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_all_pending_compactions.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.get_compaction_strategy().estimated_pending_compactions(cf.as_table_state());
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
cf::get_live_ss_table_count.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx, req->param["name"], &column_family_stats::live_sstable_count);
|
||||
return get_cf_stats(ctx, req->param["name"], &replica::column_family_stats::live_sstable_count);
|
||||
});
|
||||
|
||||
cf::get_all_live_ss_table_count.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx, &column_family_stats::live_sstable_count);
|
||||
return get_cf_stats(ctx, &replica::column_family_stats::live_sstable_count);
|
||||
});
|
||||
|
||||
cf::get_unleveled_sstables.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
@@ -601,7 +601,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_bloom_filter_false_positives.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_get_false_positive();
|
||||
@@ -610,7 +610,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_all_bloom_filter_false_positives.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_get_false_positive();
|
||||
@@ -619,7 +619,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_recent_bloom_filter_false_positives.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_get_recent_false_positive();
|
||||
@@ -628,7 +628,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_all_recent_bloom_filter_false_positives.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_get_recent_false_positive();
|
||||
@@ -637,31 +637,31 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], ratio_holder(), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], ratio_holder(), [] (replica::column_family& cf) {
|
||||
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_false_positive_as_ratio_holder), ratio_holder());
|
||||
}, std::plus<>());
|
||||
});
|
||||
|
||||
cf::get_all_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, ratio_holder(), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, ratio_holder(), [] (replica::column_family& cf) {
|
||||
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_false_positive_as_ratio_holder), ratio_holder());
|
||||
}, std::plus<>());
|
||||
});
|
||||
|
||||
cf::get_recent_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], ratio_holder(), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], ratio_holder(), [] (replica::column_family& cf) {
|
||||
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_recent_false_positive_as_ratio_holder), ratio_holder());
|
||||
}, std::plus<>());
|
||||
});
|
||||
|
||||
cf::get_all_recent_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, ratio_holder(), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, ratio_holder(), [] (replica::column_family& cf) {
|
||||
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_recent_false_positive_as_ratio_holder), ratio_holder());
|
||||
}, std::plus<>());
|
||||
});
|
||||
|
||||
cf::get_bloom_filter_disk_space_used.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_size();
|
||||
@@ -670,7 +670,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_all_bloom_filter_disk_space_used.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_size();
|
||||
@@ -679,7 +679,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_bloom_filter_off_heap_memory_used.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_memory_size();
|
||||
@@ -688,7 +688,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_all_bloom_filter_off_heap_memory_used.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->filter_memory_size();
|
||||
@@ -697,7 +697,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_index_summary_off_heap_memory_used.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->get_summary().memory_footprint();
|
||||
@@ -706,7 +706,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_all_index_summary_off_heap_memory_used.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (column_family& cf) {
|
||||
return map_reduce_cf(ctx, uint64_t(0), [] (replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
return std::accumulate(sstables->begin(), sstables->end(), uint64_t(0), [](uint64_t s, auto& sst) {
|
||||
return s + sst->get_summary().memory_footprint();
|
||||
@@ -753,7 +753,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
cf::get_true_snapshots_size.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
auto uuid = get_uuid(req->param["name"], ctx.db.local());
|
||||
return ctx.db.local().find_column_family(uuid).get_snapshot_details().then([](
|
||||
const std::unordered_map<sstring, column_family::snapshot_details>& sd) {
|
||||
const std::unordered_map<sstring, replica::column_family::snapshot_details>& sd) {
|
||||
int64_t res = 0;
|
||||
for (auto i : sd) {
|
||||
res += i.second.total;
|
||||
@@ -782,7 +782,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_row_cache_hit.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_raw(ctx, req->param["name"], utils::rate_moving_average(), [](const column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, req->param["name"], utils::rate_moving_average(), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().hits.rate();
|
||||
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
|
||||
return make_ready_future<json::json_return_type>(meter_to_json(m));
|
||||
@@ -790,7 +790,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_all_row_cache_hit.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().hits.rate();
|
||||
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
|
||||
return make_ready_future<json::json_return_type>(meter_to_json(m));
|
||||
@@ -798,7 +798,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_row_cache_miss.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_raw(ctx, req->param["name"], utils::rate_moving_average(), [](const column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, req->param["name"], utils::rate_moving_average(), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().misses.rate();
|
||||
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
|
||||
return make_ready_future<json::json_return_type>(meter_to_json(m));
|
||||
@@ -806,7 +806,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_all_row_cache_miss.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const replica::column_family& cf) {
|
||||
return cf.get_row_cache().stats().misses.rate();
|
||||
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
|
||||
return make_ready_future<json::json_return_type>(meter_to_json(m));
|
||||
@@ -815,36 +815,36 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_cas_prepare.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const column_family& cf) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const replica::column_family& cf) {
|
||||
return cf.get_stats().estimated_cas_prepare;
|
||||
});
|
||||
});
|
||||
|
||||
cf::get_cas_propose.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const column_family& cf) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const replica::column_family& cf) {
|
||||
return cf.get_stats().estimated_cas_accept;
|
||||
});
|
||||
});
|
||||
|
||||
cf::get_cas_commit.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const column_family& cf) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const replica::column_family& cf) {
|
||||
return cf.get_stats().estimated_cas_learn;
|
||||
});
|
||||
});
|
||||
|
||||
cf::get_sstables_per_read_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, req->param["name"], utils::estimated_histogram(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, req->param["name"], utils::estimated_histogram(0), [](replica::column_family& cf) {
|
||||
return cf.get_stats().estimated_sstable_per_read;
|
||||
},
|
||||
utils::estimated_histogram_merge, utils_json::estimated_histogram());
|
||||
});
|
||||
|
||||
cf::get_tombstone_scanned_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_histogram(ctx, req->param["name"], &column_family_stats::tombstone_scanned);
|
||||
return get_cf_histogram(ctx, req->param["name"], &replica::column_family_stats::tombstone_scanned);
|
||||
});
|
||||
|
||||
cf::get_live_scanned_histogram.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return get_cf_histogram(ctx, req->param["name"], &column_family_stats::live_scanned);
|
||||
return get_cf_histogram(ctx, req->param["name"], &replica::column_family_stats::live_scanned);
|
||||
});
|
||||
|
||||
cf::get_col_update_time_delta_histogram.set(r, [] (std::unique_ptr<request> req) {
|
||||
@@ -857,14 +857,14 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
|
||||
cf::get_auto_compaction.set(r, [&ctx] (const_req req) {
|
||||
const utils::UUID& uuid = get_uuid(req.param["name"], ctx.db.local());
|
||||
column_family& cf = ctx.db.local().find_column_family(uuid);
|
||||
replica::column_family& cf = ctx.db.local().find_column_family(uuid);
|
||||
return !cf.is_auto_compaction_disabled_by_user();
|
||||
});
|
||||
|
||||
cf::enable_auto_compaction.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return ctx.db.invoke_on(0, [&ctx, req = std::move(req)] (database& db) {
|
||||
auto g = database::autocompaction_toggle_guard(db);
|
||||
return foreach_column_family(ctx, req->param["name"], [](column_family &cf) {
|
||||
return ctx.db.invoke_on(0, [&ctx, req = std::move(req)] (replica::database& db) {
|
||||
auto g = replica::database::autocompaction_toggle_guard(db);
|
||||
return foreach_column_family(ctx, req->param["name"], [](replica::column_family &cf) {
|
||||
cf.enable_auto_compaction();
|
||||
}).then([g = std::move(g)] {
|
||||
return make_ready_future<json::json_return_type>(json_void());
|
||||
@@ -873,9 +873,9 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::disable_auto_compaction.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return ctx.db.invoke_on(0, [&ctx, req = std::move(req)] (database& db) {
|
||||
auto g = database::autocompaction_toggle_guard(db);
|
||||
return foreach_column_family(ctx, req->param["name"], [](column_family &cf) {
|
||||
return ctx.db.invoke_on(0, [&ctx, req = std::move(req)] (replica::database& db) {
|
||||
auto g = replica::database::autocompaction_toggle_guard(db);
|
||||
return foreach_column_family(ctx, req->param["name"], [](replica::column_family &cf) {
|
||||
return cf.disable_auto_compaction();
|
||||
}).then([g = std::move(g)] {
|
||||
return make_ready_future<json::json_return_type>(json_void());
|
||||
@@ -896,7 +896,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
}
|
||||
std::vector<sstring> res;
|
||||
auto uuid = get_uuid(ks, cf_name, ctx.db.local());
|
||||
column_family& cf = ctx.db.local().find_column_family(uuid);
|
||||
replica::column_family& cf = ctx.db.local().find_column_family(uuid);
|
||||
res.reserve(cf.get_index_manager().list_indexes().size());
|
||||
for (auto&& i : cf.get_index_manager().list_indexes()) {
|
||||
if (!vp.contains(secondary_index::index_table_name(i.metadata().name()))) {
|
||||
@@ -924,8 +924,8 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
cf::get_compression_ratio.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
auto uuid = get_uuid(req->param["name"], ctx.db.local());
|
||||
|
||||
return ctx.db.map_reduce(sum_ratio<double>(), [uuid](database& db) {
|
||||
column_family& cf = db.find_column_family(uuid);
|
||||
return ctx.db.map_reduce(sum_ratio<double>(), [uuid](replica::database& db) {
|
||||
replica::column_family& cf = db.find_column_family(uuid);
|
||||
return make_ready_future<double>(get_compression_ratio(cf));
|
||||
}).then([] (const double& result) {
|
||||
return make_ready_future<json::json_return_type>(result);
|
||||
@@ -933,20 +933,20 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_read_latency_estimated_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const column_family& cf) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const replica::column_family& cf) {
|
||||
return cf.get_stats().estimated_read;
|
||||
});
|
||||
});
|
||||
|
||||
cf::get_write_latency_estimated_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const column_family& cf) {
|
||||
return map_reduce_cf_time_histogram(ctx, req->param["name"], [](const replica::column_family& cf) {
|
||||
return cf.get_stats().estimated_write;
|
||||
});
|
||||
});
|
||||
|
||||
cf::set_compaction_strategy_class.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
sstring strategy = req->get_query_param("class_name");
|
||||
return foreach_column_family(ctx, req->param["name"], [strategy](column_family& cf) {
|
||||
return foreach_column_family(ctx, req->param["name"], [strategy](replica::column_family& cf) {
|
||||
cf.set_compaction_strategy(sstables::compaction_strategy::type(strategy));
|
||||
}).then([] {
|
||||
return make_ready_future<json::json_return_type>(json_void());
|
||||
@@ -970,7 +970,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cf::get_sstable_count_per_level.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return map_reduce_cf_raw(ctx, req->param["name"], std::vector<uint64_t>(), [](const column_family& cf) {
|
||||
return map_reduce_cf_raw(ctx, req->param["name"], std::vector<uint64_t>(), [](const replica::column_family& cf) {
|
||||
return cf.sstable_count_per_level();
|
||||
}, concat_sstable_count_per_level).then([](const std::vector<uint64_t>& res) {
|
||||
return make_ready_future<json::json_return_type>(res);
|
||||
@@ -981,7 +981,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
auto key = req->get_query_param("key");
|
||||
auto uuid = get_uuid(req->param["name"], ctx.db.local());
|
||||
|
||||
return ctx.db.map_reduce0([key, uuid] (database& db) {
|
||||
return ctx.db.map_reduce0([key, uuid] (replica::database& db) {
|
||||
return db.find_column_family(uuid).get_sstables_by_partition_key(key);
|
||||
}, std::unordered_set<sstring>(),
|
||||
[](std::unordered_set<sstring> a, std::unordered_set<sstring>&& b) mutable {
|
||||
@@ -1013,7 +1013,7 @@ void set_column_family(http_context& ctx, routes& r) {
|
||||
if (req->get_query_param("split_output") != "") {
|
||||
fail(unimplemented::cause::API);
|
||||
}
|
||||
return foreach_column_family(ctx, req->param["name"], [](column_family &cf) {
|
||||
return foreach_column_family(ctx, req->param["name"], [](replica::column_family &cf) {
|
||||
return cf.compact_all_sstables();
|
||||
}).then([] {
|
||||
return make_ready_future<json::json_return_type>(json_void());
|
||||
|
||||
@@ -31,17 +31,17 @@ namespace api {
|
||||
|
||||
void set_column_family(http_context& ctx, routes& r);
|
||||
|
||||
const utils::UUID& get_uuid(const sstring& name, const database& db);
|
||||
future<> foreach_column_family(http_context& ctx, const sstring& name, std::function<void(column_family&)> f);
|
||||
const utils::UUID& get_uuid(const sstring& name, const replica::database& db);
|
||||
future<> foreach_column_family(http_context& ctx, const sstring& name, std::function<void(replica::column_family&)> f);
|
||||
|
||||
|
||||
template<class Mapper, class I, class Reducer>
|
||||
future<I> map_reduce_cf_raw(http_context& ctx, const sstring& name, I init,
|
||||
Mapper mapper, Reducer reducer) {
|
||||
auto uuid = get_uuid(name, ctx.db.local());
|
||||
using mapper_type = std::function<std::unique_ptr<std::any>(database&)>;
|
||||
using mapper_type = std::function<std::unique_ptr<std::any>(replica::database&)>;
|
||||
using reducer_type = std::function<std::unique_ptr<std::any>(std::unique_ptr<std::any>, std::unique_ptr<std::any>)>;
|
||||
return ctx.db.map_reduce0(mapper_type([mapper, uuid](database& db) {
|
||||
return ctx.db.map_reduce0(mapper_type([mapper, uuid](replica::database& db) {
|
||||
return std::make_unique<std::any>(I(mapper(db.find_column_family(uuid))));
|
||||
}), std::make_unique<std::any>(std::move(init)), reducer_type([reducer = std::move(reducer)] (std::unique_ptr<std::any> a, std::unique_ptr<std::any> b) mutable {
|
||||
return std::make_unique<std::any>(I(reducer(std::any_cast<I>(std::move(*a)), std::any_cast<I>(std::move(*b)))));
|
||||
@@ -68,15 +68,15 @@ future<json::json_return_type> map_reduce_cf(http_context& ctx, const sstring& n
|
||||
});
|
||||
}
|
||||
|
||||
future<json::json_return_type> map_reduce_cf_time_histogram(http_context& ctx, const sstring& name, std::function<utils::time_estimated_histogram(const column_family&)> f);
|
||||
future<json::json_return_type> map_reduce_cf_time_histogram(http_context& ctx, const sstring& name, std::function<utils::time_estimated_histogram(const replica::column_family&)> f);
|
||||
|
||||
struct map_reduce_column_families_locally {
|
||||
std::any init;
|
||||
std::function<std::unique_ptr<std::any>(column_family&)> mapper;
|
||||
std::function<std::unique_ptr<std::any>(replica::column_family&)> mapper;
|
||||
std::function<std::unique_ptr<std::any>(std::unique_ptr<std::any>, std::unique_ptr<std::any>)> reducer;
|
||||
future<std::unique_ptr<std::any>> operator()(database& db) const {
|
||||
future<std::unique_ptr<std::any>> operator()(replica::database& db) const {
|
||||
auto res = seastar::make_lw_shared<std::unique_ptr<std::any>>(std::make_unique<std::any>(init));
|
||||
return do_for_each(db.get_column_families(), [res, this](const std::pair<utils::UUID, seastar::lw_shared_ptr<table>>& i) {
|
||||
return do_for_each(db.get_column_families(), [res, this](const std::pair<utils::UUID, seastar::lw_shared_ptr<replica::table>>& i) {
|
||||
*res = reducer(std::move(*res), mapper(*i.second.get()));
|
||||
}).then([res] {
|
||||
return std::move(*res);
|
||||
@@ -87,9 +87,9 @@ struct map_reduce_column_families_locally {
|
||||
template<class Mapper, class I, class Reducer>
|
||||
future<I> map_reduce_cf_raw(http_context& ctx, I init,
|
||||
Mapper mapper, Reducer reducer) {
|
||||
using mapper_type = std::function<std::unique_ptr<std::any>(column_family&)>;
|
||||
using mapper_type = std::function<std::unique_ptr<std::any>(replica::column_family&)>;
|
||||
using reducer_type = std::function<std::unique_ptr<std::any>(std::unique_ptr<std::any>, std::unique_ptr<std::any>)>;
|
||||
auto wrapped_mapper = mapper_type([mapper = std::move(mapper)] (column_family& cf) mutable {
|
||||
auto wrapped_mapper = mapper_type([mapper = std::move(mapper)] (replica::column_family& cf) mutable {
|
||||
return std::make_unique<std::any>(I(mapper(cf)));
|
||||
});
|
||||
auto wrapped_reducer = reducer_type([reducer = std::move(reducer)] (std::unique_ptr<std::any> a, std::unique_ptr<std::any> b) mutable {
|
||||
@@ -111,10 +111,10 @@ future<json::json_return_type> map_reduce_cf(http_context& ctx, I init,
|
||||
}
|
||||
|
||||
future<json::json_return_type> get_cf_stats(http_context& ctx, const sstring& name,
|
||||
int64_t column_family_stats::*f);
|
||||
int64_t replica::column_family_stats::*f);
|
||||
|
||||
future<json::json_return_type> get_cf_stats(http_context& ctx,
|
||||
int64_t column_family_stats::*f);
|
||||
int64_t replica::column_family_stats::*f);
|
||||
|
||||
|
||||
std::tuple<sstring, sstring> parse_fully_qualified_cf_name(sstring name);
|
||||
|
||||
@@ -31,7 +31,7 @@ template<typename T>
|
||||
static auto acquire_cl_metric(http_context& ctx, std::function<T (db::commitlog*)> func) {
|
||||
typedef T ret_type;
|
||||
|
||||
return ctx.db.map_reduce0([func = std::move(func)](database& db) {
|
||||
return ctx.db.map_reduce0([func = std::move(func)](replica::database& db) {
|
||||
if (db.commitlog() == nullptr) {
|
||||
return make_ready_future<ret_type>();
|
||||
}
|
||||
@@ -47,7 +47,7 @@ void set_commitlog(http_context& ctx, routes& r) {
|
||||
auto res = make_shared<std::vector<sstring>>();
|
||||
return ctx.db.map_reduce([res](std::vector<sstring> names) {
|
||||
res->insert(res->end(), names.begin(), names.end());
|
||||
}, [](database& db) {
|
||||
}, [](replica::database& db) {
|
||||
if (db.commitlog() == nullptr) {
|
||||
return make_ready_future<std::vector<sstring>>(std::vector<sstring>());
|
||||
}
|
||||
|
||||
@@ -38,7 +38,7 @@ using namespace json;
|
||||
|
||||
static future<json::json_return_type> get_cm_stats(http_context& ctx,
|
||||
int64_t compaction_manager::stats::*f) {
|
||||
return ctx.db.map_reduce0([f](database& db) {
|
||||
return ctx.db.map_reduce0([f](replica::database& db) {
|
||||
return db.get_compaction_manager().get_stats().*f;
|
||||
}, int64_t(0), std::plus<int64_t>()).then([](const int64_t& res) {
|
||||
return make_ready_future<json::json_return_type>(res);
|
||||
@@ -57,7 +57,7 @@ static std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_ha
|
||||
|
||||
void set_compaction_manager(http_context& ctx, routes& r) {
|
||||
cm::get_compactions.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return ctx.db.map_reduce0([](database& db) {
|
||||
return ctx.db.map_reduce0([](replica::database& db) {
|
||||
std::vector<cm::summary> summaries;
|
||||
const compaction_manager& cm = db.get_compaction_manager();
|
||||
|
||||
@@ -79,10 +79,10 @@ void set_compaction_manager(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cm::get_pending_tasks_by_table.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return ctx.db.map_reduce0([&ctx](database& db) {
|
||||
return ctx.db.map_reduce0([&ctx](replica::database& db) {
|
||||
return do_with(std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>(), [&ctx, &db](std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>& tasks) {
|
||||
return do_for_each(db.get_column_families(), [&tasks](const std::pair<utils::UUID, seastar::lw_shared_ptr<table>>& i) {
|
||||
table& cf = *i.second.get();
|
||||
return do_for_each(db.get_column_families(), [&tasks](const std::pair<utils::UUID, seastar::lw_shared_ptr<replica::table>>& i) {
|
||||
replica::table& cf = *i.second.get();
|
||||
tasks[std::make_pair(cf.schema()->ks_name(), cf.schema()->cf_name())] = cf.get_compaction_strategy().estimated_pending_compactions(cf.as_table_state());
|
||||
return make_ready_future<>();
|
||||
}).then([&tasks] {
|
||||
@@ -113,7 +113,7 @@ void set_compaction_manager(http_context& ctx, routes& r) {
|
||||
|
||||
cm::stop_compaction.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
auto type = req->get_query_param("type");
|
||||
return ctx.db.invoke_on_all([type] (database& db) {
|
||||
return ctx.db.invoke_on_all([type] (replica::database& db) {
|
||||
auto& cm = db.get_compaction_manager();
|
||||
return cm.stop_compaction(type);
|
||||
}).then([] {
|
||||
@@ -128,7 +128,7 @@ void set_compaction_manager(http_context& ctx, routes& r) {
|
||||
table_names = map_keys(ctx.db.local().find_keyspace(ks_name).metadata().get()->cf_meta_data());
|
||||
}
|
||||
auto type = req->get_query_param("type");
|
||||
co_await ctx.db.invoke_on_all([&ks_name, &table_names, type] (database& db) {
|
||||
co_await ctx.db.invoke_on_all([&ks_name, &table_names, type] (replica::database& db) {
|
||||
auto& cm = db.get_compaction_manager();
|
||||
return parallel_for_each(table_names, [&db, &cm, &ks_name, type] (sstring& table_name) {
|
||||
auto& t = db.find_column_family(ks_name, table_name);
|
||||
@@ -139,7 +139,7 @@ void set_compaction_manager(http_context& ctx, routes& r) {
|
||||
});
|
||||
|
||||
cm::get_pending_tasks.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](column_family& cf) {
|
||||
return map_reduce_cf(ctx, int64_t(0), [](replica::column_family& cf) {
|
||||
return cf.get_compaction_strategy().estimated_pending_compactions(cf.as_table_state());
|
||||
}, std::plus<int64_t>());
|
||||
});
|
||||
|
||||
@@ -35,7 +35,7 @@ static logging::logger alogger("lsa-api");
|
||||
void set_lsa(http_context& ctx, routes& r) {
|
||||
httpd::lsa_json::lsa_compact.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
alogger.info("Triggering compaction");
|
||||
return ctx.db.invoke_on_all([] (database&) {
|
||||
return ctx.db.invoke_on_all([] (replica::database&) {
|
||||
logalloc::shard_tracker().reclaim(std::numeric_limits<size_t>::max());
|
||||
}).then([] {
|
||||
return json::json_return_type(json::json_void());
|
||||
|
||||
@@ -74,7 +74,7 @@ sstring validate_keyspace(http_context& ctx, const parameters& param) {
|
||||
if (ctx.db.local().has_keyspace(ks_name)) {
|
||||
return ks_name;
|
||||
}
|
||||
throw bad_param_exception(no_such_keyspace(ks_name).what());
|
||||
throw bad_param_exception(replica::no_such_keyspace(ks_name).what());
|
||||
}
|
||||
|
||||
// splits a request parameter assumed to hold a comma-separated list of table names
|
||||
@@ -90,7 +90,7 @@ std::vector<sstring> parse_tables(const sstring& ks_name, http_context& ctx, con
|
||||
for (const auto& table_name : names) {
|
||||
ctx.db.local().find_column_family(ks_name, table_name);
|
||||
}
|
||||
} catch (const no_such_column_family& e) {
|
||||
} catch (const replica::no_such_column_family& e) {
|
||||
throw bad_param_exception(e.what());
|
||||
}
|
||||
return names;
|
||||
@@ -165,11 +165,11 @@ future<json::json_return_type> set_tables_autocompaction(http_context& ctx, serv
|
||||
|
||||
apilog.info("set_tables_autocompaction: enabled={} keyspace={} tables={}", enabled, keyspace, tables);
|
||||
return do_with(keyspace, std::move(tables), [&ctx, enabled] (const sstring &keyspace, const std::vector<sstring>& tables) {
|
||||
return ctx.db.invoke_on(0, [&ctx, &keyspace, &tables, enabled] (database& db) {
|
||||
auto g = database::autocompaction_toggle_guard(db);
|
||||
return ctx.db.invoke_on_all([&keyspace, &tables, enabled] (database& db) {
|
||||
return ctx.db.invoke_on(0, [&ctx, &keyspace, &tables, enabled] (replica::database& db) {
|
||||
auto g = replica::database::autocompaction_toggle_guard(db);
|
||||
return ctx.db.invoke_on_all([&keyspace, &tables, enabled] (replica::database& db) {
|
||||
return parallel_for_each(tables, [&db, &keyspace, enabled] (const sstring& table) {
|
||||
column_family& cf = db.find_column_family(keyspace, table);
|
||||
replica::column_family& cf = db.find_column_family(keyspace, table);
|
||||
if (enabled) {
|
||||
cf.enable_auto_compaction();
|
||||
} else {
|
||||
@@ -562,7 +562,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
});
|
||||
|
||||
ss::get_load.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx, &column_family_stats::live_disk_space_used);
|
||||
return get_cf_stats(ctx, &replica::column_family_stats::live_disk_space_used);
|
||||
});
|
||||
|
||||
ss::get_load_map.set(r, [&ctx] (std::unique_ptr<request> req) {
|
||||
@@ -606,7 +606,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
if (column_families.empty()) {
|
||||
column_families = map_keys(ctx.db.local().find_keyspace(keyspace).metadata().get()->cf_meta_data());
|
||||
}
|
||||
return ctx.db.invoke_on_all([keyspace, column_families] (database& db) -> future<> {
|
||||
return ctx.db.invoke_on_all([keyspace, column_families] (replica::database& db) -> future<> {
|
||||
auto table_ids = boost::copy_range<std::vector<utils::UUID>>(column_families | boost::adaptors::transformed([&] (auto& cf_name) {
|
||||
return db.find_uuid(keyspace, cf_name);
|
||||
}));
|
||||
@@ -636,7 +636,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
return make_exception_future<json::json_return_type>(
|
||||
std::runtime_error("Can not perform cleanup operation when topology changes"));
|
||||
}
|
||||
return ctx.db.invoke_on_all([keyspace, column_families] (database& db) -> future<> {
|
||||
return ctx.db.invoke_on_all([keyspace, column_families] (replica::database& db) -> future<> {
|
||||
auto table_ids = boost::copy_range<std::vector<utils::UUID>>(column_families | boost::adaptors::transformed([&] (auto& table_name) {
|
||||
return db.find_uuid(keyspace, table_name);
|
||||
}));
|
||||
@@ -647,7 +647,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
auto& cm = db.get_compaction_manager();
|
||||
// as a table can be dropped during loop below, let's find it before issuing the cleanup request.
|
||||
for (auto& id : table_ids) {
|
||||
table& t = db.find_column_family(id);
|
||||
replica::table& t = db.find_column_family(id);
|
||||
co_await cm.perform_cleanup(db, &t);
|
||||
}
|
||||
co_return;
|
||||
@@ -660,7 +660,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
ss::upgrade_sstables.set(r, wrap_ks_cf(ctx, [] (http_context& ctx, std::unique_ptr<request> req, sstring keyspace, std::vector<sstring> column_families) {
|
||||
bool exclude_current_version = req_param<bool>(*req, "exclude_current_version", false);
|
||||
|
||||
return ctx.db.invoke_on_all([=] (database& db) {
|
||||
return ctx.db.invoke_on_all([=] (replica::database& db) {
|
||||
return do_for_each(column_families, [=, &db](sstring cfname) {
|
||||
auto& cm = db.get_compaction_manager();
|
||||
auto& cf = db.find_column_family(keyspace, cfname);
|
||||
@@ -677,7 +677,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
if (column_families.empty()) {
|
||||
column_families = map_keys(ctx.db.local().find_keyspace(keyspace).metadata().get()->cf_meta_data());
|
||||
}
|
||||
return ctx.db.invoke_on_all([keyspace, column_families] (database& db) {
|
||||
return ctx.db.invoke_on_all([keyspace, column_families] (replica::database& db) {
|
||||
return parallel_for_each(column_families, [&db, keyspace](const sstring& cf) mutable {
|
||||
return db.find_column_family(keyspace, cf).flush();
|
||||
});
|
||||
@@ -766,7 +766,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
});
|
||||
|
||||
ss::get_drain_progress.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return ctx.db.map_reduce(adder<database::drain_progress>(), [] (auto& db) {
|
||||
return ctx.db.map_reduce(adder<replica::database::drain_progress>(), [] (auto& db) {
|
||||
return db.get_drain_progress();
|
||||
}).then([] (auto&& progress) {
|
||||
auto progress_str = format("Drained {}/{} ColumnFamilies", progress.remaining_cfs, progress.total_cfs);
|
||||
@@ -873,7 +873,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
ss::is_incremental_backups_enabled.set(r, [&ctx](std::unique_ptr<request> 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.
|
||||
return ctx.db.map_reduce(adder<bool>(), [] (database& db) {
|
||||
return ctx.db.map_reduce(adder<bool>(), [] (replica::database& db) {
|
||||
for (auto& pair: db.get_keyspaces()) {
|
||||
auto& ks = pair.second;
|
||||
if (ks.incremental_backups_enabled()) {
|
||||
@@ -889,7 +889,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
ss::set_incremental_backups_enabled.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
auto val_str = req->get_query_param("value");
|
||||
bool value = (val_str == "True") || (val_str == "true") || (val_str == "1");
|
||||
return ctx.db.invoke_on_all([value] (database& db) {
|
||||
return ctx.db.invoke_on_all([value] (replica::database& db) {
|
||||
db.set_enable_incremental_backups(value);
|
||||
|
||||
// Change both KS and CF, so they are in sync
|
||||
@@ -1087,7 +1087,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
});
|
||||
|
||||
ss::get_metrics_load.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
return get_cf_stats(ctx, &column_family_stats::live_disk_space_used);
|
||||
return get_cf_stats(ctx, &replica::column_family_stats::live_disk_space_used);
|
||||
});
|
||||
|
||||
ss::get_exceptions.set(r, [&ss](const_req req) {
|
||||
@@ -1149,7 +1149,7 @@ void set_storage_service(http_context& ctx, routes& r, sharded<service::storage_
|
||||
}
|
||||
}
|
||||
}
|
||||
}, [ks, cf](const database& db) {
|
||||
}, [ks, cf](const replica::database& db) {
|
||||
// see above
|
||||
table_sstables_list res;
|
||||
|
||||
@@ -1374,7 +1374,7 @@ void set_snapshot(http_context& ctx, routes& r, sharded<db::snapshot_ctl>& snap_
|
||||
throw std::invalid_argument(fmt::format("Unknown argument for 'quarantine_mode' parameter: {}", quarantine_mode_str));
|
||||
}
|
||||
return f.then([&ctx, keyspace, column_families, opts] {
|
||||
return ctx.db.invoke_on_all([=] (database& db) {
|
||||
return ctx.db.invoke_on_all([=] (replica::database& db) {
|
||||
return do_for_each(column_families, [=, &db](sstring cfname) {
|
||||
auto& cm = db.get_compaction_manager();
|
||||
auto& cf = db.find_column_family(keyspace, cfname);
|
||||
|
||||
@@ -76,7 +76,7 @@ void set_system(http_context& ctx, routes& r) {
|
||||
|
||||
hs::drop_sstable_caches.set(r, [&ctx](std::unique_ptr<request> req) {
|
||||
apilog.info("Dropping sstable caches");
|
||||
return ctx.db.invoke_on_all([] (database& db) {
|
||||
return ctx.db.invoke_on_all([] (replica::database& db) {
|
||||
return db.drop_caches();
|
||||
}).then([] {
|
||||
apilog.info("Caches dropped");
|
||||
|
||||
@@ -92,12 +92,12 @@ future<> create_metadata_table_if_missing(
|
||||
return futurize_invoke(create_metadata_table_if_missing_impl, table_name, qp, cql, mm);
|
||||
}
|
||||
|
||||
future<> wait_for_schema_agreement(::service::migration_manager& mm, const database& db, seastar::abort_source& as) {
|
||||
future<> wait_for_schema_agreement(::service::migration_manager& mm, const replica::database& db, seastar::abort_source& as) {
|
||||
static const auto pause = [] { return sleep(std::chrono::milliseconds(500)); };
|
||||
|
||||
return do_until([&db, &as] {
|
||||
as.check();
|
||||
return db.get_version() != database::empty_version;
|
||||
return db.get_version() != replica::database::empty_version;
|
||||
}, pause).then([&mm, &as] {
|
||||
return do_until([&mm, &as] {
|
||||
as.check();
|
||||
|
||||
@@ -39,7 +39,10 @@
|
||||
|
||||
using namespace std::chrono_literals;
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
class timeout_config;
|
||||
|
||||
namespace service {
|
||||
@@ -79,7 +82,7 @@ future<> create_metadata_table_if_missing(
|
||||
std::string_view cql,
|
||||
::service::migration_manager&) noexcept;
|
||||
|
||||
future<> wait_for_schema_agreement(::service::migration_manager&, const database&, seastar::abort_source&);
|
||||
future<> wait_for_schema_agreement(::service::migration_manager&, const replica::database&, seastar::abort_source&);
|
||||
|
||||
///
|
||||
/// Time-outs for internal, non-local CQL queries.
|
||||
|
||||
@@ -684,7 +684,7 @@ constexpr char could_not_retrieve_msg_template[]
|
||||
generation_service::generation_service(
|
||||
config cfg, gms::gossiper& g, sharded<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
abort_source& abort_src, const locator::shared_token_metadata& stm, gms::feature_service& f,
|
||||
database& db)
|
||||
replica::database& db)
|
||||
: _cfg(std::move(cfg))
|
||||
, _gossiper(g)
|
||||
, _sys_dist_ks(sys_dist_ks)
|
||||
|
||||
@@ -68,7 +68,7 @@ private:
|
||||
abort_source& _abort_src;
|
||||
const locator::shared_token_metadata& _token_metadata;
|
||||
gms::feature_service& _feature_service;
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
|
||||
/* Maintains the set of known CDC generations used to pick streams for log writes (i.e., the partition keys of these log writes).
|
||||
* Updated in response to certain gossip events (see the handle_cdc_generation function).
|
||||
@@ -92,7 +92,7 @@ private:
|
||||
public:
|
||||
generation_service(config cfg, gms::gossiper&,
|
||||
sharded<db::system_distributed_keyspace>&, abort_source&, const locator::shared_token_metadata&,
|
||||
gms::feature_service&, database& db);
|
||||
gms::feature_service&, replica::database& db);
|
||||
|
||||
future<> stop();
|
||||
~generation_service();
|
||||
|
||||
10
cdc/log.cc
10
cdc/log.cc
@@ -254,7 +254,7 @@ public:
|
||||
future<> append_mutations(Iter i, Iter e, schema_ptr s, lowres_clock::time_point, std::vector<mutation>&);
|
||||
|
||||
private:
|
||||
static void check_for_attempt_to_create_nested_cdc_log(database& db, const schema& schema) {
|
||||
static void check_for_attempt_to_create_nested_cdc_log(replica::database& db, const schema& schema) {
|
||||
const auto& cf_name = schema.cf_name();
|
||||
const auto cf_name_view = std::string_view(cf_name.data(), cf_name.size());
|
||||
if (is_log_for_some_table(db, schema.ks_name(), cf_name_view)) {
|
||||
@@ -263,7 +263,7 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
static void check_that_cdc_log_table_does_not_exist(database& db, const schema& schema, const sstring& logname) {
|
||||
static void check_that_cdc_log_table_does_not_exist(replica::database& db, const schema& schema, const sstring& logname) {
|
||||
if (db.has_schema(schema.ks_name(), logname)) {
|
||||
throw exceptions::invalid_request_exception(format("Cannot create CDC log table for table {}.{} because a table of name {}.{} already exists",
|
||||
schema.ks_name(), schema.cf_name(),
|
||||
@@ -428,7 +428,7 @@ bool is_cdc_metacolumn_name(const sstring& name) {
|
||||
return name.compare(0, cdc_meta_column_prefix.size(), cdc_meta_column_prefix) == 0;
|
||||
}
|
||||
|
||||
bool is_log_for_some_table(const database& db, const sstring& ks_name, const std::string_view& table_name) {
|
||||
bool is_log_for_some_table(const replica::database& db, const sstring& ks_name, const std::string_view& table_name) {
|
||||
auto base_schema = get_base_table(db, ks_name, table_name);
|
||||
if (!base_schema) {
|
||||
return false;
|
||||
@@ -436,11 +436,11 @@ bool is_log_for_some_table(const database& db, const sstring& ks_name, const std
|
||||
return base_schema->cdc_options().enabled();
|
||||
}
|
||||
|
||||
schema_ptr get_base_table(const database& db, const schema& s) {
|
||||
schema_ptr get_base_table(const replica::database& db, const schema& s) {
|
||||
return get_base_table(db, s.ks_name(), s.cf_name());
|
||||
}
|
||||
|
||||
schema_ptr get_base_table(const database& db, sstring_view ks_name,std::string_view table_name) {
|
||||
schema_ptr get_base_table(const replica::database& db, sstring_view ks_name,std::string_view table_name) {
|
||||
if (!is_log_name(table_name)) {
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
@@ -62,7 +62,10 @@ class query_state;
|
||||
|
||||
class mutation;
|
||||
class partition_key;
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace cdc {
|
||||
|
||||
@@ -114,10 +117,10 @@ enum class operation : int8_t {
|
||||
post_image = 9,
|
||||
};
|
||||
|
||||
bool is_log_for_some_table(const database& db, const sstring& ks_name, const std::string_view& table_name);
|
||||
bool is_log_for_some_table(const replica::database& db, const sstring& ks_name, const std::string_view& table_name);
|
||||
|
||||
schema_ptr get_base_table(const database&, const schema&);
|
||||
schema_ptr get_base_table(const database&, sstring_view, std::string_view);
|
||||
schema_ptr get_base_table(const replica::database&, const schema&);
|
||||
schema_ptr get_base_table(const replica::database&, sstring_view, std::string_view);
|
||||
|
||||
seastar::sstring base_name(std::string_view log_name);
|
||||
seastar::sstring log_name(std::string_view table_name);
|
||||
|
||||
@@ -162,7 +162,7 @@ unsigned compaction_manager::current_compaction_fan_in_threshold() const {
|
||||
return std::min(unsigned(32), largest_fan_in);
|
||||
}
|
||||
|
||||
bool compaction_manager::can_register_compaction(table* t, int weight, unsigned fan_in) const {
|
||||
bool compaction_manager::can_register_compaction(replica::table* t, int weight, unsigned fan_in) const {
|
||||
// Only one weight is allowed if parallel compaction is disabled.
|
||||
if (!t->get_compaction_strategy().parallel_compaction() && has_table_ongoing_compaction(t)) {
|
||||
return false;
|
||||
@@ -194,7 +194,7 @@ void compaction_manager::deregister_weight(int weight) {
|
||||
reevaluate_postponed_compactions();
|
||||
}
|
||||
|
||||
std::vector<sstables::shared_sstable> compaction_manager::get_candidates(const table& t) {
|
||||
std::vector<sstables::shared_sstable> compaction_manager::get_candidates(const replica::table& t) {
|
||||
std::vector<sstables::shared_sstable> candidates;
|
||||
candidates.reserve(t.sstables_count());
|
||||
// prevents sstables that belongs to a partial run being generated by ongoing compaction from being
|
||||
@@ -254,7 +254,7 @@ private:
|
||||
virtual void remove_sstable(sstables::shared_sstable sst) override { }
|
||||
};
|
||||
|
||||
compaction_manager::compaction_state& compaction_manager::get_compaction_state(table* t) {
|
||||
compaction_manager::compaction_state& compaction_manager::get_compaction_state(replica::table* t) {
|
||||
try {
|
||||
return _compaction_state.at(t);
|
||||
} catch (std::out_of_range&) {
|
||||
@@ -263,7 +263,7 @@ compaction_manager::compaction_state& compaction_manager::get_compaction_state(t
|
||||
}
|
||||
}
|
||||
|
||||
future<> compaction_manager::perform_major_compaction(table* t) {
|
||||
future<> compaction_manager::perform_major_compaction(replica::table* t) {
|
||||
if (_state != state::enabled) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
@@ -319,7 +319,7 @@ future<> compaction_manager::perform_major_compaction(table* t) {
|
||||
return task->compaction_done.get_future().then([task] {});
|
||||
}
|
||||
|
||||
future<> compaction_manager::run_custom_job(table* t, sstables::compaction_type type, noncopyable_function<future<>(sstables::compaction_data&)> job) {
|
||||
future<> compaction_manager::run_custom_job(replica::table* t, sstables::compaction_type type, noncopyable_function<future<>(sstables::compaction_data&)> job) {
|
||||
if (_state != state::enabled) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
@@ -362,7 +362,7 @@ future<> compaction_manager::run_custom_job(table* t, sstables::compaction_type
|
||||
}
|
||||
|
||||
future<>
|
||||
compaction_manager::run_with_compaction_disabled(table* t, std::function<future<> ()> func) {
|
||||
compaction_manager::run_with_compaction_disabled(replica::table* t, std::function<future<> ()> func) {
|
||||
auto& c_state = _compaction_state[t];
|
||||
auto holder = c_state.gate.hold();
|
||||
|
||||
@@ -528,7 +528,7 @@ void compaction_manager::reevaluate_postponed_compactions() {
|
||||
_postponed_reevaluation.signal();
|
||||
}
|
||||
|
||||
void compaction_manager::postpone_compaction_for_table(table* t) {
|
||||
void compaction_manager::postpone_compaction_for_table(replica::table* t) {
|
||||
_postponed.insert(t);
|
||||
}
|
||||
|
||||
@@ -554,7 +554,7 @@ future<> compaction_manager::stop_tasks(std::vector<lw_shared_ptr<task>> tasks,
|
||||
});
|
||||
}
|
||||
|
||||
future<> compaction_manager::stop_ongoing_compactions(sstring reason, table* t, std::optional<sstables::compaction_type> type_opt) {
|
||||
future<> compaction_manager::stop_ongoing_compactions(sstring reason, replica::table* t, std::optional<sstables::compaction_type> type_opt) {
|
||||
auto ongoing_compactions = get_compactions(t).size();
|
||||
auto tasks = boost::copy_range<std::vector<lw_shared_ptr<task>>>(_tasks | boost::adaptors::filtered([t, type_opt] (auto& task) {
|
||||
return (!t || task->compacting_table == t) && (!type_opt || task->type == *type_opt);
|
||||
@@ -657,7 +657,7 @@ inline bool compaction_manager::maybe_stop_on_error(std::exception_ptr err, bool
|
||||
return retry;
|
||||
}
|
||||
|
||||
void compaction_manager::submit(table* t) {
|
||||
void compaction_manager::submit(replica::table* t) {
|
||||
if (t->is_auto_compaction_disabled_by_user()) {
|
||||
return;
|
||||
}
|
||||
@@ -674,7 +674,7 @@ void compaction_manager::submit(table* t) {
|
||||
}
|
||||
return with_lock(task->compaction_state.lock.for_read(), [this, task] () mutable {
|
||||
return with_scheduling_group(_compaction_controller.sg(), [this, task = std::move(task)] () mutable {
|
||||
table& t = *task->compacting_table;
|
||||
replica::table& t = *task->compacting_table;
|
||||
sstables::compaction_strategy cs = t.get_compaction_strategy();
|
||||
sstables::compaction_descriptor descriptor = cs.get_sstables_for_compaction(t.as_table_state(), get_strategy_control(), get_candidates(t));
|
||||
int weight = calculate_weight(descriptor);
|
||||
@@ -730,7 +730,7 @@ void compaction_manager::submit(table* t) {
|
||||
});
|
||||
}
|
||||
|
||||
void compaction_manager::submit_offstrategy(table* t) {
|
||||
void compaction_manager::submit_offstrategy(replica::table* t) {
|
||||
auto task = make_lw_shared<compaction_manager::task>(t, sstables::compaction_type::Reshape, get_compaction_state(t));
|
||||
_tasks.push_back(task);
|
||||
_stats.pending_tasks++;
|
||||
@@ -779,7 +779,7 @@ void compaction_manager::submit_offstrategy(table* t) {
|
||||
});
|
||||
}
|
||||
|
||||
future<> compaction_manager::rewrite_sstables(table* t, sstables::compaction_type_options options, get_candidates_func get_func, can_purge_tombstones can_purge) {
|
||||
future<> compaction_manager::rewrite_sstables(replica::table* t, sstables::compaction_type_options options, get_candidates_func get_func, can_purge_tombstones can_purge) {
|
||||
auto task = make_lw_shared<compaction_manager::task>(t, options.type(), get_compaction_state(t));
|
||||
_tasks.push_back(task);
|
||||
cmlog.debug("{} task {} table={}: started", options.type(), fmt::ptr(task.get()), fmt::ptr(task->compacting_table));
|
||||
@@ -813,7 +813,7 @@ future<> compaction_manager::rewrite_sstables(table* t, sstables::compaction_typ
|
||||
auto rewrite_sstable = [this, &task, &options, &compacting, can_purge] (const sstables::shared_sstable& sst) mutable -> future<> {
|
||||
stop_iteration completed = stop_iteration::no;
|
||||
do {
|
||||
table& t = *task->compacting_table;
|
||||
replica::table& t = *task->compacting_table;
|
||||
auto sstable_level = sst->get_sstable_level();
|
||||
auto run_identifier = sst->run_identifier();
|
||||
auto sstable_set_snapshot = can_purge ? std::make_optional(t.get_sstable_set()) : std::nullopt;
|
||||
@@ -878,7 +878,7 @@ future<> compaction_manager::rewrite_sstables(table* t, sstables::compaction_typ
|
||||
}
|
||||
}
|
||||
|
||||
future<> compaction_manager::perform_sstable_scrub_validate_mode(table* t) {
|
||||
future<> compaction_manager::perform_sstable_scrub_validate_mode(replica::table* t) {
|
||||
// All sstables must be included, even the ones being compacted, such that everything in table is validated.
|
||||
auto all_sstables = boost::copy_range<std::vector<sstables::shared_sstable>>(*t->get_sstables());
|
||||
return run_custom_job(t, sstables::compaction_type::Scrub, [this, &t = *t, sstables = std::move(all_sstables)] (sstables::compaction_data& info) mutable -> future<> {
|
||||
@@ -952,7 +952,7 @@ bool needs_cleanup(const sstables::shared_sstable& sst,
|
||||
return true;
|
||||
}
|
||||
|
||||
future<> compaction_manager::perform_cleanup(database& db, table* t) {
|
||||
future<> compaction_manager::perform_cleanup(replica::database& db, replica::table* t) {
|
||||
auto check_for_cleanup = [this, t] {
|
||||
return boost::algorithm::any_of(_tasks, [t] (auto& task) {
|
||||
return task->compacting_table == t && task->type == sstables::compaction_type::Cleanup;
|
||||
@@ -981,7 +981,7 @@ future<> compaction_manager::perform_cleanup(database& db, table* t) {
|
||||
}
|
||||
|
||||
// Submit a table to be upgraded and wait for its termination.
|
||||
future<> compaction_manager::perform_sstable_upgrade(database& db, table* t, bool exclude_current_version) {
|
||||
future<> compaction_manager::perform_sstable_upgrade(replica::database& db, replica::table* t, bool exclude_current_version) {
|
||||
auto get_sstables = [this, &db, t, exclude_current_version] {
|
||||
std::vector<sstables::shared_sstable> tables;
|
||||
|
||||
@@ -1009,7 +1009,7 @@ future<> compaction_manager::perform_sstable_upgrade(database& db, table* t, boo
|
||||
}
|
||||
|
||||
// Submit a table to be scrubbed and wait for its termination.
|
||||
future<> compaction_manager::perform_sstable_scrub(table* t, sstables::compaction_type_options::scrub opts) {
|
||||
future<> compaction_manager::perform_sstable_scrub(replica::table* t, sstables::compaction_type_options::scrub opts) {
|
||||
auto scrub_mode = opts.operation_mode;
|
||||
if (scrub_mode == sstables::compaction_type_options::scrub::mode::validate) {
|
||||
return perform_sstable_scrub_validate_mode(t);
|
||||
@@ -1034,7 +1034,7 @@ future<> compaction_manager::perform_sstable_scrub(table* t, sstables::compactio
|
||||
}, can_purge_tombstones::no);
|
||||
}
|
||||
|
||||
void compaction_manager::add(table* t) {
|
||||
void compaction_manager::add(replica::table* t) {
|
||||
auto [_, inserted] = _compaction_state.insert({t, compaction_state{}});
|
||||
if (!inserted) {
|
||||
auto s = t->schema();
|
||||
@@ -1042,7 +1042,7 @@ void compaction_manager::add(table* t) {
|
||||
}
|
||||
}
|
||||
|
||||
future<> compaction_manager::remove(table* t) {
|
||||
future<> compaction_manager::remove(replica::table* t) {
|
||||
auto handle = _compaction_state.extract(t);
|
||||
|
||||
if (!handle.empty()) {
|
||||
@@ -1077,7 +1077,7 @@ future<> compaction_manager::remove(table* t) {
|
||||
#endif
|
||||
}
|
||||
|
||||
const std::vector<sstables::compaction_info> compaction_manager::get_compactions(table* t) const {
|
||||
const std::vector<sstables::compaction_info> compaction_manager::get_compactions(replica::table* t) const {
|
||||
auto to_info = [] (const lw_shared_ptr<task>& task) {
|
||||
sstables::compaction_info ret;
|
||||
ret.compaction_uuid = task->compaction_data.compaction_uuid;
|
||||
@@ -1094,7 +1094,7 @@ const std::vector<sstables::compaction_info> compaction_manager::get_compactions
|
||||
}) | boost::adaptors::transformed(to_info));
|
||||
}
|
||||
|
||||
future<> compaction_manager::stop_compaction(sstring type, table* table) {
|
||||
future<> compaction_manager::stop_compaction(sstring type, replica::table* table) {
|
||||
sstables::compaction_type target_type;
|
||||
try {
|
||||
target_type = sstables::to_compaction_type(type);
|
||||
@@ -1113,7 +1113,7 @@ future<> compaction_manager::stop_compaction(sstring type, table* table) {
|
||||
return stop_ongoing_compactions("user request", table, target_type);
|
||||
}
|
||||
|
||||
void compaction_manager::propagate_replacement(table* t,
|
||||
void compaction_manager::propagate_replacement(replica::table* t,
|
||||
const std::vector<sstables::shared_sstable>& removed, const std::vector<sstables::shared_sstable>& added) {
|
||||
for (auto& task : _tasks) {
|
||||
if (task->compacting_table == t && task->compaction_running) {
|
||||
|
||||
@@ -44,7 +44,10 @@
|
||||
#include "backlog_controller.hh"
|
||||
#include "seastarx.hh"
|
||||
|
||||
namespace replica {
|
||||
class table;
|
||||
}
|
||||
|
||||
class compacting_sstable_registration;
|
||||
|
||||
// Compaction manager provides facilities to submit and track compaction jobs on
|
||||
@@ -83,7 +86,7 @@ private:
|
||||
};
|
||||
|
||||
struct task {
|
||||
table* compacting_table = nullptr;
|
||||
replica::table* compacting_table = nullptr;
|
||||
shared_future<> compaction_done = make_ready_future<>();
|
||||
exponential_backoff_retry compaction_retry = exponential_backoff_retry(std::chrono::seconds(5), std::chrono::seconds(300));
|
||||
bool stopping = false;
|
||||
@@ -94,7 +97,7 @@ private:
|
||||
compaction_state& compaction_state;
|
||||
gate::holder gate_holder;
|
||||
|
||||
explicit task(table* t, sstables::compaction_type type, struct compaction_state& cs)
|
||||
explicit task(replica::table* t, sstables::compaction_type type, struct compaction_state& cs)
|
||||
: compacting_table(t)
|
||||
, type(type)
|
||||
, compaction_state(cs)
|
||||
@@ -146,12 +149,12 @@ private:
|
||||
future<> _waiting_reevalution = make_ready_future<>();
|
||||
condition_variable _postponed_reevaluation;
|
||||
// tables that wait for compaction but had its submission postponed due to ongoing compaction.
|
||||
std::unordered_set<table*> _postponed;
|
||||
std::unordered_set<replica::table*> _postponed;
|
||||
// tracks taken weights of ongoing compactions, only one compaction per weight is allowed.
|
||||
// weight is value assigned to a compaction job that is log base N of total size of all input sstables.
|
||||
std::unordered_set<int> _weight_tracker;
|
||||
|
||||
std::unordered_map<table*, compaction_state> _compaction_state;
|
||||
std::unordered_map<replica::table*, compaction_state> _compaction_state;
|
||||
|
||||
// Purpose is to serialize all maintenance (non regular) compaction activity to reduce aggressiveness and space requirement.
|
||||
// If the operation must be serialized with regular, then the per-table write lock must be taken.
|
||||
@@ -170,7 +173,7 @@ private:
|
||||
unsigned current_compaction_fan_in_threshold() const;
|
||||
|
||||
// Return true if compaction can be initiated
|
||||
bool can_register_compaction(table* t, int weight, unsigned fan_in) const;
|
||||
bool can_register_compaction(replica::table* t, int weight, unsigned fan_in) const;
|
||||
// Register weight for a table. Do that only if can_register_weight()
|
||||
// returned true.
|
||||
void register_weight(int weight);
|
||||
@@ -178,14 +181,14 @@ private:
|
||||
void deregister_weight(int weight);
|
||||
|
||||
// Get candidates for compaction strategy, which are all sstables but the ones being compacted.
|
||||
std::vector<sstables::shared_sstable> get_candidates(const table& t);
|
||||
std::vector<sstables::shared_sstable> get_candidates(const replica::table& t);
|
||||
|
||||
void register_compacting_sstables(const std::vector<sstables::shared_sstable>& sstables);
|
||||
void deregister_compacting_sstables(const std::vector<sstables::shared_sstable>& sstables);
|
||||
|
||||
// gets the table's compaction state
|
||||
// throws std::out_of_range exception if not found.
|
||||
compaction_state& get_compaction_state(table* t);
|
||||
compaction_state& get_compaction_state(replica::table* t);
|
||||
|
||||
// Return true if compaction manager and task weren't asked to stop.
|
||||
inline bool can_proceed(const lw_shared_ptr<task>& task);
|
||||
@@ -201,9 +204,9 @@ private:
|
||||
void reevaluate_postponed_compactions();
|
||||
// Postpone compaction for a table that couldn't be executed due to ongoing
|
||||
// similar-sized compaction.
|
||||
void postpone_compaction_for_table(table* t);
|
||||
void postpone_compaction_for_table(replica::table* t);
|
||||
|
||||
future<> perform_sstable_scrub_validate_mode(table* t);
|
||||
future<> perform_sstable_scrub_validate_mode(replica::table* t);
|
||||
|
||||
compaction_controller _compaction_controller;
|
||||
compaction_backlog_manager _backlog_manager;
|
||||
@@ -214,7 +217,7 @@ private:
|
||||
class can_purge_tombstones_tag;
|
||||
using can_purge_tombstones = bool_class<can_purge_tombstones_tag>;
|
||||
|
||||
future<> rewrite_sstables(table* t, sstables::compaction_type_options options, get_candidates_func, can_purge_tombstones can_purge = can_purge_tombstones::yes);
|
||||
future<> rewrite_sstables(replica::table* t, sstables::compaction_type_options options, get_candidates_func, can_purge_tombstones can_purge = can_purge_tombstones::yes);
|
||||
|
||||
optimized_optional<abort_source::subscription> _early_abort_subscription;
|
||||
|
||||
@@ -246,10 +249,10 @@ public:
|
||||
void really_do_stop();
|
||||
|
||||
// Submit a table to be compacted.
|
||||
void submit(table* t);
|
||||
void submit(replica::table* t);
|
||||
|
||||
// Submit a table to be off-strategy compacted.
|
||||
void submit_offstrategy(table* t);
|
||||
void submit_offstrategy(replica::table* t);
|
||||
|
||||
// Submit a table to be cleaned up and wait for its termination.
|
||||
//
|
||||
@@ -258,16 +261,16 @@ public:
|
||||
// Cleanup is about discarding keys that are no longer relevant for a
|
||||
// given sstable, e.g. after node loses part of its token range because
|
||||
// of a newly added node.
|
||||
future<> perform_cleanup(database& db, table* t);
|
||||
future<> perform_cleanup(replica::database& db, replica::table* t);
|
||||
|
||||
// Submit a table to be upgraded and wait for its termination.
|
||||
future<> perform_sstable_upgrade(database& db, table* t, bool exclude_current_version);
|
||||
future<> perform_sstable_upgrade(replica::database& db, replica::table* t, bool exclude_current_version);
|
||||
|
||||
// Submit a table to be scrubbed and wait for its termination.
|
||||
future<> perform_sstable_scrub(table* t, sstables::compaction_type_options::scrub opts);
|
||||
future<> perform_sstable_scrub(replica::table* t, sstables::compaction_type_options::scrub opts);
|
||||
|
||||
// Submit a table for major compaction.
|
||||
future<> perform_major_compaction(table* t);
|
||||
future<> perform_major_compaction(replica::table* t);
|
||||
|
||||
|
||||
// Run a custom job for a given table, defined by a function
|
||||
@@ -277,42 +280,42 @@ public:
|
||||
// parameter type is the compaction type the operation can most closely be
|
||||
// associated with, use compaction_type::Compaction, if none apply.
|
||||
// parameter job is a function that will carry the operation
|
||||
future<> run_custom_job(table* t, sstables::compaction_type type, noncopyable_function<future<>(sstables::compaction_data&)> job);
|
||||
future<> run_custom_job(replica::table* t, sstables::compaction_type type, noncopyable_function<future<>(sstables::compaction_data&)> job);
|
||||
|
||||
// Run a function with compaction temporarily disabled for a table T.
|
||||
future<> run_with_compaction_disabled(table* t, std::function<future<> ()> func);
|
||||
future<> run_with_compaction_disabled(replica::table* t, std::function<future<> ()> func);
|
||||
|
||||
// Adds a table to the compaction manager.
|
||||
// Creates a compaction_state structure that can be used for submitting
|
||||
// compaction jobs of all types.
|
||||
void add(table* t);
|
||||
void add(replica::table* t);
|
||||
|
||||
// Remove a table from the compaction manager.
|
||||
// Cancel requests on table and wait for possible ongoing compactions.
|
||||
future<> remove(table* t);
|
||||
future<> remove(replica::table* t);
|
||||
|
||||
const stats& get_stats() const {
|
||||
return _stats;
|
||||
}
|
||||
|
||||
const std::vector<sstables::compaction_info> get_compactions(table* t = nullptr) const;
|
||||
const std::vector<sstables::compaction_info> get_compactions(replica::table* t = nullptr) const;
|
||||
|
||||
// Returns true if table has an ongoing compaction, running on its behalf
|
||||
bool has_table_ongoing_compaction(const table* t) const {
|
||||
bool has_table_ongoing_compaction(const replica::table* t) const {
|
||||
return std::any_of(_tasks.begin(), _tasks.end(), [t] (const lw_shared_ptr<task>& task) {
|
||||
return task->compacting_table == t && task->compaction_running;
|
||||
});
|
||||
};
|
||||
|
||||
bool compaction_disabled(table* t) const {
|
||||
bool compaction_disabled(replica::table* t) const {
|
||||
return _compaction_state.contains(t) && _compaction_state.at(t).compaction_disabled();
|
||||
}
|
||||
|
||||
// Stops ongoing compaction of a given type.
|
||||
future<> stop_compaction(sstring type, table* table = nullptr);
|
||||
future<> stop_compaction(sstring type, replica::table* table = nullptr);
|
||||
|
||||
// Stops ongoing compaction of a given table and/or compaction_type.
|
||||
future<> stop_ongoing_compactions(sstring reason, table* t = nullptr, std::optional<sstables::compaction_type> type_opt = {});
|
||||
future<> stop_ongoing_compactions(sstring reason, replica::table* t = nullptr, std::optional<sstables::compaction_type> type_opt = {});
|
||||
|
||||
double backlog() {
|
||||
return _backlog_manager.backlog();
|
||||
@@ -323,7 +326,7 @@ public:
|
||||
}
|
||||
|
||||
// Propagate replacement of sstables to all ongoing compaction of a given table
|
||||
void propagate_replacement(table* t, const std::vector<sstables::shared_sstable>& removed, const std::vector<sstables::shared_sstable>& added);
|
||||
void propagate_replacement(replica::table* t, const std::vector<sstables::shared_sstable>& removed, const std::vector<sstables::shared_sstable>& added);
|
||||
|
||||
static sstables::compaction_data create_compaction_data();
|
||||
|
||||
|
||||
@@ -169,7 +169,7 @@ database::get_config() const {
|
||||
return _ops->get_config(*this);
|
||||
}
|
||||
|
||||
::database&
|
||||
replica::database&
|
||||
database::real_database() const {
|
||||
return _ops->real_database(*this);
|
||||
}
|
||||
|
||||
@@ -28,7 +28,10 @@
|
||||
#include "seastarx.hh"
|
||||
#include "utils/UUID.hh"
|
||||
|
||||
namespace replica {
|
||||
class database; // For transition; remove
|
||||
}
|
||||
|
||||
class schema;
|
||||
using schema_ptr = lw_shared_ptr<const schema>;
|
||||
class view_ptr;
|
||||
@@ -127,7 +130,7 @@ public:
|
||||
schema_ptr get_cdc_base_table(const schema&) const;
|
||||
const db::extensions& extensions() const;
|
||||
const gms::feature_service& features() const;
|
||||
::database& real_database() const; // For transition; remove
|
||||
replica::database& real_database() const; // For transition; remove
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
@@ -46,7 +46,7 @@ public:
|
||||
virtual const db::config& get_config(database db) const = 0;
|
||||
virtual const db::extensions& get_extensions(database db) const = 0;
|
||||
virtual const gms::feature_service& get_features(database db) const = 0;
|
||||
virtual ::database& real_database(database db) const = 0;
|
||||
virtual replica::database& real_database(database db) const = 0;
|
||||
protected:
|
||||
// Tools for type erasing an unerasing
|
||||
static database make_database(const impl* i, const void* db) {
|
||||
|
||||
@@ -80,7 +80,7 @@ class db::commitlog_replayer::impl {
|
||||
|
||||
friend class db::commitlog_replayer;
|
||||
public:
|
||||
impl(seastar::sharded<database>& db);
|
||||
impl(seastar::sharded<replica::database>& db);
|
||||
|
||||
future<> init();
|
||||
|
||||
@@ -133,7 +133,7 @@ public:
|
||||
return j != i->second.end() ? j->second : replay_position();
|
||||
}
|
||||
|
||||
seastar::sharded<database>&
|
||||
seastar::sharded<replica::database>&
|
||||
_db;
|
||||
shard_rpm_map
|
||||
_rpm;
|
||||
@@ -141,7 +141,7 @@ public:
|
||||
_min_pos;
|
||||
};
|
||||
|
||||
db::commitlog_replayer::impl::impl(seastar::sharded<database>& db)
|
||||
db::commitlog_replayer::impl::impl(seastar::sharded<replica::database>& db)
|
||||
: _db(db)
|
||||
{}
|
||||
|
||||
@@ -158,7 +158,7 @@ future<> db::commitlog_replayer::impl::init() {
|
||||
}
|
||||
}
|
||||
}
|
||||
}, [this](database& db) {
|
||||
}, [this](replica::database& db) {
|
||||
return do_with(shard_rpm_map{}, [this, &db](shard_rpm_map& map) {
|
||||
return parallel_for_each(db.get_column_families(), [&map, &db](auto& cfp) {
|
||||
auto uuid = cfp.first;
|
||||
@@ -276,7 +276,7 @@ future<> db::commitlog_replayer::impl::process(stats* s, commitlog::buffer_and_r
|
||||
}
|
||||
|
||||
auto shard = _db.local().shard_of(fm);
|
||||
return _db.invoke_on(shard, [this, cer = std::move(cer), &src_cm, rp, shard, s] (database& db) mutable -> future<> {
|
||||
return _db.invoke_on(shard, [this, cer = std::move(cer), &src_cm, rp, shard, s] (replica::database& db) mutable -> future<> {
|
||||
auto& fm = cer.mutation();
|
||||
// TODO: might need better verification that the deserialized mutation
|
||||
// is schema compatible. My guess is that just applying the mutation
|
||||
@@ -321,7 +321,7 @@ future<> db::commitlog_replayer::impl::process(stats* s, commitlog::buffer_and_r
|
||||
rlogger.warn("error replaying: {}", std::current_exception());
|
||||
}
|
||||
});
|
||||
} catch (no_such_column_family&) {
|
||||
} catch (replica::no_such_column_family&) {
|
||||
// No such CF now? Origin just ignores this.
|
||||
} catch (...) {
|
||||
s->invalid_mutations++;
|
||||
@@ -332,7 +332,7 @@ future<> db::commitlog_replayer::impl::process(stats* s, commitlog::buffer_and_r
|
||||
return make_ready_future<>();
|
||||
}
|
||||
|
||||
db::commitlog_replayer::commitlog_replayer(seastar::sharded<database>& db)
|
||||
db::commitlog_replayer::commitlog_replayer(seastar::sharded<replica::database>& db)
|
||||
: _impl(std::make_unique<impl>(db))
|
||||
{}
|
||||
|
||||
@@ -343,7 +343,7 @@ db::commitlog_replayer::commitlog_replayer(commitlog_replayer&& r) noexcept
|
||||
db::commitlog_replayer::~commitlog_replayer()
|
||||
{}
|
||||
|
||||
future<db::commitlog_replayer> db::commitlog_replayer::create_replayer(seastar::sharded<database>& db) {
|
||||
future<db::commitlog_replayer> db::commitlog_replayer::create_replayer(seastar::sharded<replica::database>& db) {
|
||||
return do_with(commitlog_replayer(db), [](auto&& rp) {
|
||||
auto f = rp._impl->init();
|
||||
return f.then([rp = std::move(rp)]() mutable {
|
||||
|
||||
@@ -47,7 +47,9 @@
|
||||
|
||||
#include "seastarx.hh"
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace db {
|
||||
|
||||
@@ -58,13 +60,13 @@ public:
|
||||
commitlog_replayer(commitlog_replayer&&) noexcept;
|
||||
~commitlog_replayer();
|
||||
|
||||
static future<commitlog_replayer> create_replayer(seastar::sharded<database>&);
|
||||
static future<commitlog_replayer> create_replayer(seastar::sharded<replica::database>&);
|
||||
|
||||
future<> recover(std::vector<sstring> files, sstring fname_prefix);
|
||||
future<> recover(sstring file, sstring fname_prefix);
|
||||
|
||||
private:
|
||||
commitlog_replayer(seastar::sharded<database>&);
|
||||
commitlog_replayer(seastar::sharded<replica::database>&);
|
||||
|
||||
class impl;
|
||||
std::unique_ptr<impl> _impl;
|
||||
|
||||
@@ -60,12 +60,12 @@ namespace db {
|
||||
|
||||
logging::logger cl_logger("consistency");
|
||||
|
||||
size_t quorum_for(const keyspace& ks) {
|
||||
size_t quorum_for(const replica::keyspace& ks) {
|
||||
size_t replication_factor = ks.get_effective_replication_map()->get_replication_factor();
|
||||
return replication_factor ? (replication_factor / 2) + 1 : 0;
|
||||
}
|
||||
|
||||
size_t local_quorum_for(const keyspace& ks, const sstring& dc) {
|
||||
size_t local_quorum_for(const replica::keyspace& ks, const sstring& dc) {
|
||||
using namespace locator;
|
||||
|
||||
auto& rs = ks.get_replication_strategy();
|
||||
@@ -80,7 +80,7 @@ size_t local_quorum_for(const keyspace& ks, const sstring& dc) {
|
||||
return quorum_for(ks);
|
||||
}
|
||||
|
||||
size_t block_for_local_serial(keyspace& ks) {
|
||||
size_t block_for_local_serial(replica::keyspace& ks) {
|
||||
using namespace locator;
|
||||
|
||||
//
|
||||
@@ -95,7 +95,7 @@ size_t block_for_local_serial(keyspace& ks) {
|
||||
return local_quorum_for(ks, snitch_ptr->get_datacenter(local_addr));
|
||||
}
|
||||
|
||||
size_t block_for_each_quorum(keyspace& ks) {
|
||||
size_t block_for_each_quorum(replica::keyspace& ks) {
|
||||
using namespace locator;
|
||||
|
||||
auto& rs = ks.get_replication_strategy();
|
||||
@@ -115,7 +115,7 @@ size_t block_for_each_quorum(keyspace& ks) {
|
||||
}
|
||||
}
|
||||
|
||||
size_t block_for(keyspace& ks, consistency_level cl) {
|
||||
size_t block_for(replica::keyspace& ks, consistency_level cl) {
|
||||
switch (cl) {
|
||||
case consistency_level::ONE:
|
||||
case consistency_level::LOCAL_ONE:
|
||||
@@ -158,7 +158,7 @@ bool is_local(gms::inet_address endpoint) {
|
||||
|
||||
template <typename Range, typename PendingRange = std::array<gms::inet_address, 0>>
|
||||
std::unordered_map<sstring, dc_node_count> count_per_dc_endpoints(
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
const Range& live_endpoints,
|
||||
const PendingRange& pending_endpoints = std::array<gms::inet_address, 0>()) {
|
||||
using namespace locator;
|
||||
@@ -193,7 +193,7 @@ std::unordered_map<sstring, dc_node_count> count_per_dc_endpoints(
|
||||
template<typename Range, typename PendingRange>
|
||||
bool assure_sufficient_live_nodes_each_quorum(
|
||||
consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
const Range& live_endpoints,
|
||||
const PendingRange& pending_endpoints) {
|
||||
using namespace locator;
|
||||
@@ -220,7 +220,7 @@ bool assure_sufficient_live_nodes_each_quorum(
|
||||
template<typename Range, typename PendingRange>
|
||||
void assure_sufficient_live_nodes(
|
||||
consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
const Range& live_endpoints,
|
||||
const PendingRange& pending_endpoints) {
|
||||
size_t need = block_for(ks, cl);
|
||||
@@ -267,17 +267,17 @@ void assure_sufficient_live_nodes(
|
||||
}
|
||||
}
|
||||
|
||||
template void assure_sufficient_live_nodes(consistency_level, keyspace&, const inet_address_vector_replica_set&, const std::array<gms::inet_address, 0>&);
|
||||
template void assure_sufficient_live_nodes(db::consistency_level, keyspace&, const inet_address_vector_replica_set&, const utils::small_vector<gms::inet_address, 1ul>&);
|
||||
template void assure_sufficient_live_nodes(consistency_level, replica::keyspace&, const inet_address_vector_replica_set&, const std::array<gms::inet_address, 0>&);
|
||||
template void assure_sufficient_live_nodes(db::consistency_level, replica::keyspace&, const inet_address_vector_replica_set&, const utils::small_vector<gms::inet_address, 1ul>&);
|
||||
|
||||
inet_address_vector_replica_set
|
||||
filter_for_query(consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
inet_address_vector_replica_set live_endpoints,
|
||||
const inet_address_vector_replica_set& preferred_endpoints,
|
||||
read_repair_decision read_repair,
|
||||
gms::inet_address* extra,
|
||||
column_family* cf) {
|
||||
replica::column_family* cf) {
|
||||
size_t local_count;
|
||||
|
||||
if (read_repair == read_repair_decision::GLOBAL) { // take RRD.GLOBAL out of the way
|
||||
@@ -383,16 +383,16 @@ filter_for_query(consistency_level cl,
|
||||
}
|
||||
|
||||
inet_address_vector_replica_set filter_for_query(consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
inet_address_vector_replica_set& live_endpoints,
|
||||
const inet_address_vector_replica_set& preferred_endpoints,
|
||||
column_family* cf) {
|
||||
replica::column_family* cf) {
|
||||
return filter_for_query(cl, ks, live_endpoints, preferred_endpoints, read_repair_decision::NONE, nullptr, cf);
|
||||
}
|
||||
|
||||
bool
|
||||
is_sufficient_live_nodes(consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
const inet_address_vector_replica_set& live_endpoints) {
|
||||
using namespace locator;
|
||||
|
||||
|
||||
@@ -57,15 +57,15 @@ namespace db {
|
||||
|
||||
extern logging::logger cl_logger;
|
||||
|
||||
size_t quorum_for(const keyspace& ks);
|
||||
size_t quorum_for(const replica::keyspace& ks);
|
||||
|
||||
size_t local_quorum_for(const keyspace& ks, const sstring& dc);
|
||||
size_t local_quorum_for(const replica::keyspace& ks, const sstring& dc);
|
||||
|
||||
size_t block_for_local_serial(keyspace& ks);
|
||||
size_t block_for_local_serial(replica::keyspace& ks);
|
||||
|
||||
size_t block_for_each_quorum(keyspace& ks);
|
||||
size_t block_for_each_quorum(replica::keyspace& ks);
|
||||
|
||||
size_t block_for(keyspace& ks, consistency_level cl);
|
||||
size_t block_for(replica::keyspace& ks, consistency_level cl);
|
||||
|
||||
bool is_datacenter_local(consistency_level l);
|
||||
|
||||
@@ -78,18 +78,18 @@ inline size_t count_local_endpoints(const Range& live_endpoints) {
|
||||
|
||||
inet_address_vector_replica_set
|
||||
filter_for_query(consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
inet_address_vector_replica_set live_endpoints,
|
||||
const inet_address_vector_replica_set& preferred_endpoints,
|
||||
read_repair_decision read_repair,
|
||||
gms::inet_address* extra,
|
||||
column_family* cf);
|
||||
replica::column_family* cf);
|
||||
|
||||
inet_address_vector_replica_set filter_for_query(consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
inet_address_vector_replica_set& live_endpoints,
|
||||
const inet_address_vector_replica_set& preferred_endpoints,
|
||||
column_family* cf);
|
||||
replica::column_family* cf);
|
||||
|
||||
struct dc_node_count {
|
||||
size_t live = 0;
|
||||
@@ -98,17 +98,17 @@ struct dc_node_count {
|
||||
|
||||
bool
|
||||
is_sufficient_live_nodes(consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
const inet_address_vector_replica_set& live_endpoints);
|
||||
|
||||
template<typename Range, typename PendingRange = std::array<gms::inet_address, 0>>
|
||||
void assure_sufficient_live_nodes(
|
||||
consistency_level cl,
|
||||
keyspace& ks,
|
||||
replica::keyspace& ks,
|
||||
const Range& live_endpoints,
|
||||
const PendingRange& pending_endpoints = std::array<gms::inet_address, 0>());
|
||||
|
||||
extern template void assure_sufficient_live_nodes(consistency_level, keyspace&, const inet_address_vector_replica_set&, const std::array<gms::inet_address, 0>&);
|
||||
extern template void assure_sufficient_live_nodes(db::consistency_level, keyspace&, const inet_address_vector_replica_set&, const utils::small_vector<gms::inet_address, 1ul>&);
|
||||
extern template void assure_sufficient_live_nodes(consistency_level, replica::keyspace&, const inet_address_vector_replica_set&, const std::array<gms::inet_address, 0>&);
|
||||
extern template void assure_sufficient_live_nodes(db::consistency_level, replica::keyspace&, const inet_address_vector_replica_set&, const utils::small_vector<gms::inet_address, 1ul>&);
|
||||
|
||||
}
|
||||
|
||||
@@ -71,16 +71,16 @@ data_type db::cql_type_parser::parse(const sstring& keyspace, const sstring& str
|
||||
}
|
||||
|
||||
const auto& sp = service::get_storage_proxy();
|
||||
const user_types_metadata& user_types =
|
||||
const replica::user_types_metadata& user_types =
|
||||
sp.local_is_initialized() ? sp.local().get_db().local().find_keyspace(keyspace).metadata()->user_types()
|
||||
: user_types_metadata{};
|
||||
: replica::user_types_metadata{};
|
||||
auto raw = parse_raw(str);
|
||||
return raw->prepare_internal(keyspace, user_types).get_type();
|
||||
}
|
||||
|
||||
class db::cql_type_parser::raw_builder::impl {
|
||||
public:
|
||||
impl(keyspace_metadata &ks)
|
||||
impl(replica::keyspace_metadata &ks)
|
||||
: _ks(ks)
|
||||
{}
|
||||
|
||||
@@ -91,7 +91,7 @@ public:
|
||||
std::vector<sstring> field_names;
|
||||
std::vector<::shared_ptr<cql3::cql3_type::raw>> field_types;
|
||||
|
||||
user_type prepare(const sstring& keyspace, user_types_metadata& user_types) const {
|
||||
user_type prepare(const sstring& keyspace, replica::user_types_metadata& user_types) const {
|
||||
std::vector<data_type> fields;
|
||||
fields.reserve(field_types.size());
|
||||
std::transform(field_types.begin(), field_types.end(), std::back_inserter(fields), [&](auto& r) {
|
||||
@@ -151,7 +151,7 @@ public:
|
||||
// Create a copy of the existing types, so that we don't
|
||||
// modify the one in the keyspace. It is up to the caller to
|
||||
// do that.
|
||||
user_types_metadata types = _ks.user_types();
|
||||
replica::user_types_metadata types = _ks.user_types();
|
||||
|
||||
const auto &ks_name = _ks.name();
|
||||
std::vector<user_type> created;
|
||||
|
||||
@@ -63,7 +63,7 @@ toppartitions_item_key::operator sstring() const {
|
||||
return oss.str();
|
||||
}
|
||||
|
||||
toppartitions_data_listener::toppartitions_data_listener(database& db, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash> table_filters,
|
||||
toppartitions_data_listener::toppartitions_data_listener(replica::database& db, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash> table_filters,
|
||||
std::unordered_set<sstring> keyspace_filters) : _db(db), _table_filters(std::move(table_filters)), _keyspace_filters(std::move(keyspace_filters)) {
|
||||
dblog.debug("toppartitions_data_listener: installing {}", fmt::ptr(this));
|
||||
_db.data_listeners().install(this);
|
||||
@@ -126,7 +126,7 @@ toppartitions_data_listener::localize(const global_top_k::results& r) {
|
||||
return n;
|
||||
}
|
||||
|
||||
toppartitions_query::toppartitions_query(distributed<database>& xdb, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash>&& table_filters,
|
||||
toppartitions_query::toppartitions_query(distributed<replica::database>& xdb, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash>&& table_filters,
|
||||
std::unordered_set<sstring>&& keyspace_filters, std::chrono::milliseconds duration, size_t list_size, size_t capacity)
|
||||
: _xdb(xdb), _table_filters(std::move(table_filters)), _keyspace_filters(std::move(keyspace_filters)), _duration(duration), _list_size(list_size), _capacity(capacity),
|
||||
_query(std::make_unique<sharded<toppartitions_data_listener>>()) {
|
||||
|
||||
@@ -126,7 +126,7 @@ struct toppartitions_global_item_key {
|
||||
class toppartitions_data_listener : public data_listener, public weakly_referencable<toppartitions_data_listener> {
|
||||
friend class toppartitions_query;
|
||||
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash> _table_filters;
|
||||
std::unordered_set<sstring> _keyspace_filters;
|
||||
|
||||
@@ -141,7 +141,7 @@ private:
|
||||
top_k _top_k_write;
|
||||
|
||||
public:
|
||||
toppartitions_data_listener(database& db, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash> table_filters, std::unordered_set<sstring> keyspace_filters);
|
||||
toppartitions_data_listener(replica::database& db, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash> table_filters, std::unordered_set<sstring> keyspace_filters);
|
||||
~toppartitions_data_listener();
|
||||
|
||||
virtual flat_mutation_reader on_read(const schema_ptr& s, const dht::partition_range& range,
|
||||
@@ -153,7 +153,7 @@ public:
|
||||
};
|
||||
|
||||
class toppartitions_query {
|
||||
distributed<database>& _xdb;
|
||||
distributed<replica::database>& _xdb;
|
||||
std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash> _table_filters;
|
||||
std::unordered_set<sstring> _keyspace_filters;
|
||||
std::chrono::milliseconds _duration;
|
||||
@@ -162,7 +162,7 @@ class toppartitions_query {
|
||||
std::unique_ptr<sharded<toppartitions_data_listener>> _query;
|
||||
|
||||
public:
|
||||
toppartitions_query(seastar::distributed<database>& xdb, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash>&& table_filters,
|
||||
toppartitions_query(seastar::distributed<replica::database>& xdb, std::unordered_set<std::tuple<sstring, sstring>, utils::tuple_hash>&& table_filters,
|
||||
std::unordered_set<sstring>&& keyspace_filters, std::chrono::milliseconds duration, size_t list_size, size_t capacity);
|
||||
|
||||
struct results {
|
||||
|
||||
@@ -58,7 +58,7 @@ const std::string manager::FILENAME_PREFIX("HintsLog" + commitlog::descriptor::S
|
||||
const std::chrono::seconds manager::hint_file_write_timeout = std::chrono::seconds(2);
|
||||
const std::chrono::seconds manager::hints_flush_period = std::chrono::seconds(10);
|
||||
|
||||
manager::manager(sstring hints_directory, host_filter filter, int64_t max_hint_window_ms, resource_manager& res_manager, distributed<database>& db)
|
||||
manager::manager(sstring hints_directory, host_filter filter, int64_t max_hint_window_ms, resource_manager& res_manager, distributed<replica::database>& db)
|
||||
: _hints_dir(fs::path(hints_directory) / format("{:d}", this_shard_id()))
|
||||
, _host_filter(std::move(filter))
|
||||
, _local_snitch_ptr(locator::i_endpoint_snitch::get_local_snitch_ptr())
|
||||
@@ -754,7 +754,7 @@ void manager::drain_for(gms::inet_address endpoint) {
|
||||
});
|
||||
}
|
||||
|
||||
manager::end_point_hints_manager::sender::sender(end_point_hints_manager& parent, service::storage_proxy& local_storage_proxy, database& local_db, gms::gossiper& local_gossiper) noexcept
|
||||
manager::end_point_hints_manager::sender::sender(end_point_hints_manager& parent, service::storage_proxy& local_storage_proxy,replica::database& local_db, gms::gossiper& local_gossiper) noexcept
|
||||
: _stopped(make_ready_future<>())
|
||||
, _ep_key(parent.end_point_key())
|
||||
, _ep_manager(parent)
|
||||
@@ -864,7 +864,7 @@ void manager::end_point_hints_manager::sender::start() {
|
||||
}
|
||||
|
||||
future<> manager::end_point_hints_manager::sender::send_one_mutation(frozen_mutation_and_schema m) {
|
||||
keyspace& ks = _db.find_keyspace(m.s->ks_name());
|
||||
replica::keyspace& ks = _db.find_keyspace(m.s->ks_name());
|
||||
auto token = dht::get_token(*m.s, m.fm.key());
|
||||
inet_address_vector_replica_set natural_endpoints = ks.get_effective_replication_map()->get_natural_endpoints(std::move(token));
|
||||
|
||||
@@ -897,10 +897,10 @@ future<> manager::end_point_hints_manager::sender::send_one_hint(lw_shared_ptr<s
|
||||
});
|
||||
|
||||
// ignore these errors and move on - probably this hint is too old and the KS/CF has been deleted...
|
||||
} catch (no_such_column_family& e) {
|
||||
} catch (replica::no_such_column_family& e) {
|
||||
manager_logger.debug("send_hints(): no_such_column_family: {}", e.what());
|
||||
++this->shard_stats().discarded;
|
||||
} catch (no_such_keyspace& e) {
|
||||
} catch (replica::no_such_keyspace& e) {
|
||||
manager_logger.debug("send_hints(): no_such_keyspace: {}", e.what());
|
||||
++this->shard_stats().discarded;
|
||||
} catch (no_column_mapping& e) {
|
||||
|
||||
@@ -158,7 +158,7 @@ public:
|
||||
manager& _shard_manager;
|
||||
resource_manager& _resource_manager;
|
||||
service::storage_proxy& _proxy;
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
seastar::scheduling_group _hints_cpu_sched_group;
|
||||
gms::gossiper& _gossiper;
|
||||
seastar::shared_mutex& _file_update_mutex;
|
||||
@@ -166,7 +166,7 @@ public:
|
||||
std::multimap<db::replay_position, lw_shared_ptr<std::optional<promise<>>>> _replay_waiters;
|
||||
|
||||
public:
|
||||
sender(end_point_hints_manager& parent, service::storage_proxy& local_storage_proxy, database& local_db, gms::gossiper& local_gossiper) noexcept;
|
||||
sender(end_point_hints_manager& parent, service::storage_proxy& local_storage_proxy, replica::database& local_db, gms::gossiper& local_gossiper) noexcept;
|
||||
~sender();
|
||||
|
||||
/// \brief A constructor that should be called from the copy/move-constructor of end_point_hints_manager.
|
||||
@@ -530,7 +530,7 @@ private:
|
||||
shared_ptr<gms::gossiper> _gossiper_anchor;
|
||||
locator::snitch_ptr& _local_snitch_ptr;
|
||||
int64_t _max_hint_window_us = 0;
|
||||
database& _local_db;
|
||||
replica::database& _local_db;
|
||||
|
||||
seastar::gate _draining_eps_gate; // gate used to control the progress of ep_managers stopping not in the context of manager::stop() call
|
||||
|
||||
@@ -543,7 +543,7 @@ private:
|
||||
seastar::named_semaphore _drain_lock = {1, named_semaphore_exception_factory{"drain lock"}};
|
||||
|
||||
public:
|
||||
manager(sstring hints_directory, host_filter filter, int64_t max_hint_window_ms, resource_manager&res_manager, distributed<database>& db);
|
||||
manager(sstring hints_directory, host_filter filter, int64_t max_hint_window_ms, resource_manager&res_manager, distributed<replica::database>& db);
|
||||
virtual ~manager();
|
||||
manager(manager&&) = delete;
|
||||
manager& operator=(manager&&) = delete;
|
||||
@@ -746,7 +746,7 @@ private:
|
||||
return *_gossiper_anchor;
|
||||
}
|
||||
|
||||
database& local_db() noexcept {
|
||||
replica::database& local_db() noexcept {
|
||||
return _local_db;
|
||||
}
|
||||
|
||||
|
||||
@@ -81,7 +81,7 @@ class migrator {
|
||||
public:
|
||||
static const std::unordered_set<sstring> legacy_schema_tables;
|
||||
|
||||
migrator(sharded<service::storage_proxy>& sp, sharded<database>& db, cql3::query_processor& qp)
|
||||
migrator(sharded<service::storage_proxy>& sp, sharded<replica::database>& db, cql3::query_processor& qp)
|
||||
: _sp(sp), _db(db), _qp(qp) {
|
||||
}
|
||||
migrator(migrator&&) = default;
|
||||
@@ -568,7 +568,7 @@ public:
|
||||
return parallel_for_each(legacy_schema_tables, [this](const sstring& cfname) {
|
||||
return do_with(utils::make_joinpoint([] { return db_clock::now();}),[this, cfname](auto& tsf) {
|
||||
auto with_snapshot = !_keyspaces.empty();
|
||||
return _db.invoke_on_all([&tsf, cfname, with_snapshot](database& db) {
|
||||
return _db.invoke_on_all([&tsf, cfname, with_snapshot](replica::database& db) {
|
||||
return db.drop_column_family(db::system_keyspace::NAME, cfname, [&tsf] { return tsf.value(); }, with_snapshot);
|
||||
});
|
||||
});
|
||||
@@ -602,7 +602,7 @@ public:
|
||||
}
|
||||
|
||||
future<> flush_schemas() {
|
||||
return _qp.proxy().get_db().invoke_on_all([this] (database& db) {
|
||||
return _qp.proxy().get_db().invoke_on_all([this] (replica::database& db) {
|
||||
return parallel_for_each(db::schema_tables::all_table_names(schema_features::full()), [this, &db](const sstring& cf_name) {
|
||||
auto& cf = db.find_column_family(db::schema_tables::NAME, cf_name);
|
||||
return cf.flush();
|
||||
@@ -621,7 +621,7 @@ public:
|
||||
}
|
||||
|
||||
sharded<service::storage_proxy>& _sp;
|
||||
sharded<database>& _db;
|
||||
sharded<replica::database>& _db;
|
||||
cql3::query_processor& _qp;
|
||||
std::vector<keyspace> _keyspaces;
|
||||
};
|
||||
@@ -640,7 +640,7 @@ const std::unordered_set<sstring> migrator::legacy_schema_tables = {
|
||||
}
|
||||
|
||||
future<>
|
||||
db::legacy_schema_migrator::migrate(sharded<service::storage_proxy>& sp, sharded<database>& db, cql3::query_processor& qp) {
|
||||
db::legacy_schema_migrator::migrate(sharded<service::storage_proxy>& sp, sharded<replica::database>& db, cql3::query_processor& qp) {
|
||||
return do_with(migrator(sp, db, qp), std::bind(&migrator::migrate, std::placeholders::_1));
|
||||
}
|
||||
|
||||
|
||||
@@ -45,7 +45,9 @@
|
||||
|
||||
#include "seastarx.hh"
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace cql3 {
|
||||
class query_processor;
|
||||
@@ -58,7 +60,7 @@ class storage_proxy;
|
||||
namespace db {
|
||||
namespace legacy_schema_migrator {
|
||||
|
||||
future<> migrate(sharded<service::storage_proxy>&, sharded<database>& db, cql3::query_processor&);
|
||||
future<> migrate(sharded<service::storage_proxy>&, sharded<replica::database>& db, cql3::query_processor&);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@@ -123,11 +123,11 @@ schema_ctxt::schema_ctxt(const db::config& cfg)
|
||||
, _schema_registry_grace_period(cfg.schema_registry_grace_period())
|
||||
{}
|
||||
|
||||
schema_ctxt::schema_ctxt(const database& db)
|
||||
schema_ctxt::schema_ctxt(const replica::database& db)
|
||||
: schema_ctxt(db.get_config())
|
||||
{}
|
||||
|
||||
schema_ctxt::schema_ctxt(distributed<database>& db)
|
||||
schema_ctxt::schema_ctxt(distributed<replica::database>& db)
|
||||
: schema_ctxt(db.local())
|
||||
{}
|
||||
|
||||
@@ -889,7 +889,7 @@ static
|
||||
future<> update_schema_version_and_announce(distributed<service::storage_proxy>& proxy, schema_features features) {
|
||||
auto uuid = co_await calculate_schema_digest(proxy, features);
|
||||
co_await db::system_keyspace::update_schema_version(uuid);
|
||||
co_await proxy.local().get_db().invoke_on_all([uuid] (database& db) {
|
||||
co_await proxy.local().get_db().invoke_on_all([uuid] (replica::database& db) {
|
||||
db.update_version(uuid);
|
||||
});
|
||||
slogger.info("Schema version changed to {}", uuid);
|
||||
@@ -1070,7 +1070,7 @@ static future<> do_merge_schema(distributed<service::storage_proxy>& proxy, std:
|
||||
co_await proxy.local().mutate_locally(std::move(mutations), tracing::trace_state_ptr());
|
||||
|
||||
if (do_flush) {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (database& db) -> future<> {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (replica::database& db) -> future<> {
|
||||
auto& cfs = column_families;
|
||||
co_await parallel_for_each(cfs.begin(), cfs.end(), [&] (const utils::UUID& id) -> future<> {
|
||||
auto& cf = db.find_column_family(id);
|
||||
@@ -1096,7 +1096,7 @@ static future<> do_merge_schema(distributed<service::storage_proxy>& proxy, std:
|
||||
co_await merge_aggregates(proxy, std::move(old_aggregates), std::move(new_aggregates));
|
||||
co_await types_to_drop.drop();
|
||||
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (database& db) -> future<> {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (replica::database& db) -> future<> {
|
||||
// it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
|
||||
for (auto keyspace_to_drop : keyspaces_to_drop) {
|
||||
db.drop_keyspace(keyspace_to_drop);
|
||||
@@ -1136,7 +1136,7 @@ future<std::set<sstring>> merge_keyspaces(distributed<service::storage_proxy>& p
|
||||
slogger.info("Altering keyspace {}", key);
|
||||
altered.emplace_back(key);
|
||||
}
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (database& db) -> future<> {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (replica::database& db) -> future<> {
|
||||
for (auto&& val : created) {
|
||||
auto ksm = create_keyspace_from_schema_partition(val);
|
||||
co_await db.create_keyspace(ksm, proxy.local().get_erm_factory());
|
||||
@@ -1263,7 +1263,7 @@ static future<> merge_tables_and_views(distributed<service::storage_proxy>& prox
|
||||
return vp;
|
||||
});
|
||||
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (database& db) -> future<> {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (replica::database& db) -> future<> {
|
||||
// First drop views and *only then* the tables, if interleaved it can lead
|
||||
// to a mv not finding its schema when snapshoting since the main table
|
||||
// was already dropped (see https://github.com/scylladb/scylla/issues/5614)
|
||||
@@ -1442,7 +1442,7 @@ template <typename T> static std::vector<user_type> create_types(keyspace_metada
|
||||
// Given a set of rows that is sorted by keyspace, create types for each keyspace.
|
||||
// The topological sort in each keyspace is necessary when creating types, since we can only create a type when the
|
||||
// types it reference have already been created.
|
||||
static std::vector<user_type> create_types(database& db, const std::vector<const query::result_set_row*>& rows) {
|
||||
static std::vector<user_type> create_types(replica::database& db, const std::vector<const query::result_set_row*>& rows) {
|
||||
std::vector<user_type> ret;
|
||||
for (auto i = rows.begin(), e = rows.end(); i != e;) {
|
||||
const auto &row = *i;
|
||||
@@ -1467,7 +1467,7 @@ static future<user_types_to_drop> merge_types(distributed<service::storage_proxy
|
||||
// use those types. Similarly, defer dropping until after tables/views that may use
|
||||
// some of these user types are dropped.
|
||||
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (database& db) -> future<> {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (replica::database& db) -> future<> {
|
||||
for (auto&& user_type : create_types(db, diff.created)) {
|
||||
db.find_keyspace(user_type->_keyspace).add_user_type(user_type);
|
||||
co_await db.get_notifier().create_user_type(user_type);
|
||||
@@ -1479,7 +1479,7 @@ static future<user_types_to_drop> merge_types(distributed<service::storage_proxy
|
||||
});
|
||||
|
||||
co_return user_types_to_drop{[&proxy, before = std::move(before), rows = std::move(diff.dropped)] () mutable -> future<> {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (database& db) -> future<> {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (replica::database& db) -> future<> {
|
||||
auto dropped = create_types(db, rows);
|
||||
for (auto& user_type : dropped) {
|
||||
db.find_keyspace(user_type->_keyspace).remove_user_type(user_type);
|
||||
@@ -1552,7 +1552,7 @@ static std::vector<data_type> read_arg_types(const query::result_set_row& row, c
|
||||
}
|
||||
#endif
|
||||
|
||||
static shared_ptr<cql3::functions::user_function> create_func(database& db, const query::result_set_row& row) {
|
||||
static shared_ptr<cql3::functions::user_function> create_func(replica::database& db, const query::result_set_row& row) {
|
||||
cql3::functions::function_name name{
|
||||
row.get_nonnull<sstring>("keyspace_name"), row.get_nonnull<sstring>("function_name")};
|
||||
auto arg_types = read_arg_types(row, name.keyspace);
|
||||
@@ -1590,7 +1590,7 @@ static shared_ptr<cql3::functions::user_function> create_func(database& db, cons
|
||||
}
|
||||
}
|
||||
|
||||
static shared_ptr<cql3::functions::user_aggregate> create_aggregate(database& db, const query::result_set_row& row) {
|
||||
static shared_ptr<cql3::functions::user_aggregate> create_aggregate(replica::database& db, const query::result_set_row& row) {
|
||||
cql3::functions::function_name name{
|
||||
row.get_nonnull<sstring>("keyspace_name"), row.get_nonnull<sstring>("aggregate_name")};
|
||||
auto arg_types = read_arg_types(row, name.keyspace);
|
||||
@@ -1619,10 +1619,10 @@ static shared_ptr<cql3::functions::user_aggregate> create_aggregate(database& db
|
||||
}
|
||||
|
||||
static future<> merge_functions(distributed<service::storage_proxy>& proxy, schema_result before, schema_result after,
|
||||
std::function<shared_ptr<cql3::functions::function>(database& db, const query::result_set_row& row)> create) {
|
||||
std::function<shared_ptr<cql3::functions::function>(replica::database& db, const query::result_set_row& row)> create) {
|
||||
auto diff = diff_rows(before, after);
|
||||
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (database& db) {
|
||||
co_await proxy.local().get_db().invoke_on_all([&] (replica::database& db) {
|
||||
for (const auto& val : diff.created) {
|
||||
cql3::functions::functions::add_function(create(db, *val));
|
||||
}
|
||||
@@ -1794,7 +1794,7 @@ std::vector<user_type> create_types_from_schema_partition(
|
||||
}
|
||||
|
||||
std::vector<shared_ptr<cql3::functions::user_function>> create_functions_from_schema_partition(
|
||||
database& db, lw_shared_ptr<query::result_set> result) {
|
||||
replica::database& db, lw_shared_ptr<query::result_set> result) {
|
||||
std::vector<shared_ptr<cql3::functions::user_function>> ret;
|
||||
for (const auto& row : result->rows()) {
|
||||
ret.emplace_back(create_func(db, row));
|
||||
@@ -2181,7 +2181,7 @@ static schema_mutations make_view_mutations(view_ptr view, api::timestamp_type t
|
||||
static void make_drop_table_or_view_mutations(schema_ptr schema_table, schema_ptr table_or_view, api::timestamp_type timestamp, std::vector<mutation>& mutations);
|
||||
|
||||
static void make_update_indices_mutations(
|
||||
database& db,
|
||||
replica::database& db,
|
||||
schema_ptr old_table,
|
||||
schema_ptr new_table,
|
||||
api::timestamp_type timestamp,
|
||||
@@ -2279,7 +2279,7 @@ static void make_update_columns_mutations(schema_ptr old_table,
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<mutation> make_update_table_mutations(database& db,
|
||||
std::vector<mutation> make_update_table_mutations(replica::database& db,
|
||||
lw_shared_ptr<keyspace_metadata> keyspace,
|
||||
schema_ptr old_table,
|
||||
schema_ptr new_table,
|
||||
@@ -3168,7 +3168,7 @@ std::vector<sstring> all_table_names(schema_features features) {
|
||||
boost::adaptors::transformed([] (auto schema) { return schema->cf_name(); }));
|
||||
}
|
||||
|
||||
view_ptr maybe_fix_legacy_secondary_index_mv_schema(database& db, const view_ptr& v, schema_ptr base_schema, preserve_version preserve_version) {
|
||||
view_ptr maybe_fix_legacy_secondary_index_mv_schema(replica::database& db, const view_ptr& v, schema_ptr base_schema, preserve_version preserve_version) {
|
||||
// Legacy format for a secondary index used a hardcoded "token" column, which ensured a proper
|
||||
// order for indexed queries. This "token" column is now implemented as a computed column,
|
||||
// but for the sake of compatibility we assume that there might be indexes created in the legacy
|
||||
|
||||
@@ -61,7 +61,9 @@ class keyspace_metadata;
|
||||
|
||||
using keyspace_metadata = data_dictionary::keyspace_metadata;
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace query {
|
||||
class result_set;
|
||||
@@ -88,8 +90,8 @@ class config;
|
||||
class schema_ctxt {
|
||||
public:
|
||||
schema_ctxt(const config&);
|
||||
schema_ctxt(const database&);
|
||||
schema_ctxt(distributed<database>&);
|
||||
schema_ctxt(const replica::database&);
|
||||
schema_ctxt(distributed<replica::database>&);
|
||||
schema_ctxt(distributed<service::storage_proxy>&);
|
||||
|
||||
const db::extensions& extensions() const {
|
||||
@@ -217,7 +219,7 @@ std::vector<mutation> make_create_type_mutations(lw_shared_ptr<keyspace_metadata
|
||||
|
||||
std::vector<user_type> create_types_from_schema_partition(keyspace_metadata& ks, lw_shared_ptr<query::result_set> result);
|
||||
|
||||
std::vector<shared_ptr<cql3::functions::user_function>> create_functions_from_schema_partition(database& db, lw_shared_ptr<query::result_set> result);
|
||||
std::vector<shared_ptr<cql3::functions::user_function>> create_functions_from_schema_partition(replica::database& db, lw_shared_ptr<query::result_set> result);
|
||||
|
||||
std::vector<mutation> make_create_function_mutations(shared_ptr<cql3::functions::user_function> func, api::timestamp_type timestamp);
|
||||
|
||||
@@ -234,7 +236,7 @@ void add_type_to_schema_mutation(user_type type, api::timestamp_type timestamp,
|
||||
std::vector<mutation> make_create_table_mutations(lw_shared_ptr<keyspace_metadata> keyspace, schema_ptr table, api::timestamp_type timestamp);
|
||||
|
||||
std::vector<mutation> make_update_table_mutations(
|
||||
database& db,
|
||||
replica::database& db,
|
||||
lw_shared_ptr<keyspace_metadata> keyspace,
|
||||
schema_ptr old_table,
|
||||
schema_ptr new_table,
|
||||
@@ -264,7 +266,7 @@ std::vector<mutation> make_drop_view_mutations(lw_shared_ptr<keyspace_metadata>
|
||||
|
||||
class preserve_version_tag {};
|
||||
using preserve_version = bool_class<preserve_version_tag>;
|
||||
view_ptr maybe_fix_legacy_secondary_index_mv_schema(database& db, const view_ptr& v, schema_ptr base_schema, preserve_version preserve_version);
|
||||
view_ptr maybe_fix_legacy_secondary_index_mv_schema(replica::database& db, const view_ptr& v, schema_ptr base_schema, preserve_version preserve_version);
|
||||
|
||||
sstring serialize_kind(column_kind kind);
|
||||
column_kind deserialize_kind(sstring kind);
|
||||
|
||||
@@ -123,7 +123,7 @@ public:
|
||||
/**
|
||||
* Returns the keyspaces, ordered by name, as selected by the partition_range.
|
||||
*/
|
||||
static std::vector<sstring> get_keyspaces(const schema& s, const database& db, dht::partition_range range) {
|
||||
static std::vector<sstring> get_keyspaces(const schema& s, const replica::database& db, dht::partition_range range) {
|
||||
struct keyspace_less_comparator {
|
||||
const schema& _s;
|
||||
keyspace_less_comparator(const schema& s) : _s(s) { }
|
||||
@@ -174,7 +174,7 @@ static dht::partition_range as_ring_position_range(dht::token_range& r) {
|
||||
/**
|
||||
* Add a new range_estimates for the specified range, considering the sstables associated with `cf`.
|
||||
*/
|
||||
static system_keyspace::range_estimates estimate(const column_family& cf, const token_range& r) {
|
||||
static system_keyspace::range_estimates estimate(const replica::column_family& cf, const token_range& r) {
|
||||
int64_t count{0};
|
||||
utils::estimated_histogram hist{0};
|
||||
auto from_bytes = [] (auto& b) {
|
||||
@@ -198,7 +198,7 @@ static system_keyspace::range_estimates estimate(const column_family& cf, const
|
||||
/**
|
||||
* Returns the primary ranges for the local node.
|
||||
*/
|
||||
static future<std::vector<token_range>> get_local_ranges(database& db) {
|
||||
static future<std::vector<token_range>> get_local_ranges(replica::database& db) {
|
||||
return db::system_keyspace::get_local_tokens().then([&db] (auto&& tokens) {
|
||||
auto ranges = db.get_token_metadata().get_primary_ranges_for(std::move(tokens));
|
||||
std::vector<token_range> local_ranges;
|
||||
@@ -230,11 +230,11 @@ static future<std::vector<token_range>> get_local_ranges(database& db) {
|
||||
});
|
||||
}
|
||||
|
||||
future<std::vector<token_range>> test_get_local_ranges(database& db) {
|
||||
future<std::vector<token_range>> test_get_local_ranges(replica::database& db) {
|
||||
return get_local_ranges(db);
|
||||
}
|
||||
|
||||
size_estimates_mutation_reader::size_estimates_mutation_reader(database& db, schema_ptr schema, reader_permit permit, const dht::partition_range& prange,
|
||||
size_estimates_mutation_reader::size_estimates_mutation_reader(replica::database& db, schema_ptr schema, reader_permit permit, const dht::partition_range& prange,
|
||||
const query::partition_slice& slice, streamed_mutation::forwarding fwd)
|
||||
: impl(std::move(schema), std::move(permit))
|
||||
, _db(db)
|
||||
|
||||
@@ -24,7 +24,9 @@
|
||||
#include "flat_mutation_reader_v2.hh"
|
||||
#include "db/system_keyspace.hh"
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace db {
|
||||
|
||||
@@ -36,7 +38,7 @@ struct token_range {
|
||||
};
|
||||
|
||||
class size_estimates_mutation_reader final : public flat_mutation_reader_v2::impl {
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
const dht::partition_range* _prange;
|
||||
const query::partition_slice& _slice;
|
||||
using ks_range = std::vector<sstring>;
|
||||
@@ -45,7 +47,7 @@ class size_estimates_mutation_reader final : public flat_mutation_reader_v2::imp
|
||||
streamed_mutation::forwarding _fwd;
|
||||
flat_mutation_reader_v2_opt _partition_reader;
|
||||
public:
|
||||
size_estimates_mutation_reader(database& db, schema_ptr, reader_permit, const dht::partition_range&, const query::partition_slice&, streamed_mutation::forwarding);
|
||||
size_estimates_mutation_reader(replica::database& db, schema_ptr, reader_permit, const dht::partition_range&, const query::partition_slice&, streamed_mutation::forwarding);
|
||||
|
||||
virtual future<> fill_buffer() override;
|
||||
virtual future<> next_partition() override;
|
||||
@@ -61,7 +63,7 @@ private:
|
||||
};
|
||||
|
||||
struct virtual_reader {
|
||||
database& db;
|
||||
replica::database& db;
|
||||
|
||||
flat_mutation_reader_v2 operator()(schema_ptr schema,
|
||||
reader_permit permit,
|
||||
@@ -74,10 +76,10 @@ struct virtual_reader {
|
||||
return make_flat_mutation_reader_v2<size_estimates_mutation_reader>(db, std::move(schema), std::move(permit), range, slice, fwd);
|
||||
}
|
||||
|
||||
virtual_reader(database& db_) noexcept : db(db_) {}
|
||||
virtual_reader(replica::database& db_) noexcept : db(db_) {}
|
||||
};
|
||||
|
||||
future<std::vector<token_range>> test_get_local_ranges(database& db);
|
||||
future<std::vector<token_range>> test_get_local_ranges(replica::database& db);
|
||||
|
||||
} // namespace size_estimates
|
||||
|
||||
|
||||
@@ -91,7 +91,7 @@ future<> snapshot_ctl::take_snapshot(sstring tag, std::vector<sstring> keyspace_
|
||||
return parallel_for_each(keyspace_names, [tag, this] (auto& ks_name) {
|
||||
return check_snapshot_not_exist(ks_name, tag);
|
||||
}).then([this, tag, keyspace_names, sf] {
|
||||
return _db.invoke_on_all([tag = std::move(tag), keyspace_names, sf] (database& db) {
|
||||
return _db.invoke_on_all([tag = std::move(tag), keyspace_names, sf] (replica::database& db) {
|
||||
return parallel_for_each(keyspace_names, [&db, tag = std::move(tag), sf] (auto& ks_name) {
|
||||
auto& ks = db.find_keyspace(ks_name);
|
||||
return parallel_for_each(ks.metadata()->cf_meta_data(), [&db, tag = std::move(tag), sf] (auto& pair) {
|
||||
@@ -122,7 +122,7 @@ future<> snapshot_ctl::take_column_family_snapshot(sstring ks_name, std::vector<
|
||||
if (table_name.find(".") != sstring::npos) {
|
||||
throw std::invalid_argument("Cannot take a snapshot of a secondary index by itself. Run snapshot on the table that owns the index.");
|
||||
}
|
||||
return _db.invoke_on_all([ks_name, table_name, tag, sf] (database &db) {
|
||||
return _db.invoke_on_all([ks_name, table_name, tag, sf] (replica::database &db) {
|
||||
auto& cf = db.find_column_family(ks_name, table_name);
|
||||
return cf.snapshot(db, tag, bool(sf));
|
||||
});
|
||||
@@ -144,7 +144,7 @@ future<> snapshot_ctl::clear_snapshot(sstring tag, std::vector<sstring> keyspace
|
||||
|
||||
future<std::unordered_map<sstring, std::vector<snapshot_ctl::snapshot_details>>>
|
||||
snapshot_ctl::get_snapshot_details() {
|
||||
using cf_snapshot_map = std::unordered_map<utils::UUID, column_family::snapshot_details>;
|
||||
using cf_snapshot_map = std::unordered_map<utils::UUID, replica::column_family::snapshot_details>;
|
||||
using snapshot_map = std::unordered_map<sstring, cf_snapshot_map>;
|
||||
|
||||
class snapshot_reducer {
|
||||
@@ -175,7 +175,7 @@ snapshot_ctl::get_snapshot_details() {
|
||||
};
|
||||
|
||||
return run_snapshot_list_operation([this] {
|
||||
return _db.map_reduce(snapshot_reducer(), [] (database& db) {
|
||||
return _db.map_reduce(snapshot_reducer(), [] (replica::database& db) {
|
||||
auto local_snapshots = make_lw_shared<snapshot_map>();
|
||||
return parallel_for_each(db.get_column_families(), [local_snapshots] (auto& cf_pair) {
|
||||
return cf_pair.second->get_snapshot_details().then([uuid = cf_pair.first, local_snapshots] (auto map) {
|
||||
@@ -207,7 +207,7 @@ snapshot_ctl::get_snapshot_details() {
|
||||
|
||||
future<int64_t> snapshot_ctl::true_snapshots_size() {
|
||||
return run_snapshot_list_operation([this] {
|
||||
return _db.map_reduce(adder<int64_t>(), [] (database& db) {
|
||||
return _db.map_reduce(adder<int64_t>(), [] (replica::database& db) {
|
||||
return do_with(int64_t(0), [&db] (auto& local_total) {
|
||||
return parallel_for_each(db.get_column_families(), [&local_total] (auto& cf_pair) {
|
||||
return cf_pair.second->get_snapshot_details().then([&local_total] (auto map) {
|
||||
|
||||
@@ -62,7 +62,7 @@ public:
|
||||
sstring cf;
|
||||
sstring ks;
|
||||
};
|
||||
explicit snapshot_ctl(sharded<database>& db) : _db(db) {}
|
||||
explicit snapshot_ctl(sharded<replica::database>& db) : _db(db) {}
|
||||
|
||||
future<> stop() {
|
||||
return _ops.close();
|
||||
@@ -114,7 +114,7 @@ public:
|
||||
|
||||
future<int64_t> true_snapshots_size();
|
||||
private:
|
||||
sharded<database>& _db;
|
||||
sharded<replica::database>& _db;
|
||||
seastar::rwlock _lock;
|
||||
seastar::gate _ops;
|
||||
|
||||
|
||||
@@ -41,7 +41,7 @@ void feature_enabled_listener::on_enabled() {
|
||||
}
|
||||
}
|
||||
|
||||
sstables_format_selector::sstables_format_selector(gms::gossiper& g, sharded<gms::feature_service>& f, sharded<database>& db)
|
||||
sstables_format_selector::sstables_format_selector(gms::gossiper& g, sharded<gms::feature_service>& f, sharded<replica::database>& db)
|
||||
: _gossiper(g)
|
||||
, _features(f)
|
||||
, _db(db)
|
||||
@@ -96,7 +96,7 @@ future<> sstables_format_selector::read_sstables_format() {
|
||||
future<> sstables_format_selector::select_format(sstables::sstable_version_types format) {
|
||||
logger.info("Selected {} sstables format", to_string(format));
|
||||
_selected_format = format;
|
||||
return _db.invoke_on_all([this] (database& db) {
|
||||
return _db.invoke_on_all([this] (replica::database& db) {
|
||||
db.set_format(_selected_format);
|
||||
});
|
||||
}
|
||||
|
||||
@@ -31,7 +31,9 @@
|
||||
|
||||
using namespace seastar;
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace gms {
|
||||
class gossiper;
|
||||
@@ -57,7 +59,7 @@ public:
|
||||
class sstables_format_selector {
|
||||
gms::gossiper& _gossiper;
|
||||
sharded<gms::feature_service>& _features;
|
||||
sharded<database>& _db;
|
||||
sharded<replica::database>& _db;
|
||||
seastar::named_semaphore _sem = {1, named_semaphore_exception_factory{"feature listeners"}};
|
||||
seastar::gate _sel;
|
||||
|
||||
@@ -70,7 +72,7 @@ class sstables_format_selector {
|
||||
future<> do_maybe_select_format(sstables::sstable_version_types new_format);
|
||||
|
||||
public:
|
||||
sstables_format_selector(gms::gossiper& g, sharded<gms::feature_service>& f, sharded<database>& db);
|
||||
sstables_format_selector(gms::gossiper& g, sharded<gms::feature_service>& f, sharded<replica::database>& db);
|
||||
|
||||
future<> start();
|
||||
future<> stop();
|
||||
|
||||
@@ -184,7 +184,7 @@ static std::vector<schema_ptr> ensured_tables() {
|
||||
}
|
||||
|
||||
// Precondition: `ks_name` is either "system_distributed" or "system_distributed_everywhere".
|
||||
static void check_exists(std::string_view ks_name, std::string_view cf_name, const database& db) {
|
||||
static void check_exists(std::string_view ks_name, std::string_view cf_name, const replica::database& db) {
|
||||
if (!db.has_schema(ks_name, cf_name)) {
|
||||
on_internal_error(dlogger, format("expected {}.{} to exist but it doesn't", ks_name, cf_name));
|
||||
}
|
||||
@@ -200,7 +200,7 @@ system_distributed_keyspace::system_distributed_keyspace(cql3::query_processor&
|
||||
, _sp(sp) {
|
||||
}
|
||||
|
||||
static future<> add_new_columns_if_missing(database& db, ::service::migration_manager& mm) noexcept {
|
||||
static future<> add_new_columns_if_missing(replica::database& db, ::service::migration_manager& mm) noexcept {
|
||||
static thread_local std::pair<std::string_view, data_type> new_columns[] {
|
||||
{"timeout", duration_type},
|
||||
{"workload_type", utf8_type}
|
||||
@@ -431,7 +431,7 @@ system_distributed_keyspace::read_cdc_topology_description(
|
||||
}
|
||||
|
||||
static future<utils::chunked_vector<mutation>> get_cdc_generation_mutations(
|
||||
const database& db,
|
||||
const replica::database& db,
|
||||
utils::UUID id,
|
||||
size_t num_replicas,
|
||||
size_t concurrency,
|
||||
@@ -548,7 +548,7 @@ system_distributed_keyspace::read_cdc_generation(utils::UUID id) {
|
||||
}
|
||||
|
||||
static future<std::vector<mutation>> get_cdc_streams_descriptions_v2_mutation(
|
||||
const database& db,
|
||||
const replica::database& db,
|
||||
db_clock::time_point time,
|
||||
const cdc::topology_description& desc) {
|
||||
auto s = db.find_schema(system_distributed_keyspace::NAME, system_distributed_keyspace::CDC_DESC_V2);
|
||||
|
||||
@@ -1374,7 +1374,7 @@ void system_keyspace::minimal_setup(distributed<cql3::query_processor>& qp) {
|
||||
qctx = std::make_unique<query_context>(qp);
|
||||
}
|
||||
|
||||
future<> system_keyspace::setup(distributed<database>& db,
|
||||
future<> system_keyspace::setup(distributed<replica::database>& db,
|
||||
distributed<cql3::query_processor>& qp,
|
||||
sharded<netw::messaging_service>& ms) {
|
||||
const db::config& cfg = db.local().get_config();
|
||||
@@ -1436,18 +1436,18 @@ future<truncation_record> system_keyspace::get_truncation_record(utils::UUID cf_
|
||||
}
|
||||
|
||||
// Read system.truncate table and cache last truncation time in `table` object for each table on every shard
|
||||
future<> system_keyspace::cache_truncation_record(distributed<database>& db) {
|
||||
future<> system_keyspace::cache_truncation_record(distributed<replica::database>& db) {
|
||||
sstring req = format("SELECT DISTINCT table_uuid, truncated_at from system.{}", TRUNCATED);
|
||||
return qctx->qp().execute_internal(req).then([&db] (::shared_ptr<cql3::untyped_result_set> rs) {
|
||||
return parallel_for_each(rs->begin(), rs->end(), [&db] (const cql3::untyped_result_set_row& row) {
|
||||
auto table_uuid = row.get_as<utils::UUID>("table_uuid");
|
||||
auto ts = row.get_as<db_clock::time_point>("truncated_at");
|
||||
|
||||
return db.invoke_on_all([table_uuid, ts] (database& db) mutable {
|
||||
return db.invoke_on_all([table_uuid, ts] (replica::database& db) mutable {
|
||||
try {
|
||||
table& cf = db.find_column_family(table_uuid);
|
||||
replica::table& cf = db.find_column_family(table_uuid);
|
||||
cf.cache_truncation_record(ts);
|
||||
} catch (no_such_column_family&) {
|
||||
} catch (replica::no_such_column_family&) {
|
||||
slogger.debug("Skip caching truncation time for {} since the table is no longer present", table_uuid);
|
||||
}
|
||||
});
|
||||
@@ -1462,7 +1462,7 @@ future<> system_keyspace::save_truncation_record(utils::UUID id, db_clock::time_
|
||||
});
|
||||
}
|
||||
|
||||
future<> system_keyspace::save_truncation_record(const column_family& cf, db_clock::time_point truncated_at, db::replay_position rp) {
|
||||
future<> system_keyspace::save_truncation_record(const replica::column_family& cf, db_clock::time_point truncated_at, db::replay_position rp) {
|
||||
return save_truncation_record(cf.schema()->id(), truncated_at, rp);
|
||||
}
|
||||
|
||||
@@ -1908,10 +1908,10 @@ public:
|
||||
|
||||
class token_ring_table : public streaming_virtual_table {
|
||||
private:
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
service::storage_service& _ss;
|
||||
public:
|
||||
token_ring_table(database& db, service::storage_service& ss)
|
||||
token_ring_table(replica::database& db, service::storage_service& ss)
|
||||
: streaming_virtual_table(build_schema())
|
||||
, _db(db)
|
||||
, _ss(ss)
|
||||
@@ -1998,9 +1998,9 @@ public:
|
||||
};
|
||||
|
||||
class snapshots_table : public streaming_virtual_table {
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
public:
|
||||
explicit snapshots_table(distributed<database>& db)
|
||||
explicit snapshots_table(distributed<replica::database>& db)
|
||||
: streaming_virtual_table(build_schema())
|
||||
, _db(db)
|
||||
{
|
||||
@@ -2051,7 +2051,7 @@ public:
|
||||
return less(l.key, r.key);
|
||||
});
|
||||
|
||||
using snapshots_by_tables_map = std::map<sstring, std::map<sstring, table::snapshot_details>>;
|
||||
using snapshots_by_tables_map = std::map<sstring, std::map<sstring, replica::table::snapshot_details>>;
|
||||
|
||||
class snapshot_reducer {
|
||||
private:
|
||||
@@ -2082,14 +2082,14 @@ public:
|
||||
for (auto& ks_data : keyspace_names) {
|
||||
co_await result.emit_partition_start(ks_data.key);
|
||||
|
||||
const auto snapshots_by_tables = co_await _db.map_reduce(snapshot_reducer(), [ks_name = ks_data.name] (database& db) -> future<snapshots_by_tables_map> {
|
||||
const auto snapshots_by_tables = co_await _db.map_reduce(snapshot_reducer(), [ks_name = ks_data.name] (replica::database& db) -> future<snapshots_by_tables_map> {
|
||||
snapshots_by_tables_map snapshots_by_tables;
|
||||
for (auto& [_, table] : db.get_column_families()) {
|
||||
if (table->schema()->ks_name() != ks_name) {
|
||||
continue;
|
||||
}
|
||||
const auto unordered_snapshots = co_await table->get_snapshot_details();
|
||||
snapshots_by_tables.emplace(table->schema()->cf_name(), std::map<sstring, table::snapshot_details>(unordered_snapshots.begin(), unordered_snapshots.end()));
|
||||
snapshots_by_tables.emplace(table->schema()->cf_name(), std::map<sstring, replica::table::snapshot_details>(unordered_snapshots.begin(), unordered_snapshots.end()));
|
||||
}
|
||||
co_return snapshots_by_tables;
|
||||
});
|
||||
@@ -2171,7 +2171,7 @@ public:
|
||||
|
||||
class runtime_info_table : public memtable_filling_virtual_table {
|
||||
private:
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
service::storage_service& _ss;
|
||||
std::optional<dht::decorated_key> _generic_key;
|
||||
|
||||
@@ -2220,7 +2220,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
future<T> map_reduce_tables(std::function<T(table&)> map, std::function<T(T, T)> reduce = std::plus<T>{}) {
|
||||
future<T> map_reduce_tables(std::function<T(replica::table&)> map, std::function<T(T, T)> reduce = std::plus<T>{}) {
|
||||
class shard_reducer {
|
||||
T _v{};
|
||||
std::function<T(T, T)> _reduce;
|
||||
@@ -2232,7 +2232,7 @@ private:
|
||||
}
|
||||
T get() && { return std::move(_v); }
|
||||
};
|
||||
co_return co_await _db.map_reduce(shard_reducer(reduce), [map, reduce] (database& db) {
|
||||
co_return co_await _db.map_reduce(shard_reducer(reduce), [map, reduce] (replica::database& db) {
|
||||
T val = {};
|
||||
for (auto& [_, table] : db.get_column_families()) {
|
||||
val = reduce(val, map(*table));
|
||||
@@ -2254,7 +2254,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
explicit runtime_info_table(distributed<database>& db, service::storage_service& ss)
|
||||
explicit runtime_info_table(distributed<replica::database>& db, service::storage_service& ss)
|
||||
: memtable_filling_virtual_table(build_schema())
|
||||
, _db(db)
|
||||
, _ss(ss) {
|
||||
@@ -2280,7 +2280,7 @@ public:
|
||||
});
|
||||
});
|
||||
co_await add_partition(mutation_sink, "load", [this] () -> future<sstring> {
|
||||
return map_reduce_tables<int64_t>([] (table& tbl) {
|
||||
return map_reduce_tables<int64_t>([] (replica::table& tbl) {
|
||||
return tbl.get_stats().live_disk_space_used;
|
||||
}).then([] (int64_t load) {
|
||||
return format("{}", load);
|
||||
@@ -2311,7 +2311,7 @@ public:
|
||||
uint64_t entries = 0;
|
||||
static stats reduce(stats a, stats b) { return stats{a.total + b.total, a.free + b.free, a.entries + b.entries}; }
|
||||
};
|
||||
return map_reduce_tables<stats>([] (table& t) {
|
||||
return map_reduce_tables<stats>([] (replica::table& t) {
|
||||
const auto s = t.active_memtable().region().occupancy();
|
||||
return stats{s.total_space(), s.free_space(), t.active_memtable().partition_count()};
|
||||
}, stats::reduce).then([] (stats s) {
|
||||
@@ -2342,7 +2342,7 @@ public:
|
||||
a.requests_moving_average + b.requests_moving_average};
|
||||
}
|
||||
};
|
||||
return _db.map_reduce0([] (database& db) {
|
||||
return _db.map_reduce0([] (replica::database& db) {
|
||||
stats res{};
|
||||
auto occupancy = db.row_cache_tracker().region().occupancy();
|
||||
res.total = occupancy.total_space();
|
||||
@@ -2371,7 +2371,7 @@ public:
|
||||
});
|
||||
});
|
||||
co_await add_partition(mutation_sink, "incremental_backup_enabled", [this] () {
|
||||
return _db.map_reduce0([] (database& db) {
|
||||
return _db.map_reduce0([] (replica::database& db) {
|
||||
return boost::algorithm::any_of(db.get_keyspaces(), [] (const auto& id_and_ks) {
|
||||
return id_and_ks.second.incremental_backups_enabled();
|
||||
});
|
||||
@@ -2510,7 +2510,7 @@ public:
|
||||
// Map from table's schema ID to table itself. Helps avoiding accidental duplication.
|
||||
static thread_local std::map<utils::UUID, std::unique_ptr<virtual_table>> virtual_tables;
|
||||
|
||||
void register_virtual_tables(distributed<database>& dist_db, distributed<service::storage_service>& dist_ss, sharded<gms::gossiper>& dist_gossiper, db::config& cfg) {
|
||||
void register_virtual_tables(distributed<replica::database>& dist_db, distributed<service::storage_service>& dist_ss, sharded<gms::gossiper>& dist_gossiper, db::config& cfg) {
|
||||
auto add_table = [] (std::unique_ptr<virtual_table>&& tbl) {
|
||||
virtual_tables[tbl->schema()->id()] = std::move(tbl);
|
||||
};
|
||||
@@ -2562,7 +2562,7 @@ std::vector<schema_ptr> system_keyspace::all_tables(const db::config& cfg) {
|
||||
return r;
|
||||
}
|
||||
|
||||
static void install_virtual_readers(database& db) {
|
||||
static void install_virtual_readers(replica::database& db) {
|
||||
db.find_column_family(system_keyspace::size_estimates()).set_virtual_reader(mutation_source(db::size_estimates::virtual_reader(db)));
|
||||
db.find_column_family(system_keyspace::v3::views_builds_in_progress()).set_virtual_reader(mutation_source(db::view::build_progress_virtual_reader(db)));
|
||||
db.find_column_family(system_keyspace::built_indexes()).set_virtual_reader(mutation_source(db::index::built_indexes_virtual_reader(db)));
|
||||
@@ -2574,13 +2574,13 @@ static void install_virtual_readers(database& db) {
|
||||
}
|
||||
}
|
||||
|
||||
static bool maybe_write_in_user_memory(schema_ptr s, database& db) {
|
||||
static bool maybe_write_in_user_memory(schema_ptr s, replica::database& db) {
|
||||
return (s.get() == system_keyspace::batchlog().get()) || (s.get() == system_keyspace::paxos().get())
|
||||
|| s == system_keyspace::v3::scylla_views_builds_in_progress()
|
||||
|| s == system_keyspace::raft();
|
||||
}
|
||||
|
||||
future<> system_keyspace_make(distributed<database>& dist_db, distributed<service::storage_service>& dist_ss, sharded<gms::gossiper>& dist_gossiper, db::config& cfg) {
|
||||
future<> system_keyspace_make(distributed<replica::database>& dist_db, distributed<service::storage_service>& dist_ss, sharded<gms::gossiper>& dist_gossiper, db::config& cfg) {
|
||||
register_virtual_tables(dist_db, dist_ss, dist_gossiper, cfg);
|
||||
|
||||
auto& db = dist_db.local();
|
||||
@@ -2595,7 +2595,7 @@ future<> system_keyspace_make(distributed<database>& dist_db, distributed<servic
|
||||
std::map<sstring, sstring>{},
|
||||
durable
|
||||
);
|
||||
co_await db.create_keyspace(ksm, dist_ss.local().get_erm_factory(), true, database::system_keyspace::yes);
|
||||
co_await db.create_keyspace(ksm, dist_ss.local().get_erm_factory(), true, replica::database::system_keyspace::yes);
|
||||
}
|
||||
auto& ks = db.find_keyspace(ks_name);
|
||||
auto cfg = ks.make_column_family_config(*table, db);
|
||||
@@ -2611,7 +2611,7 @@ future<> system_keyspace_make(distributed<database>& dist_db, distributed<servic
|
||||
install_virtual_readers(db);
|
||||
}
|
||||
|
||||
future<> system_keyspace::make(distributed<database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg) {
|
||||
future<> system_keyspace::make(distributed<replica::database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg) {
|
||||
return system_keyspace_make(db, ss, g, cfg);
|
||||
}
|
||||
|
||||
@@ -2650,7 +2650,7 @@ system_keyspace::load_dc_rack_info() {
|
||||
|
||||
future<foreign_ptr<lw_shared_ptr<reconcilable_result>>>
|
||||
system_keyspace::query_mutations(distributed<service::storage_proxy>& proxy, const sstring& ks_name, const sstring& cf_name) {
|
||||
database& db = proxy.local().get_db().local();
|
||||
replica::database& db = proxy.local().get_db().local();
|
||||
schema_ptr schema = db.find_schema(ks_name, cf_name);
|
||||
auto slice = partition_slice_builder(*schema).build();
|
||||
auto cmd = make_lw_shared<query::read_command>(schema->id(), schema->version(), std::move(slice), proxy.local().get_max_result_size(slice));
|
||||
@@ -2660,7 +2660,7 @@ system_keyspace::query_mutations(distributed<service::storage_proxy>& proxy, con
|
||||
|
||||
future<lw_shared_ptr<query::result_set>>
|
||||
system_keyspace::query(distributed<service::storage_proxy>& proxy, const sstring& ks_name, const sstring& cf_name) {
|
||||
database& db = proxy.local().get_db().local();
|
||||
replica::database& db = proxy.local().get_db().local();
|
||||
schema_ptr schema = db.find_schema(ks_name, cf_name);
|
||||
auto slice = partition_slice_builder(*schema).build();
|
||||
auto cmd = make_lw_shared<query::read_command>(schema->id(), schema->version(), std::move(slice), proxy.local().get_max_result_size(slice));
|
||||
|
||||
@@ -122,7 +122,7 @@ class system_keyspace {
|
||||
static future<> force_blocking_flush(sstring cfname);
|
||||
static future<> build_dc_rack_info();
|
||||
static future<> build_bootstrap_info();
|
||||
static future<> cache_truncation_record(distributed<database>& db);
|
||||
static future<> cache_truncation_record(distributed<replica::database>& db);
|
||||
template <typename Value>
|
||||
static future<> update_cached_values(gms::inet_address ep, sstring column_name, Value value);
|
||||
public:
|
||||
@@ -240,7 +240,7 @@ public:
|
||||
|
||||
static future<> init_local_cache();
|
||||
static future<> deinit_local_cache();
|
||||
static future<> setup(distributed<database>& db,
|
||||
static future<> setup(distributed<replica::database>& db,
|
||||
distributed<cql3::query_processor>& qp,
|
||||
sharded<netw::messaging_service>& ms);
|
||||
static future<> update_schema_version(utils::UUID version);
|
||||
@@ -267,7 +267,7 @@ public:
|
||||
static future<std::optional<sstring>> get_scylla_local_param(const sstring& key);
|
||||
|
||||
static std::vector<schema_ptr> all_tables(const db::config& cfg);
|
||||
static future<> make(distributed<database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg);
|
||||
static future<> make(distributed<replica::database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg);
|
||||
|
||||
/// overloads
|
||||
|
||||
@@ -325,7 +325,7 @@ public:
|
||||
typedef std::vector<db::replay_position> replay_positions;
|
||||
|
||||
static future<> save_truncation_record(utils::UUID, db_clock::time_point truncated_at, db::replay_position);
|
||||
static future<> save_truncation_record(const column_family&, db_clock::time_point truncated_at, db::replay_position);
|
||||
static future<> save_truncation_record(const replica::column_family&, db_clock::time_point truncated_at, db::replay_position);
|
||||
static future<replay_positions> get_truncated_position(utils::UUID);
|
||||
static future<db::replay_position> get_truncated_position(utils::UUID, uint32_t shard);
|
||||
static future<db_clock::time_point> get_truncated_at(utils::UUID);
|
||||
@@ -439,7 +439,7 @@ public:
|
||||
static future<> save_local_supported_features(const std::set<std::string_view>& feats);
|
||||
}; // class system_keyspace
|
||||
|
||||
future<> system_keyspace_make(distributed<database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g);
|
||||
future<> system_keyspace_make(distributed<replica::database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g);
|
||||
extern const char *const system_keyspace_CLIENTS;
|
||||
|
||||
} // namespace db
|
||||
|
||||
@@ -50,7 +50,7 @@ namespace db::view {
|
||||
// the same as the previous one (as a result of trimming cpu_id),
|
||||
// the duplicated fragment is ignored.
|
||||
class build_progress_virtual_reader {
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
|
||||
struct build_progress_reader : flat_mutation_reader::impl {
|
||||
column_id _scylla_next_token_col;
|
||||
@@ -65,7 +65,7 @@ class build_progress_virtual_reader {
|
||||
build_progress_reader(
|
||||
schema_ptr legacy_schema,
|
||||
reader_permit permit,
|
||||
column_family& scylla_views_build_progress,
|
||||
replica::column_family& scylla_views_build_progress,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
@@ -189,7 +189,7 @@ class build_progress_virtual_reader {
|
||||
};
|
||||
|
||||
public:
|
||||
build_progress_virtual_reader(database& db)
|
||||
build_progress_virtual_reader(replica::database& db)
|
||||
: _db(db) {
|
||||
}
|
||||
|
||||
|
||||
@@ -1271,7 +1271,7 @@ future<> mutate_MV(
|
||||
dht::token base_token,
|
||||
utils::chunked_vector<frozen_mutation_and_schema> view_updates,
|
||||
db::view::stats& stats,
|
||||
cf_stats& cf_stats,
|
||||
replica::cf_stats& cf_stats,
|
||||
tracing::trace_state_ptr tr_state,
|
||||
db::timeout_semaphore_units pending_view_updates,
|
||||
service::allow_hints allow_hints,
|
||||
@@ -1387,7 +1387,7 @@ future<> mutate_MV(
|
||||
});
|
||||
}
|
||||
|
||||
view_builder::view_builder(database& db, db::system_distributed_keyspace& sys_dist_ks, service::migration_notifier& mn)
|
||||
view_builder::view_builder(replica::database& db, db::system_distributed_keyspace& sys_dist_ks, service::migration_notifier& mn)
|
||||
: _db(db)
|
||||
, _sys_dist_ks(sys_dist_ks)
|
||||
, _mnotifier(mn)
|
||||
@@ -1625,7 +1625,7 @@ void view_builder::setup_shard_build_step(
|
||||
try {
|
||||
_db.find_schema(view->view_info()->base_id());
|
||||
return true;
|
||||
} catch (const no_such_column_family&) {
|
||||
} catch (const replica::no_such_column_family&) {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
@@ -1640,7 +1640,7 @@ void view_builder::setup_shard_build_step(
|
||||
}
|
||||
// The view was dropped and a table was re-created with the same name,
|
||||
// but the write to the view-related system tables didn't make it.
|
||||
} catch (const no_such_column_family&) {
|
||||
} catch (const replica::no_such_column_family&) {
|
||||
// Fall-through
|
||||
}
|
||||
if (this_shard_id() == 0) {
|
||||
@@ -1687,7 +1687,7 @@ future<> view_builder::calculate_shard_build_step(view_builder_init_state& vbi)
|
||||
try {
|
||||
_db.find_schema(view->view_info()->base_id());
|
||||
return true;
|
||||
} catch (const no_such_column_family&) {
|
||||
} catch (const replica::no_such_column_family&) {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
@@ -1749,7 +1749,7 @@ future<> view_builder::add_new_view(view_ptr view, build_step& step) {
|
||||
system_keyspace::register_view_for_building(view->ks_name(), view->cf_name(), step.current_token())).discard_result();
|
||||
}
|
||||
|
||||
static future<> flush_base(lw_shared_ptr<column_family> base, abort_source& as) {
|
||||
static future<> flush_base(lw_shared_ptr<replica::column_family> base, abort_source& as) {
|
||||
struct empty_state { };
|
||||
return exponential_backoff_retry::do_until_value(1s, 1min, as, [base = std::move(base)] {
|
||||
return base->flush().then_wrapped([base] (future<> f) -> std::optional<empty_state> {
|
||||
@@ -1784,7 +1784,7 @@ void view_builder::on_create_view(const sstring& ks_name, const sstring& view_na
|
||||
});
|
||||
});
|
||||
});
|
||||
}).handle_exception_type([] (no_such_column_family&) { });
|
||||
}).handle_exception_type([] (replica::no_such_column_family&) { });
|
||||
}
|
||||
|
||||
void view_builder::on_update_view(const sstring& ks_name, const sstring& view_name, bool) {
|
||||
@@ -1801,7 +1801,7 @@ void view_builder::on_update_view(const sstring& ks_name, const sstring& view_na
|
||||
if (status_it != step_it->second.build_status.end()) {
|
||||
status_it->view = std::move(view);
|
||||
}
|
||||
}).handle_exception_type([] (no_such_column_family&) { });
|
||||
}).handle_exception_type([] (replica::no_such_column_family&) { });
|
||||
}
|
||||
|
||||
void view_builder::on_drop_view(const sstring& ks_name, const sstring& view_name) {
|
||||
@@ -2157,7 +2157,7 @@ future<bool> check_view_build_ongoing(db::system_distributed_keyspace& sys_dist_
|
||||
});
|
||||
}
|
||||
|
||||
future<bool> check_needs_view_update_path(db::system_distributed_keyspace& sys_dist_ks, const table& t, streaming::stream_reason reason) {
|
||||
future<bool> check_needs_view_update_path(db::system_distributed_keyspace& sys_dist_ks, const replica::table& t, streaming::stream_reason reason) {
|
||||
if (is_internal_keyspace(t.schema()->ks_name())) {
|
||||
return make_ready_future<bool>(false);
|
||||
}
|
||||
@@ -2206,7 +2206,7 @@ void view_updating_consumer::maybe_flush_buffer_mid_partition() {
|
||||
}
|
||||
}
|
||||
|
||||
view_updating_consumer::view_updating_consumer(schema_ptr schema, reader_permit permit, table& table, std::vector<sstables::shared_sstable> excluded_sstables, const seastar::abort_source& as,
|
||||
view_updating_consumer::view_updating_consumer(schema_ptr schema, reader_permit permit, replica::table& table, std::vector<sstables::shared_sstable> excluded_sstables, const seastar::abort_source& as,
|
||||
evictable_reader_handle& staging_reader_handle)
|
||||
: view_updating_consumer(std::move(schema), std::move(permit), as, staging_reader_handle,
|
||||
[table = table.shared_from_this(), excluded_sstables = std::move(excluded_sstables)] (mutation m) mutable {
|
||||
|
||||
@@ -29,7 +29,10 @@
|
||||
#include "frozen_mutation.hh"
|
||||
|
||||
class frozen_mutation_and_schema;
|
||||
|
||||
namespace replica {
|
||||
struct cf_stats;
|
||||
}
|
||||
|
||||
namespace service {
|
||||
struct allow_hints_tag;
|
||||
@@ -232,7 +235,7 @@ future<> mutate_MV(
|
||||
dht::token base_token,
|
||||
utils::chunked_vector<frozen_mutation_and_schema> view_updates,
|
||||
db::view::stats& stats,
|
||||
cf_stats& cf_stats,
|
||||
replica::cf_stats& cf_stats,
|
||||
tracing::trace_state_ptr tr_state,
|
||||
db::timeout_semaphore_units pending_view_updates,
|
||||
service::allow_hints allow_hints,
|
||||
|
||||
@@ -57,7 +57,10 @@ namespace service {
|
||||
class migration_manager;
|
||||
} // namespace service
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
class exponential_backoff_retry;
|
||||
|
||||
namespace db::view {
|
||||
@@ -142,7 +145,7 @@ class view_builder final : public service::migration_listener::only_view_notific
|
||||
struct build_step final {
|
||||
// Ensure we pin the column_family. It may happen that all views are removed,
|
||||
// and that the base table is too before we can detect it.
|
||||
lw_shared_ptr<column_family> base;
|
||||
lw_shared_ptr<replica::column_family> base;
|
||||
query::partition_slice pslice;
|
||||
dht::partition_range prange;
|
||||
flat_mutation_reader reader{nullptr};
|
||||
@@ -156,7 +159,7 @@ class view_builder final : public service::migration_listener::only_view_notific
|
||||
|
||||
using base_to_build_step_type = std::unordered_map<utils::UUID, build_step>;
|
||||
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
db::system_distributed_keyspace& _sys_dist_ks;
|
||||
service::migration_notifier& _mnotifier;
|
||||
reader_permit _permit;
|
||||
@@ -195,7 +198,7 @@ public:
|
||||
static constexpr size_t batch_memory_max = 1024*1024;
|
||||
|
||||
public:
|
||||
view_builder(database&, db::system_distributed_keyspace&, service::migration_notifier&);
|
||||
view_builder(replica::database&, db::system_distributed_keyspace&, service::migration_notifier&);
|
||||
view_builder(view_builder&&) = delete;
|
||||
|
||||
/**
|
||||
|
||||
@@ -25,7 +25,9 @@
|
||||
#include "streaming/stream_reason.hh"
|
||||
#include "seastarx.hh"
|
||||
|
||||
namespace replica {
|
||||
class table;
|
||||
}
|
||||
|
||||
namespace db {
|
||||
|
||||
@@ -36,6 +38,6 @@ class system_distributed_keyspace;
|
||||
namespace db::view {
|
||||
|
||||
future<bool> check_view_build_ongoing(db::system_distributed_keyspace& sys_dist_ks, const sstring& ks_name, const sstring& cf_name);
|
||||
future<bool> check_needs_view_update_path(db::system_distributed_keyspace& sys_dist_ks, const table& t, streaming::stream_reason reason);
|
||||
future<bool> check_needs_view_update_path(db::system_distributed_keyspace& sys_dist_ks, const replica::table& t, streaming::stream_reason reason);
|
||||
|
||||
}
|
||||
|
||||
@@ -149,7 +149,7 @@ bool view_update_generator::should_throttle() const {
|
||||
return !_started.available();
|
||||
}
|
||||
|
||||
future<> view_update_generator::register_staging_sstable(sstables::shared_sstable sst, lw_shared_ptr<table> table) {
|
||||
future<> view_update_generator::register_staging_sstable(sstables::shared_sstable sst, lw_shared_ptr<replica::table> table) {
|
||||
if (_as.abort_requested()) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
@@ -184,7 +184,7 @@ void view_update_generator::setup_metrics() {
|
||||
|
||||
void view_update_generator::discover_staging_sstables() {
|
||||
for (auto& x : _db.get_column_families()) {
|
||||
table& t = *(x.second);
|
||||
replica::table& t = *(x.second);
|
||||
for (auto sstables = t.get_sstables(); sstables::shared_sstable sst : *sstables) {
|
||||
if (sst->requires_view_building()) {
|
||||
_sstables_with_tables[t.shared_from_this()].push_back(std::move(sst));
|
||||
|
||||
@@ -35,23 +35,23 @@ public:
|
||||
static constexpr size_t registration_queue_size = 5;
|
||||
|
||||
private:
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
seastar::abort_source _as;
|
||||
future<> _started = make_ready_future<>();
|
||||
seastar::condition_variable _pending_sstables;
|
||||
named_semaphore _registration_sem{registration_queue_size, named_semaphore_exception_factory{"view update generator"}};
|
||||
std::unordered_map<lw_shared_ptr<table>, std::vector<sstables::shared_sstable>> _sstables_with_tables;
|
||||
std::unordered_map<lw_shared_ptr<table>, std::vector<sstables::shared_sstable>> _sstables_to_move;
|
||||
std::unordered_map<lw_shared_ptr<replica::table>, std::vector<sstables::shared_sstable>> _sstables_with_tables;
|
||||
std::unordered_map<lw_shared_ptr<replica::table>, std::vector<sstables::shared_sstable>> _sstables_to_move;
|
||||
metrics::metric_groups _metrics;
|
||||
public:
|
||||
view_update_generator(database& db) : _db(db) {
|
||||
view_update_generator(replica::database& db) : _db(db) {
|
||||
setup_metrics();
|
||||
discover_staging_sstables();
|
||||
}
|
||||
|
||||
future<> start();
|
||||
future<> stop();
|
||||
future<> register_staging_sstable(sstables::shared_sstable sst, lw_shared_ptr<table> table);
|
||||
future<> register_staging_sstable(sstables::shared_sstable sst, lw_shared_ptr<replica::table> table);
|
||||
|
||||
ssize_t available_register_units() const { return _registration_sem.available_units(); }
|
||||
private:
|
||||
|
||||
@@ -71,7 +71,7 @@ public:
|
||||
, _view_update_pusher(std::move(view_update_pusher))
|
||||
{ }
|
||||
|
||||
view_updating_consumer(schema_ptr schema, reader_permit permit, table& table, std::vector<sstables::shared_sstable> excluded_sstables, const seastar::abort_source& as,
|
||||
view_updating_consumer(schema_ptr schema, reader_permit permit, replica::table& table, std::vector<sstables::shared_sstable> excluded_sstables, const seastar::abort_source& as,
|
||||
evictable_reader_handle& staging_reader_handle);
|
||||
|
||||
view_updating_consumer(view_updating_consumer&&) = default;
|
||||
|
||||
4
debug.hh
4
debug.hh
@@ -23,11 +23,13 @@
|
||||
|
||||
#include <seastar/core/sharded.hh>
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace debug {
|
||||
|
||||
extern seastar::sharded<database>* the_database;
|
||||
extern seastar::sharded<replica::database>* the_database;
|
||||
|
||||
|
||||
}
|
||||
|
||||
@@ -86,7 +86,7 @@ future<> boot_strapper::bootstrap(streaming::stream_reason reason, gms::gossiper
|
||||
}
|
||||
}
|
||||
|
||||
std::unordered_set<token> boot_strapper::get_bootstrap_tokens(const token_metadata_ptr tmptr, database& db) {
|
||||
std::unordered_set<token> boot_strapper::get_bootstrap_tokens(const token_metadata_ptr tmptr, replica::database& db) {
|
||||
auto initial_tokens = db.get_initial_tokens();
|
||||
// if user specified tokens, use those
|
||||
if (initial_tokens.size() > 0) {
|
||||
|
||||
@@ -55,7 +55,7 @@ class boot_strapper {
|
||||
using token_metadata = locator::token_metadata;
|
||||
using token_metadata_ptr = locator::token_metadata_ptr;
|
||||
using token = dht::token;
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
sharded<streaming::stream_manager>& _stream_manager;
|
||||
abort_source& _abort_source;
|
||||
/* endpoint that needs to be bootstrapped */
|
||||
@@ -64,7 +64,7 @@ class boot_strapper {
|
||||
std::unordered_set<token> _tokens;
|
||||
const token_metadata_ptr _token_metadata_ptr;
|
||||
public:
|
||||
boot_strapper(distributed<database>& db, sharded<streaming::stream_manager>& sm, abort_source& abort_source, inet_address addr, std::unordered_set<token> tokens, const token_metadata_ptr tmptr)
|
||||
boot_strapper(distributed<replica::database>& db, sharded<streaming::stream_manager>& sm, abort_source& abort_source, inet_address addr, std::unordered_set<token> tokens, const token_metadata_ptr tmptr)
|
||||
: _db(db)
|
||||
, _stream_manager(sm)
|
||||
, _abort_source(abort_source)
|
||||
@@ -80,7 +80,7 @@ public:
|
||||
* otherwise, if num_tokens == 1, pick a token to assume half the load of the most-loaded node.
|
||||
* else choose num_tokens tokens at random
|
||||
*/
|
||||
static std::unordered_set<token> get_bootstrap_tokens(const token_metadata_ptr tmptr, database& db);
|
||||
static std::unordered_set<token> get_bootstrap_tokens(const token_metadata_ptr tmptr, replica::database& db);
|
||||
|
||||
static std::unordered_set<token> get_random_tokens(const token_metadata_ptr tmptr, size_t num_tokens);
|
||||
#if 0
|
||||
|
||||
@@ -50,7 +50,10 @@
|
||||
#include <unordered_map>
|
||||
#include <memory>
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace gms { class gossiper; }
|
||||
|
||||
namespace dht {
|
||||
@@ -102,7 +105,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
range_streamer(distributed<database>& db, sharded<streaming::stream_manager>& sm, const token_metadata_ptr tmptr, abort_source& abort_source, std::unordered_set<token> tokens, inet_address address, sstring description, streaming::stream_reason reason)
|
||||
range_streamer(distributed<replica::database>& db, sharded<streaming::stream_manager>& sm, const token_metadata_ptr tmptr, abort_source& abort_source, std::unordered_set<token> tokens, inet_address address, sstring description, streaming::stream_reason reason)
|
||||
: _db(db)
|
||||
, _stream_manager(sm)
|
||||
, _token_metadata_ptr(std::move(tmptr))
|
||||
@@ -115,7 +118,7 @@ public:
|
||||
_abort_source.check();
|
||||
}
|
||||
|
||||
range_streamer(distributed<database>& db, sharded<streaming::stream_manager>& sm, const token_metadata_ptr tmptr, abort_source& abort_source, inet_address address, sstring description, streaming::stream_reason reason)
|
||||
range_streamer(distributed<replica::database>& db, sharded<streaming::stream_manager>& sm, const token_metadata_ptr tmptr, abort_source& abort_source, inet_address address, sstring description, streaming::stream_reason reason)
|
||||
: range_streamer(db, sm, std::move(tmptr), abort_source, std::unordered_set<token>(), address, description, reason) {
|
||||
}
|
||||
|
||||
@@ -170,7 +173,7 @@ public:
|
||||
future<> do_stream_async();
|
||||
size_t nr_ranges_to_stream();
|
||||
private:
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
sharded<streaming::stream_manager>& _stream_manager;
|
||||
const token_metadata_ptr _token_metadata_ptr;
|
||||
abort_source& _abort_source;
|
||||
|
||||
@@ -77,7 +77,7 @@ class dirty_memory_manager: public logalloc::region_group_reclaimer {
|
||||
// mounting, in which case the pressure flag could be set back on if we force it off.
|
||||
bool _db_shutdown_requested = false;
|
||||
|
||||
database* _db;
|
||||
replica::database* _db;
|
||||
// The _real_region_group protects against actual dirty memory usage hitting the maximum. Usage
|
||||
// for this group is the real dirty memory usage of the system.
|
||||
logalloc::region_group _real_region_group;
|
||||
@@ -151,7 +151,7 @@ public:
|
||||
//
|
||||
// We then set the soft limit to 80 % of the virtual dirty hard limit, which is equal to 40 % of
|
||||
// the user-supplied threshold.
|
||||
dirty_memory_manager(database& db, size_t threshold, double soft_limit, scheduling_group deferred_work_sg)
|
||||
dirty_memory_manager(replica::database& db, size_t threshold, double soft_limit, scheduling_group deferred_work_sg)
|
||||
: logalloc::region_group_reclaimer(threshold / 2, threshold * soft_limit / 2)
|
||||
, _real_dirty_reclaimer(threshold)
|
||||
, _db(&db)
|
||||
|
||||
@@ -77,20 +77,20 @@ io_error_handler error_handler_gen_for_upload_dir(disk_error_signal_type& dummy)
|
||||
|
||||
// global_column_family_ptr provides a way to easily retrieve local instance of a given column family.
|
||||
class global_column_family_ptr {
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
utils::UUID _id;
|
||||
private:
|
||||
column_family& get() const { return _db.local().find_column_family(_id); }
|
||||
replica::column_family& get() const { return _db.local().find_column_family(_id); }
|
||||
public:
|
||||
global_column_family_ptr(distributed<database>& db, sstring ks_name, sstring cf_name)
|
||||
global_column_family_ptr(distributed<replica::database>& db, sstring ks_name, sstring cf_name)
|
||||
: _db(db)
|
||||
, _id(_db.local().find_column_family(ks_name, cf_name).schema()->id()) {
|
||||
}
|
||||
|
||||
column_family* operator->() const {
|
||||
replica::column_family* operator->() const {
|
||||
return &get();
|
||||
}
|
||||
column_family& operator*() const {
|
||||
replica::column_family& operator*() const {
|
||||
return get();
|
||||
}
|
||||
};
|
||||
@@ -115,7 +115,7 @@ distributed_loader::process_sstable_dir(sharded<sstables::sstable_directory>& di
|
||||
}
|
||||
|
||||
future<>
|
||||
distributed_loader::lock_table(sharded<sstables::sstable_directory>& dir, sharded<database>& db, sstring ks_name, sstring cf_name) {
|
||||
distributed_loader::lock_table(sharded<sstables::sstable_directory>& dir, sharded<replica::database>& db, sstring ks_name, sstring cf_name) {
|
||||
return dir.invoke_on_all([&db, ks_name, cf_name] (sstables::sstable_directory& d) {
|
||||
auto& table = db.local().find_column_family(ks_name, cf_name);
|
||||
d.store_phaser(table.write_in_progress());
|
||||
@@ -195,7 +195,7 @@ distribute_reshard_jobs(sstables::sstable_directory::sstable_info_vector source)
|
||||
}
|
||||
|
||||
future<> run_resharding_jobs(sharded<sstables::sstable_directory>& dir, std::vector<reshard_shard_descriptor> reshard_jobs,
|
||||
sharded<database>& db, sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator) {
|
||||
sharded<replica::database>& db, sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator) {
|
||||
|
||||
uint64_t total_size = boost::accumulate(reshard_jobs | boost::adaptors::transformed(std::mem_fn(&reshard_shard_descriptor::size)), uint64_t(0));
|
||||
if (total_size == 0) {
|
||||
@@ -229,7 +229,7 @@ future<> run_resharding_jobs(sharded<sstables::sstable_directory>& dir, std::vec
|
||||
// - The second part calls each shard's distributed object to reshard the SSTables they were
|
||||
// assigned.
|
||||
future<>
|
||||
distributed_loader::reshard(sharded<sstables::sstable_directory>& dir, sharded<database>& db, sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator) {
|
||||
distributed_loader::reshard(sharded<sstables::sstable_directory>& dir, sharded<replica::database>& db, sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator) {
|
||||
return collect_all_shared_sstables(dir).then([] (sstables::sstable_directory::sstable_info_vector all_jobs) mutable {
|
||||
return distribute_reshard_jobs(std::move(all_jobs));
|
||||
}).then([&dir, &db, ks_name, table_name, creator = std::move(creator)] (std::vector<reshard_shard_descriptor> destinations) mutable {
|
||||
@@ -253,7 +253,7 @@ highest_version_seen(sharded<sstables::sstable_directory>& dir, sstables::sstabl
|
||||
}
|
||||
|
||||
future<>
|
||||
distributed_loader::reshape(sharded<sstables::sstable_directory>& dir, sharded<database>& db, sstables::reshape_mode mode,
|
||||
distributed_loader::reshape(sharded<sstables::sstable_directory>& dir, sharded<replica::database>& db, sstables::reshape_mode mode,
|
||||
sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator) {
|
||||
|
||||
auto start = std::chrono::steady_clock::now();
|
||||
@@ -273,7 +273,7 @@ distributed_loader::reshape(sharded<sstables::sstable_directory>& dir, sharded<d
|
||||
|
||||
// Loads SSTables into the main directory (or staging) and returns how many were loaded
|
||||
future<size_t>
|
||||
distributed_loader::make_sstables_available(sstables::sstable_directory& dir, sharded<database>& db,
|
||||
distributed_loader::make_sstables_available(sstables::sstable_directory& dir, sharded<replica::database>& db,
|
||||
sharded<db::view::view_update_generator>& view_update_generator, fs::path datadir, sstring ks, sstring cf) {
|
||||
|
||||
auto& table = db.local().find_column_family(ks, cf);
|
||||
@@ -319,7 +319,7 @@ distributed_loader::make_sstables_available(sstables::sstable_directory& dir, sh
|
||||
}
|
||||
|
||||
future<>
|
||||
distributed_loader::process_upload_dir(distributed<database>& db, distributed<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
distributed_loader::process_upload_dir(distributed<replica::database>& db, distributed<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
distributed<db::view::view_update_generator>& view_update_generator, sstring ks, sstring cf) {
|
||||
seastar::thread_attributes attr;
|
||||
attr.sched_group = db.local().get_streaming_scheduling_group();
|
||||
@@ -388,7 +388,7 @@ distributed_loader::process_upload_dir(distributed<database>& db, distributed<db
|
||||
}
|
||||
|
||||
future<std::tuple<utils::UUID, std::vector<std::vector<sstables::shared_sstable>>>>
|
||||
distributed_loader::get_sstables_from_upload_dir(distributed<database>& db, sstring ks, sstring cf) {
|
||||
distributed_loader::get_sstables_from_upload_dir(distributed<replica::database>& db, sstring ks, sstring cf) {
|
||||
return seastar::async([&db, ks = std::move(ks), cf = std::move(cf)] {
|
||||
global_column_family_ptr global_table(db, ks, cf);
|
||||
sharded<sstables::sstable_directory> directory;
|
||||
@@ -464,7 +464,7 @@ future<> distributed_loader::handle_sstables_pending_delete(sstring pending_dele
|
||||
});
|
||||
}
|
||||
|
||||
future<> distributed_loader::populate_column_family(distributed<database>& db, sstring sstdir, sstring ks, sstring cf, bool must_exist) {
|
||||
future<> distributed_loader::populate_column_family(distributed<replica::database>& db, sstring sstdir, sstring ks, sstring cf, bool must_exist) {
|
||||
return async([&db, sstdir = std::move(sstdir), ks = std::move(ks), cf = std::move(cf), must_exist] {
|
||||
assert(this_shard_id() == 0);
|
||||
|
||||
@@ -509,7 +509,7 @@ future<> distributed_loader::populate_column_family(distributed<database>& db, s
|
||||
auto sst_version = highest_version_seen(directory, sys_format).get0();
|
||||
auto generation = highest_generation_seen(directory).get0();
|
||||
|
||||
db.invoke_on_all([&global_table, generation] (database& db) {
|
||||
db.invoke_on_all([&global_table, generation] (replica::database& db) {
|
||||
global_table->update_sstables_known_generation(generation);
|
||||
return global_table->disable_auto_compaction();
|
||||
}).get();
|
||||
@@ -537,7 +537,7 @@ future<> distributed_loader::populate_column_family(distributed<database>& db, s
|
||||
});
|
||||
}
|
||||
|
||||
future<> distributed_loader::populate_keyspace(distributed<database>& db, sstring datadir, sstring ks_name) {
|
||||
future<> distributed_loader::populate_keyspace(distributed<replica::database>& db, sstring datadir, sstring ks_name) {
|
||||
auto ksdir = datadir + "/" + ks_name;
|
||||
auto& keyspaces = db.local().get_keyspaces();
|
||||
auto i = keyspaces.find(ks_name);
|
||||
@@ -552,7 +552,7 @@ future<> distributed_loader::populate_keyspace(distributed<database>& db, sstrin
|
||||
return parallel_for_each(ks.metadata()->cf_meta_data() | boost::adaptors::map_values,
|
||||
[ks_name, ksdir, &ks, &column_families, &db] (schema_ptr s) {
|
||||
utils::UUID uuid = s->id();
|
||||
lw_shared_ptr<column_family> cf = column_families[uuid];
|
||||
lw_shared_ptr<replica::column_family> cf = column_families[uuid];
|
||||
sstring cfname = cf->schema()->cf_name();
|
||||
auto sstdir = ks.column_family_directory(ksdir, cfname, uuid);
|
||||
dblog.info("Keyspace {}: Reading CF {} id={} version={}", ks_name, cfname, uuid, s->version());
|
||||
@@ -580,9 +580,9 @@ future<> distributed_loader::populate_keyspace(distributed<database>& db, sstrin
|
||||
}
|
||||
}
|
||||
|
||||
future<> distributed_loader::init_system_keyspace(distributed<database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg) {
|
||||
future<> distributed_loader::init_system_keyspace(distributed<replica::database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg) {
|
||||
return seastar::async([&db, &ss, &cfg, &g] {
|
||||
db.invoke_on_all([&db, &ss, &cfg, &g] (database&) {
|
||||
db.invoke_on_all([&db, &ss, &cfg, &g] (replica::database&) {
|
||||
return db::system_keyspace::make(db, ss, g, cfg);
|
||||
}).get();
|
||||
|
||||
@@ -593,7 +593,7 @@ future<> distributed_loader::init_system_keyspace(distributed<database>& db, dis
|
||||
}
|
||||
}
|
||||
|
||||
db.invoke_on_all([] (database& db) {
|
||||
db.invoke_on_all([] (replica::database& db) {
|
||||
for (auto ksname : system_keyspaces) {
|
||||
auto& ks = db.find_keyspace(ksname);
|
||||
for (auto& pair : ks.metadata()->cf_meta_data()) {
|
||||
@@ -612,7 +612,7 @@ future<> distributed_loader::init_system_keyspace(distributed<database>& db, dis
|
||||
});
|
||||
}
|
||||
|
||||
future<> distributed_loader::ensure_system_table_directories(distributed<database>& db) {
|
||||
future<> distributed_loader::ensure_system_table_directories(distributed<replica::database>& db) {
|
||||
return parallel_for_each(system_keyspaces, [&db](std::string_view ksname) {
|
||||
auto& ks = db.local().find_keyspace(ksname);
|
||||
return parallel_for_each(ks.metadata()->cf_meta_data(), [&ks] (auto& pair) {
|
||||
@@ -622,10 +622,10 @@ future<> distributed_loader::ensure_system_table_directories(distributed<databas
|
||||
});
|
||||
}
|
||||
|
||||
future<> distributed_loader::init_non_system_keyspaces(distributed<database>& db,
|
||||
future<> distributed_loader::init_non_system_keyspaces(distributed<replica::database>& db,
|
||||
distributed<service::storage_proxy>& proxy) {
|
||||
return seastar::async([&db, &proxy] {
|
||||
db.invoke_on_all([&proxy] (database& db) {
|
||||
db.invoke_on_all([&proxy] (replica::database& db) {
|
||||
return db.parse_system_tables(proxy);
|
||||
}).get();
|
||||
|
||||
@@ -644,7 +644,7 @@ future<> distributed_loader::init_non_system_keyspaces(distributed<database>& db
|
||||
});
|
||||
}).get();
|
||||
|
||||
db.invoke_on_all([&dirs] (database& db) {
|
||||
db.invoke_on_all([&dirs] (replica::database& db) {
|
||||
for (auto& [name, ks] : db.get_keyspaces()) {
|
||||
// mark all user keyspaces that are _not_ on disk as already
|
||||
// populated.
|
||||
@@ -668,7 +668,7 @@ future<> distributed_loader::init_non_system_keyspaces(distributed<database>& db
|
||||
auto& datadir = p.second;
|
||||
return distributed_loader::populate_keyspace(db, datadir, ks_name);
|
||||
}).finally([&] {
|
||||
return db.invoke_on_all([ks_name] (database& db) {
|
||||
return db.invoke_on_all([ks_name] (replica::database& db) {
|
||||
// can be false if running test environment
|
||||
// or ks_name was just a borked directory not representing
|
||||
// a keyspace in schema tables.
|
||||
@@ -682,8 +682,8 @@ future<> distributed_loader::init_non_system_keyspaces(distributed<database>& db
|
||||
|
||||
when_all_succeed(futures.begin(), futures.end()).discard_result().get();
|
||||
|
||||
db.invoke_on_all([] (database& db) {
|
||||
return parallel_for_each(db.get_non_system_column_families(), [] (lw_shared_ptr<table> table) {
|
||||
db.invoke_on_all([] (replica::database& db) {
|
||||
return parallel_for_each(db.get_non_system_column_families(), [] (lw_shared_ptr<replica::table> table) {
|
||||
// Make sure this is called even if the table is empty
|
||||
table->mark_ready_for_writes();
|
||||
return make_ready_future<>();
|
||||
|
||||
@@ -32,9 +32,12 @@
|
||||
#include "seastarx.hh"
|
||||
#include "compaction/compaction_descriptor.hh"
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
class table;
|
||||
using column_family = table;
|
||||
}
|
||||
|
||||
namespace db {
|
||||
class config;
|
||||
class system_distributed_keyspace;
|
||||
@@ -65,30 +68,30 @@ class gossiper;
|
||||
class distributed_loader {
|
||||
friend class distributed_loader_for_tests;
|
||||
|
||||
static future<> reshape(sharded<sstables::sstable_directory>& dir, sharded<database>& db, sstables::reshape_mode mode,
|
||||
static future<> reshape(sharded<sstables::sstable_directory>& dir, sharded<replica::database>& db, sstables::reshape_mode mode,
|
||||
sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator);
|
||||
static future<> reshard(sharded<sstables::sstable_directory>& dir, sharded<database>& db, sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator);
|
||||
static future<> reshard(sharded<sstables::sstable_directory>& dir, sharded<replica::database>& db, sstring ks_name, sstring table_name, sstables::compaction_sstable_creator_fn creator);
|
||||
static future<> process_sstable_dir(sharded<sstables::sstable_directory>& dir, bool sort_sstables_according_to_owner = true);
|
||||
static future<> lock_table(sharded<sstables::sstable_directory>& dir, sharded<database>& db, sstring ks_name, sstring cf_name);
|
||||
static future<> lock_table(sharded<sstables::sstable_directory>& dir, sharded<replica::database>& db, sstring ks_name, sstring cf_name);
|
||||
static future<size_t> make_sstables_available(sstables::sstable_directory& dir,
|
||||
sharded<database>& db, sharded<db::view::view_update_generator>& view_update_generator,
|
||||
sharded<replica::database>& db, sharded<db::view::view_update_generator>& view_update_generator,
|
||||
std::filesystem::path datadir, sstring ks, sstring cf);
|
||||
static future<> populate_column_family(distributed<database>& db, sstring sstdir, sstring ks, sstring cf, bool must_exist = true);
|
||||
static future<> populate_keyspace(distributed<database>& db, sstring datadir, sstring ks_name);
|
||||
static future<> populate_column_family(distributed<replica::database>& db, sstring sstdir, sstring ks, sstring cf, bool must_exist = true);
|
||||
static future<> populate_keyspace(distributed<replica::database>& db, sstring datadir, sstring ks_name);
|
||||
static future<> cleanup_column_family_temp_sst_dirs(sstring sstdir);
|
||||
static future<> handle_sstables_pending_delete(sstring pending_deletes_dir);
|
||||
|
||||
public:
|
||||
static future<> init_system_keyspace(distributed<database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg);
|
||||
static future<> init_non_system_keyspaces(distributed<database>& db, distributed<service::storage_proxy>& proxy);
|
||||
static future<> ensure_system_table_directories(distributed<database>& db);
|
||||
static future<> init_system_keyspace(distributed<replica::database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg);
|
||||
static future<> init_non_system_keyspaces(distributed<replica::database>& db, distributed<service::storage_proxy>& proxy);
|
||||
static future<> ensure_system_table_directories(distributed<replica::database>& db);
|
||||
|
||||
// Scan sstables under upload directory. Return a vector with smp::count entries.
|
||||
// Each entry with index of idx should be accessed on shard idx only.
|
||||
// Each entry contains a vector of sstables for this shard.
|
||||
// The table UUID is returned too.
|
||||
static future<std::tuple<utils::UUID, std::vector<std::vector<sstables::shared_sstable>>>>
|
||||
get_sstables_from_upload_dir(distributed<database>& db, sstring ks, sstring cf);
|
||||
static future<> process_upload_dir(distributed<database>& db, distributed<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
get_sstables_from_upload_dir(distributed<replica::database>& db, sstring ks, sstring cf);
|
||||
static future<> process_upload_dir(distributed<replica::database>& db, distributed<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
distributed<db::view::view_update_generator>& view_update_generator, sstring ks_name, sstring cf_name);
|
||||
};
|
||||
|
||||
@@ -35,10 +35,10 @@
|
||||
namespace db::index {
|
||||
|
||||
class built_indexes_virtual_reader {
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
|
||||
struct built_indexes_reader : flat_mutation_reader::impl {
|
||||
database& _db;
|
||||
replica::database& _db;
|
||||
schema_ptr _schema;
|
||||
query::partition_slice _view_names_slice;
|
||||
flat_mutation_reader _underlying;
|
||||
@@ -111,10 +111,10 @@ class built_indexes_virtual_reader {
|
||||
}
|
||||
|
||||
built_indexes_reader(
|
||||
database& db,
|
||||
replica::database& db,
|
||||
schema_ptr schema,
|
||||
reader_permit permit,
|
||||
column_family& built_views,
|
||||
replica::column_family& built_views,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
@@ -141,7 +141,7 @@ class built_indexes_virtual_reader {
|
||||
try {
|
||||
auto s = _db.find_schema(_current_keyspace, view_name);
|
||||
return s->is_view() && _db.find_column_family(s->view_info()->base_id()).get_index_manager().is_index(view_ptr(s));
|
||||
} catch (const no_such_column_family&) {
|
||||
} catch (const replica::no_such_column_family&) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -222,7 +222,7 @@ class built_indexes_virtual_reader {
|
||||
};
|
||||
|
||||
public:
|
||||
built_indexes_virtual_reader(database& db)
|
||||
built_indexes_virtual_reader(replica::database& db)
|
||||
: _db(db) {
|
||||
}
|
||||
|
||||
|
||||
@@ -73,7 +73,7 @@ const index_metadata& index::metadata() const {
|
||||
return _im;
|
||||
}
|
||||
|
||||
secondary_index_manager::secondary_index_manager(column_family& cf)
|
||||
secondary_index_manager::secondary_index_manager(replica::column_family& cf)
|
||||
: _cf{cf}
|
||||
{}
|
||||
|
||||
|
||||
@@ -79,11 +79,11 @@ public:
|
||||
};
|
||||
|
||||
class secondary_index_manager {
|
||||
column_family& _cf;
|
||||
replica::column_family& _cf;
|
||||
/// The key of the map is the name of the index as stored in system tables.
|
||||
std::unordered_map<sstring, index> _indices;
|
||||
public:
|
||||
secondary_index_manager(column_family& cf);
|
||||
secondary_index_manager(replica::column_family& cf);
|
||||
void reload();
|
||||
view_ptr create_view_for_index(const index_metadata& index, bool new_token_column_computation) const;
|
||||
std::vector<index_metadata> get_dependent_indices(const column_definition& cdef) const;
|
||||
|
||||
@@ -33,7 +33,9 @@
|
||||
#include "utils/maybe_yield.hh"
|
||||
|
||||
// forward declaration since replica/database.hh includes this file
|
||||
namespace replica {
|
||||
class keyspace;
|
||||
}
|
||||
|
||||
namespace locator {
|
||||
|
||||
|
||||
@@ -54,7 +54,9 @@
|
||||
#include <seastar/core/semaphore.hh>
|
||||
|
||||
// forward declaration since replica/database.hh includes this file
|
||||
namespace replica {
|
||||
class keyspace;
|
||||
}
|
||||
|
||||
namespace locator {
|
||||
|
||||
|
||||
24
main.cc
24
main.cc
@@ -397,7 +397,7 @@ sharded<cql3::query_processor>* the_query_processor;
|
||||
sharded<qos::service_level_controller>* the_sl_controller;
|
||||
sharded<service::migration_manager>* the_migration_manager;
|
||||
sharded<service::storage_service>* the_storage_service;
|
||||
sharded<database>* the_database;
|
||||
sharded<replica::database>* the_database;
|
||||
sharded<streaming::stream_manager> *the_stream_manager;
|
||||
}
|
||||
|
||||
@@ -491,7 +491,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
sharded<locator::effective_replication_map_factory> erm_factory;
|
||||
sharded<service::migration_notifier> mm_notifier;
|
||||
sharded<service::endpoint_lifecycle_notifier> lifecycle_notifier;
|
||||
distributed<database> db;
|
||||
distributed<replica::database> db;
|
||||
seastar::sharded<service::cache_hitrate_calculator> cf_cache_hitrate_calculator;
|
||||
service::load_meter load_meter;
|
||||
auto& proxy = service::get_storage_proxy();
|
||||
@@ -747,7 +747,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
api::set_server_config(ctx, *cfg).get();
|
||||
|
||||
// Note: changed from using a move here, because we want the config object intact.
|
||||
database_config dbcfg;
|
||||
replica::database_config dbcfg;
|
||||
dbcfg.compaction_scheduling_group = make_sched_group("compaction", 1000);
|
||||
dbcfg.memory_compaction_scheduling_group = make_sched_group("mem_compaction", 1000);
|
||||
dbcfg.streaming_scheduling_group = maintenance_scheduling_group;
|
||||
@@ -900,14 +900,14 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
// #293 - do not stop anything - not even db (for real)
|
||||
//return db.stop();
|
||||
// call stop on each db instance, but leave the shareded<database> pointers alive.
|
||||
db.invoke_on_all(&database::stop).get();
|
||||
db.invoke_on_all(&replica::database::stop).get();
|
||||
});
|
||||
|
||||
// We need to init commitlog on shard0 before it is inited on other shards
|
||||
// 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(&database::start).get();
|
||||
db.invoke_on_all(&replica::database::start).get();
|
||||
|
||||
// Initialization of a keyspace is done by shard 0 only. For system
|
||||
// keyspace, the procedure will go through the hardcoded column
|
||||
@@ -957,7 +957,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
supervisor::notify("starting query processor");
|
||||
cql3::query_processor::memory_config qp_mcfg = {memory::stats().total_memory() / 256, memory::stats().total_memory() / 2560};
|
||||
debug::the_query_processor = &qp;
|
||||
auto local_data_dict = seastar::sharded_parameter([] (const database& db) { return db.as_data_dictionary(); }, std::ref(db));
|
||||
auto local_data_dict = seastar::sharded_parameter([] (const replica::database& db) { return db.as_data_dictionary(); }, std::ref(db));
|
||||
qp.start(std::ref(proxy), std::move(local_data_dict), std::ref(mm_notifier), std::ref(mm), qp_mcfg, std::ref(cql_config)).get();
|
||||
// #293 - do not stop anything
|
||||
// engine().at_exit([&qp] { return qp.stop(); });
|
||||
@@ -1013,7 +1013,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
auto rp = db::commitlog_replayer::create_replayer(db).get0();
|
||||
rp.recover(paths, db::commitlog::descriptor::FILENAME_PREFIX).get();
|
||||
supervisor::notify("replaying commit log - flushing memtables");
|
||||
db.invoke_on_all([] (database& db) {
|
||||
db.invoke_on_all([] (replica::database& db) {
|
||||
return db.flush_all_memtables();
|
||||
}).get();
|
||||
supervisor::notify("replaying commit log - removing old commitlog segments");
|
||||
@@ -1022,9 +1022,9 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
}
|
||||
}
|
||||
|
||||
db.invoke_on_all([] (database& db) {
|
||||
db.invoke_on_all([] (replica::database& db) {
|
||||
for (auto& x : db.get_column_families()) {
|
||||
table& t = *(x.second);
|
||||
replica::table& t = *(x.second);
|
||||
t.enable_auto_compaction();
|
||||
}
|
||||
}).get();
|
||||
@@ -1040,9 +1040,9 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
// group that was effectively used in the bulk of it (compaction). Soon it will become
|
||||
// streaming
|
||||
|
||||
db.invoke_on_all([&proxy] (database& db) {
|
||||
db.invoke_on_all([&proxy] (replica::database& db) {
|
||||
for (auto& x : db.get_column_families()) {
|
||||
column_family& cf = *(x.second);
|
||||
replica::column_family& cf = *(x.second);
|
||||
cf.trigger_compaction();
|
||||
}
|
||||
}).get();
|
||||
@@ -1342,7 +1342,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
|
||||
// Truncate `clients' CF - this table should not persist between server restarts.
|
||||
clear_clientlist().get();
|
||||
|
||||
db.invoke_on_all([] (database& db) {
|
||||
db.invoke_on_all([] (replica::database& db) {
|
||||
db.revert_initial_system_read_concurrency_boost();
|
||||
}).get();
|
||||
|
||||
|
||||
@@ -123,7 +123,7 @@ void memtable::memtable_encoding_stats_collector::update(const ::schema& s, cons
|
||||
}
|
||||
}
|
||||
|
||||
memtable::memtable(schema_ptr schema, dirty_memory_manager& dmm, table_stats& table_stats,
|
||||
memtable::memtable(schema_ptr schema, dirty_memory_manager& dmm, replica::table_stats& table_stats,
|
||||
memtable_list* memtable_list, seastar::scheduling_group compaction_scheduling_group)
|
||||
: logalloc::region(dmm.region_group())
|
||||
, _dirty_mgr(dmm)
|
||||
@@ -135,7 +135,7 @@ memtable::memtable(schema_ptr schema, dirty_memory_manager& dmm, table_stats& ta
|
||||
}
|
||||
|
||||
static thread_local dirty_memory_manager mgr_for_tests;
|
||||
static thread_local table_stats stats_for_tests;
|
||||
static thread_local replica::table_stats stats_for_tests;
|
||||
|
||||
memtable::memtable(schema_ptr schema)
|
||||
: memtable(std::move(schema), mgr_for_tests, stats_for_tests)
|
||||
|
||||
@@ -106,7 +106,10 @@ public:
|
||||
};
|
||||
|
||||
class dirty_memory_manager;
|
||||
|
||||
namespace replica {
|
||||
struct table_stats;
|
||||
}
|
||||
|
||||
// Managed by lw_shared_ptr<>.
|
||||
class memtable final : public enable_lw_shared_from_this<memtable>, private logalloc::region {
|
||||
@@ -133,7 +136,7 @@ private:
|
||||
// monotonic. That combined source in this case is cache + memtable.
|
||||
mutation_source_opt _underlying;
|
||||
uint64_t _flushed_memory = 0;
|
||||
table_stats& _table_stats;
|
||||
replica::table_stats& _table_stats;
|
||||
|
||||
class memtable_encoding_stats_collector : public encoding_stats_collector {
|
||||
private:
|
||||
@@ -178,7 +181,7 @@ private:
|
||||
void clear() noexcept;
|
||||
uint64_t dirty_size() const;
|
||||
public:
|
||||
explicit memtable(schema_ptr schema, dirty_memory_manager&, table_stats& table_stats, memtable_list *memtable_list = nullptr,
|
||||
explicit memtable(schema_ptr schema, dirty_memory_manager&, replica::table_stats& table_stats, memtable_list *memtable_list = nullptr,
|
||||
seastar::scheduling_group compaction_scheduling_group = seastar::current_scheduling_group());
|
||||
// Used for testing that want to control the flush process.
|
||||
explicit memtable(schema_ptr schema);
|
||||
|
||||
@@ -201,7 +201,7 @@ class read_context : public reader_lifecycle_policy {
|
||||
}
|
||||
};
|
||||
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
schema_ptr _schema;
|
||||
reader_permit _permit;
|
||||
const query::read_command& _cmd;
|
||||
@@ -219,7 +219,7 @@ class read_context : public reader_lifecycle_policy {
|
||||
future<> save_reader(shard_id shard, const dht::decorated_key& last_pkey, const std::optional<clustering_key_prefix>& last_ckey);
|
||||
|
||||
public:
|
||||
read_context(distributed<database>& db, schema_ptr s, const query::read_command& cmd, const dht::partition_range_vector& ranges,
|
||||
read_context(distributed<replica::database>& db, schema_ptr s, const query::read_command& cmd, const dht::partition_range_vector& ranges,
|
||||
tracing::trace_state_ptr trace_state, db::timeout_clock::time_point timeout)
|
||||
: _db(db)
|
||||
, _schema(std::move(s))
|
||||
@@ -238,7 +238,7 @@ public:
|
||||
read_context& operator=(read_context&&) = delete;
|
||||
read_context& operator=(const read_context&) = delete;
|
||||
|
||||
distributed<database>& db() {
|
||||
distributed<replica::database>& db() {
|
||||
return _db;
|
||||
}
|
||||
|
||||
@@ -383,7 +383,7 @@ future<> read_context::destroy_reader(stopped_reader reader) noexcept {
|
||||
future<> read_context::stop() {
|
||||
return parallel_for_each(smp::all_cpus(), [this] (unsigned shard) {
|
||||
if (_readers[shard].rparts) {
|
||||
return _db.invoke_on(shard, [&rparts_fptr = _readers[shard].rparts] (database& db) mutable {
|
||||
return _db.invoke_on(shard, [&rparts_fptr = _readers[shard].rparts] (replica::database& db) mutable {
|
||||
auto rparts = rparts_fptr.release();
|
||||
if (rparts->handle) {
|
||||
auto reader_opt = rparts->permit.semaphore().unregister_inactive_read(std::move(*rparts->handle));
|
||||
@@ -486,7 +486,7 @@ read_context::dismantle_buffer_stats read_context::dismantle_compaction_state(de
|
||||
future<> read_context::save_reader(shard_id shard, const dht::decorated_key& last_pkey, const std::optional<clustering_key_prefix>& last_ckey) {
|
||||
return do_with(std::exchange(_readers[shard], {}), [this, shard, &last_pkey, &last_ckey] (reader_meta& rm) mutable {
|
||||
return _db.invoke_on(shard, [this, query_uuid = _cmd.query_uuid, query_ranges = _ranges, &rm,
|
||||
&last_pkey, &last_ckey, gts = tracing::global_trace_state_ptr(_trace_state)] (database& db) mutable {
|
||||
&last_pkey, &last_ckey, gts = tracing::global_trace_state_ptr(_trace_state)] (replica::database& db) mutable {
|
||||
try {
|
||||
auto rparts = rm.rparts.release(); // avoid another round-trip when destroying rparts
|
||||
auto reader_opt = rparts->permit.semaphore().unregister_inactive_read(std::move(*rparts->handle));
|
||||
@@ -559,7 +559,7 @@ future<> read_context::lookup_readers(db::timeout_clock::time_point timeout) {
|
||||
|
||||
return parallel_for_each(boost::irange(0u, smp::count), [this, timeout] (shard_id shard) {
|
||||
return _db.invoke_on(shard, [this, shard, cmd = &_cmd, ranges = &_ranges, gs = global_schema_ptr(_schema),
|
||||
gts = tracing::global_trace_state_ptr(_trace_state), timeout] (database& db) mutable {
|
||||
gts = tracing::global_trace_state_ptr(_trace_state), timeout] (replica::database& db) mutable {
|
||||
auto schema = gs.get();
|
||||
auto querier_opt = db.get_querier_cache().lookup_shard_mutation_querier(cmd->query_uuid, *schema, *ranges, cmd->slice, gts.get(), timeout);
|
||||
auto& table = db.find_column_family(schema);
|
||||
@@ -747,7 +747,7 @@ future<page_consume_result<ResultBuilder>> read_page(
|
||||
|
||||
template <typename ResultBuilder>
|
||||
future<typename ResultBuilder::result_type> do_query(
|
||||
distributed<database>& db,
|
||||
distributed<replica::database>& db,
|
||||
schema_ptr s,
|
||||
const query::read_command& cmd,
|
||||
const dht::partition_range_vector& ranges,
|
||||
@@ -780,7 +780,7 @@ future<typename ResultBuilder::result_type> do_query(
|
||||
|
||||
template <typename ResultBuilder>
|
||||
static future<std::tuple<foreign_ptr<lw_shared_ptr<typename ResultBuilder::result_type>>, cache_temperature>> do_query_on_all_shards(
|
||||
distributed<database>& db,
|
||||
distributed<replica::database>& db,
|
||||
schema_ptr s,
|
||||
const query::read_command& cmd,
|
||||
const dht::partition_range_vector& ranges,
|
||||
@@ -866,7 +866,7 @@ public:
|
||||
} // anonymous namespace
|
||||
|
||||
future<std::tuple<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>> query_mutations_on_all_shards(
|
||||
distributed<database>& db,
|
||||
distributed<replica::database>& db,
|
||||
schema_ptr table_schema,
|
||||
const query::read_command& cmd,
|
||||
const dht::partition_range_vector& ranges,
|
||||
@@ -881,7 +881,7 @@ future<std::tuple<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_tempera
|
||||
}
|
||||
|
||||
future<std::tuple<foreign_ptr<lw_shared_ptr<query::result>>, cache_temperature>> query_data_on_all_shards(
|
||||
distributed<database>& db,
|
||||
distributed<replica::database>& db,
|
||||
schema_ptr table_schema,
|
||||
const query::read_command& cmd,
|
||||
const dht::partition_range_vector& ranges,
|
||||
|
||||
@@ -79,7 +79,7 @@ namespace tracing {
|
||||
/// \see multishard_combined_reader
|
||||
/// \see querier_cache
|
||||
future<std::tuple<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>> query_mutations_on_all_shards(
|
||||
distributed<database>& db,
|
||||
distributed<replica::database>& db,
|
||||
schema_ptr s,
|
||||
const query::read_command& cmd,
|
||||
const dht::partition_range_vector& ranges,
|
||||
@@ -91,7 +91,7 @@ future<std::tuple<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_tempera
|
||||
/// Identical to `query_mutations_on_all_shards()` except that it builds results
|
||||
/// in the `query::result` format instead of in the `reconcilable_result` one.
|
||||
future<std::tuple<foreign_ptr<lw_shared_ptr<query::result>>, cache_temperature>> query_data_on_all_shards(
|
||||
distributed<database>& db,
|
||||
distributed<replica::database>& db,
|
||||
schema_ptr s,
|
||||
const query::read_command& cmd,
|
||||
const dht::partition_range_vector& ranges,
|
||||
|
||||
@@ -52,7 +52,9 @@ namespace gms {
|
||||
class gossiper;
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace redis {
|
||||
|
||||
|
||||
@@ -31,7 +31,7 @@ namespace redis {
|
||||
|
||||
distributed<query_processor> _the_query_processor;
|
||||
|
||||
query_processor::query_processor(service::storage_proxy& proxy, distributed<database>& db)
|
||||
query_processor::query_processor(service::storage_proxy& proxy, distributed<replica::database>& db)
|
||||
: _proxy(proxy)
|
||||
, _db(db)
|
||||
{
|
||||
|
||||
@@ -26,7 +26,10 @@
|
||||
#include <seastar/core/gate.hh>
|
||||
#include <seastar/core/metrics_registration.hh>
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
class service_permit;
|
||||
|
||||
namespace service {
|
||||
@@ -42,15 +45,15 @@ class redis_message;
|
||||
|
||||
class query_processor {
|
||||
service::storage_proxy& _proxy;
|
||||
seastar::sharded<database>& _db;
|
||||
seastar::sharded<replica::database>& _db;
|
||||
seastar::metrics::metric_groups _metrics;
|
||||
seastar::gate _pending_command_gate;
|
||||
public:
|
||||
query_processor(service::storage_proxy& proxy, seastar::sharded<database>& db);
|
||||
query_processor(service::storage_proxy& proxy, seastar::sharded<replica::database>& db);
|
||||
|
||||
~query_processor();
|
||||
|
||||
seastar::sharded<database>& db() {
|
||||
seastar::sharded<replica::database>& db() {
|
||||
return _db;
|
||||
}
|
||||
|
||||
|
||||
@@ -153,7 +153,7 @@ std::ostream& operator<<(std::ostream& out, row_level_diff_detect_algorithm algo
|
||||
return out << "unknown";
|
||||
}
|
||||
|
||||
static std::vector<sstring> list_column_families(const database& db, const sstring& keyspace) {
|
||||
static std::vector<sstring> list_column_families(const replica::database& db, const sstring& keyspace) {
|
||||
std::vector<sstring> ret;
|
||||
for (auto &&e : db.get_column_families_mapping()) {
|
||||
if (e.first.first == keyspace) {
|
||||
@@ -168,7 +168,7 @@ std::ostream& operator<<(std::ostream& os, const repair_uniq_id& x) {
|
||||
}
|
||||
|
||||
// Must run inside a seastar thread
|
||||
static std::vector<utils::UUID> get_table_ids(const database& db, const sstring& keyspace, const std::vector<sstring>& tables) {
|
||||
static std::vector<utils::UUID> get_table_ids(const replica::database& db, const sstring& keyspace, const std::vector<sstring>& tables) {
|
||||
std::vector<utils::UUID> table_ids;
|
||||
table_ids.reserve(tables.size());
|
||||
for (auto& table : tables) {
|
||||
@@ -178,7 +178,7 @@ static std::vector<utils::UUID> get_table_ids(const database& db, const sstring&
|
||||
return table_ids;
|
||||
}
|
||||
|
||||
static std::vector<sstring> get_table_names(const database& db, const std::vector<utils::UUID>& table_ids) {
|
||||
static std::vector<sstring> get_table_names(const replica::database& db, const std::vector<utils::UUID>& table_ids) {
|
||||
std::vector<sstring> table_names;
|
||||
table_names.reserve(table_ids.size());
|
||||
for (auto& table_id : table_ids) {
|
||||
@@ -196,13 +196,13 @@ void remove_item(Collection& c, T& item) {
|
||||
}
|
||||
|
||||
// Return all of the neighbors with whom we share the provided range.
|
||||
static std::vector<gms::inet_address> get_neighbors(database& db,
|
||||
static std::vector<gms::inet_address> get_neighbors(replica::database& db,
|
||||
const sstring& ksname, query::range<dht::token> range,
|
||||
const std::vector<sstring>& data_centers,
|
||||
const std::vector<sstring>& hosts,
|
||||
const std::unordered_set<gms::inet_address>& ignore_nodes) {
|
||||
|
||||
keyspace& ks = db.find_keyspace(ksname);
|
||||
replica::keyspace& ks = db.find_keyspace(ksname);
|
||||
auto erm = ks.get_effective_replication_map();
|
||||
|
||||
dht::token tok = range.end() ? range.end()->value() : dht::maximum_token();
|
||||
@@ -315,7 +315,7 @@ static std::vector<gms::inet_address> get_neighbors(database& db,
|
||||
#endif
|
||||
}
|
||||
|
||||
static future<std::list<gms::inet_address>> get_hosts_participating_in_repair(database& db,
|
||||
static future<std::list<gms::inet_address>> get_hosts_participating_in_repair(replica::database& db,
|
||||
const sstring& ksname,
|
||||
const dht::token_range_vector& ranges,
|
||||
const std::vector<sstring>& data_centers,
|
||||
@@ -539,7 +539,7 @@ void check_in_shutdown() {
|
||||
repair_tracker().check_in_shutdown();
|
||||
}
|
||||
|
||||
future<uint64_t> estimate_partitions(seastar::sharded<database>& db, const sstring& keyspace,
|
||||
future<uint64_t> estimate_partitions(seastar::sharded<replica::database>& db, const sstring& keyspace,
|
||||
const sstring& cf, const dht::token_range& range) {
|
||||
return db.map_reduce0(
|
||||
[keyspace, cf, range] (auto& db) {
|
||||
@@ -559,14 +559,14 @@ future<uint64_t> estimate_partitions(seastar::sharded<database>& db, const sstri
|
||||
|
||||
static
|
||||
const dht::sharder&
|
||||
get_sharder_for_tables(seastar::sharded<database>& db, const sstring& keyspace, const std::vector<utils::UUID>& table_ids) {
|
||||
get_sharder_for_tables(seastar::sharded<replica::database>& db, const sstring& keyspace, const std::vector<utils::UUID>& table_ids) {
|
||||
schema_ptr last_s;
|
||||
for (size_t idx = 0 ; idx < table_ids.size(); idx++) {
|
||||
schema_ptr s;
|
||||
try {
|
||||
s = db.local().find_column_family(table_ids[idx]).schema();
|
||||
} catch(...) {
|
||||
throw no_such_column_family(keyspace, table_ids[idx]);
|
||||
throw replica::no_such_column_family(keyspace, table_ids[idx]);
|
||||
}
|
||||
if (last_s && last_s->get_sharder() != s->get_sharder()) {
|
||||
throw std::runtime_error(
|
||||
@@ -695,14 +695,14 @@ future<> repair_info::repair_range(const dht::token_range& range) {
|
||||
sstring cf;
|
||||
try {
|
||||
cf = db.local().find_column_family(table_id).schema()->cf_name();
|
||||
} catch (no_such_column_family&) {
|
||||
} catch (replica::no_such_column_family&) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
// Row level repair
|
||||
if (dropped_tables.contains(cf)) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
return repair_cf_range_row_level(*this, cf, table_id, range, neighbors).handle_exception_type([this, cf] (no_such_column_family&) mutable {
|
||||
return repair_cf_range_row_level(*this, cf, table_id, range, neighbors).handle_exception_type([this, cf] (replica::no_such_column_family&) mutable {
|
||||
dropped_tables.insert(cf);
|
||||
return make_ready_future<>();
|
||||
}).handle_exception([this] (std::exception_ptr ep) mutable {
|
||||
@@ -715,12 +715,12 @@ future<> repair_info::repair_range(const dht::token_range& range) {
|
||||
}
|
||||
|
||||
static dht::token_range_vector get_primary_ranges_for_endpoint(
|
||||
database& db, sstring keyspace, gms::inet_address ep) {
|
||||
replica::database& db, sstring keyspace, gms::inet_address ep) {
|
||||
return db.find_keyspace(keyspace).get_effective_replication_map()->get_primary_ranges(ep);
|
||||
}
|
||||
|
||||
static dht::token_range_vector get_primary_ranges(
|
||||
database& db, sstring keyspace) {
|
||||
replica::database& db, sstring keyspace) {
|
||||
return get_primary_ranges_for_endpoint(db, keyspace,
|
||||
utils::fb_utilities::get_broadcast_address());
|
||||
}
|
||||
@@ -730,7 +730,7 @@ static dht::token_range_vector get_primary_ranges(
|
||||
// across the entire cluster, here each range is assigned a primary
|
||||
// owner in each of the clusters.
|
||||
static dht::token_range_vector get_primary_ranges_within_dc(
|
||||
database& db, sstring keyspace) {
|
||||
replica::database& db, sstring keyspace) {
|
||||
return db.find_keyspace(keyspace).get_effective_replication_map()->get_primary_ranges_within_dc(utils::fb_utilities::get_broadcast_address());
|
||||
}
|
||||
|
||||
@@ -1027,7 +1027,7 @@ static future<> repair_ranges(lw_shared_ptr<repair_info> ri) {
|
||||
// repairs). It is fine to always do this on one CPU, because the function
|
||||
// itself does very little (mainly tell other nodes and CPUs what to do).
|
||||
int repair_service::do_repair_start(sstring keyspace, std::unordered_map<sstring, sstring> options_map) {
|
||||
seastar::sharded<database>& db = get_db();
|
||||
seastar::sharded<replica::database>& db = get_db();
|
||||
check_in_shutdown();
|
||||
|
||||
repair_options options(options_map);
|
||||
@@ -1237,20 +1237,20 @@ future<int> repair_start(seastar::sharded<repair_service>& repair,
|
||||
});
|
||||
}
|
||||
|
||||
future<std::vector<int>> get_active_repairs(seastar::sharded<database>& db) {
|
||||
return db.invoke_on(0, [] (database& localdb) {
|
||||
future<std::vector<int>> get_active_repairs(seastar::sharded<replica::database>& db) {
|
||||
return db.invoke_on(0, [] (replica::database& localdb) {
|
||||
return repair_tracker().get_active();
|
||||
});
|
||||
}
|
||||
|
||||
future<repair_status> repair_get_status(seastar::sharded<database>& db, int id) {
|
||||
return db.invoke_on(0, [id] (database& localdb) {
|
||||
future<repair_status> repair_get_status(seastar::sharded<replica::database>& db, int id) {
|
||||
return db.invoke_on(0, [id] (replica::database& localdb) {
|
||||
return repair_tracker().get(id);
|
||||
});
|
||||
}
|
||||
|
||||
future<repair_status> repair_await_completion(seastar::sharded<database>& db, int id, std::chrono::steady_clock::time_point timeout) {
|
||||
return db.invoke_on(0, [id, timeout] (database& localdb) {
|
||||
future<repair_status> repair_await_completion(seastar::sharded<replica::database>& db, int id, std::chrono::steady_clock::time_point timeout) {
|
||||
return db.invoke_on(0, [id, timeout] (replica::database& localdb) {
|
||||
return repair_tracker().repair_await_completion(id, timeout);
|
||||
});
|
||||
}
|
||||
@@ -1262,8 +1262,8 @@ future<> repair_service::shutdown() {
|
||||
}
|
||||
}
|
||||
|
||||
future<> repair_abort_all(seastar::sharded<database>& db) {
|
||||
return db.invoke_on_all([] (database& localdb) {
|
||||
future<> repair_abort_all(seastar::sharded<replica::database>& db) {
|
||||
return db.invoke_on_all([] (replica::database& localdb) {
|
||||
repair_tracker().abort_all_repairs();
|
||||
});
|
||||
}
|
||||
@@ -1288,7 +1288,7 @@ future<> repair_service::do_sync_data_using_repair(
|
||||
std::unordered_map<dht::token_range, repair_neighbors> neighbors,
|
||||
streaming::stream_reason reason,
|
||||
std::optional<utils::UUID> ops_uuid) {
|
||||
seastar::sharded<database>& db = get_db();
|
||||
seastar::sharded<replica::database>& db = get_db();
|
||||
|
||||
repair_uniq_id id = repair_tracker().next_repair_command();
|
||||
rlogger.info("repair id {} to sync data for keyspace={}, status=started", id, keyspace);
|
||||
@@ -1344,7 +1344,7 @@ future<> repair_service::do_sync_data_using_repair(
|
||||
future<> repair_service::bootstrap_with_repair(locator::token_metadata_ptr tmptr, std::unordered_set<dht::token> bootstrap_tokens) {
|
||||
using inet_address = gms::inet_address;
|
||||
return seastar::async([this, tmptr = std::move(tmptr), tokens = std::move(bootstrap_tokens)] () mutable {
|
||||
seastar::sharded<database>& db = get_db();
|
||||
seastar::sharded<replica::database>& db = get_db();
|
||||
auto keyspaces = db.local().get_non_system_keyspaces();
|
||||
auto myip = utils::fb_utilities::get_broadcast_address();
|
||||
auto reason = streaming::stream_reason::bootstrap;
|
||||
@@ -1360,7 +1360,7 @@ future<> repair_service::bootstrap_with_repair(locator::token_metadata_ptr tmptr
|
||||
seastar::thread::maybe_yield();
|
||||
nr_ranges_total += desired_ranges.size();
|
||||
}
|
||||
db.invoke_on_all([nr_ranges_total] (database&) {
|
||||
db.invoke_on_all([nr_ranges_total] (replica::database&) {
|
||||
_node_ops_metrics.bootstrap_finished_ranges = 0;
|
||||
_node_ops_metrics.bootstrap_total_ranges = nr_ranges_total;
|
||||
}).get();
|
||||
@@ -1518,7 +1518,7 @@ future<> repair_service::bootstrap_with_repair(locator::token_metadata_ptr tmptr
|
||||
future<> repair_service::do_decommission_removenode_with_repair(locator::token_metadata_ptr tmptr, gms::inet_address leaving_node, shared_ptr<node_ops_info> ops) {
|
||||
using inet_address = gms::inet_address;
|
||||
return seastar::async([this, tmptr = std::move(tmptr), leaving_node = std::move(leaving_node), ops] () mutable {
|
||||
seastar::sharded<database>& db = get_db();
|
||||
seastar::sharded<replica::database>& db = get_db();
|
||||
auto myip = utils::fb_utilities::get_broadcast_address();
|
||||
auto keyspaces = db.local().get_non_system_keyspaces();
|
||||
bool is_removenode = myip != leaving_node;
|
||||
@@ -1534,12 +1534,12 @@ future<> repair_service::do_decommission_removenode_with_repair(locator::token_m
|
||||
nr_ranges_total += ranges.size();
|
||||
}
|
||||
if (reason == streaming::stream_reason::decommission) {
|
||||
db.invoke_on_all([nr_ranges_total] (database&) {
|
||||
db.invoke_on_all([nr_ranges_total] (replica::database&) {
|
||||
_node_ops_metrics.decommission_finished_ranges = 0;
|
||||
_node_ops_metrics.decommission_total_ranges = nr_ranges_total;
|
||||
}).get();
|
||||
} else if (reason == streaming::stream_reason::removenode) {
|
||||
db.invoke_on_all([nr_ranges_total] (database&) {
|
||||
db.invoke_on_all([nr_ranges_total] (replica::database&) {
|
||||
_node_ops_metrics.removenode_finished_ranges = 0;
|
||||
_node_ops_metrics.removenode_total_ranges = nr_ranges_total;
|
||||
}).get();
|
||||
@@ -1690,11 +1690,11 @@ future<> repair_service::do_decommission_removenode_with_repair(locator::token_m
|
||||
}
|
||||
temp.clear_gently().get();
|
||||
if (reason == streaming::stream_reason::decommission) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (database&) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (replica::database&) {
|
||||
_node_ops_metrics.decommission_finished_ranges += nr_ranges_skipped;
|
||||
}).get();
|
||||
} else if (reason == streaming::stream_reason::removenode) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (database&) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (replica::database&) {
|
||||
_node_ops_metrics.removenode_finished_ranges += nr_ranges_skipped;
|
||||
}).get();
|
||||
}
|
||||
@@ -1718,8 +1718,8 @@ future<> repair_service::decommission_with_repair(locator::token_metadata_ptr tm
|
||||
future<> repair_service::removenode_with_repair(locator::token_metadata_ptr tmptr, gms::inet_address leaving_node, shared_ptr<node_ops_info> ops) {
|
||||
return do_decommission_removenode_with_repair(std::move(tmptr), std::move(leaving_node), std::move(ops)).then([this] {
|
||||
rlogger.debug("Triggering off-strategy compaction for all non-system tables on removenode completion");
|
||||
seastar::sharded<database>& db = get_db();
|
||||
return db.invoke_on_all([](database &db) {
|
||||
seastar::sharded<replica::database>& db = get_db();
|
||||
return db.invoke_on_all([](replica::database &db) {
|
||||
for (auto& table : db.get_non_system_column_families()) {
|
||||
table->trigger_offstrategy_compaction();
|
||||
}
|
||||
@@ -1735,7 +1735,7 @@ future<> abort_repair_node_ops(utils::UUID ops_uuid) {
|
||||
|
||||
future<> repair_service::do_rebuild_replace_with_repair(locator::token_metadata_ptr tmptr, sstring op, sstring source_dc, streaming::stream_reason reason, std::list<gms::inet_address> ignore_nodes) {
|
||||
return seastar::async([this, tmptr = std::move(tmptr), source_dc = std::move(source_dc), op = std::move(op), reason, ignore_nodes = std::move(ignore_nodes)] () mutable {
|
||||
seastar::sharded<database>& db = get_db();
|
||||
seastar::sharded<replica::database>& db = get_db();
|
||||
auto keyspaces = db.local().get_non_system_keyspaces();
|
||||
auto myip = utils::fb_utilities::get_broadcast_address();
|
||||
size_t nr_ranges_total = 0;
|
||||
@@ -1751,12 +1751,12 @@ future<> repair_service::do_rebuild_replace_with_repair(locator::token_metadata_
|
||||
|
||||
}
|
||||
if (reason == streaming::stream_reason::rebuild) {
|
||||
db.invoke_on_all([nr_ranges_total] (database&) {
|
||||
db.invoke_on_all([nr_ranges_total] (replica::database&) {
|
||||
_node_ops_metrics.rebuild_finished_ranges = 0;
|
||||
_node_ops_metrics.rebuild_total_ranges = nr_ranges_total;
|
||||
}).get();
|
||||
} else if (reason == streaming::stream_reason::replace) {
|
||||
db.invoke_on_all([nr_ranges_total] (database&) {
|
||||
db.invoke_on_all([nr_ranges_total] (replica::database&) {
|
||||
_node_ops_metrics.replace_finished_ranges = 0;
|
||||
_node_ops_metrics.replace_total_ranges = nr_ranges_total;
|
||||
}).get();
|
||||
@@ -1800,11 +1800,11 @@ future<> repair_service::do_rebuild_replace_with_repair(locator::token_metadata_
|
||||
}
|
||||
}
|
||||
if (reason == streaming::stream_reason::rebuild) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (database&) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (replica::database&) {
|
||||
_node_ops_metrics.rebuild_finished_ranges += nr_ranges_skipped;
|
||||
}).get();
|
||||
} else if (reason == streaming::stream_reason::replace) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (database&) {
|
||||
db.invoke_on_all([nr_ranges_skipped] (replica::database&) {
|
||||
_node_ops_metrics.replace_finished_ranges += nr_ranges_skipped;
|
||||
}).get();
|
||||
}
|
||||
@@ -1823,7 +1823,7 @@ future<> repair_service::rebuild_with_repair(locator::token_metadata_ptr tmptr,
|
||||
}
|
||||
auto reason = streaming::stream_reason::rebuild;
|
||||
co_await do_rebuild_replace_with_repair(std::move(tmptr), std::move(op), std::move(source_dc), reason, {});
|
||||
co_await get_db().invoke_on_all([](database& db) {
|
||||
co_await get_db().invoke_on_all([](replica::database& db) {
|
||||
for (auto& t : db.get_non_system_column_families()) {
|
||||
t->trigger_offstrategy_compaction();
|
||||
}
|
||||
|
||||
@@ -42,7 +42,10 @@
|
||||
|
||||
class flat_mutation_reader;
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
class repair_service;
|
||||
namespace db {
|
||||
namespace view {
|
||||
@@ -102,19 +105,19 @@ enum class repair_status { RUNNING, SUCCESSFUL, FAILED };
|
||||
|
||||
// repair_get_status() returns a future because it needs to run code on a
|
||||
// different CPU (cpu 0) and that might be a deferring operation.
|
||||
future<repair_status> repair_get_status(seastar::sharded<database>& db, int id);
|
||||
future<repair_status> repair_get_status(seastar::sharded<replica::database>& db, int id);
|
||||
|
||||
// If the repair job is finished (SUCCESSFUL or FAILED), it returns immediately.
|
||||
// It blocks if the repair job is still RUNNING until timeout.
|
||||
future<repair_status> repair_await_completion(seastar::sharded<database>& db, int id, std::chrono::steady_clock::time_point timeout);
|
||||
future<repair_status> repair_await_completion(seastar::sharded<replica::database>& db, int id, std::chrono::steady_clock::time_point timeout);
|
||||
|
||||
// returns a vector with the ids of the active repairs
|
||||
future<std::vector<int>> get_active_repairs(seastar::sharded<database>& db);
|
||||
future<std::vector<int>> get_active_repairs(seastar::sharded<replica::database>& db);
|
||||
|
||||
void check_in_shutdown();
|
||||
|
||||
// Abort all the repairs
|
||||
future<> repair_abort_all(seastar::sharded<database>& db);
|
||||
future<> repair_abort_all(seastar::sharded<replica::database>& db);
|
||||
|
||||
enum class repair_checksum {
|
||||
legacy = 0,
|
||||
@@ -169,7 +172,7 @@ public:
|
||||
class repair_info {
|
||||
public:
|
||||
repair_service& rs;
|
||||
seastar::sharded<database>& db;
|
||||
seastar::sharded<replica::database>& db;
|
||||
seastar::sharded<netw::messaging_service>& messaging;
|
||||
sharded<db::system_distributed_keyspace>& sys_dist_ks;
|
||||
sharded<db::view::view_update_generator>& view_update_generator;
|
||||
@@ -290,7 +293,7 @@ public:
|
||||
void abort_repair_node_ops(utils::UUID ops_uuid);
|
||||
};
|
||||
|
||||
future<uint64_t> estimate_partitions(seastar::sharded<database>& db, const sstring& keyspace,
|
||||
future<uint64_t> estimate_partitions(seastar::sharded<replica::database>& db, const sstring& keyspace,
|
||||
const sstring& cf, const dht::token_range& range);
|
||||
|
||||
// Represent a position of a mutation_fragment read from a flat mutation
|
||||
|
||||
@@ -376,8 +376,8 @@ private:
|
||||
|
||||
public:
|
||||
repair_reader(
|
||||
seastar::sharded<database>& db,
|
||||
column_family& cf,
|
||||
seastar::sharded<replica::database>& db,
|
||||
replica::column_family& cf,
|
||||
schema_ptr s,
|
||||
reader_permit permit,
|
||||
dht::token_range range,
|
||||
@@ -539,11 +539,11 @@ public:
|
||||
return sstables::offstrategy(operations_supported.contains(reason));
|
||||
}
|
||||
|
||||
void create_writer(sharded<database>& db, sharded<db::system_distributed_keyspace>& sys_dist_ks, sharded<db::view::view_update_generator>& view_update_gen) {
|
||||
void create_writer(sharded<replica::database>& db, sharded<db::system_distributed_keyspace>& sys_dist_ks, sharded<db::view::view_update_generator>& view_update_gen) {
|
||||
if (_writer_done) {
|
||||
return;
|
||||
}
|
||||
table& t = db.local().find_column_family(_schema->id());
|
||||
replica::table& t = db.local().find_column_family(_schema->id());
|
||||
auto [queue_reader, queue_handle] = make_queue_reader(_schema, _permit);
|
||||
_mq = std::move(queue_handle);
|
||||
auto writer = shared_from_this();
|
||||
@@ -644,11 +644,11 @@ public:
|
||||
using msg_addr = netw::messaging_service::msg_addr;
|
||||
using tracker_link_type = boost::intrusive::list_member_hook<bi::link_mode<boost::intrusive::auto_unlink>>;
|
||||
private:
|
||||
seastar::sharded<database>& _db;
|
||||
seastar::sharded<replica::database>& _db;
|
||||
seastar::sharded<netw::messaging_service>& _messaging;
|
||||
seastar::sharded<db::system_distributed_keyspace>& _sys_dist_ks;
|
||||
seastar::sharded<db::view::view_update_generator>& _view_update_generator;
|
||||
column_family& _cf;
|
||||
replica::column_family& _cf;
|
||||
schema_ptr _schema;
|
||||
reader_permit _permit;
|
||||
dht::token_range _range;
|
||||
@@ -735,11 +735,11 @@ public:
|
||||
|
||||
public:
|
||||
repair_meta(
|
||||
seastar::sharded<database>& db,
|
||||
seastar::sharded<replica::database>& db,
|
||||
seastar::sharded<netw::messaging_service>& ms,
|
||||
seastar::sharded<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
seastar::sharded<db::view::view_update_generator>& view_update_generator,
|
||||
column_family& cf,
|
||||
replica::column_family& cf,
|
||||
schema_ptr s,
|
||||
reader_permit permit,
|
||||
dht::token_range range,
|
||||
@@ -1665,7 +1665,7 @@ public:
|
||||
_master_node_shard_config.shard, _master_node_shard_config.shard_count, _master_node_shard_config.ignore_msb,
|
||||
remote_partitioner_name, std::move(schema_version), reason).then([ks_name, cf_name] (rpc::optional<repair_row_level_start_response> resp) {
|
||||
if (resp && resp->status == repair_row_level_start_status::no_such_column_family) {
|
||||
return make_exception_future<>(no_such_column_family(ks_name, cf_name));
|
||||
return make_exception_future<>(replica::no_such_column_family(ks_name, cf_name));
|
||||
} else {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
@@ -1681,7 +1681,7 @@ public:
|
||||
utils::fb_utilities::get_broadcast_address(), from, repair_meta_id, ks_name, cf_name, schema_version, range, seed, max_row_buf_size);
|
||||
return insert_repair_meta(repair, from, src_cpu_id, repair_meta_id, std::move(range), algo, max_row_buf_size, seed, std::move(master_node_shard_config), std::move(schema_version), reason).then([] {
|
||||
return repair_row_level_start_response{repair_row_level_start_status::ok};
|
||||
}).handle_exception_type([] (no_such_column_family&) {
|
||||
}).handle_exception_type([] (replica::no_such_column_family&) {
|
||||
return repair_row_level_start_response{repair_row_level_start_status::no_such_column_family};
|
||||
});
|
||||
}
|
||||
@@ -2290,7 +2290,7 @@ future<repair_update_system_table_response> repair_service::repair_update_system
|
||||
if (!is_valid_range) {
|
||||
throw std::runtime_error(format("repair[{}]: range {} is not in the format of (start, end]", req.repair_uuid, req.range));
|
||||
}
|
||||
co_await db.invoke_on_all([&req] (database& local_db) {
|
||||
co_await db.invoke_on_all([&req] (replica::database& local_db) {
|
||||
auto& table = local_db.find_column_family(req.table_uuid);
|
||||
return ::update_repair_time(table.schema(), req.range, req.repair_time);
|
||||
});
|
||||
@@ -2559,7 +2559,7 @@ class row_level_repair {
|
||||
utils::UUID _table_id;
|
||||
dht::token_range _range;
|
||||
inet_address_vector_replica_set _all_live_peer_nodes;
|
||||
column_family& _cf;
|
||||
replica::column_family& _cf;
|
||||
|
||||
// Repair master and followers will propose a sync boundary. Each of them
|
||||
// read N bytes of rows from disk, the row with largest
|
||||
@@ -2996,7 +2996,7 @@ public:
|
||||
}
|
||||
send_missing_rows_to_follower_nodes(master);
|
||||
}
|
||||
} catch (no_such_column_family& e) {
|
||||
} catch (replica::no_such_column_family& e) {
|
||||
table_dropped = true;
|
||||
rlogger.warn("repair id {} on shard {}, keyspace={}, cf={}, range={}, got error in row level repair: {}",
|
||||
_ri.id, this_shard_id(), _ri.keyspace, _cf_name, _range, e);
|
||||
@@ -3018,7 +3018,7 @@ public:
|
||||
_ri.update_statistics(master.stats());
|
||||
if (_failed) {
|
||||
if (table_dropped) {
|
||||
throw no_such_column_family(_ri.keyspace, _cf_name);
|
||||
throw replica::no_such_column_family(_ri.keyspace, _cf_name);
|
||||
} else {
|
||||
throw std::runtime_error(format("Failed to repair for keyspace={}, cf={}, range={}", _ri.keyspace, _cf_name, _range));
|
||||
}
|
||||
@@ -3096,7 +3096,7 @@ class row_level_repair_gossip_helper : public gms::i_endpoint_state_change_subsc
|
||||
|
||||
repair_service::repair_service(distributed<gms::gossiper>& gossiper,
|
||||
netw::messaging_service& ms,
|
||||
sharded<database>& db,
|
||||
sharded<replica::database>& db,
|
||||
sharded<service::storage_proxy>& sp,
|
||||
sharded<db::batchlog_manager>& bm,
|
||||
sharded<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
@@ -3197,11 +3197,11 @@ future<> repair_service::load_history() {
|
||||
auto range = dht::token_range(dht::token_range::bound(start, false), dht::token_range::bound(end, true));
|
||||
rlogger.debug("Loading repair history for keyspace={}, table={}, table_uuid={}, repair_time={}, range={}",
|
||||
keyspace_name, table_name, table_uuid, repair_time, range);
|
||||
co_await get_db().invoke_on_all([table_uuid, range, repair_time, keyspace_name, table_name] (database& local_db) -> future<> {
|
||||
co_await get_db().invoke_on_all([table_uuid, range, repair_time, keyspace_name, table_name] (replica::database& local_db) -> future<> {
|
||||
try {
|
||||
auto& table = local_db.find_column_family(table_uuid);
|
||||
::update_repair_time(table.schema(), range, repair_time);
|
||||
} catch (no_such_column_family&) {
|
||||
} catch (replica::no_such_column_family&) {
|
||||
rlogger.trace("Table {}.{} with {} does not exist", keyspace_name, table_name, table_uuid);
|
||||
} catch (...) {
|
||||
rlogger.warn("Failed to load repair history for keyspace={}, table={}, range={}, repair_time={}",
|
||||
|
||||
@@ -54,7 +54,7 @@ public:
|
||||
class repair_service : public seastar::peering_sharded_service<repair_service> {
|
||||
distributed<gms::gossiper>& _gossiper;
|
||||
netw::messaging_service& _messaging;
|
||||
sharded<database>& _db;
|
||||
sharded<replica::database>& _db;
|
||||
sharded<service::storage_proxy>& _sp;
|
||||
sharded<db::batchlog_manager>& _bm;
|
||||
sharded<db::system_distributed_keyspace>& _sys_dist_ks;
|
||||
@@ -78,7 +78,7 @@ class repair_service : public seastar::peering_sharded_service<repair_service> {
|
||||
public:
|
||||
repair_service(distributed<gms::gossiper>& gossiper,
|
||||
netw::messaging_service& ms,
|
||||
sharded<database>& db,
|
||||
sharded<replica::database>& db,
|
||||
sharded<service::storage_proxy>& sp,
|
||||
sharded<db::batchlog_manager>& bm,
|
||||
sharded<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
@@ -133,7 +133,7 @@ private:
|
||||
|
||||
public:
|
||||
netw::messaging_service& get_messaging() noexcept { return _messaging; }
|
||||
sharded<database>& get_db() noexcept { return _db; }
|
||||
sharded<replica::database>& get_db() noexcept { return _db; }
|
||||
service::migration_manager& get_migration_manager() noexcept { return _mm; }
|
||||
sharded<db::system_distributed_keyspace>& get_sys_dist_ks() noexcept { return _sys_dist_ks; }
|
||||
sharded<db::view::view_update_generator>& get_view_update_generator() noexcept { return _view_update_generator; }
|
||||
|
||||
@@ -84,6 +84,8 @@ logging::logger dblog("database");
|
||||
// dirty_memory manager in that case.
|
||||
thread_local dirty_memory_manager default_dirty_memory_manager;
|
||||
|
||||
namespace replica {
|
||||
|
||||
inline
|
||||
flush_controller
|
||||
make_flush_controller(const db::config& cfg, seastar::scheduling_group sg, const ::io_priority_class& iop, std::function<double()> fn) {
|
||||
@@ -383,6 +385,8 @@ const db::extensions& database::extensions() const {
|
||||
return get_config().extensions();
|
||||
}
|
||||
|
||||
} // namespace replica
|
||||
|
||||
void backlog_controller::adjust() {
|
||||
auto backlog = _current_backlog();
|
||||
|
||||
@@ -447,6 +451,8 @@ dirty_memory_manager::setup_collectd(sstring namestr) {
|
||||
});
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
|
||||
static const metrics::label class_label("class");
|
||||
|
||||
void
|
||||
@@ -1285,6 +1291,8 @@ database::existing_index_names(const sstring& ks_name, const sstring& cf_to_excl
|
||||
return names;
|
||||
}
|
||||
|
||||
} // namespace replica
|
||||
|
||||
// Based on:
|
||||
// - org.apache.cassandra.db.AbstractCell#reconcile()
|
||||
// - org.apache.cassandra.db.BufferExpiringCell#reconcile()
|
||||
@@ -1323,6 +1331,8 @@ compare_atomic_cell_for_merge(atomic_cell_view left, atomic_cell_view right) {
|
||||
return std::strong_ordering::equal;
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
|
||||
future<std::tuple<lw_shared_ptr<query::result>, cache_temperature>>
|
||||
database::query(schema_ptr s, const query::read_command& cmd, query::result_options opts, const dht::partition_range_vector& ranges,
|
||||
tracing::trace_state_ptr trace_state, db::timeout_clock::time_point timeout) {
|
||||
@@ -1617,6 +1627,8 @@ future<mutation> database::do_apply_counter_update(column_family& cf, const froz
|
||||
});
|
||||
}
|
||||
|
||||
} // namespace replica
|
||||
|
||||
future<> dirty_memory_manager::shutdown() {
|
||||
_db_shutdown_requested = true;
|
||||
_should_flush.signal();
|
||||
@@ -1718,6 +1730,8 @@ void dirty_memory_manager::start_reclaiming() noexcept {
|
||||
_should_flush.signal();
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
|
||||
future<> database::apply_in_memory(const frozen_mutation& m, schema_ptr m_schema, db::rp_handle&& h, db::timeout_clock::time_point timeout) {
|
||||
auto& cf = find_column_family(m.column_family_id());
|
||||
|
||||
@@ -1902,6 +1916,8 @@ database::make_keyspace_config(const keyspace_metadata& ksm) {
|
||||
return cfg;
|
||||
}
|
||||
|
||||
} // namespace replica
|
||||
|
||||
namespace db {
|
||||
|
||||
std::ostream& operator<<(std::ostream& os, const write_type& t) {
|
||||
@@ -1944,6 +1960,8 @@ operator<<(std::ostream& os, const exploded_clustering_prefix& ecp) {
|
||||
return os;
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
|
||||
sstring database::get_available_index_name(const sstring &ks_name, const sstring &cf_name,
|
||||
std::optional<sstring> index_name_root) const
|
||||
{
|
||||
@@ -2258,28 +2276,32 @@ future<> database::drain() {
|
||||
co_await _commitlog->shutdown();
|
||||
}
|
||||
|
||||
} // namespace replica
|
||||
|
||||
namespace cdc {
|
||||
schema_ptr get_base_table(const database& db, const schema& s);
|
||||
schema_ptr get_base_table(const replica::database& db, const schema& s);
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
|
||||
class database::data_dictionary_impl : public data_dictionary::impl {
|
||||
const data_dictionary::database wrap(const ::database& db) const {
|
||||
const data_dictionary::database wrap(const replica::database& db) const {
|
||||
return make_database(this, &db);
|
||||
}
|
||||
data_dictionary::keyspace wrap(const ::keyspace& ks) const {
|
||||
data_dictionary::keyspace wrap(const replica::keyspace& ks) const {
|
||||
return make_keyspace(this, &ks);
|
||||
}
|
||||
data_dictionary::table wrap(const ::table& t) const {
|
||||
data_dictionary::table wrap(const replica::table& t) const {
|
||||
return make_table(this, &t);
|
||||
}
|
||||
static const ::database& unwrap(data_dictionary::database db) {
|
||||
return *static_cast<const ::database*>(extract(db));
|
||||
static const replica::database& unwrap(data_dictionary::database db) {
|
||||
return *static_cast<const replica::database*>(extract(db));
|
||||
}
|
||||
static const ::keyspace& unwrap(data_dictionary::keyspace ks) {
|
||||
return *static_cast<const ::keyspace*>(extract(ks));
|
||||
static const replica::keyspace& unwrap(data_dictionary::keyspace ks) {
|
||||
return *static_cast<const replica::keyspace*>(extract(ks));
|
||||
}
|
||||
static const ::table& unwrap(data_dictionary::table t) {
|
||||
return *static_cast<const ::table*>(extract(t));
|
||||
static const replica::table& unwrap(data_dictionary::table t) {
|
||||
return *static_cast<const replica::table*>(extract(t));
|
||||
}
|
||||
friend class database;
|
||||
public:
|
||||
@@ -2311,7 +2333,7 @@ public:
|
||||
return unwrap(t).views();
|
||||
}
|
||||
virtual const secondary_index::secondary_index_manager& get_index_manager(data_dictionary::table t) const override {
|
||||
return const_cast<::table&>(unwrap(t)).get_index_manager();
|
||||
return const_cast<replica::table&>(unwrap(t)).get_index_manager();
|
||||
}
|
||||
virtual lw_shared_ptr<keyspace_metadata> get_keyspace_metadata(data_dictionary::keyspace ks) const override {
|
||||
return unwrap(ks).metadata();
|
||||
@@ -2338,8 +2360,8 @@ public:
|
||||
virtual const gms::feature_service& get_features(data_dictionary::database db) const override {
|
||||
return unwrap(db).features();
|
||||
}
|
||||
virtual ::database& real_database(data_dictionary::database db) const override {
|
||||
return const_cast<::database&>(unwrap(db));
|
||||
virtual replica::database& real_database(data_dictionary::database db) const override {
|
||||
return const_cast<replica::database&>(unwrap(db));
|
||||
}
|
||||
virtual schema_ptr get_cdc_base_table(data_dictionary::database db, const schema& s) const override {
|
||||
return cdc::get_base_table(unwrap(db), s);
|
||||
@@ -2352,10 +2374,12 @@ database::as_data_dictionary() const {
|
||||
return _impl.wrap(*this);
|
||||
}
|
||||
|
||||
} // namespace replica
|
||||
|
||||
template <typename T>
|
||||
using foreign_unique_ptr = foreign_ptr<std::unique_ptr<T>>;
|
||||
|
||||
flat_mutation_reader make_multishard_streaming_reader(distributed<database>& db, schema_ptr schema, reader_permit permit,
|
||||
flat_mutation_reader make_multishard_streaming_reader(distributed<replica::database>& db, schema_ptr schema, reader_permit permit,
|
||||
std::function<std::optional<dht::partition_range>()> range_generator) {
|
||||
class streaming_reader_lifecycle_policy
|
||||
: public reader_lifecycle_policy
|
||||
@@ -2365,11 +2389,11 @@ flat_mutation_reader make_multishard_streaming_reader(distributed<database>& db,
|
||||
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
|
||||
reader_concurrency_semaphore* semaphore;
|
||||
};
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
utils::UUID _table_id;
|
||||
std::vector<reader_context> _contexts;
|
||||
public:
|
||||
streaming_reader_lifecycle_policy(distributed<database>& db, utils::UUID table_id) : _db(db), _table_id(table_id), _contexts(smp::count) {
|
||||
streaming_reader_lifecycle_policy(distributed<replica::database>& db, utils::UUID table_id) : _db(db), _table_id(table_id), _contexts(smp::count) {
|
||||
}
|
||||
virtual flat_mutation_reader create_reader(
|
||||
schema_ptr schema,
|
||||
|
||||
@@ -87,6 +87,8 @@ class mutation;
|
||||
class frozen_mutation;
|
||||
class reconcilable_result;
|
||||
|
||||
class distributed_loader;
|
||||
|
||||
namespace service {
|
||||
class storage_proxy;
|
||||
class storage_service;
|
||||
@@ -130,7 +132,7 @@ class rp_handle;
|
||||
class data_listeners;
|
||||
class large_data_handler;
|
||||
|
||||
future<> system_keyspace_make(distributed<database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg);
|
||||
future<> system_keyspace_make(distributed<replica::database>& db, distributed<service::storage_service>& ss, sharded<gms::gossiper>& g, db::config& cfg);
|
||||
|
||||
}
|
||||
|
||||
@@ -141,7 +143,10 @@ class engine;
|
||||
class mutation_reordered_with_truncate_exception : public std::exception {};
|
||||
|
||||
using shared_memtable = lw_shared_ptr<memtable>;
|
||||
|
||||
class memtable_list;
|
||||
class column_family_test;
|
||||
class database_test;
|
||||
|
||||
// We could just add all memtables, regardless of types, to a single list, and
|
||||
// then filter them out when we read them. Here's why I have chosen not to do
|
||||
@@ -172,7 +177,7 @@ private:
|
||||
dirty_memory_manager* _dirty_memory_manager;
|
||||
std::optional<shared_future<>> _flush_coalescing;
|
||||
seastar::scheduling_group _compaction_scheduling_group;
|
||||
table_stats& _table_stats;
|
||||
replica::table_stats& _table_stats;
|
||||
public:
|
||||
using iterator = decltype(_memtables)::iterator;
|
||||
using const_iterator = decltype(_memtables)::const_iterator;
|
||||
@@ -181,7 +186,7 @@ public:
|
||||
seal_immediate_fn_type seal_immediate_fn,
|
||||
std::function<schema_ptr()> cs,
|
||||
dirty_memory_manager* dirty_memory_manager,
|
||||
table_stats& table_stats,
|
||||
replica::table_stats& table_stats,
|
||||
seastar::scheduling_group compaction_scheduling_group = seastar::current_scheduling_group())
|
||||
: _memtables({})
|
||||
, _seal_immediate_fn(seal_immediate_fn)
|
||||
@@ -193,7 +198,7 @@ public:
|
||||
}
|
||||
|
||||
memtable_list(std::function<schema_ptr()> cs, dirty_memory_manager* dirty_memory_manager,
|
||||
table_stats& table_stats,
|
||||
replica::table_stats& table_stats,
|
||||
seastar::scheduling_group compaction_scheduling_group = seastar::current_scheduling_group())
|
||||
: memtable_list({}, std::move(cs), dirty_memory_manager, table_stats, compaction_scheduling_group) {
|
||||
}
|
||||
@@ -286,6 +291,8 @@ private:
|
||||
|
||||
using sstable_list = sstables::sstable_list;
|
||||
|
||||
namespace replica {
|
||||
|
||||
// The CF has a "stats" structure. But we don't want all fields here,
|
||||
// since some of them are fairly complex for exporting to collectd. Also,
|
||||
// that structure matches what we export via the API, so better leave it
|
||||
@@ -375,7 +382,7 @@ public:
|
||||
::dirty_memory_manager* dirty_memory_manager = &default_dirty_memory_manager;
|
||||
reader_concurrency_semaphore* streaming_read_concurrency_semaphore;
|
||||
reader_concurrency_semaphore* compaction_concurrency_semaphore;
|
||||
::cf_stats* cf_stats = nullptr;
|
||||
replica::cf_stats* cf_stats = nullptr;
|
||||
seastar::scheduling_group memtable_scheduling_group;
|
||||
seastar::scheduling_group memtable_to_cache_scheduling_group;
|
||||
seastar::scheduling_group compaction_scheduling_group;
|
||||
@@ -900,7 +907,7 @@ public:
|
||||
return _view_stats;
|
||||
}
|
||||
|
||||
::cf_stats* cf_stats() {
|
||||
replica::cf_stats* cf_stats() {
|
||||
return _config.cf_stats;
|
||||
}
|
||||
|
||||
@@ -1060,9 +1067,9 @@ public:
|
||||
|
||||
friend std::ostream& operator<<(std::ostream& out, const column_family& cf);
|
||||
// Testing purposes.
|
||||
friend class column_family_test;
|
||||
friend class ::column_family_test;
|
||||
|
||||
friend class distributed_loader;
|
||||
friend class ::distributed_loader;
|
||||
|
||||
private:
|
||||
timer<> _off_strategy_trigger;
|
||||
@@ -1094,7 +1101,7 @@ public:
|
||||
::dirty_memory_manager* dirty_memory_manager = &default_dirty_memory_manager;
|
||||
reader_concurrency_semaphore* streaming_read_concurrency_semaphore;
|
||||
reader_concurrency_semaphore* compaction_concurrency_semaphore;
|
||||
::cf_stats* cf_stats = nullptr;
|
||||
replica::cf_stats* cf_stats = nullptr;
|
||||
seastar::scheduling_group memtable_scheduling_group;
|
||||
seastar::scheduling_group memtable_to_cache_scheduling_group;
|
||||
seastar::scheduling_group compaction_scheduling_group;
|
||||
@@ -1205,7 +1212,7 @@ struct string_pair_eq {
|
||||
// use shard_of() for data
|
||||
|
||||
class database {
|
||||
friend class database_test;
|
||||
friend class ::database_test;
|
||||
public:
|
||||
enum class table_kind {
|
||||
system,
|
||||
@@ -1224,7 +1231,7 @@ public:
|
||||
};
|
||||
|
||||
private:
|
||||
::cf_stats _cf_stats;
|
||||
replica::cf_stats _cf_stats;
|
||||
static constexpr size_t max_count_concurrent_reads{100};
|
||||
size_t max_memory_concurrent_reads() { return _dbcfg.available_memory * 0.02; }
|
||||
// Assume a queued read takes up 1kB of memory, and allow 2% of memory to be filled up with such reads.
|
||||
@@ -1608,13 +1615,15 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
future<> start_large_data_handler(sharded<database>& db);
|
||||
} // namespace replica
|
||||
|
||||
future<> start_large_data_handler(sharded<replica::database>& db);
|
||||
|
||||
// Creates a streaming reader that reads from all shards.
|
||||
//
|
||||
// Shard readers are created via `table::make_streaming_reader()`.
|
||||
// Range generator must generate disjoint, monotonically increasing ranges.
|
||||
flat_mutation_reader make_multishard_streaming_reader(distributed<database>& db, schema_ptr schema, reader_permit permit,
|
||||
flat_mutation_reader make_multishard_streaming_reader(distributed<replica::database>& db, schema_ptr schema, reader_permit permit,
|
||||
std::function<std::optional<dht::partition_range>()> range_generator);
|
||||
|
||||
bool is_internal_keyspace(std::string_view name);
|
||||
|
||||
@@ -21,11 +21,16 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
namespace replica {
|
||||
|
||||
// replica/database.hh
|
||||
class database;
|
||||
class keyspace;
|
||||
class table;
|
||||
using column_family = table;
|
||||
|
||||
}
|
||||
|
||||
class memtable_list;
|
||||
|
||||
// mutation.hh
|
||||
|
||||
@@ -55,6 +55,8 @@
|
||||
#include <boost/range/algorithm/remove_if.hpp>
|
||||
#include <boost/range/algorithm.hpp>
|
||||
|
||||
namespace replica {
|
||||
|
||||
static logging::logger tlogger("table");
|
||||
static seastar::metrics::label column_family_label("cf");
|
||||
static seastar::metrics::label keyspace_label("ks");
|
||||
@@ -1546,7 +1548,7 @@ future<db::replay_position> table::discard_sstables(db_clock::time_point truncat
|
||||
db::replay_position rp;
|
||||
struct removed_sstable {
|
||||
sstables::shared_sstable sst;
|
||||
::enable_backlog_tracker enable_backlog_tracker;
|
||||
replica::enable_backlog_tracker enable_backlog_tracker;
|
||||
};
|
||||
std::vector<removed_sstable> remove;
|
||||
|
||||
@@ -1561,7 +1563,7 @@ future<db::replay_position> table::discard_sstables(db_clock::time_point truncat
|
||||
|
||||
auto prune = [this, &gc_trunc] (lw_shared_ptr<sstables::sstable_set>& pruned,
|
||||
lw_shared_ptr<sstables::sstable_set>& pruning,
|
||||
::enable_backlog_tracker enable_backlog_tracker) mutable {
|
||||
replica::enable_backlog_tracker enable_backlog_tracker) mutable {
|
||||
pruning->for_each_sstable([&] (const sstables::shared_sstable& p) mutable {
|
||||
if (p->max_data_age() <= gc_trunc) {
|
||||
rp = std::max(p->get_stats_metadata().position, rp);
|
||||
@@ -1957,6 +1959,8 @@ future<> table::apply(const frozen_mutation& m, schema_ptr m_schema, db::rp_hand
|
||||
|
||||
template void table::do_apply(db::rp_handle&&, const frozen_mutation&, const schema_ptr&);
|
||||
|
||||
}
|
||||
|
||||
future<>
|
||||
write_memtable_to_sstable(flat_mutation_reader reader,
|
||||
memtable& mt, sstables::shared_sstable sst,
|
||||
@@ -1982,7 +1986,7 @@ write_memtable_to_sstable(reader_permit permit, memtable& mt, sstables::shared_s
|
||||
future<>
|
||||
write_memtable_to_sstable(memtable& mt, sstables::shared_sstable sst, sstables::sstable_writer_config cfg) {
|
||||
return do_with(
|
||||
permit_monitor(make_lw_shared(sstable_write_permit::unconditional())),
|
||||
replica::permit_monitor(make_lw_shared(sstable_write_permit::unconditional())),
|
||||
std::make_unique<reader_concurrency_semaphore>(reader_concurrency_semaphore::no_limits{}, "write_memtable_to_sstable"),
|
||||
cfg,
|
||||
[&mt, sst] (auto& monitor, auto& semaphore, auto& cfg) {
|
||||
@@ -1993,6 +1997,7 @@ write_memtable_to_sstable(memtable& mt, sstables::shared_sstable sst, sstables::
|
||||
});
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
|
||||
struct query_state {
|
||||
explicit query_state(schema_ptr s,
|
||||
@@ -2430,3 +2435,5 @@ public:
|
||||
compaction::table_state& table::as_table_state() const noexcept {
|
||||
return *_table_state;
|
||||
}
|
||||
|
||||
} // namespace replica
|
||||
|
||||
@@ -755,16 +755,16 @@ static std::ostream& column_definition_as_cql_key(std::ostream& os, const column
|
||||
return os;
|
||||
}
|
||||
|
||||
static bool is_global_index(database& db, const utils::UUID& id, const schema& s) {
|
||||
static bool is_global_index(replica::database& db, const utils::UUID& id, const schema& s) {
|
||||
return db.find_column_family(id).get_index_manager().is_global_index(s);
|
||||
}
|
||||
|
||||
static bool is_index(database& db, const utils::UUID& id, const schema& s) {
|
||||
static bool is_index(replica::database& db, const utils::UUID& id, const schema& s) {
|
||||
return db.find_column_family(id).get_index_manager().is_index(s);
|
||||
}
|
||||
|
||||
|
||||
std::ostream& schema::describe(database& db, std::ostream& os) const {
|
||||
std::ostream& schema::describe(replica::database& db, std::ostream& os) const {
|
||||
os << "CREATE ";
|
||||
int n = 0;
|
||||
|
||||
|
||||
@@ -54,7 +54,9 @@ namespace cdc {
|
||||
class options;
|
||||
}
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
using column_count_type = uint32_t;
|
||||
|
||||
@@ -960,7 +962,7 @@ public:
|
||||
* Index or Local Index).
|
||||
*
|
||||
*/
|
||||
std::ostream& describe(database& db, std::ostream& os) const;
|
||||
std::ostream& describe(replica::database& db, std::ostream& os) const;
|
||||
friend bool operator==(const schema&, const schema&);
|
||||
const column_mapping& get_column_mapping() const;
|
||||
friend class schema_registry_entry;
|
||||
|
||||
@@ -1005,7 +1005,7 @@ def for_each_table(db=None):
|
||||
db = find_db()
|
||||
cfs = db['_column_families']
|
||||
for (key, value) in unordered_map(cfs):
|
||||
yield value['_p'].reinterpret_cast(gdb.lookup_type('column_family').pointer()).dereference() # it's a lw_shared_ptr
|
||||
yield value['_p'].reinterpret_cast(lookup_type(['replica::table', 'column_family'])[1].pointer()).dereference() # it's a lw_shared_ptr
|
||||
|
||||
|
||||
def lookup_type(type_names):
|
||||
@@ -1163,7 +1163,7 @@ class scylla_databases(gdb.Command):
|
||||
def invoke(self, arg, from_tty):
|
||||
for shard in range(cpus()):
|
||||
db = find_db(shard)
|
||||
gdb.write('{:5} (database*){}\n'.format(shard, db))
|
||||
gdb.write('{:5} (replica::database*){}\n'.format(shard, db))
|
||||
|
||||
|
||||
class scylla_keyspaces(gdb.Command):
|
||||
@@ -1175,7 +1175,7 @@ class scylla_keyspaces(gdb.Command):
|
||||
db = find_db(shard)
|
||||
keyspaces = db['_keyspaces']
|
||||
for (key, value) in unordered_map(keyspaces):
|
||||
gdb.write('{:5} {:20} (keyspace*){}\n'.format(shard, str(key), value.address))
|
||||
gdb.write('{:5} {:20} (replica::keyspace*){}\n'.format(shard, str(key), value.address))
|
||||
|
||||
|
||||
class scylla_column_families(gdb.Command):
|
||||
@@ -1187,11 +1187,11 @@ class scylla_column_families(gdb.Command):
|
||||
db = find_db(shard)
|
||||
cfs = db['_column_families']
|
||||
for (key, value) in unordered_map(cfs):
|
||||
value = value['_p'].reinterpret_cast(gdb.lookup_type('column_family').pointer()).dereference() # it's a lw_shared_ptr
|
||||
value = value['_p'].reinterpret_cast(lookup_type(['replica::table', 'column_family'])[1].pointer()).dereference() # it's a lw_shared_ptr
|
||||
schema = value['_schema']['_p'].reinterpret_cast(gdb.lookup_type('schema').pointer())
|
||||
name = str(schema['_raw']['_ks_name']) + '/' + str(schema['_raw']['_cf_name'])
|
||||
schema_version = str(schema['_raw']['_version'])
|
||||
gdb.write('{:5} {} v={} {:45} (column_family*){}\n'.format(shard, key, schema_version, name, value.address))
|
||||
gdb.write('{:5} {} v={} {:45} (replica::table*){}\n'.format(shard, key, schema_version, name, value.address))
|
||||
|
||||
|
||||
class scylla_task_histogram(gdb.Command):
|
||||
@@ -1751,20 +1751,21 @@ class scylla_memory(gdb.Command):
|
||||
for key, sem in [('user_mem_str', db['_read_concurrency_sem']), ('streaming_mem_str', db['_streaming_concurrency_sem']), ('system_mem_str', db['_system_read_concurrency_sem'])]:
|
||||
mem_stats[key] = 'remaining mem: {:>13} B'.format(int(sem['_resources']['memory']))
|
||||
|
||||
database_typename = lookup_type(['replica::database', 'database'])[1].name
|
||||
gdb.write('Replica:\n')
|
||||
gdb.write(' Read Concurrency Semaphores:\n'
|
||||
' user sstable reads: {user_sst_rd_count:>3}/{user_sst_rd_max_count:>3}, {user_mem_str}, queued: {user_sst_rd_queued}\n'
|
||||
' streaming sstable reads: {streaming_sst_rd_count:>3}/{streaming_sst_rd_max_count:>3}, {streaming_mem_str}, queued: {streaming_sst_rd_queued}\n'
|
||||
' system sstable reads: {system_sst_rd_count:>3}/{system_sst_rd_max_count:>3}, {system_mem_str}, queued: {system_sst_rd_queued}\n'
|
||||
.format(
|
||||
user_sst_rd_count=int(gdb.parse_and_eval('database::max_count_concurrent_reads')) - int(db['_read_concurrency_sem']['_resources']['count']),
|
||||
user_sst_rd_max_count=int(gdb.parse_and_eval('database::max_count_concurrent_reads')),
|
||||
user_sst_rd_count=int(gdb.parse_and_eval(f'{database_typename}::max_count_concurrent_reads')) - int(db['_read_concurrency_sem']['_resources']['count']),
|
||||
user_sst_rd_max_count=int(gdb.parse_and_eval(f'{database_typename}::max_count_concurrent_reads')),
|
||||
user_sst_rd_queued=int(db['_read_concurrency_sem']['_wait_list']['_size']),
|
||||
streaming_sst_rd_count=int(gdb.parse_and_eval('database::max_count_streaming_concurrent_reads')) - int(db['_streaming_concurrency_sem']['_resources']['count']),
|
||||
streaming_sst_rd_max_count=int(gdb.parse_and_eval('database::max_count_streaming_concurrent_reads')),
|
||||
streaming_sst_rd_count=int(gdb.parse_and_eval(f'{database_typename}::max_count_streaming_concurrent_reads')) - int(db['_streaming_concurrency_sem']['_resources']['count']),
|
||||
streaming_sst_rd_max_count=int(gdb.parse_and_eval(f'{database_typename}::max_count_streaming_concurrent_reads')),
|
||||
streaming_sst_rd_queued=int(db['_streaming_concurrency_sem']['_wait_list']['_size']),
|
||||
system_sst_rd_count=int(gdb.parse_and_eval('database::max_count_system_concurrent_reads')) - int(db['_system_read_concurrency_sem']['_resources']['count']),
|
||||
system_sst_rd_max_count=int(gdb.parse_and_eval('database::max_count_system_concurrent_reads')),
|
||||
system_sst_rd_count=int(gdb.parse_and_eval(f'{database_typename}::max_count_system_concurrent_reads')) - int(db['_system_read_concurrency_sem']['_resources']['count']),
|
||||
system_sst_rd_max_count=int(gdb.parse_and_eval(f'{database_typename}::max_count_system_concurrent_reads')),
|
||||
system_sst_rd_queued=int(db['_system_read_concurrency_sem']['_wait_list']['_size']),
|
||||
**mem_stats))
|
||||
|
||||
|
||||
@@ -43,7 +43,7 @@ class cache_hitrate_calculator : public seastar::async_sharded_service<cache_hit
|
||||
}
|
||||
};
|
||||
|
||||
seastar::sharded<database>& _db;
|
||||
seastar::sharded<replica::database>& _db;
|
||||
gms::gossiper& _gossiper;
|
||||
timer<lowres_clock> _timer;
|
||||
bool _stopped = false;
|
||||
@@ -56,7 +56,7 @@ class cache_hitrate_calculator : public seastar::async_sharded_service<cache_hit
|
||||
future<lowres_clock::duration> recalculate_hitrates();
|
||||
void recalculate_timer();
|
||||
public:
|
||||
cache_hitrate_calculator(seastar::sharded<database>& db, gms::gossiper& g);
|
||||
cache_hitrate_calculator(seastar::sharded<replica::database>& db, gms::gossiper& g);
|
||||
void run_on(size_t master, lowres_clock::duration d = std::chrono::milliseconds(2000));
|
||||
|
||||
future<> stop();
|
||||
|
||||
@@ -105,13 +105,13 @@ future<> service::client_state::has_all_keyspaces_access(
|
||||
co_return co_await ensure_has_permission({p, r});
|
||||
}
|
||||
|
||||
future<> service::client_state::has_keyspace_access(const database& db, const sstring& ks,
|
||||
future<> service::client_state::has_keyspace_access(const replica::database& db, const sstring& ks,
|
||||
auth::permission p) const {
|
||||
auth::resource r = auth::make_data_resource(ks);
|
||||
co_return co_await has_access(db, ks, {p, r});
|
||||
}
|
||||
|
||||
future<> service::client_state::has_column_family_access(const database& db, const sstring& ks,
|
||||
future<> service::client_state::has_column_family_access(const replica::database& db, const sstring& ks,
|
||||
const sstring& cf, auth::permission p, auth::command_desc::type t) const {
|
||||
// NOTICE: callers of this function tend to assume that this error will be thrown
|
||||
// synchronously and will be intercepted in a try-catch block. Thus, this function can only
|
||||
@@ -123,12 +123,12 @@ future<> service::client_state::has_column_family_access(const database& db, con
|
||||
});
|
||||
}
|
||||
|
||||
future<> service::client_state::has_schema_access(const database& db, const schema& s, auth::permission p) const {
|
||||
future<> service::client_state::has_schema_access(const replica::database& db, const schema& s, auth::permission p) const {
|
||||
auth::resource r = auth::make_data_resource(s.ks_name(), s.cf_name());
|
||||
co_return co_await has_access(db, s.ks_name(), {p, r});
|
||||
}
|
||||
|
||||
future<> service::client_state::has_access(const database& db, const sstring& ks, auth::command_desc cmd) const {
|
||||
future<> service::client_state::has_access(const replica::database& db, const sstring& ks, auth::command_desc cmd) const {
|
||||
if (ks.empty()) {
|
||||
return make_exception_future<>(exceptions::invalid_request_exception("You have not set a keyspace for this session"));
|
||||
}
|
||||
@@ -246,7 +246,7 @@ future<> service::client_state::ensure_has_permission(auth::command_desc cmd) co
|
||||
});
|
||||
}
|
||||
|
||||
void service::client_state::set_keyspace(database& db, std::string_view keyspace) {
|
||||
void service::client_state::set_keyspace(replica::database& db, std::string_view keyspace) {
|
||||
// Skip keyspace validation for non-authenticated users. Apparently, some client libraries
|
||||
// call set_keyspace() before calling login(), and we have to handle that.
|
||||
if (_user && !db.has_keyspace(keyspace)) {
|
||||
|
||||
@@ -325,7 +325,7 @@ public:
|
||||
}
|
||||
|
||||
public:
|
||||
void set_keyspace(database& db, std::string_view keyspace);
|
||||
void set_keyspace(replica::database& db, std::string_view keyspace);
|
||||
|
||||
void set_raw_keyspace(sstring new_keyspace) noexcept {
|
||||
_keyspace = std::move(new_keyspace);
|
||||
@@ -347,13 +347,13 @@ public:
|
||||
future<> check_user_can_login();
|
||||
|
||||
future<> has_all_keyspaces_access(auth::permission) const;
|
||||
future<> has_keyspace_access(const database& db, const sstring&, auth::permission) const;
|
||||
future<> has_column_family_access(const database& db, const sstring&, const sstring&, auth::permission,
|
||||
future<> has_keyspace_access(const replica::database& db, const sstring&, auth::permission) const;
|
||||
future<> has_column_family_access(const replica::database& db, const sstring&, const sstring&, auth::permission,
|
||||
auth::command_desc::type = auth::command_desc::type::OTHER) const;
|
||||
future<> has_schema_access(const database& db, const schema& s, auth::permission p) const;
|
||||
future<> has_schema_access(const replica::database& db, const schema& s, auth::permission p) const;
|
||||
|
||||
private:
|
||||
future<> has_access(const database& db, const sstring& keyspace, auth::command_desc) const;
|
||||
future<> has_access(const replica::database& db, const sstring& keyspace, auth::command_desc) const;
|
||||
|
||||
public:
|
||||
future<bool> check_has_permission(auth::command_desc) const;
|
||||
|
||||
@@ -51,7 +51,7 @@ public:
|
||||
static constexpr std::chrono::milliseconds BROADCAST_INTERVAL{60 * 1000};
|
||||
|
||||
private:
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
gms::gossiper& _gossiper;
|
||||
std::unordered_map<gms::inet_address, double> _load_info;
|
||||
timer<> _timer;
|
||||
@@ -59,7 +59,7 @@ private:
|
||||
bool _stopped = false;
|
||||
|
||||
public:
|
||||
load_broadcaster(distributed<database>& db, gms::gossiper& g) : _db(db), _gossiper(g) {
|
||||
load_broadcaster(distributed<replica::database>& db, gms::gossiper& g) : _db(db), _gossiper(g) {
|
||||
_gossiper.register_(shared_from_this());
|
||||
}
|
||||
~load_broadcaster() {
|
||||
|
||||
@@ -27,7 +27,10 @@
|
||||
|
||||
using namespace seastar;
|
||||
|
||||
namespace replica {
|
||||
class database;
|
||||
}
|
||||
|
||||
namespace gms { class gossiper; }
|
||||
|
||||
namespace service {
|
||||
@@ -43,7 +46,7 @@ private:
|
||||
public:
|
||||
future<std::map<sstring, double>> get_load_map();
|
||||
|
||||
future<> init(distributed<database>& db, gms::gossiper& gossiper);
|
||||
future<> init(distributed<replica::database>& db, gms::gossiper& gossiper);
|
||||
future<> exit();
|
||||
};
|
||||
|
||||
|
||||
@@ -257,7 +257,7 @@ future<> migration_manager::maybe_schedule_schema_pull(const utils::UUID& their_
|
||||
return make_ready_future<>();
|
||||
}
|
||||
|
||||
if (db.get_version() == database::empty_version || runtime::get_uptime() < migration_delay) {
|
||||
if (db.get_version() == replica::database::empty_version || runtime::get_uptime() < migration_delay) {
|
||||
// If we think we may be bootstrapping or have recently started, submit MigrationTask immediately
|
||||
mlogger.debug("Submitting migration task for {}", endpoint);
|
||||
return submit_migration_task(endpoint);
|
||||
@@ -354,7 +354,7 @@ future<> migration_manager::merge_schema_from(netw::messaging_service::msg_addr
|
||||
mutations.emplace_back(cm.to_mutation(
|
||||
tbl.schema()));
|
||||
}
|
||||
} catch (no_such_column_family& e) {
|
||||
} catch (replica::no_such_column_family& e) {
|
||||
mlogger.error("Error while applying schema mutations from {}: {}", src, e);
|
||||
return make_exception_future<>(std::make_exception_ptr<std::runtime_error>(
|
||||
std::runtime_error(fmt::format("Error while applying schema mutations: {}", e))));
|
||||
@@ -702,7 +702,7 @@ future<std::vector<mutation>> migration_manager::prepare_new_column_family_annou
|
||||
}).then([this, ksm](std::vector<mutation> mutations) {
|
||||
return include_keyspace(*ksm, std::move(mutations));
|
||||
});
|
||||
} catch (const no_such_keyspace& e) {
|
||||
} 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()));
|
||||
}
|
||||
}
|
||||
@@ -732,7 +732,7 @@ future<std::vector<mutation>> migration_manager::prepare_column_family_update_an
|
||||
}
|
||||
get_notifier().before_update_column_family(*cfm, *old_schema, mutations, ts);
|
||||
co_return co_await include_keyspace(*keyspace, std::move(mutations));
|
||||
} catch (const no_such_column_family& e) {
|
||||
} catch (const replica::no_such_column_family& e) {
|
||||
co_return coroutine::make_exception(exceptions::configuration_exception(format("Cannot update non existing table '{}' in keyspace '{}'.",
|
||||
cfm->cf_name(), cfm->ks_name())));
|
||||
}
|
||||
@@ -885,7 +885,7 @@ future<std::vector<mutation>> migration_manager::prepare_column_family_drop_anno
|
||||
get_notifier().before_drop_column_family(*schema, mutations, ts);
|
||||
});
|
||||
co_return co_await include_keyspace(*keyspace, std::move(mutations));
|
||||
} catch (const no_such_column_family& e) {
|
||||
} catch (const replica::no_such_column_family& e) {
|
||||
co_return coroutine::make_exception(exceptions::configuration_exception(format("Cannot drop non existing table '{}' in keyspace '{}'.", cf_name, ks_name)));
|
||||
}
|
||||
}
|
||||
@@ -918,7 +918,7 @@ future<std::vector<mutation>> migration_manager::prepare_new_view_announcement(v
|
||||
mlogger.info("Create new view: {}", view);
|
||||
auto mutations = db::schema_tables::make_create_view_mutations(keyspace, std::move(view), api::new_timestamp());
|
||||
co_return co_await include_keyspace(*keyspace, std::move(mutations));
|
||||
} catch (const no_such_keyspace& e) {
|
||||
} catch (const replica::no_such_keyspace& e) {
|
||||
co_return coroutine::make_exception(exceptions::configuration_exception(format("Cannot add view '{}' to non existing keyspace '{}'.", view->cf_name(), view->ks_name())));
|
||||
}
|
||||
}
|
||||
@@ -964,7 +964,7 @@ future<std::vector<mutation>> migration_manager::prepare_view_drop_announcement(
|
||||
mlogger.info("Drop view '{}.{}'", view->ks_name(), view->cf_name());
|
||||
auto mutations = db::schema_tables::make_drop_view_mutations(keyspace, view_ptr(std::move(view)), api::new_timestamp());
|
||||
return include_keyspace(*keyspace, std::move(mutations));
|
||||
} catch (const no_such_column_family& e) {
|
||||
} catch (const replica::no_such_column_family& e) {
|
||||
throw exceptions::configuration_exception(format("Cannot drop non existing materialized view '{}' in keyspace '{}'.",
|
||||
cf_name, ks_name));
|
||||
}
|
||||
@@ -1205,7 +1205,7 @@ future<schema_ptr> migration_manager::get_schema_for_write(table_schema_version
|
||||
});
|
||||
}
|
||||
|
||||
future<> migration_manager::sync_schema(const database& db, const std::vector<gms::inet_address>& nodes) {
|
||||
future<> migration_manager::sync_schema(const replica::database& db, const std::vector<gms::inet_address>& nodes) {
|
||||
using schema_and_hosts = std::unordered_map<utils::UUID, std::vector<gms::inet_address>>;
|
||||
return do_with(schema_and_hosts(), db.get_version(), [this, &nodes] (schema_and_hosts& schema_map, utils::UUID& my_version) {
|
||||
return parallel_for_each(nodes, [this, &schema_map, &my_version] (const gms::inet_address& node) {
|
||||
|
||||
@@ -104,7 +104,7 @@ public:
|
||||
future<> submit_migration_task(const gms::inet_address& endpoint, bool can_ignore_down_node = true);
|
||||
|
||||
// Makes sure that this node knows about all schema changes known by "nodes" that were made prior to this call.
|
||||
future<> sync_schema(const database& db, const std::vector<gms::inet_address>& nodes);
|
||||
future<> sync_schema(const replica::database& db, const std::vector<gms::inet_address>& nodes);
|
||||
|
||||
// Fetches schema from remote node and applies it locally.
|
||||
// Differs from submit_migration_task() in that all errors are propagated.
|
||||
|
||||
@@ -57,7 +57,7 @@ constexpr std::chrono::milliseconds load_broadcaster::BROADCAST_INTERVAL;
|
||||
|
||||
logging::logger llogger("load_broadcaster");
|
||||
|
||||
future<> load_meter::init(distributed<database>& db, gms::gossiper& gms) {
|
||||
future<> load_meter::init(distributed<replica::database>& db, gms::gossiper& gms) {
|
||||
_lb = make_shared<load_broadcaster>(db, gms);
|
||||
_lb->start_broadcasting();
|
||||
return make_ready_future<>();
|
||||
@@ -111,7 +111,7 @@ void load_broadcaster::start_broadcasting() {
|
||||
|
||||
_timer.set_callback([this] {
|
||||
llogger.debug("Disseminating load info ...");
|
||||
_done = _db.map_reduce0([](database& db) {
|
||||
_done = _db.map_reduce0([](replica::database& db) {
|
||||
int64_t res = 0;
|
||||
for (auto i : db.get_column_families()) {
|
||||
res += i.second->get_stats().live_disk_space_used;
|
||||
@@ -140,7 +140,7 @@ future<> load_broadcaster::stop_broadcasting() {
|
||||
|
||||
|
||||
// cache_hitrate_calculator implementation
|
||||
cache_hitrate_calculator::cache_hitrate_calculator(seastar::sharded<database>& db, gms::gossiper& g)
|
||||
cache_hitrate_calculator::cache_hitrate_calculator(seastar::sharded<replica::database>& db, gms::gossiper& g)
|
||||
: _db(db), _gossiper(g),
|
||||
_timer(std::bind(std::mem_fn(&cache_hitrate_calculator::recalculate_timer), this))
|
||||
{}
|
||||
@@ -167,13 +167,13 @@ void cache_hitrate_calculator::run_on(size_t master, lowres_clock::duration d) {
|
||||
}
|
||||
|
||||
future<lowres_clock::duration> cache_hitrate_calculator::recalculate_hitrates() {
|
||||
auto non_system_filter = [&] (const std::pair<utils::UUID, lw_shared_ptr<column_family>>& cf) {
|
||||
auto non_system_filter = [&] (const std::pair<utils::UUID, lw_shared_ptr<replica::column_family>>& cf) {
|
||||
return _db.local().find_keyspace(cf.second->schema()->ks_name()).get_replication_strategy().get_type() != locator::replication_strategy_type::local;
|
||||
};
|
||||
|
||||
auto cf_to_cache_hit_stats = [non_system_filter] (database& db) {
|
||||
auto cf_to_cache_hit_stats = [non_system_filter] (replica::database& db) {
|
||||
return boost::copy_range<std::unordered_map<utils::UUID, stat>>(db.get_column_families() | boost::adaptors::filtered(non_system_filter) |
|
||||
boost::adaptors::transformed([] (const std::pair<utils::UUID, lw_shared_ptr<column_family>>& cf) {
|
||||
boost::adaptors::transformed([] (const std::pair<utils::UUID, lw_shared_ptr<replica::column_family>>& cf) {
|
||||
auto& stats = cf.second->get_row_cache().stats();
|
||||
return std::make_pair(cf.first, stat{float(stats.reads_with_no_misses.rate().rates[0]), float(stats.reads_with_misses.rate().rates[0])});
|
||||
}));
|
||||
@@ -192,7 +192,7 @@ future<lowres_clock::duration> cache_hitrate_calculator::recalculate_hitrates()
|
||||
_slen = 0;
|
||||
_rates = std::move(rates);
|
||||
// set calculated rates on all shards
|
||||
return _db.invoke_on_all([this, cpuid = this_shard_id(), non_system_filter] (database& db) {
|
||||
return _db.invoke_on_all([this, cpuid = this_shard_id(), non_system_filter] (replica::database& db) {
|
||||
return do_for_each(_rates, [this, cpuid, &db] (auto&& r) mutable {
|
||||
auto it = db.get_column_families().find(r.first);
|
||||
if (it == db.get_column_families().end()) { // a table may be added before map/reduce completes and this code runs
|
||||
|
||||
@@ -188,7 +188,7 @@ future<> paxos_state::learn(storage_proxy& sp, schema_ptr schema, proposal decis
|
||||
return do_with(std::move(decision), [&sp, tr_state = std::move(tr_state), schema, timeout] (proposal& decision) {
|
||||
auto f = utils::get_local_injector().inject("paxos_state_learn_timeout", timeout);
|
||||
|
||||
table& cf = sp.get_db().local().find_column_family(schema);
|
||||
replica::table& cf = sp.get_db().local().find_column_family(schema);
|
||||
db_clock::time_point t = cf.get_truncation_record();
|
||||
auto truncated_at = std::chrono::duration_cast<std::chrono::milliseconds>(t.time_since_epoch());
|
||||
// When saving a decision, also delete the last accepted proposal. This is just an
|
||||
|
||||
@@ -409,7 +409,7 @@ protected:
|
||||
}
|
||||
|
||||
public:
|
||||
abstract_write_response_handler(shared_ptr<storage_proxy> p, keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
abstract_write_response_handler(shared_ptr<storage_proxy> p, replica::keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
std::unique_ptr<mutation_holder> mh, inet_address_vector_replica_set targets, tracing::trace_state_ptr trace_state,
|
||||
storage_proxy::write_stats& stats, service_permit permit, size_t pending_endpoints = 0, inet_address_vector_topology_change dead_endpoints = {})
|
||||
: _id(p->get_next_response_id()), _proxy(std::move(p)), _trace_state(trace_state), _cl(cl), _type(type), _mutation_holder(std::move(mh)), _targets(std::move(targets)),
|
||||
@@ -659,7 +659,7 @@ class datacenter_write_response_handler : public abstract_write_response_handler
|
||||
}
|
||||
|
||||
public:
|
||||
datacenter_write_response_handler(shared_ptr<storage_proxy> p, keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
datacenter_write_response_handler(shared_ptr<storage_proxy> p, replica::keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
std::unique_ptr<mutation_holder> mh, inet_address_vector_replica_set targets,
|
||||
const inet_address_vector_topology_change& pending_endpoints, inet_address_vector_topology_change dead_endpoints, tracing::trace_state_ptr tr_state,
|
||||
storage_proxy::write_stats& stats, service_permit permit) :
|
||||
@@ -674,7 +674,7 @@ class write_response_handler : public abstract_write_response_handler {
|
||||
return true;
|
||||
}
|
||||
public:
|
||||
write_response_handler(shared_ptr<storage_proxy> p, keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
write_response_handler(shared_ptr<storage_proxy> p, replica::keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
std::unique_ptr<mutation_holder> mh, inet_address_vector_replica_set targets,
|
||||
const inet_address_vector_topology_change& pending_endpoints, inet_address_vector_topology_change dead_endpoints, tracing::trace_state_ptr tr_state,
|
||||
storage_proxy::write_stats& stats, service_permit permit) :
|
||||
@@ -686,7 +686,7 @@ public:
|
||||
|
||||
class view_update_write_response_handler : public write_response_handler, public bi::list_base_hook<bi::link_mode<bi::auto_unlink>> {
|
||||
public:
|
||||
view_update_write_response_handler(shared_ptr<storage_proxy> p, keyspace& ks, db::consistency_level cl,
|
||||
view_update_write_response_handler(shared_ptr<storage_proxy> p, replica::keyspace& ks, db::consistency_level cl,
|
||||
std::unique_ptr<mutation_holder> mh, inet_address_vector_replica_set targets,
|
||||
const inet_address_vector_topology_change& pending_endpoints, inet_address_vector_topology_change dead_endpoints, tracing::trace_state_ptr tr_state,
|
||||
storage_proxy::write_stats& stats, service_permit permit):
|
||||
@@ -766,7 +766,7 @@ class datacenter_sync_write_response_handler : public abstract_write_response_ha
|
||||
return false;
|
||||
}
|
||||
public:
|
||||
datacenter_sync_write_response_handler(shared_ptr<storage_proxy> p, keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
datacenter_sync_write_response_handler(shared_ptr<storage_proxy> p, replica::keyspace& ks, db::consistency_level cl, db::write_type type,
|
||||
std::unique_ptr<mutation_holder> mh, inet_address_vector_replica_set targets, const inet_address_vector_topology_change& pending_endpoints,
|
||||
inet_address_vector_topology_change dead_endpoints, tracing::trace_state_ptr tr_state, storage_proxy::write_stats& stats, service_permit permit) :
|
||||
abstract_write_response_handler(std::move(p), ks, cl, type, std::move(mh), targets, std::move(tr_state), stats, std::move(permit), 0, dead_endpoints) {
|
||||
@@ -1450,7 +1450,7 @@ future<> storage_proxy::response_wait(storage_proxy::response_id_type id, clock_
|
||||
return _response_handlers.find(id)->second;
|
||||
}
|
||||
|
||||
storage_proxy::response_id_type storage_proxy::create_write_response_handler(keyspace& ks, db::consistency_level cl, db::write_type type, std::unique_ptr<mutation_holder> m,
|
||||
storage_proxy::response_id_type storage_proxy::create_write_response_handler(replica::keyspace& ks, db::consistency_level cl, db::write_type type, std::unique_ptr<mutation_holder> m,
|
||||
inet_address_vector_replica_set targets, const inet_address_vector_topology_change& pending_endpoints, inet_address_vector_topology_change dead_endpoints, tracing::trace_state_ptr tr_state,
|
||||
storage_proxy::write_stats& stats, service_permit permit)
|
||||
{
|
||||
@@ -1815,7 +1815,7 @@ inline std::ostream& operator<<(std::ostream& os, const hint_wrapper& h) {
|
||||
using namespace std::literals::chrono_literals;
|
||||
|
||||
storage_proxy::~storage_proxy() {}
|
||||
storage_proxy::storage_proxy(distributed<database>& db, gms::gossiper& gossiper, storage_proxy::config cfg, db::view::node_update_backlog& max_view_update_backlog,
|
||||
storage_proxy::storage_proxy(distributed<replica::database>& db, gms::gossiper& gossiper, storage_proxy::config cfg, db::view::node_update_backlog& max_view_update_backlog,
|
||||
scheduling_group_key stats_key, gms::feature_service& feat, const locator::shared_token_metadata& stm, locator::effective_replication_map_factory& erm_factory, netw::messaging_service& ms)
|
||||
: _db(db)
|
||||
, _gossiper(gossiper)
|
||||
@@ -1892,7 +1892,7 @@ storage_proxy::mutate_locally(const mutation& m, tracing::trace_state_ptr tr_sta
|
||||
m = freeze(m),
|
||||
gtr = tracing::global_trace_state_ptr(std::move(tr_state)),
|
||||
timeout,
|
||||
sync] (database& db) mutable -> future<> {
|
||||
sync] (replica::database& db) mutable -> future<> {
|
||||
return db.apply(s, m, gtr.get(), sync, timeout);
|
||||
});
|
||||
}
|
||||
@@ -1903,7 +1903,7 @@ storage_proxy::mutate_locally(const schema_ptr& s, const frozen_mutation& m, tra
|
||||
auto shard = _db.local().shard_of(m);
|
||||
get_stats().replica_cross_shard_ops += shard != this_shard_id();
|
||||
return _db.invoke_on(shard, {smp_grp, timeout},
|
||||
[&m, gs = global_schema_ptr(s), gtr = tracing::global_trace_state_ptr(std::move(tr_state)), timeout, sync] (database& db) mutable -> future<> {
|
||||
[&m, gs = global_schema_ptr(s), gtr = tracing::global_trace_state_ptr(std::move(tr_state)), timeout, sync] (replica::database& db) mutable -> future<> {
|
||||
return db.apply(gs, m, gtr.get(), sync, timeout);
|
||||
});
|
||||
}
|
||||
@@ -1925,7 +1925,7 @@ future<>
|
||||
storage_proxy::mutate_hint(const schema_ptr& s, const frozen_mutation& m, tracing::trace_state_ptr tr_state, clock_type::time_point timeout) {
|
||||
auto shard = _db.local().shard_of(m);
|
||||
get_stats().replica_cross_shard_ops += shard != this_shard_id();
|
||||
return _db.invoke_on(shard, {_hints_write_smp_service_group, timeout}, [&m, gs = global_schema_ptr(s), tr_state = std::move(tr_state), timeout] (database& db) mutable -> future<> {
|
||||
return _db.invoke_on(shard, {_hints_write_smp_service_group, timeout}, [&m, gs = global_schema_ptr(s), tr_state = std::move(tr_state), timeout] (replica::database& db) mutable -> future<> {
|
||||
return db.apply_hint(gs, m, std::move(tr_state), timeout);
|
||||
});
|
||||
}
|
||||
@@ -1947,7 +1947,7 @@ storage_proxy::mutate_counter_on_leader_and_replicate(const schema_ptr& s, froze
|
||||
auto shard = _db.local().shard_of(fm);
|
||||
bool local = shard == this_shard_id();
|
||||
get_stats().replica_cross_shard_ops += !local;
|
||||
return _db.invoke_on(shard, {_write_smp_service_group, timeout}, [gs = global_schema_ptr(s), fm = std::move(fm), cl, timeout, gt = tracing::global_trace_state_ptr(std::move(trace_state)), permit = std::move(permit), local] (database& db) {
|
||||
return _db.invoke_on(shard, {_write_smp_service_group, timeout}, [gs = global_schema_ptr(s), fm = std::move(fm), cl, timeout, gt = tracing::global_trace_state_ptr(std::move(trace_state)), permit = std::move(permit), local] (replica::database& db) {
|
||||
auto trace_state = gt.get();
|
||||
auto p = local ? std::move(permit) : /* FIXME: either obtain a real permit on this shard or hold original one across shard */ empty_service_permit();
|
||||
return db.apply_counter_update(gs, fm, timeout, trace_state).then([cl, timeout, trace_state, p = std::move(p)] (mutation m) mutable {
|
||||
@@ -1960,7 +1960,7 @@ storage_proxy::response_id_type
|
||||
storage_proxy::create_write_response_handler_helper(schema_ptr s, const dht::token& token, std::unique_ptr<mutation_holder> mh,
|
||||
db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
|
||||
auto keyspace_name = s->ks_name();
|
||||
keyspace& ks = _db.local().find_keyspace(keyspace_name);
|
||||
replica::keyspace& ks = _db.local().find_keyspace(keyspace_name);
|
||||
auto erm = ks.get_effective_replication_map();
|
||||
inet_address_vector_replica_set natural_endpoints = erm->get_natural_endpoints_without_node_being_replaced(token);
|
||||
inet_address_vector_topology_change pending_endpoints = erm->get_token_metadata_ptr()->pending_endpoints_for(token, keyspace_name);
|
||||
@@ -2044,7 +2044,7 @@ storage_proxy::create_write_response_handler(const std::unordered_map<gms::inet_
|
||||
tracing::trace(tr_state, "Creating write handler for read repair token: {} endpoint: {}", mh->token(), endpoints);
|
||||
|
||||
auto keyspace_name = mh->schema()->ks_name();
|
||||
keyspace& ks = _db.local().find_keyspace(keyspace_name);
|
||||
replica::keyspace& ks = _db.local().find_keyspace(keyspace_name);
|
||||
|
||||
return create_write_response_handler(ks, cl, type, std::move(mh), std::move(endpoints), inet_address_vector_topology_change(), inet_address_vector_topology_change(), std::move(tr_state), get_stats(), std::move(permit));
|
||||
}
|
||||
@@ -2067,7 +2067,7 @@ storage_proxy::create_write_response_handler(const std::tuple<lw_shared_ptr<paxo
|
||||
tracing::trace(tr_state, "Creating write handler for paxos repair token: {} endpoint: {}", token, endpoints);
|
||||
|
||||
auto keyspace_name = s->ks_name();
|
||||
keyspace& ks = _db.local().find_keyspace(keyspace_name);
|
||||
replica::keyspace& ks = _db.local().find_keyspace(keyspace_name);
|
||||
|
||||
return create_write_response_handler(ks, cl, db::write_type::CAS, std::make_unique<cas_mutation>(std::move(commit), s, nullptr), std::move(endpoints),
|
||||
inet_address_vector_topology_change(), inet_address_vector_topology_change(), std::move(tr_state), get_stats(), std::move(permit));
|
||||
@@ -2165,7 +2165,7 @@ future<> storage_proxy::mutate_end(future<> mutate_result, utils::latency_counte
|
||||
mutate_result.get();
|
||||
tracing::trace(trace_state, "Mutation successfully completed");
|
||||
return make_ready_future<>();
|
||||
} catch (no_such_keyspace& ex) {
|
||||
} catch (replica::no_such_keyspace& ex) {
|
||||
tracing::trace(trace_state, "Mutation failed: write to non existing keyspace: {}", ex.what());
|
||||
slogger.trace("Write to non existing keyspace: {}", ex.what());
|
||||
return make_exception_future<>(std::current_exception());
|
||||
@@ -2285,7 +2285,7 @@ future<> storage_proxy::mutate_counters(Range&& mutations, db::consistency_level
|
||||
|
||||
storage_proxy::paxos_participants
|
||||
storage_proxy::get_paxos_participants(const sstring& ks_name, const dht::token &token, db::consistency_level cl_for_paxos) {
|
||||
keyspace& ks = _db.local().find_keyspace(ks_name);
|
||||
replica::keyspace& ks = _db.local().find_keyspace(ks_name);
|
||||
auto erm = ks.get_effective_replication_map();
|
||||
inet_address_vector_replica_set natural_endpoints = erm->get_natural_endpoints_without_node_being_replaced(token);
|
||||
inet_address_vector_topology_change pending_endpoints = erm->get_token_metadata_ptr()->pending_endpoints_for(token, ks_name);
|
||||
@@ -3507,7 +3507,7 @@ protected:
|
||||
inet_address_vector_replica_set _used_targets;
|
||||
promise<foreign_ptr<lw_shared_ptr<query::result>>> _result_promise;
|
||||
tracing::trace_state_ptr _trace_state;
|
||||
lw_shared_ptr<column_family> _cf;
|
||||
lw_shared_ptr<replica::column_family> _cf;
|
||||
bool _foreground = true;
|
||||
service_permit _permit; // holds admission permit until operation completes
|
||||
|
||||
@@ -3518,7 +3518,7 @@ private:
|
||||
_foreground = false;
|
||||
}
|
||||
public:
|
||||
abstract_read_executor(schema_ptr s, lw_shared_ptr<column_family> cf, shared_ptr<storage_proxy> proxy, lw_shared_ptr<query::read_command> cmd, dht::partition_range pr, db::consistency_level cl, size_t block_for,
|
||||
abstract_read_executor(schema_ptr s, lw_shared_ptr<replica::column_family> cf, shared_ptr<storage_proxy> proxy, lw_shared_ptr<query::read_command> cmd, dht::partition_range pr, db::consistency_level cl, size_t block_for,
|
||||
inet_address_vector_replica_set targets, tracing::trace_state_ptr trace_state, service_permit permit) :
|
||||
_schema(std::move(s)), _proxy(std::move(proxy)), _cmd(std::move(cmd)), _partition_range(std::move(pr)), _cl(cl), _block_for(block_for), _targets(std::move(targets)), _trace_state(std::move(trace_state)),
|
||||
_cf(std::move(cf)), _permit(std::move(permit)) {
|
||||
@@ -3820,7 +3820,7 @@ public:
|
||||
return _result_promise.get_future();
|
||||
}
|
||||
|
||||
lw_shared_ptr<column_family>& get_cf() {
|
||||
lw_shared_ptr<replica::column_family>& get_cf() {
|
||||
return _cf;
|
||||
}
|
||||
|
||||
@@ -3844,7 +3844,7 @@ private:
|
||||
|
||||
class never_speculating_read_executor : public abstract_read_executor {
|
||||
public:
|
||||
never_speculating_read_executor(schema_ptr s, lw_shared_ptr<column_family> cf, shared_ptr<storage_proxy> proxy, lw_shared_ptr<query::read_command> cmd, dht::partition_range pr, db::consistency_level cl, inet_address_vector_replica_set targets, tracing::trace_state_ptr trace_state,
|
||||
never_speculating_read_executor(schema_ptr s, lw_shared_ptr<replica::column_family> cf, shared_ptr<storage_proxy> proxy, lw_shared_ptr<query::read_command> cmd, dht::partition_range pr, db::consistency_level cl, inet_address_vector_replica_set targets, tracing::trace_state_ptr trace_state,
|
||||
service_permit permit) :
|
||||
abstract_read_executor(std::move(s), std::move(cf), std::move(proxy), std::move(cmd), std::move(pr), cl, 0, std::move(targets), std::move(trace_state), std::move(permit)) {
|
||||
_block_for = _targets.size();
|
||||
@@ -3943,7 +3943,7 @@ db::read_repair_decision storage_proxy::new_read_repair_decision(const schema& s
|
||||
bool& is_read_non_local,
|
||||
service_permit permit) {
|
||||
const dht::token& token = pr.start()->value().token();
|
||||
keyspace& ks = _db.local().find_keyspace(schema->ks_name());
|
||||
replica::keyspace& ks = _db.local().find_keyspace(schema->ks_name());
|
||||
speculative_retry::type retry_type = schema->speculative_retry().get_type();
|
||||
gms::inet_address extra_replica;
|
||||
|
||||
@@ -4023,7 +4023,7 @@ storage_proxy::query_result_local(schema_ptr s, lw_shared_ptr<query::read_comman
|
||||
if (pr.is_singular()) {
|
||||
unsigned shard = dht::shard_of(*s, pr.start()->value().token());
|
||||
get_stats().replica_cross_shard_ops += shard != this_shard_id();
|
||||
return _db.invoke_on(shard, _read_smp_service_group, [gs = global_schema_ptr(s), prv = dht::partition_range_vector({pr}) /* FIXME: pr is copied */, cmd, opts, timeout, gt = tracing::global_trace_state_ptr(std::move(trace_state))] (database& db) mutable {
|
||||
return _db.invoke_on(shard, _read_smp_service_group, [gs = global_schema_ptr(s), prv = dht::partition_range_vector({pr}) /* FIXME: pr is copied */, cmd, opts, timeout, gt = tracing::global_trace_state_ptr(std::move(trace_state))] (replica::database& db) mutable {
|
||||
auto trace_state = gt.get();
|
||||
tracing::trace(trace_state, "Start querying singular range {}", prv.front());
|
||||
return db.query(gs, *cmd, opts, prv, trace_state, timeout).then([trace_state](std::tuple<lw_shared_ptr<query::result>, cache_temperature>&& f_ht) {
|
||||
@@ -4150,7 +4150,7 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
|
||||
replicas_per_token_range preferred_replicas,
|
||||
service_permit permit) {
|
||||
schema_ptr schema = local_schema_registry().get(cmd->schema_version);
|
||||
keyspace& ks = _db.local().find_keyspace(schema->ks_name());
|
||||
replica::keyspace& ks = _db.local().find_keyspace(schema->ks_name());
|
||||
std::vector<::shared_ptr<abstract_read_executor>> exec;
|
||||
auto p = shared_from_this();
|
||||
auto& cf= _db.local().find_column_family(schema);
|
||||
@@ -4247,7 +4247,7 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
|
||||
// check that merged set hit rate is not to low
|
||||
auto find_min = [pcf] (const inet_address_vector_replica_set& range) {
|
||||
struct {
|
||||
column_family* cf = nullptr;
|
||||
replica::column_family* cf = nullptr;
|
||||
float operator()(const gms::inet_address& ep) const {
|
||||
return float(cf->get_hit_rate(ep).rate);
|
||||
}
|
||||
@@ -4343,7 +4343,7 @@ storage_proxy::query_partition_key_range(lw_shared_ptr<query::read_command> cmd,
|
||||
db::consistency_level cl,
|
||||
storage_proxy::coordinator_query_options query_options) {
|
||||
schema_ptr schema = local_schema_registry().get(cmd->schema_version);
|
||||
keyspace& ks = _db.local().find_keyspace(schema->ks_name());
|
||||
replica::keyspace& ks = _db.local().find_keyspace(schema->ks_name());
|
||||
|
||||
// when dealing with LocalStrategy keyspaces, we can skip the range splitting and merging (which can be
|
||||
// expensive in clusters with vnodes)
|
||||
@@ -4465,7 +4465,7 @@ storage_proxy::do_query(schema_ptr s,
|
||||
p->get_stats().estimated_read.add(lc.latency());
|
||||
}
|
||||
});
|
||||
} catch (const no_such_column_family&) {
|
||||
} catch (const replica::no_such_column_family&) {
|
||||
get_stats().read.mark(lc.stop().latency());
|
||||
return make_empty();
|
||||
}
|
||||
@@ -4749,7 +4749,7 @@ future<bool> storage_proxy::cas(schema_ptr schema, shared_ptr<cas_request> reque
|
||||
co_return condition_met;
|
||||
}
|
||||
|
||||
inet_address_vector_replica_set storage_proxy::get_live_endpoints(keyspace& ks, const dht::token& token) const {
|
||||
inet_address_vector_replica_set storage_proxy::get_live_endpoints(replica::keyspace& ks, const dht::token& token) const {
|
||||
auto erm = ks.get_effective_replication_map();
|
||||
inet_address_vector_replica_set eps = erm->get_natural_endpoints_without_node_being_replaced(token);
|
||||
auto itend = boost::range::remove_if(eps, std::not1(std::bind1st(std::mem_fn(&gms::gossiper::is_alive), &_gossiper)));
|
||||
@@ -4766,7 +4766,7 @@ void storage_proxy::sort_endpoints_by_proximity(inet_address_vector_replica_set&
|
||||
}
|
||||
}
|
||||
|
||||
inet_address_vector_replica_set storage_proxy::get_live_sorted_endpoints(keyspace& ks, const dht::token& token) const {
|
||||
inet_address_vector_replica_set storage_proxy::get_live_sorted_endpoints(replica::keyspace& ks, const dht::token& token) const {
|
||||
auto eps = get_live_endpoints(ks, token);
|
||||
sort_endpoints_by_proximity(eps);
|
||||
return eps;
|
||||
@@ -5329,7 +5329,7 @@ storage_proxy::query_mutations_locally(schema_ptr s, lw_shared_ptr<query::read_c
|
||||
if (pr.is_singular()) {
|
||||
unsigned shard = dht::shard_of(*s, pr.start()->value().token());
|
||||
get_stats().replica_cross_shard_ops += shard != this_shard_id();
|
||||
return _db.invoke_on(shard, _read_smp_service_group, [cmd, &pr, gs=global_schema_ptr(s), timeout, gt = tracing::global_trace_state_ptr(std::move(trace_state))] (database& db) mutable {
|
||||
return _db.invoke_on(shard, _read_smp_service_group, [cmd, &pr, gs=global_schema_ptr(s), timeout, gt = tracing::global_trace_state_ptr(std::move(trace_state))] (replica::database& db) mutable {
|
||||
return db.query_mutations(gs, *cmd, pr, gt, timeout).then([] (std::tuple<reconcilable_result, cache_temperature> result_ht) {
|
||||
auto&& [result, ht] = result_ht;
|
||||
return make_ready_future<rpc::tuple<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>>(rpc::tuple(make_foreign(make_lw_shared<reconcilable_result>(std::move(result))), ht));
|
||||
|
||||
@@ -265,7 +265,7 @@ public:
|
||||
query::max_result_size get_max_result_size(const query::partition_slice& slice) const;
|
||||
|
||||
private:
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
gms::gossiper& _gossiper;
|
||||
const locator::shared_token_metadata& _shared_token_metadata;
|
||||
locator::effective_replication_map_factory& _erm_factory;
|
||||
@@ -348,7 +348,7 @@ private:
|
||||
response_id_type create_write_response_handler_helper(schema_ptr s, const dht::token& token,
|
||||
std::unique_ptr<mutation_holder> mh, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state,
|
||||
service_permit permit);
|
||||
response_id_type create_write_response_handler(keyspace& ks, db::consistency_level cl, db::write_type type, std::unique_ptr<mutation_holder> m, inet_address_vector_replica_set targets,
|
||||
response_id_type create_write_response_handler(replica::keyspace& ks, db::consistency_level cl, db::write_type type, std::unique_ptr<mutation_holder> m, inet_address_vector_replica_set targets,
|
||||
const inet_address_vector_topology_change& pending_endpoints, inet_address_vector_topology_change, tracing::trace_state_ptr tr_state, storage_proxy::write_stats& stats, service_permit permit);
|
||||
response_id_type create_write_response_handler(const mutation&, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit);
|
||||
response_id_type create_write_response_handler(const hint_wrapper&, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit);
|
||||
@@ -366,9 +366,9 @@ private:
|
||||
bool cannot_hint(const Range& targets, db::write_type type) const;
|
||||
bool hints_enabled(db::write_type type) const noexcept;
|
||||
db::hints::manager& hints_manager_for(db::write_type type);
|
||||
inet_address_vector_replica_set get_live_endpoints(keyspace& ks, const dht::token& token) const;
|
||||
inet_address_vector_replica_set get_live_endpoints(replica::keyspace& ks, const dht::token& token) const;
|
||||
static void sort_endpoints_by_proximity(inet_address_vector_replica_set& eps);
|
||||
inet_address_vector_replica_set get_live_sorted_endpoints(keyspace& ks, const dht::token& token) const;
|
||||
inet_address_vector_replica_set get_live_sorted_endpoints(replica::keyspace& ks, const dht::token& token) const;
|
||||
db::read_repair_decision new_read_repair_decision(const schema& s);
|
||||
::shared_ptr<abstract_read_executor> get_read_executor(lw_shared_ptr<query::read_command> cmd,
|
||||
schema_ptr schema,
|
||||
@@ -463,22 +463,22 @@ private:
|
||||
void retire_view_response_handlers(noncopyable_function<bool(const abstract_write_response_handler&)> filter_fun);
|
||||
void connection_dropped(gms::inet_address);
|
||||
public:
|
||||
storage_proxy(distributed<database>& db, gms::gossiper& gossiper, config cfg, db::view::node_update_backlog& max_view_update_backlog,
|
||||
storage_proxy(distributed<replica::database>& db, gms::gossiper& gossiper, config cfg, db::view::node_update_backlog& max_view_update_backlog,
|
||||
scheduling_group_key stats_key, gms::feature_service& feat, const locator::shared_token_metadata& stm, locator::effective_replication_map_factory& erm_factory, netw::messaging_service& ms);
|
||||
~storage_proxy();
|
||||
const distributed<database>& get_db() const {
|
||||
const distributed<replica::database>& get_db() const {
|
||||
return _db;
|
||||
}
|
||||
distributed<database>& get_db() {
|
||||
distributed<replica::database>& get_db() {
|
||||
return _db;
|
||||
}
|
||||
const database& local_db() const noexcept {
|
||||
const replica::database& local_db() const noexcept {
|
||||
return _db.local();
|
||||
}
|
||||
|
||||
const data_dictionary::database data_dictionary() const;
|
||||
|
||||
database& local_db() noexcept {
|
||||
replica::database& local_db() noexcept {
|
||||
return _db.local();
|
||||
}
|
||||
|
||||
|
||||
@@ -107,7 +107,7 @@ namespace service {
|
||||
static logging::logger slogger("storage_service");
|
||||
|
||||
storage_service::storage_service(abort_source& abort_source,
|
||||
distributed<database>& db, gms::gossiper& gossiper,
|
||||
distributed<replica::database>& db, gms::gossiper& gossiper,
|
||||
sharded<db::system_distributed_keyspace>& sys_dist_ks,
|
||||
gms::feature_service& feature_service,
|
||||
storage_service_config config,
|
||||
@@ -594,7 +594,7 @@ void storage_service::join_token_ring(int delay) {
|
||||
}
|
||||
|
||||
void storage_service::mark_existing_views_as_built() {
|
||||
_db.invoke_on(0, [this] (database& db) {
|
||||
_db.invoke_on(0, [this] (replica::database& db) {
|
||||
return do_with(db.get_views(), [this] (std::vector<view_ptr>& views) {
|
||||
return parallel_for_each(views, [this] (view_ptr& view) {
|
||||
return db::system_keyspace::mark_view_as_built(view->ks_name(), view->cf_name()).then([this, view] {
|
||||
@@ -700,7 +700,7 @@ void storage_service::bootstrap() {
|
||||
}
|
||||
}
|
||||
|
||||
_db.invoke_on_all([this] (database& db) {
|
||||
_db.invoke_on_all([this] (replica::database& db) {
|
||||
for (auto& cf : db.get_non_system_column_families()) {
|
||||
cf->notify_bootstrap_or_replace_start();
|
||||
}
|
||||
@@ -717,7 +717,7 @@ void storage_service::bootstrap() {
|
||||
bs.bootstrap(streaming::stream_reason::bootstrap, _gossiper).get();
|
||||
}
|
||||
}
|
||||
_db.invoke_on_all([this] (database& db) {
|
||||
_db.invoke_on_all([this] (replica::database& db) {
|
||||
for (auto& cf : db.get_non_system_column_families()) {
|
||||
cf->notify_bootstrap_or_replace_end();
|
||||
}
|
||||
@@ -1621,7 +1621,7 @@ future<std::map<gms::inet_address, float>> storage_service::effective_ownership(
|
||||
return run_with_no_api_lock([keyspace_name] (storage_service& ss) mutable -> future<std::map<gms::inet_address, float>> {
|
||||
if (keyspace_name != "") {
|
||||
//find throws no such keyspace if it is missing
|
||||
const keyspace& ks = ss._db.local().find_keyspace(keyspace_name);
|
||||
const replica::keyspace& ks = ss._db.local().find_keyspace(keyspace_name);
|
||||
// This is ugly, but it follows origin
|
||||
auto&& rs = ks.get_replication_strategy(); // clang complains about typeid(ks.get_replication_strategy());
|
||||
if (typeid(rs) == typeid(locator::local_strategy)) {
|
||||
@@ -1674,7 +1674,7 @@ future<std::map<gms::inet_address, float>> storage_service::effective_ownership(
|
||||
}
|
||||
}
|
||||
final_ownership[endpoint] = ownership;
|
||||
} catch (no_such_keyspace&) {
|
||||
} catch (replica::no_such_keyspace&) {
|
||||
// In case ss.get_ranges_for_endpoint(keyspace_name, endpoint) is not found, just mark it as zero and continue
|
||||
final_ownership[endpoint] = 0;
|
||||
}
|
||||
@@ -2402,7 +2402,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
|
||||
return resp;
|
||||
} else if (req.cmd == node_ops_cmd::repair_updater) {
|
||||
slogger.debug("repair[{}]: Got repair_updater request from {}", ops_uuid, coordinator);
|
||||
_db.invoke_on_all([coordinator, ops_uuid, tables = req.repair_tables] (database &db) {
|
||||
_db.invoke_on_all([coordinator, ops_uuid, tables = req.repair_tables] (replica::database &db) {
|
||||
for (const auto& table_id : tables) {
|
||||
auto& table = db.find_column_family(table_id);
|
||||
table.update_off_strategy_trigger();
|
||||
@@ -2496,7 +2496,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
|
||||
} else if (req.cmd == node_ops_cmd::decommission_done) {
|
||||
slogger.info("decommission[{}]: Marked ops done from coordinator={}", req.ops_uuid, coordinator);
|
||||
slogger.debug("Triggering off-strategy compaction for all non-system tables on decommission completion");
|
||||
_db.invoke_on_all([](database &db) {
|
||||
_db.invoke_on_all([](replica::database &db) {
|
||||
for (auto& table : db.get_non_system_column_families()) {
|
||||
table->trigger_offstrategy_compaction();
|
||||
}
|
||||
@@ -2636,7 +2636,7 @@ future<> storage_service::do_drain() {
|
||||
_migration_manager.invoke_on_all(&service::migration_manager::drain).get();
|
||||
|
||||
set_mode(mode::DRAINING, "flushing column families", false);
|
||||
_db.invoke_on_all(&database::drain).get();
|
||||
_db.invoke_on_all(&replica::database::drain).get();
|
||||
});
|
||||
}
|
||||
|
||||
@@ -3381,7 +3381,7 @@ future<> storage_service::force_remove_completion() {
|
||||
* Takes an ordered list of adjacent tokens and divides them in the specified number of ranges.
|
||||
*/
|
||||
static std::vector<std::pair<dht::token_range, uint64_t>>
|
||||
calculate_splits(std::vector<dht::token> tokens, uint64_t split_count, column_family& cf) {
|
||||
calculate_splits(std::vector<dht::token> tokens, uint64_t split_count, replica::column_family& cf) {
|
||||
auto sstables = cf.get_sstables();
|
||||
const double step = static_cast<double>(tokens.size() - 1) / split_count;
|
||||
auto prev_token_idx = 0;
|
||||
|
||||
@@ -171,7 +171,7 @@ private:
|
||||
|
||||
abort_source& _abort_source;
|
||||
gms::feature_service& _feature_service;
|
||||
distributed<database>& _db;
|
||||
distributed<replica::database>& _db;
|
||||
gms::gossiper& _gossiper;
|
||||
// Container for all Raft instances running on this shard.
|
||||
raft_group_registry& _raft_gr;
|
||||
@@ -198,7 +198,7 @@ private:
|
||||
void node_ops_singal_abort(std::optional<utils::UUID> ops_uuid);
|
||||
future<> node_ops_abort_thread();
|
||||
public:
|
||||
storage_service(abort_source& as, distributed<database>& db,
|
||||
storage_service(abort_source& as, distributed<replica::database>& db,
|
||||
gms::gossiper& gossiper,
|
||||
sharded<db::system_distributed_keyspace>&,
|
||||
gms::feature_service& feature_service,
|
||||
|
||||
@@ -335,7 +335,7 @@ sstable_directory::collect_output_sstables_from_reshaping(std::vector<sstables::
|
||||
});
|
||||
}
|
||||
|
||||
future<uint64_t> sstable_directory::reshape(compaction_manager& cm, table& table, sstables::compaction_sstable_creator_fn creator, const ::io_priority_class& iop, sstables::reshape_mode mode)
|
||||
future<uint64_t> sstable_directory::reshape(compaction_manager& cm, replica::table& table, sstables::compaction_sstable_creator_fn creator, const ::io_priority_class& iop, sstables::reshape_mode mode)
|
||||
{
|
||||
return do_with(uint64_t(0), [this, &cm, &table, creator = std::move(creator), iop, mode] (uint64_t & reshaped_size) mutable {
|
||||
return repeat([this, &cm, &table, creator = std::move(creator), iop, &reshaped_size, mode] () mutable {
|
||||
@@ -378,7 +378,7 @@ future<uint64_t> sstable_directory::reshape(compaction_manager& cm, table& table
|
||||
}
|
||||
|
||||
future<>
|
||||
sstable_directory::reshard(sstable_info_vector shared_info, compaction_manager& cm, table& table,
|
||||
sstable_directory::reshard(sstable_info_vector shared_info, compaction_manager& cm, replica::table& table,
|
||||
unsigned max_sstables_per_job, sstables::compaction_sstable_creator_fn creator, const ::io_priority_class& iop)
|
||||
{
|
||||
// Resharding doesn't like empty sstable sets, so bail early. There is nothing
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user