Browse Source

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
Ashwin Agate 1 year ago
parent
commit
0cf08c45c5
24 changed files with 1057 additions and 850 deletions
  1. 4
    24
      docs/data_formats.md
  2. 91
    18
      docs/generic-aggregation-components.md
  3. 15
    41
      monasca_transform/component/insert/__init__.py
  4. 6
    25
      monasca_transform/component/setter/pre_hourly_calculate_rate.py
  5. 18
    19
      monasca_transform/component/setter/rollup_quantity.py
  6. 5
    10
      monasca_transform/component/setter/set_aggregated_metric_name.py
  7. 5
    10
      monasca_transform/component/setter/set_aggregated_period.py
  8. 0
    8
      monasca_transform/component/usage/calculate_rate.py
  9. 56
    85
      monasca_transform/component/usage/fetch_quantity.py
  10. 9
    7
      monasca_transform/component/usage/fetch_quantity_util.py
  11. 1
    2
      monasca_transform/config/config_initializer.py
  12. 2
    2
      monasca_transform/data_driven_specs/pre_transform_specs/pre_transform_specs.json
  13. 5
    5
      monasca_transform/data_driven_specs/transform_specs/transform_specs.json
  14. 30
    76
      monasca_transform/driver/mon_metrics_kafka.py
  15. 0
    16
      monasca_transform/transform/grouping/group_sort_by_timestamp.py
  16. 0
    183
      monasca_transform/transform/grouping/group_sortby_timestamp.py
  17. 0
    231
      monasca_transform/transform/grouping/group_sortby_timestamp_partition.py
  18. 196
    35
      monasca_transform/transform/transform_utils.py
  19. 7
    7
      tests/functional/data_driven_specs/pre_transform_specs/pre_transform_specs.json
  20. 2
    2
      tests/functional/data_driven_specs/test_data_driven_specs.py
  21. 12
    12
      tests/functional/data_driven_specs/transform_specs/transform_specs.json
  22. 12
    0
      tests/functional/test_resources/fetch_quantity_data/fetch_quantity_data.txt
  23. 32
    32
      tests/functional/test_resources/metrics_pre_hourly_data/metrics_pre_hourly_data.txt
  24. 549
    0
      tests/functional/usage/test_pod_net_usage_agg.py

+ 4
- 24
docs/data_formats.md View File

@@ -52,15 +52,12 @@ Example of a monasca metric:
52 52
 Data Frame Schema:
53 53
 
54 54
 | Column Name | Column Data Type | Description |
55
-| ----------- | ---------------- | ----------- |
55
+| :---------- | :--------------- | :---------- |
56 56
 | event_quantity | `pyspark.sql.types.DoubleType` | mapped to `metric.value`|
57 57
 | event_timestamp_unix | `pyspark.sql.types.DoubleType` | calculated as `metric.timestamp`/`1000` from source metric|
58 58
 | event_timestamp_string | `pyspark.sql.types.StringType` | mapped to `metric.timestamp` from the source metric|
59 59
 | event_type | `pyspark.sql.types.StringType` | placeholder for the future. mapped to `metric.name` from source metric|
60 60
 | event_quantity_name | `pyspark.sql.types.StringType` | mapped to `metric.name` from source metric|
61
-| event_status | `pyspark.sql.types.StringType` | placeholder for the future. Currently mapped to `metric.dimensions.state` from the source metric  |
62
-| event_version | `pyspark.sql.types.StringType` | placeholder for the future. Set to "1.0" |
63
-| record_type | `pyspark.sql.types.StringType` | placeholder for the future. Set to "metrics" |
64 61
 | resource_uuid | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.instanceId` or `metric.dimensions.resource_id` from source metric  |
65 62
 | tenant_id | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.tenant_id` or `metric.dimensions.tenantid` or `metric.dimensions.project_id`  |
66 63
 | user_id | `pyspark.sql.types.StringType` | mapped to `meta.userId` |
@@ -68,28 +65,19 @@ Data Frame Schema:
68 65
 | 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`) |
69 66
 | host | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.hostname` or `metric.value_meta.host` |
70 67
 | project_id | `pyspark.sql.types.StringType` | mapped to metric tenant_id |
71
-| service_group | `pyspark.sql.types.StringType` | placeholder for the future. mapped to `service_id` in `pre_transform_spec` |
72
-| service_id | `pyspark.sql.types.StringType` | placeholder for the future. mapped to `service_id` in `pre_transform_spec` |
73 68
 | event_date | `pyspark.sql.types.StringType` | "YYYY-mm-dd". Extracted from `metric.timestamp` |
74 69
 | event_hour | `pyspark.sql.types.StringType` | "HH". Extracted from `metric.timestamp` |
75 70
 | event_minute | `pyspark.sql.types.StringType` | "MM". Extracted from `metric.timestamp` |
76 71
 | event_second | `pyspark.sql.types.StringType` | "SS". Extracted from `metric.timestamp` |
77 72
 | metric_group | `pyspark.sql.types.StringType` | identifier for transform spec group |
78 73
 | metric_id | `pyspark.sql.types.StringType` | identifier for transform spec |
79
-| namespace | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.namespace` |
80
-| pod_name | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.pod_name` |
81
-| app | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.app` |
82
-| container_name | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.container_name`|
83
-| interface | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.interface` |
84
-| deployment | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.deployment` |
85
-| daemon_set | `pyspark.sql.types.StringType` | mapped to `metric.dimensions.daemon_set` |
86 74
 
87 75
 ## Instance Usage Data Format ##
88 76
 
89 77
 Data Frame Schema:
90 78
 
91 79
 | Column Name | Column Data Type | Description |
92
-| ----------- | ---------------- | ----------- |
80
+| :---------- | :--------------- | :---------- |
93 81
 | tenant_id | `pyspark.sql.types.StringType` | project_id, defaults to `NA` |
94 82
 | user_id | `pyspark.sql.types.StringType` | user_id, defaults to `NA`|
95 83
 | resource_uuid | `pyspark.sql.types.StringType` | resource_id, defaults to `NA`|
@@ -101,24 +89,16 @@ Data Frame Schema:
101 89
 | aggregated_metric_name | `pyspark.sql.types.StringType` | aggregated metric name, defaults to `NA`|
102 90
 | firstrecord_timestamp_string | `pyspark.sql.types.StringType` | timestamp of the first metric used to derive this aggregated metric|
103 91
 | lastrecord_timestamp_string | `pyspark.sql.types.StringType` | timestamp of the last metric used to derive this aggregated metric|
104
-| service_group | `pyspark.sql.types.StringType` | placeholder for the future, defaults to `NA`|
105
-| service_id | `pyspark.sql.types.StringType` | placeholder for the future, defaults to `NA`|
106 92
 | usage_date | `pyspark.sql.types.StringType` | "YYYY-mm-dd" date|
107 93
 | usage_hour | `pyspark.sql.types.StringType` | "HH" hour|
108 94
 | usage_minute | `pyspark.sql.types.StringType` | "MM" minute|
109 95
 | aggregation_period | `pyspark.sql.types.StringType` | "hourly" or "minutely"  |
110
-| namespace | `pyspark.sql.types.StringType` | |
111
-| pod_name | `pyspark.sql.types.StringType` | |
112
-| app | `pyspark.sql.types.StringType` | |
113
-| container_name | `pyspark.sql.types.StringType` | |
114
-| interface | `pyspark.sql.types.StringType` | |
115
-| deployment | `pyspark.sql.types.StringType` | |
116
-| daemon_set | `pyspark.sql.types.StringType` | |
117 96
 | firstrecord_timestamp_unix | `pyspark.sql.types.DoubleType` | epoch timestamp of the first metric used to derive this aggregated metric |
118 97
 | lastrecord_timestamp_unix | `pyspark.sql.types.DoubleType` | epoch timestamp of the first metric used to derive this aggregated metric |
119 98
 | quantity | `pyspark.sql.types.DoubleType` | aggregated metric quantity |
120 99
 | record_count | `pyspark.sql.types.DoubleType` | number of source metrics that were used to derive this aggregated metric. For informational purposes only. |
121
-| 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|
100
+| 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 |
101
+| 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 |
122 102
 
123 103
 ## References
124 104
 

+ 91
- 18
docs/generic-aggregation-components.md View File

