-
Notifications
You must be signed in to change notification settings - Fork 134
/
configuration_test.py
199 lines (159 loc) · 8.59 KB
/
configuration_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
import os
import logging
import parse
import pytest
import tempfile
from cassandra.concurrent import execute_concurrent_with_args
from dtest import Tester, create_ks
from tools.jmxutils import (JolokiaAgent, make_mbean)
from distutils.version import LooseVersion
logger = logging.getLogger(__name__)
class TestConfiguration(Tester):
def test_compression_chunk_length(self):
""" Verify the setting of compression chunk_length [#3558]"""
cluster = self.cluster
cluster.populate(1).start()
node = cluster.nodelist()[0]
session = self.patient_cql_connection(node)
create_ks(session, 'ks', 1)
create_table_query = "CREATE TABLE test_table (row varchar, name varchar, value int, PRIMARY KEY (row, name));"
if self.cluster.version() >= LooseVersion('5.0'):
alter_chunk_len_query = "ALTER TABLE test_table WITH " \
"compression = {{'class' : 'SnappyCompressor', " \
"'chunk_length_in_kb' : {chunk_length}}};"
else:
alter_chunk_len_query = "ALTER TABLE test_table WITH " \
"compression = {{'sstable_compression' : 'SnappyCompressor', " \
"'chunk_length_kb' : {chunk_length}}};"
session.execute(create_table_query)
session.execute(alter_chunk_len_query.format(chunk_length=32))
self._check_chunk_length(session, 32)
session.execute(alter_chunk_len_query.format(chunk_length=64))
self._check_chunk_length(session, 64)
@pytest.mark.timeout(60*30)
def test_change_durable_writes(self):
"""
@jira_ticket CASSANDRA-9560
Test that changes to the DURABLE_WRITES option on keyspaces is
respected in subsequent writes.
This test starts by writing a dataset to a cluster and asserting that
the commitlogs have been written to. The subsequent test depends on
the assumption that this dataset triggers an fsync.
After checking this assumption, the test destroys the cluster and
creates a fresh one. Then it tests that DURABLE_WRITES is respected by:
- creating a keyspace with DURABLE_WRITES set to false,
- using ALTER KEYSPACE to set its DURABLE_WRITES option to true,
- writing a dataset to this keyspace that is known to trigger a commitlog fsync,
- asserting that the commitlog has grown in size since the data was written.
"""
cluster = self.cluster
cluster.set_batch_commitlog(enabled=True, use_batch_window = cluster.version() < '5.0')
cluster.set_configuration_options(values={'commitlog_segment_size_in_mb': 1})
cluster.populate(1).start()
durable_node = cluster.nodelist()[0]
durable_init_size = commitlog_size(durable_node)
durable_session = self.patient_exclusive_cql_connection(durable_node)
# test assumption that write_to_trigger_fsync actually triggers a commitlog fsync
durable_session.execute("CREATE KEYSPACE ks WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1} "
"AND DURABLE_WRITES = true")
durable_session.execute('CREATE TABLE ks.tab (key int PRIMARY KEY, a int, b int, c int)')
logger.debug('commitlog size diff = ' + str(commitlog_size(durable_node) - durable_init_size))
write_to_trigger_fsync(durable_session, 'ks', 'tab')
logger.debug('commitlog size diff = ' + str(commitlog_size(durable_node) - durable_init_size))
assert commitlog_size(durable_node) > durable_init_size, \
"This test will not work in this environment; write_to_trigger_fsync does not trigger fsync."
durable_session.shutdown()
cluster.stop()
cluster.clear()
cluster.set_batch_commitlog(enabled=True, use_batch_window = cluster.version() < '5.0')
cluster.set_configuration_options(values={'commitlog_segment_size_in_mb': 1})
cluster.start()
node = cluster.nodelist()[0]
session = self.patient_exclusive_cql_connection(node)
# set up a keyspace without durable writes, then alter it to use them
session.execute("CREATE KEYSPACE ks WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1} "
"AND DURABLE_WRITES = false")
session.execute('CREATE TABLE ks.tab (key int PRIMARY KEY, a int, b int, c int)')
init_size = commitlog_size(node)
write_to_trigger_fsync(session, 'ks', 'tab')
assert commitlog_size(node) == init_size, "Commitlog was written with durable writes disabled"
def test_relative_paths(self):
"""
@jira_ticket CASSANDRA-17084
"""
self.cluster.populate(1)
node1 = self.cluster.nodelist()[0]
cassdir = tempfile.mkdtemp()
os.mkdir(os.path.join(cassdir, 'bin'))
os.chdir(cassdir)
node1.set_configuration_options({'commitlog_directory': 'bin/../data/commitlog', 'data_file_directories': ['bin/../data/data']})
self.cluster.start()
self.cluster.stop()
def overlapping_data_folders(self):
"""
@jira_ticket CASSANDRA-10902
"""
self.cluster.populate(1)
node1 = self.cluster.nodelist()[0]
default_path = node1.data_directories()[0]
node1.set_configuration_options({'saved_caches_directory': os.path.join(default_path, 'saved_caches')})
self.cluster.start()
session = self.patient_exclusive_cql_connection(node1)
session.execute("CREATE KEYSPACE ks WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1}")
session.execute("CREATE TABLE ks.tab (key int PRIMARY KEY, a int)")
session.execute("INSERT INTO ks.tab (key, a) VALUES (%s, %s)", [0, 0])
session.execute("SELECT * FROM ks.tab WHERE key = %s", [0])
cache_service = make_mbean('db', type="Caches")
with JolokiaAgent(node1) as jmx:
jmx.execute_method(cache_service, 'saveCaches')
self.cluster.stop()
self.cluster.start()
def _check_chunk_length(self, session, value):
result = session.cluster.metadata.keyspaces['ks'].tables['test_table'].as_cql_query()
# Now extract the param list
params = ''
if self.cluster.version() < '3.0':
if 'sstable_compression' in result:
params = result
else:
if 'compression' in result:
params = result
assert not params == '', "Looking for the string 'sstable_compression', but could not find " \
"it in {str}".format(str=result)
chunk_string = "chunk_length_kb" if self.cluster.version() < '3.0' else "chunk_length_in_kb"
chunk_length = parse.search("'" + chunk_string + "': '{chunk_length:d}'", result).named['chunk_length']
assert chunk_length == value, "Expected chunk_length: {}. We got: {}".format(value, chunk_length)
def write_to_trigger_fsync(session, ks, table):
"""
Given a session, a keyspace name, and a table name, inserts enough values
to trigger an fsync to the commitlog, assuming the cluster's
commitlog_segment_size_in_mb is 1. Assumes the table's columns are
(key int, a int, b int, c int).
"""
"""
From https://github.com/datastax/python-driver/pull/877/files
"Note: in the case that `generators` are used, it is important to ensure the consumers do not
block or attempt further synchronous requests, because no further IO will be processed until
the consumer returns. This may also produce a deadlock in the IO event thread."
"""
execute_concurrent_with_args(session,
session.prepare('INSERT INTO "{ks}"."{table}" (key, a, b, c) '
'VALUES (?, ?, ?, ?)'.format(ks=ks, table=table)),
((x, x + 1, x + 2, x + 3)
for x in range(50000)), concurrency=5)
def commitlog_size(node):
commitlog_size_mbean = make_mbean('metrics', type='CommitLog', name='TotalCommitLogSize')
with JolokiaAgent(node) as jmx:
return jmx.read_attribute(commitlog_size_mbean, 'Value')
# not used but left for debugging
def commitlog_size_nojmx(node):
total = 0
path = os.path.join(node.get_path(), 'commitlogs')
with os.scandir(path) as it:
for entry in it:
if entry.is_file():
total += entry.stat().st_size
elif entry.is_dir():
total += get_dir_size(entry.path)
logger.debug("added {}, {}".format(entry, entry.stat()))
return total