-
Notifications
You must be signed in to change notification settings - Fork 132
/
auth_join_ring_false_test.py
214 lines (168 loc) · 9.36 KB
/
auth_join_ring_false_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
import pytest
from cassandra import AuthenticationFailed, Unauthorized
from cassandra.cluster import NoHostAvailable
from dtest import Tester
since = pytest.mark.since
@since('2.2')
class TestAuth(Tester):
def test_login_existing_node(self):
"""
* Launch a three node cluster
* Restart the third node in `join_ring=false` mode
* Connect as the default user/password
* Verify that default user w/ bad password gives AuthenticationFailed exception
* Verify that bad user gives AuthenticationFailed exception
"""
# also tests default user creation (cassandra/cassandra)
self.prepare(nodes=3)
node1, node2, node3 = self.cluster.nodelist()
node3.stop(wait_other_notice=True)
node3.start(join_ring=False, wait_other_notice=False, wait_for_binary_proto=True)
self.patient_exclusive_cql_connection(node=node3, user='cassandra', password='cassandra')
try:
self.patient_exclusive_cql_connection(node=node3, user='cassandra', password='badpassword')
except NoHostAvailable as e:
assert isinstance(list(e.errors.values())[0], AuthenticationFailed)
try:
self.patient_exclusive_cql_connection(node=node3, user='doesntexist', password='doesntmatter')
except NoHostAvailable as e:
assert isinstance(list(e.errors.values())[0], AuthenticationFailed)
def test_login_new_node(self):
"""
* Launch a two node cluster
* Add a third node in `join_ring=false` mode
* Connect as the default user/password
* Verify that default user w/ bad password gives AuthenticationFailed exception
* Verify that bad user gives AuthenticationFailed exception
"""
# also tests default user creation (cassandra/cassandra)
self.prepare(nodes=2)
node3 = self.cluster.create_node('node3', False,
('127.0.0.3', 9160),
('127.0.0.3', 7000),
'7300', '2002', None,
binary_interface=('127.0.0.3', 9042))
self.cluster.add(node3, False, data_center="dc1")
node3.start(join_ring=False, wait_other_notice=False, wait_for_binary_proto=True)
self.patient_exclusive_cql_connection(node=node3, user='cassandra', password='cassandra')
try:
self.patient_exclusive_cql_connection(node=node3, user='cassandra', password='badpassword')
except NoHostAvailable as e:
assert isinstance(list(e.errors.values())[0], AuthenticationFailed)
try:
self.patient_exclusive_cql_connection(node=node3, user='doesntexist', password='doesntmatter')
except NoHostAvailable as e:
assert isinstance(list(e.errors.values())[0], AuthenticationFailed)
def test_list_users(self):
"""
* Launch a one node cluster
* Connect as the default superuser
* Create two new users, and two new superusers.
* Verify that LIST USERS shows all five users.
* Verify that the correct users are listed as super users.
* Add a second node in `join_ring=false` mode
* Connect (through the non-ring node) as one of the new users, and check that the LIST USERS behavior is also correct there.
"""
self.prepare()
session = self.get_session(user='cassandra', password='cassandra')
session.execute("CREATE USER alex WITH PASSWORD '12345' NOSUPERUSER")
session.execute("CREATE USER bob WITH PASSWORD '12345' SUPERUSER")
session.execute("CREATE USER cathy WITH PASSWORD '12345' NOSUPERUSER")
session.execute("CREATE USER dave WITH PASSWORD '12345' SUPERUSER")
node2 = self.cluster.create_node('node2', False,
('127.0.0.2', 9160),
('127.0.0.2', 7000),
'7200', '2001', None,
binary_interface=('127.0.0.2', 9042))
self.cluster.add(node2, False, data_center="dc1")
node2.start(join_ring=False, wait_other_notice=False, wait_for_binary_proto=True)
self.patient_exclusive_cql_connection(node=node2, user='cassandra', password='cassandra')
session = self.get_session(user='cassandra', password='cassandra')
rows = list(session.execute("LIST USERS"))
assert 5 == len(rows)
# {username: isSuperuser} dict.
users = dict([(r[0], r[1]) for r in rows])
assert users['cassandra']
assert not users['alex']
assert users['bob']
assert not users['cathy']
assert users['dave']
self.get_session(node_idx=1, user='dave', password='12345')
rows = list(session.execute("LIST USERS"))
assert 5 == len(rows)
# {username: isSuperuser} dict.
users = dict([(r[0], r[1]) for r in rows])
assert users['cassandra']
assert not users['alex']
assert users['bob']
assert not users['cathy']
assert users['dave']
def test_modify_and_select_auth(self):
self.prepare()
cassandra = self.get_session(user='cassandra', password='cassandra')
cassandra.execute("CREATE USER cathy WITH PASSWORD '12345'")
cassandra.execute("CREATE KEYSPACE ks WITH replication = {'class':'SimpleStrategy', 'replication_factor':3}")
cassandra.execute("CREATE TABLE ks.cf (id int primary key, val int)")
node2 = self.cluster.create_node('node2', False,
('127.0.0.2', 9160),
('127.0.0.2', 7000),
'7200', '2001', None,
binary_interface=('127.0.0.2', 9042))
self.cluster.add(node2, False, data_center="dc1")
node2.start(join_ring=False, wait_other_notice=False, wait_for_binary_proto=True)
cathy = self.get_session(node_idx=1, user='cathy', password='12345')
self.assert_unauthorized("User cathy has no SELECT permission on <table ks.cf> or any of its parents",
cathy, "SELECT * FROM ks.cf")
node2.stop()
cassandra = self.get_session(user='cassandra', password='cassandra')
cassandra.execute("GRANT SELECT ON ks.cf TO cathy")
node2.start(join_ring=False, wait_other_notice=False, wait_for_binary_proto=True)
cathy = self.get_session(node_idx=1, user='cathy', password='12345')
rows = list(cathy.execute("SELECT * FROM ks.cf"))
assert 0 == len(rows)
self.assert_unauthorized("User cathy has no MODIFY permission on <table ks.cf> or any of its parents",
cathy, "INSERT INTO ks.cf (id, val) VALUES (0, 0)")
self.assert_unauthorized("User cathy has no MODIFY permission on <table ks.cf> or any of its parents",
cathy, "UPDATE ks.cf SET val = 1 WHERE id = 1")
self.assert_unauthorized("User cathy has no MODIFY permission on <table ks.cf> or any of its parents",
cathy, "DELETE FROM ks.cf WHERE id = 1")
self.assert_unauthorized("User cathy has no MODIFY permission on <table ks.cf> or any of its parents",
cathy, "TRUNCATE ks.cf")
node2.stop()
cassandra = self.get_session(user='cassandra', password='cassandra')
cassandra.execute("GRANT MODIFY ON ks.cf TO cathy")
node2.start(join_ring=False, wait_other_notice=False, wait_for_binary_proto=True)
cathy = self.get_session(node_idx=1, user='cathy', password='12345')
cathy.execute("INSERT INTO ks.cf (id, val) VALUES (0, 0)")
cathy.execute("UPDATE ks.cf SET val = 1 WHERE id = 1")
rows = list(cathy.execute("SELECT * FROM ks.cf"))
assert 2 == len(rows)
cathy.execute("DELETE FROM ks.cf WHERE id = 1")
rows = list(cathy.execute("SELECT * FROM ks.cf"))
assert 1 == len(rows)
rows = list(cathy.execute("TRUNCATE ks.cf"))
assert len(rows) == 0
def assert_unauthorized(self, message, session, query):
with pytest.raises(Unauthorized) as cm:
session.execute(query)
assert_regexp_matches(repr(cm._excinfo[1]), message)
def get_session(self, node_idx=0, user=None, password=None):
"""
Connect with a set of credentials to a given node. Connection is not exclusive to that node.
@param node_idx Initial node to connect to
@param user User to connect as
@param password Password to use
@return Session as user, to specified node
"""
node = self.cluster.nodelist()[node_idx]
session = self.patient_cql_connection(node, user=user, password=password)
return session
def prepare(self, nodes=1, roles_expiry=0):
config = {'authenticator': 'org.apache.cassandra.auth.PasswordAuthenticator',
'authorizer': 'org.apache.cassandra.auth.CassandraAuthorizer',
'role_manager': 'org.apache.cassandra.auth.CassandraRoleManager',
'permissions_validity_in_ms': 0,
'roles_validity_in_ms': roles_expiry}
self.cluster.set_configuration_options(values=config)
self.cluster.populate(nodes).start()
self.cluster.wait_for_any_log('Created default superuser', 25)