Commit Graph

136 Commits

Author SHA1 Message Date
Asias He
72cc596842 repair: Wire off-strategy compaction for regular repair
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 #8677

Closes #8678
2021-05-26 11:41:27 +03:00
Avi Kivity
d6d6758857 Merge 'Switch to use NODE_OPS_CMD for decommission and bootstrap operation' from Asias He
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 #8472
Fixes #8471

Closes #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
2021-05-06 17:28:19 +03:00
Avi Kivity
6ffd813b7b Merge 'hints: delay repair until hints are replayed' from Piotr Dulikowski
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
2021-05-03 18:47:27 +03:00
Asias He
84a78f4558 repair: Switch to use NODE_OPS_CMD for bootstrap operation
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
2021-04-28 09:53:04 +08:00
Piotr Dulikowski
82c419870a messaging_service: add verbs for hint sync points
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.
2021-04-27 15:06:39 +02:00
Avi Kivity
0af7a22c21 repair: remove partition_checksum and related code
80ebedd242 made row-level repair mandatory, so there remain no
callers to partition_checksum. Remove it.

Closes #8537
2021-04-22 18:56:53 +03:00
Asias He
1513de633b repair: Switch to use NODE_OPS_CMD for decommission operation
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
2021-04-21 20:35:54 +08:00
Avi Kivity
40b60e8f09 Merge 'repair: Switch to use NODE_OPS_CMD for replace operation' from Asias He
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 #8013

Closes #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
2021-04-04 12:54:09 +03:00
Asias He
323f72e48a repair: Switch to use NODE_OPS_CMD for replace operation
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
2021-04-01 09:38:54 +08:00
Gleb Natapov
9d6bf7f351 raft: introduce leader stepdown procedure
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.
2021-03-22 10:28:43 +02:00
Konstantin Osipov
4afa662d62 raft: respond with snapshot_reply to send_snapshot RPC
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.
2021-03-18 16:56:42 +03:00
Gleb Natapov
1f868d516e raft: implement prevoting stage in leader election
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.
2021-03-12 11:09:21 +02:00
Gleb Natapov
dd6ba3d507 raft: add non-voting member support
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>
2021-03-09 13:47:48 +01:00
Konstantin Osipov
1ca738d9a2 raft: joint consensus, use unordered_set for server_address list 2021-01-29 22:07:07 +03:00
Pavel Solodovnikov
e1504bbf0e raft: add IDL definitions for raft types
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.
2021-01-29 01:59:10 +03:00
Wojciech Mitros
59769efd3b idl: change the type of mutation_partition_view::rows() to a chunked_vector
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>
2021-01-13 04:25:53 +01:00
Avi Kivity
fdb47c954d Merge "idl: allow IDL compiler to parse const specifiers for template arguments" from Pavel S
"
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
2020-12-16 14:05:33 +02:00
Pavel Solodovnikov
1e6df841a5 idl: add unit-test for const specifiers feature
Signed-off-by: Pavel Solodovnikov <pa.solodovnikov@scylladb.com>
2020-12-15 16:03:18 +03:00
Asias He
829b4c1438 repair: Make removenode safe by default
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 #7359

Closes #7626
2020-12-10 10:14:39 +02:00
Avi Kivity
3b1ff90a1a Merge "Get rid of seed concept in gossip" from Asias
"
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
2020-08-17 09:50:51 +03:00
Wojciech Mitros
45215746fe increase the maximum size of query results to 2^64
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.
2020-08-03 17:32:49 +02:00
Botond Dénes
92a7b16cba query: read_command: add max_result_size
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.
2020-07-28 18:00:29 +03:00
Asias He
cd7d64f588 gossip: Introduce GOSSIP_GET_ENDPOINT_STATES verb
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
2020-07-27 09:15:11 +08:00
Avi Kivity
9afd599d7c Merge 'range_streamer: Handle table of RF 1 in get_range_fetch_map' from Asias
"
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
2020-06-10 16:03:13 +03:00
Asias He
c02fea5f04 repair: Ignore table removed in sync_data_using_repair
Commit 75cf255c67 (repair: Ignore keyspace
that is removed in sync_data_using_repair) is not enough to fix the
issue because when the repair master checks if the table is dropped, the
table might not be dropped yet on the repair master.

