Compute node report to placement

Change-Id: I7e2d440b2cc5f397bf7ceeb0d718ee1a12b61971
This commit is contained in:
Hongbin Lu 2019-08-04 17:38:42 +00:00
parent 5fa55e9c08
commit b8af5ebca6
13 changed files with 464 additions and 18 deletions

View File

@ -240,6 +240,10 @@ function create_zun_conf {
iniset $ZUN_CONF DEFAULT enabled_ssl_apis "$ZUN_ENABLED_APIS"
fi
if is_service_enabled n-cpu; then
iniset $ZUN_CONF compute host_shared_with_nova "True"
fi
}
function create_api_paste_conf {

View File

@ -34,6 +34,7 @@ SQLAlchemy!=1.1.5,!=1.1.6,!=1.1.7,!=1.1.8,>=1.0.10 # MIT
stevedore>=1.20.0 # Apache-2.0
docker>=2.4.2 # Apache-2.0
neutron-lib>=1.13.0 # Apache-2.0
retrying>=1.3.3 # Apache-2.0
tenacity>=4.9.0 # Apache-2.0
websockify>=0.8.0 # LGPLv3
websocket-client>=0.44.0 # LGPLv2+

View File

@ -67,3 +67,9 @@ TYPE_CONTAINER = 0
TYPE_CAPSULE = 1
TYPE_CAPSULE_CONTAINER = 2
TYPE_CAPSULE_INIT_CONTAINER = 3
CUSTOM_TRAITS = (
ZUN_COMPUTE_STATUS_DISABLED,
) = (
'CUSTOM_ZUN_COMPUTE_STATUS_DISABLED',
)

View File

@ -849,6 +849,12 @@ class InvalidInventory(Invalid):
"resource provider '%(resource_provider)s' invalid.")
# An exception with this name is used on both sides of the placement/
# nova interaction.
class InventoryInUse(InvalidInventory):
pass
class UsagesRetrievalFailed(ZunException):
message = _("Failed to retrieve usages for project '%(project_id)s' and "
"user '%(user_id)s'.")
@ -882,3 +888,7 @@ class AllocationMoveFailed(ZunException):
class ResourceProviderAllocationRetrievalFailed(ZunException):
message = _("Failed to retrieve allocations for resource provider "
"%(rp_uuid)s: %(error)s")
class ComputeHostNotFound(NotFound):
message = _("Compute host %(host)s could not be found.")

View File

@ -22,6 +22,7 @@ import contextlib
import eventlet
import functools
import inspect
import math
import mimetypes
from oslo_concurrency import lockutils
@ -728,3 +729,12 @@ else:
def nested_contexts(*contexts):
with contextlib.ExitStack() as stack:
yield [stack.enter_context(c) for c in contexts]
def convert_mb_to_ceil_gb(mb_value):
gb_int = 0
if mb_value:
gb_float = mb_value / 1024.0
# ensure we reserve/allocate enough space by rounding up to nearest GB
gb_int = int(math.ceil(gb_float))
return gb_int

View File

