Added filter capability for transform specs.

Change-Id: Ie5b456039c9810da19c1699cc7d5a44277496843
This commit is contained in:
Flint Calvin 2016-07-13 21:52:27 +00:00
parent b3f85e134e
commit c7128b0136
9 changed files with 785 additions and 442 deletions

View File

@ -15,8 +15,10 @@
from collections import namedtuple
import datetime
from pyspark.sql import functions
from pyspark.sql import SQLContext
from monasca_transform.component import Component
from monasca_transform.component.component_utils import ComponentUtils
from monasca_transform.component.usage import UsageComponent
@ -327,6 +329,43 @@ class FetchQuantity(UsageComponent):
group_by_period_list = ComponentUtils._get_group_by_period_list(
aggregation_period)
# retrieve filter specifications
agg_params = transform_spec_df.select(
"aggregation_params_map.filter_by_list"). \
collect()[0].asDict()
filter_by_list = \
agg_params["filter_by_list"]
# if filter(s) have been specified, apply them one at a time
if filter_by_list:
for filter_element in filter_by_list:
field_to_filter = filter_element["field_to_filter"]
filter_expression = filter_element["filter_expression"]
filter_operation = filter_element["filter_operation"]
if (field_to_filter and
filter_expression and
filter_operation and
(filter_operation == "include" or
filter_operation == "exclude")):
if filter_operation == "include":
match = True
else:
match = False
# apply the specified filter to the record store
record_store_df = record_store_df.where(
functions.col(str(field_to_filter)).rlike(
str(filter_expression)) == match)
else:
raise FetchQuantityException(
"Encountered invalid filter details: "
"field to filter = %s, filter expression = %s, "
"filter operation = %s. All values must be "
"supplied and filter operation must be either "
"'include' or 'exclude'." % (field_to_filter,
filter_expression,
filter_operation))
# get what we want to group by
agg_params = transform_spec_df.select(
"aggregation_params_map.aggregation_group_by_list"). \

View File

@ -16,6 +16,8 @@ import os
from monasca_transform.data_driven_specs.data_driven_specs_repo \
import DataDrivenSpecsRepo
from monasca_transform.transform.transform_utils import PreTransformSpecsUtils
from monasca_transform.transform.transform_utils import TransformSpecsUtils
class JSONDataDrivenSpecsRepo(DataDrivenSpecsRepo):
@ -32,14 +34,17 @@ class JSONDataDrivenSpecsRepo(DataDrivenSpecsRepo):
"monasca_transform/data_driven_specs/"
"transform_specs/transform_specs.json"
))
if os.path.exists(path):
# read file to json
return TransformSpecsUtils.create_df_from_json(
sql_context, path)
elif data_driven_spec_type == self.pre_transform_specs_type:
path = (os.path.join(
self._common_file_system_stub_path,
"monasca_transform/data_driven_specs/"
"pre_transform_specs/pre_transform_specs.json"
))
if os.path.exists(path):
# read file to json
return sql_context.read.json(path)
if os.path.exists(path):
# read file to json
return PreTransformSpecsUtils.create_df_from_json(
sql_context, path)

View File

