Backend support for magnum service

To support 'magnum service-list' after 'nova service-list', we need to
introduce periodic status update functionality for internal services.

Change-Id: Ia0c09222405c87cb61e5de4a43ba345ae3405b50
Partially-Implements: blueprint magnum-service-list
Closes-bug: #1492501
This commit is contained in:
Surojit Pathak 2015-09-03 00:19:47 +00:00
parent 529444146d
commit 3674ce278d
17 changed files with 665 additions and 10 deletions

View File

@ -72,6 +72,7 @@ def main():
'coreos_template': cfg.CONF.bay.k8s_coreos_template_path})
server = rpc_service.Service.create(cfg.CONF.conductor.topic,
conductor_id, endpoints)
conductor_id, endpoints,
binary='magnum-conductor')
launcher = service.launch(cfg.CONF, server)
launcher.wait()

View File

@ -495,3 +495,11 @@ class CertificateValidationError(Invalid):
class KeyPairNotFound(ResourceNotFound):
message = _("Unable to find keypair %(keypair)s.")
class MagnumServiceNotFound(ResourceNotFound):
message = _("A magnum service %(magnum_service_id)s could not be found.")
class MagnumServiceAlreadyExists(Conflict):
message = _("A magnum service with ID %(id)s already exists.")

View File

@ -56,7 +56,7 @@ CONF.register_opts(periodic_opts)
class Service(service.Service):
def __init__(self, topic, server, handlers):
def __init__(self, topic, server, handlers, binary):
super(Service, self).__init__()
serializer = rpc.RequestContextSerializer(
objects_base.MagnumObjectSerializer())
@ -66,18 +66,19 @@ class Service(service.Service):
target = messaging.Target(topic=topic, server=server)
self._server = messaging.get_rpc_server(transport, target, handlers,
serializer=serializer)
self.binary = binary
def start(self):
if CONF.periodic_enable:
self.tg = periodic.setup(CONF)
self.tg = periodic.setup(CONF, self.binary)
self._server.start()
def wait(self):
self._server.wait()
@classmethod
def create(cls, topic, server, handlers):
service_obj = cls(topic, server, handlers)
def create(cls, topic, server, handlers, binary):
service_obj = cls(topic, server, handlers, binary)
return service_obj

View File