@@ -32,6 +32,9 @@ Team and repository tags
32 32
       - [insert_data_pre_hourly](#insert_data_pre_hourly)
33 33
   - [Processors](#processors)
34 34
     - [pre_hourly_processor](#pre_hourly_processor)
35
+  - [Special notation](#special-notation)
36
+    - [pre_transform spec](#pre_transform-spec)
37
+    - [transform spec](#transform-spec)
35 38
 - [Putting it all together](#putting-it-all-together)
36 39
 
37 40
 <!-- END doctoc generated TOC please keep comment here to allow auto update -->
@@ -74,19 +77,17 @@ Example *pre_transform_spec* for metric
74 77
   "event_type":"cpu.total_logical_cores",
75 78
   "metric_id_list":["cpu_total_all","cpu_total_host","cpu_util_all","cpu_util_host"],
76 79
   "required_raw_fields_list":["creation_time"],
77
-  "service_id":"host_metrics"
78 80
 }
79 81
 ```
80 82
 
81 83
 *List of fields*
82 84
 
83 85
 | field name | values | description |
84
-| ---------- | ------ | ----------- |
86
+| :--------- | :----- | :---------- |
85 87
 | 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 |
86 88
 | event_type | Name of the metric | identifies metric that needs to be aggregated |
87 89
 | 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|
88
-| 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 |
89
-| service_id | service identifier | Identifies the service to which this metric belongs to. Note: this field not yet used  |
90
+| 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. |
90 91
 
91 92
 ## 2: Data aggregation using generic aggregation components ##
92 93
 
@@ -140,7 +141,7 @@ pipelines as required.
140 141
 *List of fields*
141 142
 
142 143
 | field name | values | description |
143
-| ---------- | ------ | ----------- |
144
+| :--------- | :----- | :---------- |
144 145
 | source | ```streaming``` | source is ```streaming```. In the future this can be used to specify a component which can fetch data directly from monasca datastore |
145 146
 | usage | ```fetch_quantity```, ```fetch_quantity_util```, ```calculate_rate``` | [Usage Components](https://github.com/openstack/monasca-transform/tree/master/monasca_transform/component/usage)|
146 147
 | 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
154 155
 *List of Other parameters*
155 156
 
156 157
 | Parameter Name | Values | Description | Used by |
157
-| -------------- | ------ | ----------- | ------- |
158
+| :------------- | :----- | :---------- | :------ |
158 159
 | aggregated_metric_name| e.g. "cpu.total_logical_cores_agg" | Name of the aggregated metric | [set_aggregated_metric_name](#set_aggregated_metric_name) |
159 160
 | 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) |
160
-| 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) |
161
+| 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) |
161 162
 | 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) |
162 163
 | 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) |
163
-| setter_rollup_group_by_list | e.g. "project_id" | Group by these set of fields | [rollup_quantity](#rollup_quantity) |
164
-| 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) |
165
-| 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)|
166
-| 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) |
164
+| 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) |
165
+| 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) |
166
+| 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)|
167
+| 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) |
167 168
 | 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) |
168 169
 
169 170
 ### metric_group and metric_id ###
@@ -175,7 +176,7 @@ produce the final aggregated metric.
175 176
 *List of fields*
176 177
 
177 178
 | field name | values | description |
178
-| ---------- | ------ | ----------- |
179
+| :--------- | :----- | :---------- |
179 180
 | metric_group | unique transform spec group identifier | group identifier for this transform spec e.g. "cpu_total_all" |
180 181
 | metric_id | unique transform spec identifier | identifier for this transform spec e.g. "cpu_total_all" |
181 182
 
@@ -210,7 +211,7 @@ calculation.
210 211
 
211 212
     List of fields to group by.
212 213
 
213
-    Possible values: any set of fields in record store data.
214
+    Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation).
214 215
 
215 216
     Example:
216 217
 
@@ -281,7 +282,7 @@ using the formula given above.
281 282
 
282 283
     List of fields to group by.
283 284
 
284
-    Possible values: any set of fields in record store data.
285
+    Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation) below.
285 286
 
286 287
     Example:
287 288
 
@@ -374,7 +375,7 @@ where,
374 375
 
375 376
     List of fields to group by.
376 377
 
377
-    Possible values: any set of fields in record store data.
378
+    Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation) below.
378 379
 
379 380
     Example:
380 381
 
@@ -485,7 +486,7 @@ group by timestamp field, finds usage based on `setter_fetch_operation`.
485 486
 
486 487
     List of fields to group by.
487 488
 
488
-    Possible values: any set of fields in record store data.
489
+    Possible values: any set of fields in record store data. Please also see [Special notation](#special-notation) below.
489 490
 
490 491
     Example:
491 492
     ```
@@ -536,7 +537,7 @@ This component converts `instance_usage` data into monasca metric format and wri
536 537
 
537 538
     List of fields in `instance_usage` data that should be converted to monasca metric dimensions.
538 539
 
539
-    *Possible values:* any fields in `instance_usage` data
540
+    *Possible values:* any fields in `instance_usage` data or use [Special notation](#special-notation) below.
540 541
 
541 542
     Example:
542 543
     ```
@@ -598,7 +599,8 @@ data from previous hour. `instance_usage` data is grouped by `pre_hourly_group_b
598 599
 
599 600
     List of fields to group by.
600 601
 
601
-    Possible values: any set of fields in `instance_usage` data or to `default`
602
+    Possible values: any set of fields in `instance_usage` data or to `default`. Please also see
603
+    [Special notation](#special-notation) below.
602 604
 
603 605
     Note: setting to `default` will group `instance_usage` data by `tenant_id`, `user_id`,
604 606
     `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
627 629
     "pre_hourly_operation": "avg"
628 630
     ```
629 631
 
632
+
633
+## Special notation ##
634
+
635
+### pre_transform spec ###
636
+
637
+To specify `required_raw_fields_list` please use special notation
638
+`dimensions#{$field_name}` or `meta#{$field_name}` or`value_meta#{$field_name}` to refer to any field in
639
+dimension, meta or value_meta field in the incoming raw metric.
640
+
641
+For example if you want to check that for a particular metric say dimension called "pod_name" is
642
+present and is non-empty, then simply add `dimensions#pod_name` to the
643
+`required_raw_fields_list`.
644
+
645
+Example `pre_transform` spec
646
+```
647
+{"event_processing_params":{"set_default_zone_to":"1",
648
+                            "set_default_geolocation_to":"1",
649
+                            "set_default_region_to":"W"},
650
+ "event_type":"pod.net.in_bytes_sec",
651
+ "metric_id_list":["pod_net_in_b_per_sec_per_namespace"],
652
+ "required_raw_fields_list":["creation_time",
653
+                             "meta#tenantId",
654
+                             "dimensions#namespace",
655
+                             "dimensions#pod_name",
656
+                             "dimensions#app"]
657
+}
658
+```
659
+
660
+### transform spec ###
661
+
662
+To specify `aggregation_group_by_list`, `setter_rollup_group_by_list`, `pre_hourly_group_by_list`,
663
+`dimension_list`, you can also use special notation `dimensions#{$field_name}` or `meta#{$field_name}`
664
+or`value_meta#$field_name` to refer to any field in dimension, meta or value_meta field in the
665
+incoming raw metric.
666
+
667
+For example following `transform_spec` will aggregate by "app", "namespace" and "pod_name"
668
+dimensions, then will do a rollup of the aggregated data by "namespace" dimension, and write final
669
+aggregated metric with "app", "namespace" and "pod_name" dimensions.  Note that "app" and "pod_name"
670
+will be set to "all" since the final rollup operation was done only based on "namespace" dimension.
671
+
672
+```
673
+{
674
+ "aggregation_params_map":{
675
+       "aggregation_pipeline":{"source":"streaming",
676
+       "usage":"fetch_quantity",
677
+       "setters":["rollup_quantity",
678
+                  "set_aggregated_metric_name",
679
+                  "set_aggregated_period"],
680
+       "insert":["prepare_data",
681
+                 "insert_data_pre_hourly"]},
682
+       "aggregated_metric_name":"pod.net.in_bytes_sec_agg",
683
+       "aggregation_period":"hourly",
684
+       "aggregation_group_by_list": ["tenant_id",
685
+                                     "dimensions#app",
686
+                                     "dimensions#namespace",
687
+                                     "dimensions#pod_name"],
688
+       "usage_fetch_operation": "avg",
689
+       "filter_by_list": [],
690
+       "setter_rollup_group_by_list":["dimensions#namespace"],
691
+       "setter_rollup_operation": "sum",
692
+       "dimension_list":["aggregation_period",
693
+                         "dimensions#app",
694
+                         "dimensions#namespace",
695
+                         "dimensions#pod_name"],
696
+       "pre_hourly_operation":"avg",
697
+       "pre_hourly_group_by_list":["aggregation_period",
698
+                                   "dimensions#namespace]'"]},
699
+ "metric_group":"pod_net_in_b_per_sec_per_namespace",
700
+ "metric_id":"pod_net_in_b_per_sec_per_namespace"}
701
+```
702
+
630 703
 # Putting it all together
631 704
 Please refer to [Create a new aggregation pipeline](create-new-aggregation-pipeline.md) document to
632 705
 create a new aggregation pipeline.

+ 15
- 41
monasca_transform/component/insert/__init__.py View File

@@ -19,6 +19,7 @@ from monasca_common.validation import metrics as metric_validator
19 19
 from monasca_transform.component import Component
20 20
 from monasca_transform.config.config_initializer import ConfigInitializer
21 21
 from monasca_transform.log_utils import LogUtils
22
+from monasca_transform.transform.transform_utils import InstanceUsageUtils
22 23
 
23 24
 from oslo_config import cfg
24 25
 
@@ -76,13 +77,12 @@ class InsertComponent(Component):
76 77
         current_epoch_seconds = time.time()
77 78
         current_epoch_milliseconds = current_epoch_seconds * 1000
78 79
 
80
+        log.debug(instance_usage_dict)
81
+
82
+        # extract dimensions
79 83
         dimension_list = agg_params["dimension_list"]
80
-        # build dimensions dynamically
81
-        dimensions_part = {}
82
-        for dim in dimension_list:
83
-            dimensions_part[dim] = \
84
-                instance_usage_dict.get(dim,
85
-                                        Component.DEFAULT_UNAVAILABLE_VALUE)
84
+        dimensions_part = InstanceUsageUtils.extract_dimensions(instance_usage_dict,
85
+                                                                dimension_list)
86 86
 
87 87
         meta_part = {}
88 88
 
@@ -112,13 +112,15 @@ class InsertComponent(Component):
112 112
                        "dimensions": dimensions_part,
113 113
                        "timestamp": int(current_epoch_milliseconds),
114 114
                        "value": instance_usage_dict.get(
115
-                       "quantity", 0.0),
115
+                           "quantity", 0.0),
116 116
                        "value_meta": value_meta_part}
117 117
 
118 118
         metric = {"metric": metric_part,
119 119
                   "meta": meta_part,
120 120
                   "creation_time": int(current_epoch_seconds)}
121 121
 
122
+        log.debug(metric)
123
+
122 124
         return metric
123 125
 
124 126
     @staticmethod
@@ -140,27 +142,13 @@ class InsertComponent(Component):
140 142
                                "lastrecord_timestamp_string":
141 143
                                    row.lastrecord_timestamp_string,
142 144
                                "record_count": row.record_count,
143
-                               "service_group": row.service_group,
144
-                               "service_id": row.service_id,
145 145
                                "usage_date": row.usage_date,
146 146
                                "usage_hour": row.usage_hour,
147 147
                                    "usage_minute": row.usage_minute,
148 148
                                "aggregation_period":
149 149
                                    row.aggregation_period,
150
-                               "namespace":
151
-                                   row.namespace,
152
-                               "pod_name":
153
-                                   row.pod_name,
154
-                               "app":
155
-                                   row.app,
156
-                               "container_name":
157
-                                   row.container_name,
158
-                               "interface":
159
-                                   row.interface,
160
-                               "deployment":
161
-                                   row.deployment,
162
-                               "daemon_set":
163
-                                   row.daemon_set}
150
+                               "extra_data_map":
151
+                                   row.extra_data_map}
164 152
         metric = InsertComponent._prepare_metric(instance_usage_dict,
165 153
                                                  agg_params)
166 154
         return metric
@@ -171,11 +159,12 @@ class InsertComponent(Component):
171 159
         """write data to kafka. extracts and formats metric data and writes the data to kafka"""
172 160
         # retrieve the processing meta from the row
173 161
         processing_meta = row.processing_meta
162
+
174 163
         # add transform spec metric id to the processing meta
175 164
         if processing_meta:
176
-                processing_meta["metric_id"] = metric_id
165
+            processing_meta["metric_id"] = metric_id
177 166
         else:
178
-                processing_meta = {"metric_id": metric_id}
167
+            processing_meta = {"metric_id": metric_id}
179 168
 
180 169
         instance_usage_dict = {"tenant_id": row.tenant_id,
181 170
                                "user_id": row.user_id,
@@ -197,28 +186,13 @@ class InsertComponent(Component):
197 186
                                "lastrecord_timestamp_unix":
198 187
                                    row.lastrecord_timestamp_unix,
199 188
                                "record_count": row.record_count,
200
-                               "service_group": row.service_group,
201
-                               "service_id": row.service_id,
202 189
                                "usage_date": row.usage_date,
203 190
                                "usage_hour": row.usage_hour,
204 191
                                "usage_minute": row.usage_minute,
205 192
                                "aggregation_period":
206 193
                                    row.aggregation_period,
207 194
                                "processing_meta": processing_meta,
208
-                               "namespace":
209
-                                   row.namespace,
210
-                               "pod_name":
211
-                                   row.pod_name,
212
-                               "app":
213
-                                   row.app,
214
-                               "container_name":
215
-                                   row.container_name,
216
-                               "interface":
217
-                                   row.interface,
218
-                               "deployment":
219
-                                   row.deployment,
220
-                               "daemon_set":
221
-                                   row.daemon_set}
195
+                               "extra_data_map": row.extra_data_map}
222 196
         return instance_usage_dict
223 197
 
224 198
     @staticmethod

+ 6
- 25
monasca_transform/component/setter/pre_hourly_calculate_rate.py View File

@@ -15,7 +15,6 @@
15 15
 from pyspark.sql import functions
16 16
 from pyspark.sql import SQLContext
17 17
 
18
-from monasca_transform.component import Component
19 18
 from monasca_transform.component.setter import SetterComponent
20 19
 from monasca_transform.transform.transform_utils import InstanceUsageUtils
21 20
 
@@ -60,11 +59,14 @@ class PreHourlyCalculateRate(SetterComponent):
60 59
 
61 60
             rate_percentage = 100 * (
62 61
                 (oldest_quantity - latest_quantity) / oldest_quantity)
62
+
63
+            # get any extra data
64
+            extra_data_map = getattr(sorted_oldest_ascending_df.collect()[0],
65
+                                     "extra_data_map", {})
63 66
         except Exception as e:
64 67
             raise PreHourlyCalculateRateException(
65 68
                 "Exception occurred in pre-hourly rate calculation. Error: %s"
66 69
                 % str(e))
67
-
68 70
         #  create a new instance usage dict
69 71
         instance_usage_dict = {"tenant_id":
70 72
                                latest_dict.get("tenant_id", "all"),
@@ -72,20 +74,6 @@ class PreHourlyCalculateRate(SetterComponent):
72 74
                                latest_dict.get("user_id", "all"),
73 75
                                "resource_uuid":
74 76
                                latest_dict.get("resource_uuid", "all"),
75
-                               "namespace":
76
-                                   latest_dict.get("namespace", "all"),
77
-                               "pod_name":
78
-                                   latest_dict.get("pod_name", "all"),
79
-                               "app":
80
-                                   latest_dict.get("app", "all"),
81
-                               "container_name":
82
-                                   latest_dict.get("container_name", "all"),
83
-                               "interface":
84
-                                   latest_dict.get("interface", "all"),
85
-                               "deployment":
86
-                                   latest_dict.get("deployment", "all"),
87
-                               "daemon_set":
88
-                                   latest_dict.get("daemon_set", "all"),
89 77
                                "geolocation":
90 78
                                latest_dict.get("geolocation", "all"),
91 79
                                "region":
@@ -109,19 +97,12 @@ class PreHourlyCalculateRate(SetterComponent):
109 97
                                latest_dict["lastrecord_timestamp_string"],
110 98
                                "record_count": oldest_dict["record_count"] +
111 99
                                latest_dict["record_count"],
112
-                               "service_group":
113
-                               latest_dict.get("service_group",
114
-                                               Component.
115
-                                               DEFAULT_UNAVAILABLE_VALUE),
116
-                               "service_id":
117
-                               latest_dict.get("service_id",
118
-                                               Component.
119
-                                               DEFAULT_UNAVAILABLE_VALUE),
120 100
                                "usage_date": latest_dict["usage_date"],
121 101
                                "usage_hour": latest_dict["usage_hour"],
122 102
                                "usage_minute": latest_dict["usage_minute"],
123 103
                                "aggregation_period":
124
-                               latest_dict["aggregation_period"]
104
+                               latest_dict["aggregation_period"],
105
+                               "extra_data_map": extra_data_map
125 106
                                }
126 107
 
127 108
         instance_usage_data_json = json.dumps(instance_usage_dict)

+ 18
- 19
monasca_transform/component/setter/rollup_quantity.py View File

@@ -110,6 +110,14 @@ class RollupQuantity(SetterComponent):
110 110
             except AttributeError:
111 111
                 processing_meta = {}
112 112
 
113
+            # create a column name, value pairs from grouped data
114
+            extra_data_map = InstanceUsageUtils.grouped_data_to_map(row,
115
+                                                                    setter_rollup_group_by_list)
116
+
117
+            # convert column names, so that values can be accessed by components
118
+            # later in the pipeline
119
+            extra_data_map = InstanceUsageUtils.prepare_extra_data_map(extra_data_map)
120
+
113 121
             # create a new instance usage dict
114 122
             instance_usage_dict = {"tenant_id": getattr(row, "tenant_id",
115 123
                                                         "all"),
@@ -117,20 +125,6 @@ class RollupQuantity(SetterComponent):
117 125
                                        getattr(row, "user_id", "all"),
118 126
                                    "resource_uuid":
119 127
                                        getattr(row, "resource_uuid", "all"),
120
-                                   "namespace":
121
-                                       getattr(row, "namespace", "all"),
122
-                                   "pod_name":
123
-                                       getattr(row, "pod_name", "all"),
124
-                                   "app":
125
-                                       getattr(row, "app", "all"),
126
-                                   "container_name":
127
-                                       getattr(row, "container_name", "all"),
128
-                                   "interface":
129
-                                       getattr(row, "interface", "all"),
130
-                                   "deployment":
131
-                                       getattr(row, "deployment", "all"),
132
-                                   "daemon_set":
133
-                                       getattr(row, "daemon_set", "all"),
134 128
                                    "geolocation":
135 129
                                        getattr(row, "geolocation", "all"),
136 130
                                    "region":
@@ -155,10 +149,6 @@ class RollupQuantity(SetterComponent):
155 149
                                    "lastrecord_timestamp_string":
156 150
                                        latest_record_timestamp_string,
157 151
                                    "record_count": record_count,
158
-                                   "service_group":
159
-                                       getattr(row, "service_group", "all"),
160
-                                   "service_id":
161
-                                       getattr(row, "service_id", "all"),
162 152
                                    "usage_date":
163 153
                                        getattr(row, "usage_date", "all"),
164 154
                                    "usage_hour":
@@ -168,7 +158,8 @@ class RollupQuantity(SetterComponent):
168 158
                                    "aggregation_period":
169 159
                                        getattr(row, "aggregation_period",
170 160
                                                "all"),
171
-                                   "processing_meta": processing_meta
161
+                                   "processing_meta": processing_meta,
162
+                                   "extra_data_map": extra_data_map
172 163
                                    }
173 164
 
174 165
             instance_usage_data_json = json.dumps(instance_usage_dict)
@@ -220,6 +211,10 @@ class RollupQuantity(SetterComponent):
220 211
         group_by_columns_list = \
221 212
             group_by_period_list + setter_rollup_group_by_list
222 213
 
214
+        # prepare for group by
215
+        group_by_columns_list = InstanceUsageUtils.prepare_instance_usage_group_by_list(
216
+            group_by_columns_list)
217
+
223 218
         # perform rollup operation
224 219
         instance_usage_json_rdd = RollupQuantity._rollup_quantity(
225 220
             instance_usage_df,
@@ -248,6 +243,10 @@ class RollupQuantity(SetterComponent):
248 243
         group_by_columns_list = group_by_period_list + \
249 244
             setter_rollup_group_by_list
250 245
 
246
+        # prepare for group by
247
+        group_by_columns_list = InstanceUsageUtils.prepare_instance_usage_group_by_list(
248
+            group_by_columns_list)
249
+
251 250
         # perform rollup operation
252 251
         instance_usage_json_rdd = RollupQuantity._rollup_quantity(
253 252
             instance_usage_df,

+ 5
- 10
monasca_transform/component/setter/set_aggregated_metric_name.py View File

@@ -39,16 +39,12 @@ class SetAggregatedMetricName(SetterComponent):
39 39
         except AttributeError:
40 40
             processing_meta = {}
41 41
 
42
+        # get any extra data
43
+        extra_data_map = getattr(row, "extra_data_map", {})
44
+
42 45
         instance_usage_dict = {"tenant_id": row.tenant_id,
43 46
                                "user_id": row.user_id,
44 47
                                "resource_uuid": row.resource_uuid,
45
-                               "namespace": row.namespace,
46
-                               "pod_name": row.pod_name,
47
-                               "app": row.app,
48
-                               "container_name": row.container_name,
49
-                               "interface": row.interface,
50
-                               "deployment": row.deployment,
51
-                               "daemon_set": row.daemon_set,
52 48
                                "geolocation": row.geolocation,
53 49
                                "region": row.region,
54 50
                                "zone": row.zone,
@@ -66,13 +62,12 @@ class SetAggregatedMetricName(SetterComponent):
66 62
                                "lastrecord_timestamp_string":
67 63
                                    row.lastrecord_timestamp_string,
68 64
                                "record_count": row.record_count,
69
-                               "service_group": row.service_group,
70
-                               "service_id": row.service_id,
71 65
                                "usage_date": row.usage_date,
72 66
                                "usage_hour": row.usage_hour,
73 67
                                "usage_minute": row.usage_minute,
74 68
                                "aggregation_period": row.aggregation_period,
75
-                               "processing_meta": processing_meta}
69
+                               "processing_meta": processing_meta,
70
+                               "extra_data_map": extra_data_map}
76 71
 
77 72
         instance_usage_data_json = json.dumps(instance_usage_dict)
78 73
 

+ 5
- 10
monasca_transform/component/setter/set_aggregated_period.py View File

@@ -39,16 +39,12 @@ class SetAggregatedPeriod(SetterComponent):
39 39
         except AttributeError:
40 40
             processing_meta = {}
41 41
 
42
+        # get any extra data
43
+        extra_data_map = getattr(row, "extra_data_map", {})
44
+
42 45
         instance_usage_dict = {"tenant_id": row.tenant_id,
43 46
                                "user_id": row.user_id,
44 47
                                "resource_uuid": row.resource_uuid,
45
-                               "namespace": row.namespace,
46
-                               "pod_name": row.pod_name,
47
-                               "app": row.app,
48
-                               "container_name": row.container_name,
49
-                               "interface": row.interface,
50
-                               "deployment": row.deployment,
51
-                               "daemon_set": row.daemon_set,
52 48
                                "geolocation": row.geolocation,
53 49
                                "region": row.region,
54 50
                                "zone": row.zone,
@@ -66,14 +62,13 @@ class SetAggregatedPeriod(SetterComponent):
66 62
                                "lastrecord_timestamp_string":
67 63
                                    row.lastrecord_timestamp_string,
68 64
                                "record_count": row.record_count,
69
-                               "service_group": row.service_group,
70
-                               "service_id": row.service_id,
71 65
                                "usage_date": row.usage_date,
72 66
                                "usage_hour": row.usage_hour,
73 67
                                "usage_minute": row.usage_minute,
74 68
                                "aggregation_period":
75 69
                                    agg_params["aggregation_period"],
76
-                               "processing_meta": processing_meta}
70
+                               "processing_meta": processing_meta,
71
+                               "extra_data_map": extra_data_map}
77 72
 
78 73
         instance_usage_data_json = json.dumps(instance_usage_dict)
79 74
 

+ 0
- 8
monasca_transform/component/usage/calculate_rate.py View File

@@ -128,14 +128,6 @@ class CalculateRate(UsageComponent):
128 128
                                    latest_dict["lastrecord_timestamp_string"],
129 129
                                "record_count": oldest_dict["record_count"] +
130 130
                                    latest_dict["record_count"],
131
-                               "service_group":
132
-                                   latest_dict.get("service_group",
133
-                                                   Component.
134
-                                                   DEFAULT_UNAVAILABLE_VALUE),
135
-                               "service_id":
136
-                                   latest_dict.get("service_id",
137
-                                                   Component.
138
-                                                   DEFAULT_UNAVAILABLE_VALUE),
139 131
                                "usage_date": latest_dict["usage_date"],
140 132
                                "usage_hour": latest_dict["usage_hour"],
141 133
                                "usage_minute": latest_dict["usage_minute"],

+ 56
- 85
monasca_transform/component/usage/fetch_quantity.py View File

@@ -27,6 +27,7 @@ from monasca_transform.transform.grouping.group_sort_by_timestamp \
27 27
 from monasca_transform.transform.grouping.group_sort_by_timestamp_partition \
28 28
     import GroupSortbyTimestampPartition
29 29
 from monasca_transform.transform.transform_utils import InstanceUsageUtils
30
+from monasca_transform.transform.transform_utils import RecordStoreUtils
30 31
 
31 32
 import json
32 33
 
@@ -45,18 +46,20 @@ class FetchQuantityException(Exception):
45 46
         return repr(self.value)
46 47
 
47 48
 
48
-GroupedDataWithOperation = namedtuple("GroupedDataWithOperation",
49
-                                      ["grouped_data",
50
-                                       "usage_fetch_operation"])
49
+GroupedDataNamedTuple = namedtuple("GroupedDataWithOperation",
50
+                                   ["grouped_data",
51
+                                    "usage_fetch_operation",
52
+                                    "group_by_columns_list"])
51 53
 
52 54
 
53
-class GroupedDataWithOperation(GroupedDataWithOperation):
55
+class GroupedDataNamedTuple(GroupedDataNamedTuple):
54 56
     """A tuple which is a wrapper containing record store data and the usage operation
55 57
 
56 58
     namdetuple contains:
57 59
 
58 60
     grouped_data - grouped record store data
59
-    usage_fetch_operation - operation
61
+    usage_fetch_operation - operation to be performed on
62
+    grouped data group_by_columns_list - list of group by columns
60 63
     """
61 64
 
62 65
 
@@ -75,21 +78,23 @@ class FetchQuantity(UsageComponent):
75 78
             return False
76 79
 
77 80
     @staticmethod
78
-    def _get_latest_oldest_quantity(grouping_results_with_operation):
79
-        """Method to return an instance usage data
81
+    def _get_latest_oldest_quantity(grouped_data_named_tuple):
82
+        """Get quantity for each group.
80 83
 
81
-        Get quantity for each group by performing the requested
82
-        usage operation and return an instance usage data.
84
+        By performing the requested usage operation and return a instance usage data.
83 85
         """
84
-
85 86
         # row
86
-        grouping_results = grouping_results_with_operation.\
87
+        grouping_results = grouped_data_named_tuple.\
87 88
             grouped_data
88 89
 
89 90
         # usage fetch operation
90
-        usage_fetch_operation = grouping_results_with_operation.\
91
+        usage_fetch_operation = grouped_data_named_tuple.\
91 92
             usage_fetch_operation
92 93
 
94
+        # group_by_columns_list
95
+        group_by_columns_list = grouped_data_named_tuple.\
96
+            group_by_columns_list
97
+
93 98
         group_by_dict = grouping_results.grouping_key_dict
94 99
 
95 100
         #
@@ -99,20 +104,6 @@ class FetchQuantity(UsageComponent):
99 104
                                           Component.DEFAULT_UNAVAILABLE_VALUE)
100 105
         user_id = group_by_dict.get("user_id",
101 106
                                     Component.DEFAULT_UNAVAILABLE_VALUE)
102
-        namespace = group_by_dict.get("namespace",
103
-                                      Component.DEFAULT_UNAVAILABLE_VALUE)
104
-        pod_name = group_by_dict.get("pod_name",
105
-                                     Component.DEFAULT_UNAVAILABLE_VALUE)
106
-        app = group_by_dict.get("app",
107
-                                Component.DEFAULT_UNAVAILABLE_VALUE)
108
-        container_name = group_by_dict.get("container_name",
109
-                                           Component.DEFAULT_UNAVAILABLE_VALUE)
110
-        interface = group_by_dict.get("interface",
111
-                                      Component.DEFAULT_UNAVAILABLE_VALUE)
112
-        deployment = group_by_dict.get("deployment",
113
-                                       Component.DEFAULT_UNAVAILABLE_VALUE)
114
-        daemon_set = group_by_dict.get("daemon_set",
115
-                                       Component.DEFAULT_UNAVAILABLE_VALUE)
116 107
 
117 108
         geolocation = group_by_dict.get("geolocation",
118 109
                                         Component.DEFAULT_UNAVAILABLE_VALUE)
@@ -148,10 +139,6 @@ class FetchQuantity(UsageComponent):
148 139
         lastrecord_timestamp_string = agg_stats["lastrecord_timestamp_string"]
149 140
         record_count = agg_stats["record_count"]
150 141
 
151
-        # service id
152
-        service_group = Component.DEFAULT_UNAVAILABLE_VALUE
153
-        service_id = Component.DEFAULT_UNAVAILABLE_VALUE
154
-
155 142
         # aggregation period
156 143
         aggregation_period = Component.DEFAULT_UNAVAILABLE_VALUE
157 144
 
@@ -159,15 +146,16 @@ class FetchQuantity(UsageComponent):
159 146
         event_type = group_by_dict.get("event_type",
160 147
                                        Component.DEFAULT_UNAVAILABLE_VALUE)
161 148
 
149
+        # add group by fields data to extra data map
150
+        # get existing extra_data_map if any
151
+        extra_data_map = group_by_dict.get("extra_data_map", {})
152
+        for column_name in group_by_columns_list:
153
+            column_value = group_by_dict.get(column_name, Component.
154
+                                             DEFAULT_UNAVAILABLE_VALUE)
155
+            extra_data_map[column_name] = column_value
156
+
162 157
         instance_usage_dict = {"tenant_id": tenant_id, "user_id": user_id,
163 158
                                "resource_uuid": resource_uuid,
164
-                               "namespace": namespace,
165
-                               "pod_name": pod_name,
166
-                               "app": app,
167
-                               "container_name": container_name,
168
-                               "interface": interface,
169
-                               "deployment": deployment,
170
-                               "daemon_set": daemon_set,
171 159
                                "geolocation": geolocation, "region": region,
172 160
                                "zone": zone, "host": host,
173 161
                                "aggregated_metric_name":
@@ -182,28 +170,32 @@ class FetchQuantity(UsageComponent):
182 170
                                "lastrecord_timestamp_string":
183 171
                                    lastrecord_timestamp_string,
184 172
                                "record_count": record_count,
185
-                               "service_group": service_group,
186
-                               "service_id": service_id,
187 173
                                "usage_date": usage_date,
188 174
                                "usage_hour": usage_hour,
189 175
                                "usage_minute": usage_minute,
190 176
                                "aggregation_period": aggregation_period,
191
-                               "processing_meta": {"event_type": event_type}
177
+                               "processing_meta": {"event_type": event_type},
178
+                               "extra_data_map": extra_data_map
192 179
                                }
193 180
         instance_usage_data_json = json.dumps(instance_usage_dict)
194 181
 
195 182
         return instance_usage_data_json
196 183
 
197 184
     @staticmethod
198
-    def _get_quantity(grouped_record_with_operation):
185
+    def _get_quantity(grouped_data_named_tuple):
199 186
 
200 187
         # row
201
-        row = grouped_record_with_operation.grouped_data
188
+        row = grouped_data_named_tuple.grouped_data
202 189
 
203 190
         # usage fetch operation
204
-        usage_fetch_operation = grouped_record_with_operation.\
191
+        usage_fetch_operation = grouped_data_named_tuple.\
205 192
             usage_fetch_operation
206 193
 
194
+        # group by columns list
195
+
196
+        group_by_columns_list = grouped_data_named_tuple.\
197
+            group_by_columns_list
198
+
207 199
         # first record timestamp # FIXME: beginning of epoch?
208 200
         earliest_record_timestamp_unix = getattr(
209 201
             row, "min(event_timestamp_unix_for_min)",
@@ -230,6 +222,14 @@ class FetchQuantity(UsageComponent):
230 222
         select_quant_str = "".join((usage_fetch_operation, "(event_quantity)"))
231 223
         quantity = getattr(row, select_quant_str, 0.0)
232 224
 
225
+        # create a column name, value pairs from grouped data
226
+        extra_data_map = InstanceUsageUtils.grouped_data_to_map(row,
227
+                                                                group_by_columns_list)
228
+
229
+        # convert column names, so that values can be accessed by components
230
+        # later in the pipeline
231
+        extra_data_map = InstanceUsageUtils.prepare_extra_data_map(extra_data_map)
232
+
233 233
         #  create a new instance usage dict
234 234
         instance_usage_dict = {"tenant_id": getattr(row, "tenant_id",
235 235
                                                     Component.
@@ -262,34 +262,6 @@ class FetchQuantity(UsageComponent):
262 262
                                    getattr(row, "tenant_id",
263 263
                                            Component.
264 264
                                            DEFAULT_UNAVAILABLE_VALUE),
265
-                               "namespace":
266
-                                   getattr(row, "namespace",
267
-                                           Component.
268
-                                           DEFAULT_UNAVAILABLE_VALUE),
269
-                               "pod_name":
270
-                                   getattr(row, "pod_name",
271
-                                           Component.
272
-                                           DEFAULT_UNAVAILABLE_VALUE),
273
-                               "app":
274
-                                   getattr(row, "app",
275
-                                           Component.
276
-                                           DEFAULT_UNAVAILABLE_VALUE),
277
-                               "container_name":
278
-                                   getattr(row, "container_name",
279
-                                           Component.
280
-                                           DEFAULT_UNAVAILABLE_VALUE),
281
-                               "interface":
282
-                                   getattr(row, "interface",
283
-                                           Component.
284
-                                           DEFAULT_UNAVAILABLE_VALUE),
285
-                               "deployment":
286
-                                   getattr(row, "deployment",
287
-                                           Component.
288
-                                           DEFAULT_UNAVAILABLE_VALUE),
289
-                               "daemon_set":
290
-                                   getattr(row, "daemon_set",
291
-                                           Component.
292
-                                           DEFAULT_UNAVAILABLE_VALUE),
293 265
                                "aggregated_metric_name":
294 266
                                    getattr(row, "aggregated_metric_name",
295 267
                                            Component.
@@ -305,14 +277,6 @@ class FetchQuantity(UsageComponent):
305 277
                                "lastrecord_timestamp_string":
306 278
                                    latest_record_timestamp_string,
307 279
                                "record_count": record_count,
308
-                               "service_group":
309
-                                   getattr(row, "service_group",
310
-                                           Component.
311
-                                           DEFAULT_UNAVAILABLE_VALUE),
312
-                               "service_id":
313
-                                   getattr(row, "service_id",
314
-                                           Component.
315
-                                           DEFAULT_UNAVAILABLE_VALUE),
316 280
                                "usage_date":
317 281
                                    getattr(row, "event_date",
318 282
                                            Component.
@@ -331,7 +295,8 @@ class FetchQuantity(UsageComponent):
331 295
                                            DEFAULT_UNAVAILABLE_VALUE),
332 296
                                "processing_meta": {"event_type": getattr(
333 297
                                    row, "event_type",
334
-                                   Component.DEFAULT_UNAVAILABLE_VALUE)}
298
+                                   Component.DEFAULT_UNAVAILABLE_VALUE)},
299
+                               "extra_data_map": extra_data_map
335 300
                                }
336 301
 
337 302
         instance_usage_data_json = json.dumps(instance_usage_dict)
@@ -431,6 +396,10 @@ class FetchQuantity(UsageComponent):
431 396
         group_by_columns_list = group_by_period_list + \
432 397
             aggregation_group_by_list
433 398
 
399
+        # prepare group by columns list
400
+        group_by_columns_list = RecordStoreUtils.prepare_recordstore_group_by_list(
401
+            group_by_columns_list)
402
+
434 403
         instance_usage_json_rdd = None
435 404
         if (usage_fetch_operation == "latest" or
436 405
                 usage_fetch_operation == "oldest"):
@@ -466,14 +435,14 @@ class FetchQuantity(UsageComponent):
466 435
 
467 436
             grouped_data_rdd_with_operation = grouped_rows_rdd.map(
468 437
                 lambda x:
469
-                GroupedDataWithOperation(x,
470
-                                         str(usage_fetch_operation)))
438
+                GroupedDataNamedTuple(x,
439
+                                      str(usage_fetch_operation),
440
+                                      group_by_columns_list))
471 441
 
472 442
             instance_usage_json_rdd = \
473 443
                 grouped_data_rdd_with_operation.map(
474 444
                     FetchQuantity._get_latest_oldest_quantity)
475 445
         else:
476
-
477 446
             record_store_df_int = \
478 447
                 record_store_df.select(
479 448
                     record_store_df.event_timestamp_unix.alias(
@@ -488,14 +457,16 @@ class FetchQuantity(UsageComponent):
488 457
                 "event_timestamp_unix_for_min": "min",
489 458
                 "event_timestamp_unix_for_max": "max",
490 459
                 "event_timestamp_unix": "count"}
460
+
491 461
             # do a group by
492 462
             grouped_data = record_store_df_int.groupBy(*group_by_columns_list)
493 463
             grouped_record_store_df = grouped_data.agg(agg_operations_map)
494 464
 
495 465
             grouped_data_rdd_with_operation = grouped_record_store_df.rdd.map(
496 466
                 lambda x:
497
-                GroupedDataWithOperation(x,
498
-                                         str(usage_fetch_operation)))
467
+                GroupedDataNamedTuple(x,
468
+                                      str(usage_fetch_operation),
469
+                                      group_by_columns_list))
499 470
 
500 471
             instance_usage_json_rdd = grouped_data_rdd_with_operation.map(
501 472
                 FetchQuantity._get_quantity)

+ 9
- 7
monasca_transform/component/usage/fetch_quantity_util.py View File

@@ -107,13 +107,16 @@ class FetchQuantityUtil(UsageComponent):
107 107
         record_count = getattr(row, "record_count",
108 108
                                Component.DEFAULT_UNAVAILABLE_VALUE)
109 109
 
110
-        # service id
111
-        service_group = Component.DEFAULT_UNAVAILABLE_VALUE
112
-        service_id = Component.DEFAULT_UNAVAILABLE_VALUE
113
-
114 110
         # aggregation period
115 111
         aggregation_period = Component.DEFAULT_UNAVAILABLE_VALUE
116 112
 
113
+        # get extra_data_map, if any
114
+        extra_data_map = getattr(row, "extra_data_map", {})
115
+        # filter out event_type
116
+        extra_data_map_filtered = \
117
+            {key: extra_data_map[key] for key in list(extra_data_map)
118
+             if key != 'event_type'}
119
+
117 120
         instance_usage_dict = {"tenant_id": tenant_id, "user_id": user_id,
118 121
                                "resource_uuid": resource_uuid,
119 122
                                "geolocation": geolocation, "region": region,
@@ -130,12 +133,11 @@ class FetchQuantityUtil(UsageComponent):
130 133
                                "lastrecord_timestamp_string":
131 134
                                    lastrecord_timestamp_string,
132 135
                                "record_count": record_count,
133
-                               "service_group": service_group,
134
-                               "service_id": service_id,
135 136
                                "usage_date": usage_date,
136 137
                                "usage_hour": usage_hour,
137 138
                                "usage_minute": usage_minute,
138
-                               "aggregation_period": aggregation_period}
139
+                               "aggregation_period": aggregation_period,
140
+                               "extra_data_map": extra_data_map_filtered}
139 141
 
140 142
         instance_usage_data_json = json.dumps(instance_usage_dict)
141 143
 

+ 1
- 2
monasca_transform/config/config_initializer.py View File

@@ -27,8 +27,7 @@ class ConfigInitializer(object):
27 27
         ConfigInitializer.load_stage_processors_options()
28 28
         ConfigInitializer.load_pre_hourly_processor_options()
29 29
         if not default_config_files:
30
-            default_config_files = ['/etc/monasca-transform.conf',
31
-                                    'etc/monasca-transform.conf']
30
+            default_config_files = ['/etc/monasca-transform.conf']
32 31
         cfg.CONF(args=[],
33 32
                  project='monasca_transform',
34 33
                  default_config_files=default_config_files)

+ 2
- 2
monasca_transform/data_driven_specs/pre_transform_specs/pre_transform_specs.json View File

@@ -12,6 +12,6 @@
12 12
 {"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"}
13 13
 {"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"}
14 14
 {"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"}
15
-{"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"}
16
-{"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"}
15
+{"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"}
16
+{"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"}
17 17
 {"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"}

+ 5
- 5
monasca_transform/data_driven_specs/transform_specs/transform_specs.json View File

@@ -18,9 +18,9 @@
18 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":"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"}
19 19
 {"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"}
20 20
 {"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"}
21
-{"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"}
22
-{"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"}
23
-{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"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"}
24
-{"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"}
25
-{"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"}
21
+{"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"}
22
+{"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"}
23
+{"aggregation_params_map":{"aggregation_pipeline":{"source":"streaming","usage":"fetch_quantity","setters":["rollup_quantity","set_aggregated_metric_name","set_aggregated_period"],"insert":["prepare_data","insert_data_pre_hourly"]},"aggregated_metric_name":"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"}
24
+{"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"}
25
+{"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"}
26 26
 {"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"}

+ 30
- 76
monasca_transform/driver/mon_metrics_kafka.py View File

@@ -21,7 +21,6 @@ from pyspark.streaming import StreamingContext
21 21
 
22 22
 from pyspark.sql.functions import explode
23 23
 from pyspark.sql.functions import from_unixtime
24
-from pyspark.sql.functions import lit
25 24
 from pyspark.sql.functions import when
26 25
 from pyspark.sql import SQLContext
27 26
 
@@ -52,6 +51,7 @@ from monasca_transform.transform.storage_utils import \
52 51
     InvalidCacheStorageLevelException
53 52
 from monasca_transform.transform.storage_utils import StorageUtils
54 53
 from monasca_transform.transform.transform_utils import MonMetricUtils
54
+from monasca_transform.transform.transform_utils import PreTransformSpecsUtils
55 55
 from monasca_transform.transform import TransformContextUtils
56 56
 
57 57
 ConfigInitializer.basic_config()
@@ -177,8 +177,11 @@ class MonMetricsKafkaProcessor(object):
177 177
 
178 178
         required_fields = row.required_raw_fields_list
179 179
 
180
-        invalid_list = []
180
+        # prepare list of required fields, to a rdd syntax to retrieve value
181
+        required_fields = PreTransformSpecsUtils.prepare_required_raw_fields_list(
182
+            required_fields)
181 183
 
184
+        invalid_list = []
182 185
         for required_field in required_fields:
183 186
             required_field_value = None
184 187
 
@@ -188,35 +191,18 @@ class MonMetricsKafkaProcessor(object):
188 191
             except Exception:
189 192
                 pass
190 193
 
191
-            if (required_field_value is None or required_field_value == "" and
192
-                    row.metric is not None and
193
-                    row.metric.dimensions is not None):
194
-                # Look for the field in the dimensions layer of the row
195
-                try:
196
-                    required_field_value = eval(
197
-                        ".".join(("row.metric.dimensions", required_field)))
198
-                except Exception:
199
-                    pass
200
-
201
-            if (required_field_value is None or required_field_value == "" and
202
-                    row.meta is not None):
203
-                # Look for the field in the meta layer of the row
204
-                try:
205
-                    required_field_value = eval(
206
-                        ".".join(("row.meta", required_field)))
207
-                except Exception:
208
-                    pass
209
-
210 194
             if required_field_value is None \
211 195
                     or required_field_value == "":
212
-                invalid_list.append("invalid")
196
+                invalid_list.append((required_field,
197
+                                     required_field_value))
213 198
 
214 199
         if len(invalid_list) <= 0:
215 200
             return row
216 201
         else:
217
-            print("_validate_raw_mon_metrics : found invalid : ** %s: %s" % (
218
-                (".".join(("row", required_field))),
219
-                required_field_value))
202
+            for field_name, field_value in invalid_list:
203
+                MonMetricsKafkaProcessor.log_debug(
204
+                    "_validate_raw_mon_metrics : found invalid field : ** %s: %s" % (
205
+                        field_name, field_value))
220 206
 
221 207
     @staticmethod
222 208
     def process_metric(transform_context, record_store_df):
@@ -225,7 +211,6 @@ class MonMetricsKafkaProcessor(object):
225 211
         All the parameters to drive processing should be available
226 212
         in transform_spec_df dataframe.
227 213
         """
228
-
229 214
         # call processing chain
230 215
         return GenericTransformBuilder.do_transform(
231 216
             transform_context, record_store_df)
@@ -378,11 +363,6 @@ class MonMetricsKafkaProcessor(object):
378 363
                 gen_mon_metrics_df.event_type.alias("event_quantity_name"),
379 364
                 (gen_mon_metrics_df.metric.value / 1.0).alias(
380 365
                     "event_quantity"),
381
-                when(gen_mon_metrics_df.metric.dimensions.state != '',
382
-                     gen_mon_metrics_df.metric.dimensions.state).otherwise(
383
-                    'NA').alias("event_status"),
384
-                lit('1.0').alias('event_version'),
385
-                lit('metrics').alias("record_type"),
386 366
 
387 367
                 # resource_uuid
388 368
                 when(gen_mon_metrics_df.metric.dimensions.instanceId != '',
@@ -391,6 +371,7 @@ class MonMetricsKafkaProcessor(object):
391 371
                     gen_mon_metrics_df.metric.dimensions.resource_id).
392 372
                 otherwise('NA').alias("resource_uuid"),
393 373
 
374
+                # tenant_id
394 375
                 when(gen_mon_metrics_df.metric.dimensions.tenantId != '',
395 376
                      gen_mon_metrics_df.metric.dimensions.tenantId).when(
396 377
                     gen_mon_metrics_df.metric.dimensions.tenant_id != '',
@@ -399,46 +380,12 @@ class MonMetricsKafkaProcessor(object):
399 380
                     gen_mon_metrics_df.metric.dimensions.project_id).otherwise(
400 381
                     'NA').alias("tenant_id"),
401 382
 
402
-                when(gen_mon_metrics_df.metric.dimensions.mount != '',
403
-                     gen_mon_metrics_df.metric.dimensions.mount).otherwise(
404
-                    'NA').alias("mount"),
405
-
406
-                when(gen_mon_metrics_df.metric.dimensions.device != '',
407
-                     gen_mon_metrics_df.metric.dimensions.device).otherwise(
408
-                    'NA').alias("device"),
409
-
410
-                when(gen_mon_metrics_df.metric.dimensions.namespace != '',
411
-                     gen_mon_metrics_df.metric.dimensions.namespace).otherwise(
412
-                    'NA').alias("namespace"),
413
-
414
-                when(gen_mon_metrics_df.metric.dimensions.pod_name != '',
415
-                     gen_mon_metrics_df.metric.dimensions.pod_name).otherwise(
416
-                    'NA').alias("pod_name"),
417
-
418
-                when(gen_mon_metrics_df.metric.dimensions.container_name != '',
419
-                     gen_mon_metrics_df.metric.dimensions
420
-                     .container_name).otherwise('NA').alias("container_name"),
421
-
422
-                when(gen_mon_metrics_df.metric.dimensions.app != '',
423
-                     gen_mon_metrics_df.metric.dimensions.app).otherwise(
424
-                    'NA').alias("app"),
425
-
426
-                when(gen_mon_metrics_df.metric.dimensions.interface != '',
427
-                     gen_mon_metrics_df.metric.dimensions.interface).otherwise(
428
-                    'NA').alias("interface"),
429
-
430
-                when(gen_mon_metrics_df.metric.dimensions.deployment != '',
431
-                     gen_mon_metrics_df.metric.dimensions
432
-                     .deployment).otherwise('NA').alias("deployment"),
433
-
434
-                when(gen_mon_metrics_df.metric.dimensions.daemon_set != '',
435
-                     gen_mon_metrics_df.metric.dimensions
436
-                     .daemon_set).otherwise('NA').alias("daemon_set"),
437
-
383
+                # user_id
438 384
                 when(gen_mon_metrics_df.meta.userId != '',
439 385
                      gen_mon_metrics_df.meta.userId).otherwise('NA').alias(
440 386
                     "user_id"),
441 387
 
388
+                # region
442 389
                 when(gen_mon_metrics_df.meta.region != '',
443 390
                      gen_mon_metrics_df.meta.region).when(
444 391
                     gen_mon_metrics_df.event_processing_params
@@ -447,6 +394,7 @@ class MonMetricsKafkaProcessor(object):
447 394
                     .set_default_region_to).otherwise(
448 395
                     'NA').alias("region"),
449 396
 
397
+                # zone
450 398
                 when(gen_mon_metrics_df.meta.zone != '',
451 399
                      gen_mon_metrics_df.meta.zone).when(
452 400
                     gen_mon_metrics_df.event_processing_params
@@ -455,30 +403,36 @@ class MonMetricsKafkaProcessor(object):
455 403
                     .set_default_zone_to).otherwise(
456 404
                     'NA').alias("zone"),
457 405
 
406
+                # host
458 407
                 when(gen_mon_metrics_df.metric.dimensions.hostname != '',
459 408
                      gen_mon_metrics_df.metric.dimensions.hostname).when(
460 409
                     gen_mon_metrics_df.metric.value_meta.host != '',
461 410
                     gen_mon_metrics_df.metric.value_meta.host).otherwise(
462 411
                     'NA').alias("host"),
463 412
 
464
-                when(gen_mon_metrics_df.service_id != '',
465
-                     gen_mon_metrics_df.service_id).otherwise(
466
-                    'NA').alias("service_group"),
467
-
468
-                when(gen_mon_metrics_df.service_id != '',
469
-                     gen_mon_metrics_df.service_id).otherwise(
470
-                    'NA').alias("service_id"),
471
-
413
+                # event_date
472 414
                 from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000,
473 415
                               'yyyy-MM-dd').alias("event_date"),
416
+                # event_hour
474 417
                 from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000,
475 418
                               'HH').alias("event_hour"),
419
+                # event_minute
476 420
                 from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000,
477 421
                               'mm').alias("event_minute"),