@ -1,23 +1,23 @@
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"mem.total_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"mem_total_all","metric_id":"mem_total_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"mem.usable_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"mem_usable_all","metric_id":"mem_usable_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.total_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_total_mb_all","metric_id":"vm_mem_total_mb_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.total_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_total_mb_project","metric_id":"vm_mem_total_mb_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.used_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_used_mb_all","metric_id":"vm_mem_used_mb_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.used_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_used_mb_project","metric_id":"vm_mem_used_mb_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"disk.total_space_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"disk_total_all","metric_id":"disk_total_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"disk.total_used_space_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"disk_usable_all","metric_id":"disk_usable_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"nova.vm.disk.total_allocated_gb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"nova_disk_total_allocated_gb_all","metric_id":"nova_disk_total_allocated_gb_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.disk.allocation_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_disk_allocation_all","metric_id":"vm_disk_allocation_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.disk.allocation_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_disk_allocation_project","metric_id":"vm_disk_allocation_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.total_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_total_all","metric_id":"cpu_total_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.total_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_total_host","metric_id":"cpu_total_host"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity_util","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.utilized_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["event_type", "host"],"usage_fetch_operation": "avg","usage_fetch_util_quantity_event_type": "cpu.total_logical_cores","usage_fetch_util_idle_perc_event_type": "cpu.idle_perc","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_util_all","metric_id":"cpu_util_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity_util","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.utilized_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["event_type", "host"],"usage_fetch_operation": "avg","usage_fetch_util_quantity_event_type": "cpu.total_logical_cores","usage_fetch_util_idle_perc_event_type": "cpu.idle_perc","setter_rollup_group_by_list": ["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_util_host","metric_id":"cpu_util_host"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vcpus_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vcpus_all","metric_id":"vcpus_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vcpus_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vcpus_project","metric_id":"vcpus_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.cpu.utilization_perc_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_cpu_util_perc_project","metric_id":"vm_cpu_util_perc_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_usage_all","metric_id":"swift_usage_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_usage_all","metric_id":"swift_usage_host"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"calculate_rate","setters":["set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.rate_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"setter_rollup_group_by_list": [],"dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_usage_rate","metric_id":"swift_usage_rate"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.avail_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_avail_all","metric_id":"swift_avail_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.avail_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","setter_rollup_group_by_list": ["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_avail_all","metric_id":"swift_avail_host"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"mem.total_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"mem_total_all","metric_id":"mem_total_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"mem.usable_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"mem_usable_all","metric_id":"mem_usable_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.total_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_total_mb_all","metric_id":"vm_mem_total_mb_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.total_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_total_mb_project","metric_id":"vm_mem_total_mb_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.used_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_used_mb_all","metric_id":"vm_mem_used_mb_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.mem.used_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_mem_used_mb_project","metric_id":"vm_mem_used_mb_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"disk.total_space_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"disk_total_all","metric_id":"disk_total_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"disk.total_used_space_mb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"disk_usable_all","metric_id":"disk_usable_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"nova.vm.disk.total_allocated_gb_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"nova_disk_total_allocated_gb_all","metric_id":"nova_disk_total_allocated_gb_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.disk.allocation_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_disk_allocation_all","metric_id":"vm_disk_allocation_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.disk.allocation_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_disk_allocation_project","metric_id":"vm_disk_allocation_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.total_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list": [],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_total_all","metric_id":"cpu_total_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.total_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id"],"usage_fetch_operation": "avg","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list":["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_total_host","metric_id":"cpu_total_host"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity_util","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.utilized_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["event_type", "host"],"usage_fetch_operation": "avg","usage_fetch_util_quantity_event_type": "cpu.total_logical_cores","usage_fetch_util_idle_perc_event_type": "cpu.idle_perc","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_util_all","metric_id":"cpu_util_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity_util","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"cpu.utilized_logical_cores_agg","aggregation_period":"hourly","aggregation_group_by_list": ["event_type", "host"],"usage_fetch_operation": "avg","usage_fetch_util_quantity_event_type": "cpu.total_logical_cores","usage_fetch_util_idle_perc_event_type": "cpu.idle_perc","filter_by_list": [{"field_to_filter": "host","filter_expression": "-comp[0-9]+-","filter_operation": "include"}],"setter_rollup_group_by_list":["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_util_host","metric_id":"cpu_util_host"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vcpus_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vcpus_all","metric_id":"vcpus_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vcpus_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vcpus_project","metric_id":"vcpus_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"vm.cpu.utilization_perc_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "tenant_id", "resource_uuid"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["tenant_id"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"vm_cpu_util_perc_project","metric_id":"vm_cpu_util_perc_project"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_usage_all","metric_id":"swift_usage_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_usage_all","metric_id":"swift_usage_host"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"calculate_rate","setters":["set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.rate_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"filter_by_list": [],"setter_rollup_group_by_list": [],"dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_usage_rate","metric_id":"swift_usage_rate"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.avail_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_avail_all","metric_id":"swift_avail_all"}
{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"swiftlm.diskusage.val.avail_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount", "device"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"swift_avail_all","metric_id":"swift_avail_host"}

View File

@ -159,6 +159,12 @@ class TransformSpecsUtils(TransformUtils):
StructField("usage_fetch_operation",
StringType(),
True),
StructField("filter_by_list",
ArrayType(MapType(StringType(),
StringType(),
True)
)
),
StructField(
"usage_fetch_util_quantity_event_type",
StringType(),

View File

@ -90,7 +90,7 @@ class SparkTest(SparkContextTest):
MockComponentManager.get_insert_cmpt_mgr()
# Create an emulated set of Kafka messages (these were gathered
# by extracting Monasca messages from the Metrics queue on mini-mon).
# by extracting Monasca messages from the Metrics queue).
# Create an RDD out of the mocked Monasca metrics
with open(DataProvider.kafka_data_path) as f:
@ -663,7 +663,7 @@ class SparkTest(SparkContextTest):
self.assertTrue(disk_total_space_agg_metric is not None)
self.assertEqual(2574044.0,
self.assertEqual(1282121.0,
disk_total_space_agg_metric
.get('metric').get('value'))
self.assertEqual('useast',
@ -681,14 +681,14 @@ class SparkTest(SparkContextTest):
.get('metric').get('dimensions')
.get('aggregation_period'))
self.assertEqual(2.0,
self.assertEqual(1.0,
disk_total_space_agg_metric
.get('metric').get('value_meta').get('record_count'))
self.assertEqual('2016-06-01 21:09:21',
disk_total_space_agg_metric
.get('metric').get('value_meta')
.get('firstrecord_timestamp'))
self.assertEqual('2016-06-01 21:09:24',
self.assertEqual('2016-06-01 21:09:21',
disk_total_space_agg_metric
.get('metric').get('value_meta')
.get('lastrecord_timestamp'))
@ -701,7 +701,7 @@ class SparkTest(SparkContextTest):
self.assertTrue(disk_total_used_agg_metric is not None)
self.assertEqual(34043.0,
self.assertEqual(2329.0,
disk_total_used_agg_metric
.get('metric').get('value'))
self.assertEqual('useast',
@ -719,14 +719,14 @@ class SparkTest(SparkContextTest):
.get('metric').get('dimensions')
.get('aggregation_period'))
self.assertEqual(2.0,
self.assertEqual(1.0,
disk_total_used_agg_metric
.get('metric').get('value_meta').get('record_count'))
self.assertEqual('2016-06-01 21:09:21',
disk_total_used_agg_metric
.get('metric').get('value_meta')
.get('firstrecord_timestamp'))
self.assertEqual('2016-06-01 21:09:24',
self.assertEqual('2016-06-01 21:09:21',
disk_total_used_agg_metric
.get('metric').get('value_meta')
.get('lastrecord_timestamp'))
@ -772,13 +772,14 @@ class SparkTest(SparkContextTest):
.get('metric').get('value_meta')
.get('lastrecord_timestamp'))
# Verify cpu.utilized_logical_cores_agg metrics for mini-mon host
# Verify cpu.utilized_logical_cores_agg metrics for
# test-cp1-comp0294-mgmt host
cpu_util_cores_agg_metric = [
value for value in metrics
if value.get('metric').get('name') ==
'cpu.utilized_logical_cores_agg' and
value.get('metric').get('dimensions').get('host') ==
'mini-mon'][0]
'test-cp1-comp0294-mgmt'][0]
self.assertTrue(cpu_util_cores_agg_metric is not None)
@ -813,13 +814,14 @@ class SparkTest(SparkContextTest):
.get('metric').get('value_meta')
.get('lastrecord_timestamp'))
# Verify vm.cpu.utilization_perc_agg metrics for devstack host
# Verify vm.cpu.utilization_perc_agg metrics for
# test-cp1-comp0037-mgmt host
cpu_util_cores_agg_metric = [
value for value in metrics
if value.get('metric').get('name') ==
'cpu.utilized_logical_cores_agg' and
value.get('metric').get('dimensions').get('host') ==
'devstack'][0]
'test-cp1-comp0037-mgmt'][0]
self.assertTrue(cpu_util_cores_agg_metric is not None)

View File

@ -1,93 +1,93 @@
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":45.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457366959000,"value":0.143361},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":3.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":2.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":3.3000000000000003},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":8.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":8.1},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":53.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457366974000,"value":0.143374},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":3.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":2.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":3.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":5.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":6.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":0.1},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":75.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457366989000,"value":0.143414},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":6.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":4.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":4.9},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":5.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":93.3},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":65.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367004000,"value":0.143409},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":2.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":3.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":5.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":75.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":29.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367019000,"value":0.143412},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":3.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":2.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":3.3},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":5.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":52.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":92.9},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367034000,"value":0.143419},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":3.9},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":3.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":3.5999999999999996},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":5.3},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":10.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":33.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":86.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367049000,"value":0.143422},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":3.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":3.1},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367053000,"value":4.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367057})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366963000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366963000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366977000,"value":18.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366993000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457367008000,"value":2.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457367023000,"value":2.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457367038000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0027-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366959000,"value":45.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0027-mgmt","apache_host":"test-cp1-comp0027-mgmt"},"timestamp":1457366959000,"value":0.143361},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366959000,"value":3.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366959000,"value":2.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366963})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366963000,"value":3.3000000000000003},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366963000,"value":8.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366963000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366963000,"value":8.1},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0027-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366974000,"value":53.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0027-mgmt","apache_host":"test-cp1-comp0027-mgmt"},"timestamp":1457366974000,"value":0.143374},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366974000,"value":3.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366974000,"value":2.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366977})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366977000,"value":3.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366977000,"value":5.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366977000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366977000,"value":6.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366989000,"value":0.1},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0027-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366989000,"value":75.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0027-mgmt","apache_host":"test-cp1-comp0027-mgmt"},"timestamp":1457366989000,"value":0.143414},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366989000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366989000,"value":6.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366989000,"value":4.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366991})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366993000,"value":4.9},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366993000,"value":5.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366993000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366993000,"value":93.3},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0027-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367004000,"value":65.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0027-mgmt","apache_host":"test-cp1-comp0027-mgmt"},"timestamp":1457367004000,"value":0.143409},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367004000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367004000,"value":2.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367005})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367008000,"value":3.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367008000,"value":5.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367008000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367008000,"value":75.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0027-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367019000,"value":29.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0027-mgmt","apache_host":"test-cp1-comp0027-mgmt"},"timestamp":1457367019000,"value":0.143412},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367019000,"value":3.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367019000,"value":2.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367026})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367023000,"value":3.3},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367023000,"value":5.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367023000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367023000,"value":52.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0027-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367034000,"value":92.9},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0027-mgmt","apache_host":"test-cp1-comp0027-mgmt"},"timestamp":1457367034000,"value":0.143419},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367034000,"value":3.9},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367034000,"value":3.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367040})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367038000,"value":3.5999999999999996},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367038000,"value":5.3},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367038000,"value":10.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367038000,"value":33.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})
('',{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"103e4d4d14bc4fdda4a9c73d1643e1d7","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":0.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0027-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131})
('',{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367049000,"value":86.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0027-mgmt","apache_host":"test-cp1-comp0027-mgmt"},"timestamp":1457367049000,"value":0.143422},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367049000,"value":3.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367049000,"value":3.1},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367054})
('',{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0333-mgmt"},"timestamp":1457367053000,"value":4.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367057})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366963000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366963000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366966})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366977000,"value":18.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366980})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457366993000,"value":4.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457366994})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367008000,"value":2.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367008})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367023000,"value":2.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367029})
('',{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0027-mgmt"},"timestamp":1457367038000,"value":8.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1457367043})

View File

@ -1,310 +1,310 @@
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"devstack"},"timestamp":1453307999000,"value":0.48},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"devstack"},"timestamp":1453307999000,"value":1.12},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453307999000,"value":3161.125},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_packets_dropped_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"devstack"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453307999000,"value":3.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"mini-mon","observer_host":"devstack","test_type":"ssh"},"timestamp":1453307983000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"devstack","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453307999000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453307999000,"value":5.133333333333334},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.com_delete_multi","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453307999000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453307999000,"value":0.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453307999000,"value":0.7333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453307999000,"value":0.06666666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453307999000,"value":0.06666666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"devstack","mount_point":"/vagrant"},"timestamp":1453307999000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"devstack","mount_point":"/vagrant_home"},"timestamp":1453307999000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453307999000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308000000,"value":1056.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308000000,"value":63492.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"raw-sql.time.max","dimensions":{"component":"monasca-api","service":"monitoring","hostname":"mini-mon","url":"http://localhost:8081/metrics"},"timestamp":1453307985000,"value":0.004438382},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"mini-mon","url":"http://localhost:8091/metrics"},"timestamp":1453308000000,"value":1.8666666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"zookeeper.outstanding_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"mini-mon","mode":"standalone"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"zookeeper.out_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"mini-mon","mode":"standalone"},"timestamp":1453308000000,"value":1.0004907E7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"mini-mon"},"timestamp":1453308000000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.innodb.mutex_spin_rounds","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.3333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.swap_free_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":881.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":739.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":134.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":3456.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"mini-mon","mount_point":"/vagrant"},"timestamp":1453308000000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308000000,"value":233.06666666666666},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric": {"timestamp": 1453252739000, "name": "POCAggregatedMetric", "value": 0.39, "dimensions": {"hostname": "devstack"}}, "meta": {"region": "useast", "tenantId": "583d49ab6aca482da653dd838deaf538"}, "creation_time": 1453252745}')
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"devstack"},"timestamp":1453308015000,"value":0.48},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"devstack"},"timestamp":1453308015000,"value":0.94},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308015000,"value":3171.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_packets_dropped_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"devstack"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308015000,"value":3.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"mini-mon","observer_host":"devstack","test_type":"ssh"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"devstack","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453308015000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308015000,"value":6.6875},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308015000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308015000,"value":0.375},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308015000,"value":0.75},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453308015000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"devstack","mount_point":"/vagrant"},"timestamp":1453308015000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"devstack","mount_point":"/vagrant_home"},"timestamp":1453308015000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453308015000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308015000,"value":2390.266666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"mini-mon"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308015000,"value":143538.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"raw-sql.time.max","dimensions":{"component":"monasca-api","service":"monitoring","hostname":"mini-mon","url":"http://localhost:8081/metrics"},"timestamp":1453308000000,"value":0.004438382},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"mini-mon","url":"http://localhost:8091/metrics"},"timestamp":1453308015000,"value":3.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"zookeeper.outstanding_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"mini-mon","mode":"standalone"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"zookeeper.out_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"mini-mon","mode":"standalone"},"timestamp":1453308015000,"value":1.0005063E7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"mini-mon"},"timestamp":1453308016000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.mutex_spin_rounds","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.375},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":818.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":689.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":123.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":4567.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"mini-mon","mount_point":"/vagrant"},"timestamp":1453308016000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308016000,"value":149.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"devstack"},"timestamp":1453308030000,"value":0.49},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"devstack"},"timestamp":1453308030000,"value":0.87},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308030000,"value":3371.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"devstack"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308030000,"value":3.2666666666666666},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"mini-mon","observer_host":"devstack","test_type":"ssh"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"devstack","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453308031000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308031000,"value":6.5625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_delete_multi","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308031000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308031000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308031000,"value":0.8125},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453308031000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453308031000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"devstack","mount_point":"/vagrant"},"timestamp":1453308031000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"devstack","mount_point":"/vagrant_home"},"timestamp":1453308031000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453308031000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308031000,"value":197.75},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308031000,"value":11972.625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"raw-sql.time.max","dimensions":{"component":"monasca-api","service":"monitoring","hostname":"mini-mon","url":"http://localhost:8081/metrics"},"timestamp":1453308016000,"value":0.004438382},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"mini-mon","url":"http://localhost:8091/metrics"},"timestamp":1453308031000,"value":3.466666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"zookeeper.outstanding_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"mini-mon","mode":"standalone"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"zookeeper.out_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"mini-mon","mode":"standalone"},"timestamp":1453308031000,"value":1.0005199E7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"mini-mon"},"timestamp":1453308031000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.swap_free_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":836.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":692.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":138.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":5678.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"mini-mon","mount_point":"/vagrant"},"timestamp":1453308031000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308031000,"value":12.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"devstack"},"timestamp":1453308046000,"value":0.47},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"devstack"},"timestamp":1453308046000,"value":0.69},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308046000,"value":3228.375},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_packets_dropped_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"devstack"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"devstack"},"timestamp":1453308046000,"value":3.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"devstack"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"mini-mon","observer_host":"devstack","test_type":"ssh"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"devstack","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453308046000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308046000,"value":10.733333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_delete_multi","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308046000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308046000,"value":0.26666666666666666},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"devstack"},"timestamp":1453308046000,"value":0.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453308046000,"value":0.13333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1453308046000,"value":0.5333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"devstack","mount_point":"/vagrant"},"timestamp":1453308046000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"devstack","mount_point":"/vagrant_home"},"timestamp":1453308046000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"devstack","mount_point":"/"},"timestamp":1453308046000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"mini-mon","url":"http://localhost:8091/metrics"},"timestamp":1453308046000,"value":3.1875},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"mini-mon"},"timestamp":1453308046000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.mutex_spin_rounds","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.swap_free_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":837.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":693.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":137.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":1234.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"mini-mon","mount_point":"/vagrant"},"timestamp":1453308046000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"mini-mon","mount_point":"/"},"timestamp":1453308046000,"value":1.7333333333333334},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308020000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308030000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308032000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308038000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308042000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"devstack","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308005000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"mini-mon","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308010000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"dummy001","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308015000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"dummy002","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308025000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"dummy001","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308034000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"dummy002","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308036000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"mini-mon","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308040000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"dummy001","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308046000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"mini-mon","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457366959000,"value":0.143361},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":3.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":2.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":3.3000000000000003},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":8.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457366974000,"value":0.143374},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":3.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":2.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":3.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":5.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":0.1},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457366989000,"value":0.143414},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":6.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":4.4},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":4.9},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":5.4},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367004000,"value":0.143409},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":2.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":3.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":5.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367019000,"value":0.143412},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":3.8},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":2.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":3.3},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":5.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367034000,"value":0.143419},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":3.9},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":3.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":3.5999999999999996},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":5.3},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366977000,"value":90.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366959000,"value":93.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366963000,"value":88.0},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366974000,"value":93.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457366989000,"value":89.0},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457366993000,"value":89.8},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367004000,"value":93.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367008000,"value":91.2},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367019000,"value":93.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367023000,"value":91.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367034000,"value":92.9},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367038000,"value":91.1},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":93.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"devstack","apache_host":"devstack"},"timestamp":1457367049000,"value":0.143422},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":3.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"devstack"},"timestamp":1457367049000,"value":3.1},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1457367053000,"value":4.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367057}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366963000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366963000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366977000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457366993000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457367008000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457367023000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"devstack"},"timestamp":1457367038000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.48},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":1.12},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":3161.125},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_packets_dropped_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":3.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"test-cp1-comp0294-mgmt","observer_host":"test-cp1-comp0037-mgmt","test_type":"ssh"},"timestamp":1453307983000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"test-cp1-comp0037-mgmt","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453307999000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":5.133333333333334},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.com_delete_multi","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.7333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.06666666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453307999000,"value":0.06666666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant"},"timestamp":1453307999000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant_home"},"timestamp":1453307999000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453307999000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308004}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":1056.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":63492.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"raw-sql.time.max","dimensions":{"component":"monasca-api","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","url":"http://localhost:8081/metrics"},"timestamp":1453307985000,"value":0.004438382},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","url":"http://localhost:8091/metrics"},"timestamp":1453308000000,"value":1.8666666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"zookeeper.outstanding_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","mode":"standalone"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"zookeeper.out_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","mode":"standalone"},"timestamp":1453308000000,"value":1.0004907E7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.innodb.mutex_spin_rounds","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.3333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.swap_free_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":881.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":739.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":134.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":3456.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308000000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"test-cp1-comp0294-mgmt","mount_point":"/vagrant"},"timestamp":1453308000000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308000000,"value":233.06666666666666},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308005}')
('<message id>','{"metric": {"timestamp": 1453252739000, "name": "POCAggregatedMetric", "value": 0.39, "dimensions": {"hostname": "test-cp1-comp0037-mgmt"}}, "meta": {"region": "useast", "tenantId": "583d49ab6aca482da653dd838deaf538"}, "creation_time": 1453252745}')
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.48},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.94},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":3171.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_packets_dropped_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":3.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"test-cp1-comp0294-mgmt","observer_host":"test-cp1-comp0037-mgmt","test_type":"ssh"},"timestamp":1453307999000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"test-cp1-comp0037-mgmt","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453308015000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":6.6875},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.375},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.75},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant"},"timestamp":1453308015000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant_home"},"timestamp":1453308015000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453308015000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308015000,"value":2390.266666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308015000,"value":143538.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"raw-sql.time.max","dimensions":{"component":"monasca-api","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","url":"http://localhost:8081/metrics"},"timestamp":1453308000000,"value":0.004438382},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","url":"http://localhost:8091/metrics"},"timestamp":1453308015000,"value":3.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"zookeeper.outstanding_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","mode":"standalone"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"zookeeper.out_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","mode":"standalone"},"timestamp":1453308015000,"value":1.0005063E7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.mutex_spin_rounds","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.375},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":818.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":689.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":123.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":4567.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308016000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"test-cp1-comp0294-mgmt","mount_point":"/vagrant"},"timestamp":1453308016000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308016000,"value":149.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308018}')
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":0.49},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":0.87},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":3371.8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":3.2666666666666666},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"test-cp1-comp0294-mgmt","observer_host":"test-cp1-comp0037-mgmt","test_type":"ssh"},"timestamp":1453308015000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"test-cp1-comp0037-mgmt","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453308031000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":6.5625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_delete_multi","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":0.8125},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453308031000,"value":0.0625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant"},"timestamp":1453308031000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant_home"},"timestamp":1453308031000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453308031000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":197.75},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":11972.625},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"raw-sql.time.max","dimensions":{"component":"monasca-api","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","url":"http://localhost:8081/metrics"},"timestamp":1453308016000,"value":0.004438382},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","url":"http://localhost:8091/metrics"},"timestamp":1453308031000,"value":3.466666666666667},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"zookeeper.outstanding_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","mode":"standalone"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"zookeeper.out_bytes","dimensions":{"component":"zookeeper","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","mode":"standalone"},"timestamp":1453308031000,"value":1.0005199E7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.2},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.swap_free_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":836.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":692.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":138.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":5678.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308031000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"test-cp1-comp0294-mgmt","mount_point":"/vagrant"},"timestamp":1453308031000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308031000,"value":12.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308032}')
('<message id>','{"metric":{"name":"load.avg_5_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.47},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"load.avg_1_min","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.69},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_bytes_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":3228.375},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_errors_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_packets_dropped_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_packets_dropped_sec","dimensions":{"device":"eth0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_packets_sec","dimensions":{"device":"eth1","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":3.5},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"device":"virbr0","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"host_alive_status","dimensions":{"hostname":"test-cp1-comp0294-mgmt","observer_host":"test-cp1-comp0037-mgmt","test_type":"ssh"},"timestamp":1453308030000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"vm.host_alive_status","dimensions":{"resource_id":"9913eeb5-2978-48a0-9587-5e872808d8d6","component":"vm","service":"compute","hostname":"test-cp1-comp0037-mgmt","tenant_id":"583d49ab6aca482da653dd838deaf538","zone":"nova"},"timestamp":1453308046000,"value":1.0,"value_meta":{"detail":"Host failed ping check"}},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.questions","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":10.733333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_delete_multi","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_used","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":1.34201344E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.user_time","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.26666666666666666},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_update","dimensions":{"component":"mysql","service":"mysql","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.6},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"apache.net.requests_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.13333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"apache.net.kbytes_sec","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1453308046000,"value":0.5333333333333333},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant"},"timestamp":1453308046000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"device":"vagrant_home","hostname":"test-cp1-comp0037-mgmt","mount_point":"/vagrant_home"},"timestamp":1453308046000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"io.read_kbytes_sec","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.inode_used_perc","dimensions":{"device":"sda1","hostname":"test-cp1-comp0037-mgmt","mount_point":"/"},"timestamp":1453308046000,"value":6.92},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_packets_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.out_errors_sec","dimensions":{"service":"monitoring","device":"eth1","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"net.in_bytes_sec","dimensions":{"service":"monitoring","device":"eth0","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"metrics-added-to-batch-counter[0]","dimensions":{"component":"monasca-persister","service":"monitoring","hostname":"test-cp1-comp0294-mgmt","url":"http://localhost:8091/metrics"},"timestamp":1453308046000,"value":3.1875},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"process.pid_count","dimensions":{"process_name":"mysqld","service":"mysql","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":1.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_disk_tables","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_insert_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.open_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":16.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.data_reads","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.mutex_spin_rounds","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.innodb.buffer_pool_free","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":1.24059648E8},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.com_select","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.4},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mysql.performance.created_tmp_files","dimensions":{"component":"mysql","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.swap_free_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.usable_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":837.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.used_cache","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":693.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.free_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":137.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":1234.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"mem.swap_total_mb","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"kafka.consumer_lag","dimensions":{"topic":"events","consumer_group":"thresh-event","component":"kafka","service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"io.write_time_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308046000,"value":0.0},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"disk.space_used_perc","dimensions":{"service":"monitoring","device":"vagrant","hostname":"test-cp1-comp0294-mgmt","mount_point":"/vagrant"},"timestamp":1453308046000,"value":7.7},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"io.read_req_sec","dimensions":{"service":"monitoring","device":"sda1","hostname":"test-cp1-comp0294-mgmt","mount_point":"/"},"timestamp":1453308046000,"value":1.7333333333333334},"meta":{"tenantId":"tenantId of metric writer","region":"useast"},"creation_time":1453308053}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308000000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0037-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308020000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0037-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308030000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0037-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308032000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0037-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308038000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0037-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"8647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308042000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0037-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308005000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0294-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308010000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0158-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308015000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0009-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308025000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0158-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308034000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0009-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308036000,"value":2.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0294-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308040000,"value":1.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0158-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"vcpus","dimensions":{"region":"useast","unit":"vcpu","resource_id":"b1a7f31d-3f85-4475-a50b-77094b62b078","source":"openstack","project_id":"9647fd5030b04a799b0411cc38c4102d","user_id":"a69aabac824c4e1688702b8eb7e22af2","type":"gauge"},"timestamp":1453308046000,"value":3.0,"value_meta":{"audit_period_beginning":"2016-05-13T23:00:00.000000","availability_zone":"None","host":"test-cp1-comp0294-mgmt","event_type":"compute.instance.update","launched_at":"2016-05-13T23:14:06.000000","deleted_at":"","state":"active","memory_mb":"512","root_gb":"1","vcpus":"1","audit_period_ending":"2016-05-13T23:15:29.476223","ephemeral_gb":"0","instance_flavor_id":"1","created_at":"2016-05-13 23:13:44+00:00","disk_gb":"1","image_ref_url":"http://10.241.104.4:9292/images/12990d03-8059-4052-8b93-d4c6376fea57"}},"meta":{"tenantId":"225bbc4c5c1c4dfaa910cb44b5a4d7b7","region":"unset"},"creation_time":1458166131}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1457366959000,"value":0.143361},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366959000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366959000,"value":3.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366959000,"value":2.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366963000,"value":3.3000000000000003},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366963000,"value":8.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366963000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366963000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1457366974000,"value":0.143374},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366974000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366974000,"value":3.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366974000,"value":2.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366977000,"value":3.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366977000,"value":5.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366977000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366977000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366989000,"value":0.1},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1457366989000,"value":0.143414},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366989000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366989000,"value":6.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366989000,"value":4.4},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366993000,"value":4.9},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366993000,"value":5.4},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366993000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366993000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1457367004000,"value":0.143409},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367004000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367004000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367004000,"value":2.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367008000,"value":3.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367008000,"value":5.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367008000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367008000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1457367019000,"value":0.143412},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367019000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367019000,"value":3.8},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367019000,"value":2.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367023000,"value":3.3},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367023000,"value":5.5},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367023000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367023000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1457367034000,"value":0.143419},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367034000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367034000,"value":3.9},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367034000,"value":3.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367038000,"value":3.5999999999999996},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367038000,"value":5.3},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367038000,"value":4.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367038000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.wait_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366977000,"value":90.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366959000,"value":93.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366963}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366963000,"value":88.0},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366974000,"value":93.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366977}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366989000,"value":89.0},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366991}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457366993000,"value":89.8},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367004000,"value":93.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367005}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367008000,"value":91.2},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367019000,"value":93.7},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367026}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367023000,"value":91.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367034000,"value":92.9},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367040}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367038000,"value":91.1},"meta":{"tenantId":"203e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"cpu.idle_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367049000,"value":93.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"apache.performance.cpu_load_perc","dimensions":{"apache_port":"80","component":"apache","service":"apache","hostname":"test-cp1-comp0037-mgmt","apache_host":"test-cp1-comp0037-mgmt"},"timestamp":1457367049000,"value":0.143422},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.stolen_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367049000,"value":0.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.user_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367049000,"value":3.6},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367049000,"value":3.1},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367054}')
('<message id>','{"metric":{"name":"cpu.system_perc","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0294-mgmt"},"timestamp":1457367053000,"value":4.2},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367057}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366963000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366963000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366966}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366977000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366980}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457366993000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457366994}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367008000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367008}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367023000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367029}')
('<message id>','{"metric":{"name":"cpu.total_logical_cores","dimensions":{"service":"monitoring","hostname":"test-cp1-comp0037-mgmt"},"timestamp":1457367038000,"value":2.0},"meta":{"tenantId":"103e4d4d14bc4fdda4a9c73d1643e1d7","region":"useast"},"creation_time":1457367043}')
('<message id>','{"metric":{"name":"nova.vm.disk.total_allocated_gb","dimensions":{"cloud_name":"helion-poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","hostname":"hlm004-cp1-comp0003-mgmt","cluster":"compute"},"timestamp":1463498048000,"value":5.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1463498052}')
('<message id>','{"metric":{"name":"nova.vm.disk.total_allocated_gb","dimensions":{"cloud_name":"helion-poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","hostname":"hlm004-cp1-comp0002-mgmt","cluster":"compute"},"timestamp":1463498054000,"value":44.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1463498056}')
('<message id>','{"metric":{"name":"nova.vm.disk.total_allocated_gb","dimensions":{"cloud_name":"helion-poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","hostname":"hlm004-cp1-comp0006-mgmt","cluster":"compute"},"timestamp":1463498061000,"value":82.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1463498062}')
('<message id>','{"metric":{"name":"nova.vm.disk.total_allocated_gb","dimensions":{"cloud_name":"helion-poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","hostname":"hlm004-cp1-comp0003-mgmt","cluster":"compute"},"timestamp":1463498078000,"value":15.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1463498080}')
('<message id>','{"metric":{"name":"nova.vm.disk.total_allocated_gb","dimensions":{"cloud_name":"helion-poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","hostname":"hlm004-cp1-comp0002-mgmt","cluster":"compute"},"timestamp":1463498084000,"value":132.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1463498086}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"a","component":"vm","service":"compute","hostname":"devstack","tenant_id":"5f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000000,"value":24.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"a","component":"vm","service":"compute","hostname":"devstack","tenant_id":"5f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308005000,"value":24.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"a","component":"vm","service":"compute","hostname":"devstack","tenant_id":"5f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308040000,"value":24.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"b","component":"vm","service":"compute","hostname":"mini-mon","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000010,"value":26.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"c","component":"vm","service":"compute","hostname":"dummy1","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000020,"value":28.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"d","component":"vm","service":"compute","hostname":"dummy2","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000030,"value":30.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"e","component":"vm","service":"compute","hostname":"dummy3","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000040,"value":32.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"b","component":"vm","service":"compute","hostname":"mini-mon","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308006000,"value":26.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"b","component":"vm","service":"compute","hostname":"mini-mon","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308046000,"value":26.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"a","component":"vm","service":"compute","hostname":"test-cp1-comp0037-mgmt","tenant_id":"5f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000000,"value":24.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"a","component":"vm","service":"compute","hostname":"test-cp1-comp0037-mgmt","tenant_id":"5f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308005000,"value":24.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"a","component":"vm","service":"compute","hostname":"test-cp1-comp0037-mgmt","tenant_id":"5f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308040000,"value":24.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"b","component":"vm","service":"compute","hostname":"test-cp1-comp0294-mgmt","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000010,"value":26.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"c","component":"vm","service":"compute","hostname":"test-cp1-comp0029-mgmt","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000020,"value":28.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"d","component":"vm","service":"compute","hostname":"test-cp1-comp0329-mgmt","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000030,"value":30.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"e","component":"vm","service":"compute","hostname":"test-cp1-comp0330-mgmt","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308000040,"value":32.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"b","component":"vm","service":"compute","hostname":"test-cp1-comp0294-mgmt","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308006000,"value":26.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.disk.allocation","dimensions":{"resource_id":"b","component":"vm","service":"compute","hostname":"test-cp1-comp0294-mgmt","tenant_id":"6f681592f7084c5fbcd4e8a20a4fef15"},"timestamp":1453308046000,"value":26.0},"meta":{"tenant_id":"tenant_id of metric writer","region":"useast"},"creation_time":1458578365}')
('<message id>','{"metric":{"name":"vm.cpu.utilization_perc","dimensions":{"resource_id":"7b805207-3c97-4ef9-8b28-3fac171f2a96","cloud_name":"poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","cluster":"compute","hostname":"004-cp1-comp0006-mgmt","tenant_id":"817331145b804dc9a7accb6edfb0674d","zone":"nova"},"timestamp":1464283861000,"value":5.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1464283972}')
('<message id>','{"metric":{"name":"vm.cpu.utilization_perc","dimensions":{"resource_id":"a07858d1-aa13-4259-9873-f23170bc9e1b","cloud_name":"poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","cluster":"compute","hostname":"004-cp1-comp0006-mgmt","tenant_id":"817331145b804dc9a7accb6edfb0674d","zone":"nova"},"timestamp":1464283895000,"value":35.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1464283972}')
('<message id>','{"metric":{"name":"vm.cpu.utilization_perc","dimensions":{"resource_id":"4cd09eac-854d-49eb-993d-914271f8e1fe","cloud_name":"poc-hlm-004","component":"vm","control_plane":"ccp","service":"compute","cluster":"compute","hostname":"004-cp1-comp0006-mgmt","tenant_id":"817331145b804dc9a7accb6edfb0674d","zone":"nova"},"timestamp":1464283896000,"value":15.0},"meta":{"tenantId":"e675b49896624464bb2e3152ef92cd11","region":"unset"},"creation_time":1464283972}')

