The pager::state() function returns a valid paging object even
if the pager itself is exhausted. It may also not contain the partition
key, so using it unconditionally was a bug - now, in case there is no
partition key present, paging state will contain an empty partition key.
Fixes#3829
Message-Id: <28401eb21ab8f12645c0a33d9e92ada9de83e96b.1539074813.git.sarna@scylladb.com>
If service::pager is exhausted, state() function used to return
a nullptr instead of a pointer to a valid paging state and the
documented return type in this case was 'unspecified'.
Sometimes a paging state may be needed anyway, even if the pager
is already exhausted - thus, state() return value becomes defined
after this commit. Exhausted pagers will return a valid object
to a state with _remaining field set to 0.
A standard way for passing a timeout parameter is specifying
a time_point, while pagers used to take a duration in order
to compute time points on the fly. This patch adds a timeout
parameter, which is a time_point, to fetch_page().
There is a bad interaction between may_need_paging() and query result
size limiter. The former is trying to avoid the complexity of paged
queries when the number of returned rows is going to be smaller than the
page size. The latter uses the fact that paged queries need not return
all requested rows to limit the size of a query results. Since
may_need_paging() may turn a paged query into non-paged one as a side
effect it disables the oversized result protection.
This patch limits the cases when may_need_paging() disables paging to
the situations when we know for sure that query result size limiter
won't be needed, i.e.: the result is not going to contain more than one
row. If the client knows for sure that the paging is not needed and
the performance impact is worthwhile it can disable paging on its side.
Otherwise, let's default to the safer behaviour.
Fixes#3620.
Message-Id: <20180925134431.24329-1-pdziepak@scylladb.com>
Query options may contain bound values needed for checking filtering
restrictions. Previously, empty query_options{} were used, which
caused prepared statements to fail.
Fixes#3677
query::result_visitor provides get_last_partition_and_clustering_key()
which allows getting those without iterating through the whole result.
Moreover, row count may be precomputed in the result, if it isn't there
is query::result_view::count_partitions_and_rows() for getting it.
"
The main idea of this series is to provide a filtering_visitor
as a specialised result_set_builder::visitor implementation
that keeps restriction info and applies it on query results.
Also, since allow_filtering checking is not correct now (e.g. #2025)
on select_statement level, this series tries to fix any issues
related to it.
Still in TODO:
* handling CONTAINS relation in single column restriction filtering
* handling multi-column restrictions - especially EQ, which can be
split into multiple single-column restrictions
* more tests - it's never enough; especially esoteric cases
like filtering queries which also use secondary indexes,
paging tests, etc.
Tests: unit (release)
"
* 'allow_filtering_6' of https://github.com/psarna/scylla:
tests: add allow_filtering tests to cql_query_test
cql3: enable ALLOW FILTERING
service: add filtering_pager
cql3: optimize filtering partition keys and static rows
cql3: add filtering visitor
cql3: move result_set_builder functions to header
cql3: amend need_filtering()
cql3: add single column primary key restrictions getters
cql3: expose single column primary key restrictions
cql3: add needs_filtering to primary key restrictions
cql3: add simpler single_column_restriction::is_satisfied_by
Use query::is_single_partition() to check whether the queried ranges are
singular or not. The current method of using
`dht::partition_range::is_singular()` is incorrect, as it is possible to
build a singular range that doesn't represent a single partition.
`query::is_single_partition()` correctly checks for this so use it
instead.
Found during code-review.
Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Message-Id: <f671f107e8069910a2f84b14c8d22638333d571c.1530675889.git.bdenes@scylladb.com>
do_fetch_page() checks in the beginning whether there is a saved query
state already, meaning this is not the first page. If there is not it
checks whether the query is for a singulular partitions or a range scan
to decide whether to enable the stateful queries or not. This check
assumed that there is at least one range in _ranges which will not hold
under some circumstances. Add a check for _ranges being empty.
Fixes: #3564
Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Message-Id: <cbe64473f8013967a93ef7b2104c7ca0507afac9.1530610709.git.bdenes@scylladb.com>
The presence of a plain reference prohibits the bound_view class from
being copyable. The trick employed to work around that was to use
'placement new' for copy-assigning bound_view objects, but this approach
is ill-formed and causes undefined behaviour for classes that have const
and/or reference members.
The solution is to use a std::reference_wrapper instead.
Signed-off-by: Vladimir Krivopalov <vladimir@scylladb.com>
Message-Id: <a0c951649c7aef2f66612fc006c44f8a33713931.1530113273.git.vladimir@scylladb.com>
So far query_result_visitor was tied to result_set_builder. The goal is
to enable result_generator to work with paged queries as well so we need
to decouple them.
Shared pointers make code harder to reason about, it is not easy to get
rid of them in this piece of the code, but we can restore at least a bit
of sanity by adding consts.
There is just a single implementation of query_pager and there is no
reason to make anything virtual. Devirtualising this code will allow
higher layers to pass visitors via templates.
"
This patchset implements separate timeouts for range queries, and lays
the foundations for separate timeouts for other query types.
While the feature in itself is worthy, the real motivation is to have
the timeouts decided by the caller, instead of storage_proxy. This in
turn is required to disentangle each layer behaving differently
depending on whether the query is internal or not; instead, the goal
is to have each caller declare its needs in terms of consistency level
and timeouts, and have the lower layers implement its requirements
instead of making their own decisions.
Fixes#3013.
Tests: unit (release)
"
* tag '3013/v1.1' of https://github.com/avikivity/scylla:
storage_proxy: remove default_query_timeout()
storage_proxy: don't use default timeouts
query_options: augment with timeout_config
thrift: configure thrift transport and handler with a timeout_config
transport: configure native transport with a timeout_config
cql3: define and populate timeout_config_selector
timeout_config: introduce timeout configuration
Make the read-repair decision on the first page of a paged-query and use
it for all the remaining pages. This helps querier-cache hit-rates as
reads to nodes will be sent consistently throught the query.
As yet more parameters and return-values are about to be added to all
storage_proxy::query_* methods we need a way that scales better than
changing the signatures every time. To this end we aggregate all
non-mandatory query parameters into `coordinator_query_options` and all
return values into `coordinator_query_result`.
This way new fields can be simply added to the respective structs while
the signatures of the methods themselves and their client code can
remain unchanged.
This new field will store the repair-decision made on the first page of
the query. This decision will be sticky to all pages of the query.
In mixed clusters the decision might not happen on the first page and it
might even change during the query as old coordinators will not store
nor respect the decision.
Pass the last_replicas from the page_state as the preferred_replicas
for query() and save the returned last_replicas as the last_replicas
field of the next page_state. The circle is now complete. The first page
of any query will pass an empty list as the preferred replicas (having
no previous paging_state) so the replicas will be selected according to
the load-balancing strategy. Any subsequent page will use the last
replicas from the last page as the preferred ones for the current one.
Thus if all goes well all pages of a query will hit the same replicas.
preferred_replicas are added to the parameters and last_replicas are
added to the return type. The preferred replicas will be used as a hint
for the selection of the replicas to send the read requests to. The last
replicas (returned) are the replicas actually selected for the read.
This will allow queries to consistently hit the same replicas for each
page thus reusing readers created on these replicas.
For convenience a query() overload is provided that doesn't take or
return the preferred and last replicas.
This patch only adds the parameters and propagates them down to
query_singular() and query_partition_key_range(). The code to actually
use these preferred-replicas will be added in later patches.
This reason for separating this is to reduce noise and improve
reviewability for those functional changes later.
Helps paged queries consistently hit the same replicas for each
subsequent page. Replicas that already served a page will keep the
readers used for filling it around in a cache. Subsequent page request
hitting the same replicas can reuse these readers to fill the pages
avoiding the work of creating these readers from scratch on every page.
In a mixed cluster older coordinators will ignore this value.
The value of last_replicas may change between pages as nodes may become
available/unavailable or the coordinator may decide to send the read
requests to different replicas at its discretion.
Replicas are identified by an opaque uuid which should only make sense
to the storage-proxy.
This patch adds the parameter to read_command which is needed for
caching of readers during multiple pages of a paged queries, which
we will introduce in the next patches.
The query_uuid is a UUID of a previously saved reader, which
the replica is now asked to recall and resume (if this saved reader is
no longer in the cache, it is fine, a new reader will be started).
Additionally a helper flag is_first_page is added so that the replica
can avoid doing any cache lookups (and incrementing miss counters) for
the first page.
Signed-off-by: Nadav Har'El <nyh@scylladb.com>
This patch adds to the "paging_state", the opaque cookie that clients are
supposed to provide when asking for the next page on a paged query, a
unique id field. This new field will be used to tell that a new request
for a page really continues the previous page, and doesn't just by chance
start at the same position the previous page stopped.
We need to support setups with mixed versions - a client may get a paging
state from a coordinator running a new version of Scylla and send it to
a different coordinator running an old version - or vice versa. So the new
uuid field is set up to have a default uuid of UUID() (a recognizable
invalid uuid 0), so new versions receiving no uuid from an old version will
set this invalid uuid, and old versions receiving a uuid from a new version
will simply ignore it.
Signed-off-by: Nadav Har'El <nyh@scylladb.com>
- introcduced "seastarx.hh" header, which does a "using namespace seastar";
- 'net' namespace conflicts with seastar::net, renamed to 'netw'.
- 'transport' namespace conflicts with seastar::transport, renamed to
cql_transport.
- "logger" global variables now conflict with logger global type, renamed
to xlogger.
- other minor changes
Some code paths were obtaining db_clock timestamp to only convert it
to gc_clock later. Avoid this. In the future we could make gc_clock
cheaper cause it has low precision.
Message-Id: <1482401190-2035-1-git-send-email-tgrabiec@scylladb.com>
This patch fixes a regression introduced in 0518895, where we counted
one extra row per partition when it contained live, non static rows.
We also simplify the visitor logic further, since now we don't need to
count rows one by one. Also remove a bunch of unused fields.
Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <1482234083-2447-1-git-send-email-duarte@scylladb.com>
Since storage_proxy::query() now respects the read_command limits, we
can remove the trimming logic from query_pagers.
Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Currently, the paging implementation assumes that the server retunrs
either as many rows as it was asked for all reached the end. Soon,
that's not going to be true so instead of making any assumptions about
the number of the rows returned use the new "short read" flag to
determine whether there is going to be more data.
Signed-off-by: Paweł Dziepak <pdziepak@scylladb.com>
Query pager needs to handle results that contain partitions with
possibly multiple clustering rows quite differently than results with
just one row per partition (for example a page may end in a middle of
partition). However, the logic dealing with partitions with clustering
rows doesn't work correctly for SELECT DISTINCT queries, which are
much more similar to the ones for schemas without clustering key.
The solution is to set _has_clustering_keys to false in case of SELECT
DISTINCT queries regardless of the schema which will make pager
correctly expect each partition to return at most one rows.
Fixes#1822.
Signed-off-by: Paweł Dziepak <pdziepak@scylladb.com>
Message-Id: <1478612486-13421-1-git-send-email-pdziepak@scylladb.com>
Currently, the code responsible for calculating ranges for the next
request could produce a wrap-around partition range. For example, if the
original range was (unimportant, A] and the last partition key A then
the output range would be (A, A].
This patch adds checks to make sure that in such cases the range is
removed.
Signed-off-by: Paweł Dziepak <pdziepak@scylladb.com>
Message-Id: <1475497244-2790-1-git-send-email-pdziepak@scylladb.com>
Paging code assumes that clustering row range [a, a] contains only one
row which may not be true. Another problem is that it tries to use
range<> interface for dealing with clustering key ranges which doesn't
work because of the lack of correct comparator.
Refs #1446.
Fixes#1684.
Signed-off-by: Paweł Dziepak <pdziepak@scylladb.com>
Message-Id: <1475236805-16223-1-git-send-email-pdziepak@scylladb.com>
Having a trace_state_ptr in the storage_proxy level is needed to trace code bits in this level.
Signed-off-by: Vlad Zolotarov <vladz@cloudius-systems.com>