@ -16,7 +16,9 @@ import collections
import copy
from oslo_log import log as logging
import retrying
from zun.common import consts
from zun.common import exception
from zun.common import utils
from zun.compute import claims
@ -33,7 +35,7 @@ COMPUTE_RESOURCE_SEMAPHORE = "compute_resources"
class ComputeNodeTracker(object):
def __init__(self, host, container_driver):
def __init__(self, host, container_driver, reportclient):
self.host = host
self.container_driver = container_driver
self.compute_node = None
@ -41,6 +43,7 @@ class ComputeNodeTracker(object):
self.old_resources = collections.defaultdict(objects.ComputeNode)
self.scheduler_client = scheduler_client.SchedulerClient()
self.pci_tracker = None
self.reportclient = reportclient
def _setup_pci_tracker(self, context, compute_node):
if not self.pci_tracker:
@ -130,7 +133,7 @@ class ComputeNodeTracker(object):
self._set_container_host(context, container)
self._update_usage_from_container(context, container)
# persist changes to the compute node:
self._update(self.compute_node)
self._update(context, self.compute_node)
return claim
@ -166,12 +169,16 @@ class ComputeNodeTracker(object):
self._update_usage_from_container_update(context, new_container,
old_container)
# persist changes to the compute node:
self._update(self.compute_node)
self._update(context, self.compute_node)
return claim
def disabled(self, hostname):
return not self.container_driver.node_is_available(hostname)
if not self.compute_node:
return True
return (hostname != self.compute_node.hostname or
not self.container_driver.node_is_available(hostname))
def _set_container_host(self, context, container):
"""Tag the container as belonging to this host.
@ -272,12 +279,14 @@ class ComputeNodeTracker(object):
numa_node.unpin_cpus(cpuset_cpus_usage)
cn._changed_fields.add('numa_topology')
def _update(self, compute_node):
def _update(self, context, compute_node):
if not self._resource_change(compute_node):
return
# Persist the stats to the Scheduler
self.scheduler_client.update_resource(compute_node)
self._update_to_placement(context, compute_node)
if self.pci_tracker:
self.pci_tracker.save()
@ -291,6 +300,111 @@ class ComputeNodeTracker(object):
return True
return False
@retrying.retry(stop_max_attempt_number=4,
retry_on_exception=lambda e: isinstance(
e, exception.ResourceProviderUpdateConflict))
def _update_to_placement(self, context, compute_node):
"""Send resource and inventory changes to placement."""
nodename = compute_node.hostname
node_rp_uuid = compute_node.uuid
if CONF.compute.host_shared_with_nova:
try:
node_rp_uuid = self.reportclient.get_provider_by_name(
context, nodename)['uuid']
except exception.ResourceProviderNotFound:
raise exception.ComputeHostNotFound(host=nodename)
# Persist the stats to the Scheduler
# First try update_provider_tree
# Retrieve the provider tree associated with this compute node. If
# it doesn't exist yet, this will create it with a (single, root)
# provider corresponding to the compute node.
prov_tree = self.reportclient.get_provider_tree_and_ensure_root(
context, node_rp_uuid, name=compute_node.hostname)
# Let the container driver rearrange the provider tree and set/update
# the inventory, traits, and aggregates throughout.
self.container_driver.update_provider_tree(prov_tree, nodename)
# Inject driver capabilities traits into the provider
# tree. We need to determine the traits that the container
# driver owns - so those that come from the tree itself
# (via the container driver) plus the compute capabilities
# traits, and then merge those with the traits set
# externally that the driver does not own - and remove any
# set on the provider externally that the driver owns but
# aren't in the current list of supported traits. For
# example, let's say we reported multiattach support as a
# trait at t1 and then at t2 it's not, so we need to
# remove it. But at both t1 and t2 there is a
# CUSTOM_VENDOR_TRAIT_X which we can't touch because it
# was set externally on the provider.
# We also want to sync the COMPUTE_STATUS_DISABLED trait based
# on the related zun-compute service's disabled status.
traits = self._get_traits(
context, nodename, provider_tree=prov_tree)
prov_tree.update_traits(nodename, traits)
self.reportclient.update_from_provider_tree(context, prov_tree)
def _get_traits(self, context, nodename, provider_tree):
"""Synchronizes internal and external traits for the node provider.
This works in conjunction with the ComptueDriver.update_provider_tree
flow and is used to synchronize traits reported by the compute driver,
traits based on information in the ComputeNode record, and traits set
externally using the placement REST API.
:param context: RequestContext for cell database access
:param nodename: ComputeNode.hostname for the compute node
resource provider whose traits are being synchronized; the node
must be in the ProviderTree.
:param provider_tree: ProviderTree being updated
"""
# Get the traits from the ProviderTree which will be the set
# of driver-owned traits plus any externally defined traits set
# on the provider that aren't owned by the container driver.
traits = provider_tree.data(nodename).traits
# Now get the driver's capabilities and add any supported
# traits that are missing, and remove any existing set traits
# that are not currently supported.
capabilities_traits = self.container_driver.capabilities_as_traits()
for trait, supported in capabilities_traits.items():
if supported:
traits.add(trait)
elif trait in traits:
traits.remove(trait)
self._sync_compute_service_disabled_trait(context, traits)
return list(traits)
def _sync_compute_service_disabled_trait(self, context, traits):
"""Synchronize the ZUN_COMPUTE_STATUS_DISABLED trait on the node provider.
Determines if the ZUN_COMPUTE_STATUS_DISABLED trait should be added to
or removed from the provider's set of traits based on the related
zun-compute service disabled status.
:param context: RequestContext for cell database access
:param traits: set of traits for the compute node resource provider;
this is modified by reference
"""
trait = consts.ZUN_COMPUTE_STATUS_DISABLED
try:
service = objects.ZunService.get_by_host_and_binary(
context, self.host, 'zun-compute')
if service.disabled:
# The service is disabled so make sure the trait is reported.
traits.add(trait)
else:
# The service is not disabled so do not report the trait.
traits.discard(trait)
except exception.NotFound:
# This should not happen but handle it gracefully. The scheduler
# should ignore this node if the compute service record is gone.
LOG.error('Unable to find services table record for zun-compute '
'host %s', self.host)
@utils.synchronized(COMPUTE_RESOURCE_SEMAPHORE)
def _update_available_resource(self, context):
@ -310,7 +424,7 @@ class ComputeNodeTracker(object):
cn = self.compute_node
# update the compute_node
self._update(cn)
self._update(context, cn)
LOG.debug('Compute_service record updated for %(host)s',
{'host': self.host})
@ -345,7 +459,7 @@ class ComputeNodeTracker(object):
"""Remove usage from the given container."""
self._update_usage_from_container(context, container, is_removed=True)
self._update(self.compute_node)
self._update(context, self.compute_node)
@utils.synchronized(COMPUTE_RESOURCE_SEMAPHORE)
def abort_container_update_claim(self, context, new_container,
@ -363,4 +477,4 @@ class ComputeNodeTracker(object):
# We need to get the latest compute node info
self.compute_node = self._get_compute_node(context)
self._update_usage_from_container(context, container, is_removed)
self._update(self.compute_node)
self._update(context, self.compute_node)

View File

@ -39,6 +39,7 @@ from zun.container import driver
from zun.image.glance import driver as glance
from zun.network import neutron
from zun import objects
from zun.scheduler.client import report
CONF = zun.conf.CONF
LOG = logging.getLogger(__name__)
@ -52,6 +53,7 @@ class Manager(periodic_task.PeriodicTasks):
self.driver = driver.load_container_driver(container_driver)
self.host = CONF.host
self._resource_tracker = None
self.reportclient = report.SchedulerReportClient()
def restore_running_container(self, context, container, current_status):
if (container.status == consts.RUNNING and
@ -1117,8 +1119,8 @@ class Manager(periodic_task.PeriodicTasks):
def _get_resource_tracker(self):
if not self._resource_tracker:
rt = compute_node_tracker.ComputeNodeTracker(self.host,
self.driver)
rt = compute_node_tracker.ComputeNodeTracker(
self.host, self.driver, self.reportclient)
self._resource_tracker = rt
return self._resource_tracker

View File

@ -48,6 +48,10 @@ process, causing it to be repopulated the next time the data is accessed.
Possible values:
* Any positive integer in seconds, or zero to disable refresh.
"""),
cfg.BoolOpt(
'host_shared_with_nova',
default=False,
help='Whether this compute node is shared with nova'),
]
service_opts = [
@ -79,10 +83,153 @@ Possible values:
"""),
]
resource_tracker_opts = [
cfg.IntOpt('reserved_host_disk_mb',
min=0,
default=0,
help="""
Amount of disk resources in MB to make them always available to host. The
disk usage gets reported back to the scheduler from zun-compute running
on the compute nodes. To prevent the disk resources from being considered
as available, this option can be used to reserve disk space for that host.
Possible values:
* Any positive integer representing amount of disk in MB to reserve
for the host.
"""),
cfg.IntOpt('reserved_host_memory_mb',
default=512,
min=0,
help="""
Amount of memory in MB to reserve for the host so that it is always available
to host processes. The host resources usage is reported back to the scheduler
continuously from zun-compute running on the compute node. To prevent the host
memory from being considered as available, this option is used to reserve
memory for the host.
Possible values:
* Any positive integer representing amount of memory in MB to reserve
for the host.
"""),
cfg.IntOpt('reserved_host_cpus',
default=0,
min=0,
help="""
Number of physical CPUs to reserve for the host. The host resources usage is
reported back to the scheduler continuously from zun-compute running on the
compute node. To prevent the host CPU from being considered as available,
this option is used to reserve random pCPU(s) for the host.
Possible values:
* Any positive integer representing number of physical CPUs to reserve
for the host.
"""),
]
allocation_ratio_opts = [
cfg.FloatOpt('cpu_allocation_ratio',
default=None,
min=0.0,
help="""
Virtual CPU to physical CPU allocation ratio.
This option is used to influence the hosts selected by the Placement API. In
addition, the ``AggregateCoreFilter`` will fall back to this configuration
value if no per-aggregate setting is found.
.. note::
If this option is set to something *other than* ``None`` or ``0.0``, the
allocation ratio will be overwritten by the value of this option, otherwise,
the allocation ratio will not change. Once set to a non-default value, it is
not possible to "unset" the config to get back to the default behavior. If
you want to reset back to the initial value, explicitly specify it to the
value of ``initial_cpu_allocation_ratio``.
Possible values:
* Any valid positive integer or float value
Related options:
* ``initial_cpu_allocation_ratio``
"""),
cfg.FloatOpt('ram_allocation_ratio',
default=None,
min=0.0,
help="""
Virtual RAM to physical RAM allocation ratio.
This option is used to influence the hosts selected by the Placement API. In
addition, the ``AggregateRamFilter`` will fall back to this configuration value
if no per-aggregate setting is found.
.. note::
If this option is set to something *other than* ``None`` or ``0.0``, the
allocation ratio will be overwritten by the value of this option, otherwise,
the allocation ratio will not change. Once set to a non-default value, it is
not possible to "unset" the config to get back to the default behavior. If
you want to reset back to the initial value, explicitly specify it to the
value of ``initial_ram_allocation_ratio``.
Possible values:
* Any valid positive integer or float value
Related options:
* ``initial_ram_allocation_ratio``
"""),
cfg.FloatOpt('disk_allocation_ratio',
default=None,
min=0.0,
help="""
Virtual disk to physical disk allocation ratio.
This option is used to influence the hosts selected by the Placement API. In
addition, the ``AggregateDiskFilter`` will fall back to this configuration
value if no per-aggregate setting is found.
When configured, a ratio greater than 1.0 will result in over-subscription of
the available physical disk, which can be useful for more efficiently packing
containers created with images that do not use the entire virtual disk.
It can be set to a value between 0.0 and 1.0 in
order to preserve a percentage of the disk for uses other than containers.
.. note::
If the value is set to ``>1``, we recommend keeping track of the free disk
space, as the value approaching ``0`` may result in the incorrect
functioning of instances using it at the moment.
.. note::
If this option is set to something *other than* ``None`` or ``0.0``, the
allocation ratio will be overwritten by the value of this option, otherwise,
the allocation ratio will not change. Once set to a non-default value, it is
not possible to "unset" the config to get back to the default behavior. If
you want to reset back to the initial value, explicitly specify it to the
value of ``initial_disk_allocation_ratio``.
Possible values:
* Any valid positive integer or float value
Related options:
* ``initial_disk_allocation_ratio``
"""),
cfg.FloatOpt('initial_cpu_allocation_ratio',
default=16.0,
min=0.0,
help="""
Initial virtual CPU to physical CPU allocation ratio.
This is only used when initially creating the ``computes_nodes`` table record
for a given zun-compute service.
Related options:
* ``cpu_allocation_ratio``
"""),
cfg.FloatOpt('initial_ram_allocation_ratio',
default=1.5,
min=0.0,
help="""
Initial virtual RAM to physical RAM allocation ratio.
This is only used when initially creating the ``computes_nodes`` table record
for a given zun-compute service.
Related options:
* ``ram_allocation_ratio``
"""),
cfg.FloatOpt('initial_disk_allocation_ratio',
default=1.0,
min=0.0,
help="""
Initial virtual disk to physical disk allocation ratio.
This is only used when initially creating the ``computes_nodes`` table record
for a given zun-compute service.
Related options:
* ``disk_allocation_ratio``
""")
]
opt_group = cfg.OptGroup(
name='compute', title='Options for the zun-compute service')
ALL_OPTS = (service_opts + db_opts + compute_opts)
ALL_OPTS = (service_opts + db_opts + compute_opts + resource_tracker_opts +
allocation_ratio_opts)
def register_opts(conf):

View File

@ -108,6 +108,9 @@ def wrap_docker_error(function):
class DockerDriver(driver.ContainerDriver):
"""Implementation of container drivers for Docker."""
# TODO(hongbin): define a list of capabilities of this driver.
capabilities = {}
def __init__(self):
super(DockerDriver, self).__init__()
self._host = host.Host()
@ -1126,7 +1129,10 @@ class DockerDriver(driver.ContainerDriver):
# give another try with system root
disk_usage = psutil.disk_usage('/')
total_disk = disk_usage.total / 1024 ** 3
return int(total_disk * (1 - CONF.compute.reserve_disk_for_image))
# TODO(hongbin): deprecate reserve_disk_for_image in flavor of
# reserved_host_disk_mb
return (int(total_disk),
int(total_disk * CONF.compute.reserve_disk_for_image))
def add_security_group(self, context, container, security_group):

View File

@ -11,13 +11,16 @@
# See the License for the specific language governing permissions and
# limitations under the License.
import copy
import sys
import os_resource_classes as orc
from oslo_log import log as logging
from oslo_utils import importutils
from oslo_utils import units
from zun.common.i18n import _
from zun.common import utils
import zun.conf
from zun.container.os_capability.linux import os_capability_linux
from zun import objects
@ -26,6 +29,10 @@ LOG = logging.getLogger(__name__)
CONF = zun.conf.CONF
# TODO(hongbin): define a list of standard traits keyed by capabilities
CAPABILITY_TRAITS_MAP = {}
def load_container_driver(container_driver=None):
"""Load a container driver module.
@ -205,7 +212,7 @@ class ContainerDriver(object):
def get_available_resources(self):
"""Retrieve resource information.
This method is called when nova-compute launches, and
This method is called when zun-compute launches, and
as part of a periodic task that records the results in the DB.
:returns: dictionary containing resource info
@ -231,8 +238,8 @@ class ContainerDriver(object):
data['os'] = info['os']
data['kernel_version'] = info['kernel_version']
data['labels'] = info['labels']
disk_total = self.get_total_disk_for_container()
data['disk_total'] = disk_total
disk_total, disk_reserved = self.get_total_disk_for_container()
data['disk_total'] = disk_total - disk_reserved
disk_quota_supported = self.node_support_disk_quota()
data['disk_quota_supported'] = disk_quota_supported
data['runtimes'] = info['runtimes']
@ -246,6 +253,143 @@ class ContainerDriver(object):
return True
return False
def update_provider_tree(self, provider_tree, nodename):
"""Update a ProviderTree object with current resource provider,
inventory information.
:param zun.compute.provider_tree.ProviderTree provider_tree:
A zun.compute.provider_tree.ProviderTree object representing all
the providers in the tree associated with the compute node, and any
sharing providers (those with the ``MISC_SHARES_VIA_AGGREGATE``
trait) associated via aggregate with any of those providers (but
not *their* tree- or aggregate-associated providers), as currently
known by placement.
:param nodename:
String name of the compute node (i.e. ComputeNode.hostname)
for which the caller is requesting updated provider information.
"""
def _get_local_gb_info():
return self.get_total_disk_for_container()[0]
def _get_memory_mb_total():
mem_total, mem_free, mem_ava, mem_used = self.get_host_mem()
return mem_total // units.Ki
def _get_vcpu_total():
return self.get_host_info()['cpus']
disk_gb = _get_local_gb_info()
memory_mb = _get_memory_mb_total()
vcpus = _get_vcpu_total()
# NOTE(yikun): If the inv record does not exists, the allocation_ratio
# will use the CONF.xxx_allocation_ratio value if xxx_allocation_ratio
# is set, and fallback to use the initial_xxx_allocation_ratio
# otherwise.
inv = provider_tree.data(nodename).inventory
ratios = self._get_allocation_ratios(inv)
result = {
orc.VCPU: {
'total': vcpus,
'min_unit': 1,
'max_unit': vcpus,
'step_size': 1,
'allocation_ratio': ratios[orc.VCPU],
# TODO(hongbin): handle the case that the zun's reserved value
# override the nova's one
'reserved': CONF.compute.reserved_host_cpus,
},
orc.MEMORY_MB: {
'total': memory_mb,
'min_unit': 1,
'max_unit': memory_mb,
'step_size': 1,
'allocation_ratio': ratios[orc.MEMORY_MB],
# TODO(hongbin): handle the case that the zun's reserved value
# override the nova's one
'reserved': CONF.compute.reserved_host_memory_mb,
},
}
# If a sharing DISK_GB provider exists in the provider tree, then our
# storage is shared, and we should not report the DISK_GB inventory in
# the compute node provider.
# TODO(efried): Reinstate non-reporting of shared resource by the
# compute RP once the issues from bug #1784020 have been resolved.
if provider_tree.has_sharing_provider(orc.DISK_GB):
LOG.debug('Ignoring sharing provider - see bug #1784020')
result[orc.DISK_GB] = {
'total': disk_gb,
'min_unit': 1,
'max_unit': disk_gb,
'step_size': 1,
'allocation_ratio': ratios[orc.DISK_GB],
# TODO(hongbin): handle the case that the zun's reserved value
# override the nova's one
'reserved': self._get_reserved_host_disk_gb_from_config(),
}
provider_tree.update_inventory(nodename, result)
# Now that we updated the ProviderTree, we want to store it locally
# so that spawn() or other methods can access it thru a getter
self.provider_tree = copy.deepcopy(provider_tree)
@staticmethod
def _get_allocation_ratios(inventory):
"""Get the cpu/ram/disk allocation ratios for the given inventory.
This utility method is used to get the inventory allocation ratio
for VCPU, MEMORY_MB and DISK_GB resource classes based on the following
precedence:
* Use ``[DEFAULT]/*_allocation_ratio`` if set - this overrides
everything including externally set allocation ratios on the
inventory via the placement API
* Use ``[DEFAULT]/initial_*_allocation_ratio`` if a value does not
exist for a given resource class in the ``inventory`` dict
* Use what is already in the ``inventory`` dict for the allocation
ratio if the above conditions are false
:param inventory: dict, keyed by resource class, of inventory
information.
:returns: Return a dict, keyed by resource class, of allocation ratio
"""
keys = {'cpu': orc.VCPU,
'ram': orc.MEMORY_MB,
'disk': orc.DISK_GB}
result = {}
for res, rc in keys.items():
attr = '%s_allocation_ratio' % res
conf_ratio = getattr(CONF.compute, attr)
if conf_ratio:
result[rc] = conf_ratio
elif rc not in inventory:
result[rc] = getattr(CONF.compute, 'initial_%s' % attr)
else:
result[rc] = inventory[rc]['allocation_ratio']
return result
@staticmethod
def _get_reserved_host_disk_gb_from_config():
return utils.convert_mb_to_ceil_gb(CONF.compute.reserved_host_disk_mb)
def capabilities_as_traits(self):
"""Returns this driver's capabilities dict where the keys are traits
Traits can only be standard compute capabilities traits from
the os-traits library.
:returns: dict, keyed by trait, of this driver's capabilities where the
values are booleans indicating if the driver supports the trait
"""
traits = {}
for capability, supported in self.capabilities.items():
if capability in CAPABILITY_TRAITS_MAP:
traits[CAPABILITY_TRAITS_MAP[capability]] = supported
return traits
def network_detach(self, context, container, network):
raise NotImplementedError()

View File

@ -30,6 +30,7 @@ import retrying
import six
from zun.common import clients
from zun.common import context as zun_context
from zun.common import exception
from zun.common.i18n import _
from zun.compute import provider_tree
@ -149,7 +150,7 @@ class SchedulerReportClient(object):
If unspecified, one is created based on config options in the
[placement_client] section.
"""
self._context = context
self._context = context or zun_context.get_admin_context()
# An object that contains a zun-compute-side cache of resource
# provider and inventory information
self._provider_tree = None

View File

@ -27,8 +27,9 @@ class TestNodeStracker(base.TestCase):
def setUp(self):
super(TestNodeStracker, self).setUp()
self.container_driver = fake_driver.FakeDriver()
self.report_client_mock = mock.MagicMock()
self._resource_tracker = compute_node_tracker.ComputeNodeTracker(
'testhost', self.container_driver)
'testhost', self.container_driver, self.report_client_mock)
@mock.patch.object(compute_node_tracker.ComputeNodeTracker, '_update')
@mock.patch.object(compute_node_tracker.ComputeNodeTracker,

View File

@ -929,7 +929,7 @@ class TestDockerDriver(base.DriverTestCase):
'runtimes': ['runc'],
'enable_cpu_pinning': False,
'docker_root_dir': '/var/lib/docker'}
mock_disk.return_value = 80
mock_disk.return_value = (100, 20)
data = self.driver.get_available_resources()
self.assertEqual(_numa_topo_spec, data['numa_topology'].to_list())
self.assertEqual(100, data['mem_total'])