View File

@ -18,6 +18,8 @@ import unittest
from oslo_config import cfg
from pyspark.streaming.kafka import OffsetRange
from monasca_transform.component.usage.fetch_quantity \
import FetchQuantityException
from monasca_transform.config.config_initializer import ConfigInitializer
from monasca_transform.driver.mon_metrics_kafka \
import MonMetricsKafkaProcessor
@ -89,6 +91,41 @@ class TestFetchQuantityAgg(SparkContextTest):
return [json.loads(transform_specs_json_operation)]
def get_invalid_filter_transform_specs_json(self,
field_to_filter,
filter_expression,
filter_operation):
"""get transform_specs driver table info."""
invalid_filter_transform_specs_json = """
{"aggregation_params_map":{
"aggregation_pipeline":{"source":"streaming",
"usage":"fetch_quantity",
"setters":["rollup_quantity",
"set_aggregated_metric_name",
"set_aggregated_period"],
"insert":["prepare_data",
"insert_data"]},
"aggregated_metric_name": "mem.total_mb_agg",
"aggregation_period": "hourly",
"aggregation_group_by_list": ["host", "metric_id"],
"usage_fetch_operation": "sum",
"filter_by_list": [{"field_to_filter": "%s",
"filter_expression": "%s", "filter_operation": "%s"}],
"setter_rollup_group_by_list": ["host"],
"setter_rollup_operation": "sum",
"dimension_list":["aggregation_period",
"host",
"project_id"]
},
"metric_group":"mem_total_all",
"metric_id":"mem_total_all"}"""
populated_invalid_filter_transform_specs_json = \
invalid_filter_transform_specs_json % (field_to_filter,
filter_expression,
filter_operation)
return [json.loads(populated_invalid_filter_transform_specs_json)]
@mock.patch('monasca_transform.data_driven_specs.data_driven_specs_repo.'
'DataDrivenSpecsRepoFactory.get_data_driven_specs_repo')
@mock.patch('monasca_transform.transform.builder.'
@ -731,6 +768,258 @@ class TestFetchQuantityAgg(SparkContextTest):
.get('metric').get('value_meta')
.get('lastrecord_timestamp'))
@mock.patch('monasca_transform.data_driven_specs.data_driven_specs_repo.'
'DataDrivenSpecsRepoFactory.get_data_driven_specs_repo')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_insert_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_setter_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_usage_component_manager')
def test_missing_field_to_filter(self,
usage_manager,
setter_manager,
insert_manager,
data_driven_specs_repo):
# load components
usage_manager.return_value = MockComponentManager.get_usage_cmpt_mgr()
setter_manager.return_value = \
MockComponentManager.get_setter_cmpt_mgr()
insert_manager.return_value = \
MockComponentManager.get_insert_cmpt_mgr()
# init mock driver tables
data_driven_specs_repo.return_value = \
MockDataDrivenSpecsRepo(
self.spark_context,
self.get_pre_transform_specs_json(),
self.get_invalid_filter_transform_specs_json("",
"-mgmt$",
"exclude"))
# Create an emulated set of Kafka messages (these were gathered
# by extracting Monasca messages from the Metrics queue on mini-mon).
# Create an RDD out of the mocked Monasca metrics
with open(DataProvider.fetch_quantity_data_path) as f:
raw_lines = f.read().splitlines()
raw_tuple_list = [eval(raw_line) for raw_line in raw_lines]
rdd_monasca = self.spark_context.parallelize(raw_tuple_list)
# decorate mocked RDD with dummy kafka offsets
myOffsetRanges = [
OffsetRange("metrics", 1, 10, 20)] # mimic rdd.offsetRanges()
transform_context = TransformContextUtils.get_context(
offset_info=myOffsetRanges,
batch_time_info=self.get_dummy_batch_time())
rdd_monasca_with_offsets = rdd_monasca.map(
lambda x: RddTransformContext(x, transform_context))
try:
# Call the primary method in mon_metrics_kafka
MonMetricsKafkaProcessor.rdd_to_recordstore(
rdd_monasca_with_offsets)
# In this case, it's an error if no exception is caught
self.assertTrue(False)
except FetchQuantityException as e:
self.assertTrue("Encountered invalid filter details:" in e.value)
self.assertTrue("field to filter = ," in e.value)
@mock.patch('monasca_transform.data_driven_specs.data_driven_specs_repo.'
'DataDrivenSpecsRepoFactory.get_data_driven_specs_repo')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_insert_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_setter_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_usage_component_manager')
def test_missing_filter_expression(self,
usage_manager,
setter_manager,
insert_manager,
data_driven_specs_repo):
# load components
usage_manager.return_value = MockComponentManager.get_usage_cmpt_mgr()
setter_manager.return_value = \
MockComponentManager.get_setter_cmpt_mgr()
insert_manager.return_value = \
MockComponentManager.get_insert_cmpt_mgr()
# init mock driver tables
data_driven_specs_repo.return_value = \
MockDataDrivenSpecsRepo(
self.spark_context,
self.get_pre_transform_specs_json(),
self.get_invalid_filter_transform_specs_json("host",
"",
"exclude"))
# Create an emulated set of Kafka messages (these were gathered
# by extracting Monasca messages from the Metrics queue on mini-mon).
# Create an RDD out of the mocked Monasca metrics
with open(DataProvider.fetch_quantity_data_path) as f:
raw_lines = f.read().splitlines()
raw_tuple_list = [eval(raw_line) for raw_line in raw_lines]
rdd_monasca = self.spark_context.parallelize(raw_tuple_list)
# decorate mocked RDD with dummy kafka offsets
myOffsetRanges = [
OffsetRange("metrics", 1, 10, 20)] # mimic rdd.offsetRanges()
transform_context = TransformContextUtils.get_context(
offset_info=myOffsetRanges,
batch_time_info=self.get_dummy_batch_time())
rdd_monasca_with_offsets = rdd_monasca.map(
lambda x: RddTransformContext(x, transform_context))
try:
# Call the primary method in mon_metrics_kafka
MonMetricsKafkaProcessor.rdd_to_recordstore(
rdd_monasca_with_offsets)
# In this case, it's an error if no exception is caught
self.assertTrue(False)
except FetchQuantityException as e:
self.assertTrue("Encountered invalid filter details:" in e.value)
self.assertTrue("filter expression = ," in e.value)
@mock.patch('monasca_transform.data_driven_specs.data_driven_specs_repo.'
'DataDrivenSpecsRepoFactory.get_data_driven_specs_repo')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_insert_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_setter_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_usage_component_manager')
def test_missing_filter_operation(self,
usage_manager,
setter_manager,
insert_manager,
data_driven_specs_repo):
# load components
usage_manager.return_value = MockComponentManager.get_usage_cmpt_mgr()
setter_manager.return_value = \
MockComponentManager.get_setter_cmpt_mgr()
insert_manager.return_value = \
MockComponentManager.get_insert_cmpt_mgr()
# init mock driver tables
data_driven_specs_repo.return_value = \
MockDataDrivenSpecsRepo(
self.spark_context,
self.get_pre_transform_specs_json(),
self.get_invalid_filter_transform_specs_json("host",
"-mgmt$",
""))
# Create an emulated set of Kafka messages (these were gathered
# by extracting Monasca messages from the Metrics queue on mini-mon).
# Create an RDD out of the mocked Monasca metrics
with open(DataProvider.fetch_quantity_data_path) as f:
raw_lines = f.read().splitlines()
raw_tuple_list = [eval(raw_line) for raw_line in raw_lines]
rdd_monasca = self.spark_context.parallelize(raw_tuple_list)
# decorate mocked RDD with dummy kafka offsets
myOffsetRanges = [
OffsetRange("metrics", 1, 10, 20)] # mimic rdd.offsetRanges()
transform_context = TransformContextUtils.get_context(
offset_info=myOffsetRanges,
batch_time_info=self.get_dummy_batch_time())
rdd_monasca_with_offsets = rdd_monasca.map(
lambda x: RddTransformContext(x, transform_context))
try:
# Call the primary method in mon_metrics_kafka
MonMetricsKafkaProcessor.rdd_to_recordstore(
rdd_monasca_with_offsets)
# In this case, it's an error if no exception is caught
self.assertTrue(False)
except FetchQuantityException as e:
self.assertTrue("Encountered invalid filter details:" in e.value)
self.assertTrue("filter operation = ." in e.value)
@mock.patch('monasca_transform.data_driven_specs.data_driven_specs_repo.'
'DataDrivenSpecsRepoFactory.get_data_driven_specs_repo')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_insert_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_setter_component_manager')
@mock.patch('monasca_transform.transform.builder.'
'generic_transform_builder.GenericTransformBuilder.'
'_get_usage_component_manager')
def test_invalid_filter_operation(self,
usage_manager,
setter_manager,
insert_manager,
data_driven_specs_repo):
# load components
usage_manager.return_value = MockComponentManager.get_usage_cmpt_mgr()
setter_manager.return_value = \
MockComponentManager.get_setter_cmpt_mgr()
insert_manager.return_value = \
MockComponentManager.get_insert_cmpt_mgr()
# init mock driver tables
data_driven_specs_repo.return_value = \
MockDataDrivenSpecsRepo(
self.spark_context,
self.get_pre_transform_specs_json(),
self.get_invalid_filter_transform_specs_json("host",
"-mgmt$",
"invalid"))
# Create an emulated set of Kafka messages (these were gathered
# by extracting Monasca messages from the Metrics queue on mini-mon).
# Create an RDD out of the mocked Monasca metrics
with open(DataProvider.fetch_quantity_data_path) as f:
raw_lines = f.read().splitlines()
raw_tuple_list = [eval(raw_line) for raw_line in raw_lines]
rdd_monasca = self.spark_context.parallelize(raw_tuple_list)
# decorate mocked RDD with dummy kafka offsets
myOffsetRanges = [
OffsetRange("metrics", 1, 10, 20)] # mimic rdd.offsetRanges()
transform_context = TransformContextUtils.get_context(
offset_info=myOffsetRanges,
batch_time_info=self.get_dummy_batch_time())
rdd_monasca_with_offsets = rdd_monasca.map(
lambda x: RddTransformContext(x, transform_context))
try:
# Call the primary method in mon_metrics_kafka
MonMetricsKafkaProcessor.rdd_to_recordstore(
rdd_monasca_with_offsets)
# In this case, it's an error if no exception is caught
self.assertTrue(False)
except FetchQuantityException as e:
self.assertTrue("Encountered invalid filter details:" in e.value)
self.assertTrue("filter operation = invalid." in e.value)
if __name__ == "__main__":
print("PATH *************************************************************")