To fix, the repair master should check if the follower failed the repair
because the table is dropped by checking the error returned from
follower.

With this patch, we would see

WARN  2020-04-14 11:19:00,417 [shard 0] repair - repair id 1 on shard 0
completed successfully, keyspace=ks, ignoring dropped tables={cf}

when the table is dropped during bootstrap.

Tests: update_cluster_layout_tests.py:TestUpdateClusterLayout.simple_add_new_node_while_schema_changes_test

Fixes: #5942
2020-05-24 13:39:59 +03:00
Asias He
fa9ee234a0 streaming: Use separate streaming reason for replace operation
Currently, replace and bootstrap share the same streaming reason,
stream_reason::bootstrap, because they share most of the code
in boot_strapper.

In order to distinguish the two, we need to introduce a new stream
reason, stream_reason::replace. It is safe to do so in a mixed cluster
because current code only check if the stream_reason is
stream_reason::repair.

Refs: #6351
2020-05-22 09:30:52 +08:00
Botond Dénes
e778b072b1 read_command: use bool_class for is_first_page parameter
The constructor of `read_command` is used both by IDL and clients in the
code. However, this constructor has a parameter that is not used by IDL:
`read_timestamp`. This requires that this parameter is the very last in
the list and that new parameters that are used by IDL are added before
it. One such new parameter was `bool is_first_page`. Adding this
parameter right before the read timestamp one created a situation where
the last parameter (read_timestamp) implicitly converts to the one
before it (is_first_page). This means that some call sites passing
`read_timestamp` were now silently converting this to `is_first_page`,
effectively dropping the timestamp.

This patch aims to rectify this, while also avoiding similar accidents
in the future, by making `is_first_page` a `bool_class` which doesn't
have any implicit convertions defined. This change does not break the
ABI as `bool_class` is also sent as a `bool` on the wire.

Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Tests: unit(dev)
Message-Id: <20200422073657.87241-1-bdenes@scylladb.com>
2020-04-22 11:01:22 +03:00
Piotr Jastrzebski
b569d127a0 token: change data to array<uint8_t, 8>
It is save to do such change because we support only
Murmur3Partitioner which uses only tokens that are
8 bytes long.

Signed-off-by: Piotr Jastrzebski <piotr@scylladb.com>
2020-02-05 09:30:46 +01:00
Asias He
7322b749e0 repair: Do not return working_row_buf_nr in get combined row hash verb
In commit b463d7039c (repair: Introduce
get_combined_row_hash_response), working_row_buf_nr is returned in
REPAIR_GET_COMBINED_ROW_HASH in addition to the combined hash. It is
scheduled to be part of 3.1 release. However it is not backported to 3.1
by accident.

In order to be compatible between 3.1 and 3.2 repair. We need to drop
the working_row_buf_nr in 3.2 release.

Fixes: #5490
Backports: 3.2
Tests: Run repair in a mixed 3.1 and 3.2 cluster
2019-12-21 20:13:15 +02:00
Vladimir Davydov
bf5f864d80 paxos: piggyback result query on prepare response
Current LWT implementation uses at least three network round trips:
 - first, execute PAXOS prepare phase
 - second, query the current value of the updated key
 - third, propose the change to participating replicas

(there's also learn phase, but we don't wait for it to complete).

The idea behind the optimization implemented by this patch is simple:
piggyback the current value of the updated key on the prepare response
to eliminate one round trip.

To generate less network traffic, only the closest to the coordinator
replica sends data while other participating replicas send digests which
are used to check data consistency.

Note, this patch changes the API of some RPC calls used by PAXOS, but
this should be okay as long as the feature in the early development
stage and marked experimental.

To assess the impact of this optimization on LWT performance, I ran a
simple benchmark that starts a number of concurrent clients each of
which updates its own key (uncontended case) stored in a cluster of
three AWS i3.2xlarge nodes located in the same region (us-west-1) and
measures the aggregate bandwidth and latency. The test uses shard-aware
gocql driver. Here are the results:

                latency 99% (ms)    bandwidth (rq/s)    timeouts (rq/s)
    clients     before  after       before  after       before  after
          1          2      2          626    637            0      0
          5          4      3         2616   2843            0      0
         10          3      3         4493   4767            0      0
         50          7      7        10567  10833            0      0
        100         15     15        12265  12934            0      0
        200         48     30        13593  14317            0      0
        400        185     60        14796  15549            0      0
        600        290     94        14416  15669            0      0
        800        568    118        14077  15820            2      0
       1000        710    118        13088  15830            9      0
       2000       1388    232        13342  15658           85      0
       3000       1110    363        13282  15422          233      0
       4000       1735    454        13387  15385          329      0

