forked from thepaul/cassandra-dtest
-
Notifications
You must be signed in to change notification settings - Fork 0
/
batch_test.py
295 lines (268 loc) · 13.9 KB
/
batch_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
import time
from cassandra import ConsistencyLevel, Timeout, Unavailable
from cassandra.query import SimpleStatement
from assertions import assert_invalid, assert_unavailable
from dtest import Tester, debug
from tools import since
class TestBatch(Tester):
def counter_batch_accepts_counter_mutations_test(self):
""" Test that counter batch accepts counter mutations """
session = self.prepare()
session.execute("""
BEGIN COUNTER BATCH
UPDATE clicks SET total = total + 1 WHERE userid = 1 and url = 'http://foo.com'
UPDATE clicks SET total = total + 1 WHERE userid = 1 and url = 'http://bar.com'
UPDATE clicks SET total = total + 1 WHERE userid = 2 and url = 'http://baz.com'
APPLY BATCH
""")
rows = session.execute("SELECT total FROM clicks")
assert [list(rows[0]), list(rows[1]), list(rows[2])] == [[1], [1], [1]], rows
def counter_batch_rejects_regular_mutations_test(self):
""" Test that counter batch rejects non-counter mutations """
session = self.prepare()
if self.cluster.version() < '2.1':
err = "Only counter mutations are allowed in COUNTER batches"
else:
err = "Cannot include non-counter statement in a counter batch"
assert_invalid(session, """
BEGIN COUNTER BATCH
UPDATE clicks SET total = total + 1 WHERE userid = 1 and url = 'http://foo.com'
UPDATE clicks SET total = total + 1 WHERE userid = 1 and url = 'http://bar.com'
UPDATE clicks SET total = total + 1 WHERE userid = 2 and url = 'http://baz.com'
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
APPLY BATCH
""", matching=err)
def logged_batch_accepts_regular_mutations_test(self):
""" Test that logged batch accepts regular mutations """
session = self.prepare()
session.execute("""
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""")
rows = session.execute("SELECT * FROM users")
res = sorted(rows)
assert [list(res[0]), list(res[1])] == [[0, u'Jack', u'Sparrow'], [1, u'Will', u'Turner']], res
@since('3.0')
def logged_batch_gcgs_below_threshold_single_table_test(self):
""" Test that logged batch accepts regular mutations """
session = self.prepare()
# Single table
session.execute("ALTER TABLE users WITH gc_grace_seconds = 0")
session.execute("""
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""")
node1 = self.cluster.nodelist()[0]
warning = node1.grep_log("Executing a LOGGED BATCH on table \[ks.users\], configured with a "
"gc_grace_seconds of 0. The gc_grace_seconds is used to TTL "
"batchlog entries, so setting gc_grace_seconds too low on tables "
"involved in an atomic batch might cause batchlog entries to expire "
"before being replayed.")
debug(warning)
self.assertEquals(1, len(warning), "Cannot find the gc_grace_seconds warning message.")
@since('3.0')
def logged_batch_gcgs_below_threshold_multi_table_test(self):
""" Test that logged batch accepts regular mutations """
session = self.prepare()
session.execute("ALTER TABLE users WITH gc_grace_seconds = 0")
session.execute("""
CREATE TABLE views (
userid int,
url text,
PRIMARY KEY (userid, url)
) WITH gc_grace_seconds = 0;
""")
session.execute("""
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO views (userid, url) VALUES (1, 'Will')
APPLY BATCH
""")
node1 = self.cluster.nodelist()[0]
warning = node1.grep_log("Executing a LOGGED BATCH on tables \[ks.views, ks.users\], configured with a "
"gc_grace_seconds of 0. The gc_grace_seconds is used to TTL "
"batchlog entries, so setting gc_grace_seconds too low on tables "
"involved in an atomic batch might cause batchlog entries to expire "
"before being replayed.")
debug(warning)
self.assertEquals(1, len(warning), "Cannot find the gc_grace_seconds warning message.")
@since('3.0')
def unlogged_batch_gcgs_below_threshold_should_not_print_warning_test(self):
""" Test that logged batch accepts regular mutations """
session = self.prepare()
session.execute("ALTER TABLE users WITH gc_grace_seconds = 0")
session.execute("""
BEGIN UNLOGGED BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""")
node1 = self.cluster.nodelist()[0]
warning = node1.grep_log("setting a too low gc_grace_seconds on tables involved in an atomic batch")
debug(warning)
self.assertEquals(0, len(warning), "Cannot find the gc_grace_seconds warning message.")
def logged_batch_rejects_counter_mutations_test(self):
""" Test that logged batch rejects counter mutations """
session = self.prepare()
if self.cluster.version() < '2.1':
err = "Counter mutations are only allowed in COUNTER batches"
else:
err = "Cannot include a counter statement in a logged batch"
assert_invalid(session, """
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
UPDATE clicks SET total = total + 1 WHERE userid = 1 and url = 'http://foo.com'
APPLY BATCH
""", matching=err)
def unlogged_batch_accepts_regular_mutations_test(self):
""" Test that unlogged batch accepts regular mutations """
session = self.prepare()
session.execute("""
BEGIN UNLOGGED BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (2, 'Elizabeth', 'Swann')
APPLY BATCH
""")
rows = session.execute("SELECT * FROM users")
res = sorted(rows)
assert [list(res[0]), list(res[1])] == [[0, u'Jack', u'Sparrow'], [2, u'Elizabeth', u'Swann']], res
def unlogged_batch_rejects_counter_mutations_test(self):
""" Test that unlogged batch rejects counter mutations """
session = self.prepare()
if self.cluster.version() < '2.1':
err = "Counter mutations are only allowed in COUNTER batches"
else:
err = "Counter and non-counter mutations cannot exist in the same batch"
assert_invalid(session, """
BEGIN UNLOGGED BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (2, 'Elizabeth', 'Swann')
UPDATE clicks SET total = total + 1 WHERE userid = 1 AND url = 'http://foo.com'
APPLY BATCH
""", matching=err)
def logged_batch_throws_uae_test(self):
""" Test that logged batch throws UAE if there aren't enough live nodes """
session = self.prepare(nodes=3)
[node.stop(wait_other_notice=True) for node in self.cluster.nodelist()[1:]]
session.consistency_level = 'ONE'
assert_unavailable(session.execute, """
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""")
def logged_batch_doesnt_throw_uae_test(self):
""" Test that logged batch DOES NOT throw UAE if there are at least 2 live nodes """
session = self.prepare(nodes=3)
self.cluster.nodelist()[-1].stop(wait_other_notice=True)
query = SimpleStatement("""
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""", consistency_level=ConsistencyLevel.ANY)
session.execute(query)
assert True
def acknowledged_by_batchlog_not_set_when_batchlog_write_fails_test(self):
""" Test that acknowledged_by_batchlog is False if batchlog can't be written """
session = self.prepare(nodes=3, compression=False)
# kill 2 of the 3 nodes (all the batchlog write candidates).
[node.stop(gently=False) for node in self.cluster.nodelist()[1:]]
self.assert_timedout(session, """
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""", ConsistencyLevel.ONE, received_responses=0)
def acknowledged_by_batchlog_set_when_batchlog_write_succeeds_test(self):
""" Test that acknowledged_by_batchlog is True if batchlog can be written """
session = self.prepare(nodes=3, compression=False)
# kill one of the nodes so that batchlog will be written, but the write will fail.
self.cluster.nodelist()[-1].stop(gently=False)
self.assert_timedout(session, """
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""", ConsistencyLevel.THREE, received_responses=2)
def batch_uses_proper_timestamp_test(self):
""" Test that each statement will be executed with provided BATCH timestamp """
session = self.prepare()
session.execute("""
BEGIN BATCH USING TIMESTAMP 1111111111111111
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow')
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""")
rows = session.execute("SELECT id, writetime(firstname), writetime(lastname) FROM users")
res = sorted(rows)
assert [list(res[0]), list(res[1])] == [[0, 1111111111111111, 1111111111111111], [1, 1111111111111111, 1111111111111111]], res
def only_one_timestamp_is_valid_test(self):
""" Test that TIMESTAMP must not be used in the statements within the batch. """
session = self.prepare()
assert_invalid(session, """
BEGIN BATCH USING TIMESTAMP 1111111111111111
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow') USING TIMESTAMP 2
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner')
APPLY BATCH
""", matching="Timestamp must be set either on BATCH or individual statements")
def each_statement_in_batch_uses_proper_timestamp_test(self):
""" Test that each statement will be executed with its own timestamp """
session = self.prepare()
session.execute("""
BEGIN BATCH
INSERT INTO users (id, firstname, lastname) VALUES (0, 'Jack', 'Sparrow') USING TIMESTAMP 1111111111111111
INSERT INTO users (id, firstname, lastname) VALUES (1, 'Will', 'Turner') USING TIMESTAMP 1111111111111112
APPLY BATCH
""")
rows = session.execute("SELECT id, writetime(firstname), writetime(lastname) FROM users")
res = sorted(rows)
assert [list(res[0]), list(res[1])] == [[0, 1111111111111111, 1111111111111111], [1, 1111111111111112, 1111111111111112]], res
def assert_timedout(self, session, query, cl, acknowledged_by=None,
received_responses=None):
try:
statement = SimpleStatement(query, consistency_level=cl)
session.execute(statement, timeout=None)
except Timeout as e:
if received_responses is not None:
msg = "Expecting received_responses to be {}, got: {}".format(
received_responses, e.received_responses,)
self.assertEqual(e.received_responses, received_responses, msg)
except Unavailable as e:
if received_responses is not None:
msg = "Expecting alive_replicas to be {}, got: {}".format(
received_responses, e.alive_replicas,)
self.assertEqual(e.alive_replicas, received_responses, msg)
except Exception as e:
assert False, "Expecting TimedOutException, got:" + str(e)
else:
assert False, "Expecting TimedOutException but no exception was raised"
def prepare(self, nodes=1, compression=True):
if not self.cluster.nodelist():
self.cluster.populate(nodes).start(wait_other_notice=True)
node1 = self.cluster.nodelist()[0]
session = self.patient_cql_connection(node1)
self.create_ks(session, 'ks', nodes)
session.execute("""
CREATE TABLE clicks (
userid int,
url text,
total counter,
PRIMARY KEY (userid, url)
);
""")
session.execute("""
CREATE TABLE users (
id int,
firstname text,
lastname text,
PRIMARY KEY (id)
);
""")
time.sleep(.5)
return session