View File

@ -131,13 +131,13 @@ class SparkTest(SparkContextTest):
'metric').get('value_meta')
.get('lastrecord_timestamp'))
# Verify cpu.total_logical_cores_agg for mini-mon host
# Verify cpu.total_logical_cores_agg for test-cp1-comp0333-mgmt host
total_cpu_logical_agg_metric = [
value for value in metrics
if value.get('metric').get('name') ==
'cpu.total_logical_cores_agg' and
value.get('metric').get('dimensions').get('host') ==
'mini-mon'][0]
'test-cp1-comp0333-mgmt'][0]
self.assertEqual(9.0,
total_cpu_logical_agg_metric.get(
@ -170,13 +170,13 @@ class SparkTest(SparkContextTest):
'metric').get('value_meta')
.get('lastrecord_timestamp'))
# Verify cpu.total_logical_cores_agg for devstack host
# Verify cpu.total_logical_cores_agg for test-cp1-comp0027-mgmt host
total_cpu_logical_agg_metric = [
value for value in metrics
if value.get('metric').get('name') ==
'cpu.total_logical_cores_agg' and
value.get('metric').get('dimensions').get('host') ==
'devstack'][0]
'test-cp1-comp0027-mgmt'][0]
self.assertEqual(6.0,
total_cpu_logical_agg_metric.get(
@ -249,13 +249,14 @@ class SparkTest(SparkContextTest):
'metric').get('value_meta')
.get('lastrecord_timestamp'))
# Verify cpu.utilized_logical_cores_agg for the mini-mon host
# Verify cpu.utilized_logical_cores_agg for the
# test-cp1-comp0333-mgmt host
utilized_cpu_logical_agg_metric = [
value for value in metrics
if value.get('metric').get('name') ==
'cpu.utilized_logical_cores_agg' and
value.get('metric').get('dimensions').get('host') ==
'mini-mon'][0]
'test-cp1-comp0333-mgmt'][0]
self.assertEqual(5.0,
utilized_cpu_logical_agg_metric.get(
@ -289,13 +290,14 @@ class SparkTest(SparkContextTest):
'metric').get('value_meta')
.get('lastrecord_timestamp'))
# Verify cpu.utilized_logical_cores_agg for the devstack host
# Verify cpu.utilized_logical_cores_agg for the
# test-cp1-comp0027-mgmt host
utilized_cpu_logical_agg_metric = [
value for value in metrics
if value.get('metric').get('name') ==
'cpu.utilized_logical_cores_agg' and
value.get('metric').get('dimensions').get('host') ==
'devstack'][0]
'test-cp1-comp0027-mgmt'][0]
self.assertEqual(3.0,
utilized_cpu_logical_agg_metric.get(