That is, this optimization improves max LWT bandwidth by about 15%
and allows to run 3-4x more clients while maintaining the same level
of system responsiveness.
2019-11-24 11:35:29 +02:00
Gleb Natapov
8d6201a23b lwt: Add RPC verbs needed for paxos implementation
Paxos protocol has three stages: prepare, accept, learn. This patch adds
rpc verb for each of those stages. To be term compatible with Cassandra
the patch calls those stages: prepare, propose, commit.
2019-10-27 23:21:51 +03:00
Botond Dénes
7adc764b6e messaging_service: add canonical_support to schema pull and push verbs
The verbs are:
* DEFINITIONS_UPDATE (push)
* MIGRATION_REQUEST (pull)

Support was added in a backward-compatible way. The push verb, sends
both the old frozen mutation parameter, and the new optional canonical
mutation parameter. It is expected that new nodes will use the latter,
while old nodes will fall-back to the former. The pull verb has a new
optional `options` parameter, which for now contains a single flag:
`remote_supports_canonical_mutation_retval`. This flag, if set, means
that the remote node supports the new canonical mutation return value,
thus the old frozen mutations return value can be left empty.
2019-09-04 10:32:44 +03:00
Nadav Har'El
f9d6eaf5ff reconcilable_result: switch to chunked_vector
Merged patch series from Avi Kivity:

In rare but valid cases (reconciling many tombstones, paging disabled),
a reconciled_result can grow large. This triggers large allocation
warnings. Switch to chunked_vector to avoid the large allocation.
In passing, fix chunked_vector's begin()/end() const correctness, and
add the reverse iterator function family which is needed by the conversion.

Fixes #4780.

Tests: unit (dev)

Commit Summary

    utils: chunked_vector: make begin()/end() const correct
    utils::chunked_vector: add rbegin() and related iterators
    reconcilable_result: use chunked_vector to hold partitions
2019-08-11 16:03:13 +03:00
Asias He
bac987e32a streaming: Send error code from the sender to receiver
In case of error on the sender side, the sender does not propagate the
error to the receiver. The sender will close the stream. As a result,
the receiver will get nullopt from the source in
get_next_mutation_fragment and pass mutation_fragment_opt with no value
to the generating_reader. In turn, the generating_reader generates end
of stream. However, the last element that the generating_reader has
generated can be any type of mutation_fragment. This makes the sstable
that consumes the generating_reader violates the mutation_fragment
stream rule.

To fix, we need to propagate the error. However RPC streaming does not
support propagate the error in the framework. User has to send an error
code explicitly.

Fixes: #4789
2019-08-06 16:54:56 +02:00
Avi Kivity
093d2cd7e5 reconcilable_result: use chunked_vector to hold partitions
Usually, a reconcilable_result holds very few partitions (1 is common),
since the page size is limited by 1MB. But if we have paging disabled or
if we are reconciling a range full of tombstones, we may see many more.
This can cause large allocations.

Change to chunked_vector to prevent those large allocations, as they
can be quite expensive.

Fixes #4780.
2019-08-01 18:49:13 +03:00
Piotr Sarna
c1d5aef735 db: add system_schema.computed_columns
Information on which columns of a table are 'computed' is now kept
in system_schema.computed_columns system table.
2019-07-19 11:58:42 +02:00
Calle Wilund
c540e36fe2 gms::inet_address: Make serialization ipv6 aware
Because inet_address was initially hardcoded to
ipv4, its wire format is not very forward compatible.
Since we potentially need to communicate with older version nodes, we
manually define the new serial format for inet_address to be:

ipv4: 4  bytes address
ipv6: 4  bytes marker 0xffffffff (invalid address)
      16 bytes data -> address