422
+                # event_second
478 423
                 from_unixtime(gen_mon_metrics_df.metric.timestamp / 1000,
479 424
                               'ss').alias("event_second"),
425
+
480 426
                 gen_mon_metrics_df.this_metric_id.alias("metric_group"),
481
-                gen_mon_metrics_df.this_metric_id.alias("metric_id"))
427
+
428
+                gen_mon_metrics_df.this_metric_id.alias("metric_id"),
429
+
430
+                # metric dimensions
431
+                gen_mon_metrics_df.meta.alias("meta"),
432
+                # metric dimensions
433
+                gen_mon_metrics_df.metric.dimensions.alias("dimensions"),
434
+                # metric value_meta
435
+                gen_mon_metrics_df.metric.value_meta.alias("value_meta"))
482 436
 
483 437
             #
484 438
             # get transform context

+ 0
- 16
monasca_transform/transform/grouping/group_sort_by_timestamp.py View File

@@ -24,22 +24,6 @@ class GroupSortbyTimestamp(Grouping):
24 24
         print(str)
25 25
         # LOG.debug(logStr)
26 26
 
27
-    @staticmethod
28
-    def _get_group_by_key(row_decorated):
29
-        """Build a group by key using the group by column list.
30
-
31
-        row_decorated: [[Rows(a=1, b=1, c=2, d=3)],[group_by_a,group_by_b]]
32
-        """
33
-        # LOG.debug(whoami(row_decorated))
34
-        # LOG.debug(row_decorated)
35
-
36
-        group_by_columns_list = row_decorated[1]
37
-        group_by_key = ""
38
-        for gcol in group_by_columns_list:
39
-            group_by_key = "^".join((group_by_key,
40
-                                     eval(".".join(("row", gcol)))))
41
-            return group_by_key
42
-
43 27
     @staticmethod
