From 6047d790a32ef5a65d4d6b029f673ce53c3d4141 Mon Sep 17 00:00:00 2001 From: Jim Rollenhagen Date: Mon, 1 Aug 2016 15:49:51 +0000 Subject: [PATCH] Ironic: allow multiple compute services This lifts some hash ring code from ironic (to be put into oslo soon), to be used to do consistent hashing of ironic nodes among multiple nova-compute services. The hash ring is used within the driver itself, and is refreshed at each resource tracker. get_available_nodes() will now return a subset of nodes, determined by the following rules: * any node with an instance managed by the compute service * any node that is mapped to the compute service on the hash ring * no nodes with instances managed by another compute service The virt driver finds all compute services that are running the ironic driver by joining the services table and the compute_nodes table. Since there won't be any records in the compute_nodes table for a service that is starting for the first time, the virt driver also adds its own compute service into this list. The list of all hostnames in this list is what is used to instantiate the hash ring. As nova-compute services are brought up or down, the ring will re-balance. It's important to note that this re-balance does not occur at the same time on all compute services, so for some amount of time, an ironic node may be managed by more than one compute service. In other words, there may be two compute_nodes records for a single ironic node, with a different host value. For scheduling purposes, this is okay, because either compute service is capable of actually spawning an instance on the node (because the ironic service doesn't know about this hashing). This will cause capacity reporting (e.g. nova hypervisor-stats) to over-report capacity for this time. Once all compute services in the cluster have done a resource tracker run and re-balanced the hash ring, this will be back to normal. It's also important to note that, due to the way nodes with instances are handled, if an instance is deleted while the compute service is down, that node will be removed from the compute_nodes table when the service comes back up (as each service will see an instance on the node object, and assume another compute service manages that instance). The ironic node will remain active and orphaned. Once the periodic task to reap deleted instances runs, the ironic node will be torn down and the node will again be reported in the compute_nodes table. It's all very eventually consistent, with a potentially long time to eventual. There's no configuration to enable this mode; it's always running. The code is exercised (but simple) when running with one compute service; spinning up more invokes the hard bits. As such, the release note for this change clarifies that this feature is new and untested for running with multiple compute services. Implements: blueprint ironic-multiple-compute-hosts Change-Id: I852f62b29f1faedf7ff19b42bbfb966f61d95c6e --- nova/hash_ring.py | 134 +++++++++++ nova/tests/unit/test_hash_ring.py | 145 ++++++++++++ nova/tests/unit/virt/ironic/test_driver.py | 209 +++++++++++++++++- nova/virt/ironic/driver.py | 38 +++- ...nic-driver-hash-ring-7d763d87b9236e5d.yaml | 33 +++ 5 files changed, 551 insertions(+), 8 deletions(-) create mode 100644 nova/hash_ring.py create mode 100644 nova/tests/unit/test_hash_ring.py create mode 100644 releasenotes/notes/ironic-driver-hash-ring-7d763d87b9236e5d.yaml diff --git a/nova/hash_ring.py b/nova/hash_ring.py new file mode 100644 index 000000000000..f107d5eda721 --- /dev/null +++ b/nova/hash_ring.py @@ -0,0 +1,134 @@ +# Copyright 2013 Hewlett-Packard Development Company, L.P. +# All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. You may obtain +# a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +# License for the specific language governing permissions and limitations +# under the License. + +import bisect +import hashlib + +import six + +from nova import exception +from nova.i18n import _ + + +# NOTE(jroll) these constants will be config options in Ocata, when the hash +# ring code is in oslo. +# Number of partitions per service is 2^PARTITION_EXPONENT. +# 5 should be fine for most deployments, as an experimental feature. +PARTITION_EXPONENT = 5 +# This should always be 1 in nova, as two compute daemons handling the same +# node should not be possible. +DISTRIBUTION_REPLICAS = 1 + + +class HashRing(object): + """A stable hash ring. + + We map item N to a host Y based on the closest lower hash: + + - hash(item) -> partition + - hash(host) -> divider + - closest lower divider is the host to use + - we hash each host many times to spread load more finely + as otherwise adding a host gets (on average) 50% of the load of + just one other host assigned to it. + """ + + def __init__(self, hosts): + """Create a new hash ring across the specified hosts. + + :param hosts: an iterable of hosts which will be mapped. + """ + replicas = DISTRIBUTION_REPLICAS + + try: + self.hosts = set(hosts) + self.replicas = replicas if replicas <= len(hosts) else len(hosts) + except TypeError: + raise exception.Invalid( + _("Invalid hosts supplied when building HashRing.")) + + self._host_hashes = {} + for host in hosts: + key = str(host).encode('utf8') + key_hash = hashlib.md5(key) + for p in range(2 ** PARTITION_EXPONENT): + key_hash.update(key) + hashed_key = self._hash2int(key_hash) + self._host_hashes[hashed_key] = host + # Gather the (possibly colliding) resulting hashes into a bisectable + # list. + self._partitions = sorted(self._host_hashes.keys()) + + def _hash2int(self, key_hash): + """Convert the given hash's digest to a numerical value for the ring. + + :returns: An integer equivalent value of the digest. + """ + return int(key_hash.hexdigest(), 16) + + def _get_partition(self, data): + try: + if six.PY3 and data is not None: + data = data.encode('utf-8') + key_hash = hashlib.md5(data) + hashed_key = self._hash2int(key_hash) + position = bisect.bisect(self._partitions, hashed_key) + return position if position < len(self._partitions) else 0 + except TypeError: + raise exception.Invalid( + _("Invalid data supplied to HashRing.get_hosts.")) + + def get_hosts(self, data, ignore_hosts=None): + """Get the list of hosts which the supplied data maps onto. + + :param data: A string identifier to be mapped across the ring. + :param ignore_hosts: A list of hosts to skip when performing the hash. + Useful to temporarily skip down hosts without + performing a full rebalance. + Default: None. + :returns: a list of hosts. + The length of this list depends on the number of replicas + this `HashRing` was created with. It may be less than this + if ignore_hosts is not None. + """ + hosts = [] + if ignore_hosts is None: + ignore_hosts = set() + else: + ignore_hosts = set(ignore_hosts) + ignore_hosts.intersection_update(self.hosts) + partition = self._get_partition(data) + for replica in range(0, self.replicas): + if len(hosts) + len(ignore_hosts) == len(self.hosts): + # prevent infinite loop - cannot allocate more fallbacks. + break + # Linear probing: partition N, then N+1 etc. + host = self._get_host(partition) + while host in hosts or host in ignore_hosts: + partition += 1 + if partition >= len(self._partitions): + partition = 0 + host = self._get_host(partition) + hosts.append(host) + return hosts + + def _get_host(self, partition): + """Find what host is serving a partition. + + :param partition: The index of the partition in the partition map. + e.g. 0 is the first partition, 1 is the second. + :return: The host object the ring was constructed with. + """ + return self._host_hashes[self._partitions[partition]] diff --git a/nova/tests/unit/test_hash_ring.py b/nova/tests/unit/test_hash_ring.py new file mode 100644 index 000000000000..0a84c11bb992 --- /dev/null +++ b/nova/tests/unit/test_hash_ring.py @@ -0,0 +1,145 @@ +# Copyright 2013 Hewlett-Packard Development Company, L.P. +# All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. You may obtain +# a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +# License for the specific language governing permissions and limitations +# under the License. + +import hashlib + +import mock +from testtools import matchers + +from nova import exception +from nova import hash_ring +from nova import test + + +class HashRingTestCase(test.TestCase): + + # NOTE(deva): the mapping used in these tests is as follows: + # if hosts = [foo, bar]: + # fake -> foo, bar + # if hosts = [foo, bar, baz]: + # fake -> foo, bar, baz + # fake-again -> bar, baz, foo + + @mock.patch.object(hashlib, 'md5', autospec=True) + def test__hash2int_returns_int(self, mock_md5): + r1 = 32 * 'a' + r2 = 32 * 'b' + # 2**PARTITION_EXPONENT calls to md5.update per host + # PARTITION_EXPONENT is currently always 5, so 32 calls each here + mock_md5.return_value.hexdigest.side_effect = [r1] * 32 + [r2] * 32 + + hosts = ['foo', 'bar'] + ring = hash_ring.HashRing(hosts) + + self.assertIn(int(r1, 16), ring._host_hashes) + self.assertIn(int(r2, 16), ring._host_hashes) + + def test_create_ring(self): + hosts = ['foo', 'bar'] + ring = hash_ring.HashRing(hosts) + self.assertEqual(set(hosts), ring.hosts) + self.assertEqual(1, ring.replicas) + self.assertEqual(2 ** 5 * 2, len(ring._partitions)) + + def test_distribution_one_replica(self): + hosts = ['foo', 'bar', 'baz'] + ring = hash_ring.HashRing(hosts) + fake_1_hosts = ring.get_hosts('fake') + fake_2_hosts = ring.get_hosts('fake-again') + # We should have one hosts for each thing + self.assertThat(fake_1_hosts, matchers.HasLength(1)) + self.assertThat(fake_2_hosts, matchers.HasLength(1)) + # And they must not be the same answers even on this simple data. + self.assertNotEqual(fake_1_hosts, fake_2_hosts) + + def test_ignore_hosts(self): + hosts = ['foo', 'bar', 'baz'] + ring = hash_ring.HashRing(hosts) + equals_bar_or_baz = matchers.MatchesAny( + matchers.Equals(['bar']), + matchers.Equals(['baz'])) + self.assertThat( + ring.get_hosts('fake', ignore_hosts=['foo']), + equals_bar_or_baz) + self.assertThat( + ring.get_hosts('fake', ignore_hosts=['foo', 'bar']), + equals_bar_or_baz) + self.assertEqual([], ring.get_hosts('fake', ignore_hosts=hosts)) + + def _compare_rings(self, nodes, conductors, ring, + new_conductors, new_ring): + delta = {} + mapping = {'node': ring.get_hosts(node)[0] for node in nodes} + new_mapping = {'node': new_ring.get_hosts(node)[0] for node in nodes} + + for key, old in mapping.items(): + new = new_mapping.get(key, None) + if new != old: + delta[key] = (old, new) + return delta + + def test_rebalance_stability_join(self): + num_services = 10 + num_nodes = 10000 + # Adding 1 service to a set of N should move 1/(N+1) of all nodes + # Eg, for a cluster of 10 nodes, adding one should move 1/11, or 9% + # We allow for 1/N to allow for rounding in tests. + redistribution_factor = 1.0 / num_services + + nodes = [str(x) for x in range(num_nodes)] + services = [str(x) for x in range(num_services)] + new_services = services + ['new'] + delta = self._compare_rings( + nodes, services, hash_ring.HashRing(services), + new_services, hash_ring.HashRing(new_services)) + + self.assertLess(len(delta), num_nodes * redistribution_factor) + + def test_rebalance_stability_leave(self): + num_services = 10 + num_nodes = 10000 + # Removing 1 service from a set of N should move 1/(N) of all nodes + # Eg, for a cluster of 10 nodes, removing one should move 1/10, or 10% + # We allow for 1/(N-1) to allow for rounding in tests. + redistribution_factor = 1.0 / (num_services - 1) + + nodes = [str(x) for x in range(num_nodes)] + services = [str(x) for x in range(num_services)] + new_services = services[:] + new_services.pop() + delta = self._compare_rings( + nodes, services, hash_ring.HashRing(services), + new_services, hash_ring.HashRing(new_services)) + + self.assertLess(len(delta), num_nodes * redistribution_factor) + + def test_ignore_non_existent_host(self): + hosts = ['foo', 'bar'] + ring = hash_ring.HashRing(hosts) + self.assertEqual(['foo'], ring.get_hosts('fake', + ignore_hosts=['baz'])) + + def test_create_ring_invalid_data(self): + hosts = None + self.assertRaises(exception.Invalid, + hash_ring.HashRing, + hosts) + + def test_get_hosts_invalid_data(self): + hosts = ['foo', 'bar'] + ring = hash_ring.HashRing(hosts) + self.assertRaises(exception.Invalid, + ring.get_hosts, + None) diff --git a/nova/tests/unit/virt/ironic/test_driver.py b/nova/tests/unit/virt/ironic/test_driver.py index c705af4c8321..e7a2cffae802 100644 --- a/nova/tests/unit/virt/ironic/test_driver.py +++ b/nova/tests/unit/virt/ironic/test_driver.py @@ -29,7 +29,9 @@ from nova.compute import task_states from nova.compute import vm_states from nova import context as nova_context from nova import exception +from nova import hash_ring from nova import objects +from nova import servicegroup from nova import test from nova.tests.unit import fake_instance from nova.tests.unit import utils @@ -48,6 +50,8 @@ CONF = cfg.CONF FAKE_CLIENT = ironic_utils.FakeClient() +SENTINEL = object() + class FakeClientWrapper(cw.IronicClientWrapper): def _get_client(self, retry_on_conflict=True): @@ -79,6 +83,10 @@ def _get_stats(): return {'cpu_arch': 'x86_64'} +def _make_compute_service(hostname): + return objects.Service(host=hostname) + + FAKE_CLIENT_WRAPPER = FakeClientWrapper() @@ -87,7 +95,9 @@ class IronicDriverTestCase(test.NoDBTestCase): @mock.patch.object(cw, 'IronicClientWrapper', lambda *_: FAKE_CLIENT_WRAPPER) - def setUp(self): + @mock.patch.object(ironic_driver.IronicDriver, '_refresh_hash_ring') + @mock.patch.object(servicegroup, 'API', autospec=True) + def setUp(self, mock_sg, mock_hash): super(IronicDriverTestCase, self).setUp() self.driver = ironic_driver.IronicDriver(None) @@ -695,25 +705,38 @@ class IronicDriverTestCase(test.NoDBTestCase): provision_state=ironic_states.AVAILABLE) self.assertFalse(self.driver._node_resources_used(unused_node)) + @mock.patch.object(objects.InstanceList, 'get_uuids_by_host') @mock.patch.object(FAKE_CLIENT.node, 'list') - def test_get_available_nodes(self, mock_list): + def test_get_available_nodes(self, mock_list, mock_gi): + instance = fake_instance.fake_instance_obj(self.ctx, + uuid=self.instance_uuid) + mock_gi.return_value = [instance.uuid] node_dicts = [ # a node in maintenance /w no instance and power OFF {'uuid': uuidutils.generate_uuid(), 'maintenance': True, - 'power_state': ironic_states.POWER_OFF}, - # a node /w instance and power ON + 'power_state': ironic_states.POWER_OFF, + 'expected': True}, + # a node /w instance on this compute daemon and power ON {'uuid': uuidutils.generate_uuid(), 'instance_uuid': self.instance_uuid, - 'power_state': ironic_states.POWER_ON}, + 'power_state': ironic_states.POWER_ON, + 'expected': True}, + # a node /w instance on another compute daemon and power ON + {'uuid': uuidutils.generate_uuid(), + 'instance_uuid': uuidutils.generate_uuid(), + 'power_state': ironic_states.POWER_ON, + 'expected': False}, # a node not in maintenance /w no instance and bad power state {'uuid': uuidutils.generate_uuid(), - 'power_state': ironic_states.ERROR}, + 'power_state': ironic_states.ERROR, + 'expected': True}, ] nodes = [ironic_utils.get_test_node(**n) for n in node_dicts] mock_list.return_value = nodes available_nodes = self.driver.get_available_nodes() - expected_uuids = [n['uuid'] for n in node_dicts] + mock_gi.assert_called_once_with(mock.ANY, CONF.host) + expected_uuids = [n['uuid'] for n in node_dicts if n['expected']] self.assertEqual(sorted(expected_uuids), sorted(available_nodes)) @mock.patch.object(FAKE_CLIENT.node, 'get') @@ -1696,3 +1719,175 @@ class IronicDriverGenerateConfigDriveTestCase(test.NoDBTestCase): mock_cd_builder.assert_called_once_with(instance_md='fake-instance') mock_instance_meta.assert_called_once_with(self.instance, network_info=self.network_info, extra_md={}, content=None) + + +class HashRingTestCase(test.NoDBTestCase): + @mock.patch.object(servicegroup, 'API', autospec=True) + def setUp(self, mock_sg): + super(HashRingTestCase, self).setUp() + + self.driver = ironic_driver.IronicDriver(None) + self.driver.virtapi = fake.FakeVirtAPI() + self.ctx = nova_context.get_admin_context() + self.mock_is_up = ( + self.driver.servicegroup_api.service_is_up) + + @mock.patch.object(ironic_driver.IronicDriver, '_refresh_hash_ring') + def test_hash_ring_refreshed_on_init(self, mock_hr): + ironic_driver.IronicDriver(None) + mock_hr.assert_called_once_with(mock.ANY) + + @mock.patch.object(hash_ring, 'HashRing') + @mock.patch.object(objects.ServiceList, 'get_all_computes_by_hv_type') + def _test__refresh_hash_ring(self, services, expected_hosts, mock_services, + mock_hash_ring): + services = [_make_compute_service(host) for host in services] + is_up_calls = [mock.call(svc) for svc in services] + self.flags(host='host1') + mock_services.return_value = services + mock_hash_ring.return_value = SENTINEL + + self.driver._refresh_hash_ring(self.ctx) + + mock_services.assert_called_once_with( + mock.ANY, self.driver._get_hypervisor_type()) + mock_hash_ring.assert_called_once_with(expected_hosts) + self.assertEqual(SENTINEL, self.driver.hash_ring) + self.mock_is_up.assert_has_calls(is_up_calls) + + def test__refresh_hash_ring_one_compute(self): + services = ['host1'] + expected_hosts = {'host1'} + self.mock_is_up.return_value = True + self._test__refresh_hash_ring(services, expected_hosts) + + def test__refresh_hash_ring_many_computes(self): + services = ['host1', 'host2', 'host3'] + expected_hosts = {'host1', 'host2', 'host3'} + self.mock_is_up.return_value = True + self._test__refresh_hash_ring(services, expected_hosts) + + def test__refresh_hash_ring_one_compute_new_compute(self): + services = [] + expected_hosts = {'host1'} + self.mock_is_up.return_value = True + self._test__refresh_hash_ring(services, expected_hosts) + + def test__refresh_hash_ring_many_computes_new_compute(self): + services = ['host2', 'host3'] + expected_hosts = {'host1', 'host2', 'host3'} + self.mock_is_up.return_value = True + self._test__refresh_hash_ring(services, expected_hosts) + + def test__refresh_hash_ring_some_computes_down(self): + services = ['host1', 'host2', 'host3', 'host4'] + expected_hosts = {'host1', 'host2', 'host4'} + self.mock_is_up.side_effect = [True, True, False, True] + self._test__refresh_hash_ring(services, expected_hosts) + + +class NodeCacheTestCase(test.NoDBTestCase): + def setUp(self): + super(NodeCacheTestCase, self).setUp() + + self.driver = ironic_driver.IronicDriver(None) + self.driver.virtapi = fake.FakeVirtAPI() + self.ctx = nova_context.get_admin_context() + + self.host = 'host1' + self.flags(host=self.host) + + @mock.patch.object(ironic_driver.IronicDriver, '_refresh_hash_ring') + @mock.patch.object(hash_ring.HashRing, 'get_hosts') + @mock.patch.object(ironic_driver.IronicDriver, '_get_node_list') + @mock.patch.object(objects.InstanceList, 'get_uuids_by_host') + def _test__refresh_cache(self, instances, nodes, hosts, mock_instances, + mock_nodes, mock_hosts, mock_hash_ring): + mock_instances.return_value = instances + mock_nodes.return_value = nodes + mock_hosts.side_effect = hosts + self.driver.node_cache = {} + self.driver.node_cache_time = None + + self.driver._refresh_cache() + + mock_hash_ring.assert_called_once_with(mock.ANY) + mock_instances.assert_called_once_with(mock.ANY, self.host) + mock_nodes.assert_called_once_with(detail=True, limit=0) + self.assertIsNotNone(self.driver.node_cache_time) + + def test__refresh_cache(self): + # normal operation, one compute service + instances = [] + nodes = [ + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ] + hosts = [self.host, self.host, self.host] + + self._test__refresh_cache(instances, nodes, hosts) + + expected_cache = {n.uuid: n for n in nodes} + self.assertEqual(expected_cache, self.driver.node_cache) + + def test__refresh_cache_multiple_services(self): + # normal operation, many compute services + instances = [] + nodes = [ + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ] + hosts = [self.host, 'host2', 'host3'] + + self._test__refresh_cache(instances, nodes, hosts) + + expected_cache = {n.uuid: n for n in nodes[0:1]} + self.assertEqual(expected_cache, self.driver.node_cache) + + def test__refresh_cache_our_instances(self): + # we should manage a node we have an instance for, even if it doesn't + # map to us + instances = [uuidutils.generate_uuid()] + nodes = [ + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=instances[0]), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ] + # only two calls, having the instance will short-circuit the first node + hosts = [{self.host}, {self.host}] + + self._test__refresh_cache(instances, nodes, hosts) + + expected_cache = {n.uuid: n for n in nodes} + self.assertEqual(expected_cache, self.driver.node_cache) + + def test__refresh_cache_their_instances(self): + # we should never manage a node that another compute service has + # an instance for, even if it maps to us + instances = [] + nodes = [ + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=uuidutils.generate_uuid()), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ironic_utils.get_test_node(uuid=uuidutils.generate_uuid(), + instance_uuid=None), + ] + hosts = [self.host, self.host] + + # only two calls, having the instance will short-circuit the first node + self._test__refresh_cache(instances, nodes, hosts) + + expected_cache = {n.uuid: n for n in nodes[1:]} + self.assertEqual(expected_cache, self.driver.node_cache) diff --git a/nova/virt/ironic/driver.py b/nova/virt/ironic/driver.py index f9f7ed9b8e37..4c903496aa24 100644 --- a/nova/virt/ironic/driver.py +++ b/nova/virt/ironic/driver.py @@ -42,11 +42,13 @@ from nova.compute import vm_states import nova.conf from nova import context as nova_context from nova import exception +from nova import hash_ring from nova.i18n import _ from nova.i18n import _LE from nova.i18n import _LI from nova.i18n import _LW from nova import objects +from nova import servicegroup from nova.virt import configdrive from nova.virt import driver as virt_driver from nova.virt import firewall @@ -143,6 +145,8 @@ class IronicDriver(virt_driver.ComputeDriver): default='nova.virt.firewall.NoopFirewallDriver') self.node_cache = {} self.node_cache_time = 0 + self.servicegroup_api = servicegroup.API() + self._refresh_hash_ring(nova_context.get_admin_context()) self.ironicclient = client_wrapper.IronicClientWrapper() @@ -516,12 +520,44 @@ class IronicDriver(virt_driver.ComputeDriver): except ironic.exc.NotFound: return False + def _refresh_hash_ring(self, ctxt): + service_list = objects.ServiceList.get_all_computes_by_hv_type( + ctxt, self._get_hypervisor_type()) + services = set() + for svc in service_list: + is_up = self.servicegroup_api.service_is_up(svc) + if is_up: + services.add(svc.host) + # NOTE(jroll): always make sure this service is in the list, because + # only services that have something registered in the compute_nodes + # table will be here so far, and we might be brand new. + services.add(CONF.host) + + self.hash_ring = hash_ring.HashRing(services) + def _refresh_cache(self): # NOTE(lucasagomes): limit == 0 is an indicator to continue # pagination until there're no more values to be returned. + ctxt = nova_context.get_admin_context() + self._refresh_hash_ring(ctxt) + instances = objects.InstanceList.get_uuids_by_host(ctxt, CONF.host) node_cache = {} + for node in self._get_node_list(detail=True, limit=0): - node_cache[node.uuid] = node + # NOTE(jroll): we always manage the nodes for instances we manage + if node.instance_uuid in instances: + node_cache[node.uuid] = node + + # NOTE(jroll): check if the node matches us in the hash ring, and + # does not have an instance_uuid (which would imply the node has + # an instance managed by another compute service). + # Note that this means nodes with an instance that was deleted in + # nova while the service was down, and not yet reaped, will not be + # reported until the periodic task cleans it up. + elif (node.instance_uuid is None and + CONF.host in self.hash_ring.get_hosts(node.uuid)): + node_cache[node.uuid] = node + self.node_cache = node_cache self.node_cache_time = time.time() diff --git a/releasenotes/notes/ironic-driver-hash-ring-7d763d87b9236e5d.yaml b/releasenotes/notes/ironic-driver-hash-ring-7d763d87b9236e5d.yaml new file mode 100644 index 000000000000..8e25d001f894 --- /dev/null +++ b/releasenotes/notes/ironic-driver-hash-ring-7d763d87b9236e5d.yaml @@ -0,0 +1,33 @@ +--- +features: + - | + Adds a new feature to the ironic virt driver, which allows + multiple nova-compute services to be run simultaneously. This uses + consistent hashing to divide the ironic nodes between the nova-compute + services, with the hash ring being refreshed each time the resource tracker + runs. + + Note that instances will still be owned by the same nova-compute service + for the entire life of the instance, and so the ironic node that instance + is on will also be managed by the same nova-compute service until the node + is deleted. This also means that removing a nova-compute service will + leave instances managed by that service orphaned, and as such most + instance actions will not work until a nova-compute service with the same + hostname is brought (back) online. + + When nova-compute services are brought up or down, the ring will eventually + re-balance (when the resource tracker runs on each compute). This may + result in duplicate compute_node entries for ironic nodes while the + nova-compute service pool is re-balancing. However, because any + nova-compute service running the ironic virt driver can manage any ironic + node, if a build request goes to the compute service not currently managing + the node the build request is for, it will still succeed. + + There is no configuration to do to enable this feature; it is always + enabled. There are no major changes when only one compute service is + running. If more compute services are brought online, the bigger changes + come into play. + + Note that this is tested when running with only one nova-compute service, + but not more than one. As such, this should be used with caution for + multiple compute hosts until it is properly tested in CI.