2019-07-08 14:13:09 +00:00
Asias He
c93113f3a5 idl: Add repair_row_on_wire_with_cmd 2019-07-02 21:18:54 +08:00
Asias He
a90fb24efc idl: Add repair_hash_with_cmd 2019-07-02 21:18:37 +08:00
Asias He
599d40fbe9 idl: Add repair_stream_cmd 2019-07-02 21:18:15 +08:00
Asias He
672c24f6b0 idl: Add send_full_set_rpc_stream for row_level_diff_detect_algorithm 2019-07-02 21:17:36 +08:00
Asias He
b463d7039c repair: Introduce get_combined_row_hash_response
Currently, REPAIR_GET_COMBINED_ROW_HASH RPC verb returns only the
repair_hash object. In the future, we will use set reconciliation
algorithm to decode the full row hashes in working row buf. It is useful
to return the number of rows inside working row buf in addition to the
combined row hashes to make sure the decode is successful.

It is also better to use a wrapper class for the verb response so we can
extend the return values later more easily with IDL.

Fixes #4526
Message-Id: <93be47920b523f07179ee17e418760015a142990.1559771344.git.asias@scylladb.com>
2019-06-12 13:51:29 +03:00
Piotr Sarna
acf7bedad4 idl,service: add persistent last partition row count
In order to process paged queries with per-partition limits properly,
paging state needs to keep additional information: what was the row
count of last partition returned in previous run.
That's necessary because the end of previous page and the beginning
of current one might consist of rows with the same partition key
and we need to be able to trim the results to the number indicated
by per-partition limit.
2019-02-18 11:06:44 +01:00
Duarte Nunes
fa2b0384d2 Replace std::experimental types with C++17 std version.
Replace stdx::optional and stdx::string_view with the C++ std
counterparts.

Some instances of boost::variant were also replaced with std::variant,
namely those that called seastar::visit.

Scylla now requires GCC 8 to compile.

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20190108111141.5369-1-duarte@scylladb.com>
2019-01-08 13:16:36 +02:00
Avi Kivity
c96fc1d585 Merge "Introduce row level repair" from Asias
"
=== How the the partition level repair works

- The repair master decides which ranges to work on.
- The repair master splits the ranges to sub ranges which contains around 100
partitions.
- The repair master computes the checksum of the 100 partitions and asks the
related peers to compute the checksum of the 100 partitions.
- If the checksum matches, the data in this sub range is synced.
- If the checksum mismatches, repair master fetches the data from all the peers
and sends back the merged data to peers.

=== Major problems with partition level repair

- A mismatch of a single row in any of the 100 partitions causes 100
partitions to be transferred. A single partition can be very large. Not to
mention the size of 100 partitions.

- Checksum (find the mismatch) and streaming (fix the mismatch) will read the
same data twice

=== Row level repair

Row level checksum and synchronization: detect row level mismatch and transfer
only the mismatch

=== How the row level repair works

- To solve the problem of reading data twice

Read the data only once for both checksum and synchronization between nodes.

We work on a small range which contains only a few mega bytes of rows,
We read all the rows within the small range into memory. Find the
mismatch and send the mismatch rows between peers.

We need to find a sync boundary among the nodes which contains only N bytes of
rows.

- To solve the problem of sending unnecessary data.

We need to find the mismatched rows between nodes and only send the delta.
The problem is called set reconciliation problem which is a common problem in
distributed systems.

For example:
Node1 has set1 = {row1, row2, row3}
Node2 has set2 = {      row2, row3}
Node3 has set3 = {row1, row2, row4}

To repair:
Node1 fetches nothing from Node2 (set2 - set1), fetches row4 (set3 - set1) from Node3.
Node1 sends row1 and row4 (set1 + set2 + set3 - set2) to Node2
Node1 sends row3 (set1 + set2 + set3 - set3) to Node3.

=== How to implement repair with set reconciliation

- Step A: Negotiate sync boundary

class repair_sync_boundary {
    dht::decorated_key pk;
    position_in_partition position
}

Reads rows from disk into row buffers until the size is larger than N
bytes. Return the repair_sync_boundary of the last mutation_fragment we
read from disk. The smallest repair_sync_boundary of all nodes is
set as the current_sync_boundary.

- Step B: Get missing rows from peer nodes so that repair master contains all the rows

Request combined hashes from all nodes between last_sync_boundary and
current_sync_boundary. If the combined hashes from all nodes are identical,
data is synced, goto Step A. If not, request the full hashes from peers.