44 28
     def _prepare_for_group_by(record_store_with_group_by_rdd):
45 29
         """creates a new rdd where:

+ 0
- 183
monasca_transform/transform/grouping/group_sortby_timestamp.py View File

@@ -1,183 +0,0 @@
1
-# Copyright 2016 Hewlett Packard Enterprise Development Company LP
2
-#
3
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
4
-# not use this file except in compliance with the License. You may obtain
5
-# a copy of the License at
6
-#
7
-# http://www.apache.org/licenses/LICENSE-2.0
8
-#
9
-# Unless required by applicable law or agreed to in writing, software
10
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
11
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
12
-# License for the specific language governing permissions and limitations
13
-# under the License.
14
-
15
-from monasca_transform.transform.grouping import Grouping
16
-from monasca_transform.transform.grouping import GroupingResults
17
-from monasca_transform.transform.grouping import RecordStoreWithGroupBy
18
-
19
-
20
-class GroupSortbyTimestamp(Grouping):
21
-
22
-    @staticmethod
23
-    def log_debug(logStr):
24
-        print(str)
25
-        # LOG.debug(logStr)
26
-
27
-    @staticmethod
28
-    def _get_groupby_key(row_decorated):
29
-        """Build a group by key using the group by column list.
30
-
31
-        row_decorated: [[Rows(a=1, b=1, c=2, d=3)],[groupby_a,groupby_b]]
32
-        """
33
-        # LOG.debug(whoami(row_decorated))
34
-        # LOG.debug(row_decorated)
35
-
36
-        groupby_columns_list = row_decorated[1]
37
-        groupby_key = ""
38
-        for gcol in groupby_columns_list:
39
-            groupby_key = "^".join((groupby_key,
40
-                                    eval(".".join(("row", gcol)))))
41
-            return groupby_key
42
-
43
-    @staticmethod
44
-    def _prepare_for_groupby(record_store_with_groupby_rdd):
45
-        """creates a new rdd where:
46
-
47
-        the first element of each row
48
-        contains array of grouping key and event timestamp fields.
49
-        Grouping key and event timestamp fields are used by
50
-        partitioning and sorting function to partition the data
51
-        by grouping key and then sort the elements in a group by the
52
-        timestamp
53
-        """
54
-
55
-        # get the record store data and group by columns
56
-        record_store_data = record_store_with_groupby_rdd.record_store_data
57
-
58
-        groupby_columns_list = \
59
-            record_store_with_groupby_rdd.groupby_columns_list
60
-
61
-        # construct a group by key
62
-        # key1=value1^key2=value2^...
63
-        groupby_key_value = ""
64
-        for gcol in groupby_columns_list:
65
-            groupby_key_value = \
66
-                "^".join((groupby_key_value,
67
-                          "=".join((gcol,
68
-                                    eval(".".join(("record_store_data",
69
-                                                   gcol)))))))
70
-
71
-        # return a key-value rdd
72
-        return [groupby_key_value, record_store_data]
73
-
74
-    @staticmethod
75
-    def _sortby_timestamp(result_iterable):
76
-        # LOG.debug(whoami(result_iterable.data[0]))
77
-
78
-        # sort list might cause OOM, if the group has lots of items
79
-        # use group_sortby_timestamp_partitions module instead if you run
80
-        # into OOM
81
-        sorted_list = sorted(result_iterable.data,
82
-                             key=lambda row: row.event_timestamp_string)
83
-        return sorted_list
84
-
85
-    @staticmethod
86
-    def _group_sortby_timestamp(record_store_df, groupby_columns_list):
87
-        # convert the dataframe rdd to normal rdd and add the group by column
88
-        # list
89
-        record_store_with_groupby_rdd = record_store_df.rdd.\
90
-            map(lambda x: RecordStoreWithGroupBy(x, groupby_columns_list))
91
-
92
-        # convert rdd into key-value rdd
93
-        record_store_with_groupby_rdd_key_val = record_store_with_groupby_rdd.\
94
-            map(GroupSortbyTimestamp._prepare_for_groupby)
95
-
96
-        first_step = record_store_with_groupby_rdd_key_val.groupByKey()
97
-        record_store_rdd_grouped_sorted = first_step.mapValues(
98
-            GroupSortbyTimestamp._sortby_timestamp)
99
-
100
-        return record_store_rdd_grouped_sorted
101
-
102
-    @staticmethod
103
-    def _get_group_first_last_quantity_udf(grouplistiter):
104
-        """Return stats that include:
105
-
106
-        first row key, first_event_timestamp,
107
-        first event quantity, last_event_timestamp and last event quantity
108
-        """
109
-        first_row = None
110
-        last_row = None
111
-
112
-        # extract key and value list
113
-        group_key = grouplistiter[0]
114
-        grouped_values = grouplistiter[1]
115
-
116
-        count = 0.0
117
-        for row in grouped_values:
118
-
119
-            # set the first row
120
-            if first_row is None:
121
-                first_row = row
122
-
123
-            # set the last row
124
-            last_row = row
125
-            count = count + 1
126
-
127
-        first_event_timestamp_unix = None
128
-        first_event_timestamp_string = None
129
-        first_event_quantity = None
130
-
131
-        if first_row is not None:
132
-            first_event_timestamp_unix = first_row.event_timestamp_unix
133
-            first_event_timestamp_string = first_row.event_timestamp_string
134
-            first_event_quantity = first_row.event_quantity
135
-
136
-        last_event_timestamp_unix = None
137
-        last_event_timestamp_string = None
138
-        last_event_quantity = None
139
-
140
-        if last_row is not None:
141
-            last_event_timestamp_unix = last_row.event_timestamp_unix
142
-            last_event_timestamp_string = last_row.event_timestamp_string
143
-            last_event_quantity = last_row.event_quantity
144
-
145
-        results_dict = {"firstrecord_timestamp_unix":
146
-                        first_event_timestamp_unix,
147
-                        "firstrecord_timestamp_string":
148
-                        first_event_timestamp_string,
149
-                        "firstrecord_quantity": first_event_quantity,
150
-                        "lastrecord_timestamp_unix":
151
-                        last_event_timestamp_unix,
152
-                        "lastrecord_timestamp_string":
153
-                        last_event_timestamp_string,
154
-                        "lastrecord_quantity": last_event_quantity,
155
-                        "record_count": count}
156
-
157
-        group_key_dict = Grouping._parse_grouping_key(group_key)
158
-
159
-        return GroupingResults(group_key, results_dict, group_key_dict)
160
-
161
-    @staticmethod
162
-    def fetch_group_latest_oldest_quantity(record_store_df,
163
-                                           transform_spec_df,
164
-                                           groupby_columns_list):
165
-        """To group record store data
166
-
167
-        sort by timestamp within group
168
-        and get first and last timestamp along with quantity within each group
169
-
170
-        This function uses key-value pair rdd's groupBy function to do groupby
171
-        """
172
-        # group and order elements in group
173
-        record_store_grouped_data_rdd = \
174
-            GroupSortbyTimestamp._group_sortby_timestamp(record_store_df,
175
-                                                         groupby_columns_list)
176
-
177
-        # find stats for a group
178
-        record_store_grouped_rows = \
179
-            record_store_grouped_data_rdd.\
180
-            map(GroupSortbyTimestamp.
181
-                _get_group_first_last_quantity_udf)
182
-
183
-        return record_store_grouped_rows

+ 0
- 231
monasca_transform/transform/grouping/group_sortby_timestamp_partition.py View File

@@ -1,231 +0,0 @@
1
-# Copyright 2016 Hewlett Packard Enterprise Development Company LP
2
-#
3
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
4
-# not use this file except in compliance with the License. You may obtain
5
-# a copy of the License at
6
-#
7
-# http://www.apache.org/licenses/LICENSE-2.0
8
-#
9
-# Unless required by applicable law or agreed to in writing, software
10
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
11
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
12
-# License for the specific language governing permissions and limitations
13
-# under the License.
14
-
15
-from monasca_transform.transform.grouping import Grouping
16
-from monasca_transform.transform.grouping import GroupingResults
17
-from monasca_transform.transform.grouping import RecordStoreWithGroupBy
18
-
19
-
20
-class GroupSortbyTimestampPartition(Grouping):
21
-
22
-    @staticmethod
23
-    def log_debug(logStr):
24
-        print(str)
25
-        # LOG.debug(logStr)
26
-
27
-    @staticmethod
28
-    def _get_group_first_last_quantity_udf(partitionlistiter):
29
-        """user defined function to to through a list of partitions.
30
-
31
-        Each partition contains elements for a group. All the elements are sorted by
32
-        timestamp.
33
-        The stats include first row key, first_event_timestamp,
34
-        fist event quantity, last_event_timestamp and last event quantity
35
-        """
36
-        first_row = None
37
-        last_row = None
38
-
39
-        count = 0.0
40
-        for row in partitionlistiter:
41
-
42
-            # set the first row
43
-            if first_row is None:
44
-                first_row = row
45
-
46
-            # set the last row
47
-            last_row = row
48
-            count = count + 1
49
-
50
-        first_event_timestamp_unix = None
51
-        first_event_timestamp_string = None
52
-        first_event_quantity = None
53
-        first_row_key = None
54
-        if first_row is not None:
55
-            first_event_timestamp_unix = first_row[1].event_timestamp_unix
56
-            first_event_timestamp_string = first_row[1].event_timestamp_string
57
-            first_event_quantity = first_row[1].event_quantity
58
-
59
-            # extract the grouping_key from composite grouping_key
60
-            # composite grouping key is a list, where first item is the
61
-            # grouping key and second item is the event_timestamp_string
62
-            first_row_key = first_row[0][0]
63
-
64
-        last_event_timestamp_unix = None
65
-        last_event_timestamp_string = None
66
-        last_event_quantity = None
67
-        if last_row is not None:
68
-            last_event_timestamp_unix = last_row[1].event_timestamp_unix
69
-            last_event_timestamp_string = last_row[1].event_timestamp_string
70
-            last_event_quantity = last_row[1].event_quantity
71
-
72
-        results_dict = {"firstrecord_timestamp_unix":
73
-                        first_event_timestamp_unix,
74
-                        "firstrecord_timestamp_string":
75
-                        first_event_timestamp_string,
76
-                        "firstrecord_quantity": first_event_quantity,
77
-                        "lastrecord_timestamp_unix":
78
-                        last_event_timestamp_unix,
79
-                        "lastrecord_timestamp_string":
80
-                        last_event_timestamp_string,
81
-                        "lastrecord_quantity": last_event_quantity,
82
-                        "record_count": count}
83
-
84
-        first_row_key_dict = Grouping._parse_grouping_key(first_row_key)
85
-
86
-        yield [GroupingResults(first_row_key, results_dict,
87
-                               first_row_key_dict)]
88
-
89
-    @staticmethod
90
-    def _prepare_for_groupby(record_store_with_groupby_rdd):
91
-        """creates a new rdd where:
92
-
93
-        the first element of each row
94
-        contains array of grouping key and event timestamp fields.
95
-        Grouping key and event timestamp fields are used by
96
-        partitioning and sorting function to partition the data
97
-        by grouping key and then sort the elements in a group by the
98
-        timestamp
99
-        """
100
-
101
-        # get the record store data and group by columns
102
-        record_store_data = record_store_with_groupby_rdd.record_store_data
103
-
104
-        groupby_columns_list = \
105
-            record_store_with_groupby_rdd.groupby_columns_list
106
-
107
-        # construct a group by key
108
-        # key1=value1^key2=value2^...
109
-        groupby_key_value = ""
110
-        for gcol in groupby_columns_list:
111
-            groupby_key_value = \
112
-                "^".join((groupby_key_value,
113
-                          "=".join((gcol, eval(".".join(("record_store_data",
114
-                                                         gcol)))))))
115
-
116
-        # return a key-value rdd
117
-        # key is a composite key which consists of grouping key and
118
-        # event_timestamp_string
119
-        return [[groupby_key_value,
120
-                 record_store_data.event_timestamp_string], record_store_data]
121
-
122
-    @staticmethod
123
-    def _get_partition_by_group(group_composite):
124
-        """get a hash of the grouping key,
125
-
126
-        which is then used by partitioning
127
-        function to get partition where the groups data should end up in.
128
-        It uses hash % num_partitions to get partition
129
-        """
130
-        # FIXME: find out of hash function in python gives same value on
131
-        # different machines
132
-        # Look at using portable_hash method in spark rdd
133
-        grouping_key = group_composite[0]
134
-        grouping_key_hash = hash(grouping_key)
135
-        # log_debug("groupby_sortby_timestamp_partition: got hash : %s" \
136
-        #    % str(returnhash))
137
-        return grouping_key_hash
138
-
139
-    @staticmethod
140
-    def _sortby_timestamp(group_composite):
141
-        """get timestamp which will be used to sort grouped data"""
142
-        event_timestamp_string = group_composite[1]
143
-        return event_timestamp_string
144
-
145
-    @staticmethod
146
-    def _group_sortby_timestamp_partition(record_store_df,
147
-                                          groupby_columns_list,
148
-                                          num_of_groups):
149
-        """It does a group by and then sorts all the items within the group by event timestamp."""
150
-        # convert the dataframe rdd to normal rdd and add the group by
151
-        # column list
152
-        record_store_with_groupby_rdd = record_store_df.rdd.\
153
-            map(lambda x: RecordStoreWithGroupBy(x, groupby_columns_list))
154
-
155
-        # prepare the data for repartitionAndSortWithinPartitions function
156
-        record_store_rdd_prepared = \
157
-            record_store_with_groupby_rdd.\
158
-            map(GroupSortbyTimestampPartition._prepare_for_groupby)
159
-
160
-        # repartition data based on a grouping key and sort the items within
161
-        # group by timestamp
162
-        # give high number of partitions
163
-        # numPartitions > number of groups expected, so that each group gets
164
-        # allocated a separate partition
165
-        record_store_rdd_partitioned_sorted = \
166
-            record_store_rdd_prepared.\
167
-            repartitionAndSortWithinPartitions(
168
-                numPartitions=num_of_groups,
169
-                partitionFunc=GroupSortbyTimestampPartition.
170
-                _get_partition_by_group,
171
-                keyfunc=GroupSortbyTimestampPartition.
172
-                _sortby_timestamp)
173
-
174
-        return record_store_rdd_partitioned_sorted
175
-
176
-    @staticmethod
177
-    def _remove_none_filter(row):
178
-        """remove any rows which have None as grouping key
179
-
180
-        [GroupingResults(grouping_key="key1", results={})] rows get created
181
-        when partition does not get any grouped data assigned to it
182
-        """
183
-        if len(row[0].results) > 0 and row[0].grouping_key is not None:
184
-            return row
185
-
186
-    @staticmethod
187
-    def fetch_group_first_last_quantity(record_store_df,
188
-                                        transform_spec_df,
189
-                                        groupby_columns_list,
190
-                                        num_of_groups):
191
-        """Function to group record store data
192
-
193
-        Sort by timestamp within group
194
-        and get first and last timestamp along with quantity within each group
195
-
196
-        To do group by it uses custom partitioning function which creates a new
197
-        partition
198
-        for each group and uses RDD's repartitionAndSortWithinPartitions
199
-        function to do the grouping and sorting within the group.
200
-
201
-        This is more scalable than just using RDD's groupby as using this
202
-        technique
203
-        group is not materialized into a list and stored in memory, but rather
204
-        it uses RDD's in built partitioning capability to do the sort
205
-
206
-        num_of_groups should be more than expected groups, otherwise the same
207
-        partition can get used for two groups which will cause incorrect
208
-        results.
209
-        """
210
-
211
-        # group and order elements in group using repartition
212
-        record_store_grouped_data_rdd = \
213
-            GroupSortbyTimestampPartition.\
214
-            _group_sortby_timestamp_partition(record_store_df,
215
-                                              groupby_columns_list,
216
-                                              num_of_groups)
217
-
218
-        # do some operations on all elements in the group
219
-        grouping_results_tuple_with_none = \
220
-            record_store_grouped_data_rdd.\
221
-            mapPartitions(GroupSortbyTimestampPartition.
222
-                          _get_group_first_last_quantity_udf)
223
-
224
-        # filter all rows which have no data (where grouping key is None) and
225
-        # convert resuts into grouping results tuple
226
-        grouping_results_tuple1 = grouping_results_tuple_with_none.\
227
-            filter(GroupSortbyTimestampPartition._remove_none_filter)
228
-
229
-        grouping_results_tuple = grouping_results_tuple1.map(lambda x: x[0])
230
-
231
-        return grouping_results_tuple

+ 196
- 35
monasca_transform/transform/transform_utils.py View File

@@ -20,6 +20,8 @@ from pyspark.sql.types import StringType
20 20
 from pyspark.sql.types import StructField
21 21
 from pyspark.sql.types import StructType
22 22
 
23
+from monasca_transform.component import Component
24
+
23 25
 
24 26
 class TransformUtils(object):
25 27
     """utility methods for different kinds of data."""
@@ -47,10 +49,8 @@ class InstanceUsageUtils(TransformUtils):
47 49
                    "geolocation", "region", "zone", "host", "project_id",
48 50
                    "aggregated_metric_name", "firstrecord_timestamp_string",
49 51
                    "lastrecord_timestamp_string",
50
-                   "service_group", "service_id",
51 52
                    "usage_date", "usage_hour", "usage_minute",
52
-                   "aggregation_period", "namespace", "pod_name", "app",
53
-                   "container_name", "interface", "deployment", "daemon_set"]
53
+                   "aggregation_period"]
54 54
 
55 55
         columns_struct_fields = [StructField(field_name, StringType(), True)
56 56
                                  for field_name in columns]
@@ -70,6 +70,12 @@ class InstanceUsageUtils(TransformUtils):
70 70
                                                          StringType(),
71 71
                                                          True),
72 72
                                                  True))
73
+
74
+        columns_struct_fields.append(StructField("extra_data_map",
75
+                                                 MapType(StringType(),
76
+                                                         StringType(),
77
+                                                         True),
78
+                                                 True))
73 79
         schema = StructType(columns_struct_fields)
74 80
 
75 81
         return schema
@@ -81,6 +87,103 @@ class InstanceUsageUtils(TransformUtils):
81 87
         instance_usage_schema_df = sql_context.read.json(jsonrdd, schema)
82 88
         return instance_usage_schema_df
83 89
 
90
+    @staticmethod
91
+    def prepare_instance_usage_group_by_list(group_by_list):
92
+        """Prepare group by list.
93
+
94
+        If the group by list contains any instances of "dimensions#", "meta#" or "value_meta#" then
95
+        prepend the column value by "extra_data_map." since those columns are available in
96
+        extra_data_map column.
97
+
98
+        """
99
+        return [InstanceUsageUtils.prepare_group_by_item(item) for item in group_by_list]
100
+
101
+    @staticmethod
102
+    def prepare_group_by_item(item):
103
+        """Prepare group by list item.
104
+
105
+        Convert replaces any special "dimensions#", "meta#" or "value_meta#" occurrences into
106
+        spark sql syntax to retrieve data from extra_data_map column.
107
+
108
+        """
109
+        if (item.startswith("dimensions#") or
110
+                item.startswith("meta#") or
111
+                item.startswith("value_meta#")):
112
+            return ".".join(("extra_data_map", item))
113
+        else:
114
+            return item
115
+
116
+    @staticmethod
117
+    def prepare_extra_data_map(extra_data_map):
118
+        """Prepare extra data map.
119
+
120
+        Replace any occurances of "dimensions." or "meta." or "value_meta."
121
+        to "dimensions#", "meta#" or "value_meta#" in extra_data_map.
122
+
123
+        """
124
+        prepared_extra_data_map = {}
125
+        for column_name in list(extra_data_map):
126
+            column_value = extra_data_map[column_name]
127
+            if column_name.startswith("dimensions."):
128
+                column_name = column_name.replace("dimensions.", "dimensions#")
129
+            elif column_name.startswith("meta."):
130
+                column_name = column_name.replace("meta.", "meta#")
131
+            elif column_name.startswith("value_meta."):
132
+                column_name = column_name.replace("value_meta.", "value_meta#")
133
+            elif column_name.startswith("extra_data_map."):
134
+                column_name = column_name.replace("extra_data_map.", "")
135
+            prepared_extra_data_map[column_name] = column_value
136
+        return prepared_extra_data_map
137
+
138
+    @staticmethod
139
+    def grouped_data_to_map(row, group_by_columns_list):
140
+        """Iterate through group by column values from grouped data set and extract any values.
141
+
142
+        Return a dictionary which contains original group by columns name and value pairs, if they
143
+        are available from the grouped data set.
144
+
145
+        """
146
+        extra_data_map = getattr(row, "extra_data_map", {})
147
+        # add group by fields data to extra data map
148
+        for column_name in group_by_columns_list:
149
+            column_value = getattr(row, column_name, Component.
150
+                                   DEFAULT_UNAVAILABLE_VALUE)
151
+            if (column_value == Component.DEFAULT_UNAVAILABLE_VALUE
152
+                    and (column_name.startswith("dimensions.")
153
+                         or column_name.startswith("meta.")
154
+                         or column_name.startswith("value_meta.")
155
+                         or column_name.startswith("extra_data_map."))):
156
+                split_column_name = column_name.split(".", 1)[-1]
157
+                column_value = getattr(row, split_column_name, Component.
158
+                                       DEFAULT_UNAVAILABLE_VALUE)
159
+            extra_data_map[column_name] = column_value
160
+        return extra_data_map
161
+
162
+    @staticmethod
163
+    def extract_dimensions(instance_usage_dict, dimension_list):
164
+        """Extract dimensions from instance usage.
165
+
166
+        """
167
+        dimensions_part = {}
168
+        # extra_data_map
169
+        extra_data_map = instance_usage_dict.get("extra_data_map", {})
170
+
171
+        for dim in dimension_list:
172
+            value = instance_usage_dict.get(dim)
173
+            if value is None:
174
+                # lookup for value in extra_data_map
175
+                if len(list(extra_data_map)) > 0:
176
+                    value = extra_data_map.get(dim, "all")
177
+            if dim.startswith("dimensions#"):
178
+                dim = dim.replace("dimensions#", "")
179
+            elif dim.startswith("meta#"):
180
+                dim = dim.replace("meta#", "")
181
+            elif dim.startswith("value_meta#"):
182
+                dim = dim.replace("value_meta#", "")
183
+            dimensions_part[dim] = value
184
+
185
+        return dimensions_part
186
+
84 187
 
85 188
 class RecordStoreUtils(TransformUtils):
86 189
     """utility methods to transform record store data."""
@@ -95,9 +198,7 @@ class RecordStoreUtils(TransformUtils):
95 198
                    "user_id", "region", "zone",
96 199
                    "host", "project_id", "service_group", "service_id",
97 200
                    "event_date", "event_hour", "event_minute",
98
-                   "event_second", "metric_group", "metric_id",
99
-                   "namespace", "pod_name", "app", "container_name",
100
-                   "interface", "deployment", "daemon_set"]
201
+                   "event_second", "metric_group", "metric_id"]
101 202
 
102 203
         columns_struct_fields = [StructField(field_name, StringType(), True)
103 204
                                  for field_name in columns]
@@ -110,6 +211,25 @@ class RecordStoreUtils(TransformUtils):
110 211
                                      StructField("event_quantity",
111 212
                                                  DoubleType(), True))
112 213
 
214
+        # map to metric meta
215
+        columns_struct_fields.append(StructField("meta",
216
+                                                 MapType(StringType(),
217
+                                                         StringType(),
218
+                                                         True),
219
+                                                 True))
220
+        # map to dimensions
221
+        columns_struct_fields.append(StructField("dimensions",
222
+                                                 MapType(StringType(),
223
+                                                         StringType(),
224
+                                                         True),
225
+                                                 True))
226
+        # map to value_meta
227
+        columns_struct_fields.append(StructField("value_meta",
228
+                                                 MapType(StringType(),
229
+                                                         StringType(),
230
+                                                         True),
231
+                                                 True))
232
+
113 233
         schema = StructType(columns_struct_fields)
114 234
 
115 235
         return schema
@@ -127,6 +247,33 @@ class RecordStoreUtils(TransformUtils):
127 247
         record_store_df = sql_context.read.json(jsonpath, schema)
128 248
         return record_store_df
129 249
 
250
+    @staticmethod
251
+    def prepare_recordstore_group_by_list(group_by_list):
252
+        """Prepare record store group by list.
253
+
254
+        If the group by list contains any instances of "dimensions#", "meta#" or "value_meta#" then
255
+        convert into proper dotted notation. i.e. "dimensions.", "meta." and
256
+        "value_meta." to reference columns in record_store data.
257
+
258
+        """
259
+        return [RecordStoreUtils.prepare_group_by_item(item) for item in group_by_list]
260
+
261
+    @staticmethod
262
+    def prepare_group_by_item(item):
263
+        """Prepare record store item for group by.
264
+
265
+        Convert replaces any special "dimensions#", "meta#" or "value#meta" occurrences into
266
+        "dimensions.", "meta." and value_meta.".
267
+
268
+        """
269
+        if item.startswith("dimensions#"):
270
+            item = item.replace("dimensions#", "dimensions.")
271
+        elif item.startswith("meta#"):
272
+            item = item.replace("meta#", "meta.")
273
+        elif item.startswith("value_meta#"):
274
+            item = item.replace("value_meta#", "value_meta.")
275
+        return item
276
+
130 277
 
131 278
 class TransformSpecsUtils(TransformUtils):
132 279
     """utility methods to transform_specs."""
@@ -219,50 +366,34 @@ class MonMetricUtils(TransformUtils):
219 366
     @staticmethod
220 367
     def _get_mon_metric_json_schema():
221 368
         """get the schema of the incoming monasca metric."""
222
-        dimensions = ["apache_host", "apache_port", "component",
223
-                      "consumer_group", "device", "hostname",
224
-                      "mode", "mount", "mount_point", "observer_host",
225
-                      "process_name", "project_id", "resource_id", "service",
226
-                      "test_type", "tenantId", "tenant_id", "topic", "url",
227
-                      "state", "state_description", "instanceId",
228
-                      "namespace", "pod_name", "app", "container_name",
229
-                      "interface", "deployment", "daemon_set"]
230
-
231
-        dimensions_struct_fields = [
232
-            StructField(field_name, StringType(), True)
233
-            for field_name in dimensions]
234
-
235
-        value_meta = ["host"]
236
-
237
-        value_meta_struct_fields = [
238
-            StructField(field_name, StringType(), True)
239
-            for field_name in value_meta]
240 369
 
241 370
         metric_struct_field = StructField(
242 371
             "metric",
243 372
             StructType([StructField("dimensions",
244
-                                    StructType(dimensions_struct_fields)),
373
+                                    MapType(StringType(),
374
+                                            StringType(),
375
+                                            True),
376
+                                    True),
245 377
                         StructField("value_meta",
246
-                                    StructType(value_meta_struct_fields)),
378
+                                    MapType(StringType(),
379
+                                            StringType(),
380
+                                            True),
381
+                                    True),
247 382
                         StructField("name", StringType(), True),
248 383
                         StructField("timestamp", StringType(), True),
249 384
                         StructField("value", StringType(), True)]), True)
250 385
 
251
-        meta_struct_field = StructField(
252
-            "meta",
253
-            StructType([StructField("timestamp", StringType(), True),
254
-                        StructField("region", StringType(), True),
255
-                        StructField("tenantId", StringType(), True),
256
-                        StructField("userId", StringType(), True),
257
-                        StructField("zone", StringType(), True),
258
-                        StructField("geolocation", StringType(), True)]))
386
+        meta_struct_field = StructField("meta",
387
+                                        MapType(StringType(),
388
+                                                StringType(),
389
+                                                True),
390
+                                        True)
259 391
 
260 392
         creation_time_struct_field = StructField("creation_time",
261 393
                                                  StringType(), True)
262 394
 
263 395
         schema = StructType([creation_time_struct_field,
264 396
                              meta_struct_field, metric_struct_field])
265
-
266 397
         return schema
267 398
 
268 399
     @staticmethod
@@ -323,6 +454,36 @@ class PreTransformSpecsUtils(TransformUtils):
323 454
         pre_transform_specs_df = sql_context.read.json(jsonpath, schema)
324 455
         return pre_transform_specs_df
325 456
 
457
+    @staticmethod
458
+    def prepare_required_raw_fields_list(group_by_list):
459
+        """Prepare required fields list.
460
+
461
+        If the group by list contains any instances of "dimensions#field", "meta#field" or
462
+        "value_meta#field" then convert them into metric.dimensions["field"] syntax.
463
+
464
+        """
465
+        return [PreTransformSpecsUtils.prepare_required_raw_item(item) for item in group_by_list]
466
+
467
+    @staticmethod
468
+    def prepare_required_raw_item(item):
469
+        """Prepare required field item.
470
+
471
+        Convert replaces any special "dimensions#", "meta#" or "value_meta" occurrences into
472
+        spark rdd syntax to fetch field value.
473
+
474
+        """
475
+        if item.startswith("dimensions#"):
476
+            field_name = item.replace("dimensions#", "")
477
+            return "metric.dimensions['%s']" % field_name
478
+        elif item.startswith("meta#"):
479
+            field_name = item.replace("meta#", "")
480
+            return "meta['%s']" % field_name
481
+        elif item.startswith("value_meta#"):
482
+            field_name = item.replace("value_meta#", "")
483
+            return "metric.value_meta['%s']" % field_name
484
+        else:
485
+            return item
486
+
326 487
 
327 488
 class GroupingResultsUtils(TransformUtils):
328 489
     """utility methods to transform record store data."""

+ 7
- 7
tests/functional/data_driven_specs/pre_transform_specs/pre_transform_specs.json View File

@@ -12,14 +12,14 @@
12 12
 {"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"}
13 13
 {"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"}
14 14
 {"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"}
15
-{"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"}
16
-{"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"}
15
+{"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"}
16
+{"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"}
17 17
 {"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"}
18
-{"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"}
19
-{"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"}
20
-{"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"}
21
-{"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"}
22
-{"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"}
18
+{"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"}
19
+{"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"}
20
+{"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"}
21
+{"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"}
22
+{"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"}
23 23
 {"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"}
24 24
 {"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"}
25 25
 {"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"}

+ 2
- 2
tests/functional/data_driven_specs/test_data_driven_specs.py View File

@@ -410,7 +410,7 @@ class TestDataDrivenSpecsRepo(SparkContextTest):
410 410
         self.check_list_field_for_row(
411 411
             row=swiftlm_diskusage_all_row,
412 412
             field_name='required_raw_fields_list',
413
-            expected_list=['creation_time', 'hostname', 'mount'],
413
+            expected_list=['creation_time', 'hostname', 'dimensions#mount'],
414 414
         )
415 415
         self.check_dict_field_for_row(
416 416
             row=swiftlm_diskusage_all_row,
@@ -439,7 +439,7 @@ class TestDataDrivenSpecsRepo(SparkContextTest):
439 439
         self.check_list_field_for_row(
440 440
             row=swiftlm_diskavail_all_row,
441 441
             field_name='required_raw_fields_list',
442
-            expected_list=['creation_time', 'hostname', 'mount'],
442
+            expected_list=['creation_time', 'hostname', 'dimensions#mount'],
443 443
         )
444 444
         self.check_dict_field_for_row(
445 445
             row=swiftlm_diskavail_all_row,

+ 12
- 12
tests/functional/data_driven_specs/transform_specs/transform_specs.json View File

@@ -24,18 +24,18 @@
24 24
 {"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"}
25 25
 {"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"}
26 26
 {"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"}
27
-{"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"}
28
-{"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"}
29
-{"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"}
30
-{"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"}
31
-{"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"}
32
-{"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"}
33
-{"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"}
34
-{"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"}
35
-{"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"}
36
-{"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"}
37
-{"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"}
38
-{"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"}
27
+{"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"}
28
+{"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"}
29
+{"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"}
30
+{"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"}
31
+{"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"}
32
+{"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"}
33
+{"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"}
34
+{"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"}
35
+{"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"}
36
+{"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"}
37
+{"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"}
38
+{"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"}
39 39
 {"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"}
40 40
 {"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"}
41 41
 {"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"}

+ 12
- 0
tests/functional/test_resources/fetch_quantity_data/fetch_quantity_data.txt View File

@@ -2,3 +2,15 @@
2 2
 ('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}')
3 3
 ('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}')
4 4
 ('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}')
5
+('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}')
6
+('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}')
7
+('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}')
8
+('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}')
9
+('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}')
10
+('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}')
11
+('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}')
12
+('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}')
13
+('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}')
14
+('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}')
15
+('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}')
16
+('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}')

+ 32
- 32
tests/functional/test_resources/metrics_pre_hourly_data/metrics_pre_hourly_data.txt View File

@@ -9,49 +9,49 @@
9 9
 ('',{"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})
10 10
 ('',{"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})
11 11
 ('',{"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})
12
-('',{"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})
13
-('',{"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})
12
+('',{"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"}})
13
+('',{"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"}})
14 14
 ('',{"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})
15 15
 ('',{"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})
16
-('',{"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})
17
-('',{"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})
18
-('',{"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})
19
-('',{"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})
16
+('',{"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"}})
17
+('',{"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"}})
18
+('',{"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"}})
19
+('',{"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"}})
20 20
 ('',{"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})
21 21
 ('',{"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})
22
-('',{"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})
23
-('',{"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})
22
+('',{"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"}})
23
+('',{"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"}})
24 24
 ('',{"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})
25 25
 ('',{"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})
26
-('',{"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})
27
-('',{"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})
28
-('',{"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})
29
-('',{"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})
30
-('',{"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})
31
-('',{"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})
32
-('',{"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})
33
-('',{"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})
26
+('',{"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"}})
27
+('',{"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"}})
28
+('',{"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"}})
29
+('',{"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"}})
30
+('',{"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"}})
31
+('',{"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"}})
32
+('',{"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"}})
33
+('',{"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"}})
34 34
 ('',{"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})
35 35
 ('',{"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})
36
-('',{"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})
37
-('',{"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})
38
-('',{"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})
39
-('',{"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})
36
+('',{"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"}})
37
+('',{"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"}})
38
+('',{"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"}})
39
+('',{"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"