Merge "add event pipeline"
This commit is contained in:
commit
479ac5feea
@ -17,10 +17,12 @@
|
|||||||
# License for the specific language governing permissions and limitations
|
# License for the specific language governing permissions and limitations
|
||||||
# under the License.
|
# under the License.
|
||||||
|
|
||||||
|
import abc
|
||||||
import fnmatch
|
import fnmatch
|
||||||
import os
|
import os
|
||||||
|
|
||||||
from oslo_config import cfg
|
from oslo_config import cfg
|
||||||
|
import six
|
||||||
import yaml
|
import yaml
|
||||||
|
|
||||||
from ceilometer.i18n import _
|
from ceilometer.i18n import _
|
||||||
@ -35,6 +37,10 @@ OPTS = [
|
|||||||
default="pipeline.yaml",
|
default="pipeline.yaml",
|
||||||
help="Configuration file for pipeline definition."
|
help="Configuration file for pipeline definition."
|
||||||
),
|
),
|
||||||
|
cfg.StrOpt('event_pipeline_cfg_file',
|
||||||
|
default="event_pipeline.yaml",
|
||||||
|
help="Configuration file for event pipeline definition."
|
||||||
|
),
|
||||||
]
|
]
|
||||||
|
|
||||||
cfg.CONF.register_opts(OPTS)
|
cfg.CONF.register_opts(OPTS)
|
||||||
@ -104,6 +110,7 @@ class Source(object):
|
|||||||
|
|
||||||
try:
|
try:
|
||||||
self.name = cfg['name']
|
self.name = cfg['name']
|
||||||
|
self.sinks = cfg.get('sinks')
|
||||||
except KeyError as err:
|
except KeyError as err:
|
||||||
raise PipelineException(
|
raise PipelineException(
|
||||||
"Required field %s not specified" % err.args[0], cfg)
|
"Required field %s not specified" % err.args[0], cfg)
|
||||||
@ -143,6 +150,41 @@ class Source(object):
|
|||||||
'Included %s specified with wildcard' % d_type,
|
'Included %s specified with wildcard' % d_type,
|
||||||
self.cfg)
|
self.cfg)
|
||||||
|
|
||||||
|
@staticmethod
|
||||||
|
def is_supported(dataset, data_name):
|
||||||
|
# Support wildcard like storage.* and !disk.*
|
||||||
|
# Start with negation, we consider that the order is deny, allow
|
||||||
|
if any(fnmatch.fnmatch(data_name, datapoint[1:])
|
||||||
|
for datapoint in dataset if datapoint[0] == '!'):
|
||||||
|
return False
|
||||||
|
|
||||||
|
if any(fnmatch.fnmatch(data_name, datapoint)
|
||||||
|
for datapoint in dataset if datapoint[0] != '!'):
|
||||||
|
return True
|
||||||
|
|
||||||
|
# if we only have negation, we suppose the default is allow
|
||||||
|
return all(datapoint.startswith('!') for datapoint in dataset)
|
||||||
|
|
||||||
|
|
||||||
|
class EventSource(Source):
|
||||||
|
"""Represents a source of events.
|
||||||
|
|
||||||
|
In effect it is a set of notification handlers capturing events for a set
|
||||||
|
of matching notifications.
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __init__(self, cfg):
|
||||||
|
super(EventSource, self).__init__(cfg)
|
||||||
|
try:
|
||||||
|
self.events = cfg['events']
|
||||||
|
except KeyError as err:
|
||||||
|
raise PipelineException(
|
||||||
|
"Required field %s not specified" % err.args[0], cfg)
|
||||||
|
self.check_source_filtering(self.events, 'events')
|
||||||
|
|
||||||
|
def support_event(self, event_name):
|
||||||
|
return self.is_supported(self.events, event_name)
|
||||||
|
|
||||||
|
|
||||||
class SampleSource(Source):
|
class SampleSource(Source):
|
||||||
"""Represents a source of samples.
|
"""Represents a source of samples.
|
||||||
@ -162,7 +204,6 @@ class SampleSource(Source):
|
|||||||
raise PipelineException("Invalid interval value", cfg)
|
raise PipelineException("Invalid interval value", cfg)
|
||||||
# Support 'counters' for backward compatibility
|
# Support 'counters' for backward compatibility
|
||||||
self.meters = cfg.get('meters', cfg.get('counters'))
|
self.meters = cfg.get('meters', cfg.get('counters'))
|
||||||
self.sinks = cfg.get('sinks')
|
|
||||||
except KeyError as err:
|
except KeyError as err:
|
||||||
raise PipelineException(
|
raise PipelineException(
|
||||||
"Required field %s not specified" % err.args[0], cfg)
|
"Required field %s not specified" % err.args[0], cfg)
|
||||||
@ -191,24 +232,7 @@ class SampleSource(Source):
|
|||||||
|
|
||||||
def support_meter(self, meter_name):
|
def support_meter(self, meter_name):
|
||||||
meter_name = self._variable_meter_name(meter_name)
|
meter_name = self._variable_meter_name(meter_name)
|
||||||
|
return self.is_supported(self.meters, meter_name)
|
||||||
# Special case: if we only have negation, we suppose the default is
|
|
||||||
# allow
|
|
||||||
default = all(meter.startswith('!') for meter in self.meters)
|
|
||||||
|
|
||||||
# Support wildcard like storage.* and !disk.*
|
|
||||||
# Start with negation, we consider that the order is deny, allow
|
|
||||||
if any(fnmatch.fnmatch(meter_name, meter[1:])
|
|
||||||
for meter in self.meters
|
|
||||||
if meter[0] == '!'):
|
|
||||||
return False
|
|
||||||
|
|
||||||
if any(fnmatch.fnmatch(meter_name, meter)
|
|
||||||
for meter in self.meters
|
|
||||||
if meter[0] != '!'):
|
|
||||||
return True
|
|
||||||
|
|
||||||
return default
|
|
||||||
|
|
||||||
|
|
||||||
class Sink(object):
|
class Sink(object):
|
||||||
@ -285,6 +309,24 @@ class Sink(object):
|
|||||||
return transformers
|
return transformers
|
||||||
|
|
||||||
|
|
||||||
|
class EventSink(Sink):
|
||||||
|
|
||||||
|
def publish_events(self, ctxt, events):
|
||||||
|
if events:
|
||||||
|
for p in self.publishers:
|
||||||
|
try:
|
||||||
|
p.publish_events(ctxt, events)
|
||||||
|
except Exception:
|
||||||
|
LOG.exception(_(
|
||||||
|
"Pipeline %(pipeline)s: Continue after error "
|
||||||
|
"from publisher %(pub)s") % ({'pipeline': self,
|
||||||
|
'pub': p}))
|
||||||
|
|
||||||
|
def flush(self, ctxt):
|
||||||
|
"""Flush data after all events have been injected to pipeline."""
|
||||||
|
pass
|
||||||
|
|
||||||
|
|
||||||
class SampleSink(Sink):
|
class SampleSink(Sink):
|
||||||
|
|
||||||
def _transform_sample(self, start, ctxt, sample):
|
def _transform_sample(self, start, ctxt, sample):
|
||||||
@ -299,6 +341,7 @@ class SampleSink(Sink):
|
|||||||
return
|
return
|
||||||
return sample
|
return sample
|
||||||
except Exception as err:
|
except Exception as err:
|
||||||
|
# TODO(gordc): only use one log level.
|
||||||
LOG.warning(_("Pipeline %(pipeline)s: "
|
LOG.warning(_("Pipeline %(pipeline)s: "
|
||||||
"Exit after error from transformer "
|
"Exit after error from transformer "
|
||||||
"%(trans)s for %(smp)s") % ({'pipeline': self,
|
"%(trans)s for %(smp)s") % ({'pipeline': self,
|
||||||
@ -359,6 +402,7 @@ class SampleSink(Sink):
|
|||||||
LOG.exception(err)
|
LOG.exception(err)
|
||||||
|
|
||||||
|
|
||||||
|
@six.add_metaclass(abc.ABCMeta)
|
||||||
class Pipeline(object):
|
class Pipeline(object):
|
||||||
"""Represents a coupling between a sink and a corresponding source."""
|
"""Represents a coupling between a sink and a corresponding source."""
|
||||||
|
|
||||||
@ -378,6 +422,29 @@ class Pipeline(object):
|
|||||||
def publishers(self):
|
def publishers(self):
|
||||||
return self.sink.publishers
|
return self.sink.publishers
|
||||||
|
|
||||||
|
@abc.abstractmethod
|
||||||
|
def publish_data(self, ctxt, data):
|
||||||
|
"""Publish data from pipeline."""
|
||||||
|
|
||||||
|
|
||||||
|
class EventPipeline(Pipeline):
|
||||||
|
"""Represents a pipeline for Events."""
|
||||||
|
|
||||||
|
def __str__(self):
|
||||||
|
# NOTE(gordc): prepend a namespace so we ensure event and sample
|
||||||
|
# pipelines do not have the same name.
|
||||||
|
return 'event:%s' % super(EventPipeline, self).__str__()
|
||||||
|
|
||||||
|
def support_event(self, event_type):
|
||||||
|
return self.source.support_event(event_type)
|
||||||
|
|
||||||
|
def publish_data(self, ctxt, events):
|
||||||
|
if not isinstance(events, list):
|
||||||
|
events = [events]
|
||||||
|
supported = [e for e in events
|
||||||
|
if self.source.support_event(e.event_type)]
|
||||||
|
self.sink.publish_events(ctxt, supported)
|
||||||
|
|
||||||
|
|
||||||
class SamplePipeline(Pipeline):
|
class SamplePipeline(Pipeline):
|
||||||
"""Represents a pipeline for Samples."""
|
"""Represents a pipeline for Samples."""
|
||||||
@ -407,6 +474,10 @@ SAMPLE_TYPE = {'pipeline': SamplePipeline,
|
|||||||
'source': SampleSource,
|
'source': SampleSource,
|
||||||
'sink': SampleSink}
|
'sink': SampleSink}
|
||||||
|
|
||||||
|
EVENT_TYPE = {'pipeline': EventPipeline,
|
||||||
|
'source': EventSource,
|
||||||
|
'sink': EventSink}
|
||||||
|
|
||||||
|
|
||||||
class PipelineManager(object):
|
class PipelineManager(object):
|
||||||
"""Pipeline Manager
|
"""Pipeline Manager
|
||||||
@ -556,9 +627,7 @@ class PipelineManager(object):
|
|||||||
return PublishContext(context, self.pipelines)
|
return PublishContext(context, self.pipelines)
|
||||||
|
|
||||||
|
|
||||||
def setup_pipeline(transformer_manager=None):
|
def _setup_pipeline_manager(cfg_file, transformer_manager, p_type=SAMPLE_TYPE):
|
||||||
"""Setup pipeline manager according to yaml config file."""
|
|
||||||
cfg_file = cfg.CONF.pipeline_cfg_file
|
|
||||||
if not os.path.exists(cfg_file):
|
if not os.path.exists(cfg_file):
|
||||||
cfg_file = cfg.CONF.find_file(cfg_file)
|
cfg_file = cfg.CONF.find_file(cfg_file)
|
||||||
|
|
||||||
@ -574,4 +643,16 @@ def setup_pipeline(transformer_manager=None):
|
|||||||
transformer_manager or
|
transformer_manager or
|
||||||
xformer.TransformerExtensionManager(
|
xformer.TransformerExtensionManager(
|
||||||
'ceilometer.transformer',
|
'ceilometer.transformer',
|
||||||
))
|
), p_type)
|
||||||
|
|
||||||
|
|
||||||
|
def setup_event_pipeline(transformer_manager=None):
|
||||||
|
"""Setup event pipeline manager according to yaml config file."""
|
||||||
|
cfg_file = cfg.CONF.event_pipeline_cfg_file
|
||||||
|
return _setup_pipeline_manager(cfg_file, transformer_manager, EVENT_TYPE)
|
||||||
|
|
||||||
|
|
||||||
|
def setup_pipeline(transformer_manager=None):
|
||||||
|
"""Setup pipeline manager according to yaml config file."""
|
||||||
|
cfg_file = cfg.CONF.pipeline_cfg_file
|
||||||
|
return _setup_pipeline_manager(cfg_file, transformer_manager)
|
||||||
|
@ -34,7 +34,7 @@ def get_publisher(url, namespace='ceilometer.publisher'):
|
|||||||
|
|
||||||
@six.add_metaclass(abc.ABCMeta)
|
@six.add_metaclass(abc.ABCMeta)
|
||||||
class PublisherBase(object):
|
class PublisherBase(object):
|
||||||
"""Base class for plugins that publish the sampler."""
|
"""Base class for plugins that publish data."""
|
||||||
|
|
||||||
def __init__(self, parsed_url):
|
def __init__(self, parsed_url):
|
||||||
pass
|
pass
|
||||||
@ -42,3 +42,7 @@ class PublisherBase(object):
|
|||||||
@abc.abstractmethod
|
@abc.abstractmethod
|
||||||
def publish_samples(self, context, samples):
|
def publish_samples(self, context, samples):
|
||||||
"""Publish samples into final conduit."""
|
"""Publish samples into final conduit."""
|
||||||
|
|
||||||
|
@abc.abstractmethod
|
||||||
|
def publish_events(self, context, events):
|
||||||
|
"""Publish events into final conduit."""
|
||||||
|
@ -17,6 +17,7 @@
|
|||||||
from oslo.config import cfg
|
from oslo.config import cfg
|
||||||
from oslo.utils import timeutils
|
from oslo.utils import timeutils
|
||||||
|
|
||||||
|
import ceilometer
|
||||||
from ceilometer.dispatcher import database
|
from ceilometer.dispatcher import database
|
||||||
from ceilometer import publisher
|
from ceilometer import publisher
|
||||||
from ceilometer.publisher import utils
|
from ceilometer.publisher import utils
|
||||||
@ -51,3 +52,6 @@ class DirectPublisher(publisher.PublisherBase):
|
|||||||
ts = timeutils.parse_isotime(meter['timestamp'])
|
ts = timeutils.parse_isotime(meter['timestamp'])
|
||||||
meter['timestamp'] = timeutils.normalize_time(ts)
|
meter['timestamp'] = timeutils.normalize_time(ts)
|
||||||
self.meter_conn.record_metering_data(meter)
|
self.meter_conn.record_metering_data(meter)
|
||||||
|
|
||||||
|
def publish_events(self, context, events):
|
||||||
|
raise ceilometer.NotImplementedError
|
||||||
|
@ -18,6 +18,7 @@ import logging.handlers
|
|||||||
|
|
||||||
from six.moves.urllib import parse as urlparse
|
from six.moves.urllib import parse as urlparse
|
||||||
|
|
||||||
|
import ceilometer
|
||||||
from ceilometer.i18n import _
|
from ceilometer.i18n import _
|
||||||
from ceilometer.openstack.common import log
|
from ceilometer.openstack.common import log
|
||||||
from ceilometer import publisher
|
from ceilometer import publisher
|
||||||
@ -95,3 +96,11 @@ class FilePublisher(publisher.PublisherBase):
|
|||||||
if self.publisher_logger:
|
if self.publisher_logger:
|
||||||
for sample in samples:
|
for sample in samples:
|
||||||
self.publisher_logger.info(sample.as_dict())
|
self.publisher_logger.info(sample.as_dict())
|
||||||
|
|
||||||
|
def publish_events(self, context, events):
|
||||||
|
"""Send an event message for publishing
|
||||||
|
|
||||||
|
:param context: Execution context from the service or RPC call
|
||||||
|
:param events: events from pipeline after transformation
|
||||||
|
"""
|
||||||
|
raise ceilometer.NotImplementedError
|
||||||
|
@ -24,6 +24,7 @@ from oslo_config import cfg
|
|||||||
import six
|
import six
|
||||||
import six.moves.urllib.parse as urlparse
|
import six.moves.urllib.parse as urlparse
|
||||||
|
|
||||||
|
import ceilometer
|
||||||
from ceilometer.i18n import _
|
from ceilometer.i18n import _
|
||||||
from ceilometer import messaging
|
from ceilometer import messaging
|
||||||
from ceilometer.openstack.common import log
|
from ceilometer.openstack.common import log
|
||||||
@ -160,6 +161,14 @@ class MessagingPublisher(publisher.PublisherBase):
|
|||||||
queue.pop(0)
|
queue.pop(0)
|
||||||
return []
|
return []
|
||||||
|
|
||||||
|
def publish_events(self, context, events):
|
||||||
|
"""Send an event message for publishing
|
||||||
|
|
||||||
|
:param context: Execution context from the service or RPC call
|
||||||
|
:param events: events from pipeline after transformation
|
||||||
|
"""
|
||||||
|
raise ceilometer.NotImplementedError
|
||||||
|
|
||||||
@abc.abstractmethod
|
@abc.abstractmethod
|
||||||
def _send(self, context, topic, meters):
|
def _send(self, context, topic, meters):
|
||||||
"""Send the meters to the messaging topic."""
|
"""Send the meters to the messaging topic."""
|
||||||
|
@ -23,6 +23,7 @@ class TestPublisher(publisher.PublisherBase):
|
|||||||
|
|
||||||
def __init__(self, parsed_url):
|
def __init__(self, parsed_url):
|
||||||
self.samples = []
|
self.samples = []
|
||||||
|
self.events = []
|
||||||
self.calls = 0
|
self.calls = 0
|
||||||
|
|
||||||
def publish_samples(self, context, samples):
|
def publish_samples(self, context, samples):
|
||||||
@ -33,3 +34,12 @@ class TestPublisher(publisher.PublisherBase):
|
|||||||
"""
|
"""
|
||||||
self.samples.extend(samples)
|
self.samples.extend(samples)
|
||||||
self.calls += 1
|
self.calls += 1
|
||||||
|
|
||||||
|
def publish_events(self, context, events):
|
||||||
|
"""Send an event message for publishing
|
||||||
|
|
||||||
|
:param context: Execution context from the service or RPC call
|
||||||
|
:param events: events from pipeline after transformation
|
||||||
|
"""
|
||||||
|
self.events.extend(events)
|
||||||
|
self.calls += 1
|
||||||
|
@ -21,6 +21,7 @@ import msgpack
|
|||||||
from oslo_config import cfg
|
from oslo_config import cfg
|
||||||
from oslo_utils import netutils
|
from oslo_utils import netutils
|
||||||
|
|
||||||
|
import ceilometer
|
||||||
from ceilometer.i18n import _
|
from ceilometer.i18n import _
|
||||||
from ceilometer.openstack.common import log
|
from ceilometer.openstack.common import log
|
||||||
from ceilometer import publisher
|
from ceilometer import publisher
|
||||||
@ -62,3 +63,11 @@ class UDPPublisher(publisher.PublisherBase):
|
|||||||
except Exception as e:
|
except Exception as e:
|
||||||
LOG.warn(_("Unable to send sample over UDP"))
|
LOG.warn(_("Unable to send sample over UDP"))
|
||||||
LOG.exception(e)
|
LOG.exception(e)
|
||||||
|
|
||||||
|
def publish_events(self, context, events):
|
||||||
|
"""Send an event message for publishing
|
||||||
|
|
||||||
|
:param context: Execution context from the service or RPC call
|
||||||
|
:param events: events from pipeline after transformation
|
||||||
|
"""
|
||||||
|
raise ceilometer.NotImplementedError
|
||||||
|
@ -78,7 +78,10 @@ class BasePipelineTestCase(base.BaseTestCase):
|
|||||||
return fake_drivers[url](url)
|
return fake_drivers[url](url)
|
||||||
|
|
||||||
class PublisherClassException(publisher.PublisherBase):
|
class PublisherClassException(publisher.PublisherBase):
|
||||||
def publish_samples(self, ctxt, counters):
|
def publish_samples(self, ctxt, samples):
|
||||||
|
raise Exception()
|
||||||
|
|
||||||
|
def publish_events(self, ctxt, events):
|
||||||
raise Exception()
|
raise Exception()
|
||||||
|
|
||||||
class TransformerClass(transformer.TransformerBase):
|
class TransformerClass(transformer.TransformerBase):
|
||||||
|
364
ceilometer/tests/test_event_pipeline.py
Normal file
364
ceilometer/tests/test_event_pipeline.py
Normal file
@ -0,0 +1,364 @@
|
|||||||
|
#
|
||||||
|
# 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 datetime
|
||||||
|
import traceback
|
||||||
|
import uuid
|
||||||
|
|
||||||
|
from oslotest import base
|
||||||
|
from oslotest import mockpatch
|
||||||
|
|
||||||
|
from ceilometer.event.storage import models
|
||||||
|
from ceilometer import pipeline
|
||||||
|
from ceilometer import publisher
|
||||||
|
from ceilometer.publisher import test as test_publisher
|
||||||
|
|
||||||
|
|
||||||
|
class EventPipelineTestCase(base.BaseTestCase):
|
||||||
|
|
||||||
|
def get_publisher(self, url, namespace=''):
|
||||||
|
fake_drivers = {'test://': test_publisher.TestPublisher,
|
||||||
|
'new://': test_publisher.TestPublisher,
|
||||||
|
'except://': self.PublisherClassException}
|
||||||
|
return fake_drivers[url](url)
|
||||||
|
|
||||||
|
class PublisherClassException(publisher.PublisherBase):
|
||||||
|
def publish_samples(self, ctxt, samples):
|
||||||
|
pass
|
||||||
|
|
||||||
|
def publish_events(self, ctxt, events):
|
||||||
|
raise Exception()
|
||||||
|
|
||||||
|
def setUp(self):
|
||||||
|
super(EventPipelineTestCase, self).setUp()
|
||||||
|
self.p_type = pipeline.EVENT_TYPE
|
||||||
|
self.transformer_manager = None
|
||||||
|
|
||||||
|
self.test_event = models.Event(
|
||||||
|
message_id=uuid.uuid4(),
|
||||||
|
event_type='a',
|
||||||
|
generated=datetime.datetime.utcnow(),
|
||||||
|
traits=[
|
||||||
|
models.Trait('t_text', 1, 'text_trait'),
|
||||||
|
models.Trait('t_int', 2, 'int_trait'),
|
||||||
|
models.Trait('t_float', 3, 'float_trait'),
|
||||||
|
models.Trait('t_datetime', 4, 'datetime_trait')
|
||||||
|
]
|
||||||
|
)
|
||||||
|
|
||||||
|
self.test_event2 = models.Event(
|
||||||
|
message_id=uuid.uuid4(),
|
||||||
|
event_type='b',
|
||||||
|
generated=datetime.datetime.utcnow(),
|
||||||
|
traits=[
|
||||||
|
models.Trait('t_text', 1, 'text_trait'),
|
||||||
|
models.Trait('t_int', 2, 'int_trait'),
|
||||||
|
models.Trait('t_float', 3, 'float_trait'),
|
||||||
|
models.Trait('t_datetime', 4, 'datetime_trait')
|
||||||
|
]
|
||||||
|
)
|
||||||
|
|
||||||
|
self.useFixture(mockpatch.PatchObject(
|
||||||
|
publisher, 'get_publisher', side_effect=self.get_publisher))
|
||||||
|
|
||||||
|
self._setup_pipeline_cfg()
|
||||||
|
|
||||||
|
self._reraise_exception = True
|
||||||
|
self.useFixture(mockpatch.Patch(
|
||||||
|
'ceilometer.pipeline.LOG.exception',
|
||||||
|
side_effect=self._handle_reraise_exception))
|
||||||
|
|
||||||
|
def _handle_reraise_exception(self, msg):
|
||||||
|
if self._reraise_exception:
|
||||||
|
raise Exception(traceback.format_exc())
|
||||||
|
|
||||||
|
def _setup_pipeline_cfg(self):
|
||||||
|
"""Setup the appropriate form of pipeline config."""
|
||||||
|
source = {'name': 'test_source',
|
||||||
|
'events': ['a'],
|
||||||
|
'sinks': ['test_sink']}
|
||||||
|
sink = {'name': 'test_sink',
|
||||||
|
'publishers': ['test://']}
|
||||||
|
self.pipeline_cfg = {'sources': [source], 'sinks': [sink]}
|
||||||
|
|
||||||
|
def _augment_pipeline_cfg(self):
|
||||||
|
"""Augment the pipeline config with an additional element."""
|
||||||
|
self.pipeline_cfg['sources'].append({
|
||||||
|
'name': 'second_source',
|
||||||
|
'events': ['b'],
|
||||||
|
'sinks': ['second_sink']
|
||||||
|
})
|
||||||
|
self.pipeline_cfg['sinks'].append({
|
||||||
|
'name': 'second_sink',
|
||||||
|
'publishers': ['new://'],
|
||||||
|
})
|
||||||
|
|
||||||
|
def _break_pipeline_cfg(self):
|
||||||
|
"""Break the pipeline config with a malformed element."""
|
||||||
|
self.pipeline_cfg['sources'].append({
|
||||||
|
'name': 'second_source',
|
||||||
|
'events': ['b'],
|
||||||
|
'sinks': ['second_sink']
|
||||||
|
})
|
||||||
|
self.pipeline_cfg['sinks'].append({
|
||||||
|
'name': 'second_sink',
|
||||||
|
'publishers': ['except'],
|
||||||
|
})
|
||||||
|
|
||||||
|
def _dup_pipeline_name_cfg(self):
|
||||||
|
"""Break the pipeline config with duplicate pipeline name."""
|
||||||
|
self.pipeline_cfg['sources'].append({
|
||||||
|
'name': 'test_source',
|
||||||
|
'events': ['a'],
|
||||||
|
'sinks': ['test_sink']
|
||||||
|
})
|
||||||
|
|
||||||
|
def _set_pipeline_cfg(self, field, value):
|
||||||
|
if field in self.pipeline_cfg['sources'][0]:
|
||||||
|
self.pipeline_cfg['sources'][0][field] = value
|
||||||
|
else:
|
||||||
|
self.pipeline_cfg['sinks'][0][field] = value
|
||||||
|
|
||||||
|
def _extend_pipeline_cfg(self, field, value):
|
||||||
|
if field in self.pipeline_cfg['sources'][0]:
|
||||||
|
self.pipeline_cfg['sources'][0][field].extend(value)
|
||||||
|
else:
|
||||||
|
self.pipeline_cfg['sinks'][0][field].extend(value)
|
||||||
|
|
||||||
|
def _unset_pipeline_cfg(self, field):
|
||||||
|
if field in self.pipeline_cfg['sources'][0]:
|
||||||
|
del self.pipeline_cfg['sources'][0][field]
|
||||||
|
else:
|
||||||
|
del self.pipeline_cfg['sinks'][0][field]
|
||||||
|
|
||||||
|
def _exception_create_pipelinemanager(self):
|
||||||
|
self.assertRaises(pipeline.PipelineException,
|
||||||
|
pipeline.PipelineManager,
|
||||||
|
self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
|
||||||
|
def test_no_events(self):
|
||||||
|
self._unset_pipeline_cfg('events')
|
||||||
|
self._exception_create_pipelinemanager()
|
||||||
|
|
||||||
|
def test_no_name(self):
|
||||||
|
self._unset_pipeline_cfg('name')
|
||||||
|
self._exception_create_pipelinemanager()
|
||||||
|
|
||||||
|
def test_name(self):
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
for pipe in pipeline_manager.pipelines:
|
||||||
|
self.assertTrue(pipe.name.startswith('event:'))
|
||||||
|
|
||||||
|
def test_no_publishers(self):
|
||||||
|
self._unset_pipeline_cfg('publishers')
|
||||||
|
self._exception_create_pipelinemanager()
|
||||||
|
|
||||||
|
def test_check_events_include_exclude_same(self):
|
||||||
|
event_cfg = ['a', '!a']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
self._exception_create_pipelinemanager()
|
||||||
|
|
||||||
|
def test_check_events_include_exclude(self):
|
||||||
|
event_cfg = ['a', '!b']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
self._exception_create_pipelinemanager()
|
||||||
|
|
||||||
|
def test_check_events_wildcard_included(self):
|
||||||
|
event_cfg = ['a', '*']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
self._exception_create_pipelinemanager()
|
||||||
|
|
||||||
|
def test_check_publishers_invalid_publisher(self):
|
||||||
|
publisher_cfg = ['test_invalid']
|
||||||
|
self._set_pipeline_cfg('publishers', publisher_cfg)
|
||||||
|
|
||||||
|
def test_multiple_included_events(self):
|
||||||
|
event_cfg = ['a', 'b']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event])
|
||||||
|
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[0]
|
||||||
|
self.assertEqual(1, len(publisher.events))
|
||||||
|
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event2])
|
||||||
|
|
||||||
|
self.assertEqual(2, len(publisher.events))
|
||||||
|
self.assertEqual('a', getattr(publisher.events[0], 'event_type'))
|
||||||
|
self.assertEqual('b', getattr(publisher.events[1], 'event_type'))
|
||||||
|
|
||||||
|
def test_event_non_match(self):
|
||||||
|
event_cfg = ['nomatch']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event])
|
||||||
|
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[0]
|
||||||
|
self.assertEqual(0, len(publisher.events))
|
||||||
|
self.assertEqual(0, publisher.calls)
|
||||||
|
|
||||||
|
def test_wildcard_event(self):
|
||||||
|
event_cfg = ['*']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event])
|
||||||
|
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[0]
|
||||||
|
self.assertEqual(1, len(publisher.events))
|
||||||
|
self.assertEqual('a', getattr(publisher.events[0], 'event_type'))
|
||||||
|
|
||||||
|
def test_wildcard_excluded_events(self):
|
||||||
|
event_cfg = ['*', '!a']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
self.assertFalse(pipeline_manager.pipelines[0].support_event('a'))
|
||||||
|
|
||||||
|
def test_wildcard_excluded_events_not_excluded(self):
|
||||||
|
event_cfg = ['*', '!b']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event])
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[0]
|
||||||
|
self.assertEqual(1, len(publisher.events))
|
||||||
|
self.assertEqual('a', getattr(publisher.events[0], 'event_type'))
|
||||||
|
|
||||||
|
def test_all_excluded_events_not_excluded(self):
|
||||||
|
event_cfg = ['!b', '!c']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event])
|
||||||
|
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[0]
|
||||||
|
self.assertEqual(1, len(publisher.events))
|
||||||
|
self.assertEqual('a', getattr(publisher.events[0], 'event_type'))
|
||||||
|
|
||||||
|
def test_all_excluded_events_excluded(self):
|
||||||
|
event_cfg = ['!a', '!c']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
self.assertFalse(pipeline_manager.pipelines[0].support_event('a'))
|
||||||
|
self.assertTrue(pipeline_manager.pipelines[0].support_event('b'))
|
||||||
|
self.assertFalse(pipeline_manager.pipelines[0].support_event('c'))
|
||||||
|
|
||||||
|
def test_wildcard_and_excluded_wildcard_events(self):
|
||||||
|
event_cfg = ['*', '!compute.*']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
self.assertFalse(pipeline_manager.pipelines[0].
|
||||||
|
support_event('compute.instance.create.start'))
|
||||||
|
self.assertTrue(pipeline_manager.pipelines[0].
|
||||||
|
support_event('identity.user.create'))
|
||||||
|
|
||||||
|
def test_included_event_and_wildcard_events(self):
|
||||||
|
event_cfg = ['compute.instance.create.start', 'identity.*']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
self.assertTrue(pipeline_manager.pipelines[0].
|
||||||
|
support_event('identity.user.create'))
|
||||||
|
self.assertTrue(pipeline_manager.pipelines[0].
|
||||||
|
support_event('compute.instance.create.start'))
|
||||||
|
self.assertFalse(pipeline_manager.pipelines[0].
|
||||||
|
support_event('compute.instance.create.stop'))
|
||||||
|
|
||||||
|
def test_excluded_event_and_excluded_wildcard_events(self):
|
||||||
|
event_cfg = ['!compute.instance.create.start', '!identity.*']
|
||||||
|
self._set_pipeline_cfg('events', event_cfg)
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
self.assertFalse(pipeline_manager.pipelines[0].
|
||||||
|
support_event('identity.user.create'))
|
||||||
|
self.assertFalse(pipeline_manager.pipelines[0].
|
||||||
|
support_event('compute.instance.create.start'))
|
||||||
|
self.assertTrue(pipeline_manager.pipelines[0].
|
||||||
|
support_event('compute.instance.create.stop'))
|
||||||
|
|
||||||
|
def test_multiple_pipeline(self):
|
||||||
|
self._augment_pipeline_cfg()
|
||||||
|
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event, self.test_event2])
|
||||||
|
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[0]
|
||||||
|
self.assertEqual(1, len(publisher.events))
|
||||||
|
self.assertEqual(1, publisher.calls)
|
||||||
|
self.assertEqual('a', getattr(publisher.events[0], 'event_type'))
|
||||||
|
new_publisher = pipeline_manager.pipelines[1].publishers[0]
|
||||||
|
self.assertEqual(1, len(new_publisher.events))
|
||||||
|
self.assertEqual(1, new_publisher.calls)
|
||||||
|
self.assertEqual('b', getattr(new_publisher.events[0], 'event_type'))
|
||||||
|
|
||||||
|
def test_multiple_publisher(self):
|
||||||
|
self._set_pipeline_cfg('publishers', ['test://', 'new://'])
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event])
|
||||||
|
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[0]
|
||||||
|
new_publisher = pipeline_manager.pipelines[0].publishers[1]
|
||||||
|
self.assertEqual(1, len(publisher.events))
|
||||||
|
self.assertEqual(1, len(new_publisher.events))
|
||||||
|
self.assertEqual('a', getattr(new_publisher.events[0], 'event_type'))
|
||||||
|
self.assertEqual('a', getattr(publisher.events[0], 'event_type'))
|
||||||
|
|
||||||
|
def test_multiple_publisher_isolation(self):
|
||||||
|
self._reraise_exception = False
|
||||||
|
self._set_pipeline_cfg('publishers', ['except://', 'new://'])
|
||||||
|
pipeline_manager = pipeline.PipelineManager(self.pipeline_cfg,
|
||||||
|
self.transformer_manager,
|
||||||
|
self.p_type)
|
||||||
|
with pipeline_manager.publisher(None) as p:
|
||||||
|
p([self.test_event])
|
||||||
|
|
||||||
|
publisher = pipeline_manager.pipelines[0].publishers[1]
|
||||||
|
self.assertEqual(1, len(publisher.events))
|
||||||
|
self.assertEqual('a', getattr(publisher.events[0], 'event_type'))
|
||||||
|
|
||||||
|
def test_unique_pipeline_names(self):
|
||||||
|
self._dup_pipeline_name_cfg()
|
||||||
|
self._exception_create_pipelinemanager()
|
13
etc/ceilometer/event_pipeline.yaml
Normal file
13
etc/ceilometer/event_pipeline.yaml
Normal file
@ -0,0 +1,13 @@
|
|||||||
|
---
|
||||||
|
sources:
|
||||||
|
- name: event_source
|
||||||
|
events:
|
||||||
|
- "*"
|
||||||
|
sinks:
|
||||||
|
- event_sink
|
||||||
|
sinks:
|
||||||
|
- name: event_sink
|
||||||
|
transformers:
|
||||||
|
triggers:
|
||||||
|
publishers:
|
||||||
|
- direct://
|
Loading…
Reference in New Issue
Block a user