At this point, the repair master knows exactly what rows are missing. Request the
missing rows from peer nodes.

Now, local node contains all the rows.

- Step C: Send missing rows to the peer nodes

Since local node also knows what peer nodes own, it sends the missing rows to
the peer nodes.

=== How the RPC API looks like

- repair_range_start()

Step A:
- request_sync_boundary()

Step B:
- request_combined_row_hashes()
- reqeust_full_row_hashes()
- request_row_diff()

Step C:
- send_row_diff()

- repair_range_stop()

=== Performance evaluation

We created a cluster of 3 Scylla nodes on AWS using i3.xlarge instance. We
created a keyspace with a replication factor of 3 and inserted 1 billion
rows to each of the 3 nodes. Each node has 241 GiB of data.
We tested 3 cases below.

1) 0% synced: one of the node has zero data. The other two nodes have 1 billion identical rows.

Time to repair:
   old = 87 min
   new = 70 min (rebuild took 50 minutes)
   improvement = 19.54%

2) 100% synced: all of the 3 nodes have 1 billion identical rows.
Time to repair:
   old = 43 min
   new = 24 min
   improvement = 44.18%

3) 99.9% synced: each node has 1 billion identical rows and 1 billion * 0.1% distinct rows.

Time to repair:
   old: 211 min
   new: 44 min
   improvement: 79.15%

Bytes sent on wire for repair:
   old: tx= 162 GiB,  rx = 90 GiB
   new: tx= 1.15 GiB, tx = 0.57 GiB
   improvement: tx = 99.29%, rx = 99.36%

It is worth noting that row level repair sends and receives exactly the
number of rows needed in theory.

In this test case, repair master needs to receives 2 million rows and
sends 4 million rows. Here are the details: Each node has 1 billion *
0.1% distinct rows, that is 1 million rows. So repair master receives 1
million rows from repair slave 1 and 1 million rows from repair slave 2.
Repair master sends 1 million rows from repair master and 1 million rows
received from repair slave 1 to repair slave 2. Repair master sends
sends 1 million rows from repair master and 1 million rows received from
repair slave 2 to repair slave 1.

In the result, we saw the rows on wire were as expected.

tx_row_nr  = 1000505 + 999619 + 1001257 + 998619 (4 shards, the numbers are for each shard) = 4'000'000
rx_row_nr  =  500233 + 500235 +  499559 + 499973 (4 shards, the numbers are for each shard) = 2'000'000

Fixes: #3033

Tests: dtests/repair_additional_test.py
"

* 'asias/row_level_repair_v7' of github.com:cloudius-systems/seastar-dev: (51 commits)
  repair: Enable row level repair
  repair: Add row_level_repair
  repair: Add docs for row level repair
  repair: Add repair_init_messaging_service_handler
  repair: Add repair_meta
  repair: Add repair_writer
  repair: Add repair_reader
  repair: Add repair_row
  repair: Add fragment_hasher
  repair: Add decorated_key_with_hash
  repair: Add get_random_seed
  repair: Add get_common_diff_detect_algorithm
  repair: Add shard_config
  repair: Add suportted_diff_detect_algorithms
  repair: Add repair_stats to repair_info
  repair: Introduce repair_stats
  flat_mutation_reader:  Add make_generating_reader
  storage_service: Introduce ROW_LEVEL_REPAIR feature
  messaging_service: Add RPC verbs for row level repair
  repair: Export the repair logger
  ...
2018-12-25 13:13:00 +02:00
Duarte Nunes
d54ac4961d idl: Add db::view::update_backlog
Add db::view::update_backlog to the newly created view.idl.hh.

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
2018-12-19 22:38:30 +00:00
Asias He
48341a2d4d idl: Add decorated_key support
Needed by the row level repair RPC verbs.
2018-12-12 16:49:01 +08:00
Asias He
1db4e3fd0a idl: Add row_level_diff_detect_algorithm
Needed by the row level repair RPC verbs.
2018-12-12 16:49:01 +08:00
Asias He
ccc706559f idl: Add get_sync_boundary_response
Needed by the row level repair RPC verbs.
2018-12-12 16:49:01 +08:00
Asias He
1173d1dd5a idl: Add repair_sync_boundary
Needed by the row level repair RPC verbs.
2018-12-12 16:49:01 +08:00