@ -735,3 +735,55 @@ class Connection(object):
(asc, desc)
:returns: A list of tuples of the specified columns.
"""
@abc.abstractmethod
def destroy_magnum_service(self, magnum_service_id):
"""Destroys a magnum_service record.
:param magnum_service_id: The id of a magnum_service.
"""
@abc.abstractmethod
def update_magnum_service(self, magnum_service_id, values):
"""Update properties of a magnum_service.
:param magnum_service_id: The id of a magnum_service record.
"""
@abc.abstractmethod
def get_magnum_service_by_host_and_binary(self, context, host, binary):
"""Return a magnum_service record.
:param context: The security context
:param host: The host where the binary is located.
:param binary: The name of the binary.
:returns: A magnum_service record.
"""
@abc.abstractmethod
def create_magnum_service(self, values):
"""Create a new magnum_service record.
:param values: A dict containing several items used to identify
and define the magnum_service record.
:returns: A magnum_service record.
"""
@abc.abstractmethod
def get_magnum_service_list(self, context, filters=None, limit=None,
marker=None, sort_key=None, sort_dir=None):
"""Get matching magnum_service records.
Return a list of the specified columns for all magnum_services
those match the specified filters.
:param context: The security context
:param filters: Filters to apply. Defaults to None.
:param limit: Maximum number of magnum_services to return.
:param marker: the last item of the previous page; we return the next
result set.
:param sort_key: Attribute by which results should be sorted.
:param sort_dir: direction in which results should be sorted.
(asc, desc)
:returns: A list of tuples of the specified columns.
"""

View File

@ -0,0 +1,49 @@
# 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.
"""adding magnum_service functionality
Revision ID: 27ad304554e2
Revises: 1d045384b966
Create Date: 2015-09-01 18:27:14.371860
"""
# revision identifiers, used by Alembic.
revision = '27ad304554e2'
down_revision = '1d045384b966'
from alembic import op
import sqlalchemy as sa
def upgrade():
op.create_table(
'magnum_service',
sa.Column('created_at', sa.DateTime(), nullable=True),
sa.Column('updated_at', sa.DateTime(), nullable=True),
sa.Column('id', sa.Integer(), nullable=False),
sa.Column('report_count', sa.Integer(), nullable=False),
sa.Column('host', sa.String(length=255), nullable=True),
sa.Column('binary', sa.String(length=255), nullable=True),
sa.Column('disabled', sa.Boolean(), nullable=True),
sa.Column('disabled_reason', sa.String(length=255), nullable=True),
# 'last_seen_up' has different purpose than 'updated_at'.
# 'updated_at' refers to any modification of the entry, which can
# be administrative too, whereas 'last_seen_up' is more related to
# magnum_service. Modeled after nova/servicegroup
sa.Column('last_seen_up', sa.DateTime(), nullable=True),
sa.Column('forced_down', sa.Boolean(), nullable=True),
sa.PrimaryKeyConstraint('id'),
sa.UniqueConstraint('host', 'binary',
name='uniq_magnum_service0host0binary')
)

View File

@ -1026,3 +1026,56 @@ class Connection(api.Connection):
query = self._add_x509keypairs_filters(query, filters)
return _paginate_query(models.X509KeyPair, limit, marker,
sort_key, sort_dir, query)
def destroy_magnum_service(self, magnum_service_id):
session = get_session()
with session.begin():
query = model_query(models.MagnumService, session=session)
query = add_identity_filter(query, magnum_service_id)
count = query.delete()
if count != 1:
raise exception.MagnumServiceNotFound(magnum_service_id)
def update_magnum_service(self, magnum_service_id, values):
session = get_session()
with session.begin():
query = model_query(models.MagnumService, session=session)
query = add_identity_filter(query, magnum_service_id)
try:
ref = query.with_lockmode('update').one()
except NoResultFound:
raise exception.MagnumServiceNotFound(magnum_service_id)
if 'report_count' in values:
if values['report_count'] > ref.report_count:
ref.last_seen_up = timeutils.utcnow()
ref.update(values)
return ref
def get_magnum_service_by_host_and_binary(self, context, host, binary):
query = model_query(models.MagnumService)
query = query.filter_by(host=host, binary=binary)
try:
return query.one()
except NoResultFound:
return None
def create_magnum_service(self, values):
magnum_service = models.MagnumService()
magnum_service.update(values)
try:
magnum_service.save()
except db_exc.DBDuplicateEntry:
raise exception.MagnumServiceAlreadyExists(id=magnum_service['id'])
return magnum_service
def get_magnum_service_list(self, context, disabled=None, limit=None,
marker=None, sort_key=None, sort_dir=None
):
query = model_query(models.MagnumService)
if disabled:
query = query.filter_by(disabled=disabled)
return _paginate_query(models.MagnumService, limit, marker,
sort_key, sort_dir, query)

View File

@ -23,6 +23,7 @@ from oslo_db.sqlalchemy import models
import six.moves.urllib.parse as urlparse
from sqlalchemy import Boolean
from sqlalchemy import Column
from sqlalchemy import DateTime
from sqlalchemy.ext.declarative import declarative_base
from sqlalchemy import Integer
from sqlalchemy import schema
@ -291,3 +292,22 @@ class X509KeyPair(Base):
private_key = Column(Text())
project_id = Column(String(255))
user_id = Column(String(255))
class MagnumService(Base):
"""Represents health status of various magnum services"""
__tablename__ = 'magnum_service'
__table_args__ = (
schema.UniqueConstraint("host", "binary",
name="uniq_magnum_service0host0binary"),
table_args()
)
id = Column(Integer, primary_key=True)
host = Column(String(255))
binary = Column(String(255))
disabled = Column(Boolean, default=False)
disabled_reason = Column(String(255))
last_seen_up = Column(DateTime, nullable=True)
forced_down = Column(Boolean, default=False)
report_count = Column(Integer, nullable=False, default=0)

View File

@ -17,6 +17,7 @@ from magnum.objects import baylock
from magnum.objects import baymodel
from magnum.objects import certificate
from magnum.objects import container
from magnum.objects import magnum_service
from magnum.objects import node
from magnum.objects import pod
from magnum.objects import replicationcontroller as rc
@ -28,6 +29,7 @@ Container = container.Container
Bay = bay.Bay
BayLock = baylock.BayLock
BayModel = baymodel.BayModel
MagnumService = magnum_service.MagnumService
Node = node.Node
Pod = pod.Pod
ReplicationController = rc.ReplicationController
@ -38,6 +40,7 @@ __all__ = (Bay,
BayLock,
BayModel,
Container,
MagnumService,
Node,
Pod,
ReplicationController,

View File

@ -0,0 +1,127 @@
# 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.
from oslo_versionedobjects import fields
from magnum.db import api as dbapi
from magnum.objects import base
@base.MagnumObjectRegistry.register
class MagnumService(base.MagnumPersistentObject, base.MagnumObject,
base.MagnumObjectDictCompat):
# Version 1.0: Initial version
VERSION = '1.0'
dbapi = dbapi.get_instance()
fields = {
'id': fields.IntegerField(),
'host': fields.StringField(nullable=True),
'binary': fields.StringField(nullable=True),
'disabled': fields.BooleanField(),
'disabled_reason': fields.StringField(nullable=True),
'last_seen_up': fields.DateTimeField(nullable=True),
'forced_down': fields.BooleanField(),
'report_count': fields.IntegerField(),
}
@staticmethod
def _from_db_object(magnum_service, db_magnum_service):
"""Converts a database entity to a formal object."""
for field in magnum_service.fields:
magnum_service[field] = db_magnum_service[field]
magnum_service.obj_reset_changes()
return magnum_service
@staticmethod
def _from_db_object_list(db_objects, cls, context):
"""Converts a list of database entities to a list of formal objects."""
return [MagnumService._from_db_object(cls(context), obj)
for obj in db_objects]
@base.remotable_classmethod
def get_by_host_and_binary(cls, context, host, binary):
"""Find a magnum_service based on its hostname and binary.
:param host: The host on which the binary is running.
:param binary: The name of the binary.
:returns: a :class:`MagnumService` object.
"""
db_magnum_service = cls.dbapi.get_magnum_service_by_host_and_binary(
context, host, binary)
if db_magnum_service is None:
return None
magnum_service = MagnumService._from_db_object(
cls(context), db_magnum_service)
return magnum_service
@base.remotable_classmethod
def list(cls, context, limit=None, marker=None,
sort_key=None, sort_dir=None):
"""Return a list of MagnumService objects.
:param context: Security context.
:param limit: maximum number of resources to return in a single result.
:param marker: pagination marker for large data sets.
:param sort_key: column to sort results by.
:param sort_dir: direction to sort. "asc" or "desc".
:returns: a list of :class:`MagnumService` object.
"""
db_magnum_services = cls.dbapi.get_magnum_service_list(
context, limit=limit, marker=marker, sort_key=sort_key,
sort_dir=sort_dir)
return MagnumService._from_db_object_list(db_magnum_services, cls,
context)
@base.remotable
def create(self, context=None):
"""Create a MagnumService record in the DB.
:param context: Security context.
"""
values = self.obj_get_changes()
db_magnum_service = self.dbapi.create_magnum_service(values)
self._from_db_object(self, db_magnum_service)
@base.remotable
def destroy(self, context=None):
"""Delete the MagnumService from the DB.
:param context: Security context.
"""
self.dbapi.destroy_magnum_service(self.id)
self.obj_reset_changes()
@base.remotable
def save(self, context=None):
"""Save updates to this MagnumService.
Updates will be made column by column based on the result
of self.what_changed().
:param context: Security context.
"""
updates = self.obj_get_changes()
self.dbapi.update_magnum_service(self.id, updates)
self.obj_reset_changes()
@base.remotable
def report_state_up(self, context=None):
"""Touching the magnum_service record to show aliveness.
:param context: Security context.
"""
self.report_count += 1
self.save(context)

View File

@ -48,6 +48,33 @@ class MagnumPeriodicTasks(periodic_task.PeriodicTasks):
Any periodic task job need to be added into this class
'''
def __init__(self, conf, binary):
self.magnum_service_ref = None
self.host = conf.host
self.binary = binary
super(MagnumPeriodicTasks, self).__init__(conf)
@periodic_task.periodic_task(run_immediately=True)
@set_context
def update_magnum_service(self, ctx):
LOG.debug('Update magnum_service')
if self.magnum_service_ref:
self.magnum_service_ref.report_state_up(ctx)
else:
self.magnum_service_ref = \
objects.MagnumService.get_by_host_and_binary(
ctx, self.host, self.binary)
if self.magnum_service_ref is None:
magnum_service_dict = {
'host': self.host,
'binary': self.binary
}
self.magnum_service_ref = objects.MagnumService(
ctx, **magnum_service_dict)
self.magnum_service_ref.create(ctx)
self.magnum_service_ref.report_state_up(ctx)
@periodic_task.periodic_task(run_immediately=True)
@set_context
def sync_bay_status(self, ctx):
@ -121,9 +148,9 @@ class MagnumPeriodicTasks(periodic_task.PeriodicTasks):
exc_info=True)
def setup(conf):
def setup(conf, binary):
tg = threadgroup.ThreadGroup()
pt = MagnumPeriodicTasks(conf)
pt = MagnumPeriodicTasks(conf, binary)
tg.add_dynamic_timer(
pt.run_periodic_tasks,
periodic_interval_max=conf.periodic_interval_max,

View File

View File

@ -0,0 +1,33 @@
# 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.
from oslo_utils import timeutils
from magnum.db.sqlalchemy import models
class API(object):
def __init__(self, conf):
self.service_down_time = 3 * conf.periodic_interval_max
def service_is_up(self, member):
if not isinstance(member, models.MagnumService):
raise TypeError
if member.get('forced_down'):
return False
last_heartbeat = (member.get(
'last_seen_up') or member['updated_at'] or member['created_at'])
elapsed = timeutils.delta_seconds(last_heartbeat, timeutils.utcnow())
is_up = abs(elapsed) <= self.service_down_time
return is_up

View File

@ -0,0 +1,100 @@
# 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.
"""Tests for manipulating MagnumService via the DB API"""
from magnum.common import context # NOQA
from magnum.common import exception
from magnum.tests.unit.db import base
from magnum.tests.unit.db import utils
class DbMagnumServiceTestCase(base.DbTestCase):
def test_create_magnum_service(self):
utils.create_test_magnum_service()
def test_create_magnum_service_failure_for_dup(self):
utils.create_test_magnum_service()
self.assertRaises(exception.MagnumServiceAlreadyExists,
utils.create_test_magnum_service)
def test_get_magnum_service_by_host_and_binary(self):
ms = utils.create_test_magnum_service()
res = self.dbapi.get_magnum_service_by_host_and_binary(
self.context, ms['host'], ms['binary'])
self.assertEqual(ms.id, res.id)
def test_get_magnum_service_by_host_and_binary_failure(self):
utils.create_test_magnum_service()
res = self.dbapi.get_magnum_service_by_host_and_binary(
self.context, 'fakehost1', 'fake-bin1')
self.assertEqual(res, None)
def test_update_magnum_service(self):
ms = utils.create_test_magnum_service()
d2 = True
update = {'disabled': d2}
ms1 = self.dbapi.update_magnum_service(ms['id'], update)
self.assertEqual(ms['id'], ms1['id'])
self.assertEqual(ms1['disabled'], d2)
res = self.dbapi.get_magnum_service_by_host_and_binary(
self.context, 'fakehost', 'fake-bin')
self.assertEqual(res['id'], ms1['id'])
self.assertEqual(res['disabled'], d2)
def test_update_magnum_service_failure(self):
ms = utils.create_test_magnum_service()
fake_update = {'fake_field': 'fake_value'}
self.assertRaises(exception.MagnumServiceNotFound,
self.dbapi.update_magnum_service,
ms['id'] + 1, fake_update)
def test_destroy_magnum_service(self):
ms = utils.create_test_magnum_service()
res = self.dbapi.get_magnum_service_by_host_and_binary(
self.context, 'fakehost', 'fake-bin')
self.assertEqual(res['id'], ms['id'])
self.dbapi.destroy_magnum_service(ms['id'])
res = self.dbapi.get_magnum_service_by_host_and_binary(
self.context, 'fakehost', 'fake-bin')
self.assertEqual(res, None)
def test_destroy_magnum_service_failure(self):
ms = utils.create_test_magnum_service()
self.assertRaises(exception.MagnumServiceNotFound,
self.dbapi.destroy_magnum_service,
ms['id'] + 1)
def test_get_magnum_service_list(self):
fake_ms_params = {
'report_count': 1010,
'host': 'FakeHost',
'binary': 'FakeBin',
'disabled': False,
'disabled_reason': 'FakeReason'
}
utils.create_test_magnum_service(**fake_ms_params)
res = self.dbapi.get_magnum_service_list(self.context)
self.assertEqual(1, len(res))
res = res[0]
for k, v in fake_ms_params.iteritems():
self.assertEqual(res[k], v)
fake_ms_params['binary'] = 'FakeBin1'
fake_ms_params['disabled'] = True
utils.create_test_magnum_service(**fake_ms_params)
res = self.dbapi.get_magnum_service_list(self.context, disabled=True)
self.assertEqual(1, len(res))
res = res[0]
for k, v in fake_ms_params.iteritems():
self.assertEqual(res[k], v)

View File

@ -311,3 +311,32 @@ def create_test_x509keypair(**kw):
del x509keypair['id']
dbapi = db_api.get_instance()
return dbapi.create_x509keypair(x509keypair)
def get_test_magnum_service(**kw):
return {
'id': kw.get('', 13),
'report_count': kw.get('report_count', 13),
'host': kw.get('host', 'fakehost'),
'binary': kw.get('binary', 'fake-bin'),
'disabled': kw.get('disabled', False),
'disabled_reason': kw.get('disabled_reason', 'fake-reason'),
'forced_down': kw.get('forced_down', False),
'last_seen_up': kw.get('last_seen_up'),
'created_at': kw.get('created_at'),
'updated_at': kw.get('updated_at'),
}
def create_test_magnum_service(**kw):
"""Create test magnum_service entry in DB and return magnum_service DB object.
:param kw: kwargs with overriding values for magnum_service's attributes.
:returns: Test magnum_service DB object.
"""
magnum_service = get_test_magnum_service(**kw)
# Let DB generate ID if it isn't specified explicitly
if 'id' not in kw:
del magnum_service['id']
dbapi = db_api.get_instance()
return dbapi.create_magnum_service(magnum_service)

View File

@ -0,0 +1,100 @@
# 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 mock
from magnum import objects
from magnum.tests.unit.db import base
from magnum.tests.unit.db import utils
class TestMagnumServiceObject(base.DbTestCase):
def setUp(self):
super(TestMagnumServiceObject, self).setUp()
self.fake_magnum_service = utils.get_test_magnum_service()
def test_get_by_host_and_binary(self):
with mock.patch.object(self.dbapi,
'get_magnum_service_by_host_and_binary',
autospec=True) as mock_get_magnum_service:
mock_get_magnum_service.return_value = self.fake_magnum_service
ms = objects.MagnumService.get_by_host_and_binary(self.context,
'fake-host',
'fake-bin')
mock_get_magnum_service.assert_called_once_with(self.context,
'fake-host',
'fake-bin')
self.assertEqual(self.context, ms._context)
def test_create(self):
with mock.patch.object(self.dbapi, 'create_magnum_service',
autospec=True) as mock_create_magnum_service:
mock_create_magnum_service.return_value = self.fake_magnum_service
ms_dict = {'host': 'fakehost', 'binary': 'fake-bin'}
ms = objects.MagnumService(self.context, **ms_dict)
ms.create(self.context)
mock_create_magnum_service.assert_called_once_with(ms_dict)
def test_destroy(self):
with mock.patch.object(self.dbapi,
'get_magnum_service_by_host_and_binary',
autospec=True) as mock_get_magnum_service:
mock_get_magnum_service.return_value = self.fake_magnum_service
with mock.patch.object(self.dbapi,
'destroy_magnum_service',
autospec=True) as mock_destroy_ms:
ms = objects.MagnumService.get_by_host_and_binary(
self.context, 'fake-host', 'fake-bin')
ms.destroy()
mock_get_magnum_service.assert_called_once_with(
self.context, 'fake-host', 'fake-bin')
mock_destroy_ms.assert_called_once_with(
self.fake_magnum_service['id'])
self.assertEqual(self.context, ms._context)
def test_save(self):
with mock.patch.object(self.dbapi,
'get_magnum_service_by_host_and_binary',
autospec=True) as mock_get_magnum_service:
mock_get_magnum_service.return_value = self.fake_magnum_service
with mock.patch.object(self.dbapi,
'update_magnum_service',
autospec=True) as mock_update_ms:
ms = objects.MagnumService.get_by_host_and_binary(
self.context, 'fake-host', 'fake-bin')
ms.disabled = True
ms.save()
mock_get_magnum_service.assert_called_once_with(
self.context, 'fake-host', 'fake-bin')
mock_update_ms.assert_called_once_with(
self.fake_magnum_service['id'], {'disabled': True})
self.assertEqual(self.context, ms._context)
def test_report_state_up(self):
with mock.patch.object(self.dbapi,
'get_magnum_service_by_host_and_binary',
autospec=True) as mock_get_magnum_service:
mock_get_magnum_service.return_value = self.fake_magnum_service
with mock.patch.object(self.dbapi,
'update_magnum_service',
autospec=True) as mock_update_ms:
ms = objects.MagnumService.get_by_host_and_binary(
self.context, 'fake-host', 'fake-bin')
last_report_count = self.fake_magnum_service['report_count']
ms.report_state_up()
mock_get_magnum_service.assert_called_once_with(
self.context, 'fake-host', 'fake-bin')
self.assertEqual(self.context, ms._context)
mock_update_ms.assert_called_once_with(
self.fake_magnum_service['id'],
{'report_count': last_report_count + 1})

View File

@ -435,6 +435,7 @@ object_data = {
'ReplicationController': '1.0-782b7deb9307b2807101541b7e58b8a2',
'Service': '1.0-d4b8c0f3a234aec35d273196e18f7ed1',
'X509KeyPair': '1.0-fd008eba0fbc390e0e5da247bba4eedd',
'MagnumService': '1.0-2d397ec59b0046bd5ec35cd3e06efeca',
}

View File

@ -60,6 +60,14 @@ class PeriodicTestCase(base.TestCase):
self.bay2 = objects.Bay(ctx, **bay2)
self.bay3 = objects.Bay(ctx, **bay3)
mock_magnum_service_refresh = mock.Mock()
class FakeMS(object):
report_state_up = mock_magnum_service_refresh
self.fake_ms = FakeMS()
self.fake_ms_refresh = mock_magnum_service_refresh
@mock.patch.object(objects.Bay, 'list')
@mock.patch('magnum.common.clients.OpenStackClients')
@mock.patch.object(dbapi.Connection, 'destroy_bay')
@ -80,7 +88,8 @@ class PeriodicTestCase(base.TestCase):
mock_keystone_client.client.project_id = "fake_project"
mock_osc.keystone.return_value = mock_keystone_client
periodic.MagnumPeriodicTasks(CONF).sync_bay_status(None)
periodic.MagnumPeriodicTasks(CONF,
'fake-conductor').sync_bay_status(None)
self.assertEqual(self.bay1.status, bay_status.CREATE_COMPLETE)
self.assertEqual(self.bay1.status_reason, 'fake_reason_11')
@ -102,7 +111,8 @@ class PeriodicTestCase(base.TestCase):
mock_osc = mock_oscc.return_value
mock_osc.heat.return_value = mock_heat_client
mock_bay_list.return_value = [self.bay1, self.bay2, self.bay3]
periodic.MagnumPeriodicTasks(CONF).sync_bay_status(None)
periodic.MagnumPeriodicTasks(CONF,
'fake-conductor').sync_bay_status(None)
self.assertEqual(self.bay1.status, bay_status.CREATE_IN_PROGRESS)
self.assertEqual(self.bay2.status, bay_status.DELETE_IN_PROGRESS)
@ -125,7 +135,8 @@ class PeriodicTestCase(base.TestCase):
mock_keystone_client.client.project_id = "fake_project"
mock_osc.keystone.return_value = mock_keystone_client
periodic.MagnumPeriodicTasks(CONF).sync_bay_status(None)
periodic.MagnumPeriodicTasks(CONF,
'fake-conductor').sync_bay_status(None)
self.assertEqual(self.bay1.status, bay_status.CREATE_FAILED)
self.assertEqual(self.bay1.status_reason, 'Stack with id 11 not '
@ -134,3 +145,43 @@ class PeriodicTestCase(base.TestCase):
self.assertEqual(self.bay3.status, bay_status.UPDATE_FAILED)
self.assertEqual(self.bay3.status_reason, 'Stack with id 33 not '
'found in Heat.')
@mock.patch.object(objects.MagnumService, 'get_by_host_and_binary')
@mock.patch.object(objects.MagnumService, 'create')
@mock.patch.object(objects.MagnumService, 'report_state_up')
def test_update_magnum_service_firsttime(self,
mock_ms_refresh,
mock_ms_create,
mock_ms_get
):
periodic_a = periodic.MagnumPeriodicTasks(CONF, 'fake-conductor')
mock_ms_get.return_value = None
periodic_a.update_magnum_service(None)
mock_ms_get.assert_called_once_with(mock.ANY, periodic_a.host,
periodic_a.binary)
mock_ms_create.assert_called_once_with(mock.ANY)
mock_ms_refresh.assert_called_once_with(mock.ANY)
@mock.patch.object(objects.MagnumService, 'get_by_host_and_binary')
@mock.patch.object(objects.MagnumService, 'create')
def test_update_magnum_service_on_restart(self,
mock_ms_create,
mock_ms_get):
periodic_a = periodic.MagnumPeriodicTasks(CONF, 'fake-conductor')
mock_ms_get.return_value = self.fake_ms
periodic_a.update_magnum_service(None)
mock_ms_get.assert_called_once_with(mock.ANY, periodic_a.host,
periodic_a.binary)
self.fake_ms_refresh.assert_called_once_with(mock.ANY)
def test_update_magnum_service_regular(self):
periodic_a = periodic.MagnumPeriodicTasks(CONF, 'fake-conductor')
periodic_a.magnum_service_ref = self.fake_ms
periodic_a.update_magnum_service(None)
self.fake_ms_refresh.assert_called_once_with(mock.ANY)