Implements complex query functionality for samples

New API resource /query/samples has been added

Implements: blueprint complex-filter-expressions-in-api-queries
Change-Id: I5b324f390eb3ed63514260683c1576a0fc0cb7af
This commit is contained in:
Ildiko Vancsa 2013-12-14 12:43:30 +01:00
parent 1004191f1c
commit f8e64ec982
10 changed files with 1120 additions and 38 deletions

View File

@ -29,6 +29,7 @@ import datetime
import functools
import inspect
import json
import jsonschema
import uuid
from oslo.config import cfg
@ -990,6 +991,216 @@ class SamplesController(rest.RestController):
return Sample.from_db_model(samples[0])
class ComplexQuery(_Base):
"""Holds a sample query encoded in json."""
filter = wtypes.text
"The filter expression encoded in json."
orderby = wtypes.text
"List of single-element dicts for specifing the ordering of the results."
limit = int
"The maximum number of results to be returned."
@classmethod
def sample(cls):
return cls(filter='{\"and\": [{\"and\": [{\"=\": ' +
'{\"counter_name\": \"cpu_util\"}}, ' +
'{\">\": {\"counter_volume\": 0.23}}, ' +
'{\"<\": {\"counter_volume\": 0.26}}]}, ' +
'{\"or\": [{\"and\": [{\">\": ' +
'{\"timestamp\": \"2013-12-01T18:00:00\"}}, ' +
'{\"<\": ' +
'{\"timestamp\": \"2013-12-01T18:15:00\"}}]}, ' +
'{\"and\": [{\">\": ' +
'{\"timestamp\": \"2013-12-01T18:30:00\"}}, ' +
'{\"<\": ' +
'{\"timestamp\": \"2013-12-01T18:45:00\"}}]}]}]}',
orderby='[{\"counter_volume\": \"ASC\"}, ' +
'{\"timestamp\": \"DESC\"}]',
limit=42
)
def _list_to_regexp(items):
regexp = ["^%s$" % item for item in items]
regexp = "|".join(regexp)
regexp = "(?i)" + regexp
return regexp
class ValidatedComplexQuery(object):
complex_operators = ["and", "or"]
order_directions = ["asc", "desc"]
simple_ops = ["=", "!=", "<", ">", "<=", "=<", ">=", "=>"]
complex_ops = _list_to_regexp(complex_operators)
simple_ops = _list_to_regexp(simple_ops)
order_directions = _list_to_regexp(order_directions)
schema_value = {
"oneOf": [{"type": "string"},
{"type": "number"}],
"minProperties": 1,
"maxProperties": 1}
schema_field = {
"type": "object",
"patternProperties": {"[\S]+": schema_value},
"additionalProperties": False,
"minProperties": 1,
"maxProperties": 1}
schema_leaf = {
"type": "object",
"patternProperties": {simple_ops: schema_field},
"additionalProperties": False,
"minProperties": 1,
"maxProperties": 1}
schema_and_or_array = {
"type": "array",
"items": {"$ref": "#"},
"minItems": 2}
schema_and_or = {
"type": "object",
"patternProperties": {complex_ops: schema_and_or_array},
"additionalProperties": False,
"minProperties": 1,
"maxProperties": 1}
schema = {
"oneOf": [{"$ref": "#/definitions/leaf"},
{"$ref": "#/definitions/and_or"}],
"minProperties": 1,
"maxProperties": 1,
"definitions": {"leaf": schema_leaf,
"and_or": schema_and_or}}
orderby_schema = {
"type": "array",
"items": {
"type": "object",
"patternProperties":
{"[\S]+":
{"type": "string",
"pattern": order_directions}},
"additionalProperties": False,
"minProperties": 1,
"maxProperties": 1}}
timestamp_fields = ["timestamp"]
def __init__(self, query):
self.original_query = query
def validate(self, visibility_field):
"""Validates the query content and does the necessary transformations.
"""
if self.original_query.filter is wtypes.Unset:
self.filter_expr = None
else:
self.filter_expr = json.loads(self.original_query.filter)
self._validate_filter(self.filter_expr)
self._replace_isotime_with_datetime(self.filter_expr)
self._convert_operator_to_lower_case(self.filter_expr)
self._force_visibility(visibility_field)
if self.original_query.orderby is wtypes.Unset:
self.orderby = None
else:
self.orderby = json.loads(self.original_query.orderby)
self._validate_orderby(self.orderby)
self._convert_orderby_to_lower_case(self.orderby)
if self.original_query.limit is wtypes.Unset:
self.limit = None
else:
self.limit = self.original_query.limit
if self.limit is not None and self.limit <= 0:
msg = _('Limit should be positive')
raise ClientSideError(msg)
@staticmethod
def _convert_orderby_to_lower_case(orderby):
for orderby_field in orderby:
utils.lowercase_values(orderby_field)
def _traverse_postorder(self, tree, visitor):
op = tree.keys()[0]
if op.lower() in self.complex_operators:
for i, operand in enumerate(tree[op]):
self._traverse_postorder(operand, visitor)
visitor(tree)
def _check_cross_project_references(self, own_project_id,
visibility_field):
"""Do not allow other than own_project_id
"""
def check_project_id(subfilter):
op = subfilter.keys()[0]
if (op.lower() not in self.complex_operators
and subfilter[op].keys()[0] == visibility_field
and subfilter[op][visibility_field] != own_project_id):
raise ProjectNotAuthorized(subfilter[op][visibility_field])
self._traverse_postorder(self.filter_expr, check_project_id)
def _force_visibility(self, visibility_field):
"""If the tenant is not admin insert an extra
"and <visibility_field>=<tenant's project_id>" clause to the query
"""
authorized_project = acl.get_limited_to_project(pecan.request.headers)
is_admin = authorized_project is None
if not is_admin:
self._restrict_to_project(authorized_project, visibility_field)
self._check_cross_project_references(authorized_project,
visibility_field)
def _restrict_to_project(self, project_id, visibility_field):
restriction = {"=": {visibility_field: project_id}}
if self.filter_expr is None:
self.filter_expr = restriction
else:
self.filter_expr = {"and": [restriction, self.filter_expr]}
def _replace_isotime_with_datetime(self, filter_expr):
def replace_isotime(subfilter):
op = subfilter.keys()[0]
if (op.lower() not in self.complex_operators
and subfilter[op].keys()[0] in self.timestamp_fields):
field = subfilter[op].keys()[0]
date_time = self._convert_to_datetime(subfilter[op][field])
subfilter[op][field] = date_time
self._traverse_postorder(filter_expr, replace_isotime)
def _convert_operator_to_lower_case(self, filter_expr):
self._traverse_postorder(filter_expr, utils.lowercase_keys)
@staticmethod
def _convert_to_datetime(isotime):
try:
date_time = timeutils.parse_isotime(isotime)
date_time = date_time.replace(tzinfo=None)
return date_time
except ValueError:
LOG.exception(_("String %s is not a valid isotime") % isotime)
msg = _('Failed to parse the timestamp value %s') % isotime
raise ClientSideError(msg)
def _validate_filter(self, filter_expr):
jsonschema.validate(filter_expr, self.schema)
def _validate_orderby(self, orderby_expr):
jsonschema.validate(orderby_expr, self.orderby_schema)
class Resource(_Base):
"""An externally defined object for which samples have been received.
"""
@ -1852,6 +2063,29 @@ class EventsController(rest.RestController):
traits=event.traits)
class QuerySamplesController(rest.RestController):
"""Provides complex query possibilities for samples
"""
@wsme_pecan.wsexpose([Sample], body=ComplexQuery)
def post(self, body):
"""Define query for retrieving Sample data.
:param body: Query rules for the samples to be returned.
"""
query = ValidatedComplexQuery(body)
query.validate(visibility_field="project_id")
conn = pecan.request.storage_conn
return [Sample.from_db_model(s)
for s in conn.query_samples(query.filter_expr,
query.orderby,
query.limit)]
class QueryController(rest.RestController):
samples = QuerySamplesController()
class V2Controller(object):
"""Version 2 API controller root."""
@ -1861,3 +2095,5 @@ class V2Controller(object):
alarms = AlarmsController()
event_types = EventTypesController()
events = EventsController()
query = QueryController()

