Merge "Make inactive shard readers evictable" from Botond
" This series attempts to solve the regressions recently discovered in performance of multi-partition range-scans. Namely that they: * Flood the reader concurrency semaphore's queues, trampling other reads. * Behave very badly when too many of them is running concurrently (trashing). * May deadlock if enough of them is running without a timeout. The solution for these problems is to make inactive shard readers evictable. This should address all three issues listed above, to varying degrees: * Shard readers will now not cling onto their permits for the entire duration of the scan, which might be a lot of time. * Will be less affected by infinite concurrency (more than the node can handle) as each scan now can make progress by evicting inactive shard readers belonging to other scans. * Will not deadlock at all. In addition to the above fix, this series also bundles two further improvements: * Add a mechanism to `reader_concurrecy_semaphore` to be notified of newly inserted evictables. * General cleanups and fixes for `multishard_combining_reader` and `foreign_reader`. I can unbundle these mini series and send them separately, if the maintainers so prefer, altough considering that this series will have to be backported to 3.0, I think this present form is better. Fixes: #3835 " * 'evictable-inactive-shard-readers/v7' of https://github.com/denesb/scylla: (27 commits) tests/multishard_mutation_query_test: test stateless query too tests/querier_cache: fail resource-based eviction test gracefully tests/querier_cache: simplify resource-based eviction test tests/mutation_reader_test: add test_multishard_combining_reader_next_partition tests/mutation_reader_test: restore indentation tests/mutation_reader_test: enrich pause-related multishard reader test multishard_combining_reader: use pause-resume API query::partition_slice: add clear_ranges() method position_in_partition: add region() accessor foreign_reader: add pause-resume API tests/mutation_reader_test: implement the pause-resume API query_mutations_on_all_shards(): implement pause-resume API make_multishard_streaming_reader(): implement the pause-resume API database: add accessors for user and streaming concurrency semaphores reader_lifecycle_policy: extend with a pause-resume API query_mutations_on_all_shards(): restore indentation query_mutations_on_all_shards(): simplify the state-machine multishard_combining_reader: use the reader lifecycle policy multishard_combining_reader: add reader lifecycle policy multishard_combining_reader: drop unnecessary `reader_promise` member ...
This commit is contained in:
@@ -692,6 +692,7 @@ scylla_core = (['database.cc',
|
||||
'data/cell.cc',
|
||||
'multishard_writer.cc',
|
||||
'multishard_mutation_query.cc',
|
||||
'reader_concurrency_semaphore.cc',
|
||||
] + [Antlr3Grammar('cql3/Cql.g')] + [Thrift('interface/cassandra.thrift', 'Cassandra')]
|
||||
)
|
||||
|
||||
|
||||
98
database.cc
98
database.cc
@@ -2212,9 +2212,6 @@ database::database(const db::config& cfg, database_config dbcfg)
|
||||
[this] {
|
||||
++_stats->sstable_read_queue_overloaded;
|
||||
return std::make_exception_ptr(std::runtime_error("sstable inactive read queue overloaded"));
|
||||
},
|
||||
[this] {
|
||||
return _querier_cache.evict_one();
|
||||
})
|
||||
// No timeouts or queue length limits - a failure here can kill an entire repair.
|
||||
// Trust the caller to limit concurrency.
|
||||
@@ -2226,7 +2223,7 @@ database::database(const db::config& cfg, database_config dbcfg)
|
||||
, _version(empty_version)
|
||||
, _compaction_manager(make_compaction_manager(*_cfg, dbcfg))
|
||||
, _enable_incremental_backups(cfg.incremental_backups())
|
||||
, _querier_cache(dbcfg.available_memory * 0.04)
|
||||
, _querier_cache(_read_concurrency_sem, dbcfg.available_memory * 0.04)
|
||||
, _large_partition_handler(std::make_unique<db::cql_table_large_partition_handler>(_cfg->compaction_large_partition_warning_threshold_mb()*1024*1024))
|
||||
, _result_memory_limiter(dbcfg.available_memory / 10)
|
||||
{
|
||||
@@ -2478,6 +2475,9 @@ database::setup_metrics() {
|
||||
}
|
||||
|
||||
database::~database() {
|
||||
_read_concurrency_sem.clear_inactive_reads();
|
||||
_streaming_concurrency_sem.clear_inactive_reads();
|
||||
_system_read_concurrency_sem.clear_inactive_reads();
|
||||
}
|
||||
|
||||
void database::update_version(const utils::UUID& version) {
|
||||
@@ -4704,31 +4704,87 @@ flat_mutation_reader make_range_sstable_reader(schema_ptr s,
|
||||
fwd_mr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
using foreign_unique_ptr = foreign_ptr<std::unique_ptr<T>>;
|
||||
|
||||
flat_mutation_reader make_multishard_streaming_reader(distributed<database>& db, dht::i_partitioner& partitioner, schema_ptr schema,
|
||||
std::function<std::optional<dht::partition_range>()> range_generator) {
|
||||
class streaming_reader_lifecycle_policy
|
||||
: public reader_lifecycle_policy
|
||||
, public enable_shared_from_this<streaming_reader_lifecycle_policy> {
|
||||
struct inactive_read : public reader_concurrency_semaphore::inactive_read {
|
||||
foreign_unique_ptr<flat_mutation_reader> reader;
|
||||
explicit inactive_read(foreign_unique_ptr<flat_mutation_reader> reader)
|
||||
: reader(std::move(reader)) {
|
||||
}
|
||||
virtual void evict() override {
|
||||
reader.reset();
|
||||
}
|
||||
};
|
||||
struct reader_context {
|
||||
std::unique_ptr<const dht::partition_range> range;
|
||||
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
|
||||
std::optional<reader_concurrency_semaphore::inactive_read_handle> pause_handle_opt;
|
||||
};
|
||||
distributed<database>& _db;
|
||||
std::vector<reader_context> _contexts;
|
||||
public:
|
||||
explicit streaming_reader_lifecycle_policy(distributed<database>& db) : _db(db), _contexts(smp::count) {
|
||||
}
|
||||
virtual future<foreign_unique_ptr<flat_mutation_reader>> create_reader(
|
||||
shard_id shard,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice&,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr,
|
||||
mutation_reader::forwarding fwd_mr) override {
|
||||
_contexts[shard].range = std::make_unique<dht::partition_range>(range);
|
||||
return _db.invoke_on(shard, [gs = global_schema_ptr(std::move(schema)), range = _contexts[shard].range.get(), fwd_mr] (database& db) {
|
||||
auto schema = gs.get();
|
||||
auto& cf = db.find_column_family(schema);
|
||||
return make_ready_future<foreign_unique_ptr<utils::phased_barrier::operation>, foreign_unique_ptr<flat_mutation_reader>>(
|
||||
make_foreign(std::make_unique<utils::phased_barrier::operation>(cf.read_in_progress())),
|
||||
make_foreign(std::make_unique<flat_mutation_reader>(cf.make_streaming_reader(std::move(schema), *range, fwd_mr))));
|
||||
}).then([this, zis = shared_from_this(), shard] (foreign_unique_ptr<utils::phased_barrier::operation> read_operation,
|
||||
foreign_unique_ptr<flat_mutation_reader> reader) {
|
||||
_contexts[shard].read_operation = std::move(read_operation);
|
||||
return std::move(reader);
|
||||
});
|
||||
}
|
||||
virtual void destroy_reader(shard_id shard, future<paused_or_stopped_reader> reader_fut) noexcept override {
|
||||
reader_fut.then([this, zis = shared_from_this(), shard] (paused_or_stopped_reader&& reader) mutable {
|
||||
return smp::submit_to(shard, [ctx = std::move(_contexts[shard]), reader = std::move(reader.remote_reader)] () mutable {
|
||||
reader.release();
|
||||
});
|
||||
});
|
||||
}
|
||||
virtual future<> pause(foreign_unique_ptr<flat_mutation_reader> reader) override {
|
||||
const auto shard = reader.get_owner_shard();
|
||||
return _db.invoke_on(shard, [reader = std::move(reader)] (database& db) mutable {
|
||||
return db.streaming_read_concurrency_sem().register_inactive_read(std::make_unique<inactive_read>(std::move(reader)));
|
||||
}).then([this, zis = shared_from_this(), shard] (reader_concurrency_semaphore::inactive_read_handle handle) {
|
||||
_contexts[shard].pause_handle_opt = handle;
|
||||
});
|
||||
}
|
||||
virtual future<foreign_unique_ptr<flat_mutation_reader>> try_resume(shard_id shard) override {
|
||||
return _db.invoke_on(shard, [handle = *_contexts[shard].pause_handle_opt] (database& db) mutable {
|
||||
if (auto ir_ptr = db.streaming_read_concurrency_sem().unregister_inactive_read(handle)) {
|
||||
return std::move(static_cast<inactive_read&>(*ir_ptr).reader);
|
||||
}
|
||||
return foreign_unique_ptr<flat_mutation_reader>{};
|
||||
});
|
||||
}
|
||||
};
|
||||
auto ms = mutation_source([&db, &partitioner] (schema_ptr s,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
streamed_mutation::forwarding,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
auto factory = [&db] (unsigned shard,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice&,
|
||||
const io_priority_class&,
|
||||
tracing::trace_state_ptr,
|
||||
streamed_mutation::forwarding,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return db.invoke_on(shard, [gs = global_schema_ptr(std::move(schema)), &range, fwd_mr] (database& db) {
|
||||
auto schema = gs.get();
|
||||
auto& cf = db.find_column_family(schema);
|
||||
return make_foreign(std::make_unique<flat_mutation_reader>(cf.make_streaming_reader(std::move(schema), range, fwd_mr)));
|
||||
});
|
||||
};
|
||||
return make_multishard_combining_reader(std::move(s), pr, ps, pc, partitioner, std::move(factory), std::move(trace_state),
|
||||
fwd_sm, fwd_mr);
|
||||
return make_multishard_combining_reader(make_shared<streaming_reader_lifecycle_policy>(db), partitioner, std::move(s), pr, ps, pc,
|
||||
std::move(trace_state), fwd_mr);
|
||||
});
|
||||
return make_flat_multi_range_reader(std::move(schema), std::move(ms), std::move(range_generator), schema->full_slice(),
|
||||
service::get_local_streaming_read_priority(), {}, mutation_reader::forwarding::no);
|
||||
|
||||
@@ -1428,6 +1428,12 @@ public:
|
||||
std::unordered_set<sstring> get_initial_tokens();
|
||||
std::experimental::optional<gms::inet_address> get_replace_address();
|
||||
bool is_replacing();
|
||||
reader_concurrency_semaphore& user_read_concurrency_sem() {
|
||||
return _read_concurrency_sem;
|
||||
}
|
||||
reader_concurrency_semaphore& streaming_read_concurrency_sem() {
|
||||
return _streaming_concurrency_sem;
|
||||
}
|
||||
reader_concurrency_semaphore& system_keyspace_read_concurrency_sem() {
|
||||
return _system_read_concurrency_sem;
|
||||
}
|
||||
|
||||
@@ -60,7 +60,7 @@ using foreign_unique_ptr = foreign_ptr<std::unique_ptr<T>>;
|
||||
/// 3) Both, `read_context::lookup_readers()` and `read_context::save_readers()`
|
||||
/// knows to do nothing when the query is not stateful and just short
|
||||
/// circuit.
|
||||
class read_context {
|
||||
class read_context : public reader_lifecycle_policy {
|
||||
struct reader_params {
|
||||
std::unique_ptr<const dht::partition_range> range;
|
||||
std::unique_ptr<const query::partition_slice> slice;
|
||||
@@ -80,6 +80,20 @@ class read_context {
|
||||
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
|
||||
foreign_unique_ptr<flat_mutation_reader> reader;
|
||||
};
|
||||
struct paused_reader {
|
||||
shard_id shard;
|
||||
reader_concurrency_semaphore::inactive_read_handle handle;
|
||||
bool has_pending_next_partition;
|
||||
};
|
||||
struct inactive_read : public reader_concurrency_semaphore::inactive_read {
|
||||
foreign_unique_ptr<flat_mutation_reader> reader;
|
||||
explicit inactive_read(foreign_unique_ptr<flat_mutation_reader> reader)
|
||||
: reader(std::move(reader)) {
|
||||
}
|
||||
virtual void evict() override {
|
||||
reader.reset();
|
||||
}
|
||||
};
|
||||
|
||||
using inexistent_state = std::monostate;
|
||||
struct successful_lookup_state {
|
||||
@@ -94,61 +108,64 @@ class read_context {
|
||||
struct dismantling_state {
|
||||
foreign_unique_ptr<reader_params> params;
|
||||
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
|
||||
future<stopped_foreign_reader> reader_fut;
|
||||
std::variant<foreign_unique_ptr<flat_mutation_reader>, paused_reader> reader;
|
||||
circular_buffer<mutation_fragment> buffer;
|
||||
};
|
||||
struct ready_to_save_state {
|
||||
foreign_unique_ptr<reader_params> params;
|
||||
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
|
||||
foreign_unique_ptr<flat_mutation_reader> reader;
|
||||
std::variant<foreign_unique_ptr<flat_mutation_reader>, paused_reader> reader;
|
||||
circular_buffer<mutation_fragment> buffer;
|
||||
};
|
||||
struct future_used_state {
|
||||
future<used_state> fut;
|
||||
struct paused_state {
|
||||
foreign_unique_ptr<reader_params> params;
|
||||
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
|
||||
reader_concurrency_semaphore::inactive_read_handle handle;
|
||||
};
|
||||
struct future_dismantling_state {
|
||||
future<dismantling_state> fut;
|
||||
struct evicted_state {
|
||||
};
|
||||
|
||||
// ( )
|
||||
// ( ) (O)
|
||||
// | ^
|
||||
// | |
|
||||
// +--- inexistent ---+
|
||||
// | |
|
||||
// (1) | (3) | (3)
|
||||
// | | +------ evicted -> (O)
|
||||
// successful_lookup | | ^
|
||||
// | | | | (7) |
|
||||
// | | | +-------+ | (8)
|
||||
// | | (4) | | | |
|
||||
// | +----------> used paused
|
||||
// | | | (6) ^ |
|
||||
// (2) | | +-------+ |
|
||||
// | (5) | | (5)
|
||||
// | | |
|
||||
// | | |
|
||||
// | dismantling <------+
|
||||
// | |
|
||||
// | (2) |
|
||||
// | |
|
||||
// +---------------> ready_to_save
|
||||
// |
|
||||
// +------ inexistent_state -----+
|
||||
// | |
|
||||
// (1) | (6) |
|
||||
// | |
|
||||
// successful_lookup_state future_used_state
|
||||
// | | | |
|
||||
// (2) | (3) | (7) | (8) |
|
||||
// | | | |
|
||||
// | used_state <---------+ future_dismantling_state
|
||||
// | | |
|
||||
// | (4) | (9) |
|
||||
// | | |
|
||||
// | dismantling_state <-----------------+
|
||||
// | |
|
||||
// | (5) |
|
||||
// | |
|
||||
// +----> ready_to_save_state
|
||||
// |
|
||||
// (O)
|
||||
// (O)
|
||||
//
|
||||
// 1) lookup_readers()
|
||||
// 2) save_readers()
|
||||
// 3) make_remote_reader()
|
||||
// 4) dismantle_reader()
|
||||
// 5) prepare_reader_for_saving()
|
||||
// 6) do_make_remote_reader()
|
||||
// 7) reader is created
|
||||
// 8) dismantle_reader()
|
||||
// 9) reader is created
|
||||
// 3) do_make_remote_reader()
|
||||
// 4) make_remote_reader()
|
||||
// 5) dismantle_reader()
|
||||
// 6) pause_reader()
|
||||
// 7) try_resume() - success
|
||||
// 8) try_resume() - failure
|
||||
using reader_state = std::variant<
|
||||
inexistent_state,
|
||||
successful_lookup_state,
|
||||
used_state,
|
||||
paused_state,
|
||||
evicted_state,
|
||||
dismantling_state,
|
||||
ready_to_save_state,
|
||||
future_used_state,
|
||||
future_dismantling_state>;
|
||||
ready_to_save_state>;
|
||||
|
||||
struct dismantle_buffer_stats {
|
||||
size_t partitions = 0;
|
||||
@@ -184,6 +201,8 @@ class read_context {
|
||||
// One for each shard. Index is shard id.
|
||||
std::vector<reader_state> _readers;
|
||||
|
||||
gate _dismantling_gate;
|
||||
|
||||
static future<bundled_remote_reader> do_make_remote_reader(
|
||||
distributed<database>& db,
|
||||
shard_id shard,
|
||||
@@ -200,13 +219,10 @@ class read_context {
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr);
|
||||
|
||||
void dismantle_reader(shard_id shard, future<stopped_foreign_reader>&& stopped_reader_fut);
|
||||
void dismantle_reader(shard_id shard, future<paused_or_stopped_reader>&& reader_fut);
|
||||
|
||||
ready_to_save_state* prepare_reader_for_saving(dismantling_state& current_state, future<stopped_foreign_reader>&& stopped_reader_fut,
|
||||
const dht::decorated_key& last_pkey, const std::optional<clustering_key_prefix>& last_ckey);
|
||||
dismantle_buffer_stats dismantle_combined_buffer(circular_buffer<mutation_fragment> combined_buffer, const dht::decorated_key& pkey);
|
||||
dismantle_buffer_stats dismantle_compaction_state(detached_compaction_state compaction_state);
|
||||
future<> save_reader(ready_to_save_state& current_state, const dht::decorated_key& last_pkey,
|
||||
@@ -229,26 +245,24 @@ public:
|
||||
read_context& operator=(read_context&&) = delete;
|
||||
read_context& operator=(const read_context&) = delete;
|
||||
|
||||
remote_reader_factory factory() {
|
||||
return [this] (
|
||||
shard_id shard,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return make_remote_reader(shard, std::move(schema), pr, ps, pc, std::move(trace_state), fwd_sm, fwd_mr);
|
||||
};
|
||||
virtual future<foreign_unique_ptr<flat_mutation_reader>> create_reader(
|
||||
shard_id shard,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
mutation_reader::forwarding fwd_mr) override {
|
||||
return make_remote_reader(shard, std::move(schema), pr, ps, pc, std::move(trace_state), fwd_mr);
|
||||
}
|
||||
|
||||
foreign_reader_dismantler dismantler() {
|
||||
return [this] (shard_id shard, future<stopped_foreign_reader>&& stopped_reader_fut) {
|
||||
dismantle_reader(shard, std::move(stopped_reader_fut));
|
||||
};
|
||||
virtual void destroy_reader(shard_id shard, future<paused_or_stopped_reader> reader_fut) noexcept override {
|
||||
dismantle_reader(shard, std::move(reader_fut));
|
||||
}
|
||||
|
||||
virtual future<> pause(foreign_unique_ptr<flat_mutation_reader> reader) override;
|
||||
virtual future<foreign_unique_ptr<flat_mutation_reader>> try_resume(shard_id shard) override;
|
||||
|
||||
future<> lookup_readers();
|
||||
|
||||
future<> save_readers(circular_buffer<mutation_fragment> unconsumed_buffer, detached_compaction_state compaction_state,
|
||||
@@ -289,7 +303,6 @@ future<foreign_unique_ptr<flat_mutation_reader>> read_context::make_remote_reade
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding,
|
||||
mutation_reader::forwarding) {
|
||||
auto& rs = _readers[shard];
|
||||
|
||||
@@ -306,100 +319,71 @@ future<foreign_unique_ptr<flat_mutation_reader>> read_context::make_remote_reade
|
||||
return make_ready_future<foreign_unique_ptr<flat_mutation_reader>>(std::move(reader));
|
||||
}
|
||||
|
||||
auto created = promise<used_state>();
|
||||
rs = future_used_state{created.get_future()};
|
||||
return do_make_remote_reader(_db, shard, std::move(schema), pr, ps, pc, std::move(trace_state)).then_wrapped([this, &rs,
|
||||
created = std::move(created)] (future<bundled_remote_reader>&& bundled_reader_fut) mutable {
|
||||
if (bundled_reader_fut.failed()) {
|
||||
auto ex = bundled_reader_fut.get_exception();
|
||||
if (!std::holds_alternative<future_used_state>(rs)) {
|
||||
created.set_exception(ex);
|
||||
}
|
||||
return make_exception_future<foreign_unique_ptr<flat_mutation_reader>>(std::move(ex));
|
||||
}
|
||||
|
||||
auto bundled_reader = bundled_reader_fut.get0();
|
||||
auto new_state = used_state{std::move(bundled_reader.params), std::move(bundled_reader.read_operation)};
|
||||
if (std::holds_alternative<future_used_state>(rs)) {
|
||||
rs = std::move(new_state);
|
||||
} else {
|
||||
created.set_value(std::move(new_state));
|
||||
}
|
||||
return do_make_remote_reader(_db, shard, std::move(schema), pr, ps, pc, std::move(trace_state)).then(
|
||||
[this, &rs] (bundled_remote_reader&& bundled_reader) mutable {
|
||||
rs = used_state{std::move(bundled_reader.params), std::move(bundled_reader.read_operation)};
|
||||
return make_ready_future<foreign_unique_ptr<flat_mutation_reader>>(std::move(bundled_reader.reader));
|
||||
});
|
||||
}
|
||||
|
||||
void read_context::dismantle_reader(shard_id shard, future<stopped_foreign_reader>&& stopped_reader_fut) {
|
||||
auto& rs = _readers[shard];
|
||||
void read_context::dismantle_reader(shard_id shard, future<paused_or_stopped_reader>&& reader_fut) {
|
||||
with_gate(_dismantling_gate, [this, shard, reader_fut = std::move(reader_fut)] () mutable {
|
||||
return reader_fut.then_wrapped([this, shard] (future<paused_or_stopped_reader>&& reader_fut) {
|
||||
if (reader_fut.failed()) {
|
||||
mmq_log.debug("Failed to stop reader on shard {}: {}", shard, reader_fut.get_exception());
|
||||
++_db.local().get_stats().multishard_query_failed_reader_stops;
|
||||
return;
|
||||
}
|
||||
|
||||
if (auto* maybe_used_state = std::get_if<used_state>(&rs)) {
|
||||
auto read_operation = std::move(maybe_used_state->read_operation);
|
||||
auto params = std::move(maybe_used_state->params);
|
||||
rs = dismantling_state{std::move(params), std::move(read_operation), std::move(stopped_reader_fut), circular_buffer<mutation_fragment>{}};
|
||||
} else if (auto* maybe_future_used_state = std::get_if<future_used_state>(&rs)) {
|
||||
auto f = maybe_future_used_state->fut.then([stopped_reader_fut = std::move(stopped_reader_fut)] (used_state&& current_state) mutable {
|
||||
auto read_operation = std::move(current_state.read_operation);
|
||||
auto params = std::move(current_state.params);
|
||||
return dismantling_state{std::move(params), std::move(read_operation), std::move(stopped_reader_fut),
|
||||
circular_buffer<mutation_fragment>{}};
|
||||
auto reader = reader_fut.get0();
|
||||
auto& rs = _readers[shard];
|
||||
if (auto* maybe_used_state = std::get_if<used_state>(&rs)) {
|
||||
auto read_operation = std::move(maybe_used_state->read_operation);
|
||||
auto params = std::move(maybe_used_state->params);
|
||||
rs = dismantling_state{std::move(params), std::move(read_operation), std::move(reader.remote_reader),
|
||||
std::move(reader.unconsumed_fragments)};
|
||||
} else if (auto* maybe_paused_state = std::get_if<paused_state>(&rs)) {
|
||||
auto read_operation = std::move(maybe_paused_state->read_operation);
|
||||
auto params = std::move(maybe_paused_state->params);
|
||||
auto handle = maybe_paused_state->handle;
|
||||
rs = dismantling_state{std::move(params), std::move(read_operation), paused_reader{shard, handle, reader.has_pending_next_partition},
|
||||
std::move(reader.unconsumed_fragments)};
|
||||
// Do nothing for evicted readers.
|
||||
} else if (!std::holds_alternative<evicted_state>(rs)) {
|
||||
mmq_log.warn(
|
||||
"Unexpected request to dismantle reader in state {} for shard {}."
|
||||
" Reader was not created nor is in the process of being created.",
|
||||
rs.index(),
|
||||
shard);
|
||||
}
|
||||
});
|
||||
rs = future_dismantling_state{std::move(f)};
|
||||
} else {
|
||||
mmq_log.warn("Unexpected request to dismantle reader for shard {}. Reader was not created nor is in the process of being created.", shard);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
future<> read_context::stop() {
|
||||
auto cleanup = [db = &_db.local()] (shard_id shard, dismantling_state state) {
|
||||
return state.reader_fut.then_wrapped([db, shard, params = std::move(state.params),
|
||||
read_operation = std::move(state.read_operation)] (future<stopped_foreign_reader>&& fut) mutable {
|
||||
if (fut.failed()) {
|
||||
mmq_log.debug("Failed to stop reader on shard {}: {}", shard, fut.get_exception());
|
||||
++db->get_stats().multishard_query_failed_reader_stops;
|
||||
} else {
|
||||
smp::submit_to(shard, [reader = fut.get0().remote_reader, params = std::move(params),
|
||||
read_operation = std::move(read_operation)] () mutable {
|
||||
reader.release();
|
||||
auto pr = promise<>();
|
||||
auto fut = pr.get_future();
|
||||
auto gate_fut = _dismantling_gate.is_closed() ? make_ready_future<>() : _dismantling_gate.close();
|
||||
gate_fut.then([this] {
|
||||
for (shard_id shard = 0; shard != smp::count; ++shard) {
|
||||
if (auto* maybe_dismantling_state = std::get_if<dismantling_state>(&_readers[shard])) {
|
||||
_db.invoke_on(shard, [reader = std::move(maybe_dismantling_state->reader),
|
||||
params = std::move(maybe_dismantling_state->params),
|
||||
read_operation = std::move(maybe_dismantling_state->read_operation)] (database& db) mutable {
|
||||
if (auto* maybe_stopped_reader = std::get_if<foreign_unique_ptr<flat_mutation_reader>>(&reader)) {
|
||||
maybe_stopped_reader->release();
|
||||
} else {
|
||||
db.user_read_concurrency_sem().unregister_inactive_read(std::get<paused_reader>(reader).handle);
|
||||
}
|
||||
params.release();
|
||||
read_operation.release();
|
||||
});
|
||||
}
|
||||
});
|
||||
};
|
||||
|
||||
std::vector<future<>> futures;
|
||||
auto immediate_cleanup = size_t(0);
|
||||
auto future_cleanup = size_t(0);
|
||||
|
||||
// Wait for pending read-aheads in the background.
|
||||
for (shard_id shard = 0; shard != smp::count; ++shard) {
|
||||
auto& rs = _readers[shard];
|
||||
|
||||
if (auto maybe_dismantling_state = std::get_if<dismantling_state>(&rs)) {
|
||||
++immediate_cleanup;
|
||||
cleanup(shard, std::move(*maybe_dismantling_state));
|
||||
} else if (auto maybe_future_dismantling_state = std::get_if<future_dismantling_state>(&rs)) {
|
||||
++future_cleanup;
|
||||
futures.emplace_back(maybe_future_dismantling_state->fut.then_wrapped([=] (future<dismantling_state>&& current_state_fut) {
|
||||
if (current_state_fut.failed()) {
|
||||
mmq_log.debug("Failed to stop reader on shard {}: {}", shard, current_state_fut.get_exception());
|
||||
++_db.local().get_stats().multishard_query_failed_reader_stops;
|
||||
} else {
|
||||
cleanup(shard, current_state_fut.get0());
|
||||
}
|
||||
}));
|
||||
}
|
||||
}
|
||||
|
||||
if (const auto total = immediate_cleanup + future_cleanup) {
|
||||
tracing::trace(_trace_state,
|
||||
"Stopping {} shard readers, {} ready for immediate cleanup, {} will be cleaned up after finishes read-ahead",
|
||||
total,
|
||||
immediate_cleanup,
|
||||
future_cleanup);
|
||||
}
|
||||
|
||||
return when_all(futures.begin(), futures.end()).discard_result();
|
||||
}).finally([pr = std::move(pr)] () mutable {
|
||||
pr.set_value();
|
||||
});
|
||||
return fut;
|
||||
}
|
||||
|
||||
read_context::dismantle_buffer_stats read_context::dismantle_combined_buffer(circular_buffer<mutation_fragment> combined_buffer,
|
||||
@@ -459,49 +443,35 @@ read_context::dismantle_buffer_stats read_context::dismantle_compaction_state(de
|
||||
return stats;
|
||||
}
|
||||
|
||||
read_context::ready_to_save_state* read_context::prepare_reader_for_saving(
|
||||
dismantling_state& current_state,
|
||||
future<stopped_foreign_reader>&& stopped_reader_fut,
|
||||
const dht::decorated_key& last_pkey,
|
||||
const std::optional<clustering_key_prefix>& last_ckey) {
|
||||
const auto shard = current_state.params.get_owner_shard();
|
||||
auto& rs = _readers[shard];
|
||||
|
||||
if (stopped_reader_fut.failed()) {
|
||||
mmq_log.debug("Failed to stop reader on shard {}: {}", shard, stopped_reader_fut.get_exception());
|
||||
++_db.local().get_stats().multishard_query_failed_reader_stops;
|
||||
// We don't want to leave the reader in dismantling state, lest stop()
|
||||
// will try to wait on the reader_fut again and crash the application.
|
||||
rs = {};
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
auto stopped_reader = stopped_reader_fut.get0();
|
||||
|
||||
// If the buffer is empty just overwrite it.
|
||||
// If it has some data in it append the fragments to the back.
|
||||
// The unconsumed fragments appended here come from the
|
||||
// foreign_reader which is at the lowest layer, hence its
|
||||
// fragments need to be at the back of the buffer.
|
||||
if (current_state.buffer.empty()) {
|
||||
current_state.buffer = std::move(stopped_reader.unconsumed_fragments);
|
||||
} else {
|
||||
std::move(stopped_reader.unconsumed_fragments.begin(), stopped_reader.unconsumed_fragments.end(), std::back_inserter(current_state.buffer));
|
||||
}
|
||||
rs = ready_to_save_state{std::move(current_state.params), std::move(current_state.read_operation), std::move(stopped_reader.remote_reader),
|
||||
std::move(current_state.buffer)};
|
||||
return &std::get<ready_to_save_state>(rs);
|
||||
}
|
||||
|
||||
future<> read_context::save_reader(ready_to_save_state& current_state, const dht::decorated_key& last_pkey,
|
||||
const std::optional<clustering_key_prefix>& last_ckey) {
|
||||
const auto shard = current_state.reader.get_owner_shard();
|
||||
auto* maybe_stopped_reader = std::get_if<foreign_unique_ptr<flat_mutation_reader>>(¤t_state.reader);
|
||||
const auto shard = maybe_stopped_reader
|
||||
? maybe_stopped_reader->get_owner_shard()
|
||||
: std::get<paused_reader>(current_state.reader).shard;
|
||||
|
||||
return _db.invoke_on(shard, [shard, query_uuid = _cmd.query_uuid, query_ranges = _ranges, ¤t_state, &last_pkey, &last_ckey,
|
||||
gts = tracing::global_trace_state_ptr(_trace_state)] (database& db) mutable {
|
||||
try {
|
||||
auto params = current_state.params.release();
|
||||
auto read_operation = current_state.read_operation.release();
|
||||
auto reader = current_state.reader.release();
|
||||
|
||||
flat_mutation_reader_opt reader;
|
||||
if (auto* maybe_paused_reader = std::get_if<paused_reader>(¤t_state.reader)) {
|
||||
if (auto inactive_read_ptr = db.user_read_concurrency_sem().unregister_inactive_read(maybe_paused_reader->handle)) {
|
||||
reader = std::move(*static_cast<inactive_read&>(*inactive_read_ptr).reader);
|
||||
if (maybe_paused_reader->has_pending_next_partition) {
|
||||
reader->next_partition();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
reader = std::move(*std::get<foreign_unique_ptr<flat_mutation_reader>>(current_state.reader));
|
||||
}
|
||||
|
||||
if (!reader) {
|
||||
return;
|
||||
}
|
||||
|
||||
auto& buffer = current_state.buffer;
|
||||
const auto fragments = buffer.size();
|
||||
const auto size_before = reader->buffer_size();
|
||||
@@ -544,6 +514,33 @@ future<> read_context::save_reader(ready_to_save_state& current_state, const dht
|
||||
});
|
||||
}
|
||||
|
||||
future<> read_context::pause(foreign_unique_ptr<flat_mutation_reader> reader) {
|
||||
const auto shard = reader.get_owner_shard();
|
||||
return _db.invoke_on(shard, [reader = std::move(reader)] (database& db) mutable {
|
||||
return db.user_read_concurrency_sem().register_inactive_read(std::make_unique<inactive_read>(std::move(reader)));
|
||||
}).then([this, shard] (reader_concurrency_semaphore::inactive_read_handle handle) {
|
||||
auto& current_state = std::get<used_state>(_readers[shard]);
|
||||
_readers[shard] = paused_state{std::move(current_state.params), std::move(current_state.read_operation), handle};
|
||||
});
|
||||
}
|
||||
|
||||
future<foreign_unique_ptr<flat_mutation_reader>> read_context::try_resume(shard_id shard) {
|
||||
return _db.invoke_on(shard, [handle = std::get<paused_state>(_readers[shard]).handle] (database& db) mutable {
|
||||
if (auto inactive_read_ptr = db.user_read_concurrency_sem().unregister_inactive_read(handle)) {
|
||||
return std::move(static_cast<inactive_read&>(*inactive_read_ptr).reader);
|
||||
}
|
||||
return foreign_unique_ptr<flat_mutation_reader>();
|
||||
}).then([this, shard] (foreign_unique_ptr<flat_mutation_reader> reader) {
|
||||
if (reader) {
|
||||
auto& current_state = std::get<paused_state>(_readers[shard]);
|
||||
_readers[shard] = used_state{std::move(current_state.params), std::move(current_state.read_operation)};
|
||||
} else {
|
||||
_readers[shard] = evicted_state{};
|
||||
}
|
||||
return std::move(reader);
|
||||
});
|
||||
}
|
||||
|
||||
future<> read_context::lookup_readers() {
|
||||
if (_cmd.query_uuid == utils::UUID{} || _cmd.is_first_page) {
|
||||
return make_ready_future<>();
|
||||
@@ -577,57 +574,37 @@ future<> read_context::save_readers(circular_buffer<mutation_fragment> unconsume
|
||||
return make_ready_future<>();
|
||||
}
|
||||
|
||||
auto last_pkey = compaction_state.partition_start.key();
|
||||
return _dismantling_gate.close().then([this, unconsumed_buffer = std::move(unconsumed_buffer), compaction_state = std::move(compaction_state),
|
||||
last_ckey = std::move(last_ckey)] () mutable {
|
||||
auto last_pkey = compaction_state.partition_start.key();
|
||||
|
||||
const auto cb_stats = dismantle_combined_buffer(std::move(unconsumed_buffer), last_pkey);
|
||||
tracing::trace(_trace_state, "Dismantled combined buffer: {} partitions/{} fragments/{} bytes", cb_stats.partitions, cb_stats.fragments,
|
||||
cb_stats.bytes);
|
||||
const auto cb_stats = dismantle_combined_buffer(std::move(unconsumed_buffer), last_pkey);
|
||||
tracing::trace(_trace_state, "Dismantled combined buffer: {} partitions/{} fragments/{} bytes", cb_stats.partitions, cb_stats.fragments,
|
||||
cb_stats.bytes);
|
||||
|
||||
const auto cs_stats = dismantle_compaction_state(std::move(compaction_state));
|
||||
tracing::trace(_trace_state, "Dismantled compaction state: {} partitions/{} fragments/{} bytes", cs_stats.partitions, cs_stats.fragments,
|
||||
cs_stats.bytes);
|
||||
const auto cs_stats = dismantle_compaction_state(std::move(compaction_state));
|
||||
tracing::trace(_trace_state, "Dismantled compaction state: {} partitions/{} fragments/{} bytes", cs_stats.partitions, cs_stats.fragments,
|
||||
cs_stats.bytes);
|
||||
|
||||
return do_with(std::move(last_pkey), std::move(last_ckey), [this] (const dht::decorated_key& last_pkey,
|
||||
return do_with(std::move(last_pkey), std::move(last_ckey), [this] (const dht::decorated_key& last_pkey,
|
||||
const std::optional<clustering_key_prefix>& last_ckey) {
|
||||
return parallel_for_each(_readers, [this, &last_pkey, &last_ckey] (reader_state& rs) {
|
||||
if (auto* maybe_successful_lookup_state = std::get_if<successful_lookup_state>(&rs)) {
|
||||
auto& current_state = *maybe_successful_lookup_state;
|
||||
rs = ready_to_save_state{std::move(current_state.params), std::move(current_state.read_operation),
|
||||
std::move(current_state.reader), circular_buffer<mutation_fragment>{}};
|
||||
return save_reader(std::get<ready_to_save_state>(rs), last_pkey, last_ckey);
|
||||
}
|
||||
return parallel_for_each(_readers, [this, &last_pkey, &last_ckey] (reader_state& rs) {
|
||||
if (auto* maybe_successful_lookup_state = std::get_if<successful_lookup_state>(&rs)) {
|
||||
auto& current_state = *maybe_successful_lookup_state;
|
||||
rs = ready_to_save_state{std::move(current_state.params), std::move(current_state.read_operation),
|
||||
std::move(current_state.reader), circular_buffer<mutation_fragment>{}};
|
||||
return save_reader(std::get<ready_to_save_state>(rs), last_pkey, last_ckey);
|
||||
}
|
||||
|
||||
auto finish_saving = [this, &last_pkey, &last_ckey] (dismantling_state& current_state) {
|
||||
return current_state.reader_fut.then_wrapped([this, ¤t_state, &last_pkey, &last_ckey] (
|
||||
future<stopped_foreign_reader>&& stopped_reader_fut) mutable {
|
||||
if (auto* ready_state = prepare_reader_for_saving(current_state, std::move(stopped_reader_fut), last_pkey, last_ckey)) {
|
||||
return save_reader(*ready_state, last_pkey, last_ckey);
|
||||
}
|
||||
return make_ready_future<>();
|
||||
});
|
||||
};
|
||||
if (auto* maybe_dismantling_state = std::get_if<dismantling_state>(&rs)) {
|
||||
auto& current_state = *maybe_dismantling_state;
|
||||
rs = ready_to_save_state{std::move(current_state.params), std::move(current_state.read_operation),
|
||||
std::move(current_state.reader), std::move(current_state.buffer)};
|
||||
return save_reader(std::get<ready_to_save_state>(rs), last_pkey, last_ckey);
|
||||
}
|
||||
|
||||
if (auto* maybe_dismantling_state = std::get_if<dismantling_state>(&rs)) {
|
||||
return finish_saving(*maybe_dismantling_state);
|
||||
}
|
||||
|
||||
if (auto* maybe_future_dismantling_state = std::get_if<future_dismantling_state>(&rs)) {
|
||||
return maybe_future_dismantling_state->fut.then_wrapped([this, &rs,
|
||||
finish_saving = std::move(finish_saving)] (future<dismantling_state>&& next_state_fut) mutable {
|
||||
if (next_state_fut.failed()) {
|
||||
mmq_log.debug("Failed to stop reader: {}", next_state_fut.get_exception());
|
||||
++_db.local().get_stats().multishard_query_failed_reader_stops;
|
||||
// We don't want to leave the reader in future dismantling state, lest
|
||||
// stop() will try to wait on the fut again and crash the application.
|
||||
rs = {};
|
||||
return make_ready_future<>();
|
||||
}
|
||||
rs = next_state_fut.get0();
|
||||
return finish_saving(std::get<dismantling_state>(rs));
|
||||
});
|
||||
}
|
||||
|
||||
return make_ready_future<>();
|
||||
return make_ready_future<>();
|
||||
});
|
||||
});
|
||||
});
|
||||
}
|
||||
@@ -640,8 +617,8 @@ static future<reconcilable_result> do_query_mutations(
|
||||
tracing::trace_state_ptr trace_state,
|
||||
db::timeout_clock::time_point timeout,
|
||||
query::result_memory_accounter&& accounter) {
|
||||
return do_with(std::make_unique<read_context>(db, s, cmd, ranges, trace_state), [s, &cmd, &ranges, trace_state, timeout,
|
||||
accounter = std::move(accounter)] (std::unique_ptr<read_context>& ctx) mutable {
|
||||
return do_with(seastar::make_shared<read_context>(db, s, cmd, ranges, trace_state), [s, &cmd, &ranges, trace_state, timeout,
|
||||
accounter = std::move(accounter)] (shared_ptr<read_context>& ctx) mutable {
|
||||
return ctx->lookup_readers().then([&ctx, s = std::move(s), &cmd, &ranges, trace_state, timeout,
|
||||
accounter = std::move(accounter)] () mutable {
|
||||
auto ms = mutation_source([&] (schema_ptr s,
|
||||
@@ -649,10 +626,9 @@ static future<reconcilable_result> do_query_mutations(
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
streamed_mutation::forwarding,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return make_multishard_combining_reader(std::move(s), pr, ps, pc, dht::global_partitioner(), ctx->factory(), std::move(trace_state),
|
||||
fwd_sm, fwd_mr, ctx->dismantler());
|
||||
return make_multishard_combining_reader(ctx, dht::global_partitioner(), std::move(s), pr, ps, pc, std::move(trace_state), fwd_mr);
|
||||
});
|
||||
auto reader = make_flat_multi_range_reader(s, std::move(ms), ranges, cmd.slice, service::get_local_sstable_query_read_priority(),
|
||||
trace_state, mutation_reader::forwarding::no);
|
||||
|
||||
@@ -556,128 +556,6 @@ flat_mutation_reader make_combined_reader(schema_ptr schema,
|
||||
return make_combined_reader(std::move(schema), std::move(v), fwd_sm, fwd_mr);
|
||||
}
|
||||
|
||||
void reader_concurrency_semaphore::signal(const resources& r) {
|
||||
_resources += r;
|
||||
while (!_wait_list.empty() && has_available_units(_wait_list.front().res)) {
|
||||
auto& x = _wait_list.front();
|
||||
_resources -= x.res;
|
||||
x.pr.set_value(make_lw_shared<reader_permit>(*this, x.res));
|
||||
_wait_list.pop_front();
|
||||
}
|
||||
}
|
||||
|
||||
future<lw_shared_ptr<reader_concurrency_semaphore::reader_permit>> reader_concurrency_semaphore::wait_admission(size_t memory,
|
||||
db::timeout_clock::time_point timeout) {
|
||||
if (_wait_list.size() >= _max_queue_length) {
|
||||
return make_exception_future<lw_shared_ptr<reader_permit>>(_make_queue_overloaded_exception());
|
||||
}
|
||||
auto r = resources(1, static_cast<ssize_t>(memory));
|
||||
if (!may_proceed(r) && _evict_an_inactive_reader) {
|
||||
while (_evict_an_inactive_reader() && !may_proceed(r));
|
||||
}
|
||||
if (may_proceed(r)) {
|
||||
_resources -= r;
|
||||
return make_ready_future<lw_shared_ptr<reader_permit>>(make_lw_shared<reader_permit>(*this, r));
|
||||
}
|
||||
promise<lw_shared_ptr<reader_permit>> pr;
|
||||
auto fut = pr.get_future();
|
||||
_wait_list.push_back(entry(std::move(pr), r), timeout);
|
||||
return fut;
|
||||
}
|
||||
|
||||
// A file that tracks the memory usage of buffers resulting from read
|
||||
// operations.
|
||||
class tracking_file_impl : public file_impl {
|
||||
file _tracked_file;
|
||||
lw_shared_ptr<reader_concurrency_semaphore::reader_permit> _permit;
|
||||
|
||||
// Shouldn't be called if semaphore is NULL.
|
||||
temporary_buffer<uint8_t> make_tracked_buf(temporary_buffer<uint8_t> buf) {
|
||||
return seastar::temporary_buffer<uint8_t>(buf.get_write(),
|
||||
buf.size(),
|
||||
make_deleter(buf.release(), std::bind(&reader_concurrency_semaphore::reader_permit::signal_memory, _permit, buf.size())));
|
||||
}
|
||||
|
||||
public:
|
||||
tracking_file_impl(file file, reader_resource_tracker resource_tracker)
|
||||
: _tracked_file(std::move(file))
|
||||
, _permit(resource_tracker.get_permit()) {
|
||||
}
|
||||
|
||||
tracking_file_impl(const tracking_file_impl&) = delete;
|
||||
tracking_file_impl& operator=(const tracking_file_impl&) = delete;
|
||||
tracking_file_impl(tracking_file_impl&&) = default;
|
||||
tracking_file_impl& operator=(tracking_file_impl&&) = default;
|
||||
|
||||
virtual future<size_t> write_dma(uint64_t pos, const void* buffer, size_t len, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->write_dma(pos, buffer, len, pc);
|
||||
}
|
||||
|
||||
virtual future<size_t> write_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->write_dma(pos, std::move(iov), pc);
|
||||
}
|
||||
|
||||
virtual future<size_t> read_dma(uint64_t pos, void* buffer, size_t len, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->read_dma(pos, buffer, len, pc);
|
||||
}
|
||||
|
||||
virtual future<size_t> read_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->read_dma(pos, iov, pc);
|
||||
}
|
||||
|
||||
virtual future<> flush(void) override {
|
||||
return get_file_impl(_tracked_file)->flush();
|
||||
}
|
||||
|
||||
virtual future<struct stat> stat(void) override {
|
||||
return get_file_impl(_tracked_file)->stat();
|
||||
}
|
||||
|
||||
virtual future<> truncate(uint64_t length) override {
|
||||
return get_file_impl(_tracked_file)->truncate(length);
|
||||
}
|
||||
|
||||
virtual future<> discard(uint64_t offset, uint64_t length) override {
|
||||
return get_file_impl(_tracked_file)->discard(offset, length);
|
||||
}
|
||||
|
||||
virtual future<> allocate(uint64_t position, uint64_t length) override {
|
||||
return get_file_impl(_tracked_file)->allocate(position, length);
|
||||
}
|
||||
|
||||
virtual future<uint64_t> size(void) override {
|
||||
return get_file_impl(_tracked_file)->size();
|
||||
}
|
||||
|
||||
virtual future<> close() override {
|
||||
return get_file_impl(_tracked_file)->close();
|
||||
}
|
||||
|
||||
virtual std::unique_ptr<file_handle_impl> dup() override {
|
||||
return get_file_impl(_tracked_file)->dup();
|
||||
}
|
||||
|
||||
virtual subscription<directory_entry> list_directory(std::function<future<> (directory_entry de)> next) override {
|
||||
return get_file_impl(_tracked_file)->list_directory(std::move(next));
|
||||
}
|
||||
|
||||
virtual future<temporary_buffer<uint8_t>> dma_read_bulk(uint64_t offset, size_t range_size, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->dma_read_bulk(offset, range_size, pc).then([this] (temporary_buffer<uint8_t> buf) {
|
||||
if (_permit) {
|
||||
buf = make_tracked_buf(std::move(buf));
|
||||
_permit->consume_memory(buf.size());
|
||||
}
|
||||
return make_ready_future<temporary_buffer<uint8_t>>(std::move(buf));
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
file reader_resource_tracker::track(file f) const {
|
||||
return file(make_shared<tracking_file_impl>(f, *this));
|
||||
}
|
||||
|
||||
|
||||
class restricting_mutation_reader : public flat_mutation_reader::impl {
|
||||
struct mutation_source_and_params {
|
||||
mutation_source _ms;
|
||||
@@ -838,12 +716,14 @@ class foreign_reader : public flat_mutation_reader::impl {
|
||||
template <typename T>
|
||||
using foreign_unique_ptr = foreign_ptr<std::unique_ptr<T>>;
|
||||
|
||||
using fragment_buffer = circular_buffer<mutation_fragment>;
|
||||
|
||||
foreign_unique_ptr<flat_mutation_reader> _reader;
|
||||
foreign_unique_ptr<future<>> _read_ahead_future;
|
||||
// Increase this counter every time next_partition() is called.
|
||||
// These pending calls will be executed the next time we go to the remote
|
||||
// Set this flag when next_partition() is called.
|
||||
// This pending call will be executed the next time we go to the remote
|
||||
// reader (a fill_buffer() or a fast_forward_to() call).
|
||||
unsigned _pending_next_partition = 0;
|
||||
bool _pending_next_partition = false;
|
||||
streamed_mutation::forwarding _fwd_sm;
|
||||
|
||||
// Forward an operation to the reader on the remote shard.
|
||||
@@ -857,12 +737,11 @@ class foreign_reader : public flat_mutation_reader::impl {
|
||||
Result forward_operation(db::timeout_clock::time_point timeout, Operation op) {
|
||||
return smp::submit_to(_reader.get_owner_shard(), [reader = _reader.get(),
|
||||
read_ahead_future = std::exchange(_read_ahead_future, nullptr),
|
||||
pending_next_partition = std::exchange(_pending_next_partition, 0),
|
||||
pending_next_partition = std::exchange(_pending_next_partition, false),
|
||||
timeout,
|
||||
op = std::move(op)] () mutable {
|
||||
auto exec_op_and_read_ahead = [=] () mutable {
|
||||
while (pending_next_partition) {
|
||||
--pending_next_partition;
|
||||
if (pending_next_partition) {
|
||||
reader->next_partition();
|
||||
}
|
||||
return op().then([=] (auto... results) {
|
||||
@@ -881,6 +760,8 @@ class foreign_reader : public flat_mutation_reader::impl {
|
||||
return make_ready_future<decltype(results)...>(std::move(results)...);
|
||||
});
|
||||
}
|
||||
|
||||
void update_buffer_with(foreign_unique_ptr<fragment_buffer> buffer, bool end_of_steam);
|
||||
public:
|
||||
foreign_reader(schema_ptr schema,
|
||||
foreign_unique_ptr<flat_mutation_reader> reader,
|
||||
@@ -900,10 +781,22 @@ public:
|
||||
virtual future<> fast_forward_to(position_range pr, db::timeout_clock::time_point timeout) override;
|
||||
|
||||
const mutation_fragment& peek_buffer() const { return buffer().front(); }
|
||||
const circular_buffer<mutation_fragment>& get_buffer() const { return buffer(); }
|
||||
|
||||
future<stopped_foreign_reader> stop();
|
||||
future<foreign_unique_ptr<flat_mutation_reader>> pause();
|
||||
void resume(foreign_unique_ptr<flat_mutation_reader> reader);
|
||||
|
||||
future<reader_lifecycle_policy::paused_or_stopped_reader> stop();
|
||||
};
|
||||
|
||||
void foreign_reader::update_buffer_with(foreign_unique_ptr<fragment_buffer> buffer, bool end_of_steam) {
|
||||
_end_of_stream = end_of_steam;
|
||||
for (const auto& mf : *buffer) {
|
||||
// Need a copy since the mf is on the remote shard.
|
||||
push_mutation_fragment(mutation_fragment(*_schema, mf));
|
||||
}
|
||||
}
|
||||
|
||||
foreign_reader::foreign_reader(schema_ptr schema,
|
||||
foreign_unique_ptr<flat_mutation_reader> reader,
|
||||
streamed_mutation::forwarding fwd_sm)
|
||||
@@ -929,8 +822,6 @@ future<> foreign_reader::fill_buffer(db::timeout_clock::time_point timeout) {
|
||||
return make_ready_future();
|
||||
}
|
||||
|
||||
using fragment_buffer = circular_buffer<mutation_fragment>;
|
||||
|
||||
return forward_operation(timeout, [reader = _reader.get(), timeout] () {
|
||||
auto f = reader->is_buffer_empty() ? reader->fill_buffer(timeout) : make_ready_future<>();
|
||||
return f.then([=] {
|
||||
@@ -938,12 +829,8 @@ future<> foreign_reader::fill_buffer(db::timeout_clock::time_point timeout) {
|
||||
std::make_unique<fragment_buffer>(reader->detach_buffer()),
|
||||
reader->is_end_of_stream());
|
||||
});
|
||||
}).then([this] (foreign_unique_ptr<fragment_buffer> buffer, bool end_of_steam) mutable {
|
||||
_end_of_stream = end_of_steam;
|
||||
for (const auto& mf : *buffer) {
|
||||
// Need a copy since the mf is on the remote shard.
|
||||
push_mutation_fragment(mutation_fragment(*_schema, mf));
|
||||
}
|
||||
}).then([this] (foreign_unique_ptr<fragment_buffer> buffer, bool end_of_stream) mutable {
|
||||
update_buffer_with(std::move(buffer), end_of_stream);
|
||||
});
|
||||
}
|
||||
|
||||
@@ -951,12 +838,12 @@ void foreign_reader::next_partition() {
|
||||
if (_fwd_sm == streamed_mutation::forwarding::yes) {
|
||||
clear_buffer();
|
||||
_end_of_stream = false;
|
||||
++_pending_next_partition;
|
||||
_pending_next_partition = true;
|
||||
} else {
|
||||
clear_buffer_to_next_partition();
|
||||
if (is_buffer_empty()) {
|
||||
_end_of_stream = false;
|
||||
++_pending_next_partition;
|
||||
_pending_next_partition = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -977,26 +864,61 @@ future<> foreign_reader::fast_forward_to(position_range pr, db::timeout_clock::t
|
||||
});
|
||||
}
|
||||
|
||||
future<stopped_foreign_reader> foreign_reader::stop() {
|
||||
if (_read_ahead_future || _pending_next_partition) {
|
||||
future<reader_lifecycle_policy::paused_or_stopped_reader> foreign_reader::stop() {
|
||||
if (_reader && (_read_ahead_future || _pending_next_partition)) {
|
||||
const auto owner_shard = _reader.get_owner_shard();
|
||||
return smp::submit_to(owner_shard, [reader = _reader.get(),
|
||||
read_ahead_future = std::exchange(_read_ahead_future, nullptr),
|
||||
pending_next_partition = std::exchange(_pending_next_partition, 0)] () mutable {
|
||||
pending_next_partition = std::exchange(_pending_next_partition, false)] () mutable {
|
||||
auto fut = read_ahead_future ? std::move(*read_ahead_future) : make_ready_future<>();
|
||||
return fut.then([=] () mutable {
|
||||
for (;pending_next_partition > 0; --pending_next_partition) {
|
||||
if (pending_next_partition) {
|
||||
reader->next_partition();
|
||||
}
|
||||
});
|
||||
}).then([this] {
|
||||
return stopped_foreign_reader{std::move(_reader), detach_buffer()};
|
||||
return reader_lifecycle_policy::paused_or_stopped_reader{std::move(_reader), detach_buffer(), false};
|
||||
});
|
||||
} else {
|
||||
return make_ready_future<stopped_foreign_reader>(stopped_foreign_reader{std::move(_reader), detach_buffer()});
|
||||
return make_ready_future<reader_lifecycle_policy::paused_or_stopped_reader>(
|
||||
reader_lifecycle_policy::paused_or_stopped_reader{std::move(_reader), detach_buffer(), _pending_next_partition});
|
||||
}
|
||||
}
|
||||
|
||||
future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>> foreign_reader::pause() {
|
||||
return smp::submit_to(_reader.get_owner_shard(), [reader = _reader.get(),
|
||||
read_ahead_future = std::exchange(_read_ahead_future, nullptr),
|
||||
pending_next_partition = std::exchange(_pending_next_partition, false)] () mutable {
|
||||
auto fut = read_ahead_future ? std::move(*read_ahead_future) : make_ready_future<>();
|
||||
return fut.then([=] () mutable {
|
||||
if (pending_next_partition) {
|
||||
reader->next_partition();
|
||||
}
|
||||
return make_ready_future<foreign_unique_ptr<fragment_buffer>, bool>(
|
||||
std::make_unique<fragment_buffer>(reader->detach_buffer()),
|
||||
reader->is_end_of_stream());
|
||||
});
|
||||
}).then([this] (foreign_unique_ptr<fragment_buffer>&& buffer, bool end_of_stream) mutable {
|
||||
update_buffer_with(std::move(buffer), end_of_stream);
|
||||
|
||||
// An ongoing pause() might overlap with a next_partition() call.
|
||||
// So if there is a pending next partition, try to execute it again
|
||||
// after the remote buffer was transferred. This is required for
|
||||
// correctness, otherwise some fragments belonging to the to-be-skipped
|
||||
// partition can escape the next_partition() call, both on the local and
|
||||
// the remote shard.
|
||||
if (_pending_next_partition) {
|
||||
_pending_next_partition = false;
|
||||
next_partition();
|
||||
}
|
||||
return std::move(_reader);
|
||||
});
|
||||
}
|
||||
|
||||
void foreign_reader::resume(foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader) {
|
||||
_reader = std::move(reader);
|
||||
}
|
||||
|
||||
flat_mutation_reader make_foreign_reader(schema_ptr schema,
|
||||
foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader,
|
||||
streamed_mutation::forwarding fwd_sm) {
|
||||
@@ -1008,14 +930,12 @@ flat_mutation_reader make_foreign_reader(schema_ptr schema,
|
||||
|
||||
// See make_multishard_combining_reader() for description.
|
||||
class multishard_combining_reader : public flat_mutation_reader::impl {
|
||||
shared_ptr<reader_lifecycle_policy> _lifecycle_policy;
|
||||
const dht::i_partitioner& _partitioner;
|
||||
const dht::partition_range* _pr;
|
||||
const query::partition_slice& _ps;
|
||||
const io_priority_class& _pc;
|
||||
remote_reader_factory _reader_factory;
|
||||
foreign_reader_dismantler _reader_dismantler;
|
||||
tracing::trace_state_ptr _trace_state;
|
||||
const streamed_mutation::forwarding _fwd_sm;
|
||||
const mutation_reader::forwarding _fwd_mr;
|
||||
|
||||
// Thin wrapper around a flat_mutation_reader (foreign_reader) that
|
||||
@@ -1033,14 +953,30 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
|
||||
class shard_reader {
|
||||
struct state {
|
||||
std::unique_ptr<foreign_reader> reader;
|
||||
unsigned pending_next_partition = 0;
|
||||
bool stopped = false;
|
||||
promise<> reader_promise;
|
||||
bool drop_partition_start = false;
|
||||
bool drop_static_row = false;
|
||||
};
|
||||
const multishard_combining_reader& _parent;
|
||||
const unsigned _shard;
|
||||
lw_shared_ptr<state> _state;
|
||||
std::optional<future<>> _read_ahead;
|
||||
std::optional<future<>> _pause;
|
||||
|
||||
std::optional<dht::decorated_key> _last_pkey;
|
||||
std::optional<position_in_partition> _last_position_in_partition;
|
||||
// These are used when the reader has to be recreated (after having been
|
||||
// evicted while paused) and the range and/or slice it is recreated with
|
||||
// differs from the original ones.
|
||||
std::optional<dht::partition_range> _range_override;
|
||||
std::optional<query::partition_slice> _slice_override;
|
||||
|
||||
private:
|
||||
void update_last_position();
|
||||
void adjust_partition_slice();
|
||||
future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>> recreate_reader();
|
||||
future<> resume();
|
||||
future<> do_fill_buffer(db::timeout_clock::time_point timeout);
|
||||
|
||||
public:
|
||||
shard_reader(multishard_combining_reader& parent, unsigned shard)
|
||||
@@ -1055,11 +991,7 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
|
||||
shard_reader(const shard_reader&) = delete;
|
||||
shard_reader& operator=(const shard_reader&) = delete;
|
||||
|
||||
~shard_reader() {
|
||||
if (!_state->stopped) {
|
||||
stop();
|
||||
}
|
||||
}
|
||||
~shard_reader();
|
||||
|
||||
// These methods assume the reader is already created.
|
||||
bool is_end_of_stream() const {
|
||||
@@ -1079,7 +1011,6 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
|
||||
// These methods don't assume the reader is already created.
|
||||
void next_partition();
|
||||
future<> fast_forward_to(const dht::partition_range& pr, db::timeout_clock::time_point timeout);
|
||||
future<> fast_forward_to(position_range pr, db::timeout_clock::time_point timeout);
|
||||
future<> create_reader();
|
||||
explicit operator bool() const {
|
||||
return bool(_state->reader);
|
||||
@@ -1091,7 +1022,7 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
|
||||
bool is_read_ahead_in_progress() const {
|
||||
return _read_ahead.has_value();
|
||||
}
|
||||
future<stopped_foreign_reader> stop();
|
||||
void pause();
|
||||
};
|
||||
|
||||
std::vector<shard_reader> _shard_readers;
|
||||
@@ -1104,18 +1035,15 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
|
||||
future<> handle_empty_reader_buffer(db::timeout_clock::time_point timeout);
|
||||
|
||||
public:
|
||||
multishard_combining_reader(schema_ptr s,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
const dht::i_partitioner& partitioner,
|
||||
remote_reader_factory reader_factory,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr,
|
||||
foreign_reader_dismantler reader_dismantler);
|
||||
|
||||
~multishard_combining_reader();
|
||||
multishard_combining_reader(
|
||||
shared_ptr<reader_lifecycle_policy> lifecycle_policy,
|
||||
const dht::i_partitioner& partitioner,
|
||||
schema_ptr s,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
mutation_reader::forwarding fwd_mr);
|
||||
|
||||
// this is captured.
|
||||
multishard_combining_reader(const multishard_combining_reader&) = delete;
|
||||
@@ -1129,94 +1057,284 @@ public:
|
||||
virtual future<> fast_forward_to(position_range pr, db::timeout_clock::time_point timeout) override;
|
||||
};
|
||||
|
||||
multishard_combining_reader::shard_reader::~shard_reader() {
|
||||
// Nothing to do if there was no reader created, nor is there a background
|
||||
// read ahead in progress which will create one.
|
||||
if (!_state->reader && !_read_ahead) {
|
||||
return;
|
||||
}
|
||||
|
||||
_state->stopped = true;
|
||||
|
||||
auto f = [this] {
|
||||
if (_read_ahead) {
|
||||
return std::move(*_read_ahead);
|
||||
} else if (_pause) {
|
||||
return std::move(*_pause);
|
||||
} else {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
}();
|
||||
|
||||
_parent._lifecycle_policy->destroy_reader(_shard, f.then([state = _state.get()] {
|
||||
return state->reader->stop();
|
||||
}).finally([state = _state] {}));
|
||||
}
|
||||
|
||||
void multishard_combining_reader::shard_reader::update_last_position() {
|
||||
auto& reader = *_state->reader;
|
||||
if (reader.is_buffer_empty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
auto rbegin = std::reverse_iterator(reader.get_buffer().end());
|
||||
auto rend = std::reverse_iterator(reader.get_buffer().begin());
|
||||
if (auto pk_it = std::find_if(rbegin, rend, std::mem_fn(&mutation_fragment::is_partition_start)); pk_it != rend) {
|
||||
_last_pkey = pk_it->as_partition_start().key();
|
||||
}
|
||||
|
||||
_last_position_in_partition.emplace(reader.get_buffer().back().position());
|
||||
}
|
||||
|
||||
void multishard_combining_reader::shard_reader::adjust_partition_slice() {
|
||||
if (!_slice_override) {
|
||||
_slice_override = _parent._ps;
|
||||
}
|
||||
|
||||
const auto& schema = *_parent._schema;
|
||||
_slice_override->clear_range(schema, _last_pkey->key());
|
||||
auto& last_ckey = _last_position_in_partition->key();
|
||||
|
||||
auto cmp = bound_view::compare(schema);
|
||||
auto eq = clustering_key_prefix::equality(schema);
|
||||
|
||||
auto ranges = _slice_override->default_row_ranges();
|
||||
auto it = ranges.begin();
|
||||
while (it != ranges.end()) {
|
||||
auto range = bound_view::from_range(*it);
|
||||
if (cmp(range.second, last_ckey) || eq(range.second.prefix(), last_ckey)) {
|
||||
it = ranges.erase(it);
|
||||
} else {
|
||||
if (cmp(range.first, last_ckey)) {
|
||||
assert(cmp(last_ckey, range.second));
|
||||
*it = query::clustering_range(query::clustering_range::bound{last_ckey, false}, it->end());
|
||||
}
|
||||
++it;
|
||||
}
|
||||
}
|
||||
|
||||
_slice_override->clear_ranges();
|
||||
_slice_override->set_range(schema, _last_pkey->key(), std::move(ranges));
|
||||
}
|
||||
|
||||
future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>> multishard_combining_reader::shard_reader::recreate_reader() {
|
||||
const dht::partition_range* range = _parent._pr;
|
||||
const query::partition_slice* slice = &_parent._ps;
|
||||
|
||||
if (_last_pkey) {
|
||||
bool partition_range_is_inclusive = true;
|
||||
|
||||
if (_last_position_in_partition) {
|
||||
switch (_last_position_in_partition->region()) {
|
||||
case partition_region::partition_start:
|
||||
_state->drop_partition_start = true;
|
||||
break;
|
||||
case partition_region::static_row:
|
||||
_state->drop_partition_start = true;
|
||||
_state->drop_static_row = true;
|
||||
break;
|
||||
case partition_region::clustered:
|
||||
_state->drop_partition_start = true;
|
||||
_state->drop_static_row = true;
|
||||
adjust_partition_slice();
|
||||
slice = &*_slice_override;
|
||||
break;
|
||||
case partition_region::partition_end:
|
||||
partition_range_is_inclusive = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// The original range contained a single partition and we've read it
|
||||
// all. We'd have to create a reader with an empty range that would
|
||||
// immediately be at EOS. This is not possible so just don't recreate
|
||||
// the reader.
|
||||
// This should be extremely rare (who'd create a multishard reader to
|
||||
// read a single partition) but still, let's make sure we handle it
|
||||
// correctly.
|
||||
if (_parent._pr->is_singular() && !partition_range_is_inclusive) {
|
||||
return make_ready_future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>>();
|
||||
}
|
||||
|
||||
_range_override = dht::partition_range({dht::partition_range::bound(*_last_pkey, partition_range_is_inclusive)}, _parent._pr->end());
|
||||
range = &*_range_override;
|
||||
}
|
||||
|
||||
return _parent._lifecycle_policy->create_reader(
|
||||
_shard,
|
||||
_parent._schema,
|
||||
*range,
|
||||
*slice,
|
||||
_parent._pc,
|
||||
_parent._trace_state,
|
||||
_parent._fwd_mr);
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::shard_reader::resume() {
|
||||
return std::exchange(_pause, std::nullopt)->then([this, state = _state] {
|
||||
if (state->stopped) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
return _parent._lifecycle_policy->try_resume(_shard).then(
|
||||
[this, state = std::move(state)] (foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader) mutable {
|
||||
if (reader) {
|
||||
state->reader->resume(std::move(reader));
|
||||
return make_ready_future<>();
|
||||
} else if (state->stopped) {
|
||||
return make_ready_future<>();
|
||||
} else {
|
||||
return recreate_reader().then([this, state = std::move(state)] (foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader) {
|
||||
state->reader->resume(std::move(reader));
|
||||
});
|
||||
}
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::shard_reader::do_fill_buffer(db::timeout_clock::time_point timeout) {
|
||||
return _state->reader->fill_buffer(timeout).then([this, state = _state] {
|
||||
auto& reader = *state->reader;
|
||||
|
||||
if (reader.is_buffer_empty()) {
|
||||
return;
|
||||
}
|
||||
if (state->drop_partition_start) {
|
||||
state->drop_partition_start = false;
|
||||
if (reader.peek_buffer().is_partition_start()) {
|
||||
reader.pop_mutation_fragment();
|
||||
}
|
||||
}
|
||||
|
||||
if (reader.is_buffer_empty()) {
|
||||
return;
|
||||
}
|
||||
if (state->drop_static_row) {
|
||||
state->drop_static_row = false;
|
||||
if (reader.peek_buffer().is_static_row()) {
|
||||
reader.pop_mutation_fragment();
|
||||
}
|
||||
}
|
||||
|
||||
if (!state->stopped) {
|
||||
update_last_position();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::shard_reader::fill_buffer(db::timeout_clock::time_point timeout) {
|
||||
if (_read_ahead) {
|
||||
return *std::exchange(_read_ahead, std::nullopt);
|
||||
}
|
||||
return _state->reader->fill_buffer(timeout);
|
||||
if (!_state->reader->is_buffer_empty()) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
if (_pause) {
|
||||
return resume().then([this, timeout] {
|
||||
return fill_buffer(timeout);
|
||||
});
|
||||
}
|
||||
return do_fill_buffer(timeout);
|
||||
}
|
||||
|
||||
void multishard_combining_reader::shard_reader::next_partition() {
|
||||
_last_position_in_partition = position_in_partition(position_in_partition::end_of_partition_tag_t{});
|
||||
|
||||
// The only case this can be called with an uncreated reader is when
|
||||
// `next_partition()` is called on the multishard reader before the
|
||||
// first `fill_buffer()` call. In this case we are right before the first
|
||||
// partition so this call has no effect, hence we can ignore it.
|
||||
if (_state->reader) {
|
||||
_state->reader->next_partition();
|
||||
} else {
|
||||
++_state->pending_next_partition;
|
||||
}
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::shard_reader::fast_forward_to(const dht::partition_range& pr, db::timeout_clock::time_point timeout) {
|
||||
if (_state->reader) {
|
||||
return _state->reader->fast_forward_to(pr, timeout);
|
||||
_last_pkey.reset();
|
||||
_last_position_in_partition.reset();
|
||||
|
||||
auto do_fast_forward = [this, &pr, timeout] {
|
||||
return _state->reader->fast_forward_to(pr, timeout);
|
||||
};
|
||||
|
||||
if (_pause) {
|
||||
return resume().then(std::move(do_fast_forward));
|
||||
}
|
||||
|
||||
if (_read_ahead) {
|
||||
return std::exchange(_read_ahead, std::nullopt)->then(std::move(do_fast_forward));
|
||||
}
|
||||
|
||||
return do_fast_forward();
|
||||
}
|
||||
// No need to fast-forward uncreated readers, they will be passed the new
|
||||
// range when created.
|
||||
return make_ready_future<>();
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::shard_reader::fast_forward_to(position_range pr, db::timeout_clock::time_point timeout) {
|
||||
if (_state->reader) {
|
||||
return _state->reader->fast_forward_to(pr, timeout);
|
||||
}
|
||||
return create_reader().then([this, pr = std::move(pr), timeout] {
|
||||
return _state->reader->fast_forward_to(pr, timeout);
|
||||
});
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::shard_reader::create_reader() {
|
||||
if (_state->reader) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
if (_read_ahead) {
|
||||
return _state->reader_promise.get_future();
|
||||
return *std::exchange(_read_ahead, std::nullopt);
|
||||
}
|
||||
return _parent._reader_factory(_shard, _parent._schema, *_parent._pr, _parent._ps, _parent._pc, _parent._trace_state,
|
||||
_parent._fwd_sm, _parent._fwd_mr).then(
|
||||
[schema = _parent._schema, state = _state, fwd_sm = _parent._fwd_sm] (foreign_ptr<std::unique_ptr<flat_mutation_reader>>&& r) mutable {
|
||||
state->reader = std::make_unique<foreign_reader>(std::move(schema), std::move(r), fwd_sm);
|
||||
for (;state->pending_next_partition; --state->pending_next_partition) {
|
||||
state->reader->next_partition();
|
||||
}
|
||||
|
||||
if (!state->stopped) {
|
||||
state->reader_promise.set_value();
|
||||
}
|
||||
return _parent._lifecycle_policy->create_reader(_shard, _parent._schema, *_parent._pr, _parent._ps, _parent._pc, _parent._trace_state,
|
||||
_parent._fwd_mr).then(
|
||||
[schema = _parent._schema, state = _state] (foreign_ptr<std::unique_ptr<flat_mutation_reader>>&& r) mutable {
|
||||
state->reader = std::make_unique<foreign_reader>(std::move(schema), std::move(r));
|
||||
});
|
||||
}
|
||||
|
||||
void multishard_combining_reader::shard_reader::read_ahead(db::timeout_clock::time_point timeout) {
|
||||
if (_read_ahead) {
|
||||
if (_read_ahead || (_state->reader && (_state->reader->is_end_of_stream() || !_state->reader->is_buffer_empty()))) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (_state->reader) {
|
||||
_read_ahead.emplace(_state->reader->fill_buffer(timeout));
|
||||
} else {
|
||||
_read_ahead.emplace(create_reader().then([state = _state, timeout] () mutable {
|
||||
if (state->stopped) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
return state->reader->fill_buffer(timeout);
|
||||
}));
|
||||
}
|
||||
auto f = _state->reader
|
||||
? (_pause ? resume() : make_ready_future<>())
|
||||
: create_reader();
|
||||
|
||||
_read_ahead.emplace(f.then([this, state = _state, timeout] () mutable {
|
||||
if (state->stopped) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
return do_fill_buffer(timeout);
|
||||
}));
|
||||
}
|
||||
|
||||
future<stopped_foreign_reader> multishard_combining_reader::shard_reader::stop() {
|
||||
_state->stopped = true;
|
||||
|
||||
if (!_state->reader && !_read_ahead) {
|
||||
return make_ready_future<stopped_foreign_reader>(stopped_foreign_reader{nullptr, circular_buffer<mutation_fragment>{}});
|
||||
void multishard_combining_reader::shard_reader::pause() {
|
||||
if (_pause) {
|
||||
return;
|
||||
}
|
||||
|
||||
auto f = [this] {
|
||||
if (_read_ahead) {
|
||||
return _read_ahead->then([state = _state.get()] () mutable {
|
||||
return state->reader->stop();
|
||||
});
|
||||
} else {
|
||||
return _state->reader->stop();
|
||||
auto f = _read_ahead ? *std::exchange(_read_ahead, std::nullopt) : make_ready_future<>();
|
||||
_pause = f.then([this, state = _state] () mutable {
|
||||
if (state->stopped) {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
}();
|
||||
return f.finally([state = _state] {});
|
||||
return state->reader->pause().then([this, state = std::move(state)] (foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader) {
|
||||
if (state->stopped) {
|
||||
state->reader->resume(std::move(reader));
|
||||
return make_ready_future<>();
|
||||
}
|
||||
|
||||
// When pausing, the content of the remote reader's buffer is transferred to
|
||||
// the foreign reader, so we might need to update the last position.
|
||||
update_last_position();
|
||||
|
||||
return _parent._lifecycle_policy->pause(std::move(reader));
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
void multishard_combining_reader::move_to_next_shard() {
|
||||
@@ -1229,10 +1347,11 @@ future<> multishard_combining_reader::handle_empty_reader_buffer(db::timeout_clo
|
||||
auto& reader = _shard_readers[_current_shard];
|
||||
|
||||
if (reader.is_end_of_stream()) {
|
||||
if (_fwd_sm || std::all_of(_shard_readers.begin(), _shard_readers.end(), std::mem_fn(&shard_reader::done))) {
|
||||
if (std::all_of(_shard_readers.begin(), _shard_readers.end(), std::mem_fn(&shard_reader::done))) {
|
||||
_end_of_stream = true;
|
||||
} else {
|
||||
move_to_next_shard();
|
||||
reader.pause();
|
||||
}
|
||||
return make_ready_future<>();
|
||||
} else if (reader.is_read_ahead_in_progress()) {
|
||||
@@ -1255,25 +1374,22 @@ future<> multishard_combining_reader::handle_empty_reader_buffer(db::timeout_clo
|
||||
}
|
||||
}
|
||||
|
||||
multishard_combining_reader::multishard_combining_reader(schema_ptr s,
|
||||
multishard_combining_reader::multishard_combining_reader(
|
||||
shared_ptr<reader_lifecycle_policy> lifecycle_policy,
|
||||
const dht::i_partitioner& partitioner,
|
||||
schema_ptr s,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
const dht::i_partitioner& partitioner,
|
||||
remote_reader_factory reader_factory,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr,
|
||||
foreign_reader_dismantler reader_dismantler)
|
||||
mutation_reader::forwarding fwd_mr)
|
||||
: impl(s)
|
||||
, _lifecycle_policy(std::move(lifecycle_policy))
|
||||
, _partitioner(partitioner)
|
||||
, _pr(&pr)
|
||||
, _ps(ps)
|
||||
, _pc(pc)
|
||||
, _reader_factory(std::move(reader_factory))
|
||||
, _reader_dismantler(std::move(reader_dismantler))
|
||||
, _trace_state(std::move(trace_state))
|
||||
, _fwd_sm(fwd_sm)
|
||||
, _fwd_mr(fwd_mr)
|
||||
, _current_shard(pr.start() ? _partitioner.shard_of(pr.start()->value().token()) : _partitioner.shard_of_minimum_token())
|
||||
, _next_token(_partitioner.token_for_next_shard(pr.start() ? pr.start()->value().token() : dht::minimum_token(),
|
||||
@@ -1284,25 +1400,6 @@ multishard_combining_reader::multishard_combining_reader(schema_ptr s,
|
||||
}
|
||||
}
|
||||
|
||||
multishard_combining_reader::~multishard_combining_reader() {
|
||||
for (shard_id shard = 0; shard < smp::count; ++shard) {
|
||||
auto& reader = _shard_readers[shard];
|
||||
|
||||
// Readers might also be created by background read-aheads, so it's not
|
||||
// enough to check whether the reader is created at the moment, we also
|
||||
// need to check whether there is a read-ahead in progress. If there is,
|
||||
// it will surely create a reader which also needs to be dismantled.
|
||||
if (!reader && !reader.is_read_ahead_in_progress()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
auto fut = reader.stop();
|
||||
if (_reader_dismantler) {
|
||||
_reader_dismantler(shard, std::move(fut));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::fill_buffer(db::timeout_clock::time_point timeout) {
|
||||
_crossed_shards = false;
|
||||
return do_until([this] { return is_buffer_full() || is_end_of_stream(); }, [this, timeout] {
|
||||
@@ -1318,6 +1415,7 @@ future<> multishard_combining_reader::fill_buffer(db::timeout_clock::time_point
|
||||
while (!reader.is_buffer_empty() && !is_buffer_full()) {
|
||||
if (const auto& mf = reader.peek_buffer(); mf.is_partition_start() && mf.as_partition_start().key().token() >= _next_token) {
|
||||
move_to_next_shard();
|
||||
reader.pause();
|
||||
return make_ready_future<>();
|
||||
}
|
||||
push_mutation_fragment(reader.pop_mutation_fragment());
|
||||
@@ -1327,15 +1425,9 @@ future<> multishard_combining_reader::fill_buffer(db::timeout_clock::time_point
|
||||
}
|
||||
|
||||
void multishard_combining_reader::next_partition() {
|
||||
if (_fwd_sm == streamed_mutation::forwarding::yes) {
|
||||
clear_buffer();
|
||||
_end_of_stream = false;
|
||||
clear_buffer_to_next_partition();
|
||||
if (is_buffer_empty()) {
|
||||
_shard_readers[_current_shard].next_partition();
|
||||
} else {
|
||||
clear_buffer_to_next_partition();
|
||||
if (is_buffer_empty()) {
|
||||
_shard_readers[_current_shard].next_partition();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1357,24 +1449,18 @@ future<> multishard_combining_reader::fast_forward_to(const dht::partition_range
|
||||
}
|
||||
|
||||
future<> multishard_combining_reader::fast_forward_to(position_range pr, db::timeout_clock::time_point timeout) {
|
||||
forward_buffer_to(pr.start());
|
||||
_end_of_stream = false;
|
||||
if (is_buffer_empty()) {
|
||||
return _shard_readers[_current_shard].fast_forward_to(std::move(pr), timeout);
|
||||
}
|
||||
return make_ready_future<>();
|
||||
return make_exception_future<>(std::bad_function_call());
|
||||
}
|
||||
|
||||
flat_mutation_reader make_multishard_combining_reader(schema_ptr schema,
|
||||
flat_mutation_reader make_multishard_combining_reader(
|
||||
shared_ptr<reader_lifecycle_policy> lifecycle_policy,
|
||||
const dht::i_partitioner& partitioner,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
const dht::i_partitioner& partitioner,
|
||||
remote_reader_factory reader_factory,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr,
|
||||
foreign_reader_dismantler reader_dismantler) {
|
||||
return make_flat_mutation_reader<multishard_combining_reader>(std::move(schema), pr, ps, pc, partitioner, std::move(reader_factory),
|
||||
std::move(trace_state), fwd_sm, fwd_mr, std::move(reader_dismantler));
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return make_flat_mutation_reader<multishard_combining_reader>(std::move(lifecycle_policy), partitioner, std::move(schema), pr, ps, pc,
|
||||
std::move(trace_state), fwd_mr);
|
||||
}
|
||||
|
||||
@@ -388,27 +388,81 @@ flat_mutation_reader make_foreign_reader(schema_ptr schema,
|
||||
foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader,
|
||||
streamed_mutation::forwarding fwd_sm = streamed_mutation::forwarding::no);
|
||||
|
||||
using remote_reader_factory = noncopyable_function<future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>>(unsigned,
|
||||
schema_ptr,
|
||||
const dht::partition_range&,
|
||||
const query::partition_slice&,
|
||||
const io_priority_class&,
|
||||
tracing::trace_state_ptr,
|
||||
streamed_mutation::forwarding,
|
||||
mutation_reader::forwarding)>;
|
||||
/// Reader lifecycle policy for the mulitshard combining reader.
|
||||
///
|
||||
/// This policy is expected to make sure any additional resource the readers
|
||||
/// might need is kept alive for the lifetime of the readers, not that
|
||||
/// of the multishard reader. This is a very important distinction. As
|
||||
/// destructors cannot return futures, the multishard reader will be
|
||||
/// destroyed before all it's shard readers could stop properly. Hence it
|
||||
/// is the duty of this policy to make sure all objects the shard readers
|
||||
/// depend on stay alive until they are properly destroyed on their home
|
||||
/// shards. Note that this also includes the passed in `range` and `slice`
|
||||
/// parameters because although client code is required to keep them alive as
|
||||
/// long as the top level reader lives, the shard readers might outlive the
|
||||
/// multishard reader itself.
|
||||
class reader_lifecycle_policy {
|
||||
public:
|
||||
struct paused_or_stopped_reader {
|
||||
// Null when the reader is paused.
|
||||
foreign_ptr<std::unique_ptr<flat_mutation_reader>> remote_reader;
|
||||
circular_buffer<mutation_fragment> unconsumed_fragments;
|
||||
// Only set for paused readers.
|
||||
bool has_pending_next_partition;
|
||||
};
|
||||
|
||||
struct stopped_foreign_reader {
|
||||
foreign_ptr<std::unique_ptr<flat_mutation_reader>> remote_reader;
|
||||
circular_buffer<mutation_fragment> unconsumed_fragments;
|
||||
public:
|
||||
/// Create an appropriate reader on the specified shard.
|
||||
///
|
||||
/// Will be called when the multishard reader visits a shard for the
|
||||
/// first time. This method should also enter gates, take locks or
|
||||
/// whatever is appropriate to make sure resources it is using on the
|
||||
/// remote shard stay alive, during the lifetime of the created reader.
|
||||
virtual future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>> create_reader(
|
||||
shard_id shard,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
mutation_reader::forwarding fwd_mr) = 0;
|
||||
|
||||
/// Wait on the shard reader to stop then destroy it.
|
||||
///
|
||||
/// Will be called when the multishard reader is being destroyed. It will be
|
||||
/// called for each of the shard readers. The future resolves when the
|
||||
/// reader is stopped, that is it, finishes all background and/or pending
|
||||
/// work.
|
||||
/// This method is expected to do a proper cleanup, that is, leave any gates,
|
||||
/// release any locks or whatever is appropriate for the shard reader.
|
||||
///
|
||||
/// The multishard reader couldn't wait on any future returned from this
|
||||
/// method (as it will be called from the destructor) so waiting on
|
||||
/// all the readers being cleaned up is up to the implementation.
|
||||
///
|
||||
/// This method will be called from a destructor so it cannot throw.
|
||||
virtual void destroy_reader(shard_id shard, future<paused_or_stopped_reader> reader) noexcept = 0;
|
||||
|
||||
/// Pause the reader.
|
||||
///
|
||||
/// The purpose of pausing a reader is making it evictable while it is
|
||||
/// otherwise inactive. This allows freeing up resources that are in-demand
|
||||
/// by evicting these paused readers. Most notably, this allows freeing up
|
||||
/// reader permits when the node is overloaded with reads.
|
||||
virtual future<> pause(foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader) = 0;
|
||||
|
||||
/// Try to resume the reader.
|
||||
///
|
||||
/// The pointer returned will be null when resuming fails. This can happen
|
||||
/// if the reader was evicted while paused.
|
||||
virtual future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>> try_resume(shard_id shard) = 0;
|
||||
};
|
||||
using foreign_reader_dismantler = noncopyable_function<void(shard_id, future<stopped_foreign_reader>)>;
|
||||
|
||||
/// Make a multishard_combining_reader.
|
||||
///
|
||||
/// multishard_combining_reader takes care of reading a range from all shards
|
||||
/// that own a subrange in the range. Readers are created on-demand with the
|
||||
/// supplied reader_factory. This factory function is expected to create an
|
||||
/// appropriate reader on the specified shard and return a foreign_ptr to it.
|
||||
/// that own a subrange in the range. Shard reader are created on-demand, when
|
||||
/// the shard is visited for the first time.
|
||||
///
|
||||
/// The read starts with a concurrency of one, that is the reader reads from a
|
||||
/// single shard at a time. The concurrency is exponentially increased (to a
|
||||
@@ -421,19 +475,13 @@ using foreign_reader_dismantler = noncopyable_function<void(shard_id, future<sto
|
||||
/// For dense tables (where we rarely cross shards) we rely on the
|
||||
/// foreign_reader to issue sufficient read-aheads on its own to avoid blocking.
|
||||
///
|
||||
/// Optionally a dismantler function can be passed to the multishard
|
||||
/// reader. When the multishard reader is destroyed it will invoke the
|
||||
/// dismantler functor for each of its foreign (shard) readers, passing a future
|
||||
/// to a `stopped_foreign_reader`. The future becomes available when the foreign
|
||||
/// reader has stopped, that is, it finished all of its in-progress read aheads
|
||||
/// and/or any pending `next_partition()` calls.
|
||||
flat_mutation_reader make_multishard_combining_reader(schema_ptr schema,
|
||||
/// The readers' life-cycles are managed through the supplied lifecycle policy.
|
||||
flat_mutation_reader make_multishard_combining_reader(
|
||||
shared_ptr<reader_lifecycle_policy> lifecycle_policy,
|
||||
const dht::i_partitioner& partitioner,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& pr,
|
||||
const query::partition_slice& ps,
|
||||
const io_priority_class& pc,
|
||||
const dht::i_partitioner& partitioner,
|
||||
remote_reader_factory reader_factory,
|
||||
tracing::trace_state_ptr trace_state = nullptr,
|
||||
streamed_mutation::forwarding fwd_sm = streamed_mutation::forwarding::no,
|
||||
mutation_reader::forwarding fwd_mr = mutation_reader::forwarding::no,
|
||||
foreign_reader_dismantler reader_dismantler = {});
|
||||
mutation_reader::forwarding fwd_mr = mutation_reader::forwarding::no);
|
||||
|
||||
@@ -270,6 +270,7 @@ public:
|
||||
static position_in_partition for_range_start(const query::clustering_range&);
|
||||
static position_in_partition for_range_end(const query::clustering_range&);
|
||||
|
||||
partition_region region() const { return _type; }
|
||||
bool is_partition_start() const { return _type == partition_region::partition_start; }
|
||||
bool is_partition_end() const { return _type == partition_region::partition_end; }
|
||||
bool is_static_row() const { return _type == partition_region::static_row; }
|
||||
|
||||
54
querier.cc
54
querier.cc
@@ -216,16 +216,43 @@ static querier_cache::entries::iterator find_querier(querier_cache::entries& ent
|
||||
return it->pos();
|
||||
}
|
||||
|
||||
querier_cache::querier_cache(size_t max_cache_size, std::chrono::seconds entry_ttl)
|
||||
: _expiry_timer([this] { scan_cache_entries(); })
|
||||
querier_cache::querier_cache(reader_concurrency_semaphore& sem, size_t max_cache_size, std::chrono::seconds entry_ttl)
|
||||
: _sem(sem)
|
||||
, _expiry_timer([this] { scan_cache_entries(); })
|
||||
, _entry_ttl(entry_ttl)
|
||||
, _max_queriers_memory_usage(max_cache_size) {
|
||||
_expiry_timer.arm_periodic(entry_ttl / 2);
|
||||
}
|
||||
|
||||
class querier_inactive_read : public reader_concurrency_semaphore::inactive_read {
|
||||
querier_cache::entries& _entries;
|
||||
querier_cache::entries::iterator _pos;
|
||||
querier_cache::stats& _stats;
|
||||
|
||||
public:
|
||||
querier_inactive_read(querier_cache::entries& entries, querier_cache::entries::iterator pos, querier_cache::stats& stats)
|
||||
: _entries(entries)
|
||||
, _pos(pos)
|
||||
, _stats(stats) {
|
||||
}
|
||||
virtual void evict() override {
|
||||
_entries.erase(_pos);
|
||||
++_stats.resource_based_evictions;
|
||||
--_stats.population;
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Querier>
|
||||
static void insert_querier(querier_cache::entries& entries, querier_cache::index& index, querier_cache::stats& stats,
|
||||
size_t max_queriers_memory_usage, utils::UUID key, Querier&& q, lowres_clock::time_point expires, tracing::trace_state_ptr trace_state) {
|
||||
static void insert_querier(
|
||||
reader_concurrency_semaphore& sem,
|
||||
querier_cache::entries& entries,
|
||||
querier_cache::index& index,
|
||||
querier_cache::stats& stats,
|
||||
size_t max_queriers_memory_usage,
|
||||
utils::UUID key,
|
||||
Querier&& q,
|
||||
lowres_clock::time_point expires,
|
||||
tracing::trace_state_ptr trace_state) {
|
||||
// FIXME: see #3159
|
||||
// In reverse mode flat_mutation_reader drops any remaining rows of the
|
||||
// current partition when the page ends so it cannot be reused across
|
||||
@@ -258,27 +285,30 @@ static void insert_querier(querier_cache::entries& entries, querier_cache::index
|
||||
|
||||
auto& e = entries.emplace_back(key, std::move(q), expires);
|
||||
e.set_pos(--entries.end());
|
||||
e.set_inactive_handle(sem.register_inactive_read(std::make_unique<querier_inactive_read>(entries, e.pos(), stats)));
|
||||
index.insert(e);
|
||||
++stats.population;
|
||||
}
|
||||
|
||||
void querier_cache::insert(utils::UUID key, data_querier&& q, tracing::trace_state_ptr trace_state) {
|
||||
insert_querier(_entries, _data_querier_index, _stats, _max_queriers_memory_usage, key, std::move(q), lowres_clock::now() + _entry_ttl,
|
||||
insert_querier(_sem, _entries, _data_querier_index, _stats, _max_queriers_memory_usage, key, std::move(q), lowres_clock::now() + _entry_ttl,
|
||||
std::move(trace_state));
|
||||
}
|
||||
|
||||
void querier_cache::insert(utils::UUID key, mutation_querier&& q, tracing::trace_state_ptr trace_state) {
|
||||
insert_querier(_entries, _mutation_querier_index, _stats, _max_queriers_memory_usage, key, std::move(q), lowres_clock::now() + _entry_ttl,
|
||||
insert_querier(_sem, _entries, _mutation_querier_index, _stats, _max_queriers_memory_usage, key, std::move(q), lowres_clock::now() + _entry_ttl,
|
||||
std::move(trace_state));
|
||||
}
|
||||
|
||||
void querier_cache::insert(utils::UUID key, shard_mutation_querier&& q, tracing::trace_state_ptr trace_state) {
|
||||
insert_querier(_entries, _shard_mutation_querier_index, _stats, _max_queriers_memory_usage, key, std::move(q), lowres_clock::now() + _entry_ttl,
|
||||
insert_querier(_sem, _entries, _shard_mutation_querier_index, _stats, _max_queriers_memory_usage, key, std::move(q), lowres_clock::now() + _entry_ttl,
|
||||
std::move(trace_state));
|
||||
}
|
||||
|
||||
template <typename Querier>
|
||||
static std::optional<Querier> lookup_querier(querier_cache::entries& entries,
|
||||
static std::optional<Querier> lookup_querier(
|
||||
reader_concurrency_semaphore& sem,
|
||||
querier_cache::entries& entries,
|
||||
querier_cache::index& index,
|
||||
querier_cache::stats& stats,
|
||||
utils::UUID key,
|
||||
@@ -294,6 +324,7 @@ static std::optional<Querier> lookup_querier(querier_cache::entries& entries,
|
||||
}
|
||||
|
||||
auto q = std::move(*it).template value<Querier>();
|
||||
sem.unregister_inactive_read(it->get_inactive_handle());
|
||||
entries.erase(it);
|
||||
--stats.population;
|
||||
|
||||
@@ -313,7 +344,7 @@ std::optional<data_querier> querier_cache::lookup_data_querier(utils::UUID key,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
tracing::trace_state_ptr trace_state) {
|
||||
return lookup_querier<data_querier>(_entries, _data_querier_index, _stats, key, s, range, slice, std::move(trace_state));
|
||||
return lookup_querier<data_querier>(_sem, _entries, _data_querier_index, _stats, key, s, range, slice, std::move(trace_state));
|
||||
}
|
||||
|
||||
std::optional<mutation_querier> querier_cache::lookup_mutation_querier(utils::UUID key,
|
||||
@@ -321,7 +352,7 @@ std::optional<mutation_querier> querier_cache::lookup_mutation_querier(utils::UU
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
tracing::trace_state_ptr trace_state) {
|
||||
return lookup_querier<mutation_querier>(_entries, _mutation_querier_index, _stats, key, s, range, slice, std::move(trace_state));
|
||||
return lookup_querier<mutation_querier>(_sem, _entries, _mutation_querier_index, _stats, key, s, range, slice, std::move(trace_state));
|
||||
}
|
||||
|
||||
std::optional<shard_mutation_querier> querier_cache::lookup_shard_mutation_querier(utils::UUID key,
|
||||
@@ -329,7 +360,8 @@ std::optional<shard_mutation_querier> querier_cache::lookup_shard_mutation_queri
|
||||
const dht::partition_range_vector& ranges,
|
||||
const query::partition_slice& slice,
|
||||
tracing::trace_state_ptr trace_state) {
|
||||
return lookup_querier<shard_mutation_querier>(_entries, _shard_mutation_querier_index, _stats, key, s, ranges, slice, std::move(trace_state));
|
||||
return lookup_querier<shard_mutation_querier>(_sem, _entries, _shard_mutation_querier_index, _stats, key, s, ranges, slice,
|
||||
std::move(trace_state));
|
||||
}
|
||||
|
||||
void querier_cache::set_entry_ttl(std::chrono::seconds entry_ttl) {
|
||||
|
||||
17
querier.hh
17
querier.hh
@@ -291,9 +291,8 @@ public:
|
||||
/// Inserted queriers will have a TTL. When this expires the querier is
|
||||
/// evicted. This is to avoid excess and unnecessary resource usage due to
|
||||
/// abandoned queriers.
|
||||
/// Provides a way to evict readers one-by-one via `evict_one()`. This can be
|
||||
/// used by the concurrency-limiting code to evict cached readers to free up
|
||||
/// resources for admitting new ones.
|
||||
/// Registers cached readers with the reader concurrency semaphore, as inactive
|
||||
/// readers, so the latter can evict them if needed.
|
||||
/// Keeps the total memory consumption of cached queriers
|
||||
/// below max_queriers_memory_usage by evicting older entries upon inserting
|
||||
/// new ones if the the memory consupmtion would go above the limit.
|
||||
@@ -327,6 +326,7 @@ public:
|
||||
const utils::UUID _key;
|
||||
const lowres_clock::time_point _expires;
|
||||
std::variant<data_querier, mutation_querier, shard_mutation_querier> _value;
|
||||
std::optional<reader_concurrency_semaphore::inactive_read_handle> _handle;
|
||||
|
||||
public:
|
||||
template <typename Querier>
|
||||
@@ -344,6 +344,14 @@ public:
|
||||
_pos = pos;
|
||||
}
|
||||
|
||||
void set_inactive_handle(reader_concurrency_semaphore::inactive_read_handle handle) {
|
||||
_handle = std::move(handle);
|
||||
}
|
||||
|
||||
reader_concurrency_semaphore::inactive_read_handle get_inactive_handle() const {
|
||||
return *_handle;
|
||||
}
|
||||
|
||||
const utils::UUID& key() const {
|
||||
return _key;
|
||||
}
|
||||
@@ -391,6 +399,7 @@ public:
|
||||
boost::intrusive::constant_time_size<false>>;
|
||||
|
||||
private:
|
||||
reader_concurrency_semaphore& _sem;
|
||||
entries _entries;
|
||||
index _data_querier_index;
|
||||
index _mutation_querier_index;
|
||||
@@ -403,7 +412,7 @@ private:
|
||||
void scan_cache_entries();
|
||||
|
||||
public:
|
||||
explicit querier_cache(size_t max_cache_size = 1'000'000, std::chrono::seconds entry_ttl = default_entry_ttl);
|
||||
explicit querier_cache(reader_concurrency_semaphore& sem, size_t max_cache_size = 1'000'000, std::chrono::seconds entry_ttl = default_entry_ttl);
|
||||
|
||||
querier_cache(const querier_cache&) = delete;
|
||||
querier_cache& operator=(const querier_cache&) = delete;
|
||||
|
||||
@@ -137,6 +137,9 @@ public:
|
||||
const clustering_row_ranges& row_ranges(const schema&, const partition_key&) const;
|
||||
void set_range(const schema&, const partition_key&, clustering_row_ranges);
|
||||
void clear_range(const schema&, const partition_key&);
|
||||
void clear_ranges() {
|
||||
_specific_ranges = nullptr;
|
||||
}
|
||||
// FIXME: possibly make this function return a const ref instead.
|
||||
clustering_row_ranges get_all_ranges() const;
|
||||
|
||||
|
||||
171
reader_concurrency_semaphore.cc
Normal file
171
reader_concurrency_semaphore.cc
Normal file
@@ -0,0 +1,171 @@
|
||||
/*
|
||||
* Copyright (C) 2018 ScyllaDB
|
||||
*/
|
||||
|
||||
/*
|
||||
* This file is part of Scylla.
|
||||
*
|
||||
* Scylla is free software: you can redistribute it and/or modify
|
||||
* it under the terms of the GNU Affero General Public License as published by
|
||||
* the Free Software Foundation, either version 3 of the License, or
|
||||
* (at your option) any later version.
|
||||
*
|
||||
* Scylla is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with Scylla. If not, see <http://www.gnu.org/licenses/>.
|
||||
*/
|
||||
|
||||
#include <seastar/core/reactor.hh>
|
||||
|
||||
#include "reader_concurrency_semaphore.hh"
|
||||
|
||||
void reader_concurrency_semaphore::signal(const resources& r) {
|
||||
_resources += r;
|
||||
while (!_wait_list.empty() && has_available_units(_wait_list.front().res)) {
|
||||
auto& x = _wait_list.front();
|
||||
_resources -= x.res;
|
||||
x.pr.set_value(make_lw_shared<reader_permit>(*this, x.res));
|
||||
_wait_list.pop_front();
|
||||
}
|
||||
}
|
||||
|
||||
reader_concurrency_semaphore::inactive_read_handle reader_concurrency_semaphore::register_inactive_read(std::unique_ptr<inactive_read> ir) {
|
||||
// Implies _inactive_reads.empty(), we don't queue new readers before
|
||||
// evicting all inactive reads.
|
||||
if (_wait_list.empty()) {
|
||||
const auto [it, _] = _inactive_reads.emplace(_next_id++, std::move(ir));
|
||||
(void)_;
|
||||
return inactive_read_handle(it->first);
|
||||
}
|
||||
|
||||
// The evicted reader will release its permit, hopefully allowing us to
|
||||
// admit some readers from the _wait_list.
|
||||
ir->evict();
|
||||
return inactive_read_handle();
|
||||
}
|
||||
|
||||
std::unique_ptr<reader_concurrency_semaphore::inactive_read> reader_concurrency_semaphore::unregister_inactive_read(inactive_read_handle irh) {
|
||||
if (auto it = _inactive_reads.find(irh._id); it != _inactive_reads.end()) {
|
||||
auto ir = std::move(it->second);
|
||||
_inactive_reads.erase(it);
|
||||
return ir;
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
future<lw_shared_ptr<reader_concurrency_semaphore::reader_permit>> reader_concurrency_semaphore::wait_admission(size_t memory,
|
||||
db::timeout_clock::time_point timeout) {
|
||||
if (_wait_list.size() >= _max_queue_length) {
|
||||
return make_exception_future<lw_shared_ptr<reader_permit>>(_make_queue_overloaded_exception());
|
||||
}
|
||||
auto r = resources(1, static_cast<ssize_t>(memory));
|
||||
auto it = _inactive_reads.begin();
|
||||
while (!may_proceed(r) && it != _inactive_reads.end()) {
|
||||
auto ir = std::move(it->second);
|
||||
it = _inactive_reads.erase(it);
|
||||
ir->evict();
|
||||
}
|
||||
if (may_proceed(r)) {
|
||||
_resources -= r;
|
||||
return make_ready_future<lw_shared_ptr<reader_permit>>(make_lw_shared<reader_permit>(*this, r));
|
||||
}
|
||||
promise<lw_shared_ptr<reader_permit>> pr;
|
||||
auto fut = pr.get_future();
|
||||
_wait_list.push_back(entry(std::move(pr), r), timeout);
|
||||
return fut;
|
||||
}
|
||||
|
||||
// A file that tracks the memory usage of buffers resulting from read
|
||||
// operations.
|
||||
class tracking_file_impl : public file_impl {
|
||||
file _tracked_file;
|
||||
lw_shared_ptr<reader_concurrency_semaphore::reader_permit> _permit;
|
||||
|
||||
// Shouldn't be called if semaphore is NULL.
|
||||
temporary_buffer<uint8_t> make_tracked_buf(temporary_buffer<uint8_t> buf) {
|
||||
return seastar::temporary_buffer<uint8_t>(buf.get_write(),
|
||||
buf.size(),
|
||||
make_deleter(buf.release(), std::bind(&reader_concurrency_semaphore::reader_permit::signal_memory, _permit, buf.size())));
|
||||
}
|
||||
|
||||
public:
|
||||
tracking_file_impl(file file, reader_resource_tracker resource_tracker)
|
||||
: _tracked_file(std::move(file))
|
||||
, _permit(resource_tracker.get_permit()) {
|
||||
}
|
||||
|
||||
tracking_file_impl(const tracking_file_impl&) = delete;
|
||||
tracking_file_impl& operator=(const tracking_file_impl&) = delete;
|
||||
tracking_file_impl(tracking_file_impl&&) = default;
|
||||
tracking_file_impl& operator=(tracking_file_impl&&) = default;
|
||||
|
||||
virtual future<size_t> write_dma(uint64_t pos, const void* buffer, size_t len, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->write_dma(pos, buffer, len, pc);
|
||||
}
|
||||
|
||||
virtual future<size_t> write_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->write_dma(pos, std::move(iov), pc);
|
||||
}
|
||||
|
||||
virtual future<size_t> read_dma(uint64_t pos, void* buffer, size_t len, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->read_dma(pos, buffer, len, pc);
|
||||
}
|
||||
|
||||
virtual future<size_t> read_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->read_dma(pos, iov, pc);
|
||||
}
|
||||
|
||||
virtual future<> flush(void) override {
|
||||
return get_file_impl(_tracked_file)->flush();
|
||||
}
|
||||
|
||||
virtual future<struct stat> stat(void) override {
|
||||
return get_file_impl(_tracked_file)->stat();
|
||||
}
|
||||
|
||||
virtual future<> truncate(uint64_t length) override {
|
||||
return get_file_impl(_tracked_file)->truncate(length);
|
||||
}
|
||||
|
||||
virtual future<> discard(uint64_t offset, uint64_t length) override {
|
||||
return get_file_impl(_tracked_file)->discard(offset, length);
|
||||
}
|
||||
|
||||
virtual future<> allocate(uint64_t position, uint64_t length) override {
|
||||
return get_file_impl(_tracked_file)->allocate(position, length);
|
||||
}
|
||||
|
||||
virtual future<uint64_t> size(void) override {
|
||||
return get_file_impl(_tracked_file)->size();
|
||||
}
|
||||
|
||||
virtual future<> close() override {
|
||||
return get_file_impl(_tracked_file)->close();
|
||||
}
|
||||
|
||||
virtual std::unique_ptr<file_handle_impl> dup() override {
|
||||
return get_file_impl(_tracked_file)->dup();
|
||||
}
|
||||
|
||||
virtual subscription<directory_entry> list_directory(std::function<future<> (directory_entry de)> next) override {
|
||||
return get_file_impl(_tracked_file)->list_directory(std::move(next));
|
||||
}
|
||||
|
||||
virtual future<temporary_buffer<uint8_t>> dma_read_bulk(uint64_t offset, size_t range_size, const io_priority_class& pc) override {
|
||||
return get_file_impl(_tracked_file)->dma_read_bulk(offset, range_size, pc).then([this] (temporary_buffer<uint8_t> buf) {
|
||||
if (_permit) {
|
||||
buf = make_tracked_buf(std::move(buf));
|
||||
_permit->consume_memory(buf.size());
|
||||
}
|
||||
return make_ready_future<temporary_buffer<uint8_t>>(std::move(buf));
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
file reader_resource_tracker::track(file f) const {
|
||||
return file(make_shared<tracking_file_impl>(f, *this));
|
||||
}
|
||||
@@ -21,10 +21,14 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <map>
|
||||
#include <seastar/core/file.hh>
|
||||
#include <seastar/core/future.hh>
|
||||
#include <seastar/core/semaphore.hh>
|
||||
#include "db/timeout_clock.hh"
|
||||
|
||||
using namespace seastar;
|
||||
|
||||
/// Specific semaphore for controlling reader concurrency
|
||||
///
|
||||
/// Before creating a reader one should obtain a permit by calling
|
||||
@@ -109,13 +113,24 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
class inactive_read {
|
||||
public:
|
||||
virtual void evict() = 0;
|
||||
virtual ~inactive_read() = default;
|
||||
};
|
||||
|
||||
class inactive_read_handle {
|
||||
uint64_t _id = 0;
|
||||
|
||||
friend class reader_concurrency_semaphore;
|
||||
|
||||
inactive_read_handle() = default;
|
||||
explicit inactive_read_handle(uint64_t id)
|
||||
: _id(id) {
|
||||
}
|
||||
};
|
||||
|
||||
private:
|
||||
static std::exception_ptr default_make_queue_overloaded_exception() {
|
||||
return std::make_exception_ptr(std::runtime_error("restricted mutation reader queue overload"));
|
||||
}
|
||||
|
||||
resources _resources;
|
||||
|
||||
struct entry {
|
||||
promise<lw_shared_ptr<reader_permit>> pr;
|
||||
resources res;
|
||||
@@ -126,11 +141,21 @@ private:
|
||||
e.pr.set_exception(semaphore_timed_out());
|
||||
}
|
||||
};
|
||||
|
||||
private:
|
||||
resources _resources;
|
||||
|
||||
expiring_fifo<entry, expiry_handler, db::timeout_clock> _wait_list;
|
||||
|
||||
size_t _max_queue_length = std::numeric_limits<size_t>::max();
|
||||
std::function<std::exception_ptr()> _make_queue_overloaded_exception = default_make_queue_overloaded_exception;
|
||||
std::function<bool()> _evict_an_inactive_reader;
|
||||
std::function<std::exception_ptr()> _make_queue_overloaded_exception;
|
||||
uint64_t _next_id = 1;
|
||||
std::map<uint64_t, std::unique_ptr<inactive_read>> _inactive_reads;
|
||||
|
||||
private:
|
||||
static std::exception_ptr default_make_queue_overloaded_exception() {
|
||||
return std::make_exception_ptr(std::runtime_error("restricted mutation reader queue overload"));
|
||||
}
|
||||
|
||||
bool has_available_units(const resources& r) const {
|
||||
return bool(_resources) && _resources >= r;
|
||||
@@ -153,12 +178,10 @@ public:
|
||||
reader_concurrency_semaphore(unsigned count,
|
||||
size_t memory,
|
||||
size_t max_queue_length = std::numeric_limits<size_t>::max(),
|
||||
std::function<std::exception_ptr()> raise_queue_overloaded_exception = default_make_queue_overloaded_exception,
|
||||
std::function<bool()> evict_an_inactive_reader = {})
|
||||
std::function<std::exception_ptr()> raise_queue_overloaded_exception = default_make_queue_overloaded_exception)
|
||||
: _resources(count, memory)
|
||||
, _max_queue_length(max_queue_length)
|
||||
, _make_queue_overloaded_exception(raise_queue_overloaded_exception)
|
||||
, _evict_an_inactive_reader(std::move(evict_an_inactive_reader)) {
|
||||
, _make_queue_overloaded_exception(raise_queue_overloaded_exception) {
|
||||
}
|
||||
|
||||
reader_concurrency_semaphore(const reader_concurrency_semaphore&) = delete;
|
||||
@@ -167,6 +190,29 @@ public:
|
||||
reader_concurrency_semaphore(reader_concurrency_semaphore&&) = delete;
|
||||
reader_concurrency_semaphore& operator=(reader_concurrency_semaphore&&) = delete;
|
||||
|
||||
/// Register an inactive read.
|
||||
///
|
||||
/// The semaphore will evict this read when there is a shortage of
|
||||
/// permits. This might be immediate, during this register call.
|
||||
/// Clients can use the returned handle to unregister the read, when it
|
||||
/// stops being inactive and hence evictable.
|
||||
///
|
||||
/// An inactive read is an object implementing the `inactive_read`
|
||||
/// interface.
|
||||
/// The semaphore takes ownership of the created object and destroys it if
|
||||
/// it is evicted.
|
||||
inactive_read_handle register_inactive_read(std::unique_ptr<inactive_read> ir);
|
||||
|
||||
/// Unregister the previously registered inactive read.
|
||||
///
|
||||
/// If the read was not evicted, the inactive read object, passed in to the
|
||||
/// register call, will be returned. Otherwise a nullptr is returned.
|
||||
std::unique_ptr<inactive_read> unregister_inactive_read(inactive_read_handle irh);
|
||||
|
||||
void clear_inactive_reads() {
|
||||
_inactive_reads.clear();
|
||||
}
|
||||
|
||||
future<lw_shared_ptr<reader_permit>> wait_admission(size_t memory, db::timeout_clock::time_point timeout = db::no_timeout);
|
||||
|
||||
const resources available_resources() const {
|
||||
|
||||
@@ -164,10 +164,13 @@ static std::vector<mutation> read_all_partitions_one_by_one(distributed<database
|
||||
return results;
|
||||
}
|
||||
|
||||
using stateful_query = bool_class<class stateful>;
|
||||
|
||||
static std::pair<std::vector<mutation>, size_t>
|
||||
read_all_partitions_with_paged_scan(distributed<database>& db, schema_ptr s, size_t page_size, const std::function<void(size_t)>& page_hook) {
|
||||
read_all_partitions_with_paged_scan(distributed<database>& db, schema_ptr s, size_t page_size, stateful_query is_stateful,
|
||||
const std::function<void(size_t)>& page_hook) {
|
||||
const auto max_size = std::numeric_limits<uint64_t>::max();
|
||||
const auto query_uuid = utils::make_random_uuid();
|
||||
const auto query_uuid = is_stateful ? utils::make_random_uuid() : utils::UUID{};
|
||||
std::vector<mutation> results;
|
||||
auto cmd = query::read_command(s->id(), s->version(), s->full_slice(), page_size, gc_clock::now(), stdx::nullopt, query::max_partitions,
|
||||
query_uuid, true);
|
||||
@@ -188,7 +191,9 @@ read_all_partitions_with_paged_scan(distributed<database>& db, schema_ptr s, siz
|
||||
// Loop until a page turns up with less rows than the limit.
|
||||
while (nrows == page_size) {
|
||||
page_hook(npages);
|
||||
BOOST_REQUIRE(aggregate_querier_cache_stat(db, &query::querier_cache::stats::lookups) >= npages);
|
||||
if (is_stateful) {
|
||||
BOOST_REQUIRE(aggregate_querier_cache_stat(db, &query::querier_cache::stats::lookups) >= npages);
|
||||
}
|
||||
|
||||
const auto& last_pkey = results.back().decorated_key();
|
||||
const auto& last_ckey = results.back().partition().clustered_rows().rbegin()->key();
|
||||
@@ -259,8 +264,8 @@ SEASTAR_THREAD_TEST_CASE(test_read_all) {
|
||||
// First read all partition-by-partition (not paged).
|
||||
auto results1 = read_all_partitions_one_by_one(env.db(), s, pkeys);
|
||||
|
||||
// Then do a paged range-query
|
||||
auto results2 = read_all_partitions_with_paged_scan(env.db(), s, 4, [&] (size_t) {
|
||||
// Then do a paged range-query, with reader caching
|
||||
auto results2 = read_all_partitions_with_paged_scan(env.db(), s, 4, stateful_query::yes, [&] (size_t) {
|
||||
check_cache_population(env.db(), 1);
|
||||
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::drops), 0);
|
||||
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::misses), 0);
|
||||
@@ -276,6 +281,13 @@ SEASTAR_THREAD_TEST_CASE(test_read_all) {
|
||||
|
||||
require_eventually_empty_caches(env.db());
|
||||
|
||||
// Then do a paged range-query, without reader caching
|
||||
auto results3 = read_all_partitions_with_paged_scan(env.db(), s, 4, stateful_query::no, [&] (size_t) {
|
||||
check_cache_population(env.db(), 0);
|
||||
}).first;
|
||||
|
||||
check_results_are_equal(results1, results3);
|
||||
|
||||
return make_ready_future<>();
|
||||
}).get();
|
||||
}
|
||||
@@ -295,7 +307,7 @@ SEASTAR_THREAD_TEST_CASE(test_evict_a_shard_reader_on_each_page) {
|
||||
auto results1 = read_all_partitions_one_by_one(env.db(), s, pkeys);
|
||||
|
||||
// Then do a paged range-query
|
||||
auto [results2, npages] = read_all_partitions_with_paged_scan(env.db(), s, 4, [&] (size_t page) {
|
||||
auto [results2, npages] = read_all_partitions_with_paged_scan(env.db(), s, 4, stateful_query::yes, [&] (size_t page) {
|
||||
check_cache_population(env.db(), 1);
|
||||
|
||||
env.db().invoke_on(page % smp::count, [&] (database& db) {
|
||||
@@ -334,7 +346,7 @@ SEASTAR_THREAD_TEST_CASE(test_range_tombstones) {
|
||||
auto results1 = read_all_partitions_one_by_one(env.db(), s, pkeys);
|
||||
|
||||
// Then do a paged range-query
|
||||
auto results2 = read_all_partitions_with_paged_scan(env.db(), s, 4, [&] (size_t page) {
|
||||
auto results2 = read_all_partitions_with_paged_scan(env.db(), s, 4, stateful_query::yes, [&] (size_t page) {
|
||||
check_cache_population(env.db(), 1);
|
||||
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::drops), 0);
|
||||
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::misses), 0);
|
||||
|
||||
@@ -20,6 +20,8 @@
|
||||
*/
|
||||
|
||||
|
||||
#include <random>
|
||||
|
||||
#include <boost/test/unit_test.hpp>
|
||||
#include <boost/range/irange.hpp>
|
||||
#include <boost/range/adaptor/uniqued.hpp>
|
||||
@@ -1514,6 +1516,125 @@ dht::token dummy_partitioner::token_for_next_shard(const dht::token& t, shard_id
|
||||
return *it;
|
||||
}
|
||||
|
||||
class test_reader_lifecycle_policy
|
||||
: public reader_lifecycle_policy
|
||||
, public enable_shared_from_this<test_reader_lifecycle_policy> {
|
||||
public:
|
||||
enum class operation {
|
||||
none,
|
||||
create,
|
||||
pause,
|
||||
try_resume,
|
||||
};
|
||||
|
||||
using delay_function = std::function<future<>()>;
|
||||
|
||||
static future<> no_delay() {
|
||||
return make_ready_future<>();
|
||||
}
|
||||
|
||||
private:
|
||||
using factory_function = std::function<future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>>(
|
||||
shard_id,
|
||||
schema_ptr,
|
||||
const dht::partition_range&,
|
||||
const query::partition_slice&,
|
||||
const io_priority_class&,
|
||||
tracing::trace_state_ptr,
|
||||
mutation_reader::forwarding)>;
|
||||
|
||||
struct reader_params {
|
||||
const dht::partition_range range;
|
||||
const query::partition_slice slice;
|
||||
};
|
||||
struct reader_context {
|
||||
std::unique_ptr<reader_params> params;
|
||||
foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader;
|
||||
operation operation_in_progress = operation::none;
|
||||
};
|
||||
|
||||
factory_function _factory_function;
|
||||
delay_function _delay;
|
||||
std::vector<reader_context> _contexts;
|
||||
bool _evict_paused_readers = false;
|
||||
|
||||
private:
|
||||
static sstring to_string(operation op) {
|
||||
switch (op) {
|
||||
case operation::none:
|
||||
return "none";
|
||||
case operation::create:
|
||||
return "create";
|
||||
case operation::pause:
|
||||
return "pause";
|
||||
case operation::try_resume:
|
||||
return "try_resume";
|
||||
}
|
||||
return "unknown";
|
||||
}
|
||||
|
||||
void set_current_operation(shard_id shard, operation new_operation) {
|
||||
BOOST_CHECK_MESSAGE(_contexts[shard].operation_in_progress == operation::none,
|
||||
sprint("%s(): concurrent operation detected: `%s` initiated but `%s` is still pending", __FUNCTION__,
|
||||
to_string(_contexts[shard].operation_in_progress), to_string(new_operation)));
|
||||
_contexts[shard].operation_in_progress = new_operation;
|
||||
}
|
||||
|
||||
public:
|
||||
explicit test_reader_lifecycle_policy(factory_function f, delay_function delay_func = no_delay, bool evict_paused_readers = false)
|
||||
: _factory_function(std::move(f))
|
||||
, _delay(std::move(delay_func))
|
||||
, _contexts(smp::count)
|
||||
, _evict_paused_readers(evict_paused_readers) {
|
||||
}
|
||||
virtual future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>> create_reader(
|
||||
shard_id shard,
|
||||
schema_ptr schema,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
mutation_reader::forwarding fwd_mr) override {
|
||||
set_current_operation(shard, operation::create);
|
||||
|
||||
_contexts[shard].params = std::make_unique<reader_params>(reader_params{range, slice});
|
||||
return _factory_function(shard, std::move(schema), _contexts[shard].params->range, _contexts[shard].params->slice, pc,
|
||||
std::move(trace_state), fwd_mr).finally([this, zis = shared_from_this(), shard] {
|
||||
_contexts[shard].operation_in_progress = operation::none;
|
||||
});
|
||||
}
|
||||
virtual void destroy_reader(shard_id shard, future<paused_or_stopped_reader> reader) noexcept override {
|
||||
reader.then([shard, this] (paused_or_stopped_reader&& reader) {
|
||||
return smp::submit_to(shard, [reader = std::move(reader.remote_reader), ctx = std::move(_contexts[shard])] () mutable {
|
||||
reader.release();
|
||||
});
|
||||
}).finally([zis = shared_from_this()] {});
|
||||
}
|
||||
virtual future<> pause(foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader) override {
|
||||
const auto shard = reader.get_owner_shard();
|
||||
|
||||
set_current_operation(shard, operation::pause);
|
||||
|
||||
return _delay().then([this, shard, reader = std::move(reader)] () mutable {
|
||||
_contexts[shard].reader = std::move(reader);
|
||||
}).finally([this, zis = shared_from_this(), shard] {
|
||||
_contexts[shard].operation_in_progress = operation::none;
|
||||
});
|
||||
}
|
||||
virtual future<foreign_ptr<std::unique_ptr<flat_mutation_reader>>> try_resume(shard_id shard) override {
|
||||
set_current_operation(shard, operation::try_resume);
|
||||
|
||||
return _delay().then([this, shard] {
|
||||
if (_evict_paused_readers) {
|
||||
_contexts[shard].reader.reset();
|
||||
}
|
||||
return std::move(_contexts[shard].reader);
|
||||
}).finally([this, zis = shared_from_this(), shard] {
|
||||
_contexts[shard].operation_in_progress = operation::none;
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
// Best run with SMP >= 2
|
||||
SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_as_mutation_source) {
|
||||
if (smp::count < 2) {
|
||||
@@ -1522,66 +1643,83 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_as_mutation_source) {
|
||||
}
|
||||
|
||||
do_with_cql_env([] (cql_test_env& env) -> future<> {
|
||||
auto populate = [] (schema_ptr s, const std::vector<mutation>& mutations) {
|
||||
// We need to group mutations that have the same token so they land on the same shard.
|
||||
std::map<dht::token, std::vector<mutation>> mutations_by_token;
|
||||
auto make_populate = [] (test_reader_lifecycle_policy::delay_function delay, bool evict_paused_readers) {
|
||||
return [delay = std::move(delay), evict_paused_readers] (schema_ptr s, const std::vector<mutation>& mutations) mutable {
|
||||
// We need to group mutations that have the same token so they land on the same shard.
|
||||
std::map<dht::token, std::vector<mutation>> mutations_by_token;
|
||||
|
||||
for (const auto& mut : mutations) {
|
||||
mutations_by_token[mut.token()].push_back(mut);
|
||||
}
|
||||
for (const auto& mut : mutations) {
|
||||
mutations_by_token[mut.token()].push_back(mut);
|
||||
}
|
||||
|
||||
auto partitioner = make_lw_shared<dummy_partitioner>(dht::global_partitioner(), mutations_by_token);
|
||||
auto partitioner = make_lw_shared<dummy_partitioner>(dht::global_partitioner(), mutations_by_token);
|
||||
|
||||
auto merged_mutations = boost::copy_range<std::vector<std::vector<mutation>>>(mutations_by_token | boost::adaptors::map_values);
|
||||
auto merged_mutations = boost::copy_range<std::vector<std::vector<mutation>>>(mutations_by_token | boost::adaptors::map_values);
|
||||
|
||||
auto remote_memtables = make_lw_shared<std::vector<foreign_ptr<lw_shared_ptr<memtable>>>>();
|
||||
for (unsigned shard = 0; shard < partitioner->shard_count(); ++shard) {
|
||||
auto remote_mt = smp::submit_to(shard, [shard, s = global_schema_ptr(s), &merged_mutations, partitioner = *partitioner] {
|
||||
auto mt = make_lw_shared<memtable>(s.get());
|
||||
auto remote_memtables = make_lw_shared<std::vector<foreign_ptr<lw_shared_ptr<memtable>>>>();
|
||||
for (unsigned shard = 0; shard < partitioner->shard_count(); ++shard) {
|
||||
auto remote_mt = smp::submit_to(shard, [shard, s = global_schema_ptr(s), &merged_mutations, partitioner = *partitioner] {
|
||||
auto mt = make_lw_shared<memtable>(s.get());
|
||||
|
||||
for (unsigned i = shard; i < merged_mutations.size(); i += partitioner.shard_count()) {
|
||||
for (auto& mut : merged_mutations[i]) {
|
||||
mt->apply(mut);
|
||||
for (unsigned i = shard; i < merged_mutations.size(); i += partitioner.shard_count()) {
|
||||
for (auto& mut : merged_mutations[i]) {
|
||||
mt->apply(mut);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return make_foreign(mt);
|
||||
}).get0();
|
||||
remote_memtables->emplace_back(std::move(remote_mt));
|
||||
}
|
||||
return make_foreign(mt);
|
||||
}).get0();
|
||||
remote_memtables->emplace_back(std::move(remote_mt));
|
||||
}
|
||||
|
||||
return mutation_source([partitioner, remote_memtables] (schema_ptr s,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr) mutable {
|
||||
auto factory = [remote_memtables] (unsigned shard,
|
||||
schema_ptr s,
|
||||
return mutation_source([&delay, partitioner, remote_memtables, evict_paused_readers] (schema_ptr s,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return smp::submit_to(shard, [mt = &*remote_memtables->at(shard), s = global_schema_ptr(s), &range, &slice, &pc,
|
||||
trace_state = tracing::global_trace_state_ptr(trace_state), fwd_sm, fwd_mr] () mutable {
|
||||
return make_foreign(std::make_unique<flat_mutation_reader>(mt->make_flat_reader(s.get(),
|
||||
range,
|
||||
slice,
|
||||
pc,
|
||||
trace_state.get(),
|
||||
fwd_sm,
|
||||
fwd_mr)));
|
||||
});
|
||||
};
|
||||
mutation_reader::forwarding fwd_mr) mutable {
|
||||
auto factory = [remote_memtables] (unsigned shard,
|
||||
schema_ptr s,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return smp::submit_to(shard, [mt = &*remote_memtables->at(shard), s = global_schema_ptr(s), &range, &slice, &pc,
|
||||
trace_state = tracing::global_trace_state_ptr(trace_state), fwd_mr] () mutable {
|
||||
return make_foreign(std::make_unique<flat_mutation_reader>(mt->make_flat_reader(s.get(),
|
||||
range,
|
||||
slice,
|
||||
pc,
|
||||
trace_state.get(),
|
||||
streamed_mutation::forwarding::no,
|
||||
fwd_mr)));
|
||||
});
|
||||
};
|
||||
|
||||
return make_multishard_combining_reader(s, range, slice, pc, *partitioner, factory, trace_state, fwd_sm, fwd_mr);
|
||||
});
|
||||
auto lifecycle_policy = seastar::make_shared<test_reader_lifecycle_policy>(std::move(factory), delay, evict_paused_readers);
|
||||
return make_multishard_combining_reader(std::move(lifecycle_policy), *partitioner, s, range, slice, pc, trace_state, fwd_mr);
|
||||
});
|
||||
};
|
||||
};
|
||||
|
||||
run_mutation_source_tests(populate);
|
||||
auto make_random_delay = [] (int from, int to) {
|
||||
return [gen = std::default_random_engine(std::random_device()()),
|
||||
dist = std::uniform_int_distribution(from, to)] () mutable {
|
||||
return seastar::sleep(std::chrono::milliseconds(dist(gen)));
|
||||
};
|
||||
};
|
||||
|
||||
BOOST_TEST_MESSAGE("run_mutation_source_tests(delay=no_delay, evict_readers=false)");
|
||||
run_mutation_source_tests(make_populate(test_reader_lifecycle_policy::no_delay, false), streamed_mutation::forwarding::no);
|
||||
|
||||
BOOST_TEST_MESSAGE("run_mutation_source_tests(delay=random, evict_readers=false)");
|
||||
run_mutation_source_tests(make_populate(make_random_delay(1, 10), false), streamed_mutation::forwarding::no);
|
||||
|
||||
BOOST_TEST_MESSAGE("run_mutation_source_tests(delay=random, evict_readers=true)");
|
||||
run_mutation_source_tests(make_populate(make_random_delay(1, 10), true), streamed_mutation::forwarding::no);
|
||||
|
||||
return make_ready_future<>();
|
||||
}).get();
|
||||
}
|
||||
@@ -1602,7 +1740,6 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_reading_empty_table) {
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
shards_touched[shard] = true;
|
||||
return smp::submit_to(shard, [gs = global_schema_ptr(s)] () mutable {
|
||||
@@ -1610,8 +1747,13 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_reading_empty_table) {
|
||||
});
|
||||
};
|
||||
|
||||
assert_that(make_multishard_combining_reader(s.schema(), query::full_partition_range, s.schema()->full_slice(),
|
||||
service::get_local_sstable_query_read_priority(), dht::global_partitioner(), std::move(factory)))
|
||||
assert_that(make_multishard_combining_reader(
|
||||
seastar::make_shared<test_reader_lifecycle_policy>(std::move(factory)),
|
||||
dht::global_partitioner(),
|
||||
s.schema(),
|
||||
query::full_partition_range,
|
||||
s.schema()->full_slice(),
|
||||
service::get_local_sstable_query_read_priority()))
|
||||
.produces_end_of_stream();
|
||||
|
||||
for (unsigned i = 0; i < smp::count; ++i) {
|
||||
@@ -1722,7 +1864,6 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return smp::submit_to(shard, [shard_of_interest, gs = global_simple_schema(s), remote_control] () mutable {
|
||||
if (engine().cpu_id() == shard_of_interest) {
|
||||
@@ -1741,8 +1882,8 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
|
||||
{
|
||||
const auto mutations_by_token = std::map<dht::token, std::vector<mutation>>();
|
||||
auto partitioner = dummy_partitioner(dht::global_partitioner(), mutations_by_token);
|
||||
auto reader = make_multishard_combining_reader(s.schema(), query::full_partition_range, s.schema()->full_slice(),
|
||||
service::get_local_sstable_query_read_priority(), partitioner, std::move(factory));
|
||||
auto reader = make_multishard_combining_reader(seastar::make_shared<test_reader_lifecycle_policy>(std::move(factory)), partitioner,
|
||||
s.schema(), query::full_partition_range, s.schema()->full_slice(), service::get_local_sstable_query_read_priority());
|
||||
|
||||
reader.fill_buffer(db::no_timeout).get();
|
||||
|
||||
@@ -1984,7 +2125,6 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding,
|
||||
mutation_reader::forwarding) {
|
||||
return smp::submit_to(shard, [shard, gs = global_simple_schema(s), remote_control = remote_controls.at(shard).get(),
|
||||
pkeys = shard_pkeys.at(shard)] () mutable {
|
||||
@@ -1995,8 +2135,8 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
|
||||
};
|
||||
|
||||
{
|
||||
auto reader = make_multishard_combining_reader(s.schema(), query::full_partition_range, s.schema()->full_slice(),
|
||||
service::get_local_sstable_query_read_priority(), partitioner, std::move(factory));
|
||||
auto reader = make_multishard_combining_reader(seastar::make_shared<test_reader_lifecycle_policy>(std::move(factory)), partitioner,
|
||||
s.schema(), query::full_partition_range, s.schema()->full_slice(), service::get_local_sstable_query_read_priority());
|
||||
reader.fill_buffer(db::no_timeout).get();
|
||||
BOOST_REQUIRE(reader.is_buffer_full());
|
||||
}
|
||||
@@ -2021,6 +2161,87 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
|
||||
}).get();
|
||||
}
|
||||
|
||||
SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_next_partition) {
|
||||
do_with_cql_env([] (cql_test_env& env) -> future<> {
|
||||
env.execute_cql("CREATE KEYSPACE multishard_combining_reader_next_partition_ks"
|
||||
" WITH REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 1};").get();
|
||||
env.execute_cql("CREATE TABLE multishard_combining_reader_next_partition_ks.test (pk int, v int, PRIMARY KEY(pk));").get();
|
||||
|
||||
const auto insert_id = env.prepare("INSERT INTO multishard_combining_reader_next_partition_ks.test (\"pk\", \"v\") VALUES (?, ?);").get0();
|
||||
|
||||
const auto partition_count = 1000;
|
||||
|
||||
for (int pk = 0; pk < partition_count; ++pk) {
|
||||
env.execute_prepared(insert_id, {{
|
||||
cql3::raw_value::make_value(data_value(pk).serialize()),
|
||||
cql3::raw_value::make_value(data_value(0).serialize())}}).get();
|
||||
}
|
||||
|
||||
auto schema = env.local_db().find_column_family("multishard_combining_reader_next_partition_ks", "test").schema();
|
||||
auto& partitioner = dht::global_partitioner();
|
||||
|
||||
auto pkeys = boost::copy_range<std::vector<dht::decorated_key>>(
|
||||
boost::irange(0, partition_count) |
|
||||
boost::adaptors::transformed([schema, &partitioner] (int i) {
|
||||
return partitioner.decorate_key(*schema, partition_key::from_singular(*schema, i));
|
||||
}));
|
||||
|
||||
// We want to test corner cases around next_partition() called when it
|
||||
// cannot be resolved with just the buffer and it has to be forwarded
|
||||
// to the correct shard reader, so set a buffer size so that only a
|
||||
// single fragment can fit into it at a time.
|
||||
const auto max_buffer_size = size_t{1};
|
||||
|
||||
auto factory = [db = &env.db(), max_buffer_size] (
|
||||
unsigned shard,
|
||||
schema_ptr s,
|
||||
const dht::partition_range& range,
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
mutation_reader::forwarding fwd_mr) {
|
||||
return db->invoke_on(shard, [gs = global_schema_ptr(s), &range, &slice, &pc, trace_state = tracing::global_trace_state_ptr(trace_state),
|
||||
fwd_mr, max_buffer_size] (database& db) mutable {
|
||||
auto schema = gs.get();
|
||||
auto& table = db.find_column_family(schema);
|
||||
auto reader = table.as_mutation_source().make_reader(
|
||||
schema,
|
||||
range,
|
||||
slice,
|
||||
service::get_local_sstable_query_read_priority(),
|
||||
trace_state.get(),
|
||||
streamed_mutation::forwarding::no,
|
||||
fwd_mr);
|
||||
reader.set_max_buffer_size(max_buffer_size);
|
||||
return make_foreign(std::make_unique<flat_mutation_reader>(std::move(reader)));
|
||||
});
|
||||
};
|
||||
auto reader = make_multishard_combining_reader(
|
||||
seastar::make_shared<test_reader_lifecycle_policy>(std::move(factory)),
|
||||
partitioner,
|
||||
schema,
|
||||
query::full_partition_range,
|
||||
schema->full_slice(),
|
||||
service::get_local_sstable_query_read_priority());
|
||||
|
||||
reader.set_max_buffer_size(max_buffer_size);
|
||||
|
||||
boost::sort(pkeys, [schema] (const dht::decorated_key& a, const dht::decorated_key& b) {
|
||||
return dht::ring_position_tri_compare(*schema, a, b) < 0;
|
||||
});
|
||||
|
||||
BOOST_TEST_MESSAGE("Start test");
|
||||
|
||||
auto assertions = assert_that(std::move(reader));
|
||||
for (int i = 0; i < partition_count; ++i) {
|
||||
assertions.produces(pkeys[i]);
|
||||
}
|
||||
assertions.produces_end_of_stream();
|
||||
|
||||
return make_ready_future<>();
|
||||
}).get();
|
||||
}
|
||||
|
||||
// Test the multishard streaming reader in the context it was designed to work
|
||||
// in: as a mean to read data belonging to a shard according to a different
|
||||
// sharding configuration.
|
||||
@@ -2070,21 +2291,21 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_streaming_reader) {
|
||||
const query::partition_slice& slice,
|
||||
const io_priority_class& pc,
|
||||
tracing::trace_state_ptr trace_state,
|
||||
streamed_mutation::forwarding fwd_sm,
|
||||
mutation_reader::forwarding fwd_mr) mutable {
|
||||
return db->invoke_on(shard, [gs = global_schema_ptr(std::move(schema)), &range, &slice,
|
||||
gts = tracing::global_trace_state_ptr(std::move(trace_state)), fwd_sm, fwd_mr] (database& db) {
|
||||
gts = tracing::global_trace_state_ptr(std::move(trace_state)), fwd_mr] (database& db) {
|
||||
auto s = gs.get();
|
||||
auto& table = db.find_column_family(s);
|
||||
//TODO need a way to transport io_priority_calls across shards
|
||||
auto& pc = service::get_local_sstable_query_read_priority();
|
||||
auto reader = table.as_mutation_source().make_reader(std::move(s), range, slice, pc, gts.get(), fwd_sm, fwd_mr);
|
||||
auto reader = table.as_mutation_source().make_reader(std::move(s), range, slice, pc, gts.get(), streamed_mutation::forwarding::no,
|
||||
fwd_mr);
|
||||
return make_foreign(std::make_unique<flat_mutation_reader>(std::move(reader)));
|
||||
});
|
||||
};
|
||||
auto reference_reader = make_filtering_reader(
|
||||
make_multishard_combining_reader(schema, partition_range, schema->full_slice(),
|
||||
service::get_local_sstable_query_read_priority(), local_partitioner, std::move(reader_factory)),
|
||||
make_multishard_combining_reader(seastar::make_shared<test_reader_lifecycle_policy>(std::move(reader_factory)), local_partitioner,
|
||||
schema, partition_range, schema->full_slice(), service::get_local_sstable_query_read_priority()),
|
||||
[&remote_partitioner] (const dht::decorated_key& pkey) {
|
||||
return remote_partitioner.shard_of(pkey.token()) == 0;
|
||||
});
|
||||
|
||||
@@ -1020,7 +1020,7 @@ void test_streamed_mutation_forwarding_succeeds_with_no_data(populate_fn populat
|
||||
}
|
||||
|
||||
static
|
||||
void test_slicing_with_overlapping_range_tombstones(populate_fn populate) {
|
||||
void test_slicing_with_overlapping_range_tombstones(populate_fn populate, streamed_mutation::forwarding fwd_sm) {
|
||||
simple_schema ss;
|
||||
auto s = ss.schema();
|
||||
|
||||
@@ -1058,6 +1058,10 @@ void test_slicing_with_overlapping_range_tombstones(populate_fn populate) {
|
||||
assert_that(result).is_equal_to(m1 + m2, query::clustering_row_ranges({range}));
|
||||
}
|
||||
|
||||
if (!fwd_sm) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Check fast_forward_to()
|
||||
{
|
||||
auto rd = ds.make_reader(s, query::full_partition_range, s->full_slice(), default_priority_class(),
|
||||
@@ -1093,15 +1097,21 @@ void test_slicing_with_overlapping_range_tombstones(populate_fn populate) {
|
||||
}
|
||||
}
|
||||
|
||||
void run_mutation_reader_tests(populate_fn populate) {
|
||||
void run_mutation_reader_tests(populate_fn populate, streamed_mutation::forwarding fwd_sm) {
|
||||
test_date_tiered_clustering_slicing(populate);
|
||||
test_fast_forwarding_across_partitions_to_empty_range(populate);
|
||||
test_clustering_slices(populate);
|
||||
if (fwd_sm) {
|
||||
test_clustering_slices(populate);
|
||||
}
|
||||
test_mutation_reader_fragments_have_monotonic_positions(populate);
|
||||
test_streamed_mutation_forwarding_across_range_tombstones(populate);
|
||||
test_streamed_mutation_forwarding_guarantees(populate);
|
||||
if (fwd_sm) {
|
||||
test_streamed_mutation_forwarding_across_range_tombstones(populate);
|
||||
test_streamed_mutation_forwarding_guarantees(populate);
|
||||
}
|
||||
test_streamed_mutation_slicing_returns_only_relevant_tombstones(populate);
|
||||
test_streamed_mutation_forwarding_is_consistent_with_slicing(populate);
|
||||
if (fwd_sm) {
|
||||
test_streamed_mutation_forwarding_is_consistent_with_slicing(populate);
|
||||
}
|
||||
test_range_queries(populate);
|
||||
test_query_only_static_row(populate);
|
||||
test_query_no_clustering_ranges_no_static_columns(populate);
|
||||
@@ -1141,15 +1151,17 @@ void test_next_partition(populate_fn populate) {
|
||||
.produces_end_of_stream();
|
||||
}
|
||||
|
||||
void run_flat_mutation_reader_tests(populate_fn populate) {
|
||||
void run_flat_mutation_reader_tests(populate_fn populate, streamed_mutation::forwarding fwd_sm) {
|
||||
test_next_partition(populate);
|
||||
test_streamed_mutation_forwarding_succeeds_with_no_data(populate);
|
||||
test_slicing_with_overlapping_range_tombstones(populate);
|
||||
if (fwd_sm) {
|
||||
test_streamed_mutation_forwarding_succeeds_with_no_data(populate);
|
||||
}
|
||||
test_slicing_with_overlapping_range_tombstones(populate, fwd_sm);
|
||||
}
|
||||
|
||||
void run_mutation_source_tests(populate_fn populate) {
|
||||
run_mutation_reader_tests(populate);
|
||||
run_flat_mutation_reader_tests(populate);
|
||||
void run_mutation_source_tests(populate_fn populate, streamed_mutation::forwarding fwd_sm) {
|
||||
run_mutation_reader_tests(populate, fwd_sm);
|
||||
run_flat_mutation_reader_tests(populate, fwd_sm);
|
||||
}
|
||||
|
||||
struct mutation_sets {
|
||||
|
||||
@@ -27,7 +27,9 @@
|
||||
using populate_fn = std::function<mutation_source(schema_ptr s, const std::vector<mutation>&)>;
|
||||
|
||||
// Must be run in a seastar thread
|
||||
void run_mutation_source_tests(populate_fn populate);
|
||||
// If fwd_sm == no, intra-partition (streamed_mutation) fast-forwarding tests
|
||||
// will be skipped.
|
||||
void run_mutation_source_tests(populate_fn populate, streamed_mutation::forwarding fwd_sm = streamed_mutation::forwarding::yes);
|
||||
|
||||
enum are_equal { no, yes };
|
||||
|
||||
|
||||
@@ -76,6 +76,7 @@ private:
|
||||
query::querier_cache::stats _expected_stats;
|
||||
|
||||
simple_schema _s;
|
||||
reader_concurrency_semaphore _sem;
|
||||
query::querier_cache _cache;
|
||||
const std::vector<mutation> _mutations;
|
||||
const mutation_source _mutation_source;
|
||||
@@ -157,7 +158,8 @@ public:
|
||||
};
|
||||
|
||||
test_querier_cache(const noncopyable_function<sstring(size_t)>& external_make_value, std::chrono::seconds entry_ttl = 24h, size_t cache_size = 100000)
|
||||
: _cache(cache_size, entry_ttl)
|
||||
: _sem(std::numeric_limits<unsigned>::max(), std::numeric_limits<size_t>::max())
|
||||
, _cache(_sem, cache_size, entry_ttl)
|
||||
, _mutations(make_mutations(_s, external_make_value))
|
||||
, _mutation_source([this] (schema_ptr, const dht::partition_range& range) {
|
||||
auto rd = flat_mutation_reader_from_mutations(_mutations, range);
|
||||
@@ -650,25 +652,22 @@ SEASTAR_THREAD_TEST_CASE(test_resources_based_cache_eviction) {
|
||||
nullptr,
|
||||
db::no_timeout).get();
|
||||
|
||||
// Make a fake keyspace just to obtain the configuration and
|
||||
// thus the concurrency semaphore.
|
||||
const auto dummy_ks_metadata = keyspace_metadata("dummy_ks", "SimpleStrategy", {{"replication_factor", "1"}}, false);
|
||||
auto cfg = db.make_keyspace_config(dummy_ks_metadata);
|
||||
auto& semaphore = db.user_read_concurrency_sem();
|
||||
|
||||
BOOST_REQUIRE_EQUAL(db.get_querier_cache_stats().resource_based_evictions, 0);
|
||||
BOOST_CHECK_EQUAL(db.get_querier_cache_stats().resource_based_evictions, 0);
|
||||
|
||||
// Drain all resources of the semaphore
|
||||
std::vector<lw_shared_ptr<reader_concurrency_semaphore::reader_permit>> permits;
|
||||
const auto resources = cfg.read_concurrency_semaphore->available_resources();
|
||||
const auto resources = semaphore.available_resources();
|
||||
permits.reserve(resources.count);
|
||||
const auto per_permit_memory = resources.memory / resources.count;
|
||||
|
||||
for (int i = 0; i < resources.count; ++i) {
|
||||
permits.emplace_back(cfg.read_concurrency_semaphore->wait_admission(per_permit_memory).get0());
|
||||
permits.emplace_back(semaphore.wait_admission(per_permit_memory).get0());
|
||||
}
|
||||
|
||||
BOOST_REQUIRE_EQUAL(cfg.read_concurrency_semaphore->available_resources().count, 0);
|
||||
BOOST_REQUIRE(cfg.read_concurrency_semaphore->available_resources().memory < per_permit_memory);
|
||||
BOOST_CHECK_EQUAL(semaphore.available_resources().count, 0);
|
||||
BOOST_CHECK(semaphore.available_resources().memory < per_permit_memory);
|
||||
|
||||
auto cmd2 = query::read_command(s->id(),
|
||||
s->version(),
|
||||
@@ -687,7 +686,7 @@ SEASTAR_THREAD_TEST_CASE(test_resources_based_cache_eviction) {
|
||||
nullptr,
|
||||
db::no_timeout).get();
|
||||
|
||||
BOOST_REQUIRE_EQUAL(db.get_querier_cache_stats().resource_based_evictions, 1);
|
||||
BOOST_CHECK_EQUAL(db.get_querier_cache_stats().resource_based_evictions, 1);
|
||||
|
||||
// We want to read the entire partition so that the querier
|
||||
// is not saved at the end and thus ensure it is destroyed.
|
||||
|
||||
Reference in New Issue
Block a user