The generator will be responsible for spreading view updates with the
help of mutate_MV helper. The latter needs storage proxy to operate, so
the generator gets this dependency in advance.
There's no need to change start/stop order at the moment, generator
already starts after and stops before proxy. Also, services that have
generator as dependency are not required by proxy (even indirectly) so
no circular dependency is produced at this point.
Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
In an incoming change, the wasm instance cache will be modified to be owned
by the query_processor - it will hold an optional instead of a raw
pointer to the cache, so we should stop returning the raw pointer
from the getter as well.
Consequently, the cache is also stored as a reference in wasm::cache,
as it gets the reference from the query_processor.
For consistency with the wasm engine and the wasm alien thread runner,
the name of the getter is also modified to follow the same pattern.
The wasm engine is used for compiling and executing Wasm UDFs, so
the query_processor is a more appropriate location for it than
replica::database, especially because the wasm instance cache
and the wasm alien thread runner are already there.
This patch also reduces the number of wasm engines to 1, shared by
all shards, as recommended by the wasmtime developers.
Fixes#13332
The tests user the discriminator "system" as prefix to assume keyspaces are marked
"internal" inside scylla. This is not true in enterprise universe (replicated key
provider). It maybe/probably should, but that train is sailing right now.
Fix by removing one assert (not correct) and use actual API info in the alternator
test.
Closes#13333
TWCS no longer keeps internal state, and will now rely on state
managed by each compaction group through compaction::table_state.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
LCS no longer keeps internal state, and will now rely on state
managed by each compaction group through compaction::table_state.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
That will allow compaction_strategy to access the compaction group state
through compaction::table_state, which is the interface at which replica
talks to the compaction layer.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
this is the 15th changeset of a series which tries to give an overhaul to the CMake building system. this series has two goals:
- to enable developer to use CMake for building scylla. so they can use tools (CLion for instance) with CMake integration for better developer experience
- to enable us to tweak the dependencies in a simpler way. a well-defined cross module / subsystem dependency is a prerequisite for building this project with the C++20 modules.
this changeset includes following changes:
- build: cmake: add two missing tests
- build: cmake: port more cxxflags from configure.py
Closes#13262
* github.com:scylladb/scylladb:
build: cmake: add missing source files to idl and service
build: cmake: port more cxxflags from configure.py
build: cmake: add two missing tests
We need this so that we can have multi-partition mutations which are applied atomically. If they live on different shards, we can't guarantee atomic write to the commitlog.
Fixes: #12642Closes#13134
* github.com:scylladb/scylladb:
test_raft_upgrade: add a test for schema commit log feature
scylla_cluster.py: add start flag to server_add
ServerInfo: drop host_id
scylla_cluster.py: add config to server_add
scylla_cluster.py: add expected_error to server_start
scylla_cluster.py: ScyllaServer.start, refactor error reporting
scylla_cluster.py: fix ScyllaServer.start, reset cmd if start failed
raft: check if schema commitlog is initialized Refuse to boot if neither the schema commitlog feature nor force_schema_commit_log is set. For the upgrade procedure the user should wait until the schema commitlog feature is enabled before enabling consistent_cluster_management.
raft: move raft initialization after init_system_keyspace
database: rename before_schema_keyspace_init->maybe_init_schema_commitlog
raft: use schema commitlog for raft tables
init_system_keyspace: refactoring towards explicit load phases
Task manager task implementations of classes that cover
cleanup keyspace compaction which can be started through
/storage_service/keyspace_compaction/ api.
Top level task covers the whole compaction and creates child
tasks on each shard.
Closes#12712
* github.com:scylladb/scylladb:
test: extend test_compaction_task.py to test cleanup compaction
compaction: create task manager's task for cleanup keyspace compaction on one shard
compaction: create task manager's task for cleanup keyspace compaction
api: add get_table_ids to get table ids from table infos
compaction: create cleanup_compaction_task_impl
this is a part of a series migrating from `operator<<(ostream&, ..)` based formatting to fmtlib based formatting. the goal here is to enable fmtlib to print `range_tombstone` and `range_tombstone_change` without using ostream<<. also, this change removes all existing callers of `operator<<(ostream, const range_tombstone &)` and `operator<<(ostream, const range_tombstone_change &)`, and then removes these two `operator<<`s.
Refs #13245Closes#13260
* github.com:scylladb/scylladb:
mutation: drop operator<<(ostream, const range_tombstone{_change,} &)
mutation: use fmtlib to print range_stombstone{_change,}
mutation: mutation_fragment_v2: specialize fmt::formatter<range_tombstone_change>
mutation: range_tombstone: specialize fmt::formatter<range_tombstone>
Create a local method called create_test_table that has the same
signature as test::create_test_table, but uses random schema behind the
scenes to generate the schema and the data, then migrate all the test
cases to use it instead.
To accomodate to the added randomness added by the random schema and
random data, the unreliable querier cache population checks was replaced
with more reliable lookup and miss checks, to prevent test flakiness.
Querier cache population checks worked well with a fixed and simple
schema and a fixed table population, they don't work that well with
random data.
With this, there are no more uses of test_table.hh in this test and the
include can be removed.
This keyspace exists by default and thus we don't have to create a new
one for each test. Also use `get_name()` to pass the test case's name as
table name, instead of hard-coding it. We already had some copy-pasta
creep in: two tests used the same table name. This is an error, as each
test runs in its own env, but it is confusing to see another test case's
name in the logs.
Propagate the page size to the result builder, so it can determine when
a page is short and thus it is the last page, instead of asking for more
pages until an empty one turns up. This will make tests more reliable
when dealing with random datasets.
Also change how the page counter is bumped: bump it after the current
page is executed, at which point we know whether there will be a next
page or not. This fixes an off-by-one seen in some cases.
Use the random_schema and its facilities to generate the schema and the
dataset. This allows the test to provide a much better coverage then the
previous, fixed and simplistic schema did.
Also reduce the test table population and the number of scans ran on it
to the test runs in a more reasonable time-frame.
We run these tests all the time due to CI, so no need to try to do too
much in a single run.
The tests in this file work with random schema and random data. Some
seeds can generate large partitions and rows, give the test some
more headroom to work with.
* generate lowercase names (upper-case seems to cause problems);
* preserve dependency order between UDTs when dumping them from schema;
* use built-in describe() to dump to CQL string;
* drop single arg dump_udts() overlad, which was not recursive, unlike
the vector variant;
For regular and static columns, to introduce some further randomness.
So far frozen types were generated only for primary key members and
embedded types.
before this change, we use `round(random.random(), 5)` for
the value of `bloom_filter_fp_chance` config option. there are
chances that this expression could return a number lower or equal
to 6.71e-05.
but we do have a minimal for this option, which is defined by
`utils::bloom_calculations::probs`. and the minimal false positive
rate is 6.71e-05.
we are observing test failures where the we are using 0 for
the option, and scylla right rejected it with the error message of
```
bloom_filter_fp_chance must be larger than 6.71e-05 and less than or equal to 1.0 (got 0)
```.
so, in this change, to address the test failure, we always use a number
slightly greater or equal to a number slightly greater to the minimum to
ensure that the randomly picked number is in the range of supported
false positive rate.
Fixes#13313
Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>
Closes#13314
When creating the reader, the lifecycle policy might return one that was saved on the last page and survived in the cache. This reader might have skipped some fast-forwarding ranges while sitting in the cache. To avoid using a reader reading a stale range (from the read's POV), check its read range and fast forward it if necessary.
Fixes: https://github.com/scylladb/scylladb/issues/12916Closes#12932
* github.com:scylladb/scylladb:
readers/multishard: shard_reader: fast-forward created reader to current range
readers/multishard: reader_lifecycle_policy: add get_read_range()
test/boost/multishard_mutation_query_test: paging: handle range becoming wrapping
It would have been better if `flush()` could have been called with a
keyspace and optional table param, but changing it now is too much
churn, so we add a dedicated method to flush a keyspace instead.
After each page, the read range is adjusted so it continues from/after
the last read partition. Sometimes this can result in the range becoming
wrapped like this: (pk, pk]. In this case, we can just drop this range
and continue with the rest of the ranges (if there are multiple ones).
There was an attempt to cut feature-service -> system-keyspace dependency (#13172) which turned out to require more changes. Here's a preparation squeezing from this future work.
This set
- leaves only batch-enabling API in feature service
- keeps the need for async context in feature service
- narrows down system keyspace features API to only load and store records
- relaxes features updating logic in sys.ks.
- cosmetic
Closes#13264
* github.com:scylladb/scylladb:
feature_service: Indentation fix after previous patch
feature_service: Move async context into enable()
system_keyspace: Refactor local features load/save helpers
feature_service: Mark supported_feature_set() const
feature_service: Remove single feature enabling method
boot: Enable features in batch
gossiper: Enable features in batch
The test tries to start a node with
consistent_cluster_management but without
force_schema_commit_log. This is expected to fail,
since the schema commitlog feature should be enabled
by all the cluster nodes.
Sometimes when creating a node it's useful
to just install it and not start. For example,
we may want to try to start it later with
expected error.
The ScyllaServer.install method has been made
exception safe, if an exception occurs, it
reverts to the original state. This allows
to not duplicate the try/except logic
in two of its call sites.
We are going to allow the
ScyllaCluster.add_server function not to
start the server if the caller has requested
that with a special parameter. The host_id
can only be obtained from a running node, so
add_server won't be able to return it in
this case. I've grepped the tests for host_id
and there doesn't seem to be any
reference to it in the code.
Sometimes it's useful to check that the node has failed
to start for a particular reason. If server_start can't
find expected_error in the node's log or if the
node has started without errors, it throws an exception.
Extract the function that encapsulates all the error
reporting logic. We are going to use it in several
other places to implement expected_error feature.
The ScyllaServer expects cmd to be None if the
Scylla process is not running. Otherwise, if start failed
and the test called update_config, the latter will
try to send a signal to a non-existent process via cmd.
We aim (#12642) to use the schema commit log
for raft tables. Now they are loaded at
the first call to init_system_keyspace in
main.cc, but the schema commitlog is only
initialized shortly before the second
call. This is important, since the schema
commitlog initialization
(database::before_schema_keyspace_init)
needs to access schema commitlog feature,
which is loaded from system.scylla_local
and therefore is only available after the
first init_system_keyspace call.
So the idea is to defer the loading of the raft tables
until the second call to init_system_keyspace,
just as it works for schema tables.
For this we need a tool to mark which tables
should be loaded in the first or second phase.
To do this, in this patch we introduce system_table_load_phase
enum. It's set in the schema_static_props for schema tables.
It replaces the system_keyspace::table_selector in the
signature of init_system_keyspace.
The call site for populate_keyspace in init_system_keyspace
was changed, table_selector.contains_keyspace was replaced with
db.local().has_keyspace. This check prevents calling
populate_keyspace(system_schema) on phase1, but allows for
populate_keyspace(system) on phase2 (to init raft tables).
On this second call some tables from system keyspace
(e.g. system.local) may have already been populated on phase1.
This check protects from double-populating them, since every
populated cf is marked as ready_for_writes.
This patch increases the connection timeout in the get_cql_cluster()
function in test/cql-pytest/run.py. This function is used to test
that Scylla came up, and also test/alternator/run uses it to set
up the authentication - which can only be done through CQL.
The Python driver has 2-second and 5-second default timeouts that should
have been more than enough for everybody (TM), but in #13239 we saw
that in one case it apparently wasn't enough. So to be extra safe,
let's increase the default connection-related timeouts to 60 seconds.
Note this change only affects the Scylla *boot* in the test/*/run
scripts, and it does not affect the actual tests - those have different
code to connect to Scylla (see cql_session() in test/cql-pytest/util.py),
and we already increased the timeouts there in #11289.
Fixes#13239
Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Closes#13291
before this change, alternator_timeout_in_ms is not live-updatable,
as after setting executor's default timeout right before creating
sharded executor instances, they never get updated with this option
anymore.
in this change,
* alternator_timeout_in_ms is marked as live-updateable
* executor::_s_default_timeout is changed to a thread_local variable,
so it can be updated by a per-shard updateable_value. and
it is now a updateable_value, so its variable name is updated
accordingly. this value is set in the ctor of executor, and
it is disconnected from the corresponding named_value<> option
in the dtor of executor.
* alternator_timeout_in_ms is passed to the constructor of
executor via sharded_parameter, so executor::_timeout_in_ms can
be initialized on per-shard basis
* executor::set_default_timeout() is dropped, as we already pass
the option to executor in its ctor.
please note, in the ctor of executor, we always update the cached
value of `s_default_timeout` with the value of `_timeout_in_ms`,
and we set the default timeout to 10s in `alternator_test_env`.
this is a design decision to avoid bending the production code for
testing, as in production, we always set the timeout with the value
specified either by the default value of yaml conf file.
Fixes#12232
Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>
Cassandra detects when a batch has both an IF EXISTS and IF NOT EXISTS
on the same row, and complains this is not a useful request (after all,
it can never succeed, because the batch can only succeed if both conditions
are true, and that can't be if one checks IF EXISTS and the other
IF NOT EXISTS).
This patch adds a test, test_lwt_with_batch_conflict_1, which checks
that this case results in an error. It passes on Cassandra, but xfails
on Scylla which doesn't report an error in this case.
A second test, test_lwt_with_batch_conflict_2, shows that the detection
of the EXISTS / NOT EXISTS conflict is special, and other conflicts
such as having both "r=1" and "r=2" for the same row, are NOT detected
by Cassandra.
Refs #13011.
Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Closes#13270