Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Cassandra Wiki" for 
change notification.

The "Operations" page has been changed by JonathanEllis:
http://wiki.apache.org/cassandra/Operations?action=diff&rev1=104&rev2=105

Comment:
update repair section

  See below about consistent backups.
  
  === Repairing missing or inconsistent data ===
- Cassandra repairs data in two ways:
+ Cassandra repairs missing data in three ways:
  
-  1. Read Repair: every time a read is performed, Cassandra compares the 
versions at each replica (in the background, if a low consistency was requested 
by the reader to minimize latency), and the newest version is sent to any 
out-of-date replicas.
+  1. HintedHandoff: when a replica does not acknowledge an update, the 
coordinator will store the update and replay it later.
+  1. ReadRepair: when a read is performed, Cassandra compares the versions at 
each replica (in the background, if a low consistency was requested by the 
reader to minimize latency), and the newest version is sent to any out-of-date 
replicas.  By default, Cassandra does this with 10% of all requests; this can 
be configured per-columnfamily with `read_repair_chance` and 
`dclocal_read_repair_chance`.
-  1. Anti-Entropy: when `nodetool repair` is run, Cassandra computes a Merkle 
tree for each range of data on that node, and compares it with the versions on 
other replicas, to catch any out of sync data that hasn't been read recently.  
This is intended to be run infrequently (e.g., weekly) since computing the 
Merkle tree is relatively expensive in disk i/o and CPU, since it scans ALL the 
data on the machine (but it is is very network efficient).
+  1. AntiEntropy: when `nodetool repair` is run, Cassandra computes a Merkle 
tree for each range of data on that node, and compares it with the versions on 
other replicas, to catch any out of sync data that hasn't been read recently.  
This is intended to be run gradually (e.g., continuously over a period of about 
a week) since computing the Merkle tree is relatively expensive in disk i/o and 
CPU, since it scans ALL the data on the machine (but it is is very network 
efficient).
  
- Running `nodetool repair`: Like all nodetool operations in 0.7, repair is 
blocking: it will wait for the repair to finish and then exit.  This may take a 
long time on large data sets.
+ It is safe to run repair against multiple machines at the same time, but to 
minimize the impact on your application workload it is recommended to wait for 
it to complete on one node before invoking it against the next.  Using the 
`--partitioner-range` ("partitioner range") option to repair will repair only 
the range assigned to each node by the partitioner -- i.e., not additional 
ranges added by the replication strategy.  This is recommended for continuous 
repair, since it does no redundant work when performed against different nodes.
  
- It is safe to run repair against multiple machines at the same time, but to 
minimize the impact on your application workload it is recommended to wait for 
it to complete on one node before invoking it against the next.
+ Repair shares the `compaction_throughput_mb_per_sec` limit with compaction -- 
that is, repair's scanning and compaction together will not exceed that limit.  
This keeps repair from negatively impacting your application workload.  If you 
need to get a repair done quickly, you can still minimize the impact on your 
cluster by using the `--with-snapshot` option, which will cause repair to take 
a snapshot and then have pairs of replicas compare merkle trees at a time.  
Thus, if you have three replicas, you will always leave one completely 
unencumbered by repair.
  
  === Frequency of nodetool repair ===
- Unless your application performs no deletes, it is vital that production 
clusters run `nodetool repair` periodically on all nodes in the cluster. The 
hard requirement for repair frequency is the value used for GCGraceSeconds (see 
DistributedDeletes). Running nodetool repair often enough to guarantee that all 
nodes have performed a repair in a given period GCGraceSeconds long, ensures 
that deletes are not "forgotten" in the cluster.
+ Unless your application performs no deletes, it is strongly recommended that 
production clusters run `nodetool repair` periodically on all nodes in the 
cluster. The hard requirement for repair frequency is the value used for 
GCGraceSeconds (see DistributedDeletes). Running nodetool repair often enough 
to guarantee that all nodes have performed a repair in a given period 
GCGraceSeconds long, ensures that deletes are not "forgotten" in the cluster.
  
