From 0cf08c45c59aa8ae088259c6660e6cb740d90b10 Mon Sep 17 00:00:00 2001 From: Ashwin Agate Date: Wed, 18 Apr 2018 15:09:44 -0700 Subject: [PATCH] Cleanup pre transform and transform specs * Removed unused fields "event_status", "event_version", "record_type", "mount", "device", "pod_name", "container_name" "app", "interface", "deployment" and "daemon_set" from record_store data. Now it is not required to add new dimension, meta or value_meta fields to record store data instead use special notation, e.g. "dimension#" to refer to any dimension field in the incoming metric. * Refactor and eliminate need to add any new metric.dimensions field in multiple places e.g. add to record store and instance usage dataframe schema and in all generic aggregation components. Added a new Map type column called "extra_data_map" to store any new fields, in instance usage data format. Map type column eliminates the need to add new columns to instance usage data. * Allow users to define any fields in "meta", "metric.dimensions" and "metric.value_meta" fields for aggregation in "aggregation_group_by_list" or "setter_group_by_list" using "dimensions#{$field_name}" or "meta#{$field_name}" or "value_meta#{$field_name}" * Updated generic aggregation components and data formats docs. Change-Id: I81a35e048e6bd5649c6b3031ac2722be6a309088 Story: 2001815 Task: 19605 --- docs/data_formats.md | 28 +- docs/generic-aggregation-components.md | 109 +++- .../component/insert/__init__.py | 56 +- .../setter/pre_hourly_calculate_rate.py | 31 +- .../component/setter/rollup_quantity.py | 37 +- .../setter/set_aggregated_metric_name.py | 15 +- .../component/setter/set_aggregated_period.py | 15 +- .../component/usage/calculate_rate.py | 8 - .../component/usage/fetch_quantity.py | 141 ++--- .../component/usage/fetch_quantity_util.py | 16 +- .../config/config_initializer.py | 3 +- .../pre_transform_specs.json | 4 +- .../transform_specs/transform_specs.json | 10 +- monasca_transform/driver/mon_metrics_kafka.py | 106 +--- .../grouping/group_sort_by_timestamp.py | 16 - .../grouping/group_sortby_timestamp.py | 183 ------ .../group_sortby_timestamp_partition.py | 231 -------- .../transform/transform_utils.py | 231 ++++++-- .../pre_transform_specs.json | 14 +- .../test_data_driven_specs.py | 4 +- .../transform_specs/transform_specs.json | 24 +- .../fetch_quantity_data.txt | 12 + .../metrics_pre_hourly_data.txt | 64 +- .../usage/test_pod_net_usage_agg.py | 549 ++++++++++++++++++ 24 files changed, 1057 insertions(+), 850 deletions(-) delete mode 100644 monasca_transform/transform/grouping/group_sortby_timestamp.py delete mode 100644 monasca_transform/transform/grouping/group_sortby_timestamp_partition.py create mode 100644 tests/functional/usage/test_pod_net_usage_agg.py diff --git a/docs/data_formats.md b/docs/data_formats.md index d611030..b29e45c 100644 --- a/docs/data_formats.md +++ b/docs/data_formats.md @@ -52,15 +52,12 @@ Example of a monasca metric: Data Frame Schema: | Column Name | Column Data Type | Description | -| ----------- | ---------------- | ----------- | +| :---------- | :--------------- | :---------- | | event_quantity | `pyspark.sql.types.DoubleType` | mapped to `metric.value`| | event_timestamp_unix | `pyspark.sql.types.DoubleType` | calculated as `metric.timestamp`/`1000` from source metric| | event_timestamp_string | `pyspark.sql.types.StringType` | mapped to `metric.timestamp` from the source metric| | event_type | `pyspark.sql.types.StringType` | placeholder for the future. mapped to `metric.name` from source metric| | event_quantity_name | `pyspark.sql.types.StringType` | mapped to `metric.name` from source metric| -| event_status | `pyspark.sql.types.StringType` | placeholder for the future. Currently mapped to `metric.dimensions.state` from the source metric | -| event_version | `pyspark.sql.types.StringType` | placeholder for the future. Set to "1.0" | -| record_type | `pyspark.sql.types.StringType` | placeholder for the future. Set to "metrics" | | resource_uuid | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.instanceId` or `metric.dimensions.resource_id` from source metric | | tenant_id | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.tenant_id` or `metric.dimensions.tenantid` or `metric.dimensions.project_id` | | user_id | `pyspark.sql.types.StringType` | mapped to `meta.userId` | @@ -68,28 +65,19 @@ Data Frame Schema: | zone | `pyspark.sql.types.StringType` | placeholder for the future. mapped to `meta.zone`, defaults to `event_processing_params.set_default_zone_to` (`pre_transform_spec`) | | host | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.hostname` or `metric.value_meta.host` | | project_id | `pyspark.sql.types.StringType` | mapped to metric tenant_id | -| service_group | `pyspark.sql.types.StringType` | placeholder for the future. mapped to `service_id` in `pre_transform_spec` | -| service_id | `pyspark.sql.types.StringType` | placeholder for the future. mapped to `service_id` in `pre_transform_spec` | | event_date | `pyspark.sql.types.StringType` | "YYYY-mm-dd". Extracted from `metric.timestamp` | | event_hour | `pyspark.sql.types.StringType` | "HH". Extracted from `metric.timestamp` | | event_minute | `pyspark.sql.types.StringType` | "MM". Extracted from `metric.timestamp` | | event_second | `pyspark.sql.types.StringType` | "SS". Extracted from `metric.timestamp` | | metric_group | `pyspark.sql.types.StringType` | identifier for transform spec group | | metric_id | `pyspark.sql.types.StringType` | identifier for transform spec | -| namespace | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.namespace` | -| pod_name | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.pod_name` | -| app | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.app` | -| container_name | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.container_name`| -| interface | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.interface` | -| deployment | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.deployment` | -| daemon_set | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.daemon_set` | ## Instance Usage Data Format ## Data Frame Schema: | Column Name | Column Data Type | Description | -| ----------- | ---------------- | ----------- | +| :---------- | :--------------- | :---------- | | tenant_id | `pyspark.sql.types.StringType` | project_id, defaults to `NA` | | user_id | `pyspark.sql.types.StringType` | user_id, defaults to `NA`| | resource_uuid | `pyspark.sql.types.StringType` | resource_id, defaults to `NA`| @@ -101,24 +89,16 @@ Data Frame Schema: | aggregated_metric_name | `pyspark.sql.types.StringType` | aggregated metric name, defaults to `NA`| | firstrecord_timestamp_string | `pyspark.sql.types.StringType` | timestamp of the first metric used to derive this aggregated metric| | lastrecord_timestamp_string | `pyspark.sql.types.StringType` | timestamp of the last metric used to derive this aggregated metric| -| service_group | `pyspark.sql.types.StringType` | placeholder for the future, defaults to `NA`| -| service_id | `pyspark.sql.types.StringType` | placeholder for the future, defaults to `NA`| | usage_date | `pyspark.sql.types.StringType` | "YYYY-mm-dd" date| | usage_hour | `pyspark.sql.types.StringType` | "HH" hour| | usage_minute | `pyspark.sql.types.StringType` | "MM" minute| | aggregation_period | `pyspark.sql.types.StringType` | "hourly" or "minutely" | -| namespace | `pyspark.sql.types.StringType` | | -| pod_name | `pyspark.sql.types.StringType` | | -| app | `pyspark.sql.types.StringType` | | -| container_name | `pyspark.sql.types.StringType` | | -| interface | `pyspark.sql.types.StringType` | | -| deployment | `pyspark.sql.types.StringType` | | -| daemon_set | `pyspark.sql.types.StringType` | | | firstrecord_timestamp_unix | `pyspark.sql.types.DoubleType` | epoch timestamp of the first metric used to derive this aggregated metric | | lastrecord_timestamp_unix | `pyspark.sql.types.DoubleType` | epoch timestamp of the first metric used to derive this aggregated metric | | quantity | `pyspark.sql.types.DoubleType` | aggregated metric quantity | | record_count | `pyspark.sql.types.DoubleType` | number of source metrics that were used to derive this aggregated metric. For informational purposes only. | -| processing_meta | `pyspark.sql.types.MapType(pyspark.sql.types.StringType, pyspark.sql.types.StringType, True)` | Key-Value pairs to store additional information, to aid processing| +| processing_meta | `pyspark.sql.types.MapType(pyspark.sql.types.StringType, pyspark.sql.types.StringType, True)` | Key-Value pairs to store additional information, to aid processing | +| extra_data_map | `pyspark.sql.types.MapType(pyspark.sql.types.StringType, pyspark.sql.types.StringType, True)` | Key-Value pairs to store group by column key value pair | ## References diff --git a/docs/generic-aggregation-components.md b/docs/generic-aggregation-components.md index f9cb919..cb14560 100644 --- a/docs/generic-aggregation-components.md +++ b/docs/generic-aggregation-components.md @@ -32,6 +32,9 @@ Team and repository tags - [insert_data_pre_hourly](#insert_data_pre_hourly) - [Processors](#processors) - [pre_hourly_processor](#pre_hourly_processor) + - [Special notation](#special-notation) + - [pre_transform spec](#pre_transform-spec) + - [transform spec](#transform-spec) - [Putting it all together](#putting-it-all-together) @@ -74,19 +77,17 @@ Example *pre_transform_spec* for metric "event_type":"cpu.total_logical_cores", "metric_id_list":["cpu_total_all","cpu_total_host","cpu_util_all","cpu_util_host"], "required_raw_fields_list":["creation_time"], - "service_id":"host_metrics" } ``` *List of fields* | field name | values | description | -| ---------- | ------ | ----------- | +| :--------- | :----- | :---------- | | event_processing_params | Set default field values `set_default_zone_to`, `set_default_geolocation_to`, `set_default_region_to`| Set default values for certain fields in the record store data | | event_type | Name of the metric | identifies metric that needs to be aggregated | | metric_id_list | List of `metric_id`'s | List of identifiers, should match `metric_id` in transform specs. This is used by record generation step to generate multiple records if this metric is to be aggregated in multiple ways| -| required_raw_fields_list | List of `field` | List of fields (use JSON dotted notation) that are required in the incoming metric, used for validating incoming metric | -| service_id | service identifier | Identifies the service to which this metric belongs to. Note: this field not yet used | +| required_raw_fields_list | List of `field`'s | List of fields (use [Special notation](#special-notation)) that are required in the incoming metric, used for validating incoming metric. The validator checks if field is present and is not empty. If the field is absent or empty the validator filters such metrics out from aggregation. | ## 2: Data aggregation using generic aggregation components ## @@ -140,7 +141,7 @@ pipelines as required. *List of fields* | field name | values | description | -| ---------- | ------ | ----------- | +| :--------- | :----- | :---------- | | source | ```streaming``` | source is ```streaming```. In the future this can be used to specify a component which can fetch data directly from monasca datastore | | usage | ```fetch_quantity```, ```fetch_quantity_util```, ```calculate_rate``` | [Usage Components](https://github.com/openstack/monasca-transform/tree/master/monasca_transform/component/usage)| | setters | ```pre_hourly_calculate_rate```, ```rollup_quantity```, ```set_aggregated_metric_name```, ```set_aggregated_period``` | [Setter Components](https://github.com/openstack/monasca-transform/tree/master/monasca_transform/component/setter)| @@ -154,16 +155,16 @@ Specifies parameters that generic aggregation components use to process and aggr *List of Other parameters* | Parameter Name | Values | Description | Used by | -| -------------- | ------ | ----------- | ------- | +| :------------- | :----- | :---------- | :------ | | aggregated_metric_name| e.g. "cpu.total_logical_cores_agg" | Name of the aggregated metric | [set_aggregated_metric_name](#set_aggregated_metric_name) | | aggregation_period |"hourly", "minutely" or "secondly" | Period over which to aggregate data. | [fetch_quantity](#fetch_quantity), [fetch_quantity_util](#fetch_quantity_util), [calculate_rate](#calculate_rate), [set_aggregated_period](#set_aggregated_period), [rollup_quantity](#rollup_quantity) |[fetch_quantity](#fetch_quantity), [fetch_quantity_util](#fetch_quantity_util), [calculate_rate](#calculate_rate) | -| aggregation_group_by_list | e.g. "project_id", "hostname" | Group `record_store` data with these columns | [fetch_quantity](#fetch_quantity), [fetch_quantity_util](#fetch_quantity_util), [calculate_rate](#calculate_rate) | +| aggregation_group_by_list | e.g. "project_id", "hostname" | Group `record_store` data with these columns. Please also see [Special notation](#special-notation) below | [fetch_quantity](#fetch_quantity), [fetch_quantity_util](#fetch_quantity_util), [calculate_rate](#calculate_rate) | | usage_fetch_operation | e.g "sum" | After the data is grouped by `aggregation_group_by_list`, perform this operation to find the aggregated metric value | [fetch_quantity](#fetch_quantity), [fetch_quantity_util](#fetch_quantity_util), [calculate_rate](#calculate_rate) | | filter_by_list | Filter regex | Filter data using regex on a `record_store` column value| [fetch_quantity](#fetch_quantity), [fetch_quantity_util](#fetch_quantity_util), [calculate_rate](#calculate_rate) | -| setter_rollup_group_by_list | e.g. "project_id" | Group by these set of fields | [rollup_quantity](#rollup_quantity) | -| setter_rollup_operation | e.g. "avg" | After data is grouped by `setter_rollup_group_by_list`, peform this operation to find aggregated metric value | [rollup_quantity](#rollup_quantity) | -| dimension_list | e.g. "aggregation_period", "host", "project_id" | List of fields which specify dimensions in aggregated metric | [insert_data](#insert_data), [insert_data_pre_hourly](#insert_data_pre_hourly)| -| pre_hourly_group_by_list | e.g. "default" | List of `instance usage data` fields to do a group by operation to aggregate data | [pre_hourly_processor](#pre_hourly_processor) | +| setter_rollup_group_by_list | e.g. "project_id" | Group `instance_usage` data with these columns rollup operation. Please also see [Special notation](#special-notation) below | [rollup_quantity](#rollup_quantity) | +| setter_rollup_operation | e.g. "avg" | After data is grouped by `setter_rollup_group_by_list`, perform this operation to find aggregated metric value | [rollup_quantity](#rollup_quantity) | +| dimension_list | e.g. "aggregation_period", "host", "project_id" | List of fields which specify dimensions in aggregated metric. Please also see [Special notation](#special-notation) below | [insert_data](#insert_data), [insert_data_pre_hourly](#insert_data_pre_hourly)| +| pre_hourly_group_by_list | e.g. "default" | List of `instance usage data` fields to do a group by operation to aggregate data. Please also see [Special notation](#special-notation) below | [pre_hourly_processor](#pre_hourly_processor) | | pre_hourly_operation | e.g. "avg" | When aggregating data published to `metrics_pre_hourly` every hour, perform this operation to find hourly aggregated metric value | [pre_hourly_processor](#pre_hourly_processor) | ### metric_group and metric_id ### @@ -175,7 +176,7 @@ produce the final aggregated metric. *List of fields* | field name | values | description | -| ---------- | ------ | ----------- | +| :--------- | :----- | :---------- | | metric_group | unique transform spec group identifier | group identifier for this transform spec e.g. "cpu_total_all" | | metric_id | unique transform spec identifier | identifier for this transform spec e.g. "cpu_total_all" | @@ -210,7 +211,7 @@ calculation. List of fields to group by. - Possible values: any set of fields in record store data. + Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation). Example: @@ -281,7 +282,7 @@ using the formula given above. List of fields to group by. - Possible values: any set of fields in record store data. + Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation) below. Example: @@ -374,7 +375,7 @@ where, List of fields to group by. - Possible values: any set of fields in record store data. + Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation) below. Example: @@ -485,7 +486,7 @@ group by timestamp field, finds usage based on `setter_fetch_operation`. List of fields to group by. - Possible values: any set of fields in record store data. + Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation) below. Example: ``` @@ -536,7 +537,7 @@ This component converts `instance_usage` data into monasca metric format and wri List of fields in `instance_usage` data that should be converted to monasca metric dimensions. - *Possible values:* any fields in `instance_usage` data + *Possible values:* any fields in `instance_usage` data or use [Special notation](#special-notation) below. Example: ``` @@ -598,7 +599,8 @@ data from previous hour. `instance_usage` data is grouped by `pre_hourly_group_b List of fields to group by. - Possible values: any set of fields in `instance_usage` data or to `default` + Possible values: any set of fields in `instance_usage` data or to `default`. Please also see + [Special notation](#special-notation) below. Note: setting to `default` will group `instance_usage` data by `tenant_id`, `user_id`, `resource_uuid`, `geolocation`, `region`, `zone`, `host`, `project_id`, @@ -627,6 +629,77 @@ data from previous hour. `instance_usage` data is grouped by `pre_hourly_group_b "pre_hourly_operation": "avg" ``` + +## Special notation ## + +### pre_transform spec ### + +To specify `required_raw_fields_list` please use special notation +`dimensions#{$field_name}` or `meta#{$field_name}` or`value_meta#{$field_name}` to refer to any field in +dimension, meta or value_meta field in the incoming raw metric. + +For example if you want to check that for a particular metric say dimension called "pod_name" is +present and is non-empty, then simply add `dimensions#pod_name` to the +`required_raw_fields_list`. + +Example `pre_transform` spec +``` +{"event_processing_params":{"set_default_zone_to":"1", + "set_default_geolocation_to":"1", + "set_default_region_to":"W"}, + "event_type":"pod.net.in_bytes_sec", + "metric_id_list":["pod_net_in_b_per_sec_per_namespace"], + "required_raw_fields_list":["creation_time", + "meta#tenantId", + "dimensions#namespace", + "dimensions#pod_name", + "dimensions#app"] +} +``` + +### transform spec ### + +To specify `aggregation_group_by_list`, `setter_rollup_group_by_list`, `pre_hourly_group_by_list`, +`dimension_list`, you can also use special notation `dimensions#{$field_name}` or `meta#{$field_name}` +or`value_meta#$field_name` to refer to any field in dimension, meta or value_meta field in the +incoming raw metric. + +For example following `transform_spec` will aggregate by "app", "namespace" and "pod_name" +dimensions, then will do a rollup of the aggregated data by "namespace" dimension, and write final +aggregated metric with "app", "namespace" and "pod_name" dimensions. Note that "app" and "pod_name" +will be set to "all" since the final rollup operation was done only based on "namespace" dimension. + +``` +{ + "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":"pod.net.in_bytes_sec_agg", + "aggregation_period":"hourly", + "aggregation_group_by_list": ["tenant_id", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name"], + "usage_fetch_operation": "avg", + "filter_by_list": [], + "setter_rollup_group_by_list":["dimensions#namespace"], + "setter_rollup_operation": "sum", + "dimension_list":["aggregation_period", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name"], + "pre_hourly_operation":"avg", + "pre_hourly_group_by_list":["aggregation_period", + "dimensions#namespace]'"]}, + "metric_group":"pod_net_in_b_per_sec_per_namespace", + "metric_id":"pod_net_in_b_per_sec_per_namespace"} +``` + # Putting it all together Please refer to [Create a new aggregation pipeline](create-new-aggregation-pipeline.md) document to create a new aggregation pipeline. diff --git a/monasca_transform/component/insert/__init__.py b/monasca_transform/component/insert/__init__.py index e18db5f..80c36d2 100644 --- a/monasca_transform/component/insert/__init__.py +++ b/monasca_transform/component/insert/__init__.py @@ -19,6 +19,7 @@ from monasca_common.validation import metrics as metric_validator from monasca_transform.component import Component from monasca_transform.config.config_initializer import ConfigInitializer from monasca_transform.log_utils import LogUtils +from monasca_transform.transform.transform_utils import InstanceUsageUtils from oslo_config import cfg @@ -76,13 +77,12 @@ class InsertComponent(Component): current_epoch_seconds = time.time() current_epoch_milliseconds = current_epoch_seconds * 1000 + log.debug(instance_usage_dict) + + # extract dimensions dimension_list = agg_params["dimension_list"] - # build dimensions dynamically - dimensions_part = {} - for dim in dimension_list: - dimensions_part[dim] = \ - instance_usage_dict.get(dim, - Component.DEFAULT_UNAVAILABLE_VALUE) + dimensions_part = InstanceUsageUtils.extract_dimensions(instance_usage_dict, + dimension_list) meta_part = {} @@ -112,13 +112,15 @@ class InsertComponent(Component): "dimensions": dimensions_part, "timestamp": int(current_epoch_milliseconds), "value": instance_usage_dict.get( - "quantity", 0.0), + "quantity", 0.0), "value_meta": value_meta_part} metric = {"metric": metric_part, "meta": meta_part, "creation_time": int(current_epoch_seconds)} + log.debug(metric) + return metric @staticmethod @@ -140,27 +142,13 @@ class InsertComponent(Component): "lastrecord_timestamp_string": row.lastrecord_timestamp_string, "record_count": row.record_count, - "service_group": row.service_group, - "service_id": row.service_id, "usage_date": row.usage_date, "usage_hour": row.usage_hour, "usage_minute": row.usage_minute, "aggregation_period": row.aggregation_period, - "namespace": - row.namespace, - "pod_name": - row.pod_name, - "app": - row.app, - "container_name": - row.container_name, - "interface": - row.interface, - "deployment": - row.deployment, - "daemon_set": - row.daemon_set} + "extra_data_map": + row.extra_data_map} metric = InsertComponent._prepare_metric(instance_usage_dict, agg_params) return metric @@ -171,11 +159,12 @@ class InsertComponent(Component): """write data to kafka. extracts and formats metric data and writes the data to kafka""" # retrieve the processing meta from the row processing_meta = row.processing_meta + # add transform spec metric id to the processing meta if processing_meta: - processing_meta["metric_id"] = metric_id + processing_meta["metric_id"] = metric_id else: - processing_meta = {"metric_id": metric_id} + processing_meta = {"metric_id": metric_id} instance_usage_dict = {"tenant_id": row.tenant_id, "user_id": row.user_id, @@ -197,28 +186,13 @@ class InsertComponent(Component): "lastrecord_timestamp_unix": row.lastrecord_timestamp_unix, "record_count": row.record_count, - "service_group": row.service_group, - "service_id": row.service_id, "usage_date": row.usage_date, "usage_hour": row.usage_hour, "usage_minute": row.usage_minute, "aggregation_period": row.aggregation_period, "processing_meta": processing_meta, - "namespace": - row.namespace, - "pod_name": - row.pod_name, - "app": - row.app, - "container_name": - row.container_name, - "interface": - row.interface, - "deployment": - row.deployment, - "daemon_set": - row.daemon_set} + "extra_data_map": row.extra_data_map} return instance_usage_dict @staticmethod diff --git a/monasca_transform/component/setter/pre_hourly_calculate_rate.py b/monasca_transform/component/setter/pre_hourly_calculate_rate.py index c6cd8c1..dfc35e3 100644 --- a/monasca_transform/component/setter/pre_hourly_calculate_rate.py +++ b/monasca_transform/component/setter/pre_hourly_calculate_rate.py @@ -15,7 +15,6 @@ from pyspark.sql import functions from pyspark.sql import SQLContext -from monasca_transform.component import Component from monasca_transform.component.setter import SetterComponent from monasca_transform.transform.transform_utils import InstanceUsageUtils @@ -60,11 +59,14 @@ class PreHourlyCalculateRate(SetterComponent): rate_percentage = 100 * ( (oldest_quantity - latest_quantity) / oldest_quantity) + + # get any extra data + extra_data_map = getattr(sorted_oldest_ascending_df.collect()[0], + "extra_data_map", {}) except Exception as e: raise PreHourlyCalculateRateException( "Exception occurred in pre-hourly rate calculation. Error: %s" % str(e)) - # create a new instance usage dict instance_usage_dict = {"tenant_id": latest_dict.get("tenant_id", "all"), @@ -72,20 +74,6 @@ class PreHourlyCalculateRate(SetterComponent): latest_dict.get("user_id", "all"), "resource_uuid": latest_dict.get("resource_uuid", "all"), - "namespace": - latest_dict.get("namespace", "all"), - "pod_name": - latest_dict.get("pod_name", "all"), - "app": - latest_dict.get("app", "all"), - "container_name": - latest_dict.get("container_name", "all"), - "interface": - latest_dict.get("interface", "all"), - "deployment": - latest_dict.get("deployment", "all"), - "daemon_set": - latest_dict.get("daemon_set", "all"), "geolocation": latest_dict.get("geolocation", "all"), "region": @@ -109,19 +97,12 @@ class PreHourlyCalculateRate(SetterComponent): latest_dict["lastrecord_timestamp_string"], "record_count": oldest_dict["record_count"] + latest_dict["record_count"], - "service_group": - latest_dict.get("service_group", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "service_id": - latest_dict.get("service_id", - Component. - DEFAULT_UNAVAILABLE_VALUE), "usage_date": latest_dict["usage_date"], "usage_hour": latest_dict["usage_hour"], "usage_minute": latest_dict["usage_minute"], "aggregation_period": - latest_dict["aggregation_period"] + latest_dict["aggregation_period"], + "extra_data_map": extra_data_map } instance_usage_data_json = json.dumps(instance_usage_dict) diff --git a/monasca_transform/component/setter/rollup_quantity.py b/monasca_transform/component/setter/rollup_quantity.py index 03f7cda..aacf065 100644 --- a/monasca_transform/component/setter/rollup_quantity.py +++ b/monasca_transform/component/setter/rollup_quantity.py @@ -110,6 +110,14 @@ class RollupQuantity(SetterComponent): except AttributeError: processing_meta = {} + # create a column name, value pairs from grouped data + extra_data_map = InstanceUsageUtils.grouped_data_to_map(row, + setter_rollup_group_by_list) + + # convert column names, so that values can be accessed by components + # later in the pipeline + extra_data_map = InstanceUsageUtils.prepare_extra_data_map(extra_data_map) + # create a new instance usage dict instance_usage_dict = {"tenant_id": getattr(row, "tenant_id", "all"), @@ -117,20 +125,6 @@ class RollupQuantity(SetterComponent): getattr(row, "user_id", "all"), "resource_uuid": getattr(row, "resource_uuid", "all"), - "namespace": - getattr(row, "namespace", "all"), - "pod_name": - getattr(row, "pod_name", "all"), - "app": - getattr(row, "app", "all"), - "container_name": - getattr(row, "container_name", "all"), - "interface": - getattr(row, "interface", "all"), - "deployment": - getattr(row, "deployment", "all"), - "daemon_set": - getattr(row, "daemon_set", "all"), "geolocation": getattr(row, "geolocation", "all"), "region": @@ -155,10 +149,6 @@ class RollupQuantity(SetterComponent): "lastrecord_timestamp_string": latest_record_timestamp_string, "record_count": record_count, - "service_group": - getattr(row, "service_group", "all"), - "service_id": - getattr(row, "service_id", "all"), "usage_date": getattr(row, "usage_date", "all"), "usage_hour": @@ -168,7 +158,8 @@ class RollupQuantity(SetterComponent): "aggregation_period": getattr(row, "aggregation_period", "all"), - "processing_meta": processing_meta + "processing_meta": processing_meta, + "extra_data_map": extra_data_map } instance_usage_data_json = json.dumps(instance_usage_dict) @@ -220,6 +211,10 @@ class RollupQuantity(SetterComponent): group_by_columns_list = \ group_by_period_list + setter_rollup_group_by_list + # prepare for group by + group_by_columns_list = InstanceUsageUtils.prepare_instance_usage_group_by_list( + group_by_columns_list) + # perform rollup operation instance_usage_json_rdd = RollupQuantity._rollup_quantity( instance_usage_df, @@ -248,6 +243,10 @@ class RollupQuantity(SetterComponent): group_by_columns_list = group_by_period_list + \ setter_rollup_group_by_list + # prepare for group by + group_by_columns_list = InstanceUsageUtils.prepare_instance_usage_group_by_list( + group_by_columns_list) + # perform rollup operation instance_usage_json_rdd = RollupQuantity._rollup_quantity( instance_usage_df, diff --git a/monasca_transform/component/setter/set_aggregated_metric_name.py b/monasca_transform/component/setter/set_aggregated_metric_name.py index 3268958..3852576 100644 --- a/monasca_transform/component/setter/set_aggregated_metric_name.py +++ b/monasca_transform/component/setter/set_aggregated_metric_name.py @@ -39,16 +39,12 @@ class SetAggregatedMetricName(SetterComponent): except AttributeError: processing_meta = {} + # get any extra data + extra_data_map = getattr(row, "extra_data_map", {}) + instance_usage_dict = {"tenant_id": row.tenant_id, "user_id": row.user_id, "resource_uuid": row.resource_uuid, - "namespace": row.namespace, - "pod_name": row.pod_name, - "app": row.app, - "container_name": row.container_name, - "interface": row.interface, - "deployment": row.deployment, - "daemon_set": row.daemon_set, "geolocation": row.geolocation, "region": row.region, "zone": row.zone, @@ -66,13 +62,12 @@ class SetAggregatedMetricName(SetterComponent): "lastrecord_timestamp_string": row.lastrecord_timestamp_string, "record_count": row.record_count, - "service_group": row.service_group, - "service_id": row.service_id, "usage_date": row.usage_date, "usage_hour": row.usage_hour, "usage_minute": row.usage_minute, "aggregation_period": row.aggregation_period, - "processing_meta": processing_meta} + "processing_meta": processing_meta, + "extra_data_map": extra_data_map} instance_usage_data_json = json.dumps(instance_usage_dict) diff --git a/monasca_transform/component/setter/set_aggregated_period.py b/monasca_transform/component/setter/set_aggregated_period.py index 9ee7d8a..2c70fbd 100644 --- a/monasca_transform/component/setter/set_aggregated_period.py +++ b/monasca_transform/component/setter/set_aggregated_period.py @@ -39,16 +39,12 @@ class SetAggregatedPeriod(SetterComponent): except AttributeError: processing_meta = {} + # get any extra data + extra_data_map = getattr(row, "extra_data_map", {}) + instance_usage_dict = {"tenant_id": row.tenant_id, "user_id": row.user_id, "resource_uuid": row.resource_uuid, - "namespace": row.namespace, - "pod_name": row.pod_name, - "app": row.app, - "container_name": row.container_name, - "interface": row.interface, - "deployment": row.deployment, - "daemon_set": row.daemon_set, "geolocation": row.geolocation, "region": row.region, "zone": row.zone, @@ -66,14 +62,13 @@ class SetAggregatedPeriod(SetterComponent): "lastrecord_timestamp_string": row.lastrecord_timestamp_string, "record_count": row.record_count, - "service_group": row.service_group, - "service_id": row.service_id, "usage_date": row.usage_date, "usage_hour": row.usage_hour, "usage_minute": row.usage_minute, "aggregation_period": agg_params["aggregation_period"], - "processing_meta": processing_meta} + "processing_meta": processing_meta, + "extra_data_map": extra_data_map} instance_usage_data_json = json.dumps(instance_usage_dict) diff --git a/monasca_transform/component/usage/calculate_rate.py b/monasca_transform/component/usage/calculate_rate.py index e780374..99ed186 100644 --- a/monasca_transform/component/usage/calculate_rate.py +++ b/monasca_transform/component/usage/calculate_rate.py @@ -128,14 +128,6 @@ class CalculateRate(UsageComponent): latest_dict["lastrecord_timestamp_string"], "record_count": oldest_dict["record_count"] + latest_dict["record_count"], - "service_group": - latest_dict.get("service_group", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "service_id": - latest_dict.get("service_id", - Component. - DEFAULT_UNAVAILABLE_VALUE), "usage_date": latest_dict["usage_date"], "usage_hour": latest_dict["usage_hour"], "usage_minute": latest_dict["usage_minute"], diff --git a/monasca_transform/component/usage/fetch_quantity.py b/monasca_transform/component/usage/fetch_quantity.py index 7e099bc..39c4ff4 100644 --- a/monasca_transform/component/usage/fetch_quantity.py +++ b/monasca_transform/component/usage/fetch_quantity.py @@ -27,6 +27,7 @@ from monasca_transform.transform.grouping.group_sort_by_timestamp \ from monasca_transform.transform.grouping.group_sort_by_timestamp_partition \ import GroupSortbyTimestampPartition from monasca_transform.transform.transform_utils import InstanceUsageUtils +from monasca_transform.transform.transform_utils import RecordStoreUtils import json @@ -45,18 +46,20 @@ class FetchQuantityException(Exception): return repr(self.value) -GroupedDataWithOperation = namedtuple("GroupedDataWithOperation", - ["grouped_data", - "usage_fetch_operation"]) +GroupedDataNamedTuple = namedtuple("GroupedDataWithOperation", + ["grouped_data", + "usage_fetch_operation", + "group_by_columns_list"]) -class GroupedDataWithOperation(GroupedDataWithOperation): +class GroupedDataNamedTuple(GroupedDataNamedTuple): """A tuple which is a wrapper containing record store data and the usage operation namdetuple contains: grouped_data - grouped record store data - usage_fetch_operation - operation + usage_fetch_operation - operation to be performed on + grouped data group_by_columns_list - list of group by columns """ @@ -75,21 +78,23 @@ class FetchQuantity(UsageComponent): return False @staticmethod - def _get_latest_oldest_quantity(grouping_results_with_operation): - """Method to return an instance usage data + def _get_latest_oldest_quantity(grouped_data_named_tuple): + """Get quantity for each group. - Get quantity for each group by performing the requested - usage operation and return an instance usage data. + By performing the requested usage operation and return a instance usage data. """ - # row - grouping_results = grouping_results_with_operation.\ + grouping_results = grouped_data_named_tuple.\ grouped_data # usage fetch operation - usage_fetch_operation = grouping_results_with_operation.\ + usage_fetch_operation = grouped_data_named_tuple.\ usage_fetch_operation + # group_by_columns_list + group_by_columns_list = grouped_data_named_tuple.\ + group_by_columns_list + group_by_dict = grouping_results.grouping_key_dict # @@ -99,20 +104,6 @@ class FetchQuantity(UsageComponent): Component.DEFAULT_UNAVAILABLE_VALUE) user_id = group_by_dict.get("user_id", Component.DEFAULT_UNAVAILABLE_VALUE) - namespace = group_by_dict.get("namespace", - Component.DEFAULT_UNAVAILABLE_VALUE) - pod_name = group_by_dict.get("pod_name", - Component.DEFAULT_UNAVAILABLE_VALUE) - app = group_by_dict.get("app", - Component.DEFAULT_UNAVAILABLE_VALUE) - container_name = group_by_dict.get("container_name", - Component.DEFAULT_UNAVAILABLE_VALUE) - interface = group_by_dict.get("interface", - Component.DEFAULT_UNAVAILABLE_VALUE) - deployment = group_by_dict.get("deployment", - Component.DEFAULT_UNAVAILABLE_VALUE) - daemon_set = group_by_dict.get("daemon_set", - Component.DEFAULT_UNAVAILABLE_VALUE) geolocation = group_by_dict.get("geolocation", Component.DEFAULT_UNAVAILABLE_VALUE) @@ -148,10 +139,6 @@ class FetchQuantity(UsageComponent): lastrecord_timestamp_string = agg_stats["lastrecord_timestamp_string"] record_count = agg_stats["record_count"] - # service id - service_group = Component.DEFAULT_UNAVAILABLE_VALUE - service_id = Component.DEFAULT_UNAVAILABLE_VALUE - # aggregation period aggregation_period = Component.DEFAULT_UNAVAILABLE_VALUE @@ -159,15 +146,16 @@ class FetchQuantity(UsageComponent): event_type = group_by_dict.get("event_type", Component.DEFAULT_UNAVAILABLE_VALUE) + # add group by fields data to extra data map + # get existing extra_data_map if any + extra_data_map = group_by_dict.get("extra_data_map", {}) + for column_name in group_by_columns_list: + column_value = group_by_dict.get(column_name, Component. + DEFAULT_UNAVAILABLE_VALUE) + extra_data_map[column_name] = column_value + instance_usage_dict = {"tenant_id": tenant_id, "user_id": user_id, "resource_uuid": resource_uuid, - "namespace": namespace, - "pod_name": pod_name, - "app": app, - "container_name": container_name, - "interface": interface, - "deployment": deployment, - "daemon_set": daemon_set, "geolocation": geolocation, "region": region, "zone": zone, "host": host, "aggregated_metric_name": @@ -182,28 +170,32 @@ class FetchQuantity(UsageComponent): "lastrecord_timestamp_string": lastrecord_timestamp_string, "record_count": record_count, - "service_group": service_group, - "service_id": service_id, "usage_date": usage_date, "usage_hour": usage_hour, "usage_minute": usage_minute, "aggregation_period": aggregation_period, - "processing_meta": {"event_type": event_type} + "processing_meta": {"event_type": event_type}, + "extra_data_map": extra_data_map } instance_usage_data_json = json.dumps(instance_usage_dict) return instance_usage_data_json @staticmethod - def _get_quantity(grouped_record_with_operation): + def _get_quantity(grouped_data_named_tuple): # row - row = grouped_record_with_operation.grouped_data + row = grouped_data_named_tuple.grouped_data # usage fetch operation - usage_fetch_operation = grouped_record_with_operation.\ + usage_fetch_operation = grouped_data_named_tuple.\ usage_fetch_operation + # group by columns list + + group_by_columns_list = grouped_data_named_tuple.\ + group_by_columns_list + # first record timestamp # FIXME: beginning of epoch? earliest_record_timestamp_unix = getattr( row, "min(event_timestamp_unix_for_min)", @@ -230,6 +222,14 @@ class FetchQuantity(UsageComponent): select_quant_str = "".join((usage_fetch_operation, "(event_quantity)")) quantity = getattr(row, select_quant_str, 0.0) + # create a column name, value pairs from grouped data + extra_data_map = InstanceUsageUtils.grouped_data_to_map(row, + group_by_columns_list) + + # convert column names, so that values can be accessed by components + # later in the pipeline + extra_data_map = InstanceUsageUtils.prepare_extra_data_map(extra_data_map) + # create a new instance usage dict instance_usage_dict = {"tenant_id": getattr(row, "tenant_id", Component. @@ -262,34 +262,6 @@ class FetchQuantity(UsageComponent): getattr(row, "tenant_id", Component. DEFAULT_UNAVAILABLE_VALUE), - "namespace": - getattr(row, "namespace", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "pod_name": - getattr(row, "pod_name", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "app": - getattr(row, "app", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "container_name": - getattr(row, "container_name", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "interface": - getattr(row, "interface", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "deployment": - getattr(row, "deployment", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "daemon_set": - getattr(row, "daemon_set", - Component. - DEFAULT_UNAVAILABLE_VALUE), "aggregated_metric_name": getattr(row, "aggregated_metric_name", Component. @@ -305,14 +277,6 @@ class FetchQuantity(UsageComponent): "lastrecord_timestamp_string": latest_record_timestamp_string, "record_count": record_count, - "service_group": - getattr(row, "service_group", - Component. - DEFAULT_UNAVAILABLE_VALUE), - "service_id": - getattr(row, "service_id", - Component. - DEFAULT_UNAVAILABLE_VALUE), "usage_date": getattr(row, "event_date", Component. @@ -331,7 +295,8 @@ class FetchQuantity(UsageComponent): DEFAULT_UNAVAILABLE_VALUE), "processing_meta": {"event_type": getattr( row, "event_type", - Component.DEFAULT_UNAVAILABLE_VALUE)} + Component.DEFAULT_UNAVAILABLE_VALUE)}, + "extra_data_map": extra_data_map } instance_usage_data_json = json.dumps(instance_usage_dict) @@ -431,6 +396,10 @@ class FetchQuantity(UsageComponent): group_by_columns_list = group_by_period_list + \ aggregation_group_by_list + # prepare group by columns list + group_by_columns_list = RecordStoreUtils.prepare_recordstore_group_by_list( + group_by_columns_list) + instance_usage_json_rdd = None if (usage_fetch_operation == "latest" or usage_fetch_operation == "oldest"): @@ -466,14 +435,14 @@ class FetchQuantity(UsageComponent): grouped_data_rdd_with_operation = grouped_rows_rdd.map( lambda x: - GroupedDataWithOperation(x, - str(usage_fetch_operation))) + GroupedDataNamedTuple(x, + str(usage_fetch_operation), + group_by_columns_list)) instance_usage_json_rdd = \ grouped_data_rdd_with_operation.map( FetchQuantity._get_latest_oldest_quantity) else: - record_store_df_int = \ record_store_df.select( record_store_df.event_timestamp_unix.alias( @@ -488,14 +457,16 @@ class FetchQuantity(UsageComponent): "event_timestamp_unix_for_min": "min", "event_timestamp_unix_for_max": "max", "event_timestamp_unix": "count"} + # do a group by grouped_data = record_store_df_int.groupBy(*group_by_columns_list) grouped_record_store_df = grouped_data.agg(agg_operations_map) grouped_data_rdd_with_operation = grouped_record_store_df.rdd.map( lambda x: - GroupedDataWithOperation(x, - str(usage_fetch_operation))) + GroupedDataNamedTuple(x, + str(usage_fetch_operation), + group_by_columns_list)) instance_usage_json_rdd = grouped_data_rdd_with_operation.map( FetchQuantity._get_quantity) diff --git a/monasca_transform/component/usage/fetch_quantity_util.py b/monasca_transform/component/usage/fetch_quantity_util.py index 369be93..a322551 100644 --- a/monasca_transform/component/usage/fetch_quantity_util.py +++ b/monasca_transform/component/usage/fetch_quantity_util.py @@ -107,13 +107,16 @@ class FetchQuantityUtil(UsageComponent): record_count = getattr(row, "record_count", Component.DEFAULT_UNAVAILABLE_VALUE) - # service id - service_group = Component.DEFAULT_UNAVAILABLE_VALUE - service_id = Component.DEFAULT_UNAVAILABLE_VALUE - # aggregation period aggregation_period = Component.DEFAULT_UNAVAILABLE_VALUE + # get extra_data_map, if any + extra_data_map = getattr(row, "extra_data_map", {}) + # filter out event_type + extra_data_map_filtered = \ + {key: extra_data_map[key] for key in list(extra_data_map) + if key != 'event_type'} + instance_usage_dict = {"tenant_id": tenant_id, "user_id": user_id, "resource_uuid": resource_uuid, "geolocation": geolocation, "region": region, @@ -130,12 +133,11 @@ class FetchQuantityUtil(UsageComponent): "lastrecord_timestamp_string": lastrecord_timestamp_string, "record_count": record_count, - "service_group": service_group, - "service_id": service_id, "usage_date": usage_date, "usage_hour": usage_hour, "usage_minute": usage_minute, - "aggregation_period": aggregation_period} + "aggregation_period": aggregation_period, + "extra_data_map": extra_data_map_filtered} instance_usage_data_json = json.dumps(instance_usage_dict) diff --git a/monasca_transform/config/config_initializer.py b/monasca_transform/config/config_initializer.py index 3aa5545..8f28584 100644 --- a/monasca_transform/config/config_initializer.py +++ b/monasca_transform/config/config_initializer.py @@ -27,8 +27,7 @@ class ConfigInitializer(object): ConfigInitializer.load_stage_processors_options() ConfigInitializer.load_pre_hourly_processor_options() if not default_config_files: - default_config_files = ['/etc/monasca-transform.conf', - 'etc/monasca-transform.conf'] + default_config_files = ['/etc/monasca-transform.conf'] cfg.CONF(args=[], project='monasca_transform', default_config_files=default_config_files) diff --git a/monasca_transform/data_driven_specs/pre_transform_specs/pre_transform_specs.json b/monasca_transform/data_driven_specs/pre_transform_specs/pre_transform_specs.json index e1838b3..19b81b4 100644 --- a/monasca_transform/data_driven_specs/pre_transform_specs/pre_transform_specs.json +++ b/monasca_transform/data_driven_specs/pre_transform_specs/pre_transform_specs.json @@ -12,6 +12,6 @@ {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"vcpus","metric_id_list":["vcpus_all","vcpus_project"],"required_raw_fields_list":["creation_time","project_id","resource_id"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"vm.cpu.utilization_perc","metric_id_list":["vm_cpu_util_perc_project"],"required_raw_fields_list":["creation_time","tenant_id","resource_id"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"nova.vm.cpu.total_allocated","metric_id_list":["nova_vm_cpu_total_all"],"required_raw_fields_list":["creation_time"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.size","metric_id_list":["swift_total_all","swift_total_host"],"required_raw_fields_list":["creation_time", "hostname", "mount"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.avail","metric_id_list":["swift_avail_all","swift_avail_host","swift_usage_rate"],"required_raw_fields_list":["creation_time", "hostname", "mount"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.size","metric_id_list":["swift_total_all","swift_total_host"],"required_raw_fields_list":["creation_time", "hostname", "dimensions#mount"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.avail","metric_id_list":["swift_avail_all","swift_avail_host","swift_usage_rate"],"required_raw_fields_list":["creation_time", "hostname", "dimensions#mount"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"storage.objects.size","metric_id_list":["storage_objects_size_all"],"required_raw_fields_list":["creation_time", "project_id"],"service_id":"host_metrics"} diff --git a/monasca_transform/data_driven_specs/transform_specs/transform_specs.json b/monasca_transform/data_driven_specs/transform_specs/transform_specs.json index 057c674..4ca352f 100644 --- a/monasca_transform/data_driven_specs/transform_specs/transform_specs.json +++ b/monasca_transform/data_driven_specs/transform_specs/transform_specs.json @@ -18,9 +18,9 @@ {"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":"nova.vm.cpu.total_allocated_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_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_vm_cpu_total_all","metric_id":"nova_vm_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":"swiftlm.diskusage.val.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount"],"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_total_all","metric_id":"swift_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":"swiftlm.diskusage.val.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount"],"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_total_host","metric_id":"swift_total_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":"swiftlm.diskusage.val.avail_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "mount"],"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"],"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_host","metric_id":"swift_avail_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"],"filter_by_list": [],"setter_rollup_group_by_list": [],"dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"rate","pre_hourly_group_by_list":["default"]},"metric_group":"swift_avail_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.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "dimensions#mount"],"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_total_all","metric_id":"swift_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":"swiftlm.diskusage.val.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "dimensions#mount"],"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_total_host","metric_id":"swift_total_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":"swiftlm.diskusage.val.avail_agg","aggregation_period":"hourly","aggregation_group_by_list": ["host", "metric_id", "dimensions#mount"],"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", "dimensions#mount"],"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_host","metric_id":"swift_avail_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", "dimensions#mount"],"filter_by_list": [],"setter_rollup_group_by_list": [],"dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"rate","pre_hourly_group_by_list":["default"]},"metric_group":"swift_avail_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":"storage.objects.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["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":"sum","pre_hourly_group_by_list":["default"]},"metric_group":"storage_objects_size_all","metric_id":"storage_objects_size_all"} diff --git a/monasca_transform/driver/mon_metrics_kafka.py b/monasca_transform/driver/mon_metrics_kafka.py index 66edbd0..d51467e 100644 --- a/monasca_transform/driver/mon_metrics_kafka.py +++ b/monasca_transform/driver/mon_metrics_kafka.py @@ -21,7 +21,6 @@ from pyspark.streaming import StreamingContext from pyspark.sql.functions import explode from pyspark.sql.functions import from_unixtime -from pyspark.sql.functions import lit from pyspark.sql.functions import when from pyspark.sql import SQLContext @@ -52,6 +51,7 @@ from monasca_transform.transform.storage_utils import \ InvalidCacheStorageLevelException from monasca_transform.transform.storage_utils import StorageUtils from monasca_transform.transform.transform_utils import MonMetricUtils +from monasca_transform.transform.transform_utils import PreTransformSpecsUtils from monasca_transform.transform import TransformContextUtils ConfigInitializer.basic_config() @@ -177,8 +177,11 @@ class MonMetricsKafkaProcessor(object): required_fields = row.required_raw_fields_list - invalid_list = [] + # prepare list of required fields, to a rdd syntax to retrieve value + required_fields = PreTransformSpecsUtils.prepare_required_raw_fields_list( + required_fields) + invalid_list = [] for required_field in required_fields: required_field_value = None @@ -188,35 +191,18 @@ class MonMetricsKafkaProcessor(object): except Exception: pass - if (required_field_value is None or required_field_value == "" and - row.metric is not None and - row.metric.dimensions is not None): - # Look for the field in the dimensions layer of the row - try: - required_field_value = eval( - ".".join(("row.metric.dimensions", required_field))) - except Exception: - pass - - if (required_field_value is None or required_field_value == "" and - row.meta is not None): - # Look for the field in the meta layer of the row - try: - required_field_value = eval( - ".".join(("row.meta", required_field))) - except Exception: - pass - if required_field_value is None \ or required_field_value == "": - invalid_list.append("invalid") + invalid_list.append((required_field, + required_field_value)) if len(invalid_list) <= 0: return row else: - print("_validate_raw_mon_metrics : found invalid : ** %s: %s" % ( - (".".join(("row", required_field))), - required_field_value)) + for field_name, field_value in invalid_list: + MonMetricsKafkaProcessor.log_debug( + "_validate_raw_mon_metrics : found invalid field : ** %s: %s" % ( + field_name, field_value)) @staticmethod def process_metric(transform_context, record_store_df): @@ -225,7 +211,6 @@ class MonMetricsKafkaProcessor(object): All the parameters to drive processing should be available in transform_spec_df dataframe. """ - # call processing chain return GenericTransformBuilder.do_transform( transform_context, record_store_df) @@ -378,11 +363,6 @@ class MonMetricsKafkaProcessor(object): gen_mon_metrics_df.event_type.alias("event_quantity_name"), (gen_mon_metrics_df.metric.value / 1.0).alias( "event_quantity"), - when(gen_mon_metrics_df.metric.dimensions.state != '', - gen_mon_metrics_df.metric.dimensions.state).otherwise( - 'NA').alias("event_status"), - lit('1.0').alias('event_version'), - lit('metrics').alias("record_type"), # resource_uuid when(gen_mon_metrics_df.metric.dimensions.instanceId != '', @@ -391,6 +371,7 @@ class MonMetricsKafkaProcessor(object): gen_mon_metrics_df.metric.dimensions.resource_id). otherwise('NA').alias("resource_uuid"), + # tenant_id when(gen_mon_metrics_df.metric.dimensions.tenantId != '', gen_mon_metrics_df.metric.dimensions.tenantId).when( gen_mon_metrics_df.metric.dimensions.tenant_id != '', @@ -399,46 +380,12 @@ class MonMetricsKafkaProcessor(object): gen_mon_metrics_df.metric.dimensions.project_id).otherwise( 'NA').alias("tenant_id"), - when(gen_mon_metrics_df.metric.dimensions.mount != '', - gen_mon_metrics_df.metric.dimensions.mount).otherwise( - 'NA').alias("mount"), - - when(gen_mon_metrics_df.metric.dimensions.device != '', - gen_mon_metrics_df.metric.dimensions.device).otherwise( - 'NA').alias("device"), - - when(gen_mon_metrics_df.metric.dimensions.namespace != '', - gen_mon_metrics_df.metric.dimensions.namespace).otherwise( - 'NA').alias("namespace"), - - when(gen_mon_metrics_df.metric.dimensions.pod_name != '', - gen_mon_metrics_df.metric.dimensions.pod_name).otherwise( - 'NA').alias("pod_name"), - - when(gen_mon_metrics_df.metric.dimensions.container_name != '', - gen_mon_metrics_df.metric.dimensions - .container_name).otherwise('NA').alias("container_name"), - - when(gen_mon_metrics_df.metric.dimensions.app != '', - gen_mon_metrics_df.metric.dimensions.app).otherwise( - 'NA').alias("app"), - - when(gen_mon_metrics_df.metric.dimensions.interface != '', - gen_mon_metrics_df.metric.dimensions.interface).otherwise( - 'NA').alias("interface"), - - when(gen_mon_metrics_df.metric.dimensions.deployment != '', - gen_mon_metrics_df.metric.dimensions - .deployment).otherwise('NA').alias("deployment"), - - when(gen_mon_metrics_df.metric.dimensions.daemon_set != '', - gen_mon_metrics_df.metric.dimensions - .daemon_set).otherwise('NA').alias("daemon_set"), - + # user_id when(gen_mon_metrics_df.meta.userId != '', gen_mon_metrics_df.meta.userId).otherwise('NA').alias( "user_id"), + # region when(gen_mon_metrics_df.meta.region != '', gen_mon_metrics_df.meta.region).when( gen_mon_metrics_df.event_processing_params @@ -447,6 +394,7 @@ class MonMetricsKafkaProcessor(object): .set_default_region_to).otherwise( 'NA').alias("region"), + # zone when(gen_mon_metrics_df.meta.zone != '', gen_mon_metrics_df.meta.zone).when( gen_mon_metrics_df.event_processing_params @@ -455,30 +403,36 @@ class MonMetricsKafkaProcessor(object): .set_default_zone_to).otherwise( 'NA').alias("zone"), + # host when(gen_mon_metrics_df.metric.dimensions.hostname != '', gen_mon_metrics_df.metric.dimensions.hostname).when( gen_mon_metrics_df.metric.value_meta.host != '', gen_mon_metrics_df.metric.value_meta.host).otherwise( 'NA').alias("host"), - when(gen_mon_metrics_df.service_id != '', - gen_mon_metrics_df.service_id).otherwise( - 'NA').alias("service_group"), - - when(gen_mon_metrics_df.service_id != '', - gen_mon_metrics_df.service_id).otherwise( - 'NA').alias("service_id"), - + # event_date from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000, 'yyyy-MM-dd').alias("event_date"), + # event_hour from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000, 'HH').alias("event_hour"), + # event_minute from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000, 'mm').alias("event_minute"), + # event_second from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000, 'ss').alias("event_second"), + gen_mon_metrics_df.this_metric_id.alias("metric_group"), - gen_mon_metrics_df.this_metric_id.alias("metric_id")) + + gen_mon_metrics_df.this_metric_id.alias("metric_id"), + + # metric dimensions + gen_mon_metrics_df.meta.alias("meta"), + # metric dimensions + gen_mon_metrics_df.metric.dimensions.alias("dimensions"), + # metric value_meta + gen_mon_metrics_df.metric.value_meta.alias("value_meta")) # # get transform context diff --git a/monasca_transform/transform/grouping/group_sort_by_timestamp.py b/monasca_transform/transform/grouping/group_sort_by_timestamp.py index 31a3a18..94899c9 100644 --- a/monasca_transform/transform/grouping/group_sort_by_timestamp.py +++ b/monasca_transform/transform/grouping/group_sort_by_timestamp.py @@ -24,22 +24,6 @@ class GroupSortbyTimestamp(Grouping): print(str) # LOG.debug(logStr) - @staticmethod - def _get_group_by_key(row_decorated): - """Build a group by key using the group by column list. - - row_decorated: [[Rows(a=1, b=1, c=2, d=3)],[group_by_a,group_by_b]] - """ - # LOG.debug(whoami(row_decorated)) - # LOG.debug(row_decorated) - - group_by_columns_list = row_decorated[1] - group_by_key = "" - for gcol in group_by_columns_list: - group_by_key = "^".join((group_by_key, - eval(".".join(("row", gcol))))) - return group_by_key - @staticmethod def _prepare_for_group_by(record_store_with_group_by_rdd): """creates a new rdd where: diff --git a/monasca_transform/transform/grouping/group_sortby_timestamp.py b/monasca_transform/transform/grouping/group_sortby_timestamp.py deleted file mode 100644 index 305a14e..0000000 --- a/monasca_transform/transform/grouping/group_sortby_timestamp.py +++ /dev/null @@ -1,183 +0,0 @@ -# Copyright 2016 Hewlett Packard Enterprise Development Company LP -# -# Licensed under the Apache License, Version 2.0 (the "License"); you may -# not use this file except in compliance with the License. You may obtain -# a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT -# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the -# License for the specific language governing permissions and limitations -# under the License. - -from monasca_transform.transform.grouping import Grouping -from monasca_transform.transform.grouping import GroupingResults -from monasca_transform.transform.grouping import RecordStoreWithGroupBy - - -class GroupSortbyTimestamp(Grouping): - - @staticmethod - def log_debug(logStr): - print(str) - # LOG.debug(logStr) - - @staticmethod - def _get_groupby_key(row_decorated): - """Build a group by key using the group by column list. - - row_decorated: [[Rows(a=1, b=1, c=2, d=3)],[groupby_a,groupby_b]] - """ - # LOG.debug(whoami(row_decorated)) - # LOG.debug(row_decorated) - - groupby_columns_list = row_decorated[1] - groupby_key = "" - for gcol in groupby_columns_list: - groupby_key = "^".join((groupby_key, - eval(".".join(("row", gcol))))) - return groupby_key - - @staticmethod - def _prepare_for_groupby(record_store_with_groupby_rdd): - """creates a new rdd where: - - the first element of each row - contains array of grouping key and event timestamp fields. - Grouping key and event timestamp fields are used by - partitioning and sorting function to partition the data - by grouping key and then sort the elements in a group by the - timestamp - """ - - # get the record store data and group by columns - record_store_data = record_store_with_groupby_rdd.record_store_data - - groupby_columns_list = \ - record_store_with_groupby_rdd.groupby_columns_list - - # construct a group by key - # key1=value1^key2=value2^... - groupby_key_value = "" - for gcol in groupby_columns_list: - groupby_key_value = \ - "^".join((groupby_key_value, - "=".join((gcol, - eval(".".join(("record_store_data", - gcol))))))) - - # return a key-value rdd - return [groupby_key_value, record_store_data] - - @staticmethod - def _sortby_timestamp(result_iterable): - # LOG.debug(whoami(result_iterable.data[0])) - - # sort list might cause OOM, if the group has lots of items - # use group_sortby_timestamp_partitions module instead if you run - # into OOM - sorted_list = sorted(result_iterable.data, - key=lambda row: row.event_timestamp_string) - return sorted_list - - @staticmethod - def _group_sortby_timestamp(record_store_df, groupby_columns_list): - # convert the dataframe rdd to normal rdd and add the group by column - # list - record_store_with_groupby_rdd = record_store_df.rdd.\ - map(lambda x: RecordStoreWithGroupBy(x, groupby_columns_list)) - - # convert rdd into key-value rdd - record_store_with_groupby_rdd_key_val = record_store_with_groupby_rdd.\ - map(GroupSortbyTimestamp._prepare_for_groupby) - - first_step = record_store_with_groupby_rdd_key_val.groupByKey() - record_store_rdd_grouped_sorted = first_step.mapValues( - GroupSortbyTimestamp._sortby_timestamp) - - return record_store_rdd_grouped_sorted - - @staticmethod - def _get_group_first_last_quantity_udf(grouplistiter): - """Return stats that include: - - first row key, first_event_timestamp, - first event quantity, last_event_timestamp and last event quantity - """ - first_row = None - last_row = None - - # extract key and value list - group_key = grouplistiter[0] - grouped_values = grouplistiter[1] - - count = 0.0 - for row in grouped_values: - - # set the first row - if first_row is None: - first_row = row - - # set the last row - last_row = row - count = count + 1 - - first_event_timestamp_unix = None - first_event_timestamp_string = None - first_event_quantity = None - - if first_row is not None: - first_event_timestamp_unix = first_row.event_timestamp_unix - first_event_timestamp_string = first_row.event_timestamp_string - first_event_quantity = first_row.event_quantity - - last_event_timestamp_unix = None - last_event_timestamp_string = None - last_event_quantity = None - - if last_row is not None: - last_event_timestamp_unix = last_row.event_timestamp_unix - last_event_timestamp_string = last_row.event_timestamp_string - last_event_quantity = last_row.event_quantity - - results_dict = {"firstrecord_timestamp_unix": - first_event_timestamp_unix, - "firstrecord_timestamp_string": - first_event_timestamp_string, - "firstrecord_quantity": first_event_quantity, - "lastrecord_timestamp_unix": - last_event_timestamp_unix, - "lastrecord_timestamp_string": - last_event_timestamp_string, - "lastrecord_quantity": last_event_quantity, - "record_count": count} - - group_key_dict = Grouping._parse_grouping_key(group_key) - - return GroupingResults(group_key, results_dict, group_key_dict) - - @staticmethod - def fetch_group_latest_oldest_quantity(record_store_df, - transform_spec_df, - groupby_columns_list): - """To group record store data - - sort by timestamp within group - and get first and last timestamp along with quantity within each group - - This function uses key-value pair rdd's groupBy function to do groupby - """ - # group and order elements in group - record_store_grouped_data_rdd = \ - GroupSortbyTimestamp._group_sortby_timestamp(record_store_df, - groupby_columns_list) - - # find stats for a group - record_store_grouped_rows = \ - record_store_grouped_data_rdd.\ - map(GroupSortbyTimestamp. - _get_group_first_last_quantity_udf) - - return record_store_grouped_rows diff --git a/monasca_transform/transform/grouping/group_sortby_timestamp_partition.py b/monasca_transform/transform/grouping/group_sortby_timestamp_partition.py deleted file mode 100644 index 6a63cc1..0000000 --- a/monasca_transform/transform/grouping/group_sortby_timestamp_partition.py +++ /dev/null @@ -1,231 +0,0 @@ -# Copyright 2016 Hewlett Packard Enterprise Development Company LP -# -# Licensed under the Apache License, Version 2.0 (the "License"); you may -# not use this file except in compliance with the License. You may obtain -# a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT -# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the -# License for the specific language governing permissions and limitations -# under the License. - -from monasca_transform.transform.grouping import Grouping -from monasca_transform.transform.grouping import GroupingResults -from monasca_transform.transform.grouping import RecordStoreWithGroupBy - - -class GroupSortbyTimestampPartition(Grouping): - - @staticmethod - def log_debug(logStr): - print(str) - # LOG.debug(logStr) - - @staticmethod - def _get_group_first_last_quantity_udf(partitionlistiter): - """user defined function to to through a list of partitions. - - Each partition contains elements for a group. All the elements are sorted by - timestamp. - The stats include first row key, first_event_timestamp, - fist event quantity, last_event_timestamp and last event quantity - """ - first_row = None - last_row = None - - count = 0.0 - for row in partitionlistiter: - - # set the first row - if first_row is None: - first_row = row - - # set the last row - last_row = row - count = count + 1 - - first_event_timestamp_unix = None - first_event_timestamp_string = None - first_event_quantity = None - first_row_key = None - if first_row is not None: - first_event_timestamp_unix = first_row[1].event_timestamp_unix - first_event_timestamp_string = first_row[1].event_timestamp_string - first_event_quantity = first_row[1].event_quantity - - # extract the grouping_key from composite grouping_key - # composite grouping key is a list, where first item is the - # grouping key and second item is the event_timestamp_string - first_row_key = first_row[0][0] - - last_event_timestamp_unix = None - last_event_timestamp_string = None - last_event_quantity = None - if last_row is not None: - last_event_timestamp_unix = last_row[1].event_timestamp_unix - last_event_timestamp_string = last_row[1].event_timestamp_string - last_event_quantity = last_row[1].event_quantity - - results_dict = {"firstrecord_timestamp_unix": - first_event_timestamp_unix, - "firstrecord_timestamp_string": - first_event_timestamp_string, - "firstrecord_quantity": first_event_quantity, - "lastrecord_timestamp_unix": - last_event_timestamp_unix, - "lastrecord_timestamp_string": - last_event_timestamp_string, - "lastrecord_quantity": last_event_quantity, - "record_count": count} - - first_row_key_dict = Grouping._parse_grouping_key(first_row_key) - - yield [GroupingResults(first_row_key, results_dict, - first_row_key_dict)] - - @staticmethod - def _prepare_for_groupby(record_store_with_groupby_rdd): - """creates a new rdd where: - - the first element of each row - contains array of grouping key and event timestamp fields. - Grouping key and event timestamp fields are used by - partitioning and sorting function to partition the data - by grouping key and then sort the elements in a group by the - timestamp - """ - - # get the record store data and group by columns - record_store_data = record_store_with_groupby_rdd.record_store_data - - groupby_columns_list = \ - record_store_with_groupby_rdd.groupby_columns_list - - # construct a group by key - # key1=value1^key2=value2^... - groupby_key_value = "" - for gcol in groupby_columns_list: - groupby_key_value = \ - "^".join((groupby_key_value, - "=".join((gcol, eval(".".join(("record_store_data", - gcol))))))) - - # return a key-value rdd - # key is a composite key which consists of grouping key and - # event_timestamp_string - return [[groupby_key_value, - record_store_data.event_timestamp_string], record_store_data] - - @staticmethod - def _get_partition_by_group(group_composite): - """get a hash of the grouping key, - - which is then used by partitioning - function to get partition where the groups data should end up in. - It uses hash % num_partitions to get partition - """ - # FIXME: find out of hash function in python gives same value on - # different machines - # Look at using portable_hash method in spark rdd - grouping_key = group_composite[0] - grouping_key_hash = hash(grouping_key) - # log_debug("groupby_sortby_timestamp_partition: got hash : %s" \ - # % str(returnhash)) - return grouping_key_hash - - @staticmethod - def _sortby_timestamp(group_composite): - """get timestamp which will be used to sort grouped data""" - event_timestamp_string = group_composite[1] - return event_timestamp_string - - @staticmethod - def _group_sortby_timestamp_partition(record_store_df, - groupby_columns_list, - num_of_groups): - """It does a group by and then sorts all the items within the group by event timestamp.""" - # convert the dataframe rdd to normal rdd and add the group by - # column list - record_store_with_groupby_rdd = record_store_df.rdd.\ - map(lambda x: RecordStoreWithGroupBy(x, groupby_columns_list)) - - # prepare the data for repartitionAndSortWithinPartitions function - record_store_rdd_prepared = \ - record_store_with_groupby_rdd.\ - map(GroupSortbyTimestampPartition._prepare_for_groupby) - - # repartition data based on a grouping key and sort the items within - # group by timestamp - # give high number of partitions - # numPartitions > number of groups expected, so that each group gets - # allocated a separate partition - record_store_rdd_partitioned_sorted = \ - record_store_rdd_prepared.\ - repartitionAndSortWithinPartitions( - numPartitions=num_of_groups, - partitionFunc=GroupSortbyTimestampPartition. - _get_partition_by_group, - keyfunc=GroupSortbyTimestampPartition. - _sortby_timestamp) - - return record_store_rdd_partitioned_sorted - - @staticmethod - def _remove_none_filter(row): - """remove any rows which have None as grouping key - - [GroupingResults(grouping_key="key1", results={})] rows get created - when partition does not get any grouped data assigned to it - """ - if len(row[0].results) > 0 and row[0].grouping_key is not None: - return row - - @staticmethod - def fetch_group_first_last_quantity(record_store_df, - transform_spec_df, - groupby_columns_list, - num_of_groups): - """Function to group record store data - - Sort by timestamp within group - and get first and last timestamp along with quantity within each group - - To do group by it uses custom partitioning function which creates a new - partition - for each group and uses RDD's repartitionAndSortWithinPartitions - function to do the grouping and sorting within the group. - - This is more scalable than just using RDD's groupby as using this - technique - group is not materialized into a list and stored in memory, but rather - it uses RDD's in built partitioning capability to do the sort - - num_of_groups should be more than expected groups, otherwise the same - partition can get used for two groups which will cause incorrect - results. - """ - - # group and order elements in group using repartition - record_store_grouped_data_rdd = \ - GroupSortbyTimestampPartition.\ - _group_sortby_timestamp_partition(record_store_df, - groupby_columns_list, - num_of_groups) - - # do some operations on all elements in the group - grouping_results_tuple_with_none = \ - record_store_grouped_data_rdd.\ - mapPartitions(GroupSortbyTimestampPartition. - _get_group_first_last_quantity_udf) - - # filter all rows which have no data (where grouping key is None) and - # convert resuts into grouping results tuple - grouping_results_tuple1 = grouping_results_tuple_with_none.\ - filter(GroupSortbyTimestampPartition._remove_none_filter) - - grouping_results_tuple = grouping_results_tuple1.map(lambda x: x[0]) - - return grouping_results_tuple diff --git a/monasca_transform/transform/transform_utils.py b/monasca_transform/transform/transform_utils.py index 10527e5..9b59662 100644 --- a/monasca_transform/transform/transform_utils.py +++ b/monasca_transform/transform/transform_utils.py @@ -20,6 +20,8 @@ from pyspark.sql.types import StringType from pyspark.sql.types import StructField from pyspark.sql.types import StructType +from monasca_transform.component import Component + class TransformUtils(object): """utility methods for different kinds of data.""" @@ -47,10 +49,8 @@ class InstanceUsageUtils(TransformUtils): "geolocation", "region", "zone", "host", "project_id", "aggregated_metric_name", "firstrecord_timestamp_string", "lastrecord_timestamp_string", - "service_group", "service_id", "usage_date", "usage_hour", "usage_minute", - "aggregation_period", "namespace", "pod_name", "app", - "container_name", "interface", "deployment", "daemon_set"] + "aggregation_period"] columns_struct_fields = [StructField(field_name, StringType(), True) for field_name in columns] @@ -70,6 +70,12 @@ class InstanceUsageUtils(TransformUtils): StringType(), True), True)) + + columns_struct_fields.append(StructField("extra_data_map", + MapType(StringType(), + StringType(), + True), + True)) schema = StructType(columns_struct_fields) return schema @@ -81,6 +87,103 @@ class InstanceUsageUtils(TransformUtils): instance_usage_schema_df = sql_context.read.json(jsonrdd, schema) return instance_usage_schema_df + @staticmethod + def prepare_instance_usage_group_by_list(group_by_list): + """Prepare group by list. + + If the group by list contains any instances of "dimensions#", "meta#" or "value_meta#" then + prepend the column value by "extra_data_map." since those columns are available in + extra_data_map column. + + """ + return [InstanceUsageUtils.prepare_group_by_item(item) for item in group_by_list] + + @staticmethod + def prepare_group_by_item(item): + """Prepare group by list item. + + Convert replaces any special "dimensions#", "meta#" or "value_meta#" occurrences into + spark sql syntax to retrieve data from extra_data_map column. + + """ + if (item.startswith("dimensions#") or + item.startswith("meta#") or + item.startswith("value_meta#")): + return ".".join(("extra_data_map", item)) + else: + return item + + @staticmethod + def prepare_extra_data_map(extra_data_map): + """Prepare extra data map. + + Replace any occurances of "dimensions." or "meta." or "value_meta." + to "dimensions#", "meta#" or "value_meta#" in extra_data_map. + + """ + prepared_extra_data_map = {} + for column_name in list(extra_data_map): + column_value = extra_data_map[column_name] + if column_name.startswith("dimensions."): + column_name = column_name.replace("dimensions.", "dimensions#") + elif column_name.startswith("meta."): + column_name = column_name.replace("meta.", "meta#") + elif column_name.startswith("value_meta."): + column_name = column_name.replace("value_meta.", "value_meta#") + elif column_name.startswith("extra_data_map."): + column_name = column_name.replace("extra_data_map.", "") + prepared_extra_data_map[column_name] = column_value + return prepared_extra_data_map + + @staticmethod + def grouped_data_to_map(row, group_by_columns_list): + """Iterate through group by column values from grouped data set and extract any values. + + Return a dictionary which contains original group by columns name and value pairs, if they + are available from the grouped data set. + + """ + extra_data_map = getattr(row, "extra_data_map", {}) + # add group by fields data to extra data map + for column_name in group_by_columns_list: + column_value = getattr(row, column_name, Component. + DEFAULT_UNAVAILABLE_VALUE) + if (column_value == Component.DEFAULT_UNAVAILABLE_VALUE + and (column_name.startswith("dimensions.") + or column_name.startswith("meta.") + or column_name.startswith("value_meta.") + or column_name.startswith("extra_data_map."))): + split_column_name = column_name.split(".", 1)[-1] + column_value = getattr(row, split_column_name, Component. + DEFAULT_UNAVAILABLE_VALUE) + extra_data_map[column_name] = column_value + return extra_data_map + + @staticmethod + def extract_dimensions(instance_usage_dict, dimension_list): + """Extract dimensions from instance usage. + + """ + dimensions_part = {} + # extra_data_map + extra_data_map = instance_usage_dict.get("extra_data_map", {}) + + for dim in dimension_list: + value = instance_usage_dict.get(dim) + if value is None: + # lookup for value in extra_data_map + if len(list(extra_data_map)) > 0: + value = extra_data_map.get(dim, "all") + if dim.startswith("dimensions#"): + dim = dim.replace("dimensions#", "") + elif dim.startswith("meta#"): + dim = dim.replace("meta#", "") + elif dim.startswith("value_meta#"): + dim = dim.replace("value_meta#", "") + dimensions_part[dim] = value + + return dimensions_part + class RecordStoreUtils(TransformUtils): """utility methods to transform record store data.""" @@ -95,9 +198,7 @@ class RecordStoreUtils(TransformUtils): "user_id", "region", "zone", "host", "project_id", "service_group", "service_id", "event_date", "event_hour", "event_minute", - "event_second", "metric_group", "metric_id", - "namespace", "pod_name", "app", "container_name", - "interface", "deployment", "daemon_set"] + "event_second", "metric_group", "metric_id"] columns_struct_fields = [StructField(field_name, StringType(), True) for field_name in columns] @@ -110,6 +211,25 @@ class RecordStoreUtils(TransformUtils): StructField("event_quantity", DoubleType(), True)) + # map to metric meta + columns_struct_fields.append(StructField("meta", + MapType(StringType(), + StringType(), + True), + True)) + # map to dimensions + columns_struct_fields.append(StructField("dimensions", + MapType(StringType(), + StringType(), + True), + True)) + # map to value_meta + columns_struct_fields.append(StructField("value_meta", + MapType(StringType(), + StringType(), + True), + True)) + schema = StructType(columns_struct_fields) return schema @@ -127,6 +247,33 @@ class RecordStoreUtils(TransformUtils): record_store_df = sql_context.read.json(jsonpath, schema) return record_store_df + @staticmethod + def prepare_recordstore_group_by_list(group_by_list): + """Prepare record store group by list. + + If the group by list contains any instances of "dimensions#", "meta#" or "value_meta#" then + convert into proper dotted notation. i.e. "dimensions.", "meta." and + "value_meta." to reference columns in record_store data. + + """ + return [RecordStoreUtils.prepare_group_by_item(item) for item in group_by_list] + + @staticmethod + def prepare_group_by_item(item): + """Prepare record store item for group by. + + Convert replaces any special "dimensions#", "meta#" or "value#meta" occurrences into + "dimensions.", "meta." and value_meta.". + + """ + if item.startswith("dimensions#"): + item = item.replace("dimensions#", "dimensions.") + elif item.startswith("meta#"): + item = item.replace("meta#", "meta.") + elif item.startswith("value_meta#"): + item = item.replace("value_meta#", "value_meta.") + return item + class TransformSpecsUtils(TransformUtils): """utility methods to transform_specs.""" @@ -219,50 +366,34 @@ class MonMetricUtils(TransformUtils): @staticmethod def _get_mon_metric_json_schema(): """get the schema of the incoming monasca metric.""" - dimensions = ["apache_host", "apache_port", "component", - "consumer_group", "device", "hostname", - "mode", "mount", "mount_point", "observer_host", - "process_name", "project_id", "resource_id", "service", - "test_type", "tenantId", "tenant_id", "topic", "url", - "state", "state_description", "instanceId", - "namespace", "pod_name", "app", "container_name", - "interface", "deployment", "daemon_set"] - - dimensions_struct_fields = [ - StructField(field_name, StringType(), True) - for field_name in dimensions] - - value_meta = ["host"] - - value_meta_struct_fields = [ - StructField(field_name, StringType(), True) - for field_name in value_meta] metric_struct_field = StructField( "metric", StructType([StructField("dimensions", - StructType(dimensions_struct_fields)), + MapType(StringType(), + StringType(), + True), + True), StructField("value_meta", - StructType(value_meta_struct_fields)), + MapType(StringType(), + StringType(), + True), + True), StructField("name", StringType(), True), StructField("timestamp", StringType(), True), StructField("value", StringType(), True)]), True) - meta_struct_field = StructField( - "meta", - StructType([StructField("timestamp", StringType(), True), - StructField("region", StringType(), True), - StructField("tenantId", StringType(), True), - StructField("userId", StringType(), True), - StructField("zone", StringType(), True), - StructField("geolocation", StringType(), True)])) + meta_struct_field = StructField("meta", + MapType(StringType(), + StringType(), + True), + True) creation_time_struct_field = StructField("creation_time", StringType(), True) schema = StructType([creation_time_struct_field, meta_struct_field, metric_struct_field]) - return schema @staticmethod @@ -323,6 +454,36 @@ class PreTransformSpecsUtils(TransformUtils): pre_transform_specs_df = sql_context.read.json(jsonpath, schema) return pre_transform_specs_df + @staticmethod + def prepare_required_raw_fields_list(group_by_list): + """Prepare required fields list. + + If the group by list contains any instances of "dimensions#field", "meta#field" or + "value_meta#field" then convert them into metric.dimensions["field"] syntax. + + """ + return [PreTransformSpecsUtils.prepare_required_raw_item(item) for item in group_by_list] + + @staticmethod + def prepare_required_raw_item(item): + """Prepare required field item. + + Convert replaces any special "dimensions#", "meta#" or "value_meta" occurrences into + spark rdd syntax to fetch field value. + + """ + if item.startswith("dimensions#"): + field_name = item.replace("dimensions#", "") + return "metric.dimensions['%s']" % field_name + elif item.startswith("meta#"): + field_name = item.replace("meta#", "") + return "meta['%s']" % field_name + elif item.startswith("value_meta#"): + field_name = item.replace("value_meta#", "") + return "metric.value_meta['%s']" % field_name + else: + return item + class GroupingResultsUtils(TransformUtils): """utility methods to transform record store data.""" diff --git a/tests/functional/data_driven_specs/pre_transform_specs/pre_transform_specs.json b/tests/functional/data_driven_specs/pre_transform_specs/pre_transform_specs.json index 8219984..e07871f 100644 --- a/tests/functional/data_driven_specs/pre_transform_specs/pre_transform_specs.json +++ b/tests/functional/data_driven_specs/pre_transform_specs/pre_transform_specs.json @@ -12,14 +12,14 @@ {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"vcpus","metric_id_list":["vcpus_all","vcpus_project"],"required_raw_fields_list":["creation_time","project_id","resource_id"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"vm.cpu.utilization_perc","metric_id_list":["vm_cpu_util_perc_project"],"required_raw_fields_list":["creation_time","tenant_id","resource_id"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"nova.vm.cpu.total_allocated","metric_id_list":["nova_vm_cpu_total_all"],"required_raw_fields_list":["creation_time"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.size","metric_id_list":["swift_total_all","swift_total_host"],"required_raw_fields_list":["creation_time", "hostname", "mount"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.avail","metric_id_list":["swift_avail_all","swift_avail_host","swift_usage_rate"],"required_raw_fields_list":["creation_time", "hostname", "mount"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.size","metric_id_list":["swift_total_all","swift_total_host"],"required_raw_fields_list":["creation_time", "hostname", "dimensions#mount"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"swiftlm.diskusage.host.val.avail","metric_id_list":["swift_avail_all","swift_avail_host","swift_usage_rate"],"required_raw_fields_list":["creation_time", "hostname", "dimensions#mount"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"storage.objects.size","metric_id_list":["storage_objects_size_all"],"required_raw_fields_list":["creation_time", "project_id"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.net.in_bytes_sec","metric_id_list":["pod_net_in_b_per_sec_total_all","pod_net_in_b_per_sec_per_namespace","pod_net_in_b_per_sec_per_app"],"required_raw_fields_list":["creation_time","tenantId","namespace","pod_name","app"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.net.out_bytes_sec","metric_id_list":["pod_net_out_b_per_sec_total_all","pod_net_out_b_per_sec_per_namespace"],"required_raw_fields_list":["creation_time","tenantId","namespace","pod_name"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.mem.used_bytes","metric_id_list":["pod.mem_used_b_total_all","pod.mem_used_b_per_pod_name","pod.mem_used_b_per_namespace"],"required_raw_fields_list":["creation_time","tenantId","namespace","pod_name"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.cpu.total_time","metric_id_list":["pod_cpu_total_all","pod_cpu_per_namespace"],"required_raw_fields_list":["creation_time","tenantId","namespace"],"service_id":"host_metrics"} -{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"container.cpu.total_time","metric_id_list":["container_cpu_total_time_all","container_cpu_total_time_per_container"],"required_raw_fields_list":["creation_time","tenantId","container_name"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.net.in_bytes_sec","metric_id_list":["pod_net_in_b_per_sec_total_all","pod_net_in_b_per_sec_per_namespace","pod_net_in_b_per_sec_per_app"],"required_raw_fields_list":["creation_time","tenantId","dimensions#namespace","dimensions#pod_name","dimensions#app"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.net.out_bytes_sec","metric_id_list":["pod_net_out_b_per_sec_total_all","pod_net_out_b_per_sec_per_namespace"],"required_raw_fields_list":["creation_time","tenantId","dimensions#namespace","dimensions#pod_name"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.mem.used_bytes","metric_id_list":["pod.mem_used_b_total_all","pod.mem_used_b_per_pod_name","pod.mem_used_b_per_namespace"],"required_raw_fields_list":["creation_time","tenantId","dimensions#namespace","dimensions#pod_name"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"pod.cpu.total_time","metric_id_list":["pod_cpu_total_all","pod_cpu_per_namespace"],"required_raw_fields_list":["creation_time","tenantId","dimensions#namespace"],"service_id":"host_metrics"} +{"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"container.cpu.total_time","metric_id_list":["container_cpu_total_time_all","container_cpu_total_time_per_container"],"required_raw_fields_list":["creation_time","tenantId","dimensions#container_name"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"kubernetes.node.capacity.cpu","metric_id_list":["kubernetes_node_capacity_cpu_all","kubernetes_node_capacity_cpu_per_host"],"required_raw_fields_list":["creation_time","tenantId","hostname"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"cpu.total_time_sec","metric_id_list":["cpu_total_time_sec_all","cpu_total_time_sec_per_host"],"required_raw_fields_list":["creation_time","tenantId","hostname"],"service_id":"host_metrics"} {"event_processing_params":{"set_default_zone_to":"1","set_default_geolocation_to":"1","set_default_region_to":"W"},"event_type":"kubernetes.node.allocatable.cpu","metric_id_list":["kubernetes_node_allocatable_cpu_all","kubernetes_node_allocatable_cpu_per_host"],"required_raw_fields_list":["creation_time","tenantId","hostname"],"service_id":"host_metrics"} diff --git a/tests/functional/data_driven_specs/test_data_driven_specs.py b/tests/functional/data_driven_specs/test_data_driven_specs.py index 0058a13..7ff7950 100644 --- a/tests/functional/data_driven_specs/test_data_driven_specs.py +++ b/tests/functional/data_driven_specs/test_data_driven_specs.py @@ -410,7 +410,7 @@ class TestDataDrivenSpecsRepo(SparkContextTest): self.check_list_field_for_row( row=swiftlm_diskusage_all_row, field_name='required_raw_fields_list', - expected_list=['creation_time', 'hostname', 'mount'], + expected_list=['creation_time', 'hostname', 'dimensions#mount'], ) self.check_dict_field_for_row( row=swiftlm_diskusage_all_row, @@ -439,7 +439,7 @@ class TestDataDrivenSpecsRepo(SparkContextTest): self.check_list_field_for_row( row=swiftlm_diskavail_all_row, field_name='required_raw_fields_list', - expected_list=['creation_time', 'hostname', 'mount'], + expected_list=['creation_time', 'hostname', 'dimensions#mount'], ) self.check_dict_field_for_row( row=swiftlm_diskavail_all_row, diff --git a/tests/functional/data_driven_specs/transform_specs/transform_specs.json b/tests/functional/data_driven_specs/transform_specs/transform_specs.json index 7484527..37322ac 100644 --- a/tests/functional/data_driven_specs/transform_specs/transform_specs.json +++ b/tests/functional/data_driven_specs/transform_specs/transform_specs.json @@ -24,18 +24,18 @@ {"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"],"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_host","metric_id":"swift_avail_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"],"filter_by_list": [],"setter_rollup_group_by_list": [],"dimension_list":["aggregation_period","host","project_id"],"pre_hourly_operation":"rate","pre_hourly_group_by_list":["default"]},"metric_group":"swift_avail_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":"storage.objects.size_agg","aggregation_period":"hourly","aggregation_group_by_list": ["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":"sum","pre_hourly_group_by_list":["default"]},"metric_group":"storage_objects_size_all","metric_id":"storage_objects_size_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":"pod.net.in_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "app", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","app","pod_name","interface","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod_net_in_b_per_sec_total_all","metric_id":"pod_net_in_b_per_sec_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":"pod.net.in_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "app", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","app","pod_name","interface","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "namespace"]},"metric_group":"pod_net_in_b_per_sec_per_namespace","metric_id":"pod_net_in_b_per_sec_per_namespace"} -{"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":"pod.net.in_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "app", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["app"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","app","pod_name","interface","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "app"]},"metric_group":"pod_net_in_b_per_sec_per_app","metric_id":"pod_net_in_b_per_sec_per_app"} -{"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":"pod.net.out_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","pod_name","interface","daemon_set"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod_net_out_b_per_sec_total_all","metric_id":"pod_net_out_b_per_sec_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":"pod.net.out_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","pod_name","interface","daemon_set"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "namespace"]},"metric_group":"pod_net_out_b_per_sec_per_namespace","metric_id":"pod_net_out_b_per_sec_per_namespace"} -{"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":"pod.mem.used_bytes_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","pod_name","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod.mem_used_b_total_all","metric_id":"pod.mem_used_b_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":"pod.mem.used_bytes_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["pod_name"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","pod_name","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "pod_name"]},"metric_group":"pod.mem_used_b_per_pod_name","metric_id":"pod.mem_used_b_per_pod_name"} -{"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":"pod.mem.used_bytes_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "namespace", "pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace","pod_name","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "namespace"]},"metric_group":"pod.mem_used_b_per_namespace","metric_id":"pod.mem_used_b_per_namespace"} -{"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":"pod.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","namespace"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod_cpu_total_all","metric_id":"pod_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":"pod.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","namespace"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","namespace"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "namespace"]},"metric_group":"pod_cpu_per_namespace","metric_id":"pod_cpu_per_namespace"} -{"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":"container.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","container_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","container_name"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"container_cpu_total_time_all","metric_id":"container_cpu_total_time_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":"container.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","container_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["container_name"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","container_name"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "container_name"]},"metric_group":"container_cpu_total_time_per_container","metric_id":"container_cpu_total_time_per_container"} +{"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":"pod.net.in_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#app", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#app","dimensions#pod_name","interface","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod_net_in_b_per_sec_total_all","metric_id":"pod_net_in_b_per_sec_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":"pod.net.in_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#app", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["dimensions#namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#app","dimensions#pod_name","interface","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "dimensions#namespace"]},"metric_group":"pod_net_in_b_per_sec_per_namespace","metric_id":"pod_net_in_b_per_sec_per_namespace"} +{"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":"pod.net.in_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#app", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["dimensions#app"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#app","dimensions#pod_name","interface","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "dimensions#app"]},"metric_group":"pod_net_in_b_per_sec_per_app","metric_id":"pod_net_in_b_per_sec_per_app"} +{"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":"pod.net.out_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#pod_name","interface","daemon_set"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod_net_out_b_per_sec_total_all","metric_id":"pod_net_out_b_per_sec_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":"pod.net.out_bytes_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["dimensions#namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#pod_name","interface","daemon_set"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "dimensions#namespace"]},"metric_group":"pod_net_out_b_per_sec_per_namespace","metric_id":"pod_net_out_b_per_sec_per_namespace"} +{"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":"pod.mem.used_bytes_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#pod_name","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod.mem_used_b_total_all","metric_id":"pod.mem_used_b_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":"pod.mem.used_bytes_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["dimensions#pod_name"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#pod_name","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "dimensions#pod_name"]},"metric_group":"pod.mem_used_b_per_pod_name","metric_id":"pod.mem_used_b_per_pod_name"} +{"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":"pod.mem.used_bytes_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id", "dimensions#namespace", "dimensions#pod_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["dimensions#namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace","dimensions#pod_name","deployment"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "dimensions#namespace"]},"metric_group":"pod.mem_used_b_per_namespace","metric_id":"pod.mem_used_b_per_namespace"} +{"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":"pod.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","dimensions#namespace"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"pod_cpu_total_all","metric_id":"pod_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":"pod.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","dimensions#namespace"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["dimensions#namespace"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#namespace"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "dimensions#namespace"]},"metric_group":"pod_cpu_per_namespace","metric_id":"pod_cpu_per_namespace"} +{"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":"container.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","dimensions#container_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#container_name"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"container_cpu_total_time_all","metric_id":"container_cpu_total_time_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":"container.cpu.total_time_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","dimensions#container_name"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["dimensions#container_name"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","dimensions#container_name"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "dimensions#container_name"]},"metric_group":"container_cpu_total_time_per_container","metric_id":"container_cpu_total_time_per_container"} {"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":"kubernetes.node.capacity.cpu_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","host"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"kubernetes_node_capacity_cpu_all","metric_id":"kubernetes_node_capacity_cpu_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":"kubernetes.node.capacity.cpu_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","host"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":["host"],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["geolocation", "region", "zone", "aggregated_metric_name", "aggregation_period", "host"]},"metric_group":"kubernetes_node_capacity_cpu_per_host","metric_id":"kubernetes_node_capacity_cpu_per_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":"cpu.total_time_sec_agg","aggregation_period":"hourly","aggregation_group_by_list": ["metric_id", "tenant_id","host"],"usage_fetch_operation": "avg","filter_by_list": [],"setter_rollup_group_by_list":[],"setter_rollup_operation": "sum","dimension_list":["aggregation_period","host"],"pre_hourly_operation":"avg","pre_hourly_group_by_list":["default"]},"metric_group":"cpu_total_time_sec_all","metric_id":"cpu_total_time_sec_all"} diff --git a/tests/functional/test_resources/fetch_quantity_data/fetch_quantity_data.txt b/tests/functional/test_resources/fetch_quantity_data/fetch_quantity_data.txt index 3127165..54bd6b8 100644 --- a/tests/functional/test_resources/fetch_quantity_data/fetch_quantity_data.txt +++ b/tests/functional/test_resources/fetch_quantity_data/fetch_quantity_data.txt @@ -2,3 +2,15 @@ ('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308046000,"value":1024.0},"meta":{"tenantId":"583d49ab6aca482da653dd838deaf538","region":"useast"},"creation_time":1453308053}') ('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308000000,"value":4096.0},"meta":{"tenantId":"583d49ab6aca482da653dd838deaf538","region":"useast"},"creation_time":1453308005}') ('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric":{"name":"mem.total_mb","dimensions":{"service":"monitoring","hostname":"mini-mon"},"timestamp":1453308016000,"value":2048.0},"meta":{"tenantId":"583d49ab6aca482da653dd838deaf538","region":"useast"},"creation_time":1453308018}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947372.3062, "value_meta": null, "name": "pod.net.out_bytes_sec", "value": 9554.79, "dimensions": {"interface": "weave", "namespace": "kube-system", "pod_name": "weave-net-633vh", "daemon_set": "weave-net"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947472.3062, "value_meta": null, "name": "pod.net.out_bytes_sec", "value": 8554.79, "dimensions": {"interface": "weave", "namespace": "kube-system", "pod_name": "weave-net-633vh", "daemon_set": "weave-net"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947572.3062, "value_meta": null, "name": "pod.net.out_bytes_sec", "value": 7554.79, "dimensions": {"interface": "weave", "namespace": "kube-system", "pod_name": "weave-net-633vh", "daemon_set": "weave-net"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947372.3062, "value_meta": null, "name": "pod.net.out_bytes_sec", "value": 19554.79, "dimensions": {"interface": "weave", "namespace": "kube-system", "pod_name": "weave-net-633vh", "daemon_set": "weave-net"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947472.3062, "value_meta": null, "name": "pod.net.out_bytes_sec", "value": 18554.79, "dimensions": {"interface": "weave", "namespace": "kube-system", "pod_name": "weave-net-633vh", "daemon_set": "weave-net"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288887000.724, "value_meta": null, "name": "pod.net.in_bytes_sec", "value": 22.94, "dimensions": {"interface": "eth0", "app": "wordpress", "namespace": "website", "pod_name": "wordpress-mysql-376310894-mjr8d", "deployment": "wordpress-mysql"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947479.724, "value_meta": null, "name": "pod.net.in_bytes_sec", "value": 23.94, "dimensions": {"interface": "eth0", "app": "wordpress", "namespace": "website", "pod_name": "wordpress-mysql-376310894-mjr8d", "deployment": "wordpress-mysql"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947579.724, "value_meta": null, "name": "pod.net.in_bytes_sec", "value": 21.94, "dimensions": {"interface": "eth0", "app": "wordpress", "namespace": "website", "pod_name": "wordpress-mysql-376310894-mjr8d", "deployment": "wordpress-mysql"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288887000.724, "value_meta": null, "name": "pod.net.in_bytes_sec", "value": 122.94, "dimensions": {"interface": "eth0", "app": "junk", "namespace": "website2", "pod_name": "wordpress-mysql-376310894-mjr8d", "deployment": "wordpress-mysql"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947479.724, "value_meta": null, "name": "pod.net.in_bytes_sec", "value": 123.94, "dimensions": {"interface": "eth0", "app": "junk", "namespace": "website2", "pod_name": "wordpress-mysql-376310894-mjr8d", "deployment": "wordpress-mysql"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947579.724, "value_meta": null, "name": "pod.net.in_bytes_sec", "value": 121.94, "dimensions": {"interface": "eth0", "app": "junk", "namespace": "website2", "pod_name": "wordpress-mysql-376310894-mjr8d", "deployment": "wordpress-mysql"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') +('583d49ab6aca482da653dd838deaf538load.avg_5_minhostnamedevstack','{"metric": {"timestamp": 1485288947572.3062, "value_meta": null, "name": "pod.net.out_bytes_sec", "value": 17554.79, "dimensions": {"interface": "weave", "namespace": "kube-system", "pod_name": "weave-net-633vh", "daemon_set": "weave-net"}}, "meta": {"region": "useast", "tenantId": "7b1f318cb8aa400abaa9357fbc04ab83"}, "creation_time": 1485288969}') diff --git a/tests/functional/test_resources/metrics_pre_hourly_data/metrics_pre_hourly_data.txt b/tests/functional/test_resources/metrics_pre_hourly_data/metrics_pre_hourly_data.txt index e9cb055..aed830c 100644 --- a/tests/functional/test_resources/metrics_pre_hourly_data/metrics_pre_hourly_data.txt +++ b/tests/functional/test_resources/metrics_pre_hourly_data/metrics_pre_hourly_data.txt @@ -9,49 +9,49 @@ ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"swiftlm.diskusage.rate_agg","record_count":34.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-10 20:47:01","tenant_id":"all","region":"all","usage_hour":"20","usage_date":"2016-06-10","processing_meta":{"oldest_quantity": "5291.0", "latest_timestamp_string": "2016-06-10 20:47:02", "latest_quantity": "4575.0", "metric_id":"swift_usage_rate", "oldest_timestamp_string": "2016-06-10 20:47:01"},"resource_uuid":"all","aggregation_period":"hourly","host":"all","lastrecord_timestamp_string":"2016-06-10 20:27:02","firstrecord_timestamp_unix":1465590421.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1465590422.0,"quantity":17.6502}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.in_bytes_sec_agg","record_count":2.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_in_b_per_sec_total_all"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":50.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.in_bytes_sec_agg","record_count":4.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_in_b_per_sec_total_all"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":100.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.in_bytes_sec_agg","record_count":6.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_in_b_per_sec_per_app"},"resource_uuid":"all","aggregation_period":"hourly","app":"wordpress","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":150.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.in_bytes_sec_agg","record_count":8.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_in_b_per_sec_per_app"},"resource_uuid":"all","aggregation_period":"hourly","app":"wordpress","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":200.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.in_bytes_sec_agg","record_count":6.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_in_b_per_sec_per_app"},"resource_uuid":"all","aggregation_period":"hourly","app":"wordpress","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":150.0,"extra_data_map":{"dimensions#app": "wordpress"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.in_bytes_sec_agg","record_count":8.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_in_b_per_sec_per_app"},"resource_uuid":"all","aggregation_period":"hourly","app":"wordpress","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":200.0,"extra_data_map":{"dimensions#app": "wordpress"}}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":10.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_total_all"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":250.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":12.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_total_all"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":300.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.cpu.total_time_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_cpu_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.out_bytes_sec_agg","record_count":30.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_out_b_per_sec_total_all"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":750.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.out_bytes_sec_agg","record_count":32.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_out_b_per_sec_total_all"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":800.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.out_bytes_sec_agg","record_count":34.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_out_b_per_sec_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":850.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.out_bytes_sec_agg","record_count":36.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_out_b_per_sec_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":900.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.out_bytes_sec_agg","record_count":34.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_out_b_per_sec_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":850.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.net.out_bytes_sec_agg","record_count":36.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod_net_out_b_per_sec_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":900.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":40.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_total_all"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":950.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":42.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_total_all"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1000.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":44.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1050.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":46.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1100.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":48.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1150.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":50.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1200.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":52.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"first_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1250.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":54.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"first_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1300.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":56.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"second_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1350.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":58.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"second_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1400.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":44.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1050.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":46.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1100.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":48.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1150.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":50.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_namespace"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1200.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":52.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"first_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1250.0,"extra_data_map":{"dimensions#pod_name":"first_pod"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":54.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"first_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1300.0,"extra_data_map":{"dimensions#pod_name":"first_pod"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":56.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"second_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":1350.0,"extra_data_map":{"dimensions#pod_name":"second_pod"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"pod.mem.used_bytes_agg","record_count":58.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"pod.mem_used_b_per_pod_name"},"resource_uuid":"all","aggregation_period":"hourly","app":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"second_pod","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":1400.0,"extra_data_map":{"dimensions#pod_name":"second_pod"}}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":10.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_all"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":250.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":12.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_all"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":300.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0,"extra_data_map":{"dimensions#namespace":"first_namespace","dimensions#container_name":"container_1"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0,"extra_data_map":{"dimensions#namespace":"second_namespace","dimensions#container_name":"container_1"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"container.cpu.total_time_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"container_cpu_total_time_per_container"},"resource_uuid":"all","aggregation_period":"hourly","container_name":"container_2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","host":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":10.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_all"},"resource_uuid":"all","aggregation_period":"hourly","host":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":250.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":12.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_all"},"resource_uuid":"all","aggregation_period":"hourly","host":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":300.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.capacity.cpu_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_capacity_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":10.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_all"},"resource_uuid":"all","aggregation_period":"hourly","host":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":250.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":12.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_all"},"resource_uuid":"all","aggregation_period":"hourly","host":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":300.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"cpu.total_time_sec_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"cpu_total_time_sec_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":10.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_all"},"resource_uuid":"all","aggregation_period":"hourly","host":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":250.0}) ('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":12.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_all"},"resource_uuid":"all","aggregation_period":"hourly","host":"all","deployment":"all","interface":"all","namespace":"all","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":300.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0}) -('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":14.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":350.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":16.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"first_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":400.0,"extra_data_map":{"dimensions#namespace":"first_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":18.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:24:59","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host1","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:29:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422184.0,"quantity":450.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) +('',{"service_group":"all","geolocation":"all","aggregated_metric_name":"kubernetes.node.allocatable.cpu_agg","record_count":20.0,"user_id":"all","zone":"all","firstrecord_timestamp_string":"2016-06-20 11:29:44","tenant_id":"all","region":"all","usage_hour":"11","usage_date":"2016-06-20","processing_meta":{"metric_id":"kubernetes_node_allocatable_cpu_per_host"},"resource_uuid":"all","aggregation_period":"hourly","host":"host2","deployment":"all","interface":"all","namespace":"second_namespace","pod_name":"all","lastrecord_timestamp_string":"2016-06-20 11:39:44","firstrecord_timestamp_unix":1466421899.0,"service_id":"all","project_id":"all","usage_minute":"all","lastrecord_timestamp_unix":1466422784.0,"quantity":500.0,"extra_data_map":{"dimensions#namespace":"second_namespace"}}) diff --git a/tests/functional/usage/test_pod_net_usage_agg.py b/tests/functional/usage/test_pod_net_usage_agg.py new file mode 100644 index 0000000..89f8c62 --- /dev/null +++ b/tests/functional/usage/test_pod_net_usage_agg.py @@ -0,0 +1,549 @@ +# (c) Copyright 2017 SUSE LLC +# +# Licensed under the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. You may obtain +# a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +# License for the specific language governing permissions and limitations +# under the License. +import json +import unittest + +import mock +from oslo_config import cfg +from pyspark.streaming.kafka import OffsetRange +from tests.functional.spark_context_test import SparkContextTest +from tests.functional.test_resources.fetch_quantity_data.data_provider \ + import DataProvider +from tests.functional.test_resources.mock_component_manager \ + import MockComponentManager +from tests.functional.test_resources.mock_data_driven_specs_repo \ + import MockDataDrivenSpecsRepo + +from monasca_transform.config.config_initializer import ConfigInitializer +from monasca_transform.driver.mon_metrics_kafka \ + import MonMetricsKafkaProcessor +from monasca_transform.transform import RddTransformContext +from monasca_transform.transform import TransformContextUtils +from tests.functional.messaging.adapter import DummyAdapter + + +class TestPodNetUsageAgg(SparkContextTest): + + def setUp(self): + super(TestPodNetUsageAgg, self).setUp() + # configure the system with a dummy messaging adapter + ConfigInitializer.basic_config( + default_config_files=[ + 'tests/functional/test_resources/config/' + 'test_config_with_dummy_messaging_adapter.conf']) + # reset metric_id list dummy adapter + if not DummyAdapter.adapter_impl: + DummyAdapter.init() + DummyAdapter.adapter_impl.metric_list = [] + + def get_pre_transform_specs_json_all(self): + """get pre_transform_specs driver table info.""" + pre_transform_spec_json = """ + {"event_processing_params":{"set_default_zone_to":"1", + "set_default_geolocation_to":"1", + "set_default_region_to":"W"}, + "event_type":"pod.net.in_bytes_sec", + "metric_id_list":["pod_net_in_b_per_sec_total_all"], + "required_raw_fields_list":["creation_time", + "meta#tenantId", + "dimensions#namespace", + "dimensions#pod_name", + "dimensions#app"], + "service_id":"host_metrics"}""" + return [json.loads(pre_transform_spec_json)] + + def get_transform_specs_json_all(self): + """get transform_specs driver table info.""" + transform_spec_json_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":"pod.net.in_bytes_sec_agg", + "aggregation_period":"hourly", + "aggregation_group_by_list": ["tenant_id", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name", + "dimensions#interface", + "dimensions#deployment"], + "usage_fetch_operation": "avg", + "filter_by_list": [], + "setter_rollup_group_by_list":[], + "setter_rollup_operation": "sum", + "dimension_list":["aggregation_period", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name"], + "pre_hourly_operation":"avg", + "pre_hourly_group_by_list":["default"]}, + "metric_group":"pod_net_in_b_per_sec_total_all", + "metric_id":"pod_net_in_b_per_sec_total_all"}""" + return [json.loads(transform_spec_json_all)] + + def get_pre_transform_specs_json_namespace(self): + """get pre_transform_specs driver table info.""" + pre_transform_spec_json = """ + {"event_processing_params":{"set_default_zone_to":"1", + "set_default_geolocation_to":"1", + "set_default_region_to":"W"}, + "event_type":"pod.net.in_bytes_sec", + "metric_id_list":["pod_net_in_b_per_sec_per_namespace"], + "required_raw_fields_list":["creation_time", + "meta#tenantId", + "dimensions#namespace", + "dimensions#pod_name", + "dimensions#app"], + "service_id":"host_metrics"}""" + return [json.loads(pre_transform_spec_json)] + + def get_transform_specs_json_namespace(self): + """get transform_specs driver table info.""" + transform_spec_json_namespace = """ + {"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":"pod.net.in_bytes_sec_agg", + "aggregation_period":"hourly", + "aggregation_group_by_list": ["tenant_id", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name"], + "usage_fetch_operation": "avg", + "filter_by_list": [], + "setter_rollup_group_by_list":["dimensions#namespace"], + "setter_rollup_operation": "sum", + "dimension_list":["aggregation_period", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name"], + "pre_hourly_operation":"avg", + "pre_hourly_group_by_list":["aggregation_period", + "dimensions#namespace]'"]}, + "metric_group":"pod_net_in_b_per_sec_per_namespace", + "metric_id":"pod_net_in_b_per_sec_per_namespace"}""" + return [json.loads(transform_spec_json_namespace)] + + def get_pre_transform_specs_json_app(self): + """get pre_transform_specs driver table info.""" + pre_transform_spec_json = """ + {"event_processing_params":{"set_default_zone_to":"1", + "set_default_geolocation_to":"1", + "set_default_region_to":"W"}, + "event_type":"pod.net.in_bytes_sec", + "metric_id_list":["pod_net_in_b_per_sec_per_app"], + "required_raw_fields_list":["creation_time", + "meta#tenantId", + "dimensions#namespace", + "dimensions#pod_name", + "dimensions#app"], + "service_id":"host_metrics"}""" + return [json.loads(pre_transform_spec_json)] + + def get_transform_specs_json_app(self): + """get transform_specs driver table info.""" + transform_spec_json_app = """ + {"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":"pod.net.in_bytes_sec_agg", + "aggregation_period":"hourly", + "aggregation_group_by_list": ["tenant_id", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name"], + "usage_fetch_operation": "avg", + "filter_by_list": [], + "setter_rollup_group_by_list":["dimensions#app"], + "setter_rollup_operation": "sum", + "dimension_list":["aggregation_period", + "dimensions#app", + "dimensions#namespace", + "dimensions#pod_name"], + "pre_hourly_operation":"avg", + "pre_hourly_group_by_list":["geolocation", + "region", + "zone", + "aggregated_metric_name", + "aggregation_period", + "dimensions#app"]}, + "metric_group":"pod_net_in_b_per_sec_per_app", + "metric_id":"pod_net_in_b_per_sec_per_app"}""" + return [json.loads(transform_spec_json_app)] + + @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_pod_net_in_usage_all(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_all(), + self.get_transform_specs_json_all()) + + # 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)) + + # Call the primary method in mon_metrics_kafka + MonMetricsKafkaProcessor.rdd_to_recordstore( + rdd_monasca_with_offsets) + + # get the metrics that have been submitted to the dummy message adapter + metrics = DummyAdapter.adapter_impl.metric_list + + pod_net_usage_agg_metric = [ + value for value in metrics + if value.get('metric').get('name') == + 'pod.net.in_bytes_sec_agg' and + value.get('metric').get('dimensions').get('app') == + 'all' and + value.get('metric').get('dimensions').get('namespace') == + 'all' and + value.get('metric').get('dimensions').get('pod_name') == + 'all'][0] + + self.assertTrue(pod_net_usage_agg_metric is not None) + + self.assertEqual('pod.net.in_bytes_sec_agg', + pod_net_usage_agg_metric + .get('metric').get('name')) + + self.assertEqual('all', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('app')) + + self.assertEqual('all', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('namespace')) + + self.assertEqual('all', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('pod_name')) + + self.assertEqual(145.88, + pod_net_usage_agg_metric + .get('metric').get('value')) + self.assertEqual('useast', + pod_net_usage_agg_metric + .get('meta').get('region')) + + self.assertEqual(cfg.CONF.messaging.publish_kafka_project_id, + pod_net_usage_agg_metric + .get('meta').get('tenantId')) + + self.assertEqual('hourly', + pod_net_usage_agg_metric + .get('metric').get('dimensions') + .get('aggregation_period')) + + self.assertEqual(6.0, + pod_net_usage_agg_metric + .get('metric').get('value_meta').get('record_count')) + self.assertEqual('2017-01-24 20:14:47', + pod_net_usage_agg_metric + .get('metric').get('value_meta') + .get('firstrecord_timestamp_string')) + self.assertEqual('2017-01-24 20:15:47', + pod_net_usage_agg_metric + .get('metric').get('value_meta') + .get('lastrecord_timestamp_string')) + + @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_pod_net_in_usage_namespace(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_namespace(), + self.get_transform_specs_json_namespace()) + + # 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)) + + # Call the primary method in mon_metrics_kafka + MonMetricsKafkaProcessor.rdd_to_recordstore( + rdd_monasca_with_offsets) + + # get the metrics that have been submitted to the dummy message adapter + metrics = DummyAdapter.adapter_impl.metric_list + + pod_net_usage_agg_metric = [ + value for value in metrics + if value.get('metric').get('name') == + 'pod.net.in_bytes_sec_agg' and + value.get('metric').get('dimensions').get('app') == + 'all' and + value.get('metric').get('dimensions').get('namespace') == + 'website' and + value.get('metric').get('dimensions').get('pod_name') == + 'all'][0] + + self.assertTrue(pod_net_usage_agg_metric is not None) + + self.assertEqual('pod.net.in_bytes_sec_agg', + pod_net_usage_agg_metric + .get('metric').get('name')) + + self.assertEqual('all', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('app')) + + self.assertEqual('website', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('namespace')) + + self.assertEqual('all', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('pod_name')) + + self.assertEqual(22.94, + pod_net_usage_agg_metric + .get('metric').get('value')) + self.assertEqual('useast', + pod_net_usage_agg_metric + .get('meta').get('region')) + + self.assertEqual(cfg.CONF.messaging.publish_kafka_project_id, + pod_net_usage_agg_metric + .get('meta').get('tenantId')) + + self.assertEqual('hourly', + pod_net_usage_agg_metric + .get('metric').get('dimensions') + .get('aggregation_period')) + + self.assertEqual(3.0, + pod_net_usage_agg_metric + .get('metric').get('value_meta').get('record_count')) + self.assertEqual('2017-01-24 20:14:47', + pod_net_usage_agg_metric + .get('metric').get('value_meta') + .get('firstrecord_timestamp_string')) + self.assertEqual('2017-01-24 20:15:47', + pod_net_usage_agg_metric + .get('metric').get('value_meta') + .get('lastrecord_timestamp_string')) + + @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_pod_net_in_usage_app(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_app(), + self.get_transform_specs_json_app()) + + # 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)) + + # Call the primary method in mon_metrics_kafka + MonMetricsKafkaProcessor.rdd_to_recordstore( + rdd_monasca_with_offsets) + + # get the metrics that have been submitted to the dummy message adapter + metrics = DummyAdapter.adapter_impl.metric_list + + pod_net_usage_agg_metric = [ + value for value in metrics + if value.get('metric').get('name') == + 'pod.net.in_bytes_sec_agg' and + value.get('metric').get('dimensions').get('app') == + 'junk' and + value.get('metric').get('dimensions').get('namespace') == + 'all' and + value.get('metric').get('dimensions').get('pod_name') == + 'all'][0] + + self.assertTrue(pod_net_usage_agg_metric is not None) + + self.assertEqual('pod.net.in_bytes_sec_agg', + pod_net_usage_agg_metric + .get('metric').get('name')) + + self.assertEqual('junk', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('app')) + + self.assertEqual('all', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('namespace')) + + self.assertEqual('all', + pod_net_usage_agg_metric + .get("metric").get('dimensions').get('pod_name')) + + self.assertEqual(122.94, + pod_net_usage_agg_metric + .get('metric').get('value')) + self.assertEqual('useast', + pod_net_usage_agg_metric + .get('meta').get('region')) + + self.assertEqual(cfg.CONF.messaging.publish_kafka_project_id, + pod_net_usage_agg_metric + .get('meta').get('tenantId')) + + self.assertEqual('hourly', + pod_net_usage_agg_metric + .get('metric').get('dimensions') + .get('aggregation_period')) + + self.assertEqual(3.0, + pod_net_usage_agg_metric + .get('metric').get('value_meta').get('record_count')) + self.assertEqual('2017-01-24 20:14:47', + pod_net_usage_agg_metric + .get('metric').get('value_meta') + .get('firstrecord_timestamp_string')) + self.assertEqual('2017-01-24 20:15:47', + pod_net_usage_agg_metric + .get('metric').get('value_meta') + .get('lastrecord_timestamp_string')) + +if __name__ == "__main__": + print("PATH *************************************************************") + import sys + print(sys.path) + print("PATH==============================================================") + unittest.main()