View File

@ -317,4 +317,17 @@ class Connection(object):
:param event_type: the type of the Event to filter by
:param trait_type: the name of the Trait to filter by
"""
raise NotImplementedError(_('Events not implemented.'))
@staticmethod
def query_samples(filter_expr=None, orderby=None, limit=None):
"""Return an iterable of model.Sample objects.
:param filter_expr: Filter expression for query.
:param orderby: List of field name and direction pairs for order by.
:param limit: Maximum number of results to return.
"""
raise NotImplementedError(_('Complex query for samples \
is not implemented.'))

View File

@ -338,6 +338,19 @@ class Connection(base.Connection):
return merge;
}""")
operators = {"<": "$lt",
">": "$gt",
"<=": "$lte",
"=<": "$lte",
">=": "$gte",
"=>": "$gte",
"!=": "$ne"}
complex_operators = {"or": "$or",
"and": "$and"}
ordering_functions = {"asc": pymongo.ASCENDING,
"desc": pymongo.DESCENDING}
def __init__(self, conf):
url = conf.database.connection
@ -734,21 +747,14 @@ class Connection(base.Connection):
user_id=r['user_id'],
)
def get_samples(self, sample_filter, limit=None):
"""Return an iterable of model.Sample instances.
:param sample_filter: Filter.
:param limit: Maximum number of results to return.
"""
if limit == 0:
return
q = make_query_from_filter(sample_filter, require_meter=False)
if limit:
samples = self.db.meter.find(
q, limit=limit, sort=[("timestamp", pymongo.DESCENDING)])
def _retrieve_samples(self, query, orderby, limit):
if limit is not None:
samples = self.db.meter.find(query,
limit=limit,
sort=orderby)
else:
samples = self.db.meter.find(
q, sort=[("timestamp", pymongo.DESCENDING)])
samples = self.db.meter.find(query,
sort=orderby)
for s in samples:
# Remove the ObjectId generated by the database when
@ -759,6 +765,72 @@ class Connection(base.Connection):
s['counter_unit'] = s.get('counter_unit', '')
yield models.Sample(**s)
def get_samples(self, sample_filter, limit=None):
"""Return an iterable of model.Sample instances.
:param sample_filter: Filter.
:param limit: Maximum number of results to return.
"""
if limit == 0:
return []
q = make_query_from_filter(sample_filter, require_meter=False)
return self._retrieve_samples(q,
[("timestamp", pymongo.DESCENDING)],
limit)
def query_samples(self, filter_expr=None, orderby=None, limit=None):
if limit == 0:
return []
query_filter = {}
orderby_filter = [("timestamp", pymongo.DESCENDING)]
if orderby is not None:
orderby_filter = self._transform_orderby(orderby)
if filter_expr is not None:
query_filter = self._transform_filter(
filter_expr)
return self._retrieve_samples(query_filter, orderby_filter, limit)
def _transform_orderby(self, orderby):
orderby_filter = []
for field in orderby:
field_name = field.keys()[0]
ordering = self.ordering_functions[field.values()[0]]
orderby_filter.append((field_name, ordering))
return orderby_filter
def _transform_filter(self, condition):
def process_json_tree(condition_tree):
operator_node = condition_tree.keys()[0]
nodes = condition_tree.values()[0]
if operator_node in self.complex_operators:
element_list = []
for node in nodes:
element = process_json_tree(node)
element_list.append(element)
complex_operator = self.complex_operators[operator_node]
op = {complex_operator: element_list}
return op
else:
field_name = nodes.keys()[0]
field_value = nodes.values()[0]
# no operator for equal in Mongo
if operator_node == "=":
op = {field_name: field_value}
return op
if operator_node in self.operators:
operator = self.operators[operator_node]
op = {
field_name: {
operator: field_value}}
return op
return process_json_tree(condition)
def get_meter_statistics(self, sample_filter, period=None, groupby=None):
"""Return an iterable of models.Statistics instance containing meter
statistics described by the query parameters.

View File

@ -24,8 +24,10 @@ import os
import types
from sqlalchemy import and_
from sqlalchemy import asc
from sqlalchemy import desc
from sqlalchemy import func
from sqlalchemy import or_
from sqlalchemy.orm import aliased
from ceilometer.openstack.common.db import exception as dbexc
@ -170,6 +172,19 @@ def make_query_from_filter(session, query, sample_filter, require_meter=True):
class Connection(base.Connection):
"""SqlAlchemy connection."""
operators = {"=": operator.eq,
"<": operator.lt,
">": operator.gt,
"<=": operator.le,
"=<": operator.le,
">=": operator.ge,
"=>": operator.ge,
"!=": operator.ne}
complex_operators = {"or": or_,
"and": and_}
ordering_functions = {"asc": asc,
"desc": desc}
def __init__(self, conf):
url = conf.database.connection
if url == 'sqlite://':
@ -508,23 +523,11 @@ class Connection(base.Connection):
source=resource.sources[0].id,
user_id=resource.user_id)
def get_samples(self, sample_filter, limit=None):
"""Return an iterable of api_models.Samples.
:param sample_filter: Filter.
:param limit: Maximum number of results to return.
"""
if limit == 0:
return
session = self._get_db_session()
query = session.query(models.Meter)
query = make_query_from_filter(session, query, sample_filter,
require_meter=False)
if limit:
def _retrieve_samples(self, query, orderby, limit, table):
query = self._apply_order_by(query, orderby, table)
if limit is not None:
query = query.limit(limit)
samples = query.from_self()\
.order_by(desc(models.Meter.timestamp)).all()
samples = query.all()
for s in samples:
# Remove the id generated by the database when
@ -548,6 +551,67 @@ class Connection(base.Connection):
message_signature=s.message_signature,
)
def get_samples(self, sample_filter, limit=None):
"""Return an iterable of api_models.Samples.
:param sample_filter: Filter.
:param limit: Maximum number of results to return.
"""
if limit == 0:
return []
table = models.Meter
session = self._get_db_session()
query = session.query(table)
query = make_query_from_filter(session, query, sample_filter,
require_meter=False)
return self._retrieve_samples(query, None, limit, table)
def query_samples(self, filter_expr=None, orderby=None, limit=None):
if limit == 0:
return []
table = models.Meter
session = self._get_db_session()
query = session.query(table)
if filter_expr is not None:
sql_condition = self._transform_expression(filter_expr,
table)
query = query.filter(sql_condition)
return self._retrieve_samples(query, orderby, limit, table)
def _transform_expression(self, expression_tree, table):
def transform(sub_tree):
operator = sub_tree.keys()[0]
nodes = sub_tree.values()[0]
if operator in self.complex_operators:
op = self.complex_operators[operator]
element_list = []
for node in nodes:
element = transform(node)
element_list.append(element)
return op(*element_list)
else:
op = self.operators[operator]
return op(getattr(table, nodes.keys()[0]), nodes.values()[0])
return transform(expression_tree)
def _apply_order_by(self, query, orderby, table):
if orderby is not None:
for field in orderby:
ordering_function = self.ordering_functions[field.values()[0]]
query = query.order_by(ordering_function(
getattr(table, field.keys()[0])))
else:
query = query.order_by(desc(table.timestamp))
return query
def _make_stats_query(self, sample_filter, groupby):
select = [
models.Meter.counter_unit.label('unit'),

View File

@ -0,0 +1,237 @@
# -*- encoding: utf-8 -*-
#
# Copyright Ericsson AB 2013. All rights reserved
#
# Authors: Ildiko Vancsa <ildiko.vancsa@ericsson.com>
# Balazs Gibizer <balazs.gibizer@ericsson.com>
#
# 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.
"""Test the methods related to complex query."""
import datetime
import fixtures
import jsonschema
import mock
import wsme
from ceilometer.api.controllers import v2 as api
from ceilometer.openstack.common import test
class FakeComplexQuery(api.ValidatedComplexQuery):
def __init__(self):
super(FakeComplexQuery, self).__init__(query=None)
class TestComplexQuery(test.BaseTestCase):
def setUp(self):
super(TestComplexQuery, self).setUp()
self.useFixture(fixtures.MonkeyPatch(
'pecan.response', mock.MagicMock()))
self.query = api.ValidatedComplexQuery(FakeComplexQuery())
def test_replace_isotime_utc(self):
filter_expr = {"=": {"timestamp": "2013-12-05T19:38:29Z"}}
self.query._replace_isotime_with_datetime(filter_expr)
self.assertEqual(datetime.datetime(2013, 12, 5, 19, 38, 29),
filter_expr["="]["timestamp"])
def test_replace_isotime_timezone_removed(self):
filter_expr = {"=": {"timestamp": "2013-12-05T20:38:29+01:00"}}
self.query._replace_isotime_with_datetime(filter_expr)
self.assertEqual(datetime.datetime(2013, 12, 5, 20, 38, 29),
filter_expr["="]["timestamp"])
def test_replace_isotime_wrong_syntax(self):
filter_expr = {"=": {"timestamp": "not a valid isotime string"}}
self.assertRaises(wsme.exc.ClientSideError,
self.query._replace_isotime_with_datetime,
filter_expr)
def test_replace_isotime_in_complex_filter(self):
filter_expr = {"and": [{"=": {"timestamp": "2013-12-05T19:38:29Z"}},
{"=": {"timestamp": "2013-12-06T19:38:29Z"}}]}
self.query._replace_isotime_with_datetime(filter_expr)
self.assertEqual(datetime.datetime(2013, 12, 5, 19, 38, 29),
filter_expr["and"][0]["="]["timestamp"])
self.assertEqual(datetime.datetime(2013, 12, 6, 19, 38, 29),
filter_expr["and"][1]["="]["timestamp"])
def test_replace_isotime_in_complex_filter_with_unbalanced_tree(self):
subfilter = {"and": [{"=": {"project_id": 42}},
{"=": {"timestamp": "2013-12-06T19:38:29Z"}}]}
filter_expr = {"or": [{"=": {"timestamp": "2013-12-05T19:38:29Z"}},
subfilter]}
self.query._replace_isotime_with_datetime(filter_expr)
self.assertEqual(datetime.datetime(2013, 12, 5, 19, 38, 29),
filter_expr["or"][0]["="]["timestamp"])
self.assertEqual(datetime.datetime(2013, 12, 6, 19, 38, 29),
filter_expr["or"][1]["and"][1]["="]["timestamp"])
def test_convert_operator_to_lower_case(self):
filter_expr = {"AND": [{"=": {"project_id": 42}},
{"=": {"project_id": 44}}]}
self.query._convert_operator_to_lower_case(filter_expr)
self.assertEqual("and", filter_expr.keys()[0])
filter_expr = {"Or": [{"=": {"project_id": 43}},
{"anD": [{"=": {"project_id": 44}},
{"=": {"project_id": 42}}]}]}
self.query._convert_operator_to_lower_case(filter_expr)
self.assertEqual("or", filter_expr.keys()[0])
self.assertEqual("and", filter_expr["or"][1].keys()[0])
def test_convert_orderby(self):
orderby = []
self.query._convert_orderby_to_lower_case(orderby)
self.assertEqual([], orderby)
orderby = [{"field1": "DESC"}]
self.query._convert_orderby_to_lower_case(orderby)
self.assertEqual([{"field1": "desc"}], orderby)
orderby = [{"field1": "ASC"}, {"field2": "DESC"}]
self.query._convert_orderby_to_lower_case(orderby)
self.assertEqual([{"field1": "asc"}, {"field2": "desc"}], orderby)
def test_validate_orderby_empty_direction(self):
orderby = [{"field1": ""}]
self.assertRaises(jsonschema.ValidationError,
self.query._validate_orderby,
orderby)
orderby = [{"field1": "asc"}, {"field2": ""}]
self.assertRaises(jsonschema.ValidationError,
self.query._validate_orderby,
orderby)
def test_validate_orderby_wrong_order_string(self):
orderby = [{"field1": "not a valid order"}]
self.assertRaises(jsonschema.ValidationError,
self.query._validate_orderby,
orderby)
def test_validate_orderby_wrong_multiple_item_order_string(self):
orderby = [{"field2": "not a valid order"}, {"field1": "ASC"}]
self.assertRaises(jsonschema.ValidationError,
self.query._validate_orderby,
orderby)
class TestFilterSyntaxValidation(test.BaseTestCase):
def setUp(self):
super(TestFilterSyntaxValidation, self).setUp()
self.query = api.ValidatedComplexQuery(FakeComplexQuery())
def test_simple_operator(self):
filter = {"=": {"field_name": "string_value"}}
self.query._validate_filter(filter)
filter = {"=>": {"field_name": "string_value"}}
self.query._validate_filter(filter)
def test_invalid_simple_operator(self):
filter = {"==": {"field_name": "string_value"}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
filter = {"": {"field_name": "string_value"}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_more_than_one_operator_is_invalid(self):
filter = {"=": {"field_name": "string_value"},
"<": {"": ""}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_empty_expression_is_invalid(self):
filter = {}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_invalid_field_name(self):
filter = {"=": {"": "value"}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
filter = {"=": {" ": "value"}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
filter = {"=": {"\t": "value"}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_more_than_one_field_is_invalid(self):
filter = {"=": {"field": "value", "field2": "value"}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_missing_field_after_simple_op_is_invalid(self):
filter = {"=": {}}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_and_or(self):
filter = {"and": [{"=": {"field_name": "string_value"}},
{"=": {"field2": "value"}}]}
self.query._validate_filter(filter)
filter = {"or": [{"and": [{"=": {"field_name": "string_value"}},
{"=": {"field2": "value"}}]},
{"=": {"field3": "value"}}]}
self.query._validate_filter(filter)
filter = {"or": [{"and": [{"=": {"field_name": "string_value"}},
{"=": {"field2": "value"}},
{"<": {"field3": 42}}]},
{"=": {"field3": "value"}}]}
self.query._validate_filter(filter)
def test_invalid_complex_operator(self):
filter = {"xor": [{"=": {"field_name": "string_value"}},
{"=": {"field2": "value"}}]}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_and_or_with_one_child_is_invalid(self):
filter = {"or": [{"=": {"field_name": "string_value"}}]}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_complex_operator_with_zero_child_is_invalid(self):
filter = {"or": []}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)
def test_more_than_one_complex_operator_is_invalid(self):
filter = {"and": [{"=": {"field_name": "string_value"}},
{"=": {"field2": "value"}}],
"or": [{"=": {"field_name": "string_value"}},
{"=": {"field2": "value"}}]}
self.assertRaises(jsonschema.ValidationError,
self.query._validate_filter,
filter)

View File

@ -0,0 +1,192 @@
# -*- encoding: utf-8 -*-
#
# Copyright Ericsson AB 2013. All rights reserved
#
# Authors: Ildiko Vancsa <ildiko.vancsa@ericsson.com>
# Balazs Gibizer <balazs.gibizer@ericsson.com>
#
# 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 complex queries for samples
"""
import datetime
import logging
import testscenarios
from ceilometer.openstack.common import timeutils
from ceilometer.publisher import utils
from ceilometer import sample
from ceilometer.tests.api import v2 as tests_api
from ceilometer.tests import db as tests_db
load_tests = testscenarios.load_tests_apply_scenarios
LOG = logging.getLogger(__name__)
class TestQueryMetersController(tests_api.FunctionalTest,
tests_db.MixinTestsWithBackendScenarios):
def setUp(self):
super(TestQueryMetersController, self).setUp()
self.url = '/query/samples'
self.admin_header = {"X-Roles": "admin",
"X-Project-Id":
"project-id1"}
self.non_admin_header = {"X-Roles": "Member",
"X-Project-Id":
"project-id1"}
for cnt in [
sample.Sample('meter.test',
'cumulative',
'',
1,
'user-id',
'project-id1',
'resource-id1',
timestamp=datetime.datetime(2012, 7, 2, 10, 40),
resource_metadata={'display_name': 'test-server',
'tag': 'self.sample',
'size': 123,
'util': 0.75,
'is_public': True},
source='test_source'),
sample.Sample('meter.test',
'cumulative',
'',
1,
'user-id',
'project-id2',
'resource-id2',
timestamp=datetime.datetime(2012, 7, 2, 10, 41),
resource_metadata={'display_name': 'test-server',
'tag': 'self.sample',
'size': 123,
'util': 0.75,
'is_public': True},
source='test_source')]:
msg = utils.meter_message_from_counter(
cnt,
self.CONF.publisher.metering_secret)
self.conn.record_metering_data(msg)
def test_query_fields_are_optional(self):
data = self.post_json(self.url, params={})
self.assertEqual(2, len(data.json))
def test_query_with_isotime(self):
date_time = datetime.datetime(2012, 7, 2, 10, 41)
isotime = date_time.isoformat()
data = self.post_json(self.url,
params={"filter":
'{">=": {"timestamp": "'
+ isotime + '"}}'})
self.assertEqual(1, len(data.json))
for sample in data.json:
result_time = timeutils.parse_isotime(sample['timestamp'])
result_time = result_time.replace(tzinfo=None)
self.assertTrue(result_time >= date_time)
def test_non_admin_tenant_sees_only_its_own_project(self):
data = self.post_json(self.url,
params={},
headers=self.non_admin_header)
for sample in data.json:
self.assertEqual("project-id1", sample['project_id'])
def test_non_admin_tenant_cannot_query_others_project(self):
data = self.post_json(self.url,
params={"filter":
'{"=": {"project_id": "project-id2"}}'},
expect_errors=True,
headers=self.non_admin_header)
self.assertEqual(401, data.status_int)
self.assertIn("Not Authorized to access project project-id2",
data.body)
def test_non_admin_tenant_can_explicitly_filter_for_own_project(self):
data = self.post_json(self.url,
params={"filter":
'{"=": {"project_id": "project-id1"}}'},
headers=self.non_admin_header)
for sample in data.json:
self.assertEqual("project-id1", sample['project_id'])
def test_admin_tenant_sees_every_project(self):
data = self.post_json(self.url,
params={},
headers=self.admin_header)
self.assertEqual(2, len(data.json))
for sample in data.json:
self.assertIn(sample['project_id'],
(["project-id1", "project-id2"]))
def test_admin_tenant_sees_every_project_with_complex_filter(self):
filter = ('{"OR": ' +
'[{"=": {"project_id": "project-id1"}}, ' +
'{"=": {"project_id": "project-id2"}}]}')
data = self.post_json(self.url,
params={"filter": filter},
headers=self.admin_header)
self.assertEqual(2, len(data.json))
for sample in data.json:
self.assertIn(sample['project_id'],
(["project-id1", "project-id2"]))
def test_admin_tenant_can_query_any_project(self):
data = self.post_json(self.url,
params={"filter":
'{"=": {"project_id": "project-id2"}}'},
headers=self.admin_header)
self.assertEqual(1, len(data.json))
for sample in data.json:
self.assertIn(sample['project_id'], set(["project-id2"]))
def test_query_with_orderby(self):
data = self.post_json(self.url,
params={"orderby": '[{"project_id": "DESC"}]'})
self.assertEqual(2, len(data.json))
self.assertEqual(["project-id2", "project-id1"],
[s["project_id"] for s in data.json])
def test_query_with_lower_and_upper_case_orderby(self):
data = self.post_json(self.url,
params={"orderby": '[{"project_id": "DeSc"}]'})
self.assertEqual(2, len(data.json))
self.assertEqual(["project-id2", "project-id1"],
[s["project_id"] for s in data.json])
def test_query_with_missing_order_in_orderby(self):
data = self.post_json(self.url,
params={"orderby": '[{"project_id": ""}]'},
expect_errors=True)
self.assertEqual(500, data.status_int)
def test_limit_should_be_positive(self):
data = self.post_json(self.url,
params={"limit": 0},
expect_errors=True)
self.assertEqual(400, data.status_int)
self.assertIn("Limit should be positive", data.body)