- Consider how to schedule your repairs. A repair causes additional disk and 
CPU activity on the nodes participating in the repair, and it will typically be 
a good idea to spread repairs out over time so as to minimize the chances of 
repairs running concurrently on many nodes.
+ *IF* your operations team is sufficiently on the ball, you can get by without 
repair as long as you do not have hardware failure -- in that case, 
HintedHandoff is adequate to repair successful updates that some replicas have 
missed.  Hinted handoff is active for `max_hint_window_in_ms` after a replica 
fails.
+ 
+ Full repair or re-bootstrap is necessary to re-replicate data lost to 
hardware failure (see below).
  
  ==== Dealing with the consequences of nodetool repair not running within 
GCGraceSeconds ====
  If `nodetool repair` has not been run often enough to the point that 
GCGraceSeconds has passed, you risk forgotten deletes (see DistributedDeletes). 
In addition to data popping up that has been deleted, you may see 
inconsistencies in data return from different nodes that will not self-heal by 
read-repair or further `nodetool repair`. Some further details on this latter 
effect is documented in 
[[https://issues.apache.org/jira/browse/CASSANDRA-1316|CASSANDRA-1316]].
@@ -169, +172 @@

   1. Yet another option, that will result in more forgotten deletes than the 
previous suggestion but is easier to do, is to ensure 'nodetool repair' has 
been run on all nodes, and then perform a compaction to expire toombstones. 
Following this, read-repair and regular `nodetool repair` should cause the 
cluster to converge.
  
  === Handling failure ===
- If a node goes down and comes back up, the ordinary repair mechanisms will be 
adequate to deal with any inconsistent data.  Remember though that if a node 
misses updates and is not repaired for longer than your configured 
GCGraceSeconds (default: 10 days), it could have missed remove operations 
permanently.  Unless your application performs no removes, you should wipe its 
data directory, re-bootstrap it, and removetoken its old entry in the ring (see 
below).
+ If a node fails and subsequently recovers, the ordinary repair mechanisms 
will be adequate to deal with any inconsistent data.  Remember though that if a 
node misses updates and is not repaired for longer than your configured 
GCGraceSeconds (default: 10 days), it could have missed remove operations 
permanently.  Unless your application performs no removes, you should wipe its 
data directory, re-bootstrap it, and removetoken its old entry in the ring (see 
below).
  
  If a node goes down entirely, then you have two options:
  
-  1. (Recommended approach) Bring up the replacement node with a new IP 
address, Set initial token to `(failure node's token) - 1` and !AutoBootstrap 
set to true in cassandra.yaml (storage-conf.xml for 0.6 or earlier). This will 
place the replacement node in front of the failure node. Then the bootstrap 
process begins. While this process runs, the node will not receive reads until 
finished. Once this process is finished on the replacement node, run `nodetool 
removetoken` once, supplying the token of the dead node, and `nodetool cleanup` 
on each node. You can obtain the dead node's token by running `nodetool ring` 
on any live node, unless there was some kind of outage, and the others came up 
but not the down one -- in that case, you can retrieve the token from the live 
nodes' system tables.
+  1. (Recommended approach) Bring up the replacement node with a new IP 
address, Set initial token to `(failure node's token) - 1` and !AutoBootstrap 
set to true in cassandra.yaml. This will place the replacement node in front of 
the failure node. Then the bootstrap process begins. While this process runs, 
the node will not receive reads until finished. Once this process is finished 
on the replacement node, run `nodetool removetoken` once, supplying the token 
of the dead node, and `nodetool cleanup` on each node. You can obtain the dead 
node's token by running `nodetool ring` on any live node, unless there was some 
kind of outage, and the others came up but not the down one -- in that case, 
you can retrieve the token from the live nodes' system tables.
  
   1. (Alternative approach) Bring up a replacement node with the same IP and 
token as the old, and run `nodetool repair`. Until the repair process is 
complete, clients reading only from this node may get no data back.  Using a 
higher !ConsistencyLevel on reads will avoid this.
  

Reply via email to