Merge 'Stateful range scans' from Botond

"
This series extends the query statefullness, introduced by f8613a841 to
point queries, to range scans as well. This means that queriers will be
saved and reused for range scans too.
This series builds heavily on the infrastructure introduced by stateful
point queries, namely the querier object and the querier_cache. It also
builds on another critical piece of infrastructure, the
multishard_combining_reader, introduced by 2d126a79b.
To make the range scan on a given node suspendable and resumable we move
away from the current code in
`storage_proxy::query_nonsingular_mutations_locally()` and use a
multishard_combining_reader to execute the read. When the page is filled
this reader is dismantled and its shard readers are saved in the
querier cache.
There are of course a lot more details to it but this is the gist of it.

Tests: unit(release, debug), dtest(paging_test.py, paging_additional_test.py)
"

* '1865/range-scans/v7.1' of https://github.com/denesb/scylla: (33 commits)
  query_pagers: generate query_uuid for range-scans as well
  storage_proxy: use preferred/last replicas
  storage_proxy: add preferred/last replicas to the signature of query_partition_key_range_concurrent
  db::consistency_level::filter_for_query() add preferred_endpoints
  storage_proxy: use query_mutations_from_all_shards() for range scans
  tests: add unit test for multishard_mutation_query()
  tests/mutation_assertions.hh: add missing include
  multishard_mutation_query: add badness counters
  database: add query_mutations_on_all_shards()
  mutation_compactor: add detach_state()
  flat_mutation_reader: add unpop_mutation_fragment()
  Move reconcilable_result_builder declaration to mutation_query.hh
  mutation_source_test: add an additional REQUIRE()
  mutation: add missing assert to mutation from reader
  querier: add shard_mutation_querier
  querier: prepare for multi-ranges
  tests/querier_cache: add tests specific for multiple entry-types
  querier: split querier into separate data and mutation querier types
  querier: move consume_page logic into a free function
  querier: move all matching related logic into free functions
  ...
This commit is contained in:
Duarte Nunes
2018-09-03 09:09:17 +01:00
29 changed files with 2530 additions and 776 deletions

View File

