|
5 | 5 |
|
6 | 6 | import pytest |
7 | 7 |
|
8 | | -from kafka.cluster import collect_hosts |
| 8 | +from kafka.cluster import ClusterMetadata, collect_hosts, expand_to_canonical_bootstrap_hosts |
9 | 9 | from kafka.future import Future |
10 | 10 | from kafka.protocol.metadata import MetadataResponse |
11 | 11 | from kafka.structs import TopicPartition |
@@ -241,6 +241,64 @@ def test_collect_hosts__protocol(self): |
241 | 241 | ]) |
242 | 242 |
|
243 | 243 |
|
| 244 | +class TestExpandToCanonicalBootstrapHosts: |
| 245 | + def test_expands_multi_ip_host_to_canonical_names(self): |
| 246 | + addrinfos = [ |
| 247 | + (socket.AF_INET, socket.SOCK_STREAM, 0, 'broker-1.kafka.example.com', ('10.0.0.1', 9092)), |
| 248 | + (socket.AF_INET, socket.SOCK_STREAM, 0, 'broker-2.kafka.example.com', ('10.0.0.2', 9092)), |
| 249 | + ] |
| 250 | + with patch('socket.getaddrinfo', return_value=addrinfos): |
| 251 | + expanded = expand_to_canonical_bootstrap_hosts([('kafka.example.com', 9092, socket.AF_UNSPEC)]) |
| 252 | + assert expanded == [ |
| 253 | + ('broker-1.kafka.example.com', 9092, socket.AF_INET), |
| 254 | + ('broker-2.kafka.example.com', 9092, socket.AF_INET), |
| 255 | + ] |
| 256 | + |
| 257 | + def test_deduplicates_canonical_names(self): |
| 258 | + addrinfos = [ |
| 259 | + (socket.AF_INET, socket.SOCK_STREAM, 0, 'broker-1.kafka.example.com', ('10.0.0.1', 9092)), |
| 260 | + (socket.AF_INET, socket.SOCK_STREAM, 0, 'broker-1.kafka.example.com', ('10.0.0.5', 9092)), |
| 261 | + ] |
| 262 | + with patch('socket.getaddrinfo', return_value=addrinfos): |
| 263 | + expanded = expand_to_canonical_bootstrap_hosts([('kafka.example.com', 9092, socket.AF_UNSPEC)]) |
| 264 | + assert expanded == [('broker-1.kafka.example.com', 9092, socket.AF_INET)] |
| 265 | + |
| 266 | + def test_falls_back_to_original_on_resolution_failure(self): |
| 267 | + with patch('socket.getaddrinfo', side_effect=socket.gaierror('no such host')): |
| 268 | + expanded = expand_to_canonical_bootstrap_hosts([('kafka.example.com', 9092, socket.AF_UNSPEC)]) |
| 269 | + assert expanded == [('kafka.example.com', 9092, socket.AF_UNSPEC)] |
| 270 | + |
| 271 | + def test_missing_canonname_falls_back_to_input_host(self): |
| 272 | + addrinfos = [ |
| 273 | + (socket.AF_INET, socket.SOCK_STREAM, 0, '', ('10.0.0.1', 9092)), |
| 274 | + ] |
| 275 | + with patch('socket.getaddrinfo', return_value=addrinfos): |
| 276 | + expanded = expand_to_canonical_bootstrap_hosts([('kafka.example.com', 9092, socket.AF_UNSPEC)]) |
| 277 | + assert expanded == [('kafka.example.com', 9092, socket.AF_INET)] |
| 278 | + |
| 279 | + |
| 280 | +class TestClusterMetadataClientDnsLookup: |
| 281 | + def test_default_does_not_expand_bootstrap(self): |
| 282 | + with patch('socket.getaddrinfo') as mock_gai: |
| 283 | + cluster = ClusterMetadata(bootstrap_servers='kafka.example.com:9092') |
| 284 | + assert mock_gai.call_count == 0 |
| 285 | + nodes = cluster.bootstrap_brokers() |
| 286 | + assert [n.host for n in nodes] == ['kafka.example.com'] |
| 287 | + |
| 288 | + def test_canonical_mode_expands_bootstrap(self): |
| 289 | + addrinfos = [ |
| 290 | + (socket.AF_INET, socket.SOCK_STREAM, 0, 'broker-1.kafka.example.com', ('10.0.0.1', 9092)), |
| 291 | + (socket.AF_INET, socket.SOCK_STREAM, 0, 'broker-2.kafka.example.com', ('10.0.0.2', 9092)), |
| 292 | + ] |
| 293 | + with patch('socket.getaddrinfo', return_value=addrinfos): |
| 294 | + cluster = ClusterMetadata( |
| 295 | + bootstrap_servers='kafka.example.com:9092', |
| 296 | + client_dns_lookup='resolve_canonical_bootstrap_servers_only') |
| 297 | + nodes = cluster.bootstrap_brokers() |
| 298 | + assert sorted(n.host for n in nodes) == [ |
| 299 | + 'broker-1.kafka.example.com', 'broker-2.kafka.example.com'] |
| 300 | + |
| 301 | + |
244 | 302 | class TestClusterMetadataRefresh: |
245 | 303 | def test_request_update_returns_future(self, cluster): |
246 | 304 | f = cluster.request_update() |
|
0 commit comments