View File

@ -708,6 +708,181 @@ class RawSampleTest(DBTestBase,
self.assertEqual(len(results), 2)
class ComplexSampleQueryTest(DBTestBase,
tests_db.MixinTestsWithBackendScenarios):
def setUp(self):
super(ComplexSampleQueryTest, self).setUp()
self.complex_filter = {"and":
[{"or":
[{"=": {"resource_id": "resource-id-42"}},
{"=": {"resource_id": "resource-id-44"}}]},
{"and":
[{"=": {"counter_name": "cpu_util"}},
{"and":
[{">": {"counter_volume": 0.4}},
{"<=": {"counter_volume": 0.8}}]}]}]}
or_expression = [{"=": {"resource_id": "resource-id-42"}},
{"=": {"resource_id": "resource-id-43"}},
{"=": {"resource_id": "resource-id-44"}}]
and_expression = [{">": {"counter_volume": 0.4}},
{"<=": {"counter_volume": 0.8}}]
self.complex_filter_list = {"and":
[{"or": or_expression},
{"and":
[{"=": {"counter_name": "cpu_util"}},
{"and": and_expression}]}]}
def _create_samples(self):
for resource in range(42, 45):
for volume in [0.79, 0.41, 0.4, 0.8, 0.39, 0.81]:
self.create_and_store_sample(resource_id="resource-id-%s"
% resource,
name="cpu_util",
volume=volume)
def test_no_filter(self):
results = list(self.conn.query_samples())
self.assertEqual(len(results), len(self.msgs))
for sample in results:
self.assertIn(sample.as_dict(), self.msgs)
def test_no_filter_with_zero_limit(self):
limit = 0
results = list(self.conn.query_samples(limit=limit))
self.assertEqual(len(results), limit)
def test_no_filter_with_limit(self):
limit = 3
results = list(self.conn.query_samples(limit=limit))
self.assertEqual(len(results), limit)
def test_query_simple_filter(self):
simple_filter = {"=": {"resource_id": "resource-id-8"}}
results = list(self.conn.query_samples(filter_expr=simple_filter))
self.assertEqual(len(results), 1)
for sample in results:
self.assertEqual(sample.resource_id, "resource-id-8")
def test_query_simple_filter_with_not_equal_relation(self):
simple_filter = {"!=": {"resource_id": "resource-id-8"}}
results = list(self.conn.query_samples(filter_expr=simple_filter))
self.assertEqual(len(results), len(self.msgs) - 1)
for sample in results:
self.assertNotEqual(sample.resource_id, "resource-id-8")
def test_query_complex_filter(self):
self._create_samples()
results = list(self.conn.query_samples(filter_expr=
self.complex_filter))
self.assertEqual(len(results), 6)
for sample in results:
self.assertIn(sample.resource_id,
set(["resource-id-42", "resource-id-44"]))
self.assertEqual(sample.counter_name,
"cpu_util")
self.assertTrue(sample.counter_volume > 0.4)
self.assertTrue(sample.counter_volume <= 0.8)
def test_query_complex_filter_with_limit(self):
self._create_samples()
limit = 3
results = list(self.conn.query_samples(filter_expr=self.complex_filter,
limit=limit))
self.assertEqual(len(results), limit)
def test_query_complex_filter_with_simple_orderby(self):
self._create_samples()
expected_volume_order = [0.41, 0.41, 0.79, 0.79, 0.8, 0.8]
orderby = [{"counter_volume": "asc"}]
results = list(self.conn.query_samples(filter_expr=self.complex_filter,
orderby=orderby))
self.assertEqual(expected_volume_order,
[s.counter_volume for s in results])
def test_query_complex_filter_with_complex_orderby(self):
self._create_samples()
expected_volume_order = [0.41, 0.41, 0.79, 0.79, 0.8, 0.8]
expected_resource_id_order = ["resource-id-44", "resource-id-42",
"resource-id-44", "resource-id-42",
"resource-id-44", "resource-id-42"]
orderby = [{"counter_volume": "asc"}, {"resource_id": "desc"}]
results = list(self.conn.query_samples(filter_expr=self.complex_filter,
orderby=orderby))
self.assertEqual(expected_volume_order,
[s.counter_volume for s in results])
self.assertEqual(expected_resource_id_order,
[s.resource_id for s in results])
def test_query_complex_filter_with_list(self):
self._create_samples()
results = list(
self.conn.query_samples(filter_expr=self.complex_filter_list))
self.assertEqual(len(results), 9)
for sample in results:
self.assertIn(sample.resource_id,
set(["resource-id-42",
"resource-id-43",
"resource-id-44"]))
self.assertEqual(sample.counter_name,
"cpu_util")
self.assertTrue(sample.counter_volume > 0.4)
self.assertTrue(sample.counter_volume <= 0.8)
def test_query_complex_filter_with_list_with_limit(self):
self._create_samples()
limit = 3
results = list(
self.conn.query_samples(filter_expr=self.complex_filter_list,
limit=limit))
self.assertEqual(len(results), limit)
def test_query_complex_filter_with_list_with_simple_orderby(self):
self._create_samples()
expected_volume_order = [0.41, 0.41, 0.41, 0.79, 0.79,
0.79, 0.8, 0.8, 0.8]
orderby = [{"counter_volume": "asc"}]
results = list(
self.conn.query_samples(filter_expr=self.complex_filter_list,
orderby=orderby))
self.assertEqual(expected_volume_order,
[s.counter_volume for s in results])
def test_query_complex_filterwith_list_with_complex_orderby(self):
self._create_samples()
expected_volume_order = [0.41, 0.41, 0.41, 0.79, 0.79,
0.79, 0.8, 0.8, 0.8]
expected_resource_id_order = ["resource-id-44", "resource-id-43",
"resource-id-42", "resource-id-44",
"resource-id-43", "resource-id-42",
"resource-id-44", "resource-id-43",
"resource-id-42"]
orderby = [{"counter_volume": "asc"}, {"resource_id": "desc"}]
results = list(
self.conn.query_samples(filter_expr=self.complex_filter_list,
orderby=orderby))
self.assertEqual(expected_volume_order,
[s.counter_volume for s in results])
self.assertEqual(expected_resource_id_order,
[s.resource_id for s in results])
def test_query_complex_filter_with_wrong_order_in_orderby(self):
self._create_samples()
orderby = [{"counter_volume": "not valid order"},
{"resource_id": "desc"}]
query = lambda: list(self.conn.query_samples(filter_expr=
self.complex_filter,
orderby=orderby))
self.assertRaises(KeyError, query)
class StatisticsTest(DBTestBase,
tests_db.MixinTestsWithBackendScenarios):

View File

@ -116,3 +116,18 @@ def dict_to_keyval(value, key_base=None):
yield key_gen, v
else:
yield key_gen, v
def lowercase_keys(mapping):
"""Converts the values of the keys in mapping to lowercase."""
items = mapping.items()
for key, value in items:
del mapping[key]
mapping[key.lower()] = value
def lowercase_values(mapping):
"""Converts the values in the mapping dict to lowercase."""
items = mapping.items()
for key, value in items:
mapping[key] = value.lower()

View File

@ -62,24 +62,54 @@ Alarms
Filtering Queries
=================
Ceilometer's REST API currently supports two types of queries. The Simple
Query functionality provides simple filtering on several fields of the
*Sample* type. Complex Query provides the possibility to specify queries
with logical and comparison operators on the fields of *Sample*.
You may also apply filters based on the values of one or more of the
*resource_metadata* field, which you can identify by using *metadata.<field>*
syntax in either type of query. Note, however, that given the free-form
nature of *resource_metadata* field, there is no practical or consistent way
to validate the query fields under *metadata* domain like it is done for
all other fields.
.. note:: The API call will return HTTP 200 OK status for both of the
following cases: when a query with *metadata.<field>* does not match its
value, and when *<field>* itself does not exist in any of the records being
queried.
Simple Query
++++++++++++
Many of the endpoints above accept a query filter argument, which
should be a list of Query data structures. Whatever the endpoint you
want to apply a filter on, you always filter on the fields of the *Sample*
type (for example, if you apply a filter on a query for statistics,
you won't target *duration_start* field of *Statistics*, but *timestamp*
field of *Sample*). You may also apply filters based on the values of one
or more of the *resource_metadata* field, in which case you should target
*metadata.<field>*. Notice, however, that given the free-form nature of
*resource_metadata* field, there is no practical or consistent way to
validate the query fields under *metadata* domain like it is done for all other
fields. In other words, the API call will return HTTP 200 in both scenarios:
when a query with *metadata.<field>* does not match its value, and when
*<field>* itself does not exist in any of the records being queried.
field of *Sample*).
.. autotype:: ceilometer.api.controllers.v2.Query
:members:
Complex Query
+++++++++++++
The filter expressions of the Complex Query feature operate on the fields
of *Sample*. The following comparison operators are supported: *=*, *!=*, *<*,
*<=*, *>* and *>=*; and the following logical operators can be used: *and* and *or*.
Complex Query supports defining the list of orderby expressions in the form
of [{"field_name": "asc"}, {"field_name2": "desc"}, ...].
The number of the returned items can be bounded using the *limit* option.
The *filter*, *orderby* and *limit* are all optional fields in a query.
.. rest-controller:: ceilometer.api.controllers.v2:QuerySamplesController
:webprefix: /v2/query/samples
.. autotype:: ceilometer.api.controllers.v2.ComplexQuery
:members:
Links
=====
@ -372,6 +402,53 @@ parameter to the query::
This query would only return the last 3 samples.
Functional example for Complex Query
++++++++++++++++++++++++++++++++++++
This example demonstrates how complex query filter expressions can be generated and sent
to the /v2/query/samples endpoint of Ceilometer API using POST request.
To check for *cpu_util* samples reported between 18:00-18:15 or between 18:30 - 18:45
on a particular date (2013-12-01), where the utilization is between 23 and 26 percent,
the following filter expression can be created::
{"and":
[{"and":
[{"=": {"counter_name": "cpu_util"}},
{">": {"counter_volume": 0.23}},
{"<": {"counter_volume": 0.26}}]},
{"or":
[{"and":
[{">": {"timestamp": "2013-12-01T18:00:00"}},
{"<": {"timestamp": "2013-12-01T18:15:00"}}]},
{"and":
[{">": {"timestamp": "2013-12-01T18:30:00"}},
{"<": {"timestamp": "2013-12-01T18:45:00"}}]}]}]}
Different sorting criteria can be defined for the query filter, for example the results
can be ordered in an ascending order by the *counter_volume* and descending order based on
the *timestamp*. The following order by expression has to be created for specifying this
criteria::
[{"counter_volume": "ASC"}, {"timestamp": "DESC"}]
As the current implementation accepts only string values as query filter and order by
definitions, the above defined expressions have to be converted to string values.
By adding a limit criteria to the request, which maximizes the number of returned samples
to four, the query looks like the following::
{
"filter" : "{\"and\":[{\"and\": [{\"=\": {\"counter_name\": \"cpu_util\"}}, {\">\": {\"counter_volume\": 0.23}}, {\"<\": {\"counter_volume\": 0.26}}]}, {\"or\": [{\"and\": [{\">\": {\"timestamp\": \"2013-12-01T18:00:00\"}}, {\"<\": {\"timestamp\": \"2013-12-01T18:15:00\"}}]}, {\"and\": [{\">\": {\"timestamp\": \"2013-12-01T18:30:00\"}}, {\"<\": {\"timestamp\": \"2013-12-01T18:45:00\"}}]}]}]}",
"orderby" : "[{\"counter_volume\": \"ASC\"}, {\"timestamp\": \"DESC\"}]",
"limit" : 4
}
A query request looks like the following with curl::
curl -X POST -H 'X-Auth-Token: <inserttokenhere>' -H 'Content-Type: application/json' \
-d '<insertyourqueryexpressionhere>' \
http://localhost:8777/v2/query/samples
.. _user-defined-data:
User-defined data

View File

@ -6,6 +6,7 @@ Flask>=0.10,<1.0
happybase>=0.4
iso8601>=0.1.8
jsonpath-rw>=1.2.0,<2.0
jsonschema>=2.0.0,<3.0.0
kombu>=2.4.8
lockfile>=0.8
lxml>=2.3