@@ -315,7 +315,7 @@ scylla_tests = [
'tests/fragmented_temporary_buffer_test',
'tests/json_test',
'tests/auth_passwords_test',
'tests/multishard_mutation_query_test',
]
perf_tests = [
@@ -645,6 +645,7 @@ scylla_core = (['database.cc',
'querier.cc',
'data/cell.cc',
'multishard_writer.cc',
'multishard_mutation_query.cc',
]
+ [Antlr3Grammar('cql3/Cql.g')]
+ [Thrift('interface/cassandra.thrift', 'Cassandra')]

View File

@@ -2389,6 +2389,18 @@ database::setup_metrics() {
sm::make_derive("short_mutation_queries", _stats->short_mutation_queries,
sm::description("The rate of mutation queries that returned less rows than requested due to result size limiting.")),
sm::make_derive("multishard_query_unpopped_fragments", _stats->multishard_query_unpopped_fragments,
sm::description("The total number of fragments that were extracted from the shard reader but were unconsumed by the query and moved back into the reader.")),
sm::make_derive("multishard_query_unpopped_bytes", _stats->multishard_query_unpopped_bytes,
sm::description("The total number of bytes that were extracted from the shard reader but were unconsumed by the query and moved back into the reader.")),
sm::make_derive("multishard_query_failed_reader_stops", _stats->multishard_query_failed_reader_stops,
sm::description("The number of times the stopping of a shard reader failed.")),
sm::make_derive("multishard_query_failed_reader_saves", _stats->multishard_query_failed_reader_saves,
sm::description("The number of times the saving of a shard reader failed.")),
sm::make_total_operations("counter_cell_lock_acquisition", _cl_stats->lock_acquisitions,
sm::description("The number of acquired counter cell locks.")),
@@ -3120,7 +3132,7 @@ table::query(schema_ptr s,
query::result_memory_limiter& memory_limiter,
uint64_t max_size,
db::timeout_clock::time_point timeout,
querier_cache_context cache_ctx) {
query::querier_cache_context cache_ctx) {
utils::latency_counter lc;
_stats.reads.set_latency(lc);
auto f = opts.request == query::result_request::only_digest
@@ -3176,7 +3188,7 @@ future<lw_shared_ptr<query::result>, cache_temperature>
database::query(schema_ptr s, const query::read_command& cmd, query::result_options opts, const dht::partition_range_vector& ranges,
tracing::trace_state_ptr trace_state, uint64_t max_result_size, db::timeout_clock::time_point timeout) {
column_family& cf = find_column_family(cmd.cf_id);
querier_cache_context cache_ctx(_querier_cache, cmd.query_uuid, cmd.is_first_page);
query::querier_cache_context cache_ctx(_querier_cache, cmd.query_uuid, cmd.is_first_page);
return _data_query_stage(&cf,
std::move(s),
seastar::cref(cmd),
@@ -3203,7 +3215,7 @@ future<reconcilable_result, cache_temperature>
database::query_mutations(schema_ptr s, const query::read_command& cmd, const dht::partition_range& range,
query::result_memory_accounter&& accounter, tracing::trace_state_ptr trace_state, db::timeout_clock::time_point timeout) {
column_family& cf = find_column_family(cmd.cf_id);
querier_cache_context cache_ctx(_querier_cache, cmd.query_uuid, cmd.is_first_page);
query::querier_cache_context cache_ctx(_querier_cache, cmd.query_uuid, cmd.is_first_page);
return _mutation_query_stage(std::move(s),
cf.as_mutation_source(),
seastar::cref(range),

View File

@@ -688,7 +688,7 @@ public:
query::result_memory_limiter& memory_limiter,
uint64_t max_result_size,
db::timeout_clock::time_point timeout = db::no_timeout,
querier_cache_context cache_ctx = { });
query::querier_cache_context cache_ctx = { });
void start();
future<> stop();
@@ -1167,6 +1167,11 @@ private:
uint64_t short_data_queries = 0;
uint64_t short_mutation_queries = 0;
uint64_t multishard_query_unpopped_fragments = 0;
uint64_t multishard_query_unpopped_bytes = 0;
uint64_t multishard_query_failed_reader_stops = 0;
uint64_t multishard_query_failed_reader_saves = 0;
};
lw_shared_ptr<db_stats> _stats;
@@ -1201,7 +1206,7 @@ private:
query::result_memory_limiter&,
uint64_t,
db::timeout_clock::time_point,
querier_cache_context> _data_query_stage;
query::querier_cache_context> _data_query_stage;
mutation_query_stage _mutation_query_stage;
@@ -1222,7 +1227,7 @@ private:
seastar::metrics::metric_groups _metrics;
bool _enable_incremental_backups = false;
querier_cache _querier_cache;
query::querier_cache _querier_cache;
std::unique_ptr<db::large_partition_handler> _large_partition_handler;
@@ -1410,10 +1415,14 @@ public:
_querier_cache.set_entry_ttl(entry_ttl);
}
const querier_cache::stats& get_querier_cache_stats() const {
const query::querier_cache::stats& get_querier_cache_stats() const {
return _querier_cache.get_stats();
}
query::querier_cache& get_querier_cache() {
return _querier_cache;
}
friend class distributed_loader;
};

View File

@@ -253,8 +253,12 @@ filter_for_query(consistency_level cl,
return selected_endpoints;
}
std::vector<gms::inet_address> filter_for_query(consistency_level cl, keyspace& ks, std::vector<gms::inet_address>& live_endpoints, column_family* cf) {
return filter_for_query(cl, ks, live_endpoints, {}, read_repair_decision::NONE, nullptr, cf);
std::vector<gms::inet_address> filter_for_query(consistency_level cl,
keyspace& ks,
std::vector<gms::inet_address>& live_endpoints,
const std::vector<gms::inet_address>& preferred_endpoints,
column_family* cf) {
return filter_for_query(cl, ks, live_endpoints, preferred_endpoints, read_repair_decision::NONE, nullptr, cf);
}
bool

View File

@@ -84,7 +84,11 @@ filter_for_query(consistency_level cl,
gms::inet_address* extra,
column_family* cf);
std::vector<gms::inet_address> filter_for_query(consistency_level cl, keyspace& ks, std::vector<gms::inet_address>& live_endpoints, column_family* cf);
std::vector<gms::inet_address> filter_for_query(consistency_level cl,
keyspace& ks,
std::vector<gms::inet_address>& live_endpoints,
const std::vector<gms::inet_address>& preferred_endpoints,
column_family* cf);
struct dc_node_count {
size_t live = 0;

View File

@@ -155,6 +155,26 @@ std::ostream& operator<<(std::ostream& out, const decorated_key& dk) {
return out << "{key: " << dk._key << ", token:" << dk._token << "}";
}
std::ostream& operator<<(std::ostream& out, partition_ranges_view v) {
out << "{";
if (v.empty()) {
out << " }";
return out;
}
auto it = v.begin();
out << *it;
++it;
for (;it != v.end(); ++it) {
out << ", " << *it;
}
out << "}";
return out;
}
// FIXME: make it per-keyspace
std::unique_ptr<i_partitioner> default_partitioner;

View File

@@ -663,6 +663,23 @@ std::ostream& operator<<(std::ostream& out, const token& t);
std::ostream& operator<<(std::ostream& out, const decorated_key& t);
class partition_ranges_view {
const dht::partition_range* _data = nullptr;
size_t _size = 0;
public:
partition_ranges_view() = default;
partition_ranges_view(const dht::partition_range& range) : _data(&range), _size(1) {}
partition_ranges_view(const dht::partition_range_vector& ranges) : _data(ranges.data()), _size(ranges.size()) {}
bool empty() const { return _size == 0; }
size_t size() const { return _size; }
const dht::partition_range& front() const { return *_data; }
const dht::partition_range& back() const { return *(_data + _size - 1); }
const dht::partition_range* begin() const { return _data; }
const dht::partition_range* end() const { return _data + _size; }
};
std::ostream& operator<<(std::ostream& out, partition_ranges_view v);
void set_global_partitioner(const sstring& class_name, unsigned ignore_msb = 0);
i_partitioner& global_partitioner();

447
docs/paged-queries.md Normal file
View File

@@ -0,0 +1,447 @@
# Paged queries
Queries can return any amount of data. The amount of data is only found
out when the query is actually executed. This creates all sorts of
resource management problems for the client as well as for the database.
To avoid these problems query results are transmitted in chunks of
limited size, one chunk at a time. After transmitting each chunk the
database stops and waits for the client to request the next one. This is
repeated until the entire result set is transitted. This is called
paging.
The size of pages can be limited by the client by the number of rows
they can contain. There is also a built-in (non-optional) size limit of
1MB. If a page reaches the size limit before it reaches the
client-provided row limit it's called a short page or short read.
To be able to continue the query on the next page the database has to
remember where it stopped. This is done by recording the position where
the query was interrupted when the page was filled in an opaque (to the
client) cookie called the *paging state*. This cookie is transmitted
with every page to the client and the client has to retransmit it to the
database on every page request. Since the paging state is completely
opaque (just a binary blob) to the client it can be used to store other
query-related state besides just the page end position.
## How did they work?
### Single partition queries
The coordinator selects a list of replicas for each partition to send
read requests to (IN queries are considered single partition queries
also).
The set of replicas is selected such that it satisfies required CL. An
additional replica may be selected for a speculative read. All read
requests are sent concurrently.
The replica executes the read request via `database::query()` and when
the page is filled it sends the results back to the coordinator.
At the end of each page, if there is more data expected (either the row
or the memory limits of the page were reached), the coordinator saves
the last partition key and the last clustering key in the paging state.
In case of an IN query, if the data returned from the replicas exceeds
the page size, any excess is discarded. There cannot be excess results
when a single partition is queried, the coordinator requests just
one page worth of data from the replicas.
At the beginning of each page the partition list is adjusted:
* Finished partitions are dropped.
* The partition slice of the currently read partition is adjusted, a
special clustering range is added so that the read continues after the
last clustering key.
When a single partition is queried the list contains a single entry.
### Range scans
The coordinator splits the partition range into sub-ranges that are
localized to a single vnode. It then dispatches read requests for these
sub-ranges to enough replicas to satisfy CL requirements. The reads
start with a concurrency of 1, that is a single vnode is read at a time,
exponentially increasing it if the results didnt fill the page.
On the replica the range is further split into sub-ranges that are
localized to a single shard using
`dht::ring_position_exponential_vector_sharder`. The sharder will start
reading a single sub-range exponentially increasing concurrency (reading
more and more shard-local sub ranges concurrently) until the page is
filled. Each read is executed with `database::query_mutations()`. The
results from these individual reads are then merged and sent back to the
coordinator. Care is taken to only send to the coordinator the exact
amount of data it requested. If the last round of read from the shards
yielded so much data that the page is overflown any extra data is
discarded.
The coordinator merges results from all read requests. If there are too
many results excess rows and/or partitions are discarded.
At the beginning of each page, similarly to single partition queries, the
partition range is adjusted:
* The lower bound of the range is set to the last partition of the last
page.
* The partition slice of the currently read partition is adjusted, a
special clustering range is added so that the read continues after the
last clustering key.
## Stateful queries
Before, for paged queries we threw away all readers and any associated
state accumulated during filling the page, and on the next page we
created them from scratch again. Thus on each page we threw away a
considerable amount of work, only to redo it again on the next page.
This significantly increased latency and reduced throughput as from the
point of view of a replica each page is as much work as a fresh query.
The solution is to make queries stateful: instead of throwing away all
state related to a query after filling the page on each replica, save
this state in a cache and on the next page reuse it to continue the
query where it was left off.
### Single partition queries
#### The querier
The essence of making queries stateful is saving the readers and any
associated state on the replicas. To make this easy the reader and all
associated objects that are necessary to serve a read on a shard are
wrapped in a querier object which was designed to be suspendable and
resumable, while offering a simple interface to client code.
#### The querier cache
Queriers are saved in a special-purpose cache. Queriers are not reusable
across queries even for those reading from the same table. Different
queries can have different restrictions, order, query time, etc.
Validating all this to test whether a querier can be used for an
arbitrary read request would be high-impossible and error-prone. To
avoid all this each query has a unique identifier (the `query_uuid`).
This identifier is used as the key to the cache under which the querier
is saved.
There is a querier cache object for each shard and it is stored in the
database object of the respective shard.
#### Choosing the same replicas
In order for caching to work each page of a query has to be consistently
read from the same replicas for the entire duration of the query.
Otherwise the read might miss the querier cache and won't be able to
reuse the queriers from the previous page.
To faciliate this the list of replicas used for each page is saved in
the paging state and on the next page the same replicas will be
preferred over other replicas.
#### Putting it all together
##### Coordinator
On the first page of the query the coordinator will generate a unique
identifier for the query. This identifier will be transmitted to the
replicas as part of the read request. The replicas will use this key to
lookup saved queriers from the previous page and save them after filling
the page. On the first page of the query no replicas will have any
cached queriers. To avoid a pointless lookup but even more importantly
to avoid introducing noise into the [diagnostic counters](#diagnostics)
a flag (`is_first_page`) is added to the read request. When this flag is
set replicas will not attempt to lookup queriers from the previous page.
At the end of each page, in addition to what was already saved, the
coordinator saves in the paging state:
* The `query_uuid`.
* The list of replicas used for the page (`last_replicas`).
* The [read repair decision](#probabilistic-read-repair)
##### Replica
At the start of each page, if `query_uuid` is set and `is_first_page` is
`false` a lookup of the querier from the last page will be attempted. If
this succeeds the querier will be removed from the cache and reused for
continuing the read. If it fails a new one will be created and used for
the remainder of the query.
At the end of each page, if there is still data left (at least one of
the page limits were reached) the querier is saved again in the cache.
Note that since there is no way to know whether there is more data to be
read without actually reading it the only way to determine whether the
query is done is to look at whether the page is full. If the page is not
full it means there wasn't enough data to fill it and thus the query is
done. On the other hand if the page is full there might be more data to
read. This might result in an empty last page if there was just enough
data to fill the previous page but not more.
##### Read repair
If the coordinator gets different results from the replicas (e.g.
because one of the replicas missed a write for some reason) it
reconciles them. This will result in some replicas having queriers with
the wrong position on the next page. For example replicas that sent
rows that are now dead (missed some deletes) will get a new page start
position that is ahead of their saved querier's while replicas that
excluded some rows (missed some writes) will get a new page start
position that is behind their saved querier's.
Since readers cannot be rewound to an earlier position the saved querier
has to be discarded and a new one created on these replicas. To identify
these cases on each cache lookup the position of the found querier is
validated to match *exactly* the new page's read start position. When a
mismatch is detected the saved querier is dropped and a new one is
created instead. Note that altough readers can technically be
fast-forwarded to a later position all position mismatches are treated
the same (querier is dropped) even if the reader could theoretically be
fast-forwarded to the page start position. The reason for this is that
using readers that could do that would results in significantly more
complicated code and also reduced performance.
##### Discarded results
As already mentioned, in the case of IN queries a page may be
over-filled as all partitions are read concurrently. In this case the
coordinator will discard any extra rows to fit the results into the page
limits. This poses a problem for cached queriers as those queriers,
whose results were partly or fully discarded will receive a read request
on the next page, with a start position that they already passed. The
position validation introduced in [read repair](#read-repair) will also
catch these position mismatches and the saved querier will be dropped.
##### Schema upgrades
The schema of the read table can change between two pages. Dealing with
this properly would be complicated and would not be worth the effort. So
on lookup the schema versions are also checked and in case the cached
querier's schema version differs from that of the new page's schema's it
is dropped and a new querier is created instead.
##### Concurrent reads against the same shard of the same replica
In the case of an IN query two listed partitions might be colocated on
the same shard of the same replica. This will result in two concurrent
read requests (reading different partitions) executing on said shard,
both attempting to save and/or lookup queriers using the same
`query_uuid`. This can result in the lookup finding a querier
which is reading another partition. To avoid this, on lookup, the
partition each found querier is reading is matched with that of the read
request. In case when no matching querier is found a new querier is
created as if the lookup missed.
##### Probabilistic read repair
On each page of a query there is a chance (user-changable property of
the table) that a read-repair will be attempted. This hurts stateful
queries as each page has a chance of using additional replicas in the
query and on the next page not use some of them. This will result in
cache misses when new replicas are involved and querier drops when these
abandoned replicas will be attempted to be used again (the read position
of the saved queriers that were neglected for some pages will not match
the current one). To solve this problem we make the read repair decision
apply to an entire query instead of a single page. Make it on the first
page and stick to it for the entire duration of the query. The read
repair decision is generated on the first page and saved in the paging
state to be remembered for the duration of the query.
##### Cache eviction
###### Time based
Reads may be abandoned by the client or the coordinator may chose to use
a different replica for the remainder of the query. To avoid abandoned
queriers accumulating in the cache each cached querier has a TTL. After
this expires it is evicted from the cache.
###### Resource based
The concurrency of reads executing on a given shard is limited to avoid
unbounded resource usage. For this reason each reader needs to obtain a
permit before it can start reading and holds on to this permit until it
is destroyed. Suspended readers (those that are part of a cached querier
object) also hold on to their permit and thus may prevent new readers
from being admitted to read. Since new, active readers should be
preferred over suspended ones, when there is a shortage of permits,
queriers are evicted from the cache until enough permits are recovered
to admit all new readers, or until the cache is empty. Queriers are
evicted in LRU order.
###### Memory based
To avoid excessive memory usage the size of the querier cache is
limited. To avoid crossing this limit, the cumulative size of all the
cached queriers is calculated before inserting a new one. If, together
with the to-be-added querier, the limit would be crossed, queriers
are evicted such that the memory consumption stays below the limit.
Queriers are evicted in LRU order.
#### Diagnostics
To observe the effectiveness of the caching, as well as aid in finding
any problems a number of counters are added:
1. `querier_cache_lookups` counts the total number of querier cache
lookups. Not all page-fetches will result in a querier lookup. For
example the first page of a query will not do a lookup as there was no
previous page to reuse the querier from. The second, and all
subsequent pages however should attempt to reuse the querier from the
previous page.
2. `querier_cache_misses` counts the subset of (1) where the read have
missed the querier cache (failed to find a saved querier with a
read-range matching that of the page).
3. `querier_cache_drops` counts the subset of (1) where a saved querier
was found with a matching read range but it cannot be used to continue
the read for other reasons so it was dropped. This can happen for
example if the querier was at the wrong position.
4. `querier_cache_time_based_evictions` counts the cached entries that
were evicted due to their TTL expiring.
5. `querier_cache_resource_based_evictions` counts the cached entries
that were evicted due to reader-resource (those limited by
reader-concurrency limits) shortage.
6. `querier_cache_memory_based_evictions` counts the cached entries
that were evicted due to reaching the cache's memory limits (currently
set to 4% of the shards' memory).
7. `querier_cache_querier_population` is the current number of querier
entries in the cache.
Note:
* The count of cache hits can be derived from these counters as
(1) - (2).
* A cache drop (3) also implies a cache hit (see above). This means that
the number of actually reused queriers is: (1) - (2) - (3)
Counters (2) to (6) are soft badness counters. They might be non-zero in
a healthy cluster but high values or sudden spikes can indicate
problems.
### Range scans
Stateful range scans are built on top of the infrastructure introduced
for stateful single partition queries. That is, reads on replicas are
done using a querier objects that are wrapping a reader which executes
the actual read. This querier is then saved in a cache (`querier_cache`)
at the end of the page and is reused on the next page. The major
difference is that as opposed to single partition reads range scans read
from all shards on a replica.
#### multishard_combining_reader
Using the querier mandates using a `flat_mutation_reader`. Range scans
used an open-coded algorithm on the replica for the read. As already
explained in [the introduction](#range-scans) this algorithm
uses several calls to `database::query_muations()` to the remote shards
then merging the produced `reconcilable_result`. This algoritm did not
lend itself for being wrapped in a `flat_mutation_reader` so a new,
suitable one was written from scratch. This is
`multishard_combining_reader`. In addition to implementing a
multishard-reading algorithm that is suspendable, an effort was made to
solve some of the weak points of the previous open-coded implementation,
mainly cold start and result merging.
#### query_mutations_on_all_shards()
Implementing a stateful range scan using `multishard_combining_reader`,
`querier` and `querier_cache` still has a lot of involved details to
it. To make this as accessible and resuable as possible a function was
added that takes care of all this, offering a simple interface to
clients. This is `query_mutations_on_all_shards()`, which takes care of
all details related to replica local range scans. It supports both
stateful and stateless queries transparently.
#### Suspending and resuming the multishard_combining_reader
Saving the `multishard_combining_reader` in a querier would be the
natural choice for saving the query state. This however would create
some serious problems:
* It is not enough to just save the `multishard_combining_reader` reader
in a querier. All the shard readers have to be saved on their home
shard and made individually evictable as well but this has to be
transparent to the multishard reader which, being a plain
`flat_mutation_reader`, has no way to be told that the query is
"suspended" and later "resumed" and thus could not do the
save/lookup/recreate itself.
* It mandates the consistent usage of the same shard throughout all the
pages of the query. This is problematic for load balancing.
* The querier wrapping the `multishard_combining_reader` would be a
single point of failure for the entire query. If evicted the entire
saved state (all shard readers) would have to be dropped as well.
While some of these issues could be worked around and others could be
lived with, overall they make this option unfeasable.
An alternative but less natural option is to "dismantle" the multishard
reader, that is remove and take ownership of all its shard readers and
move any fragments that were popped from a shard reader but not
consumed (included in the results) back to their originating shard
reader, so that only the shard readers need to be saved and resumed. In
other words move all state required to suspend/resume the query into the
shard readers.
This option addresses all the problems the "natural" option has:
* All shard readers are independently evictable.
* No coordinator shard is needed, each page request can be executed on
any shard.
* Evicting a shard reader has no effect on the remaining ones.
Of course it also has it own problems:
* On each page a certain amount of work is undone by moving some
fragments back to their original readers.
* The concurrency state of the multishard reader is lost and it has to
start from 1 on the next page.
But these problems are much more manageable and some (for example the
last) can be worked around if found to be a real problem.
#### Putting it all together
##### Coordinator
In principle the same as that for [single partition queries](#coordinator).
##### Replica
The storage proxy can now simply call
[query_mutations_on_all_shards()](#query_mutations_on_all_shards) with
the appropriate parameters which takes care of executing the read,
including saving and reusing the shard readers.
#### Diagnostics
Additional counters are added to detect possible problems with stateful
range scans:
1. `multishard_query_unpopped_fragments`
2. `multishard_query_unpopped_bytes`
3. `multishard_query_failed_reader_stops`
4. `multishard_query_failed_reader_saves`
(1) and (2) track the amount of data pushed back to shard readers while
[dismantling](#suspending-and-resuming-the-multishard_combining_reader)
the multishard reader. These are soft badness counters, they will not be
zero in a normally operating cluster, however sudden spikes in their
values can indicate problems.
(3) tracks the number of times stopping any of the shard readers failed.
Shard readers are said to be stopped when the page is filled, that is
any pending read-ahead is waited upon. Since saving a reader will not
fail the read itself these failures will normally go undetected. To
avoid hiding any bug or problem due to this, track these background
failures using this counters. This counter is a hard badness counter,
that is it should *always* be zero. Any other value indicates problems
in the respective shard/node.
(4) tracks the number of times saving the reader failed. This only
includes preparing the querier object and inserting it into the querier
cache. Like (3) this is a hard badness counter.
## Future work
Since the protocol specifications allows for over or underfilling a
page in the future we might get rid of discarding results on the
coordinator to free ourselves from all the [problems](#discarded-results)
it causes.
The present state optimizes for range scans on huge tables, where the
page is filled from a single shard of a single vnode. Further
optimizations are possible for scans on smaller tables, that have to
cross shards or even vnodes to fill the page. One obvious candidate is
saving and restoring the current concurrency on the coordinator (how
many vnodes have to be read concurrently) and on the replica (how many
shards we should read-ahead on).
## Further reading
* [querier.hh](../querier.hh) `querier` and `querier_cache`.
* [multishard_mutation_query.hh](../multishard_mutation_query.hh)
`query_mutations_on_all_shards()`.
* [mutation_reader.hh](../mutation_reader.hh)
`multishard_combining_reader`.

View File

@@ -117,6 +117,9 @@ public:
_buffer.reserve(_buffer.size() * 2 + 1);
}
}
const circular_buffer<mutation_fragment>& buffer() const {
return _buffer;
}
private:
static flat_mutation_reader reverse_partitions(flat_mutation_reader::impl&);
public:
@@ -136,6 +139,12 @@ public:
return mf;
}
void unpop_mutation_fragment(mutation_fragment mf) {
const auto memory_usage = mf.memory_usage(*_schema);
_buffer.emplace_front(std::move(mf));
_buffer_size += memory_usage;
}
future<mutation_fragment_opt> operator()() {
if (is_buffer_empty()) {
if (is_end_of_stream()) {
@@ -395,6 +404,7 @@ public:
bool is_buffer_empty() const { return _impl->is_buffer_empty(); }
bool is_buffer_full() const { return _impl->is_buffer_full(); }
mutation_fragment pop_mutation_fragment() { return _impl->pop_mutation_fragment(); }
void unpop_mutation_fragment(mutation_fragment mf) { _impl->unpop_mutation_fragment(std::move(mf)); }
const schema_ptr& schema() const { return _impl->_schema; }
void set_max_buffer_size(size_t size) {
_impl->max_buffer_size_in_bytes = size;

View File

@@ -0,0 +1,622 @@
/*
* 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 "schema_registry.hh"
#include "service/priority_manager.hh"
#include "multishard_mutation_query.hh"
#include <boost/range/adaptor/reversed.hpp>
logging::logger mmq_log("multishard_mutation_query");
template <typename T>
using foreign_unique_ptr = foreign_ptr<std::unique_ptr<T>>;
class read_context {
struct reader_params {
std::unique_ptr<const dht::partition_range> range;
std::unique_ptr<const query::partition_slice> slice;
reader_params(dht::partition_range range, query::partition_slice slice)
: range(std::make_unique<const dht::partition_range>(std::move(range)))
, slice(std::make_unique<const query::partition_slice>(std::move(slice))) {
}
reader_params(std::unique_ptr<const dht::partition_range> range, std::unique_ptr<const query::partition_slice> slice)
: range(std::move(range))
, slice(std::move(slice)) {
}
};
struct bundled_remote_reader {
foreign_unique_ptr<reader_params> params;
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
foreign_unique_ptr<flat_mutation_reader> reader;
};
using inexistent_state = std::monostate;
struct successful_lookup_state {
foreign_unique_ptr<reader_params> params;
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
foreign_unique_ptr<flat_mutation_reader> reader;
};
struct used_state {
foreign_unique_ptr<reader_params> params;
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
};
struct dismantling_state {
foreign_unique_ptr<reader_params> params;
foreign_unique_ptr<utils::phased_barrier::operation> read_operation;
future<stopped_foreign_reader> reader_fut;
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;
circular_buffer<mutation_fragment> buffer;
};
struct future_used_state {
future<used_state> fut;
};
struct future_dismantling_state {
future<dismantling_state> fut;
};
// ( )
// |
// +------ 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)
//
// 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
using reader_state = std::variant<
inexistent_state,
successful_lookup_state,
used_state,
dismantling_state,
ready_to_save_state,
future_used_state,
future_dismantling_state>;
distributed<database>& _db;
schema_ptr _schema;
const query::read_command& _cmd;
const dht::partition_range_vector& _ranges;
tracing::trace_state_ptr _trace_state;
// One for each shard. Index is shard id.
std::vector<reader_state> _readers;
static future<bundled_remote_reader> do_make_remote_reader(
distributed<database>& db,
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);
future<foreign_unique_ptr<flat_mutation_reader>> make_remote_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,
streamed_mutation::forwarding fwd_sm,
mutation_reader::forwarding fwd_mr);
void dismantle_reader(shard_id shard, future<stopped_foreign_reader>&& stopped_reader_fut);
future<> cleanup_readers();
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);
void dismantle_combined_buffer(circular_buffer<mutation_fragment> combined_buffer, const dht::decorated_key& pkey);
void dismantle_compaction_state(detached_compaction_state compaction_state);
future<> save_reader(ready_to_save_state& current_state, const dht::decorated_key& last_pkey,
const std::optional<clustering_key_prefix>& last_ckey);
public:
read_context(distributed<database>& db, schema_ptr s, const query::read_command& cmd, const dht::partition_range_vector& ranges,
tracing::trace_state_ptr trace_state)
: _db(db)
, _schema(std::move(s))
, _cmd(cmd)
, _ranges(ranges)
, _trace_state(std::move(trace_state)) {
_readers.resize(smp::count);
}
read_context(read_context&&) = delete;
read_context(const read_context&) = delete;
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);
};
}
foreign_reader_dismantler dismantler() {
return [this] (shard_id shard, future<stopped_foreign_reader>&& stopped_reader_fut) {
dismantle_reader(shard, std::move(stopped_reader_fut));
};
}
future<> lookup_readers();
future<> save_readers(circular_buffer<mutation_fragment> unconsumed_buffer, detached_compaction_state compaction_state,
std::optional<clustering_key_prefix> last_ckey);
};
future<read_context::bundled_remote_reader> read_context::do_make_remote_reader(
distributed<database>& db,
shard_id shard,
schema_ptr schema,
const dht::partition_range& pr,
const query::partition_slice& ps,
const io_priority_class&,
tracing::trace_state_ptr trace_state) {
return db.invoke_on(shard, [gs = global_schema_ptr(schema), &pr, &ps, gts = tracing::global_trace_state_ptr(std::move(trace_state))] (
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 params = reader_params(pr, ps);
auto read_operation = table.read_in_progress();
auto reader = table.as_mutation_source().make_reader(std::move(s), *params.range, *params.slice, pc, gts.get());
return make_ready_future<bundled_remote_reader>(bundled_remote_reader{
make_foreign(std::make_unique<reader_params>(std::move(params))),
make_foreign(std::make_unique<utils::phased_barrier::operation>(std::move(read_operation))),
make_foreign(std::make_unique<flat_mutation_reader>(std::move(reader)))});
});
}
future<foreign_unique_ptr<flat_mutation_reader>> read_context::make_remote_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,
streamed_mutation::forwarding,
mutation_reader::forwarding) {
auto& rs = _readers[shard];
if (!std::holds_alternative<successful_lookup_state>(rs) && !std::holds_alternative<inexistent_state>(rs)) {
mmq_log.warn("Unexpected request to create reader for shard {}. A reader for this shard was already created.", shard);
throw std::logic_error(sprint("Unexpected request to create reader for shard {}."
" A reader for this shard was already created in the context of this read.", shard));
}
// The reader is either in inexistent or successful lookup state.
if (auto current_state = std::get_if<successful_lookup_state>(&rs)) {
auto reader = std::move(current_state->reader);
rs = used_state{std::move(current_state->params), std::move(current_state->read_operation)};
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([this, &rs, created = std::move(created)] (
bundled_remote_reader&& bundled_reader) mutable {
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 std::move(bundled_reader.reader);
});
}
void read_context::dismantle_reader(shard_id shard, future<stopped_foreign_reader>&& stopped_reader_fut) {
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(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>{}};
});
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::cleanup_readers() {
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();
params.release();
read_operation.release();
});
}
});
};
std::vector<future<>> futures;
// 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)) {
cleanup(shard, std::move(*maybe_dismantling_state));
} else if (auto maybe_future_dismantling_state = std::get_if<future_dismantling_state>(&rs)) {
futures.emplace_back(maybe_future_dismantling_state->fut.then([=] (dismantling_state&& current_state) {
cleanup(shard, std::move(current_state));
}));
}
}
return when_all(futures.begin(), futures.end()).discard_result();
}
void read_context::dismantle_combined_buffer(circular_buffer<mutation_fragment> combined_buffer, const dht::decorated_key& pkey) {
auto& partitioner = dht::global_partitioner();
std::vector<mutation_fragment> tmp_buffer;
auto rit = std::reverse_iterator(combined_buffer.end());
const auto rend = std::reverse_iterator(combined_buffer.begin());
for (;rit != rend; ++rit) {
if (rit->is_partition_start()) {
const auto shard = partitioner.shard_of(rit->as_partition_start().key().token());
auto& shard_buffer = std::get<dismantling_state>(_readers[shard]).buffer;
for (auto& smf : tmp_buffer) {
shard_buffer.emplace_front(std::move(smf));
}
shard_buffer.emplace_front(std::move(*rit));
tmp_buffer.clear();
} else {
tmp_buffer.emplace_back(std::move(*rit));
}
}
const auto shard = partitioner.shard_of(pkey.token());
auto& shard_buffer = std::get<dismantling_state>(_readers[shard]).buffer;
for (auto& smf : tmp_buffer) {
shard_buffer.emplace_front(std::move(smf));
}
}
void read_context::dismantle_compaction_state(detached_compaction_state compaction_state) {
auto& partitioner = dht::global_partitioner();
const auto shard = partitioner.shard_of(compaction_state.partition_start.key().token());
auto& shard_buffer = std::get<dismantling_state>(_readers[shard]).buffer;
for (auto& rt : compaction_state.range_tombstones | boost::adaptors::reversed) {
shard_buffer.emplace_front(std::move(rt));
}
if (compaction_state.static_row) {
shard_buffer.emplace_front(std::move(*compaction_state.static_row));
}
shard_buffer.emplace_front(std::move(compaction_state.partition_start));
}
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;
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();
return _db.invoke_on(shard, [shard, query_uuid = _cmd.query_uuid, query_ranges = _ranges, &current_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();
auto& buffer = current_state.buffer;
const auto fragments = buffer.size();
const auto size_before = reader->buffer_size();
auto rit = std::reverse_iterator(buffer.cend());
auto rend = std::reverse_iterator(buffer.cbegin());
auto& schema = *reader->schema();
for (;rit != rend; ++rit) {
// Copy the fragment, the buffer is on another shard.
reader->unpop_mutation_fragment(mutation_fragment(schema, *rit));
}
const auto size_after = reader->buffer_size();
auto querier = query::shard_mutation_querier(
std::move(query_ranges),
std::move(params->range),
std::move(params->slice),
std::move(*reader),
last_pkey,
last_ckey);
db.get_querier_cache().insert(query_uuid, std::move(querier), gts.get());
db.get_stats().multishard_query_unpopped_fragments += fragments;
db.get_stats().multishard_query_unpopped_bytes += (size_after - size_before);
} catch (...) {
// We don't want to fail a read just because of a failure to
// save any of the readers.
mmq_log.debug("Failed to save reader: {}", std::current_exception());
++db.get_stats().multishard_query_failed_reader_saves;
}
}).handle_exception([this, shard] (std::exception_ptr e) {
// We don't want to fail a read just because of a failure to
// save any of the readers.
mmq_log.debug("Failed to save reader on shard {}: {}", shard, e);
// This will account the failure on the local shard but we don't
// know where exactly the failure happened anyway.
++_db.local().get_stats().multishard_query_failed_reader_saves;
});
}
future<> read_context::lookup_readers() {
if (_cmd.query_uuid == utils::UUID{} || _cmd.is_first_page) {
return make_ready_future<>();
}
return parallel_for_each(boost::irange(0u, smp::count), [this] (shard_id shard) {
return _db.invoke_on(shard,
[shard, cmd = &_cmd, ranges = &_ranges, gs = global_schema_ptr(_schema), gts = tracing::global_trace_state_ptr(_trace_state)] (
database& db) mutable -> reader_state {
auto schema = gs.get();
auto querier_opt = db.get_querier_cache().lookup_shard_mutation_querier(cmd->query_uuid, *schema, *ranges, cmd->slice, gts.get());
if (!querier_opt) {
return inexistent_state{};
}
auto& q = *querier_opt;
auto& table = db.find_column_family(schema);
auto params = make_foreign(std::make_unique<reader_params>(std::move(q).reader_range(), std::move(q).reader_slice()));
auto read_operation = make_foreign(std::make_unique<utils::phased_barrier::operation>(table.read_in_progress()));
auto reader = make_foreign(std::make_unique<flat_mutation_reader>(std::move(q).reader()));
return successful_lookup_state{std::move(params), std::move(read_operation), std::move(reader)};
}).then([this, shard] (reader_state&& state) {
_readers[shard] = std::move(state);
});
});
}
future<> read_context::save_readers(circular_buffer<mutation_fragment> unconsumed_buffer, detached_compaction_state compaction_state,
std::optional<clustering_key_prefix> last_ckey) {
if (_cmd.query_uuid == utils::UUID{}) {
return cleanup_readers();
}
auto last_pkey = compaction_state.partition_start.key();
dismantle_combined_buffer(std::move(unconsumed_buffer), last_pkey);
dismantle_compaction_state(std::move(compaction_state));
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);
}
auto finish_saving = [this, &last_pkey, &last_ckey] (dismantling_state& current_state) {
return current_state.reader_fut.then_wrapped([this, &current_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)) {
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([this, &rs,
finish_saving = std::move(finish_saving)] (dismantling_state&& next_state) mutable {
rs = std::move(next_state);
return finish_saving(std::get<dismantling_state>(rs));
});
}
return make_ready_future<>();
});
});
}
static future<reconcilable_result> do_query_mutations(
distributed<database>& db,
schema_ptr s,
const query::read_command& cmd,
const dht::partition_range_vector& ranges,
tracing::trace_state_ptr trace_state,
db::timeout_clock::time_point timeout,
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 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,
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_multishard_combining_reader(std::move(s), pr, ps, pc, dht::global_partitioner(), ctx->factory(), std::move(trace_state),
fwd_sm, fwd_mr, ctx->dismantler());
});
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);
auto compaction_state = make_lw_shared<compact_for_mutation_query_state>(*s, cmd.timestamp, cmd.slice, cmd.row_limit,
cmd.partition_limit);
return do_with(std::move(reader), std::move(compaction_state), [&, accounter = std::move(accounter), timeout] (
flat_mutation_reader& reader, lw_shared_ptr<compact_for_mutation_query_state>& compaction_state) mutable {
auto rrb = reconcilable_result_builder(*reader.schema(), cmd.slice, std::move(accounter));
return query::consume_page(reader,
compaction_state,
cmd.slice,
std::move(rrb),
cmd.row_limit,
cmd.partition_limit,
cmd.timestamp,
timeout).then([&] (std::optional<clustering_key_prefix>&& last_ckey, reconcilable_result&& result) mutable {
return make_ready_future<std::optional<clustering_key_prefix>,
reconcilable_result,
circular_buffer<mutation_fragment>,
lw_shared_ptr<compact_for_mutation_query_state>>(std::move(last_ckey), std::move(result), reader.detach_buffer(),
std::move(compaction_state));
});
}).then_wrapped([&ctx] (future<std::optional<clustering_key_prefix>, reconcilable_result, circular_buffer<mutation_fragment>,
lw_shared_ptr<compact_for_mutation_query_state>>&& result_fut) {
if (result_fut.failed()) {
return make_exception_future<reconcilable_result>(std::move(result_fut.get_exception()));
}
auto [last_ckey, result, unconsumed_buffer, compaction_state] = result_fut.get();
if (!compaction_state->are_limits_reached() && !result.is_short_read()) {
return make_ready_future<reconcilable_result>(std::move(result));
}
return ctx->save_readers(std::move(unconsumed_buffer), std::move(*compaction_state).detach_state(),
std::move(last_ckey)).then_wrapped([result = std::move(result)] (future<>&&) mutable {
return make_ready_future<reconcilable_result>(std::move(result));
});
});
});
});
}
future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature> query_mutations_on_all_shards(
distributed<database>& db,
schema_ptr s,
const query::read_command& cmd,
const dht::partition_range_vector& ranges,
tracing::trace_state_ptr trace_state,
uint64_t max_size,
db::timeout_clock::time_point timeout) {
if (cmd.row_limit == 0 || cmd.slice.partition_row_limit() == 0 || cmd.partition_limit == 0) {
return make_ready_future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>(
make_foreign(make_lw_shared<reconcilable_result>()),
db.local().find_column_family(s).get_global_cache_hit_rate());
}
return db.local().get_result_memory_limiter().new_mutation_read(max_size).then([&, s = std::move(s), trace_state = std::move(trace_state),
timeout] (query::result_memory_accounter accounter) mutable {
return do_query_mutations(db, s, cmd, ranges, std::move(trace_state), timeout, std::move(accounter)).then_wrapped(
[&db, s = std::move(s)] (future<reconcilable_result>&& f) {
auto& local_db = db.local();
auto& stats = local_db.get_stats();
if (f.failed()) {
++stats.total_reads_failed;
return make_exception_future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>(f.get_exception());
} else {
++stats.total_reads;
auto result = f.get0();
stats.short_mutation_queries += bool(result.is_short_read());
auto hit_rate = local_db.find_column_family(s).get_global_cache_hit_rate();
return make_ready_future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>(
make_foreign(make_lw_shared<reconcilable_result>(std::move(result))), hit_rate);
}
});
});
}

View File

@@ -0,0 +1,66 @@
/*
* 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/>.
*/
#pragma once
#include "database.hh"
/// Run the mutation query on all shards.
///
/// Under the hood it uses a multishard_combining_reader for reading the
/// range(s) from all shards.
///
/// The query uses paging. The read will stop after reaching one of the page
/// size limits. Page size is determined by the read_command (row and partition
/// limits) and by the max_size parameter (max memory size of results).
///
/// Optionally the query can be stateful. This means that after filling the
/// page, the shard readers are saved in the `querier_cache` on their home shard
/// (wrapped in a `shard_mutation_querier`). Fragments already read from
/// the shard readers, but not consumed by the results builder (due to
/// reaching the limit), are extracted from the `multishard_combining_reader`'s
/// (and the foreign readers wrapping the shard readers) buffers and pushed back
/// into the shard reader they originated from. This way only the shard readers
/// have to be cached in order to continue the query.
/// When reading the next page these querier objects are looked up from
/// their respective shard's `querier_cache`, instead of creating new shard
/// readers.
/// To enable stateful queries set the `query_uuid` field of the read command
/// to an id unique to the query. This can be easily achived by generating a
/// random uuid with `utils::make_random_uuid()`.
/// It is advisable that the `is_first_page` flag of the read command is set on
/// the first page of the query so that a pointless lookup is avoided.
///
/// Note: params passed by reference are expected to be kept alive by the caller
/// for the duration of the query. Params passed by const reference are expected
/// to *not* change during the query, as they will possibly be accessed from
/// other shards.
///
/// \see multishard_combined_reader
/// \see querier_cache
future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature> query_mutations_on_all_shards(
distributed<database>& db,
schema_ptr s,
const query::read_command& cmd,
const dht::partition_range_vector& ranges,
tracing::trace_state_ptr trace_state,
uint64_t max_size,
db::timeout_clock::time_point timeout = db::no_timeout);

View File

@@ -232,6 +232,7 @@ future<mutation_opt> read_mutation_from_flat_mutation_reader(flat_mutation_reade
adapter(schema_ptr s) : _s(std::move(s)) { }
void consume_new_partition(const dht::decorated_key& dk) {
assert(!_builder);
_builder = mutation_rebuilder(dk, std::move(_s));
}

View File

@@ -54,6 +54,12 @@ GCC6_CONCEPT(
};
)
struct detached_compaction_state {
::partition_start partition_start;
std::optional<::static_row> static_row;
std::deque<range_tombstone> range_tombstones;
};
// emit_only_live::yes will cause compact_for_query to emit only live
// static and clustering rows. It doesn't affect the way range tombstones are
// emitted.
@@ -323,6 +329,18 @@ public:
bool are_limits_reached() const {
return _row_limit == 0 || _partition_limit == 0;
}
/// Detach the internal state of the compactor
///
/// The state is represented by the last seen partition header, static row
/// and active range tombstones. Replaying these fragments through a new
/// compactor will result in the new compactor being in the same state *this
/// is (given the same outside parameters of course). Practically this
/// allows the compaction state to be stored in the compacted reader.
detached_compaction_state detach_state() && {
partition_start ps(std::move(_last_dk), _range_tombstones.get_partition_tombstone());
return {std::move(ps), std::move(_last_static_row), std::move(_range_tombstones).range_tombstones()};
}
};
template<emit_only_live_rows OnlyLive, compact_for_sstables SSTableCompaction, typename Consumer>

View File

@@ -2007,18 +2007,18 @@ future<> data_query(
query::result::builder& builder,
tracing::trace_state_ptr trace_ptr,
db::timeout_clock::time_point timeout,
querier_cache_context cache_ctx)
query::querier_cache_context cache_ctx)
{
if (row_limit == 0 || slice.partition_row_limit() == 0 || partition_limit == 0) {
return make_ready_future<>();
}
auto q = cache_ctx.lookup(emit_only_live_rows::yes, *s, range, slice, trace_ptr, [&, trace_ptr] {
return querier(source, s, range, slice, service::get_local_sstable_query_read_priority(),
std::move(trace_ptr), emit_only_live_rows::yes);
});
auto querier_opt = cache_ctx.lookup_data_querier(*s, range, slice, trace_ptr);
auto q = querier_opt
? std::move(*querier_opt)
: query::data_querier(source, s, range, slice, service::get_local_sstable_query_read_priority(), std::move(trace_ptr));
return do_with(std::move(q), [=, &builder, trace_ptr = std::move(trace_ptr), cache_ctx = std::move(cache_ctx)] (querier& q) mutable {
return do_with(std::move(q), [=, &builder, trace_ptr = std::move(trace_ptr), cache_ctx = std::move(cache_ctx)] (query::data_querier& q) mutable {
auto qrb = query_result_builder(*s, builder);
return q.consume_page(std::move(qrb), row_limit, partition_limit, query_time, timeout).then(
[=, &builder, &q, trace_ptr = std::move(trace_ptr), cache_ctx = std::move(cache_ctx)] () mutable {
@@ -2029,84 +2029,64 @@ future<> data_query(
});
}
class reconcilable_result_builder {
const schema& _schema;
const query::partition_slice& _slice;
void reconcilable_result_builder::consume_new_partition(const dht::decorated_key& dk) {
_has_ck_selector = has_ck_selector(_slice.row_ranges(_schema, dk.key()));
_static_row_is_alive = false;
_live_rows = 0;
auto is_reversed = _slice.options.contains(query::partition_slice::option::reversed);
_mutation_consumer.emplace(streamed_mutation_freezer(_schema, dk.key(), is_reversed));
}
std::vector<partition> _result;
uint32_t _live_rows{};
void reconcilable_result_builder::consume(tombstone t) {
_mutation_consumer->consume(t);
}
bool _has_ck_selector{};
bool _static_row_is_alive{};
uint32_t _total_live_rows = 0;
query::result_memory_accounter _memory_accounter;
stop_iteration _stop;
bool _short_read_allowed;
stdx::optional<streamed_mutation_freezer> _mutation_consumer;
public:
reconcilable_result_builder(const schema& s, const query::partition_slice& slice,
query::result_memory_accounter&& accounter)
: _schema(s), _slice(slice)
, _memory_accounter(std::move(accounter))
, _short_read_allowed(slice.options.contains<query::partition_slice::option::allow_short_read>())
{ }
stop_iteration reconcilable_result_builder::consume(static_row&& sr, tombstone, bool is_alive) {
_static_row_is_alive = is_alive;
_memory_accounter.update(sr.memory_usage(_schema));
return _mutation_consumer->consume(std::move(sr));
}
void consume_new_partition(const dht::decorated_key& dk) {
_has_ck_selector = has_ck_selector(_slice.row_ranges(_schema, dk.key()));
_static_row_is_alive = false;
_live_rows = 0;
auto is_reversed = _slice.options.contains(query::partition_slice::option::reversed);
_mutation_consumer.emplace(streamed_mutation_freezer(_schema, dk.key(), is_reversed));
stop_iteration reconcilable_result_builder::consume(clustering_row&& cr, row_tombstone, bool is_alive) {
_live_rows += is_alive;
auto stop = _memory_accounter.update_and_check(cr.memory_usage(_schema));
if (is_alive) {
// We are considering finishing current read only after consuming a
// live clustering row. While sending a single live row is enough to
// guarantee progress, not ending the result on a live row would
// mean that the next page fetch will read all tombstones after the
// last live row again.
_stop = stop && stop_iteration(_short_read_allowed);
}
return _mutation_consumer->consume(std::move(cr)) || _stop;
}
void consume(tombstone t) {
_mutation_consumer->consume(t);
}
stop_iteration consume(static_row&& sr, tombstone, bool is_alive) {
_static_row_is_alive = is_alive;
_memory_accounter.update(sr.memory_usage(_schema));
return _mutation_consumer->consume(std::move(sr));
}
stop_iteration consume(clustering_row&& cr, row_tombstone, bool is_alive) {
_live_rows += is_alive;
auto stop = _memory_accounter.update_and_check(cr.memory_usage(_schema));
if (is_alive) {
// We are considering finishing current read only after consuming a
// live clustering row. While sending a single live row is enough to
// guarantee progress, not ending the result on a live row would
// mean that the next page fetch will read all tombstones after the
// last live row again.
_stop = stop && stop_iteration(_short_read_allowed);
}
return _mutation_consumer->consume(std::move(cr)) || _stop;
}
stop_iteration consume(range_tombstone&& rt) {
_memory_accounter.update(rt.memory_usage(_schema));
return _mutation_consumer->consume(std::move(rt));
}
stop_iteration reconcilable_result_builder::consume(range_tombstone&& rt) {
_memory_accounter.update(rt.memory_usage(_schema));
return _mutation_consumer->consume(std::move(rt));
}
stop_iteration consume_end_of_partition() {
if (_live_rows == 0 && _static_row_is_alive && !_has_ck_selector) {
++_live_rows;
// Normally we count only live clustering rows, to guarantee that
// the next page fetch won't ask for the same range. However,
// if we return just a single static row we can stop the result as
// well. Next page fetch will ask for the next partition and if we
// don't do that we could end up with an unbounded number of
// partitions with only a static row.
_stop = _stop || (_memory_accounter.check() && stop_iteration(_short_read_allowed));
}
_total_live_rows += _live_rows;
_result.emplace_back(partition { _live_rows, _mutation_consumer->consume_end_of_stream() });
return _stop;
stop_iteration reconcilable_result_builder::consume_end_of_partition() {
if (_live_rows == 0 && _static_row_is_alive && !_has_ck_selector) {
++_live_rows;
// Normally we count only live clustering rows, to guarantee that
// the next page fetch won't ask for the same range. However,
// if we return just a single static row we can stop the result as
// well. Next page fetch will ask for the next partition and if we
// don't do that we could end up with an unbounded number of
// partitions with only a static row.
_stop = _stop || (_memory_accounter.check() && stop_iteration(_short_read_allowed));
}
_total_live_rows += _live_rows;
_result.emplace_back(partition { _live_rows, _mutation_consumer->consume_end_of_stream() });
return _stop;
}
reconcilable_result consume_end_of_stream() {
return reconcilable_result(_total_live_rows, std::move(_result),
query::short_read(bool(_stop)),
std::move(_memory_accounter).done());
}
};
reconcilable_result reconcilable_result_builder::consume_end_of_stream() {
return reconcilable_result(_total_live_rows, std::move(_result),
query::short_read(bool(_stop)),
std::move(_memory_accounter).done());
}
future<reconcilable_result>
static do_mutation_query(schema_ptr s,
@@ -2119,19 +2099,19 @@ static do_mutation_query(schema_ptr s,
query::result_memory_accounter&& accounter,
tracing::trace_state_ptr trace_ptr,
db::timeout_clock::time_point timeout,
querier_cache_context cache_ctx)
query::querier_cache_context cache_ctx)
{
if (row_limit == 0 || slice.partition_row_limit() == 0 || partition_limit == 0) {
return make_ready_future<reconcilable_result>(reconcilable_result());
}
auto q = cache_ctx.lookup(emit_only_live_rows::no, *s, range, slice, trace_ptr, [&, trace_ptr] {
return querier(source, s, range, slice, service::get_local_sstable_query_read_priority(),
std::move(trace_ptr), emit_only_live_rows::no);
});
auto querier_opt = cache_ctx.lookup_mutation_querier(*s, range, slice, trace_ptr);
auto q = querier_opt
? std::move(*querier_opt)
: query::mutation_querier(source, s, range, slice, service::get_local_sstable_query_read_priority(), std::move(trace_ptr));
return do_with(std::move(q),
[=, &slice, accounter = std::move(accounter), trace_ptr = std::move(trace_ptr), cache_ctx = std::move(cache_ctx)] (querier& q) mutable {
return do_with(std::move(q), [=, &slice, accounter = std::move(accounter), trace_ptr = std::move(trace_ptr), cache_ctx = std::move(cache_ctx)] (
query::mutation_querier& q) mutable {
auto rrb = reconcilable_result_builder(*s, slice, std::move(accounter));
return q.consume_page(std::move(rrb), row_limit, partition_limit, query_time, timeout).then(
[=, &q, trace_ptr = std::move(trace_ptr), cache_ctx = std::move(cache_ctx)] (reconcilable_result r) mutable {
@@ -2158,7 +2138,7 @@ mutation_query(schema_ptr s,
query::result_memory_accounter&& accounter,
tracing::trace_state_ptr trace_ptr,
db::timeout_clock::time_point timeout,
querier_cache_context cache_ctx)
query::querier_cache_context cache_ctx)
{
return do_mutation_query(std::move(s), std::move(source), seastar::cref(range), seastar::cref(slice),
row_limit, partition_limit, query_time, std::move(accounter), std::move(trace_ptr), timeout, std::move(cache_ctx));

View File

@@ -108,6 +108,37 @@ public:
printer pretty_printer(schema_ptr) const;
};
class reconcilable_result_builder {
const schema& _schema;
const query::partition_slice& _slice;
std::vector<partition> _result;
uint32_t _live_rows{};
bool _has_ck_selector{};
bool _static_row_is_alive{};
uint32_t _total_live_rows = 0;
query::result_memory_accounter _memory_accounter;
stop_iteration _stop;
bool _short_read_allowed;
stdx::optional<streamed_mutation_freezer> _mutation_consumer;
public:
reconcilable_result_builder(const schema& s, const query::partition_slice& slice,
query::result_memory_accounter&& accounter)
: _schema(s), _slice(slice)
, _memory_accounter(std::move(accounter))
, _short_read_allowed(slice.options.contains<query::partition_slice::option::allow_short_read>())
{ }
void consume_new_partition(const dht::decorated_key& dk);
void consume(tombstone t);
stop_iteration consume(static_row&& sr, tombstone, bool is_alive);
stop_iteration consume(clustering_row&& cr, row_tombstone, bool is_alive);
stop_iteration consume(range_tombstone&& rt);
stop_iteration consume_end_of_partition();
reconcilable_result consume_end_of_stream();
};
query::result to_data_query_result(const reconcilable_result&, schema_ptr, const query::partition_slice&, uint32_t row_limit, uint32_t partition_limit, query::result_options opts = query::result_options::only_result());
// Performs a query on given data source returning data in reconcilable form.
@@ -132,7 +163,7 @@ future<reconcilable_result> mutation_query(
query::result_memory_accounter&& accounter = { },
tracing::trace_state_ptr trace_ptr = nullptr,
db::timeout_clock::time_point timeout = db::no_timeout,
querier_cache_context cache_ctx = { });
query::querier_cache_context cache_ctx = { });
future<> data_query(
schema_ptr s,
@@ -145,7 +176,7 @@ future<> data_query(
query::result::builder& builder,
tracing::trace_state_ptr trace_ptr = nullptr,
db::timeout_clock::time_point timeout = db::no_timeout,
querier_cache_context cache_ctx = { });
query::querier_cache_context cache_ctx = { });
class mutation_query_stage {
@@ -160,7 +191,7 @@ class mutation_query_stage {
query::result_memory_accounter&&,
tracing::trace_state_ptr,
db::timeout_clock::time_point,
querier_cache_context> _execution_stage;
query::querier_cache_context> _execution_stage;
public:
explicit mutation_query_stage();
template <typename... Args>

View File

@@ -900,6 +900,10 @@ public:
virtual void next_partition() override;
virtual future<> fast_forward_to(const dht::partition_range& pr, db::timeout_clock::time_point timeout) override;
virtual future<> fast_forward_to(position_range pr, db::timeout_clock::time_point timeout) override;
const mutation_fragment& peek_buffer() const { return buffer().front(); }
future<stopped_foreign_reader> stop();
};
foreign_reader::foreign_reader(schema_ptr schema,
@@ -911,6 +915,9 @@ foreign_reader::foreign_reader(schema_ptr schema,
}
foreign_reader::~foreign_reader() {
if (!_read_ahead_future && !_reader) {
return;
}
smp::submit_to(_reader.get_owner_shard(), [reader = std::move(_reader), read_ahead_future = std::move(_read_ahead_future)] () mutable {
if (read_ahead_future) {
return read_ahead_future->finally([r = std::move(reader)] {});
@@ -972,6 +979,26 @@ 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) {
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 {
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) {
reader->next_partition();
}
});
}).then([this] {
return stopped_foreign_reader{std::move(_reader), detach_buffer()};
});
} else {
return make_ready_future<stopped_foreign_reader>(stopped_foreign_reader{std::move(_reader), detach_buffer()});
}
}
flat_mutation_reader make_foreign_reader(schema_ptr schema,
foreign_ptr<std::unique_ptr<flat_mutation_reader>> reader,
streamed_mutation::forwarding fwd_sm) {
@@ -981,11 +1008,15 @@ flat_mutation_reader make_foreign_reader(schema_ptr schema,
return make_flat_mutation_reader<foreign_reader>(std::move(schema), std::move(reader), fwd_sm);
}
// See make_foreign_reader() for description.
// See make_multishard_combining_reader() for description.
class multishard_combining_reader : public flat_mutation_reader::impl {
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;
@@ -1003,15 +1034,15 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
// pending continuations, just "run through them".
class shard_reader {
struct state {
flat_mutation_reader_opt reader;
bool abandoned = false;
std::unique_ptr<foreign_reader> reader;
unsigned pending_next_partition = 0;
bool stopped = false;
promise<> reader_promise;
};
const multishard_combining_reader& _parent;
const unsigned _shard;
lw_shared_ptr<state> _state;
unsigned _pending_next_partition = 0;
std::optional<future<>> _read_ahead;
promise<> _reader_promise;
public:
shard_reader(multishard_combining_reader& parent, unsigned shard)
@@ -1027,10 +1058,8 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
shard_reader& operator=(const shard_reader&) = delete;
~shard_reader() {
_state->abandoned = true;
if (_read_ahead) {
// Keep state (the reader) alive until the read-ahead completes.
_read_ahead->finally([state = _state] {});
if (!_state->stopped) {
stop();
}
}
@@ -1064,6 +1093,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();
};
std::vector<shard_reader> _shard_readers;
@@ -1078,10 +1108,16 @@ class multishard_combining_reader : public flat_mutation_reader::impl {
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);
mutation_reader::forwarding fwd_mr,
foreign_reader_dismantler reader_dismantler);
~multishard_combining_reader();
// this is captured.
multishard_combining_reader(const multishard_combining_reader&) = delete;
@@ -1099,14 +1135,14 @@ future<> multishard_combining_reader::shard_reader::fill_buffer(db::timeout_cloc
if (_read_ahead) {
return *std::exchange(_read_ahead, std::nullopt);
}
return _state->reader->fill_buffer();
return _state->reader->fill_buffer(timeout);
}
void multishard_combining_reader::shard_reader::next_partition() {
if (_state->reader) {
_state->reader->next_partition();
} else {
++_pending_next_partition;
++_state->pending_next_partition;
}
}
@@ -1133,22 +1169,19 @@ future<> multishard_combining_reader::shard_reader::create_reader() {
return make_ready_future<>();
}
if (_read_ahead) {
return _reader_promise.get_future();
return _state->reader_promise.get_future();
}
return _parent._reader_factory(_shard, *_parent._pr, _parent._fwd_sm, _parent._fwd_mr).then(
[this, state = _state] (foreign_ptr<std::unique_ptr<flat_mutation_reader>>&& r) mutable {
// Use the captured instance to check whether the reader is abdandoned.
// If the reader is abandoned we can't read members of this anymore.
if (state->abandoned) {
return;
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();
}
_state->reader = make_foreign_reader(_parent._schema, std::move(r), _parent._fwd_sm);
while (_pending_next_partition) {
--_pending_next_partition;
_state->reader->next_partition();
if (!state->stopped) {
state->reader_promise.set_value();
}
_reader_promise.set_value();
});
}
@@ -1157,7 +1190,7 @@ void multishard_combining_reader::shard_reader::read_ahead(db::timeout_clock::ti
_read_ahead.emplace(_state->reader->fill_buffer(timeout));
} else {
_read_ahead.emplace(create_reader().then([state = _state, timeout] () mutable {
if (state->abandoned) {
if (state->stopped) {
return make_ready_future<>();
}
return state->reader->fill_buffer(timeout);
@@ -1165,6 +1198,25 @@ void multishard_combining_reader::shard_reader::read_ahead(db::timeout_clock::ti
}
}
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>{}});
}
auto f = [this] {
if (_read_ahead) {
return _read_ahead->then([state = _state.get()] () mutable {
return state->reader->stop();
});
} else {
return _state->reader->stop();
}
}();
return f.finally([state = _state] {});
}
void multishard_combining_reader::move_to_next_shard() {
_crossed_shards = true;
_current_shard = (_current_shard + 1) % _partitioner.shard_count();
@@ -1203,14 +1255,22 @@ future<> multishard_combining_reader::handle_empty_reader_buffer(db::timeout_clo
multishard_combining_reader::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)
mutation_reader::forwarding fwd_mr,
foreign_reader_dismantler reader_dismantler)
: impl(s)
, _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())
@@ -1222,6 +1282,25 @@ 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] {
@@ -1286,9 +1365,14 @@ future<> multishard_combining_reader::fast_forward_to(position_range pr, db::tim
flat_mutation_reader make_multishard_combining_reader(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) {
return make_flat_mutation_reader<multishard_combining_reader>(schema, pr, partitioner, std::move(reader_factory), fwd_sm, fwd_mr);
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));
}

View File

@@ -389,10 +389,20 @@ flat_mutation_reader make_foreign_reader(schema_ptr schema,
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)>;
struct stopped_foreign_reader {
foreign_ptr<std::unique_ptr<flat_mutation_reader>> remote_reader;
circular_buffer<mutation_fragment> unconsumed_fragments;
};
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
@@ -410,9 +420,20 @@ using remote_reader_factory = noncopyable_function<future<foreign_ptr<std::uniqu
/// needs to move to them they have the data ready.
/// 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,
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);
mutation_reader::forwarding fwd_mr = mutation_reader::forwarding::no,
foreign_reader_dismantler reader_dismantler = {});

View File

@@ -25,32 +25,34 @@
#include <boost/range/adaptor/map.hpp>
static sstring cannot_use_reason(querier::can_use cu)
namespace query {
enum class can_use {
yes,
no_schema_version_mismatch,
no_ring_pos_mismatch,
no_clustering_pos_mismatch
};
static sstring cannot_use_reason(can_use cu)
{
switch (cu)
{
case querier::can_use::yes:
case can_use::yes:
return "can be used";
case querier::can_use::no_emit_only_live_rows_mismatch:
return "emit only live rows mismatch";
case querier::can_use::no_schema_version_mismatch:
case can_use::no_schema_version_mismatch:
return "schema version mismatch";
case querier::can_use::no_ring_pos_mismatch:
case can_use::no_ring_pos_mismatch:
return "ring pos mismatch";
case querier::can_use::no_clustering_pos_mismatch:
case can_use::no_clustering_pos_mismatch:
return "clustering pos mismatch";
}
return "unknown reason";
}
querier::position querier::current_position() const {
const dht::decorated_key* dk = std::visit([] (const auto& cs) { return cs->current_partition(); }, _compaction_state);
const clustering_key_prefix* clustering_key = *_last_ckey ? &**_last_ckey : nullptr;
return {dk, clustering_key};
}
bool querier::ring_position_matches(const dht::partition_range& range, const querier::position& pos) const {
const auto is_reversed = flat_mutation_reader::consume_reversed_partitions(_slice->options.contains(query::partition_slice::option::reversed));
static bool ring_position_matches(const schema& s, const dht::partition_range& range, const query::partition_slice& slice,
const position_view& pos) {
const auto is_reversed = flat_mutation_reader::consume_reversed_partitions(slice.options.contains(query::partition_slice::option::reversed));
const auto expected_start = dht::ring_position_view(*pos.partition_key);
// If there are no clustering columns or the select is distinct we don't
@@ -58,10 +60,10 @@ bool querier::ring_position_matches(const dht::partition_range& range, const que
// anything more in the last page's partition and thus the start bound is
// exclusive. Otherwise there migh be clustering rows still and it is
// inclusive.
const auto expected_inclusiveness = _schema->clustering_key_size() > 0 &&
!_slice->options.contains<query::partition_slice::option::distinct>() &&
const auto expected_inclusiveness = s.clustering_key_size() > 0 &&
!slice.options.contains<query::partition_slice::option::distinct>() &&
pos.clustering_key;
const auto comparator = dht::ring_position_comparator(*_schema);
const auto comparator = dht::ring_position_comparator(s);
if (is_reversed && !range.is_singular()) {
const auto& end = range.end();
@@ -72,8 +74,8 @@ bool querier::ring_position_matches(const dht::partition_range& range, const que
return start && comparator(start->value(), expected_start) == 0 && start->is_inclusive() == expected_inclusiveness;
}
bool querier::clustering_position_matches(const query::partition_slice& slice, const querier::position& pos) const {
const auto& row_ranges = slice.row_ranges(*_schema, pos.partition_key->key());
static bool clustering_position_matches(const schema& s, const query::partition_slice& slice, const position_view& pos) {
const auto& row_ranges = slice.row_ranges(s, pos.partition_key->key());
if (row_ranges.empty()) {
// This is a valid slice on the last page of a query with
@@ -89,9 +91,9 @@ bool querier::clustering_position_matches(const query::partition_slice& slice, c
return &row_ranges == &slice.default_row_ranges();
}
clustering_key_prefix::equality eq(*_schema);
clustering_key_prefix::equality eq(s);
const auto is_reversed = flat_mutation_reader::consume_reversed_partitions(_slice->options.contains(query::partition_slice::option::reversed));
const auto is_reversed = flat_mutation_reader::consume_reversed_partitions(slice.options.contains(query::partition_slice::option::reversed));
// If the page ended mid-partition the first partition range should start
// with the last clustering key (exclusive).
@@ -103,41 +105,76 @@ bool querier::clustering_position_matches(const query::partition_slice& slice, c
return !start->is_inclusive() && eq(start->value(), *pos.clustering_key);
}
bool querier::matches(const dht::partition_range& range) const {
const auto& qr = *_range;
if (qr.is_singular() != range.is_singular()) {
static bool ranges_match(const schema& s, const dht::partition_range& original_range, const dht::partition_range& new_range) {
if (original_range.is_singular() != new_range.is_singular()) {
return false;
}
const auto cmp = dht::ring_position_comparator(*_schema);
const auto cmp = dht::ring_position_comparator(s);
const auto bound_eq = [&] (const stdx::optional<dht::partition_range::bound>& a, const stdx::optional<dht::partition_range::bound>& b) {
return bool(a) == bool(b) && (!a || a->equal(*b, cmp));
};
// For singular ranges end() == start() so they are interchangable.
// For singular ranges end() == start() so they are interchangeable.
// For non-singular ranges we check only the end().
return bound_eq(qr.end(), range.end());
return bound_eq(original_range.end(), new_range.end());
}
querier::can_use querier::can_be_used_for_page(emit_only_live_rows only_live, const ::schema& s,
const dht::partition_range& range, const query::partition_slice& slice) const {
if (only_live != emit_only_live_rows(std::holds_alternative<lw_shared_ptr<compact_for_data_query_state>>(_compaction_state))) {
return can_use::no_emit_only_live_rows_mismatch;
static bool ranges_match(const schema& s, dht::partition_ranges_view original_ranges, dht::partition_ranges_view new_ranges) {
if (new_ranges.empty()) {
return false;
}
if (s.version() != _schema->version()) {
if (original_ranges.size() == 1) {
if (new_ranges.size() != 1) {
return false;
}
return ranges_match(s, original_ranges.front(), new_ranges.front());
}
// As the query progresses the number of to-be-read ranges can never surpass
// that of the original ranges.
if (original_ranges.size() < new_ranges.size()) {
return false;
}
// If there is a difference in the size of the range lists we assume we
// already read ranges from the original list and these ranges are missing
// from the head of the new list.
auto new_ranges_it = new_ranges.begin();
auto original_ranges_it = original_ranges.begin() + (original_ranges.size() - new_ranges.size());
// The first range in the new list can be partially read so we only check
// that one of its bounds match that of its original counterpart, just like
// we do with single ranges.
if (!ranges_match(s, *original_ranges_it++, *new_ranges_it++)) {
return false;
}
const auto cmp = dht::ring_position_comparator(s);
// The rest of the list, those ranges that we didn't even started reading
// yet should be *identical* to their original counterparts.
return std::equal(original_ranges_it, original_ranges.end(), new_ranges_it,
[&cmp] (const dht::partition_range& a, const dht::partition_range& b) { return a.equal(b, cmp); });
}
template <typename Querier>
static can_use can_be_used_for_page(const Querier& q, const schema& s, const dht::partition_range& range, const query::partition_slice& slice) {
if (s.version() != q.schema()->version()) {
return can_use::no_schema_version_mismatch;
}
const auto pos = current_position();
const auto pos = q.current_position();
if (!pos.partition_key) {
// There was nothing read so far so we assume we are ok.
return can_use::yes;
}
if (!ring_position_matches(range, pos)) {
if (!ring_position_matches(s, range, slice, pos)) {
return can_use::no_ring_pos_mismatch;
}
if (!clustering_position_matches(slice, pos)) {
if (!clustering_position_matches(s, slice, pos)) {
return can_use::no_clustering_pos_mismatch;
}
return can_use::yes;
@@ -158,23 +195,24 @@ void querier_cache::scan_cache_entries() {
}
}
querier_cache::entries::iterator querier_cache::find_querier(utils::UUID key, const dht::partition_range& range, tracing::trace_state_ptr trace_state) {
const auto queriers = _index.equal_range(key);
static querier_cache::entries::iterator find_querier(querier_cache::entries& entries, querier_cache::index& index, utils::UUID key,
dht::partition_ranges_view ranges, tracing::trace_state_ptr trace_state) {
const auto queriers = index.equal_range(key);
if (queriers.first == _index.end()) {
if (queriers.first == index.end()) {
tracing::trace(trace_state, "Found no cached querier for key {}", key);
return _entries.end();
return entries.end();
}
const auto it = std::find_if(queriers.first, queriers.second, [&] (const entry& e) {
return e.value().matches(range);
const auto it = std::find_if(queriers.first, queriers.second, [&] (const querier_cache::entry& e) {
return ranges_match(e.schema(), e.ranges(), ranges);
});
if (it == queriers.second) {
tracing::trace(trace_state, "Found cached querier(s) for key {} but none matches the query range {}", key, range);
return _entries.end();
tracing::trace(trace_state, "Found cached querier(s) for key {} but none matches the query range(s) {}", key, ranges);
return entries.end();
}
tracing::trace(trace_state, "Found cached querier for key {} and range {}", key, range);
tracing::trace(trace_state, "Found cached querier for key {} and range(s) {}", key, ranges);
return it->pos();
}
@@ -185,7 +223,9 @@ querier_cache::querier_cache(size_t max_cache_size, std::chrono::seconds entry_t
_expiry_timer.arm_periodic(entry_ttl / 2);
}
void querier_cache::insert(utils::UUID key, querier&& q, tracing::trace_state_ptr trace_state) {
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) {
// 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
@@ -196,7 +236,7 @@ void querier_cache::insert(utils::UUID key, querier&& q, tracing::trace_state_pt
tracing::trace(trace_state, "Caching querier with key {}", key);
auto memory_usage = boost::accumulate(_entries | boost::adaptors::transformed(std::mem_fn(&entry::memory_usage)), size_t(0));
auto memory_usage = boost::accumulate(entries | boost::adaptors::transformed(std::mem_fn(&querier_cache::entry::memory_usage)), size_t(0));
// We add the memory-usage of the to-be added querier to the memory-usage
// of all the cached queriers. We now need to makes sure this number is
@@ -205,50 +245,91 @@ void querier_cache::insert(utils::UUID key, querier&& q, tracing::trace_state_pt
// it goes below the limit.
memory_usage += q.memory_usage();
if (memory_usage >= _max_queriers_memory_usage) {
auto it = _entries.begin();
const auto end = _entries.end();
while (it != end && memory_usage >= _max_queriers_memory_usage) {
++_stats.memory_based_evictions;
if (memory_usage >= max_queriers_memory_usage) {
auto it = entries.begin();
const auto end = entries.end();
while (it != end && memory_usage >= max_queriers_memory_usage) {
++stats.memory_based_evictions;
memory_usage -= it->memory_usage();
--_stats.population;
it = _entries.erase(it);
--stats.population;
it = entries.erase(it);
}
}
auto& e = _entries.emplace_back(key, std::move(q), lowres_clock::now() + _entry_ttl);
e.set_pos(--_entries.end());
_index.insert(e);
++_stats.population;
auto& e = entries.emplace_back(key, std::move(q), expires);
e.set_pos(--entries.end());
index.insert(e);
++stats.population;
}
querier querier_cache::lookup(utils::UUID key,
emit_only_live_rows only_live,
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,
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,
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,
std::move(trace_state));
}
template <typename Querier>
static std::optional<Querier> lookup_querier(querier_cache::entries& entries,
querier_cache::index& index,
querier_cache::stats& stats,
utils::UUID key,
const schema& s,
const dht::partition_range& range,
dht::partition_ranges_view ranges,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state,
const noncopyable_function<querier()>& create_fun) {
auto it = find_querier(key, range, trace_state);
++_stats.lookups;
if (it == _entries.end()) {
++_stats.misses;
return create_fun();
tracing::trace_state_ptr trace_state) {
auto it = find_querier(entries, index, key, ranges, trace_state);
++stats.lookups;
if (it == entries.end()) {
++stats.misses;
return std::nullopt;
}
auto q = std::move(*it).value();
_entries.erase(it);
--_stats.population;
auto q = std::move(*it).template value<Querier>();
entries.erase(it);
--stats.population;
const auto can_be_used = q.can_be_used_for_page(only_live, s, range, slice);
if (can_be_used == querier::can_use::yes) {
const auto can_be_used = can_be_used_for_page(q, s, ranges.front(), slice);
if (can_be_used == can_use::yes) {
tracing::trace(trace_state, "Reusing querier");
return q;
return std::optional<Querier>(std::move(q));
}
tracing::trace(trace_state, "Dropping querier because {}", cannot_use_reason(can_be_used));
++_stats.drops;
return create_fun();
++stats.drops;
return std::nullopt;
}
std::optional<data_querier> querier_cache::lookup_data_querier(utils::UUID key,
const schema& s,
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));
}
std::optional<mutation_querier> querier_cache::lookup_mutation_querier(utils::UUID key,
const schema& s,
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));
}
std::optional<shard_mutation_querier> querier_cache::lookup_shard_mutation_querier(utils::UUID key,
const schema& s,
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));
}
void querier_cache::set_entry_ttl(std::chrono::seconds entry_ttl) {
@@ -287,20 +368,52 @@ querier_cache_context::querier_cache_context(querier_cache& cache, utils::UUID k
, _is_first_page(is_first_page) {
}
void querier_cache_context::insert(querier&& q, tracing::trace_state_ptr trace_state) {
void querier_cache_context::insert(data_querier&& q, tracing::trace_state_ptr trace_state) {
if (_cache && _key != utils::UUID{}) {
_cache->insert(_key, std::move(q), std::move(trace_state));
}
}
querier querier_cache_context::lookup(emit_only_live_rows only_live,
const schema& s,
void querier_cache_context::insert(mutation_querier&& q, tracing::trace_state_ptr trace_state) {
if (_cache && _key != utils::UUID{}) {
_cache->insert(_key, std::move(q), std::move(trace_state));
}
}
void querier_cache_context::insert(shard_mutation_querier&& q, tracing::trace_state_ptr trace_state) {
if (_cache && _key != utils::UUID{}) {
_cache->insert(_key, std::move(q), std::move(trace_state));
}
}
std::optional<data_querier> querier_cache_context::lookup_data_querier(const schema& s,
const dht::partition_range& range,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state,
const noncopyable_function<querier()>& create_fun) {
tracing::trace_state_ptr trace_state) {
if (_cache && _key != utils::UUID{} && !_is_first_page) {
return _cache->lookup(_key, only_live, s, range, slice, std::move(trace_state), create_fun);
return _cache->lookup_data_querier(_key, s, range, slice, std::move(trace_state));
}
return create_fun();
return std::nullopt;
}
std::optional<mutation_querier> querier_cache_context::lookup_mutation_querier(const schema& s,
const dht::partition_range& range,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state) {
if (_cache && _key != utils::UUID{} && !_is_first_page) {
return _cache->lookup_mutation_querier(_key, s, range, slice, std::move(trace_state));
}
return std::nullopt;
}
std::optional<shard_mutation_querier> querier_cache_context::lookup_shard_mutation_querier(const schema& s,
const dht::partition_range_vector& ranges,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state) {
if (_cache && _key != utils::UUID{} && !_is_first_page) {
return _cache->lookup_shard_mutation_querier(_key, s, ranges, slice, std::move(trace_state));
}
return std::nullopt;
}
} // namespace query

View File

@@ -28,11 +28,97 @@
#include <variant>
namespace query {
template <typename Consumer>
class clustering_position_tracker {
std::unique_ptr<Consumer> _consumer;
lw_shared_ptr<std::optional<clustering_key_prefix>> _last_ckey;
public:
clustering_position_tracker(std::unique_ptr<Consumer>&& consumer, lw_shared_ptr<std::optional<clustering_key_prefix>> last_ckey)
: _consumer(std::move(consumer))
, _last_ckey(std::move(last_ckey)) {
}
void consume_new_partition(const dht::decorated_key& dk) {
_last_ckey->reset();
_consumer->consume_new_partition(dk);
}
void consume(tombstone t) {
_consumer->consume(t);
}
stop_iteration consume(static_row&& sr, tombstone t, bool is_live) {
return _consumer->consume(std::move(sr), std::move(t), is_live);
}
stop_iteration consume(clustering_row&& cr, row_tombstone t, bool is_live) {
*_last_ckey = cr.key();
return _consumer->consume(std::move(cr), std::move(t), is_live);
}
stop_iteration consume(range_tombstone&& rt) {
return _consumer->consume(std::move(rt));
}
stop_iteration consume_end_of_partition() {
return _consumer->consume_end_of_partition();
}
auto consume_end_of_stream() {
return _consumer->consume_end_of_stream();
}
};
/// Consume a page worth of data from the reader.
///
/// Uses `compaction_state` for compacting the fragments and `consumer` for
/// building the results.
/// Returns a future containing the last consumed clustering key, or std::nullopt
/// if the last row wasn't a clustering row, and whatever the consumer's
/// `consume_end_of_stream()` method returns.
template <emit_only_live_rows OnlyLive, typename Consumer>
GCC6_CONCEPT(
requires CompactedFragmentsConsumer<Consumer>
)
auto consume_page(flat_mutation_reader& reader,
lw_shared_ptr<compact_for_query_state<OnlyLive>> compaction_state,
const query::partition_slice& slice,
Consumer&& consumer,
uint32_t row_limit,
uint32_t partition_limit,
gc_clock::time_point query_time,
db::timeout_clock::time_point timeout) {
// FIXME: #3158
// consumer cannot be moved after consume_new_partition() is called
// on it because it stores references to some of it's own members.
// Move it to the heap before any consumption begins to avoid
// accidents.
return reader.peek().then([=, &reader, consumer = std::make_unique<Consumer>(std::move(consumer)), &slice] (
mutation_fragment* next_fragment) mutable {
const auto next_fragment_kind = next_fragment ? next_fragment->mutation_fragment_kind() : mutation_fragment::kind::partition_end;
compaction_state->start_new_page(row_limit, partition_limit, query_time, next_fragment_kind, *consumer);
const auto is_reversed = flat_mutation_reader::consume_reversed_partitions(
slice.options.contains(query::partition_slice::option::reversed));
auto last_ckey = make_lw_shared<std::optional<clustering_key_prefix>>();
auto reader_consumer = make_stable_flattened_mutations_consumer<compact_for_query<OnlyLive, clustering_position_tracker<Consumer>>>(
compaction_state,
clustering_position_tracker(std::move(consumer), last_ckey));
return reader.consume(std::move(reader_consumer), is_reversed, timeout).then([last_ckey] (auto&&... results) mutable {
return make_ready_future<std::optional<clustering_key_prefix>, std::decay_t<decltype(results)>...>(std::move(*last_ckey), std::move(results)...);
});
});
}
struct position_view {
const dht::decorated_key* partition_key;
const clustering_key_prefix* clustering_key;
};
/// One-stop object for serving queries.
///
/// Encapsulates all state and logic for serving all pages for a given range
/// of a query on a given shard. Can serve mutation or data queries. Can be
/// used with any CompactedMutationsConsumer certified result-builder.
/// of a query on a given shard. Can be used with any CompactedMutationsConsumer
/// certified result-builder.
/// Intended to be created on the first page of a query then saved and reused on
/// subsequent pages.
/// (1) Create with the parameters of your query.
@@ -44,85 +130,19 @@
/// Most result builders have memory-accounters that will stop the read
/// once some memory limit was reached. This is called a short read as the
/// read stops before the row and/or partition limits are reached.
/// (4) At the beginning of the next page use can_be_used_for_page() to
/// determine whether it can be used with the page's schema and start
/// position. If a schema or position mismatch is detected the querier
/// cannot be used to produce the next page and a new one has to be created
/// (4) At the beginning of the next page validate whether it can be used with
/// the page's schema and start position. In case a schema or position
/// mismatch is detected the querier shouldn't be used to produce the next
/// page. It should be dropped instead and a new one should be created
/// instead.
template <emit_only_live_rows OnlyLive>
class querier {
public:
enum class can_use {
yes,
no_emit_only_live_rows_mismatch,
no_schema_version_mismatch,
no_ring_pos_mismatch,
no_clustering_pos_mismatch
};
private:
template <typename Consumer>
class clustering_position_tracker {
std::unique_ptr<Consumer> _consumer;
lw_shared_ptr<std::optional<clustering_key_prefix>> _last_ckey;
public:
clustering_position_tracker(std::unique_ptr<Consumer>&& consumer, lw_shared_ptr<std::optional<clustering_key_prefix>> last_ckey)
: _consumer(std::move(consumer))
, _last_ckey(std::move(last_ckey)) {
}
void consume_new_partition(const dht::decorated_key& dk) {
_last_ckey->reset();
_consumer->consume_new_partition(dk);
}
void consume(tombstone t) {
_consumer->consume(t);
}
stop_iteration consume(static_row&& sr, tombstone t, bool is_live) {
return _consumer->consume(std::move(sr), std::move(t), is_live);
}
stop_iteration consume(clustering_row&& cr, row_tombstone t, bool is_live) {
*_last_ckey = cr.key();
return _consumer->consume(std::move(cr), std::move(t), is_live);
}
stop_iteration consume(range_tombstone&& rt) {
return _consumer->consume(std::move(rt));
}
stop_iteration consume_end_of_partition() {
return _consumer->consume_end_of_partition();
}
auto consume_end_of_stream() {
return _consumer->consume_end_of_stream();
}
};
struct position {
const dht::decorated_key* partition_key;
const clustering_key_prefix* clustering_key;
};
schema_ptr _schema;
std::unique_ptr<const dht::partition_range> _range;
std::unique_ptr<const query::partition_slice> _slice;
flat_mutation_reader _reader;
std::variant<lw_shared_ptr<compact_for_mutation_query_state>, lw_shared_ptr<compact_for_data_query_state>> _compaction_state;
lw_shared_ptr<std::optional<clustering_key_prefix>> _last_ckey;
std::variant<lw_shared_ptr<compact_for_mutation_query_state>, lw_shared_ptr<compact_for_data_query_state>> make_compaction_state(
const schema& s,
gc_clock::time_point query_time,
emit_only_live_rows only_live) const {
if (only_live == emit_only_live_rows::yes) {
return make_lw_shared<compact_for_query_state<emit_only_live_rows::yes>>(s, query_time, *_slice, 0, 0);
} else {
return make_lw_shared<compact_for_query_state<emit_only_live_rows::no>>(s, query_time, *_slice, 0, 0);
}
}
position current_position() const;
bool ring_position_matches(const dht::partition_range& range, const position& pos) const;
bool clustering_position_matches(const query::partition_slice& slice, const position& pos) const;
lw_shared_ptr<compact_for_query_state<OnlyLive>> _compaction_state;
std::optional<clustering_key_prefix> _last_ckey;
public:
querier(const mutation_source& ms,
@@ -130,15 +150,13 @@ public:
dht::partition_range range,
query::partition_slice slice,
const io_priority_class& pc,
tracing::trace_state_ptr trace_ptr,
emit_only_live_rows only_live)
tracing::trace_state_ptr trace_ptr)
: _schema(schema)
, _range(std::make_unique<dht::partition_range>(std::move(range)))
, _slice(std::make_unique<query::partition_slice>(std::move(slice)))
, _reader(ms.make_reader(schema, *_range, *_slice, pc, std::move(trace_ptr),
streamed_mutation::forwarding::no, mutation_reader::forwarding::no))
, _compaction_state(make_compaction_state(*schema, gc_clock::time_point{}, only_live))
, _last_ckey(make_lw_shared<std::optional<clustering_key_prefix>>()) {
, _compaction_state(make_lw_shared<compact_for_query_state<OnlyLive>>(*schema, gc_clock::time_point{}, *_slice, 0, 0)) {
}
bool is_reversed() const {
@@ -146,28 +164,9 @@ public:
}
bool are_limits_reached() const {
return std::visit([] (const auto& cs) { return cs->are_limits_reached(); }, _compaction_state);
return _compaction_state->are_limits_reached();
}
/// Does the querier's range matches `range`?
///
/// A query can have more then one querier executing parallelly for
/// different sub-ranges on the same shard. This method helps identifying
/// the appropriate one for the `range'.
/// For the purposes of this identification it is enough to check that the
/// singulariness and end bound of the ranges matches. For non-singular
/// ranges the start bound may be adjusted from page-to-page as the query
/// progresses through it but since a query is guaranteed to be broken into
/// non-overlapping ranges just checking the end-bound is enough.
bool matches(const dht::partition_range& range) const;
/// Can the querier be used for the next page?
///
/// The querier can only be used for the next page if the only_live, the
/// schema versions, the ring and the clustering positions match.
can_use can_be_used_for_page(emit_only_live_rows only_live, const schema& s,
const dht::partition_range& range, const query::partition_slice& slice) const;
template <typename Consumer>
GCC6_CONCEPT(
requires CompactedFragmentsConsumer<Consumer>
@@ -177,28 +176,11 @@ public:
uint32_t partition_limit,
gc_clock::time_point query_time,
db::timeout_clock::time_point timeout) {
return std::visit([=, consumer = std::move(consumer)] (auto& compaction_state) mutable {
// FIXME: #3158
// consumer cannot be moved after consume_new_partition() is called
// on it because it stores references to some of it's own members.
// Move it to the heap before any consumption begins to avoid
// accidents.
return _reader.peek().then([=, consumer = std::make_unique<Consumer>(std::move(consumer))] (mutation_fragment* next_fragment) mutable {
const auto next_fragment_kind = next_fragment ? next_fragment->mutation_fragment_kind() : mutation_fragment::kind::partition_end;
compaction_state->start_new_page(row_limit, partition_limit, query_time, next_fragment_kind, *consumer);
const auto is_reversed = flat_mutation_reader::consume_reversed_partitions(
_slice->options.contains(query::partition_slice::option::reversed));
using compaction_state_type = typename std::remove_reference<decltype(*compaction_state)>::type;
constexpr auto only_live = compaction_state_type::parameters::only_live;
auto reader_consumer = make_stable_flattened_mutations_consumer<compact_for_query<only_live, clustering_position_tracker<Consumer>>>(
compaction_state,
clustering_position_tracker(std::move(consumer), _last_ckey));
return _reader.consume(std::move(reader_consumer), is_reversed, timeout);
});
}, _compaction_state);
return ::query::consume_page(_reader, _compaction_state, *_slice, std::move(consumer), row_limit, partition_limit, query_time,
timeout).then([this] (std::optional<clustering_key_prefix> last_ckey, auto&&... results) {
_last_ckey = std::move(last_ckey);
return make_ready_future<std::decay_t<decltype(results)>...>(std::move(results)...);
});
}
size_t memory_usage() const {
@@ -208,6 +190,85 @@ public:
schema_ptr schema() const {
return _schema;
}
position_view current_position() const {
const dht::decorated_key* dk = _compaction_state->current_partition();
const clustering_key_prefix* clustering_key = _last_ckey ? &*_last_ckey : nullptr;
return {dk, clustering_key};
}
dht::partition_ranges_view ranges() const {
return *_range;
}
};
using data_querier = querier<emit_only_live_rows::yes>;
using mutation_querier = querier<emit_only_live_rows::no>;
/// Local state of a multishard query.
///
/// This querier is not intended to be used directly to read pages. Instead it
/// is merely a shard local state of a suspended multishard query and is
/// intended to be used for storing the state of the query on each shard where
/// it executes. It stores the local reader and the referenced parameters it was
/// created with (similar to other queriers).
/// For position validation purposes (at lookup) the reader's position is
/// considered to be the same as that of the query.
class shard_mutation_querier {
dht::partition_range_vector _query_ranges;
std::unique_ptr<const dht::partition_range> _reader_range;
std::unique_ptr<const query::partition_slice> _reader_slice;
flat_mutation_reader _reader;
dht::decorated_key _nominal_pkey;
std::optional<clustering_key_prefix> _nominal_ckey;
public:
shard_mutation_querier(
const dht::partition_range_vector query_ranges,
std::unique_ptr<const dht::partition_range> reader_range,
std::unique_ptr<const query::partition_slice> reader_slice,
flat_mutation_reader reader,
dht::decorated_key nominal_pkey,
std::optional<clustering_key_prefix> nominal_ckey)
: _query_ranges(std::move(query_ranges))
, _reader_range(std::move(reader_range))
, _reader_slice(std::move(reader_slice))
, _reader(std::move(reader))
, _nominal_pkey(std::move(nominal_pkey))
, _nominal_ckey(std::move(nominal_ckey)) {
}
bool is_reversed() const {
return _reader_slice->options.contains(query::partition_slice::option::reversed);
}
size_t memory_usage() const {
return _reader.buffer_size();
}
schema_ptr schema() const {
return _reader.schema();
}
position_view current_position() const {
return {&_nominal_pkey, _nominal_ckey ? &*_nominal_ckey : nullptr};
}
dht::partition_ranges_view ranges() const {
return _query_ranges;
}
std::unique_ptr<const dht::partition_range> reader_range() && {
return std::move(_reader_range);
}
std::unique_ptr<const query::partition_slice> reader_slice() && {
return std::move(_reader_slice);
}
flat_mutation_reader reader() && {
return std::move(_reader);
}
};
/// Special-purpose cache for saving queriers between pages.
@@ -260,16 +321,16 @@ public:
uint64_t population = 0;
};
private:
class entry : public boost::intrusive::set_base_hook<boost::intrusive::link_mode<boost::intrusive::auto_unlink>> {
// Self reference so that we can remove the entry given an `entry&`.
std::list<entry>::iterator _pos;
const utils::UUID _key;
const lowres_clock::time_point _expires;
querier _value;
std::variant<data_querier, mutation_querier, shard_mutation_querier> _value;
public:
entry(utils::UUID key, querier q, lowres_clock::time_point expires)
template <typename Querier>
entry(utils::UUID key, Querier q, lowres_clock::time_point expires)
: _key(key)
, _expires(expires)
, _value(std::move(q)) {
@@ -288,7 +349,15 @@ private:
}
const ::schema& schema() const {
return *_value.schema();
return *std::visit([] (auto& q) {
return q.schema();
}, _value);
}
dht::partition_ranges_view ranges() const {
return std::visit([] (auto& q) {
return q.ranges();
}, _value);
}
bool is_expired(const lowres_clock::time_point& now) const {
@@ -296,15 +365,19 @@ private:
}
size_t memory_usage() const {
return _value.memory_usage();
return std::visit([] (auto& q) {
return q.memory_usage();
}, _value);
}
const querier& value() const & {
return _value;
template <typename Querier>
const Querier& value() const & {
return std::get<Querier>(_value);
}
querier value() && {
return std::move(_value);
template <typename Querier>
Querier value() && {
return std::get<Querier>(std::move(_value));
}
};
@@ -319,14 +392,14 @@ private:
private:
entries _entries;
index _index;
index _data_querier_index;
index _mutation_querier_index;
index _shard_mutation_querier_index;
timer<lowres_clock> _expiry_timer;
std::chrono::seconds _entry_ttl;
stats _stats;
size_t _max_queriers_memory_usage;
entries::iterator find_querier(utils::UUID key, const dht::partition_range& range, tracing::trace_state_ptr trace_state);
void scan_cache_entries();
public:
@@ -339,11 +412,13 @@ public:
querier_cache(querier_cache&&) = delete;
querier_cache& operator=(querier_cache&&) = delete;
void insert(utils::UUID key, querier&& q, tracing::trace_state_ptr trace_state);
void insert(utils::UUID key, data_querier&& q, tracing::trace_state_ptr trace_state);
/// Lookup a querier in the cache.
///
/// If the querier doesn't exist, use `create_fun' to create it.
void insert(utils::UUID key, mutation_querier&& q, tracing::trace_state_ptr trace_state);
void insert(utils::UUID key, shard_mutation_querier&& q, tracing::trace_state_ptr trace_state);
/// Lookup a data querier in the cache.
///
/// Queriers are found based on `key` and `range`. There may be multiple
/// queriers for the same `key` differentiated by their read range. Since
@@ -353,17 +428,32 @@ public:
/// non-overlapping ranges. Thus both bounds of any range, or in case of
/// singular ranges only the start bound are guaranteed to be unique.
///
/// The found querier is checked for a matching read-kind and schema
/// version. The start position is also checked against the current
/// position of the querier using the `range' and `slice'. If there is a
/// mismatch drop the querier and create a new one with `create_fun'.
querier lookup(utils::UUID key,
emit_only_live_rows only_live,
/// The found querier is checked for a matching position and schema version.
/// The start position of the querier is checked against the start position
/// of the page using the `range' and `slice'.
std::optional<data_querier> lookup_data_querier(utils::UUID key,
const schema& s,
const dht::partition_range& range,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state,
const noncopyable_function<querier()>& create_fun);
tracing::trace_state_ptr trace_state);
/// Lookup a mutation querier in the cache.
///
/// See \ref lookup_data_querier().
std::optional<mutation_querier> lookup_mutation_querier(utils::UUID key,
const schema& s,
const dht::partition_range& range,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state);
/// Lookup a shard mutation querier in the cache.
///
/// See \ref lookup_data_querier().
std::optional<shard_mutation_querier> lookup_shard_mutation_querier(utils::UUID key,
const schema& s,
const dht::partition_range_vector& ranges,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state);
void set_entry_ttl(std::chrono::seconds entry_ttl);
@@ -391,11 +481,21 @@ class querier_cache_context {
public:
querier_cache_context() = default;
querier_cache_context(querier_cache& cache, utils::UUID key, bool is_first_page);
void insert(querier&& q, tracing::trace_state_ptr trace_state);
querier lookup(emit_only_live_rows only_live,
const schema& s,
void insert(data_querier&& q, tracing::trace_state_ptr trace_state);
void insert(mutation_querier&& q, tracing::trace_state_ptr trace_state);
void insert(shard_mutation_querier&& q, tracing::trace_state_ptr trace_state);
std::optional<data_querier> lookup_data_querier(const schema& s,
const dht::partition_range& range,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state,
const noncopyable_function<querier()>& create_fun);
tracing::trace_state_ptr trace_state);
std::optional<mutation_querier> lookup_mutation_querier(const schema& s,
const dht::partition_range& range,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state);
std::optional<shard_mutation_querier> lookup_shard_mutation_querier(const schema& s,
const dht::partition_range_vector& ranges,
const query::partition_slice& slice,
tracing::trace_state_ptr trace_state);
};
} // namespace query

View File

@@ -263,6 +263,10 @@ public:
return _range_tombstones;
}
std::deque<range_tombstone> range_tombstones() && {
return std::move(_range_tombstones);
}
void apply(range_tombstone rt);
void clear();

View File

@@ -92,10 +92,7 @@ static bool has_clustering_keys(const schema& s, const query::read_command& cmd)
_last_replicas = state->get_last_replicas();
_query_read_repair_decision = state->get_query_read_repair_decision();
} else {
// Reusing readers is currently only supported for singular queries.
if (!_ranges.empty() && query::is_single_partition(_ranges.front())) {
_cmd->query_uuid = utils::make_random_uuid();
}
_cmd->query_uuid = utils::make_random_uuid();
_cmd->is_first_page = true;
}
qlogger.trace("fetch_page query id {}", _cmd->query_uuid);

View File

@@ -82,6 +82,7 @@
#include "core/metrics.hh"
#include <seastar/core/execution_stage.hh>
#include "db/timeout_clock.hh"
#include "multishard_mutation_query.hh"
namespace service {
@@ -3155,11 +3156,18 @@ storage_proxy::query_singular(lw_shared_ptr<query::read_command> cmd,
});
}
future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>>
storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::time_point timeout, std::vector<foreign_ptr<lw_shared_ptr<query::result>>>&& results,
lw_shared_ptr<query::read_command> cmd, db::consistency_level cl, dht::partition_range_vector::iterator&& i,
dht::partition_range_vector&& ranges, int concurrency_factor, tracing::trace_state_ptr trace_state,
uint32_t remaining_row_count, uint32_t remaining_partition_count) {
future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>, replicas_per_token_range>
storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::time_point timeout,
std::vector<foreign_ptr<lw_shared_ptr<query::result>>>&& results,
lw_shared_ptr<query::read_command> cmd,
db::consistency_level cl,
dht::partition_range_vector::iterator&& i,
dht::partition_range_vector&& ranges,
int concurrency_factor,
tracing::trace_state_ptr trace_state,
uint32_t remaining_row_count,
uint32_t remaining_partition_count,
replicas_per_token_range preferred_replicas) {
schema_ptr schema = local_schema_registry().get(cmd->schema_version);
keyspace& ks = _db.local().find_keyspace(schema->ks_name());
std::vector<::shared_ptr<abstract_read_executor>> exec;
@@ -3167,12 +3175,20 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
auto p = shared_from_this();
auto& cf= _db.local().find_column_family(schema);
auto pcf = _db.local().get_config().cache_hit_rate_read_balancing() ? &cf : nullptr;
std::unordered_map<abstract_read_executor*, std::vector<dht::token_range>> ranges_per_exec;
const auto preferred_replicas_for_range = [&preferred_replicas] (const dht::partition_range& r) {
auto it = preferred_replicas.find(r.transform(std::mem_fn(&dht::ring_position::token)));
return it == preferred_replicas.end() ? std::vector<gms::inet_address>{} : replica_ids_to_endpoints(it->second);
};
const auto to_token_range = [] (const dht::partition_range& r) { return r.transform(std::mem_fn(&dht::ring_position::token)); };
while (i != ranges.end() && std::distance(concurrent_fetch_starting_index, i) < concurrency_factor) {
dht::partition_range& range = *i;
std::vector<gms::inet_address> live_endpoints = get_live_sorted_endpoints(ks, end_token(range));
std::vector<gms::inet_address> filtered_endpoints = filter_for_query(cl, ks, live_endpoints, pcf);
std::vector<gms::inet_address> merged_preferred_replicas = preferred_replicas_for_range(*i);
std::vector<gms::inet_address> filtered_endpoints = filter_for_query(cl, ks, live_endpoints, merged_preferred_replicas, pcf);
std::vector<dht::token_range> merged_ranges{to_token_range(range)};
++i;
// getRestrictedRange has broken the queried range into per-[vnode] token ranges, but this doesn't take
@@ -3180,9 +3196,10 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
// still meets the CL requirements, then we can merge both ranges into the same RangeSliceCommand.
while (i != ranges.end())
{
const auto current_range_preferred_replicas = preferred_replicas_for_range(*i);
dht::partition_range& next_range = *i;
std::vector<gms::inet_address> next_endpoints = get_live_sorted_endpoints(ks, end_token(next_range));
std::vector<gms::inet_address> next_filtered_endpoints = filter_for_query(cl, ks, next_endpoints, pcf);
std::vector<gms::inet_address> next_filtered_endpoints = filter_for_query(cl, ks, next_endpoints, current_range_preferred_replicas, pcf);
// Origin has this to say here:
// * If the current range right is the min token, we should stop merging because CFS.getRangeSlice
@@ -3196,13 +3213,14 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
}
std::vector<gms::inet_address> merged = intersection(live_endpoints, next_endpoints);
std::vector<gms::inet_address> current_merged_preferred_replicas = intersection(merged_preferred_replicas, current_range_preferred_replicas);
// Check if there is enough endpoint for the merge to be possible.
if (!is_sufficient_live_nodes(cl, ks, merged)) {
break;
}
std::vector<gms::inet_address> filtered_merged = filter_for_query(cl, ks, merged, pcf);
std::vector<gms::inet_address> filtered_merged = filter_for_query(cl, ks, merged, current_merged_preferred_replicas, pcf);
// Estimate whether merging will be a win or not
if (!locator::i_endpoint_snitch::get_local_snitch_ptr()->is_worth_merging_for_range_query(filtered_merged, filtered_endpoints, next_filtered_endpoints)) {
@@ -3231,8 +3249,10 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
// If we get there, merge this range and the next one
range = dht::partition_range(range.start(), next_range.end());
live_endpoints = std::move(merged);
merged_preferred_replicas = std::move(current_merged_preferred_replicas);
filtered_endpoints = std::move(filtered_merged);
++i;
merged_ranges.push_back(to_token_range(next_range));
}
slogger.trace("creating range read executor with targets {}", filtered_endpoints);
try {
@@ -3244,6 +3264,7 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
}
exec.push_back(::make_shared<range_slice_read_executor>(schema, cf.shared_from_this(), p, cmd, std::move(range), cl, std::move(filtered_endpoints), trace_state));
ranges_per_exec.emplace(exec.back().get(), std::move(merged_ranges));
}
query::result_merger merger(cmd->row_limit, cmd->partition_limit);
@@ -3253,24 +3274,47 @@ storage_proxy::query_partition_key_range_concurrent(storage_proxy::clock_type::t
return rex->execute(timeout);
}, std::move(merger));
return f.then([p, exec = std::move(exec), results = std::move(results), i = std::move(i), ranges = std::move(ranges),
cl, cmd, concurrency_factor, timeout, remaining_row_count, remaining_partition_count, trace_state = std::move(trace_state)]
(foreign_ptr<lw_shared_ptr<query::result>>&& result) mutable {
return f.then([p,
exec = std::move(exec),
results = std::move(results),
i = std::move(i),
ranges = std::move(ranges),
cl,
cmd,
concurrency_factor,
timeout,
remaining_row_count,
remaining_partition_count,
trace_state = std::move(trace_state),
preferred_replicas = std::move(preferred_replicas),
ranges_per_exec = std::move(ranges_per_exec)] (foreign_ptr<lw_shared_ptr<query::result>>&& result) mutable {
result->ensure_counts();
remaining_row_count -= result->row_count().value();
remaining_partition_count -= result->partition_count().value();
results.emplace_back(std::move(result));
if (i == ranges.end() || !remaining_row_count || !remaining_partition_count) {
return make_ready_future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>>(std::move(results));
auto used_replicas = replicas_per_token_range();
for (auto& e : exec) {
// We add used replicas in separate per-vnode entries even if
// they were merged, for two reasons:
// 1) The list of replicas is determined for each vnode
// separately and thus this makes lookups more convenient.
// 2) On the next page the ranges might not be merged.
auto replica_ids = endpoints_to_replica_ids(e->used_targets());
for (auto& r : ranges_per_exec[e.get()]) {
used_replicas.emplace(std::move(r), replica_ids);
}
}
return make_ready_future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>, replicas_per_token_range>(std::move(results), std::move(used_replicas));
} else {
cmd->row_limit = remaining_row_count;
cmd->partition_limit = remaining_partition_count;
return p->query_partition_key_range_concurrent(timeout, std::move(results), cmd, cl, std::move(i),
std::move(ranges), concurrency_factor * 2, std::move(trace_state), remaining_row_count, remaining_partition_count);
return p->query_partition_key_range_concurrent(timeout, std::move(results), cmd, cl, std::move(i), std::move(ranges),
concurrency_factor * 2, std::move(trace_state), remaining_row_count, remaining_partition_count, std::move(preferred_replicas));
}
}).handle_exception([p] (std::exception_ptr eptr) {
p->handle_read_error(eptr, true);
return make_exception_future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>>(eptr);
return make_exception_future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>, replicas_per_token_range>(eptr);
});
}
@@ -3326,9 +3370,19 @@ storage_proxy::query_partition_key_range(lw_shared_ptr<query::read_command> cmd,
const auto row_limit = cmd->row_limit;
const auto partition_limit = cmd->partition_limit;
return query_partition_key_range_concurrent(query_options.timeout(*this), std::move(results), cmd, cl, ranges.begin(), std::move(ranges),
concurrency_factor, std::move(query_options.trace_state), cmd->row_limit, cmd->partition_limit)
.then([row_limit, partition_limit](std::vector<foreign_ptr<lw_shared_ptr<query::result>>> results) {
return query_partition_key_range_concurrent(query_options.timeout(*this),
std::move(results),
cmd,
cl,
ranges.begin(),
std::move(ranges),
concurrency_factor,
std::move(query_options.trace_state),
cmd->row_limit,
cmd->partition_limit,
std::move(query_options.preferred_replicas)).then([row_limit, partition_limit] (
std::vector<foreign_ptr<lw_shared_ptr<query::result>>> results,
replicas_per_token_range used_replicas) {
query::result_merger merger(row_limit, partition_limit);
merger.reserve(results.size());
@@ -3336,7 +3390,7 @@ storage_proxy::query_partition_key_range(lw_shared_ptr<query::read_command> cmd,
merger(std::move(r));
}
return make_ready_future<coordinator_query_result>(coordinator_query_result(merger.get()));
return make_ready_future<coordinator_query_result>(coordinator_query_result(merger.get(), std::move(used_replicas)));
});
}
@@ -4106,178 +4160,6 @@ void storage_proxy::uninit_messaging_service() {
ms.unregister_truncate();
}
// Merges reconcilable_result:s from different shards into one
// Drops partitions which exceed the limit.
class mutation_result_merger {
schema_ptr _schema;
lw_shared_ptr<const query::read_command> _cmd;
unsigned _row_count = 0;
unsigned _partition_count = 0;
bool _short_read_allowed;
// we get a batch of partitions each time, each with a key
// partition batches should be maintained in key order
// batches that share a key should be merged and sorted in decorated_key
// order
struct partitions_batch {
std::vector<partition> partitions;
query::short_read short_read;
};
std::multimap<unsigned, partitions_batch> _partitions;
query::result_memory_accounter _memory_accounter;
stdx::optional<unsigned> _stop_after_key;
public:
explicit mutation_result_merger(schema_ptr schema, lw_shared_ptr<const query::read_command> cmd)
: _schema(std::move(schema))
, _cmd(std::move(cmd))
, _short_read_allowed(_cmd->slice.options.contains(query::partition_slice::option::allow_short_read)) {
}
query::result_memory_accounter& memory() {
return _memory_accounter;
}
const query::result_memory_accounter& memory() const {
return _memory_accounter;
}
void add_result(unsigned key, foreign_ptr<lw_shared_ptr<reconcilable_result>> partial_result) {
if (_stop_after_key && key > *_stop_after_key) {
// A short result was added that goes before this one.
return;
}
std::vector<partition> partitions;
partitions.reserve(partial_result->partitions().size());
// Following three lines to simplify patch; can remove later
for (const partition& p : partial_result->partitions()) {
partitions.push_back(p);
_row_count += p._row_count;
_partition_count += p._row_count > 0;
}
_memory_accounter.update(partial_result->memory_usage());
if (partial_result->is_short_read()) {
_stop_after_key = key;
}
_partitions.emplace(key, partitions_batch { std::move(partitions), partial_result->is_short_read() });
}
reconcilable_result get() && {
auto unsorted = std::unordered_set<unsigned>();
struct partitions_and_last_key {
std::vector<partition> partitions;
stdx::optional<dht::decorated_key> last; // set if we had a short read
};
auto merged = std::map<unsigned, partitions_and_last_key>();
auto short_read = query::short_read(this->short_read());
// merge batches with equal keys, and note if we need to sort afterwards
for (auto&& key_value : _partitions) {
auto&& key = key_value.first;
if (_stop_after_key && key > *_stop_after_key) {
break;
}
auto&& batch = key_value.second;
auto&& dest = merged[key];
if (dest.partitions.empty()) {
dest.partitions = std::move(batch.partitions);
} else {
unsorted.insert(key);
std::move(batch.partitions.begin(), batch.partitions.end(), std::back_inserter(dest.partitions));
}
// In case of a short read we need to remove all partitions from the
// batch that come after the last partition of the short read
// result.
if (batch.short_read) {
// Nobody sends a short read with no data.
const auto& last = dest.partitions.back().mut().decorated_key(*_schema);
if (!dest.last || last.less_compare(*_schema, *dest.last)) {
dest.last = last;
}
short_read = query::short_read::yes;
}
}
// Sort batches that arrived with the same keys
for (auto key : unsorted) {
struct comparator {
const schema& s;
dht::decorated_key::less_comparator dkcmp;
bool operator()(const partition& a, const partition& b) const {
return dkcmp(a.mut().decorated_key(s), b.mut().decorated_key(s));
}
bool operator()(const dht::decorated_key& a, const partition& b) const {
return dkcmp(a, b.mut().decorated_key(s));
}
bool operator()(const partition& a, const dht::decorated_key& b) const {
return dkcmp(a.mut().decorated_key(s), b);
}
};
auto cmp = comparator { *_schema, dht::decorated_key::less_comparator(_schema) };
auto&& batch = merged[key];
boost::sort(batch.partitions, cmp);
if (batch.last) {
// This batch was built from a result that was a short read.
// We need to remove all partitions that are after that short
// read.
auto it = boost::range::upper_bound(batch.partitions, std::move(*batch.last), cmp);
batch.partitions.erase(it, batch.partitions.end());
}
}
auto final = std::vector<partition>();
final.reserve(_partition_count);
for (auto&& batch : merged | boost::adaptors::map_values) {
std::move(batch.partitions.begin(), batch.partitions.end(), std::back_inserter(final));
}
if (short_read) {
// Short read row and partition counts may be incorrect, recalculate.
_row_count = 0;
_partition_count = 0;
for (const auto& p : final) {
_row_count += p.row_count();
_partition_count += p.row_count() > 0;
}
if (_row_count >= _cmd->row_limit || _partition_count > _cmd->partition_limit) {
// Even though there was a short read contributing to the final
// result we got limited by total row limit or partition limit.
// Note that we cannot with trivial check make unset short read flag
// in case _partition_count == _cmd->partition_limit since the short
// read may have caused the last partition to contain less rows
// than asked for.
short_read = query::short_read::no;
}
}
// Trim back partition count and row count in case we overshot.
// Should be rare for dense tables.
while ((_partition_count > _cmd->partition_limit)
|| (_partition_count && (_row_count - final.back().row_count() >= _cmd->row_limit))) {
_row_count -= final.back().row_count();
_partition_count -= final.back().row_count() > 0;
final.pop_back();
}
if (_row_count > _cmd->row_limit) {
auto mut = final.back().mut().unfreeze(_schema);
static const auto all = std::vector<query::clustering_range>({query::clustering_range::make_open_ended_both_sides()});
auto is_reversed = _cmd->slice.options.contains(query::partition_slice::option::reversed);
auto final_rows = _cmd->row_limit - (_row_count - final.back().row_count());
_row_count -= final.back().row_count();
auto rc = mut.partition().compact_for_query(*_schema, _cmd->timestamp, all, is_reversed, final_rows);
final.back() = partition(rc, freeze(mut));
_row_count += rc;
}
return reconcilable_result(_row_count, std::move(final), short_read, std::move(_memory_accounter).done());
}
bool short_read() const {
return bool(_stop_after_key) || (_short_read_allowed && _row_count > 0 && _memory_accounter.check());
}
unsigned partition_count() const {
return _partition_count;
}
unsigned row_count() const {
return _row_count;
}
};
future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>
storage_proxy::query_mutations_locally(schema_ptr s, lw_shared_ptr<query::read_command> cmd, const dht::partition_range& pr,
storage_proxy::clock_type::time_point timeout,
@@ -4308,39 +4190,6 @@ storage_proxy::query_mutations_locally(schema_ptr s, lw_shared_ptr<query::read_c
}
}
}
namespace {
struct element_and_shard {
unsigned element; // element in a partition range vector
unsigned shard;
};
bool operator==(element_and_shard a, element_and_shard b) {
return a.element == b.element && a.shard == b.shard;
}
}
namespace std {
template <>
struct hash<element_and_shard> {
size_t operator()(element_and_shard es) const {
return es.element * 31 + es.shard;
}
};
}
namespace service {
struct partition_range_and_sort_key {
query::partition_range pr;
unsigned sort_key_shard_order; // for the same source partition range, we sort in shard order
};
future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>
storage_proxy::query_nonsingular_mutations_locally(schema_ptr s,
lw_shared_ptr<query::read_command> cmd,
@@ -4348,103 +4197,9 @@ storage_proxy::query_nonsingular_mutations_locally(schema_ptr s,
tracing::trace_state_ptr trace_state,
uint64_t max_size,
storage_proxy::clock_type::time_point timeout) {
// no one permitted us to modify *cmd, so make a copy
auto shard_cmd = make_lw_shared<query::read_command>(*cmd);
auto range_count = static_cast<unsigned>(prs.size());
return do_with(cmd,
shard_cmd,
0u,
false,
range_count,
std::unordered_map<element_and_shard, partition_range_and_sort_key>{},
mutation_result_merger{s, cmd},
dht::ring_position_exponential_vector_sharder{std::move(prs)},
global_schema_ptr(s),
tracing::global_trace_state_ptr(std::move(trace_state)),
cache_temperature(0.0f),
[this, s, max_size, timeout] (lw_shared_ptr<query::read_command>& cmd,
lw_shared_ptr<query::read_command>& shard_cmd,
unsigned& mutation_result_merger_key,
bool& no_more_ranges,
unsigned& partition_range_count,
std::unordered_map<element_and_shard, partition_range_and_sort_key>& shards_for_this_iteration,
mutation_result_merger& mrm,
dht::ring_position_exponential_vector_sharder& rpevs,
global_schema_ptr& gs,
tracing::global_trace_state_ptr& gt,
cache_temperature& hit_rate) {
return _db.local().get_result_memory_limiter().new_mutation_read(max_size).then([&, timeout, s] (query::result_memory_accounter ma) {
mrm.memory() = std::move(ma);
return repeat_until_value([&, s, timeout] () -> future<stdx::optional<std::pair<reconcilable_result, cache_temperature>>> {
// We don't want to query a sparsely populated table sequentially, because the latency
// will go through the roof. We don't want to query a densely populated table in parallel,
// because we'll throw away most of the results. So we'll exponentially increase
// concurrency starting at 1, so we won't waste on dense tables and at most
// `log(nr_shards) + ignore_msb_bits` latency multiplier for near-empty tables.
//
// We use the ring_position_exponential_vector_sharder to give us subranges that follow
// this scheme.
shards_for_this_iteration.clear();
// If we're reading from less than smp::count shards, then we can just append
// each shard in order without sorting. If we're reading from more, then
// we'll read from some shards at least twice, so the partitions within will be
// out-of-order wrt. other shards
auto this_iteration_subranges = rpevs.next(*s);
auto retain_shard_order = true;
no_more_ranges = true;
if (this_iteration_subranges) {
no_more_ranges = false;
retain_shard_order = this_iteration_subranges->inorder;
auto sort_key = 0u;
for (auto&& now : this_iteration_subranges->per_shard_ranges) {
shards_for_this_iteration.emplace(element_and_shard{this_iteration_subranges->element, now.shard}, partition_range_and_sort_key{now.ring_range, sort_key++});
}
}
auto key_base = mutation_result_merger_key;
// prepare for next iteration
// Each iteration uses a merger key that is either i in the loop above (so in the range [0, shards_in_parallel),
// or, the element index in prs (so in the range [0, partition_range_count). Make room for sufficient keys.
mutation_result_merger_key += std::max(smp::count, partition_range_count);
shard_cmd->partition_limit = cmd->partition_limit - mrm.partition_count();
shard_cmd->row_limit = cmd->row_limit - mrm.row_count();
return parallel_for_each(shards_for_this_iteration, [&, key_base, timeout, retain_shard_order] (const std::pair<const element_and_shard, partition_range_and_sort_key>& elem_shard_range) {
auto&& elem = elem_shard_range.first.element;
auto&& shard = elem_shard_range.first.shard;
auto&& range = elem_shard_range.second.pr;
auto sort_key_shard_order = elem_shard_range.second.sort_key_shard_order;
_stats.replica_cross_shard_ops += shard != engine().cpu_id();
return _db.invoke_on(shard, [&, range, gt, fstate = mrm.memory().state_for_another_shard(), timeout] (database& db) {
query::result_memory_accounter accounter(db.get_result_memory_limiter(), std::move(fstate));
return db.query_mutations(gs, *shard_cmd, range, std::move(accounter), std::move(gt), timeout).then([&hit_rate] (reconcilable_result&& rr, cache_temperature ht) {
hit_rate = ht;
return make_foreign(make_lw_shared(std::move(rr)));
});
}).then([&, key_base, retain_shard_order, elem, sort_key_shard_order] (foreign_ptr<lw_shared_ptr<reconcilable_result>> partial_result) {
// Each outer (sequential) iteration is in result order, so we pick increasing keys.
// Within the inner (parallel) iteration, the results can be in order (if retain_shard_order), or not (if !retain_shard_order).
// If the results are unordered, we still have to order them according to which element of prs they originated from.
auto key = key_base; // for outer loop
if (retain_shard_order) {
key += sort_key_shard_order; // inner loop is ordered
} else {
key += elem; // inner loop ordered only by position within prs
}
mrm.add_result(key, std::move(partial_result));
});
}).then([&] () -> stdx::optional<std::pair<reconcilable_result, cache_temperature>> {
if (mrm.short_read() || mrm.partition_count() >= cmd->partition_limit || mrm.row_count() >= cmd->row_limit || no_more_ranges) {
return stdx::make_optional(std::make_pair(std::move(mrm).get(), hit_rate));
}
return stdx::nullopt;
});
});
});
}).then([] (std::pair<reconcilable_result, cache_temperature>&& result) {
return make_ready_future<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_temperature>(make_foreign(make_lw_shared(std::move(result.first))), result.second);
return do_with(cmd, std::move(prs), [=, s = std::move(s), trace_state = std::move(trace_state)] (lw_shared_ptr<query::read_command>& cmd,
const dht::partition_range_vector& prs) mutable {
return query_mutations_on_all_shards(_db, std::move(s), *cmd, prs, std::move(trace_state), max_size, timeout);
});
}

View File

@@ -222,10 +222,17 @@ private:
dht::partition_range_vector get_restricted_ranges(const schema& s, dht::partition_range range);
float estimate_result_rows_per_range(lw_shared_ptr<query::read_command> cmd, keyspace& ks);
static std::vector<gms::inet_address> intersection(const std::vector<gms::inet_address>& l1, const std::vector<gms::inet_address>& l2);
future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>> query_partition_key_range_concurrent(clock_type::time_point timeout,
std::vector<foreign_ptr<lw_shared_ptr<query::result>>>&& results, lw_shared_ptr<query::read_command> cmd, db::consistency_level cl, dht::partition_range_vector::iterator&& i,
dht::partition_range_vector&& ranges, int concurrency_factor, tracing::trace_state_ptr trace_state,
uint32_t remaining_row_count, uint32_t remaining_partition_count);
future<std::vector<foreign_ptr<lw_shared_ptr<query::result>>>, replicas_per_token_range> query_partition_key_range_concurrent(clock_type::time_point timeout,
std::vector<foreign_ptr<lw_shared_ptr<query::result>>>&& results,
lw_shared_ptr<query::read_command> cmd,
db::consistency_level cl,
dht::partition_range_vector::iterator&& i,
dht::partition_range_vector&& ranges,
int concurrency_factor,
tracing::trace_state_ptr trace_state,
uint32_t remaining_row_count,
uint32_t remaining_partition_count,
replicas_per_token_range preferred_replicas);
future<coordinator_query_result> do_query(schema_ptr,
lw_shared_ptr<query::read_command> cmd,

View File

@@ -117,6 +117,7 @@ boost_tests = [
'transport_test',
'fragmented_temporary_buffer_test',
'auth_passwords_test',
'multishard_mutation_query_test',
]
other_tests = [

View File

@@ -0,0 +1,351 @@
/*
* 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 "multishard_mutation_query.hh"
#include "schema_registry.hh"
#include "tests/cql_test_env.hh"
#include "tests/eventually.hh"
#include "tests/mutation_assertions.hh"
#include <seastar/tests/test-utils.hh>
#include <experimental/source_location>
class delete_rows {
int _skip = 0;
int _delete = 0;
public:
delete_rows() = default;
delete_rows(int skip_rows, int delete_rows)
: _skip(skip_rows)
, _delete(delete_rows) {
}
bool should_delete_row(int row) {
if (!_delete) {
return false;
}
const auto total = _skip + _delete;
const auto i = row % total;
return i >= _skip;
}
};
static std::pair<schema_ptr, std::vector<dht::decorated_key>> create_test_cf(cql_test_env& env, delete_rows dr = {}) {
env.execute_cql("CREATE KEYSPACE multishard_mutation_query_cache_ks WITH REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 1};").get();
env.execute_cql("CREATE TABLE multishard_mutation_query_cache_ks.test (pk int, ck int, v int, PRIMARY KEY(pk, ck));").get();
const auto insert_id = env.prepare("INSERT INTO multishard_mutation_query_cache_ks.test (\"pk\", \"ck\", \"v\") VALUES (?, ?, ?);").get0();
const auto delete_id = env.prepare("DELETE FROM multishard_mutation_query_cache_ks.test WHERE pk = ? AND ck = ?;").get0();
auto s = env.local_db().find_column_family("multishard_mutation_query_cache_ks", "test").schema();
std::vector<dht::decorated_key> pkeys;
auto row_counter = int(0);
for (int pk = 0; pk < 10 * static_cast<int>(smp::count); ++pk) {
pkeys.emplace_back(dht::global_partitioner().decorate_key(*s, partition_key::from_single_value(*s, data_value(pk).serialize())));
for (int ck = 0; ck < 10; ++ck) {
env.execute_prepared(insert_id, {{
cql3::raw_value::make_value(data_value(pk).serialize()),
cql3::raw_value::make_value(data_value(ck).serialize()),
cql3::raw_value::make_value(data_value(pk ^ ck).serialize())}}).get();
if (dr.should_delete_row(row_counter++)) {
env.execute_prepared(delete_id, {{
cql3::raw_value::make_value(data_value(pk).serialize()),
cql3::raw_value::make_value(data_value(ck).serialize())}});
}
}
}
return std::pair(std::move(s), std::move(pkeys));
}
static uint64_t aggregate_querier_cache_stat(distributed<database>& db, uint64_t query::querier_cache::stats::*stat) {
return map_reduce(boost::irange(0u, smp::count), [stat, &db] (unsigned shard) {
return db.invoke_on(shard, [stat] (database& local_db) {
auto& stats = local_db.get_querier_cache_stats();
return stats.*stat;
});
}, 0, std::plus<size_t>()).get0();
}
static void check_cache_population(distributed<database>& db, size_t queriers,
std::experimental::source_location sl = std::experimental::source_location::current()) {
BOOST_TEST_MESSAGE(sprint("%s() called from %s() %s:%i", __FUNCTION__, sl.function_name(), sl.file_name(), sl.line()));
parallel_for_each(boost::irange(0u, smp::count), [queriers, &db] (unsigned shard) {
return db.invoke_on(shard, [queriers] (database& local_db) {
auto& stats = local_db.get_querier_cache_stats();
BOOST_REQUIRE_EQUAL(stats.population, queriers);
});
}).get0();
}
static void require_eventually_empty_caches(distributed<database>& db,
std::experimental::source_location sl = std::experimental::source_location::current()) {
BOOST_TEST_MESSAGE(sprint("%s() called from %s() %s:%i", __FUNCTION__, sl.function_name(), sl.file_name(), sl.line()));
auto aggregated_population_is_zero = [&] () mutable {
return aggregate_querier_cache_stat(db, &query::querier_cache::stats::population) == 0;
};
BOOST_REQUIRE(eventually_true(aggregated_population_is_zero));
}
// Best run with SMP>=2
SEASTAR_THREAD_TEST_CASE(test_abandoned_read) {
do_with_cql_env([] (cql_test_env& env) -> future<> {
using namespace std::chrono_literals;
env.db().invoke_on_all([] (database& db) {
db.set_querier_cache_entry_ttl(2s);
}).get();
auto [s, _] = create_test_cf(env);
(void)_;
auto cmd = query::read_command(s->id(), s->version(), s->full_slice(), 7, gc_clock::now(), stdx::nullopt, query::max_partitions,
utils::make_random_uuid(), true);
query_mutations_on_all_shards(env.db(), s, cmd, {query::full_partition_range}, nullptr, std::numeric_limits<uint64_t>::max()).get();
check_cache_population(env.db(), 1);
sleep(2s).get();
require_eventually_empty_caches(env.db());
return make_ready_future<>();
}).get();
}
static std::vector<mutation> read_all_partitions_one_by_one(distributed<database>& db, schema_ptr s, std::vector<dht::decorated_key> pkeys) {
const auto& partitioner = dht::global_partitioner();
std::vector<mutation> results;
results.reserve(pkeys.size());
for (const auto& pkey : pkeys) {
const auto res = db.invoke_on(partitioner.shard_of(pkey.token()), [gs = global_schema_ptr(s), &pkey] (database& db) {
return async([s = gs.get(), &pkey, &db] () mutable {
auto accounter = db.get_result_memory_limiter().new_mutation_read(std::numeric_limits<size_t>::max()).get0();
const auto cmd = query::read_command(s->id(), s->version(), s->full_slice(), query::max_rows);
const auto range = dht::partition_range::make_singular(pkey);
return make_foreign(std::make_unique<reconcilable_result>(
db.query_mutations(std::move(s), cmd, range, std::move(accounter), nullptr).get0()));
});
}).get0();
BOOST_REQUIRE_EQUAL(res->partitions().size(), 1);
results.emplace_back(res->partitions().front().mut().unfreeze(s));
}
return results;
}
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) {
const auto max_size = std::numeric_limits<uint64_t>::max();
const auto query_uuid = utils::make_random_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);
// First page is special, needs to have `is_first_page` set.
{
auto res = query_mutations_on_all_shards(db, s, cmd, {query::full_partition_range}, nullptr, max_size).get0();
for (auto& part : res->partitions()) {
results.emplace_back(part.mut().unfreeze(s));
}
cmd.is_first_page = false;
}
size_t nrows = page_size;
unsigned npages = 0;
// Rest of the pages.
// 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);
const auto& last_pkey = results.back().decorated_key();
const auto& last_ckey = results.back().partition().clustered_rows().rbegin()->key();
auto pkrange = dht::partition_range::make_starting_with(dht::partition_range::bound(last_pkey, true));
auto ckrange = query::clustering_range::make_starting_with(query::clustering_range::bound(last_ckey, false));
if (const auto& sr = cmd.slice.get_specific_ranges(); sr) {
cmd.slice.clear_range(*s, sr->pk());
}
cmd.slice.set_range(*s, last_pkey.key(), {ckrange});
auto res = query_mutations_on_all_shards(db, s, cmd, {pkrange}, nullptr, max_size).get0();
if (res->partitions().empty()) {
nrows = 0;
} else {
auto it = res->partitions().begin();
auto end = res->partitions().end();
auto first_mut = it->mut().unfreeze(s);
nrows = first_mut.live_row_count();
// The first partition of the new page may overlap with the last
// partition of the last page.
if (results.back().decorated_key().equal(*s, first_mut.decorated_key())) {
results.back().apply(std::move(first_mut));
} else {
results.emplace_back(std::move(first_mut));
}
++it;
for (;it != end; ++it) {
auto mut = it->mut().unfreeze(s);
nrows += mut.live_row_count();
results.emplace_back(std::move(mut));
}
}
++npages;
}
return std::pair(results, npages);
}
void check_results_are_equal(std::vector<mutation>& results1, std::vector<mutation>& results2) {
BOOST_REQUIRE_EQUAL(results1.size(), results2.size());
auto mut_less = [] (const mutation& a, const mutation& b) {
return a.decorated_key().less_compare(*a.schema(), b.decorated_key());
};
boost::sort(results1, mut_less);
boost::sort(results2, mut_less);
for (unsigned i = 0; i < results1.size(); ++i) {
BOOST_TEST_MESSAGE(sprint("Comparing mutation #%i", i));
assert_that(results2[i]).is_equal_to(results1[i]);
}
}
// Best run with SMP>=2
SEASTAR_THREAD_TEST_CASE(test_read_all) {
do_with_cql_env([] (cql_test_env& env) -> future<> {
using namespace std::chrono_literals;
env.db().invoke_on_all([] (database& db) {
db.set_querier_cache_entry_ttl(2s);
}).get();
auto [s, pkeys] = create_test_cf(env);
// 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) {
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);
}).first;
check_results_are_equal(results1, results2);
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);
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::time_based_evictions), 0);
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::resource_based_evictions), 0);
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::memory_based_evictions), 0);
require_eventually_empty_caches(env.db());
return make_ready_future<>();
}).get();
}
// Best run with SMP>=2
SEASTAR_THREAD_TEST_CASE(test_evict_a_shard_reader_on_each_page) {
do_with_cql_env([] (cql_test_env& env) -> future<> {
using namespace std::chrono_literals;
env.db().invoke_on_all([] (database& db) {
db.set_querier_cache_entry_ttl(2s);
}).get();
auto [s, pkeys] = create_test_cf(env);
// 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, npages] = read_all_partitions_with_paged_scan(env.db(), s, 4, [&] (size_t page) {
check_cache_population(env.db(), 1);
env.db().invoke_on(page % smp::count, [&] (database& db) {
db.get_querier_cache().evict_one();
}).get();
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), page);
});
check_results_are_equal(results1, results2);
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::time_based_evictions), 0);
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::resource_based_evictions), npages);
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::memory_based_evictions), 0);
require_eventually_empty_caches(env.db());
return make_ready_future<>();
}).get();
}
SEASTAR_THREAD_TEST_CASE(test_range_tombstones_at_page_boundaries) {
do_with_cql_env([] (cql_test_env& env) -> future<> {
using namespace std::chrono_literals;
env.db().invoke_on_all([] (database& db) {
db.set_querier_cache_entry_ttl(2s);
}).get();
auto [s, pkeys] = create_test_cf(env, {6, 2});
// 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, 7, [&] (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);
}).first;
check_results_are_equal(results1, results2);
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::time_based_evictions), 0);
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::resource_based_evictions), 0);
BOOST_REQUIRE_EQUAL(aggregate_querier_cache_stat(env.db(), &query::querier_cache::stats::memory_based_evictions), 0);
require_eventually_empty_caches(env.db());
return make_ready_future<>();
}).get();
}

View File

@@ -21,6 +21,8 @@
#pragma once
#include <boost/test/unit_test.hpp>
#include "mutation.hh"
class mutation_partition_assertion {

View File

@@ -46,6 +46,7 @@
#include "database.hh"
#include "partition_slice_builder.hh"
#include "schema_registry.hh"
#include "service/priority_manager.hh"
static schema_ptr make_schema() {
return schema_builder("ks", "cf")
@@ -1532,9 +1533,13 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_as_mutation_source) {
const io_priority_class& pc,
tracing::trace_state_ptr trace_state,
streamed_mutation::forwarding fwd_sm,
mutation_reader::forwarding fwd_mr) {
auto factory = [remote_memtables, s, &slice, &pc, trace_state] (unsigned shard,
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,
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,
@@ -1549,7 +1554,7 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_as_mutation_source) {
});
};
return make_multishard_combining_reader(s, range, *partitioner, factory, fwd_sm, fwd_mr);
return make_multishard_combining_reader(s, range, slice, pc, *partitioner, factory, trace_state, fwd_sm, fwd_mr);
});
};
@@ -1568,17 +1573,22 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_reading_empty_table) {
do_with_cql_env([] (cql_test_env& env) -> future<> {
std::vector<bool> shards_touched(smp::count, false);
simple_schema s;
auto factory = [&shards_touched, &s] (unsigned shard,
auto factory = [&shards_touched] (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,
streamed_mutation::forwarding fwd_sm,
mutation_reader::forwarding fwd_mr) {
shards_touched[shard] = true;
return smp::submit_to(shard, [gs = global_schema_ptr(s.schema())] () mutable {
return smp::submit_to(shard, [gs = global_schema_ptr(s)] () mutable {
return make_foreign(std::make_unique<flat_mutation_reader>(make_empty_flat_reader(gs.get())));
});
};
assert_that(make_multishard_combining_reader(s.schema(), query::full_partition_range, dht::global_partitioner(), std::move(factory)))
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)))
.produces_end_of_stream();
for (unsigned i = 0; i < smp::count; ++i) {
@@ -1683,8 +1693,12 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
auto s = simple_schema();
auto factory = [shard_of_interest, &s, remote_control = remote_control.get()] (unsigned shard,
auto factory = [&s, shard_of_interest, remote_control = remote_control.get()] (unsigned shard,
schema_ptr,
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, [shard_of_interest, gs = global_simple_schema(s), remote_control] () mutable {
@@ -1704,7 +1718,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, partitioner, std::move(factory));
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));
reader.fill_buffer().get();
@@ -1941,7 +1956,11 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
auto partitioner = dummy_partitioner(dht::global_partitioner(), std::move(pkeys_by_tokens));
auto factory = [&s, &remote_controls, &shard_pkeys] (unsigned shard,
const dht::partition_range&,
schema_ptr,
const dht::partition_range& range,
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(),
@@ -1953,7 +1972,8 @@ SEASTAR_THREAD_TEST_CASE(test_multishard_combining_reader_destroyed_with_pending
};
{
auto reader = make_multishard_combining_reader(s.schema(), query::full_partition_range, partitioner, std::move(factory));
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));
reader.fill_buffer().get();
BOOST_REQUIRE(reader.is_buffer_full());
}

View File

@@ -125,6 +125,7 @@ static void test_streamed_mutation_forwarding_is_consistent_with_slicing(populat
fwd_reader.consume(consumer(m.schema(), builder)).get0();
}
mutation_opt fwd_m = builder->consume_end_of_stream();
BOOST_REQUIRE(bool(fwd_m));
mutation_opt sliced_m = read_mutation_from_flat_mutation_reader(sliced_reader).get0();
BOOST_REQUIRE(bool(sliced_m));

View File

@@ -71,15 +71,12 @@ public:
static const size_t max_reader_buffer_size = 8 * 1024;
private:
// Actual counters updated by the cache.
unsigned _factory_invoked{};
// Expected value of the above counters, updated by this.
unsigned _expected_factory_invoked{};
querier_cache::stats _expected_stats;
query::querier_cache::stats _expected_stats;
simple_schema _s;
querier_cache _cache;
query::querier_cache _cache;
const std::vector<mutation> _mutations;
const mutation_source _mutation_source;
@@ -110,14 +107,14 @@ private:
return mutations;
}
querier make_querier(const dht::partition_range& range) {
return querier(_mutation_source,
template <typename Querier>
Querier make_querier(const dht::partition_range& range) {
return Querier(_mutation_source,
_s.schema(),
range,
_s.schema()->full_slice(),
service::get_local_sstable_query_read_priority(),
nullptr,
emit_only_live_rows::no);
nullptr);
}
static utils::UUID make_cache_key(unsigned key) {
@@ -202,11 +199,12 @@ public:
return _s.schema()->full_slice();
}
template <typename Querier>
entry_info produce_first_page_and_save_querier(unsigned key, const dht::partition_range& range,
const query::partition_slice& slice, uint32_t row_limit = 5) {
const query::partition_slice& slice, uint32_t row_limit) {
const auto cache_key = make_cache_key(key);
auto querier = make_querier(range);
auto querier = make_querier<Querier>(range);
auto [dk, ck] = querier.consume_page(dummy_result_builder{}, row_limit, std::numeric_limits<uint32_t>::max(),
gc_clock::now(), db::no_timeout).get0();
const auto memory_usage = querier.memory_usage();
@@ -248,45 +246,71 @@ public:
return {key, std::move(range), std::move(slice), row_limit, memory_usage, std::move(expected_range), std::move(expected_slice)};
}
entry_info produce_first_page_and_save_querier(unsigned key, const dht::partition_range& range, uint32_t row_limit = 5) {
return produce_first_page_and_save_querier(key, range, make_default_slice(), row_limit);
entry_info produce_first_page_and_save_data_querier(unsigned key, const dht::partition_range& range,
const query::partition_slice& slice, uint32_t row_limit = 5) {
return produce_first_page_and_save_querier<query::data_querier>(key, range, slice, row_limit);
}
entry_info produce_first_page_and_save_data_querier(unsigned key, const dht::partition_range& range, uint32_t row_limit = 5) {
return produce_first_page_and_save_data_querier(key, range, make_default_slice(), row_limit);
}
// Singular overload
entry_info produce_first_page_and_save_querier(unsigned key, std::size_t i, uint32_t row_limit = 5) {
return produce_first_page_and_save_querier(key, make_singular_partition_range(i), _s.schema()->full_slice(), row_limit);
entry_info produce_first_page_and_save_data_querier(unsigned key, std::size_t i, uint32_t row_limit = 5) {
return produce_first_page_and_save_data_querier(key, make_singular_partition_range(i), _s.schema()->full_slice(), row_limit);
}
// Use the whole range
entry_info produce_first_page_and_save_querier(unsigned key) {
return produce_first_page_and_save_querier(key, make_default_partition_range(), _s.schema()->full_slice());
entry_info produce_first_page_and_save_data_querier(unsigned key) {
return produce_first_page_and_save_data_querier(key, make_default_partition_range(), _s.schema()->full_slice());
}
// For tests testing just one insert-lookup.
entry_info produce_first_page_and_save_querier() {
return produce_first_page_and_save_querier(1);
entry_info produce_first_page_and_save_data_querier() {
return produce_first_page_and_save_data_querier(1);
}
test_querier_cache& assert_cache_lookup(unsigned lookup_key,
entry_info produce_first_page_and_save_mutation_querier(unsigned key, const dht::partition_range& range,
const query::partition_slice& slice, uint32_t row_limit = 5) {
return produce_first_page_and_save_querier<query::mutation_querier>(key, range, slice, row_limit);
}
entry_info produce_first_page_and_save_mutation_querier(unsigned key, const dht::partition_range& range, uint32_t row_limit = 5) {
return produce_first_page_and_save_mutation_querier(key, range, make_default_slice(), row_limit);
}
// Singular overload
entry_info produce_first_page_and_save_mutation_querier(unsigned key, std::size_t i, uint32_t row_limit = 5) {
return produce_first_page_and_save_mutation_querier(key, make_singular_partition_range(i), _s.schema()->full_slice(), row_limit);
}
// Use the whole range
entry_info produce_first_page_and_save_mutation_querier(unsigned key) {
return produce_first_page_and_save_mutation_querier(key, make_default_partition_range(), _s.schema()->full_slice());
}
// For tests testing just one insert-lookup.
entry_info produce_first_page_and_save_mutation_querier() {
return produce_first_page_and_save_mutation_querier(1);
}
test_querier_cache& assert_cache_lookup_data_querier(unsigned lookup_key,
const schema& lookup_schema,
const dht::partition_range& lookup_range,
const query::partition_slice& lookup_slice) {
_cache.lookup(make_cache_key(lookup_key), emit_only_live_rows::no, lookup_schema, lookup_range, lookup_slice, nullptr, [this, &lookup_range] {
++_factory_invoked;
return make_querier(lookup_range);
});
_cache.lookup_data_querier(make_cache_key(lookup_key), lookup_schema, lookup_range, lookup_slice, nullptr);
BOOST_REQUIRE_EQUAL(_cache.get_stats().lookups, ++_expected_stats.lookups);
return *this;
}
test_querier_cache& no_factory_invoked() {
BOOST_REQUIRE_EQUAL(_factory_invoked, _expected_factory_invoked);
return *this;
}
test_querier_cache& assert_cache_lookup_mutation_querier(unsigned lookup_key,
const schema& lookup_schema,
const dht::partition_range& lookup_range,
const query::partition_slice& lookup_slice) {
test_querier_cache& factory_invoked() {
BOOST_REQUIRE_EQUAL(_factory_invoked, ++_expected_factory_invoked);
_cache.lookup_mutation_querier(make_cache_key(lookup_key), lookup_schema, lookup_range, lookup_slice, nullptr);
BOOST_REQUIRE_EQUAL(_cache.get_stats().lookups, ++_expected_stats.lookups);
return *this;
}
@@ -342,14 +366,60 @@ public:
SEASTAR_THREAD_TEST_CASE(lookup_with_wrong_key_misses) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_querier();
t.assert_cache_lookup(90, *t.get_schema(), entry.expected_range, entry.expected_slice)
.factory_invoked()
const auto entry = t.produce_first_page_and_save_data_querier();
t.assert_cache_lookup_data_querier(90, *t.get_schema(), entry.expected_range, entry.expected_slice)
.misses()
.no_drops()
.no_evictions();
}
SEASTAR_THREAD_TEST_CASE(lookup_data_querier_as_mutation_querier_misses) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_data_querier();
t.assert_cache_lookup_mutation_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.misses()
.no_drops()
.no_evictions();
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
}
SEASTAR_THREAD_TEST_CASE(lookup_mutation_querier_as_data_querier_misses) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_mutation_querier();
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.misses()
.no_drops()
.no_evictions();
t.assert_cache_lookup_mutation_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
}
SEASTAR_THREAD_TEST_CASE(data_and_mutation_querier_can_coexist) {
test_querier_cache t;
const auto data_entry = t.produce_first_page_and_save_data_querier(1);
const auto mutation_entry = t.produce_first_page_and_save_mutation_querier(1);
t.assert_cache_lookup_data_querier(data_entry.key, *t.get_schema(), data_entry.expected_range, data_entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
t.assert_cache_lookup_mutation_querier(mutation_entry.key, *t.get_schema(), mutation_entry.expected_range, mutation_entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
}
/*
* Range matching tests
*/
@@ -357,9 +427,8 @@ SEASTAR_THREAD_TEST_CASE(lookup_with_wrong_key_misses) {
SEASTAR_THREAD_TEST_CASE(singular_range_lookup_with_stop_at_clustering_row) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_querier(1, t.make_singular_partition_range(1), 2);
t.assert_cache_lookup(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_factory_invoked()
const auto entry = t.produce_first_page_and_save_data_querier(1, t.make_singular_partition_range(1), 2);
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
@@ -368,9 +437,8 @@ SEASTAR_THREAD_TEST_CASE(singular_range_lookup_with_stop_at_clustering_row) {
SEASTAR_THREAD_TEST_CASE(singular_range_lookup_with_stop_at_static_row) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_querier(1, t.make_singular_partition_range(1), 1);
t.assert_cache_lookup(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_factory_invoked()
const auto entry = t.produce_first_page_and_save_data_querier(1, t.make_singular_partition_range(1), 1);
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
@@ -379,9 +447,8 @@ SEASTAR_THREAD_TEST_CASE(singular_range_lookup_with_stop_at_static_row) {
SEASTAR_THREAD_TEST_CASE(lookup_with_stop_at_clustering_row) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_querier(1, t.make_partition_range({1, true}, {3, false}), 3);
t.assert_cache_lookup(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_factory_invoked()
const auto entry = t.produce_first_page_and_save_data_querier(1, t.make_partition_range({1, true}, {3, false}), 3);
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
@@ -390,9 +457,8 @@ SEASTAR_THREAD_TEST_CASE(lookup_with_stop_at_clustering_row) {
SEASTAR_THREAD_TEST_CASE(lookup_with_stop_at_static_row) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_querier(1, t.make_partition_range({1, true}, {3, false}), 1);
t.assert_cache_lookup(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_factory_invoked()
const auto entry = t.produce_first_page_and_save_data_querier(1, t.make_partition_range({1, true}, {3, false}), 1);
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.no_misses()
.no_drops()
.no_evictions();
@@ -405,9 +471,8 @@ SEASTAR_THREAD_TEST_CASE(lookup_with_stop_at_static_row) {
SEASTAR_THREAD_TEST_CASE(lookup_with_original_range_drops) {
test_querier_cache t;
const auto entry = t.produce_first_page_and_save_querier(1);
t.assert_cache_lookup(entry.key, *t.get_schema(), entry.original_range, entry.expected_slice)
.factory_invoked()
const auto entry = t.produce_first_page_and_save_data_querier(1);
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.original_range, entry.expected_slice)
.no_misses()
.drops()
.no_evictions();
@@ -418,37 +483,32 @@ SEASTAR_THREAD_TEST_CASE(lookup_with_wrong_slice_drops) {
test_querier_cache t;
// Swap slices for different clustering keys.
const auto entry1 = t.produce_first_page_and_save_querier(1, t.make_partition_range({1, false}, {3, true}), 3);
const auto entry2 = t.produce_first_page_and_save_querier(2, t.make_partition_range({1, false}, {3, true}), 4);
t.assert_cache_lookup(entry1.key, *t.get_schema(), entry1.expected_range, entry2.expected_slice)
.factory_invoked()
const auto entry1 = t.produce_first_page_and_save_data_querier(1, t.make_partition_range({1, false}, {3, true}), 3);
const auto entry2 = t.produce_first_page_and_save_data_querier(2, t.make_partition_range({1, false}, {3, true}), 4);
t.assert_cache_lookup_data_querier(entry1.key, *t.get_schema(), entry1.expected_range, entry2.expected_slice)
.no_misses()
.drops()
.no_evictions();
t.assert_cache_lookup(entry2.key, *t.get_schema(), entry2.expected_range, entry1.expected_slice)
.factory_invoked()
t.assert_cache_lookup_data_querier(entry2.key, *t.get_schema(), entry2.expected_range, entry1.expected_slice)
.no_misses()
.drops()
.no_evictions();
// Wrong slice.
const auto entry3 = t.produce_first_page_and_save_querier(3);
t.assert_cache_lookup(entry3.key, *t.get_schema(), entry3.expected_range, t.get_schema()->full_slice())
.factory_invoked()
const auto entry3 = t.produce_first_page_and_save_data_querier(3);
t.assert_cache_lookup_data_querier(entry3.key, *t.get_schema(), entry3.expected_range, t.get_schema()->full_slice())
.no_misses()
.drops()
.no_evictions();
// Swap slices for stopped at clustering/static row.
const auto entry4 = t.produce_first_page_and_save_querier(4, t.make_partition_range({1, false}, {3, true}), 1);
const auto entry5 = t.produce_first_page_and_save_querier(5, t.make_partition_range({1, false}, {3, true}), 2);
t.assert_cache_lookup(entry4.key, *t.get_schema(), entry4.expected_range, entry5.expected_slice)
.factory_invoked()
const auto entry4 = t.produce_first_page_and_save_data_querier(4, t.make_partition_range({1, false}, {3, true}), 1);
const auto entry5 = t.produce_first_page_and_save_data_querier(5, t.make_partition_range({1, false}, {3, true}), 2);
t.assert_cache_lookup_data_querier(entry4.key, *t.get_schema(), entry4.expected_range, entry5.expected_slice)
.no_misses()
.drops()
.no_evictions();
t.assert_cache_lookup(entry5.key, *t.get_schema(), entry5.expected_range, entry4.expected_slice)
.factory_invoked()
t.assert_cache_lookup_data_querier(entry5.key, *t.get_schema(), entry5.expected_range, entry4.expected_slice)
.no_misses()
.drops()
.no_evictions();
@@ -459,9 +519,8 @@ SEASTAR_THREAD_TEST_CASE(lookup_with_different_schema_version_drops) {
auto new_schema = schema_builder(t.get_schema()).with_column("v1", utf8_type).build();
const auto entry = t.produce_first_page_and_save_querier();
t.assert_cache_lookup(entry.key, *new_schema, entry.expected_range, entry.expected_slice)
.factory_invoked()
const auto entry = t.produce_first_page_and_save_data_querier();
t.assert_cache_lookup_data_querier(entry.key, *new_schema, entry.expected_range, entry.expected_slice)
.no_misses()
.drops()
.no_evictions();
@@ -474,24 +533,22 @@ SEASTAR_THREAD_TEST_CASE(lookup_with_different_schema_version_drops) {
SEASTAR_THREAD_TEST_CASE(test_time_based_cache_eviction) {
test_querier_cache t(1s);
const auto entry1 = t.produce_first_page_and_save_querier(1);
const auto entry1 = t.produce_first_page_and_save_data_querier(1);
seastar::sleep(500ms).get();
const auto entry2 = t.produce_first_page_and_save_querier(2);
const auto entry2 = t.produce_first_page_and_save_data_querier(2);
seastar::sleep(700ms).get();
t.assert_cache_lookup(entry1.key, *t.get_schema(), entry1.expected_range, entry1.expected_slice)
.factory_invoked()
t.assert_cache_lookup_data_querier(entry1.key, *t.get_schema(), entry1.expected_range, entry1.expected_slice)
.misses()
.no_drops()
.time_based_evictions();
seastar::sleep(700ms).get();
t.assert_cache_lookup(entry2.key, *t.get_schema(), entry2.expected_range, entry2.expected_slice)
.factory_invoked()
t.assert_cache_lookup_data_querier(entry2.key, *t.get_schema(), entry2.expected_range, entry2.expected_slice)
.misses()
.no_drops()
.time_based_evictions();
@@ -520,20 +577,19 @@ SEASTAR_THREAD_TEST_CASE(test_memory_based_cache_eviction) {
}, 24h, cache_size);
size_t i = 0;
const auto entry = t.produce_first_page_and_save_querier(i++);
const auto entry = t.produce_first_page_and_save_data_querier(i++);
const size_t queriers_needed_to_fill_cache = floor(cache_size / entry.memory_usage);
// Fill the cache but don't overflow.
for (; i < queriers_needed_to_fill_cache; ++i) {
t.produce_first_page_and_save_querier(i);
t.produce_first_page_and_save_data_querier(i);
}
// Should overflow the limit and trigger the eviction of the oldest entry.
t.produce_first_page_and_save_querier(queriers_needed_to_fill_cache);
t.produce_first_page_and_save_data_querier(queriers_needed_to_fill_cache);
t.assert_cache_lookup(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.factory_invoked()
t.assert_cache_lookup_data_querier(entry.key, *t.get_schema(), entry.expected_range, entry.expected_slice)
.misses()
.no_drops()
.memory_based_evictions();