forked from thepaul/cassandra-dtest
-
Notifications
You must be signed in to change notification settings - Fork 0
/
bootstrap_test.py
432 lines (358 loc) · 16.8 KB
/
bootstrap_test.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
import os
import random
import time
import shutil
import subprocess
import tempfile
import re
from dtest import Tester, debug
from tools import new_node, query_c1c2, since, KillOnBootstrap, InterruptBootstrap
from assertions import assert_almost_equal
from ccmlib.node import NodeError
from cassandra import ConsistencyLevel
from cassandra.concurrent import execute_concurrent_with_args
class TestBootstrap(Tester):
def __init__(self, *args, **kwargs):
kwargs['cluster_options'] = {'start_rpc': 'true'}
# Ignore these log patterns:
self.ignore_log_patterns = [
# This one occurs when trying to send the migration to a
# node that hasn't started yet, and when it does, it gets
# replayed and everything is fine.
r'Can\'t send migration request: node.*is down',
# ignore streaming error during bootstrap
r'Exception encountered during startup',
r'Streaming error occurred'
]
Tester.__init__(self, *args, **kwargs)
self.allow_log_errors = True
def simple_bootstrap_test(self):
cluster = self.cluster
tokens = cluster.balanced_tokens(2)
cluster.set_configuration_options(values={'num_tokens': 1})
debug("[node1, node2] tokens: %r" % (tokens,))
keys = 10000
# Create a single node cluster
cluster.populate(1)
node1 = cluster.nodelist()[0]
node1.set_configuration_options(values={'initial_token': tokens[0]})
cluster.start(wait_other_notice=True)
session = self.patient_cql_connection(node1)
self.create_ks(session, 'ks', 1)
self.create_cf(session, 'cf', columns={'c1': 'text', 'c2': 'text'})
# record the size before inserting any of our own data
empty_size = node1.data_size()
debug("node1 empty size : %s" % float(empty_size))
insert_statement = session.prepare("INSERT INTO ks.cf (key, c1, c2) VALUES (?, 'value1', 'value2')")
execute_concurrent_with_args(session, insert_statement, [['k%d' % k] for k in range(keys)])
node1.flush()
node1.compact()
initial_size = node1.data_size()
debug("node1 size before bootstrapping node2: %s" % float(initial_size))
# Reads inserted data all during the boostrap process. We shouldn't
# get any error
reader = self.go(lambda _: query_c1c2(session, random.randint(0, keys - 1), ConsistencyLevel.ONE))
# Boostraping a new node
node2 = new_node(cluster)
node2.set_configuration_options(values={'initial_token': tokens[1]})
node2.start(wait_for_binary_proto=True)
node2.compact()
reader.check()
node1.cleanup()
debug("node1 size after cleanup: %s" % float(node1.data_size()))
node1.compact()
debug("node1 size after compacting: %s" % float(node1.data_size()))
time.sleep(.5)
reader.check()
debug("node2 size after compacting: %s" % float(node2.data_size()))
size1 = float(node1.data_size())
size2 = float(node2.data_size())
assert_almost_equal(size1, size2, error=0.3)
assert_almost_equal(float(initial_size - empty_size), 2 * (size1 - float(empty_size)))
def read_from_bootstrapped_node_test(self):
"""Test bootstrapped node sees existing data, eg. CASSANDRA-6648"""
cluster = self.cluster
cluster.populate(3)
version = cluster.version()
cluster.start()
node1 = cluster.nodes['node1']
if version < "2.1":
node1.stress(['-n', '10000'])
else:
node1.stress(['write', 'n=10000', '-rate', 'threads=8'])
session = self.patient_cql_connection(node1)
stress_table = 'keyspace1.standard1' if self.cluster.version() >= '2.1' else '"Keyspace1"."Standard1"'
original_rows = list(session.execute("SELECT * FROM %s" % (stress_table,)))
node4 = new_node(cluster)
node4.start(wait_for_binary_proto=True)
session = self.patient_exclusive_cql_connection(node4)
new_rows = list(session.execute("SELECT * FROM %s" % (stress_table,)))
self.assertEquals(original_rows, new_rows)
@since('2.2')
def resumable_bootstrap_test(self):
"""Test resuming bootstrap after data streaming failure"""
cluster = self.cluster
cluster.populate(2).start(wait_other_notice=True)
node1 = cluster.nodes['node1']
node1.stress(['write', 'n=100000', '-schema', 'replication(factor=2)'])
node1.flush()
# kill node1 in the middle of streaming to let it fail
t = InterruptBootstrap(node1)
t.start()
# start bootstrapping node3 and wait for streaming
node3 = new_node(cluster)
node3.set_configuration_options(values={'stream_throughput_outbound_megabits_per_sec': 1})
try:
node3.start()
except NodeError:
pass # node doesn't start as expected
t.join()
# wait for node3 ready to query
node3.watch_log_for("Starting listening for CQL clients")
mark = node3.mark_log()
# check if node3 is still in bootstrap mode
session = self.exclusive_cql_connection(node3)
rows = session.execute("SELECT bootstrapped FROM system.local WHERE key='local'")
assert len(rows) == 1
assert rows[0][0] == 'IN_PROGRESS', rows[0][0]
# bring back node1 and invoke nodetool bootstrap to resume bootstrapping
node1.start(wait_other_notice=True)
node3.nodetool('bootstrap resume')
# check if we skipped already retrieved ranges
node3.watch_log_for("already available. Skipping streaming.")
node3.watch_log_for("Resume complete", from_mark=mark)
rows = session.execute("SELECT bootstrapped FROM system.local WHERE key='local'")
assert rows[0][0] == 'COMPLETED', rows[0][0]
@since('2.2')
def bootstrap_with_reset_bootstrap_state_test(self):
"""Test bootstrap with resetting bootstrap progress"""
cluster = self.cluster
cluster.set_configuration_options(values={'stream_throughput_outbound_megabits_per_sec': 1})
cluster.populate(2).start(wait_other_notice=True)
node1 = cluster.nodes['node1']
node1.stress(['write', 'n=100000', '-schema', 'replication(factor=2)'])
node1.flush()
# kill node1 in the middle of streaming to let it fail
t = InterruptBootstrap(node1)
t.start()
# start bootstrapping node3 and wait for streaming
node3 = new_node(cluster)
try:
node3.start()
except NodeError:
pass # node doesn't start as expected
t.join()
node1.start()
# restart node3 bootstrap with resetting bootstrap progress
node3.stop()
mark = node3.mark_log()
node3.start(jvm_args=["-Dcassandra.reset_bootstrap_progress=true"])
# check if we reset bootstrap state
node3.watch_log_for("Resetting bootstrap progress to start fresh", from_mark=mark)
# wait for node3 ready to query
node3.watch_log_for("Listening for thrift clients...", from_mark=mark)
# check if 2nd bootstrap succeeded
session = self.exclusive_cql_connection(node3)
rows = session.execute("SELECT bootstrapped FROM system.local WHERE key='local'")
assert len(rows) == 1
assert rows[0][0] == 'COMPLETED', rows[0][0]
def manual_bootstrap_test(self):
"""Test adding a new node and bootstrappig it manually. No auto_bootstrap.
This test also verify that all data are OK after the addition of the new node.
eg. CASSANDRA-9022
"""
cluster = self.cluster
cluster.populate(2).start(wait_other_notice=True)
(node1, node2) = cluster.nodelist()
if cluster.version() < "2.1":
node1.stress(['-o', 'insert', '-n', '1000', '-l', '2', '-t', '1'])
else:
node1.stress(['write', 'n=1000', '-schema', 'replication(factor=2)',
'-rate', 'threads=1', '-pop', 'dist=UNIFORM(1..1000)'])
session = self.patient_exclusive_cql_connection(node2)
if cluster.version() < "2.1":
stress_table = '"Keyspace1"."Standard1"'
else:
stress_table = 'keyspace1.standard1'
original_rows = list(session.execute("SELECT * FROM %s" % stress_table))
# Add a new node
node3 = new_node(cluster, bootstrap=False)
node3.start(wait_for_binary_proto=True)
node3.repair()
node1.cleanup()
current_rows = list(session.execute("SELECT * FROM %s" % stress_table))
self.assertEquals(original_rows, current_rows)
def local_quorum_bootstrap_test(self):
"""Test that CL local_quorum works while a node is bootstrapping. CASSANDRA-8058"""
cluster = self.cluster
cluster.populate([1, 1])
version = cluster.version()
cluster.start()
node1 = cluster.nodes['node1']
if version < "2.1":
node1.stress(['-n', '2000000', '-t', '50', '-S', '100',
'--replication-strategy', 'NetworkTopologyStrategy',
'--strategy-properties', 'dc1:1,dc2:1'])
else:
yaml_config = """
# Create the keyspace and table
keyspace: keyspace1
keyspace_definition: |
CREATE KEYSPACE keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'dc1': 1, 'dc2': 1};
table: users
table_definition:
CREATE TABLE users (
username text,
first_name text,
last_name text,
email text,
PRIMARY KEY(username)
) WITH compaction = {'class':'SizeTieredCompactionStrategy'};
insert:
partitions: fixed(1)
batchtype: UNLOGGED
queries:
read:
cql: select * from users where username = ?
fields: samerow
"""
stress_config = tempfile.NamedTemporaryFile(mode='w+', delete=False)
stress_config.write(yaml_config)
stress_config.close()
node1.stress(['user', 'profile=' + stress_config.name, 'n=2000000',
'ops(insert=1)', '-rate', 'threads=50'])
node3 = new_node(cluster, data_center='dc2')
node3.start(no_wait=True)
time.sleep(3)
with tempfile.TemporaryFile(mode='w+') as tmpfile:
if version < "2.1":
node1.stress(['-o', 'insert', '-n', '500000', '-t', '5', '-e', 'LOCAL_QUORUM', '-K', '2'],
stdout=tmpfile, stderr=subprocess.STDOUT)
else:
node1.stress(['user', 'profile=' + stress_config.name, 'ops(insert=1)',
'n=500000', 'cl=LOCAL_QUORUM',
'-rate', 'threads=5',
'-errors', 'retries=2'],
stdout=tmpfile, stderr=subprocess.STDOUT)
os.unlink(stress_config.name)
tmpfile.seek(0)
output = tmpfile.read()
debug(output)
regex = re.compile("Operation.+error inserting key.+Exception")
failure = regex.search(output)
self.assertIsNone(failure, "Error during stress while bootstrapping")
def shutdown_wiped_node_cannot_join_test(self):
self._wiped_node_cannot_join_test(gently=True)
def killed_wiped_node_cannot_join_test(self):
self._wiped_node_cannot_join_test(gently=False)
def _wiped_node_cannot_join_test(self, gently):
"""
@jira_ticket CASSANDRA-9765
Test that if we stop a node and wipe its data then the node cannot join
when it is not a seed. Test both a nice shutdown or a forced shutdown, via
the gently parameter.
"""
cluster = self.cluster
cluster.populate(3)
cluster.start(wait_for_binary_proto=True)
version = cluster.version()
stress_table = 'keyspace1.standard1' if self.cluster.version() >= '2.1' else '"Keyspace1"."Standard1"'
# write some data
node1 = cluster.nodelist()[0]
if version < "2.1":
node1.stress(['-n', '10000'])
else:
node1.stress(['write', 'n=10000', '-rate', 'threads=8'])
session = self.patient_cql_connection(node1)
original_rows = list(session.execute("SELECT * FROM {}".format(stress_table,)))
# Add a new node, bootstrap=True ensures that it is not a seed
node2 = new_node(cluster, bootstrap=True)
node2.start(wait_for_binary_proto=True)
session = self.patient_cql_connection(node2)
self.assertEquals(original_rows, list(session.execute("SELECT * FROM {}".format(stress_table,))))
# Stop the new node and wipe its data
node2.stop(gently=gently)
data_dir = os.path.join(node2.get_path(), 'data')
commitlog_dir = os.path.join(node2.get_path(), 'commitlogs')
debug("Deleting {}".format(data_dir))
shutil.rmtree(data_dir)
shutil.rmtree(commitlog_dir)
# Now start it, it should not be allowed to join.
mark = node2.mark_log()
node2.start(no_wait=True)
node2.watch_log_for("A node with address /127.0.0.4 already exists, cancelling join", from_mark=mark)
def decommissioned_wiped_node_can_join_test(self):
"""
@jira_ticket CASSANDRA-9765
Test that if we decommission a node and then wipe its data, it can join the cluster.
"""
cluster = self.cluster
cluster.set_log_level('TRACE')
cluster.populate(3)
cluster.start(wait_for_binary_proto=True)
version = cluster.version()
stress_table = 'keyspace1.standard1' if self.cluster.version() >= '2.1' else '"Keyspace1"."Standard1"'
# write some data
node1 = cluster.nodelist()[0]
if version < "2.1":
node1.stress(['-n', '10000'])
else:
node1.stress(['write', 'n=10000', '-rate', 'threads=8'])
session = self.patient_cql_connection(node1)
original_rows = list(session.execute("SELECT * FROM {}".format(stress_table,)))
# Add a new node, bootstrap=True ensures that it is not a seed
node2 = new_node(cluster, bootstrap=True)
node2.start(wait_for_binary_proto=True)
session = self.patient_cql_connection(node2)
self.assertEquals(original_rows, list(session.execute("SELECT * FROM {}".format(stress_table,))))
session.shutdown() # Ensure all sockets to node2 are released
# Decommision the new node and wipe its data
node2.decommission()
node2.stop(gently=False)
data_dir = os.path.join(node2.get_path(), 'data')
commitlog_dir = os.path.join(node2.get_path(), 'commitlogs')
debug("Deleting {}".format(data_dir))
shutil.rmtree(data_dir)
shutil.rmtree(commitlog_dir)
# Now start it, it should be allowed to join
mark = node2.mark_log()
node2.start(wait_other_notice=True)
node2.watch_log_for("JOINING:", from_mark=mark)
def failed_bootstap_wiped_node_can_join_test(self):
"""
@jira_ticket CASSANDRA-9765
Test that if a node fails to bootstrap, it can join the cluster even if the data is wiped.
"""
cluster = self.cluster
cluster.populate(1)
cluster.start(wait_for_binary_proto=True)
version = cluster.version()
stress_table = 'keyspace1.standard1' if self.cluster.version() >= '2.1' else '"Keyspace1"."Standard1"'
# write some data, enough for the bootstrap to fail later on
node1 = cluster.nodelist()[0]
if version < "2.1":
node1.stress(['-n', '100000'])
else:
node1.stress(['write', 'n=100000', '-rate', 'threads=8'])
node1.flush()
session = self.patient_cql_connection(node1)
original_rows = list(session.execute("SELECT * FROM {}".format(stress_table,)))
# Add a new node, bootstrap=True ensures that it is not a seed
node2 = new_node(cluster, bootstrap=True)
node2.set_configuration_options(values={'stream_throughput_outbound_megabits_per_sec': 1})
# kill node2 in the middle of bootstrap
t = KillOnBootstrap(node2)
t.start()
node2.start()
t.join()
self.assertFalse(node2.is_running())
# wipe any data for node2
data_dir = os.path.join(node2.get_path(), 'data')
commitlog_dir = os.path.join(node2.get_path(), 'commitlogs')
debug("Deleting {}".format(data_dir))
shutil.rmtree(data_dir)
shutil.rmtree(commitlog_dir)
# Now start it again, it should be allowed to join
mark = node2.mark_log()
node2.start(wait_other_notice=True)
node2.watch_log_for("JOINING:", from_mark=mark)