Secondary tracing sessions used to compute the execution time
from the point of their `begin()`-ning, not the parent session's
`begin()`. As a result, replica reported a slow query if it
exceeded the entire threshold *on that replica* too.
This change augments `trace_info` with the TS of parent's session
starting point, to be used as a reference on replicas.
Fixes#9403Closes#10005
Except for the verb addition, this commit also defines forward_request
and forward_result structures, used as an argument and result of the new
rpc. forward_request is used to forward information about select
statement that does count(*) (or other aggregating functions such as
max, min, avg in the future). Due to the inability to serialize
cql3::statements::select_statement, I chose to include
query::read_command, dht::partition_range_vector and some configuration
options in forward_request. They can be serialized and are sufficient
enough to allow creation of service::pager::query_pagers::pager.
Objects of this type will be serialized and sent as commands to the
group 0 state machine. They contain a set of mutations which modify
group 0 tables (at this point: schema tables and group 0 history table),
the 'previous state ID' which is the last state ID present in the
history table when the operation described by this command has started,
and the 'new state ID' which will be appended to the history table if
this change is successful (successful = the previous state ID is still
equal to the last state ID in the history table at the moment of
application). It also contains the address of the node which constructed
this command.
The state ID mechanism will be described in more detail in a later
commit.
The MIGRATION_REQUEST verb is currently used to pull the contents of
schema tables (in the form of mutations) when nodes synchronize schemas.
We will (ab)use the verb to fetch additional data, such as the contents
of the group 0 history table, for purposes of group 0 snapshot transfer.
We extend `schema_pull_options` with a flag specifying that the puller
requests the additional data associated with group 0 snapshots. This
flag is `false` by default, so existing schema pulls will do what they
did before. If the flag is `true`, the migration request handler will
include the contents of group 0 history table.
Note that if a request is set with the flag set to `true`, that means
the entire cluster must have enabled the Raft feature, which also means
that the handler knows of the flag.
Instead of lengthy blurbs, switch to single-line, machine-readable
standardized (https://spdx.dev) license identifiers. The Linux kernel
switched long ago, so there is strong precedent.
Three cases are handled: AGPL-only, Apache-only, and dual licensed.
For the latter case, I chose (AGPL-3.0-or-later and Apache-2.0),
reasoning that our changes are extensive enough to apply our license.
The changes we applied mechanically with a script, except to
licenses/README.md.
Closes#9937
Refs: #9555
When running the "Kraken" dynamodb streams test to provoke the issued observed by QA, I noticed on my setup mainly two things: Large allocation stalls (+ warnings) and timeouts on read semaphores in DB.
This tries to address the first issue, partly by making query_result_view serialization using chunked vector instead of linear one, and by introducing a streaming option for json return objects, avoiding linearizing to string before wire.
Note that the latter has some overhead issues of its own, mainly data copying, since we essentially will be triple buffering (local, wrapped http stream, and final output stream). Still, normal string output will typically do a lot of realloc which is potential extra copies as well, so...
This is not really performance tested, but with these tweaks I no longer get large alloc stalls at least, so that is a plus. :-)
Closes#9713
* github.com:scylladb/scylla:
alternator::executor: Use streamed result for scan etc if large result
alternator::streams: Use streamed result in get_records if large result
executor/server: Add routine to make stream object return
rjson: Add print to stream of rjson::value
query_idl: Make qr_partition::rows/query_result::partitions chunked
The gc_grace_seconds is a very fragile and broken design inherited from
Cassandra. Deleted data can be resurrected if cluster wide repair is not
performed within gc_grace_seconds. This design pushes the job of making
the database consistency to the user. In practice, it is very hard to
guarantee repair is performed within gc_grace_seconds all the time. For
example, repair workload has the lowest priority in the system which can
be slowed down by the higher priority workload, so that there is no
guarantee when a repair can finish. A gc_grace_seconds value that is
used to work might not work after data volume grows in a cluster. Users
might want to avoid running repair during a specific period where
latency is the top priority for their business.
To solve this problem, an automatic mechanism to protect data
resurrection is proposed and implemented. The main idea is to remove the
tombstone only after the range that covers the tombstone is repaired.
In this patch, a new table option tombstone_gc is added. The option is
used to configure tombstone gc mode. For example:
1) GC a tombstone after gc_grace_seconds
cqlsh> ALTER TABLE ks.cf WITH tombstone_gc = {'mode':'timeout'} ;
This is the default mode. If no tombstone_gc option is specified by the
user. The old gc_grace_seconds based gc will be used.
2) Never GC a tombstone
cqlsh> ALTER TABLE ks.cf WITH tombstone_gc = {'mode':'disabled'};
3) GC a tombstone immediately
cqlsh> ALTER TABLE ks.cf WITH tombstone_gc = {'mode':'immediate'};
4) GC a tombstone after repair
cqlsh> ALTER TABLE ks.cf WITH tombstone_gc = {'mode':'repair'};
In addition to the 'mode' option, another option 'propagation_delay_in_seconds'
is added. It defines the max time a write could possibly delay before it
eventually arrives at a node.
A new gossip feature TOMBSTONE_GC_OPTIONS is added. The new tombstone_gc
option can only be used after the whole cluster supports the new
feature. A mixed cluster works with no problem.
Tests: compaction_test.py, ninja test
Fixes#3560
[avi: resolve conflicts vs data_dictionary]
With this new field comes a new member function called get_page_size.
This new function will be used by the result_memory_accounter to decide
when to cut a page.
The behaviour of get_page_size depends on whether page_size field is
set. This is distinguished by page size being equal to 0 or not. When
page_size is equal to 0 then it's not set and hard_limit will be
returned from get_page_size. Otherwise, get_page_size will return
page_size field.
When read_command is received from an old node, page_size will be equal
to 0 and hard_limit will be used to determine the page size. This is
consistent with the behaviour on the old nodes.
Signed-off-by: Piotr Jastrzebski <piotr@scylladb.com>
Operations of adding or removing a node to Raft configuration
are made idempotent: they do nothing if already done, and
they are safe to resume after a failure.
However, since topology changes are not transactional, if a
bootstrap or removal procedure fails midway, Raft group 0
configuration may go out of sync with topology state as seen by
gossip.
In future we must change gossip to avoid making any persistent
changes to the cluster: all changes to persistent topology state
will be done exclusively through Raft Group 0.
Specifically, instead of persisting the tokens by advertising
them through gossip, the bootstrap will commit a change to a system
table using Raft group 0. nodetool will switch from looking at
gossip-managed tables to consulting with Raft Group 0 configuration
or Raft-managed tables.
Once this transformation is done, naturally, adding a node to Raft
configuration (perhaps as a non-voting member at first) will become the
first persistent change to ring state applied when a node joins;
removing a node from the Raft Group 0 configuration will become the last
action when removing a node.
Until this is done, do our best to avoid a cluster state when
a removed node or a node which addition failed is stuck in Raft
configuration, but the node is no longer present in gossip-managed
system tables. In other words, keep the gossip the primary source of
truth. For this purpose, carefully chose the timing when we
join and leave Raft group 0:
Join the Raft group 0 only after we've advertised our tokens, so the
cluster is aware of this node, it's visible in nodetool status,
but before node state jumps to "normal", i.e. before it accepts
queries. Since the operation is idempotent, invoke it on each
restart.
Remove the node from Group 0 *before* its tokens are removed
from gossip-managed system tables. This guarantees
that if removal from Raft group 0 fails for whatever reason,
the node stays in the ring, so nodetool removenode and
friends are re-tried.
Add tracing.
Implement an RPC to forward add_entry calls from the follower
to leader. Bounce & retry in case of not_a_leader.
Do not retry in case of uncertainty - this can lead to adding
duplicate entries.
The feature is added to core Raft since it's needed by
all current clients - both topology and schema changes.
When forwarding an entry to a remote leader we may get back
a term/index pair that conflicts (has the same index, but is with
a higher term) with a local entry we're still waiting on.
This can happen, e.g. because there was a leader change and the
log was truncated, but we still haven't got the append_entries
RPC from the new leader, still haven't truncated the log locally,
still haven't aborted all the local waits for truncated entries.
Only remove the offending entry from the wait list and abort it.
There may be entries labeled with an older term to the right (with
higher commit index) of the conflicting entry. However, finding them,
would require a linear scan. If we allow it, we may end up doing this
linear scan for *every* conflicting entry during the transition
period, which brings us to N^2 complexity of this step. At the
same time, as soon as append_entries that commits a higher-term
entry with the same index reaches the follower, the waits
for the respective truncated entry will be aborted anyway (see
notify_waiters() which sets dropped_entry exception), so the scan
is unnecessary.
Similarly to being able to add entries, allow to modify
Raft group configuration on a follower. The implementation
works the same way as adding entries - forwards the command
to the leader.
Now that add_entry() or modify_config never throws not_a_leader,
it's more likely to throw timed_out_error, e.g. in case the
network is partitioned. Previously it was only possible due to a
semaphore wait timeout, and this scenario was not tested.
Handle timed_out_error on RPC level to let the existing tests
(specifically the randomized nemesis test) pass.
In anticipation of making system_keyspace a class instead of a
namespace, rename any member that is currently forward-declared,
since one can't forward-declare a class member. Each member
is taken out of the system_keyspace namespace and gains a
system_keyspace prefix. Aliases are added to reduce code churn.
The result isn't lovely, but can be adjusted later.
We define the native reverse format as a reversed mutation fragment
stream that is identical to one that would be emitted by a table with
the same schema but with reversed clustering order. The main difference
to the current format is how range tombstones are handled: instead of
looking at their start or end bound depending on the order, we always
use them as-usual and the reversing reader swaps their bounds to
facilitate this. This allows us to treat reversed streams completely
transparently: just pass along them a reversed schema and all the
reader, compacting and result building code is happily ignorant about
the fact that it is a reversed stream.
This patch implements RAFT extension that allows to perform linearisable
reads by accessing local state machine. The extension is described
in section 6.4 of the PhD. To sum it up to perform a read barrier on
a follower it needs to asks a leader the last committed index that it
knows about. The leader must make sure that it is still a leader before
answering by communicating with a quorum. When follower gets the index
back it waits for it to be applied and by that completes read_barrier
invocation.
The patch adds three new RPC: read_barrier, read_barrier_reply and
execute_read_barrier_on_leader. The last one is the one a follower uses
to ask a leader about safe index it can read. First two are used by a
leader to communicate with a quorum.
Adds a sync_point structure. A sync point is a (possibly incomplete)
mapping from hint queues to a replay position in it. Users will be able
to create sync points consisting of the last written positions of some
hint queues, so then they can wait until hint replay in all of the
queues reach that point.
The sync point supports serialization - first it is serialized with the
help of IDL to a binary form, and then converted to a hexadecimal
string. Deserialization is also possible.
This reverts commit 82c419870a.
This commit removes the HINT_SYNC_POINT_CREATE and HINT_SYNC_POINT_CHECK
rpc verbs.
The upcoming HTTP API for waiting for hint replay will be restricted
to waiting for hints on the node handling the request, so there is no
need for new verbs.
We have enabled off-strategy compaction for bootstrap, replace,
decommission and removenode operations when repair based node operation
is enabled. Unlike node operations like replace or decommission, it is
harder to know when the repair of a table is finished because users can
send multiple repair requests one after another, each request repairing
a few token ranges.
This patch wires off-strategy compaction for regular repair by adding
a timeout based automatic off-strategy compaction trigger mechanism.
If there is no repair activity for sometime, off-strategy compaction
will be triggered for that table automatically.
Fixes#8677Closes#8678
In commit 323f72e48a (repair: Switch to
use NODE_OPS_CMD for replace operation), we switched replace operation
to use the new NODE_OPS_CMD infrastructure.
In this patch set, we continue the work to switch decommission and bootstrap
operation to use NODE_OPS_CMD.
Fixes#8472Fixes#8471Closes#8481
* github.com:scylladb/scylla:
repair: Switch to use NODE_OPS_CMD for bootstrap operation
repair: Switch to use NODE_OPS_CMD for decommission operation
Both hinted handoff and repair are meant to improve the consistency of the cluster's data. HH does this by storing records of failed replica writes and replaying them later, while repair goes through all data on all participaring replicas and makes sure the same data is stored on all nodes. The former is generally cheaper and sometimes (but not always) can bring back full consistency on its own; repair, while being more costly, is a sure way to bring back current data to full consistency.
When hinted handoff and repair are running at the same time, some of the work can be unnecessarily duplicated. For example, if a row is repaired first, then hints towards it become unnecessary. However, repair needs to do less work if data already has good consistency, so if hints finish first, then the repair will be shorter.
This PR introduces a possibility to wait for hints to be replayed before continuing with user-issued repair. The coordinator of the repair operation asks all nodes participating in the repair operation (including itself) to mark a point at the end of all hint queues pointing towards other nodes participating in repair. Then, it waits until hint replay in all those queues reaches marked point, or configured timeout is reached.
This operation is currently opt-in and can be turned on by setting the `wait_for_hint_replay_before_repair_in_ms` config option to a positive value.
Fixes#8102
Tests:
- unit(dev)
- some manual tests:
- shutting down repair coordinator during hints replay,
- shutting down node participating in repair during hints replay,
Closes#8452
* github.com:scylladb/scylla:
repair: introduce abort_source for repair abort
repair: introduce abort_source for shutdown
storage_proxy: add abort_source to wait_for_hints_to_be_replayed
storage_proxy: stop waiting for hints replay when node goes down
hints: dismiss segment waiters when hint queue can't send
repair: plug in waiting for hints to be sent before repair
repair: add get_hosts_participating_in_repair
storage_proxy: coordinate waiting for hints to be sent
config: add wait_for_hint_replay_before_repair option
storage_proxy: implement verbs for hint sync points
messaging_service: add verbs for hint sync points
storage_proxy: add functions for syncing with hints queue
db/hints: make it possible to wait until current hints are sent
db/hints: add a metric for counting processed files
db/hints: allow to forcefully update segment list on flush
In commit 323f72e48a (repair: Switch to
use NODE_OPS_CMD for replace operation), we switched replace operation
to use the new NODE_OPS_CMD infrastructure.
In this patch, we continue the work to switch bootstrap operation to use
NODE_OPS_CMD.
The benefits:
- It is more reliable to detect pending node operations, to avoid
multiple topology changes at the same time, than using gossip status.
- The cluster reverts to a state before the bootstrap operation
automatically in case of error much faster than gossip.
- Allows users to pass a list of dead nodes to ignore during bootstrap
explicitly.
- The BOOTSTRAP gossip status is not needed any more. This is one step
closer to achieve gossip-less topology change.
Fixes#8472
Adds two verbs: HINT_SYNC_POINT_CREATE and HINT_SYNC_POINT_CHECK.
Those will make it possible to create a sync point and regularly poll
to check its existence.
In commit 323f72e48a (repair: Switch to
use NODE_OPS_CMD for replace operation), we switched replace operation
to use the new NODE_OPS_CMD infrastructure.
In this patch, we continue the work to switch decommission operation to use
NODE_OPS_CMD.
The benefits:
- A UUID is used to identify each node operation across the cluster.
- It is more reliable to detect pending node operations, to avoid
multiple topology changes at the same time.
- The cluster reverts to a state before the decommission operation
automatically in case of error. Without this patch, the node to be
decommissioned will be stuck in decommission status forever until it
is restarted and goes back to normal status.
- Allows users to pass a list of dead nodes to ignore for decommission
explicitly.
- The LEAVING gossip status is not needed any more. This is one step
closer to achieve gossip-less topology change.
- Allows us to trigger of off-strategy easily on the node receiving the
ranges
Fixes#8471
In commit c82250e0cf (gossip: Allow deferring
advertise of local node to be up), the replacing node is changed to postpone
the responding of gossip echo message to avoid other nodes sending read
requests to the replacing node. It works as following:
1) replacing node does not respond echo message to avoid other nodes to
mark replacing node as alive
2) replacing node advertises hibernate state so other nodes knows
replacing node is replacing
3) replacing node responds echo message so other nodes can mark
replacing node as alive
This is problematic because after step 2, the existing nodes in the
cluster will start to send writes to the replacing node, but at this
time it is possible that existing nodes haven't marked the replacing
node as alive, thus failing the write request unnecessarily.
For instance, we saw the following errors in issue #8013 (Cassandra
stress fails to achieve consistency when only one of the nodes is down)
```
scylla:
[shard 1] consistency - Live nodes 2 do not satisfy ConsistencyLevel (2
required, 1 pending, live_endpoints={127.0.0.2, 127.0.0.1},
pending_endpoints={127.0.0.3}) [shard 0] gossip - Fail to send
EchoMessage to 127.0.0.3: std::runtime_error (Not ready to respond
gossip echo message)
c-s:
java.io.IOException: Operation x10 on key(s) [4c4f4d37324c35304c30]:
Error executing: (UnavailableException): Not enough replicas available
for query at consistency QUORUM (2 required but only 1 alive
```
To solve this problem, we can do the replacing operation in multiple stages.
One solution is to introduce a new gossip status state as proposed
here: gossip: Introduce STATUS_PREPARE_REPLACE #7416
1) replacing node does not respond echo message
2) replacing node advertises prepare_replace state (Remove replacing
node from natural endpoint, but do not put in pending list yet)
3) replacing node responds echo message
4) replacing node advertises hibernate state (Put replacing node in
pending list)
Since we now have the node ops verb introduced in
829b4c1438 (repair: Make removenode safe
by default), we can do the multiple stage without introducing a new
gossip status state.
This patch uses the NODE_OPS_CMD infrastructure to implement replace
operation.
Improvements:
1) It solves the race between marking replacing node alive and sending
writes to replacing node
2) The cluster reverts to a state before the replace operation
automatically in case of error. As a result, it solves when the
replacing node fails in the middle of the operation, the repacing
node will be in HIBERNATE status forever issue.
3) The gossip status of the node to be replaced is not changed until the
replace operation is successful. HIBERNATE gossip status is not used
anymore.
4) Users can now pass a list of dead nodes to ignore explicitly.
Fixes#8013Closes#8330
* github.com:scylladb/scylla:
repair: Switch to use NODE_OPS_CMD for replace operation
gossip: Add advertise_to_nodes
gossip: Add helper to wait for a node to be up
gossip: Add is_normal_ring_member helper
In commit c82250e0cf (gossip: Allow deferring
advertise of local node to be up), the replacing node is changed to postpone
the responding of gossip echo message to avoid other nodes sending read
requests to the replacing node. It works as following:
1) replacing node does not respond echo message to avoid other nodes to
mark replacing node as alive
2) replacing node advertises hibernate state so other nodes knows
replacing node is replacing
3) replacing node responds echo message so other nodes can mark
replacing node as alive
This is problematic because after step 2, the existing nodes in the
cluster will start to send writes to the replacing node, but at this
time it is possible that existing nodes haven't marked the replacing
node as alive, thus failing the write request unnecessarily.
For instance, we saw the following errors in issue #8013 (Cassandra
stress fails to achieve consistency when only one of the nodes is down)
```
scylla:
[shard 1] consistency - Live nodes 2 do not satisfy ConsistencyLevel (2
required, 1 pending, live_endpoints={127.0.0.2, 127.0.0.1},
pending_endpoints={127.0.0.3}) [shard 0] gossip - Fail to send
EchoMessage to 127.0.0.3: std::runtime_error (Not ready to respond
gossip echo message)
c-s:
java.io.IOException: Operation x10 on key(s) [4c4f4d37324c35304c30]:
Error executing: (UnavailableException): Not enough replicas available
for query at consistency QUORUM (2 required but only 1 alive
```
To solve this problem, we can do the replacing operation in multiple stages.
One solution is to introduce a new gossip status state as proposed
here: gossip: Introduce STATUS_PREPARE_REPLACE #7416
1) replacing node does not respond echo message
2) replacing node advertises prepare_replace state (Remove replacing
node from natural endpoint, but do not put in pending list yet)
3) replacing node responds echo message
4) replacing node advertises hibernate state (Put replacing node in
pending list)
Since we now have the node ops verb introduced in
829b4c1438 (repair: Make removenode safe
by default), we can do the multiple stage without introducing a new
gossip status state.
This patch uses the NODE_OPS_CMD infrastructure to implement replace
operation.
Improvements:
1) It solves the race between marking replacing node alive and sending
writes to replacing node
2) The cluster reverts to a state before the replace operation
automatically in case of error. As a result, it solves when the
replacing node fails in the middle of the operation, the repacing
node will be in HIBERNATE status forever issue.
3) The gossip status of the node to be replaced is not changed until the
replace operation is successful. HIBERNATE gossip status is not used
anymore.
4) Users can now pass a list of dead nodes to ignore explicitly.
Refs #8013
Section 3.10 of the PhD describes two cases for which the extension can
be helpful:
1. Sometimes the leader must step down. For example, it may need to reboot
for maintenance, or it may be removed from the cluster. When it steps
down, the cluster will be idle for an election timeout until another
server times out and wins an election. This brief unavailability can be
avoided by having the leader transfer its leadership to another server
before it steps down.
2. In some cases, one or more servers may be more suitable to lead the
cluster than others. For example, a server with high load would not make
a good leader, or in a WAN deployment, servers in a primary datacenter
may be preferred in order to minimize the latency between clients and
the leader. Other consensus algorithms may be able to accommodate these
preferences during leader election, but Raft needs a server with a
sufficiently up-to-date log to become leader, which might not be the
most preferred one. Instead, a leader in Raft can periodically check
to see whether one of its available followers would be more suitable,
and if so, transfer its leadership to that server. (If only human leaders
were so graceful.)
The patch here implements the extension and employs it automatically
when a leader removes itself from a cluster.
Raft send_snapshot RPC is actually two-way, the follower
responds with snapshot_reply message. This message until now
was, however, muted by RPC.
Do not mute snapshot_reply any more:
- to make it obvious the RPC is two way
- to feed the follower response directly into leader's FSM and
thus ensure that FSM testing results produced when using a test
transport are representative of the real world uses of
raft::rpc.
This is how PhD explain the need for prevoting stage:
One downside of Raft's leader election algorithm is that a server that
has been partitioned from the cluster is likely to cause a disruption
when it regains connectivity. When a server is partitioned, it will
not receive heartbeats. It will soon increment its term to start
an election, although it won't be able to collect enough votes to
become leader. When the server regains connectivity sometime later, its
larger term number will propagate to the rest of the cluster (either
through the server's RequestVote requests or through its AppendEntries
response). This will force the cluster leader to step down, and a new
election will have to take place to select a new leader.
Prevoting stage is addressing that. In the Prevote algorithm, a
candidate only increments its term if it first learns from a majority of
the cluster that they would be willing to grant the candidate their votes
(if the candidate's log is sufficiently up-to-date, and the voters have
not received heartbeats from a valid leader for at least a baseline
election timeout).
The Prevote algorithm solves the issue of a partitioned server disrupting
the cluster when it rejoins. While a server is partitioned, it won't
be able to increment its term, since it can't receive permission
from a majority of the cluster. Then, when it rejoins the cluster, it
still won't be able to increment its term, since the other servers
will have been receiving regular heartbeats from the leader. Once the
server receives a heartbeat from the leader itself, it will return to
the follower state(in the same term).
In our implementation we have "stable leader" extension that prevents
spurious RequestVote to dispose an active leader, but AppendEntries with
higher term will still do that, so prevoting extension is also required.
This patch adds a support for non-voting members. Non voting member is a
member which vote is not counted for leader election purposes and commit
index calculation purposes and it cannot become a leader. But otherwise
it is a normal raft node. The state is needed to let new nodes to catch
up their log without disturbing a cluster.
All kind of transitions are allowed. A node may be added as a voting member
directly or it may be added as non-voting and then changed to be voting
one through additional configuration change. A node can be demoted from
voting to non-voting member through a configuration change as well.
Message-Id: <20210304101158.1237480-2-gleb@scylladb.com>
Changes to the `configuration` and `tagged_uint64` classes are needed
to overcome limitations of the IDL compiler tool, i.e. we need to
supply a constructor to the struct initializing all the
members (raft::configuration) and also need to make an accessor
function for private members (in case of raft::tagged_uint64).
All other structs mirror raft definitions in exactly the same way
they are declared in `raft.hh`.
`tagged_id` and `tagged_uint64` are used directly instead of their
typedef-ed companions defined in `raft.hh` since we don't want
to introduce indirect dependencies. In such case it can be guaranteed
that no accidental changes made outside of the idl file will affect idl
definitions.
This patch also fixes a minor typo in `snapshot_id_tag` struct used
in `snapshot_id` typedef.
The value of mutation_partition_view::rows() may be very large, but is
used almost exclusively for iteration, so in order to avoid a big allocation
for an std::vector, we change its type to an utils::chunked_vector.
Fixes#7918
Signed-off-by: Wojciech Mitros <wojciech.mitros@scylladb.com>
"
This patch series consists of the following patches:
1. The first one turned out to be a massive rewrite of almost
everything in `idl-compiler.py`. It aims to decouple parser
structures from the internal representation which is used
in the code-generation itself.
Prior to the patch everything was working with raw token lists and
the code was extremely fragile and hard to understand and modify.
Moreover, every change in the parser code caused a cascade effect
of breaking things at many different places, since they were relying
on the exact format of output produced by parsing rules.
Now there is a bunch of supplementary AST structures which provide
hierarchical and strongly typed structure as the output of parsing
routine.
It is much easier to verify (by the means of `isinstance`, for example)
and extend since the internal structures used in code-generation are
decoupled from the structure of parsing rules, which are now controlled
by custom parse actions providing high-level abstractions.
It is tested manually by checking that the old code produces exactly
the same autogenerated sources for all Scylla IDLs as the new one.
2 and 3. Cosmetics changes only: fixed a few typos and moved from
old-fashioned `string.Template` to python f-strings.
This improves readability of the idl-compiler code by a lot.
Only one non-functional whitespace change introduced.
4. This patch adds a very basic support for the parser to
understand `const` specifier in case it's used with a template
parameter for a data member in a class, e.g.
struct my_struct {
std::vector<const raft::log_entry> entries;
};
It actually does two things:
* Adjusts `static_asserts` in corresponding serializer methods
to match const-ness of fields.
* Defines a second serializer specialization for const type in
`.dist.hh` right next to non-const one.
This seems to be sufficient for raft-related uses for now.
Please note there is no support for the following cases, though:
const std::vector<raft::log_entry> entries;
const raft::term_t term;
None of the existing IDLs are affected by the change, so that
we can gradually improve on the feature and write the idl
unit-tests to increase test coverage with time.
5. A basic unit-test that writes a test struct with an
`std::vector<S<const T>>` field and reads it back to verify
that serialization works correctly.
6. Basic documentation for AST classes.
TODO: should also update the docs in `docs/IDL.md`. But it is already
quite outdated, and some changes would even be out of scope for this
patch set.
"
* 'idl-compiler-refactor-v5' of https://github.com/ManManson/scylla:
idl: add docstrings for AST classes
idl: add unit-test for `const` specifiers feature
idl: allow to parse `const` specifiers for template arguments
idl: fix a few typos in idl-compiler
idl: switch from `string.Template` to python f-strings and format string in idl-compiler
idl: Decouple idl-compiler data structures from grammar structure
Currently removenode works like below:
- The coordinator node advertises the node to be removed in
REMOVING_TOKEN status in gossip
- Existing nodes learn the node in REMOVING_TOKEN status
- Existing nodes sync data for the range it owns
- Existing nodes send notification to the coordinator
- The coordinator node waits for notification and announce the node in
REMOVED_TOKEN
Current problems:
- Existing nodes do not tell the coordinator if the data sync is ok or failed.
- The coordinator can not abort the removenode operation in case of error
- Failed removenode operation will make the node to be removed in
REMOVING_TOKEN forever.
- The removenode runs in best effort mode which may cause data
consistency issues.
It means if a node that owns the range after the removenode
operation is down during the operation, the removenode node operation
will continue to succeed without requiring that node to perform data
syncing. This can cause data consistency issues.
For example, Five nodes in the cluster, RF = 3, for a range, n1, n2,
n3 is the old replicas, n2 is being removed, after the removenode
operation, the new replicas are n1, n5, n3. If n3 is down during the
removenode operation, only n1 will be used to sync data with the new
owner n5. This will break QUORUM read consistency if n1 happens to
miss some writes.
Improvements in this patch:
- This patch makes the removenode safe by default.
We require all nodes in the cluster to participate in the removenode operation and
sync data if needed. We fail the removenode operation if any of them is down or
fails.
If the user want the removenode operation to succeed even if some of the nodes
are not available, the user has to explicitly pass a list of nodes that can be
skipped for the operation.
$ nodetool removenode --ignore-dead-nodes <list_of_dead_nodes_to_ignore> <host_id>
Example restful api:
$ curl -X POST "http://127.0.0.1:10000/storage_service/remove_node/?host_id=7bd303e9-4c7b-4915-84f6-343d0dbd9a49&ignore_nodes=127.0.0.3,127.0.0.5"
- The coordinator can abort data sync on existing nodes
For example, if one of the nodes fails to sync data. It makes no sense for
other nodes to continue to sync data because the whole operation will
fail anyway.
- The coordinator can decide which nodes to ignore and pass the decision
to other nodes
Previously, there is no way for the coordinator to tell existing nodes
to run in strict mode or best effort mode. Users will have to modify
config file or run a restful api cmd on all the nodes to select strict
or best effort mode. With this patch, the cluster wide configuration is
eliminated.
Fixes#7359Closes#7626
"
gossip: Get rid of seed concept
The concept of seed and the different behaviour between seed nodes and
non seed nodes generate a lot of confusion, complication and error for
users. For example, how to add a seed node into into a cluster, how to
promote a non seed node to a seed node, how to choose seeds node in
multiple DC setup, edit config files for seeds, why seed node does not
bootstrap.
If we remove the concept of seed, it will get much easier for users.
After this series, seed config option is only used once when a new node
joins a cluster.
Major changes:
Seed nodes are only used as the initial contact point nodes.
Seed nodes now perform bootstrap. The only exception is the first node
in the cluster.
The unsafe auto_bootstrap option is now ignored.
Gossip shadow round now talks to all nodes instead of just seed nodes.
Refs: #6845
Tests: update_cluster_layout_tests.py + manual test
"
* 'gossip_no_seed_v2' of github.com:asias/scylla:
gossip: Get rid of seed concept
gossip: Introduce GOSSIP_GET_ENDPOINT_STATES verb
gossip: Add do_apply_state_locally helper
gossip: Do not talk to seed node explicitly
gossip: Talk to live endpoints in a shuffled fashion
Currently, we cannot select more than 2^32 rows from a table because we are limited by types of
variables containing the numbers of rows. This patch changes these types and sets new limits.
The new limits take effect while selecting all rows from a table - custom limits of rows in a result
stay the same (2^32-1).
In classes which are being serialized and used in messaging, in order to be able to process queries
originating from older nodes, the top 32 bits of new integers are optional and stay at the end
of the class - if they're absent we assume they equal 0.
The backward compatibility was tested by querying an older node for a paged selection, using the
received paging_state with the same select statement on an upgraded node, and comparing the returned
rows with the result generated for the same query by the older node, additionally checking if the
paging_state returned by the upgraded node contained new fields with correct values. Also verified
if the older node simply ignores the top 32 bits of the remaining rows number when handling a query
with a paging_state originating from an upgraded node by generating and sending such a query to
an older node and checking the paging_state in the reply(using python driver).
Fixes#5101.
This field will replace max size which is currently passed once per
established rpc connection via the CLIENT_ID verb and stored as an
auxiliary value on the client_info. For now it is unused, but we update
all sites creating a read command to pass the correct value to it. In the
next patch we will phase out the old max size and use this field to pass
max size on each verb instead.
The new verb is used to replace the current gossip shadow round
implementation. Current shadow round implementation reuses the gossip
syn and ack async message, which has plenty of drawbacks. It is hard to
tell if the syn messages to a specific peer node has responded. The
delayed responses from shadow round can apply to the normal gossip
states even if the shadow round is done. The syn and ack message
handler are full special cases due to the shadow round. All gossip
application states including the one that are not relevant are sent
back. The gossip application states are applied and the gossip
listeners are called as if is in the normal gossip operation. It is
completely unnecessary to call the gossip listeners in the shadow round.
This patch introduces a new verb to request the exact gossip application
states the shadow round needed with a synchronous verb and applies the
application states without calling the gossip listeners. This patch
makes the shadow round easier to reason about, more robust and
efficient.
Refs: #6845
Tests: update_cluster_layout_tests.py
"
After "Make replacing node take writes" series, with repair based node
operations disabled, we saw the replace operation fail like:
```
[shard 0] init - Startup failed: std::runtime_error (unable to find
sufficient sources for streaming range (9203926935651910749, +inf) in
keyspace system_auth)
```
The reason is the system_auth keyspace has default RF of 1. It is
impossible to find a source node to stream from for the ranges owned by
the replaced node.
In the past, the replace operation with keyspace of RF 1 passes, because
the replacing node calls token_metadata.update_normal_tokens(tokens,
ip_of_replacing_node) before streaming. We saw:
```
[shard 0] range_streamer - Bootstrap : keyspace system_auth range
(-9021954492552185543, -9016289150131785593] exists on {127.0.0.6}
```
Node 127.0.0.6 is the replacing node 127.0.0.5. The source node check in
range_streamer::get_range_fetch_map will pass if the source is the node
itself. However, it will not stream from the node itself. As a result,
the system_auth keyspace will not get any data.
After the "Make replacing node take writes" series, the replacing node
calls token_metadata.update_normal_tokens(tokens, ip_of_replacing_node)
after the streaming finishes. We saw:
```
[shard 0] range_streamer - Bootstrap : keyspace system_auth range
(-9049647518073030406, -9048297455405660225] exists on {127.0.0.5}
```
Since 127.0.0.5 was dead, the source node check failed, so the bootstrap
operation.
Ta fix, we ignore the table of RF 1 when it is unable to find a source
node to stream.
Fixes#6351
"
* asias-fix_bootstrap_with_rf_one_in_range_streamer:
range_streamer: Handle table of RF 1 in get_range_fetch_map
streaming: Use separate streaming reason for replace operation