forked from thepaul/cassandra-dtest
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathcommitlog_test.py
341 lines (281 loc) · 13.2 KB
/
commitlog_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
import glob
import os
import stat
import subprocess
import time
from cassandra import WriteTimeout
from cassandra.cluster import NoHostAvailable, OperationTimedOut
from ccmlib.common import is_win
from assertions import assert_almost_equal, assert_none, assert_one
from dtest import Tester, debug
from tools import since, rows_to_list
class TestCommitLog(Tester):
""" CommitLog Tests """
def __init__(self, *argv, **kwargs):
kwargs['cluster_options'] = {'start_rpc': 'true'}
super(TestCommitLog, self).__init__(*argv, **kwargs)
self.allow_log_errors = True
def setUp(self):
super(TestCommitLog, self).setUp()
self.cluster.populate(1)
[self.node1] = self.cluster.nodelist()
def tearDown(self):
self._change_commitlog_perms(stat.S_IWRITE | stat.S_IREAD | stat.S_IEXEC)
super(TestCommitLog, self).tearDown()
def prepare(self, configuration={}, create_test_keyspace=True, **kwargs):
conf = {'commitlog_sync_period_in_ms': 1000}
conf.update(configuration)
self.cluster.set_configuration_options(values=conf, **kwargs)
self.cluster.start()
self.session1 = self.patient_cql_connection(self.node1)
if create_test_keyspace:
self.session1.execute("DROP KEYSPACE IF EXISTS ks;")
self.create_ks(self.session1, 'ks', 1)
self.session1.execute("DROP TABLE IF EXISTS test;")
query = """
CREATE TABLE test (
key int primary key,
col1 int
)
"""
self.session1.execute(query)
def _change_commitlog_perms(self, mod):
path = self._get_commitlog_path()
os.chmod(path, mod)
commitlogs = glob.glob(path + '/*')
for commitlog in commitlogs:
os.chmod(commitlog, mod)
def _get_commitlog_path(self):
""" Returns the commitlog path """
return os.path.join(self.node1.get_path(), 'commitlogs')
def _get_commitlog_files(self):
""" Returns the number of commitlog files in the directory """
path = self._get_commitlog_path()
return [os.path.join(path, p) for p in os.listdir(path)]
def _get_commitlog_size(self):
""" Returns the commitlog directory size in MB """
path = self._get_commitlog_path()
cmd_args = ['du', '-m', path]
p = subprocess.Popen(cmd_args, stdout=subprocess.PIPE,
stderr=subprocess.PIPE)
stdout, stderr = p.communicate()
exit_status = p.returncode
self.assertEqual(0, exit_status,
"du exited with a non-zero status: %d" % exit_status)
size = int(stdout.split('\t')[0])
return size
def _segment_size_test(self, segment_size_in_mb, compressed=False):
""" Execute a basic commitlog test and validate the commitlog files """
conf = {'commitlog_segment_size_in_mb': segment_size_in_mb}
if compressed:
conf['commitlog_compression'] = [{'class_name': 'LZ4Compressor'}]
if self.cluster.version() >= "2.1":
conf['memtable_heap_space_in_mb'] = 512
self.prepare(configuration=conf, create_test_keyspace=False)
segment_size = segment_size_in_mb * 1024 * 1024
self.node1.stress(['write', 'n=150000', '-rate', 'threads=25'])
time.sleep(1)
commitlogs = self._get_commitlog_files()
self.assertTrue(len(commitlogs) > 0, "No commit log files were created")
# the most recently-written segment of the commitlog may be smaller
# than the expected size, so we allow exactly one segment to be smaller
smaller_found = False
for i, f in enumerate(commitlogs):
size = os.path.getsize(f)
size_in_mb = int(size / 1024 / 1024)
debug('segment file {} {}; smaller already found: {}'.format(f, size_in_mb, smaller_found))
if size_in_mb < 1 or size < (segment_size * 0.1):
continue # commitlog not yet used
try:
if compressed:
# if compression is used, we assume there will be at most a 50% compression ratio
self.assertLess(size, segment_size)
self.assertGreater(size, segment_size / 2)
else:
# if no compression is used, the size will be close to what we expect
assert_almost_equal(size, segment_size, error=0.05)
except AssertionError as e:
# the last segment may be smaller
if not smaller_found:
self.assertLessEqual(size, segment_size)
smaller_found = True
else:
raise e
def _provoke_commitlog_failure(self):
""" Provoke the commitlog failure """
# Test things are ok at this point
self.session1.execute("""
INSERT INTO test (key, col1) VALUES (1, 1);
""")
assert_one(
self.session1,
"SELECT * FROM test where key=1;",
[1, 1]
)
self._change_commitlog_perms(0)
if self.cluster.version() < "2.1":
with open(os.devnull, 'w') as devnull:
self.node1.stress(['--num-keys=1000000', '-S', '1000'],
stdout=devnull, stderr=subprocess.STDOUT)
else:
with open(os.devnull, 'w') as devnull:
self.node1.stress(['write', 'n=1M', '-col', 'size=FIXED(1000)', '-rate', 'threads=25'],
stdout=devnull, stderr=subprocess.STDOUT)
def test_commitlog_replay_on_startup(self):
""" Test commit log replay """
node1 = self.node1
node1.set_configuration_options(batch_commitlog=True)
node1.start()
debug("Insert data")
session = self.patient_cql_connection(node1)
self.create_ks(session, 'Test', 1)
session.execute("""
CREATE TABLE users (
user_name varchar PRIMARY KEY,
password varchar,
gender varchar,
state varchar,
birth_year bigint
);
""")
session.execute("INSERT INTO Test. users (user_name, password, gender, state, birth_year) "
"VALUES('gandalf', 'p@$$', 'male', 'WA', 1955);")
debug("Verify data is present")
session = self.patient_cql_connection(node1)
res = session.execute("SELECT * FROM Test. users")
self.assertItemsEqual(rows_to_list(res),
[[u'gandalf', 1955, u'male', u'p@$$', u'WA']])
debug("Stop node abruptly")
node1.stop(gently=False)
debug("Verify commitlog was written before abrupt stop")
commitlog_dir = os.path.join(node1.get_path(), 'commitlogs')
commitlog_files = os.listdir(commitlog_dir)
self.assertTrue(len(commitlog_files) > 0)
debug("Verify no SSTables were flushed before abrupt stop")
data_dir = os.path.join(node1.get_path(), 'data')
cf_id = [s for s in os.listdir(os.path.join(data_dir, "test")) if s.startswith("users")][0]
cf_data_dir = glob.glob("{data_dir}/test/{cf_id}".format(**locals()))[0]
cf_data_dir_files = os.listdir(cf_data_dir)
if "backups" in cf_data_dir_files:
cf_data_dir_files.remove("backups")
self.assertEqual(0, len(cf_data_dir_files))
debug("Verify commit log was replayed on startup")
node1.start()
node1.watch_log_for("Log replay complete")
# Here we verify there was more than 0 replayed mutations
zero_replays = node1.grep_log(" 0 replayed mutations")
self.assertEqual(0, len(zero_replays))
debug("Make query and ensure data is present")
session = self.patient_cql_connection(node1)
res = session.execute("SELECT * FROM Test. users")
self.assertItemsEqual(rows_to_list(res),
[[u'gandalf', 1955, u'male', u'p@$$', u'WA']])
@since('2.1')
def default_segment_size_test(self):
""" Test default commitlog_segment_size_in_mb (32MB) """
self._segment_size_test(32)
@since('2.1')
def small_segment_size_test(self):
""" Test a small commitlog_segment_size_in_mb (5MB) """
self._segment_size_test(5)
@since('2.2')
def default_compressed_segment_size_test(self):
""" Test default compressed commitlog_segment_size_in_mb (32MB) """
self._segment_size_test(32, compressed=True)
@since('2.2')
def small_compressed_segment_size_test(self):
""" Test a small compressed commitlog_segment_size_in_mb (5MB) """
self._segment_size_test(5, compressed=True)
def stop_failure_policy_test(self):
""" Test the stop commitlog failure policy (default one) """
self.prepare()
self._provoke_commitlog_failure()
failure = self.node1.grep_log("Failed .+ commit log segments. Commit disk failure policy is stop; terminating thread")
debug(failure)
self.assertTrue(failure, "Cannot find the commitlog failure message in logs")
self.assertTrue(self.node1.is_running(), "Node1 should still be running")
# Cannot write anymore after the failure
with self.assertRaises(NoHostAvailable):
self.session1.execute("""
INSERT INTO test (key, col1) VALUES (2, 2);
""")
# Should not be able to read neither
with self.assertRaises(NoHostAvailable):
self.session1.execute("""
"SELECT * FROM test;"
""")
def stop_commit_failure_policy_test(self):
""" Test the stop_commit commitlog failure policy """
self.prepare(configuration={
'commit_failure_policy': 'stop_commit'
})
self.session1.execute("""
INSERT INTO test (key, col1) VALUES (2, 2);
""")
self._provoke_commitlog_failure()
failure = self.node1.grep_log("Failed .+ commit log segments. Commit disk failure policy is stop_commit; terminating thread")
debug(failure)
self.assertTrue(failure, "Cannot find the commitlog failure message in logs")
self.assertTrue(self.node1.is_running(), "Node1 should still be running")
# Cannot write anymore after the failure
with self.assertRaises((OperationTimedOut, WriteTimeout)):
self.session1.execute("""
INSERT INTO test (key, col1) VALUES (2, 2);
""")
# Should be able to read
assert_one(
self.session1,
"SELECT * FROM test where key=2;",
[2, 2]
)
@since('2.1')
def die_failure_policy_test(self):
""" Test the die commitlog failure policy """
self.prepare(configuration={
'commit_failure_policy': 'die'
})
self._provoke_commitlog_failure()
failure = self.node1.grep_log("ERROR \[COMMIT-LOG-ALLOCATOR\].+JVM state determined to be unstable. Exiting forcefully")
debug(failure)
self.assertTrue(failure, "Cannot find the commitlog failure message in logs")
self.assertFalse(self.node1.is_running(), "Node1 should not be running")
def ignore_failure_policy_test(self):
""" Test the ignore commitlog failure policy """
self.prepare(configuration={
'commit_failure_policy': 'ignore'
})
self._provoke_commitlog_failure()
failure = self.node1.grep_log("ERROR \[COMMIT-LOG-ALLOCATOR\].+Failed .+ commit log segments")
self.assertTrue(failure, "Cannot find the commitlog failure message in logs")
self.assertTrue(self.node1.is_running(), "Node1 should still be running")
# on Windows, we can't delete the segments if they're chmod to 0 so they'll still be available for use by CLSM,
# and we can still create new segments since os.chmod is limited to stat.S_IWRITE and stat.S_IREAD to set files
# as read-only. New mutations will still be allocated and WriteTimeouts will not be raised. It's sufficient that
# we confirm that a) the node isn't dead (stop) and b) the node doesn't terminate the thread (stop_commit)
query = "INSERT INTO test (key, col1) VALUES (2, 2);"
if is_win():
# We expect this to succeed
self.session1.execute(query)
self.assertFalse(self.node1.grep_log("terminating thread"), "thread was terminated but CL error should have been ignored.")
self.assertTrue(self.node1.is_running(), "Node1 should still be running after an ignore error on CL")
else:
with self.assertRaises((OperationTimedOut, WriteTimeout)):
self.session1.execute(query)
# Should not exist
assert_none(self.session1, "SELECT * FROM test where key=2;")
# bring back the node commitlogs
self._change_commitlog_perms(stat.S_IWRITE | stat.S_IREAD | stat.S_IEXEC)
self.session1.execute("""
INSERT INTO test (key, col1) VALUES (3, 3);
""")
assert_one(
self.session1,
"SELECT * FROM test where key=3;",
[3, 3]
)
time.sleep(2)
assert_one(
self.session1,
"SELECT * FROM test where key=2;",
[2, 2]
)