Initial commit moving openolt adapter from voltha-go to the new repo.
This version works with ponsim rather than openolt, this is temporary.
It is currently being fixed to work with openolt.

Change-Id: I34a800c98f050140b367e2d474b7aa8b79f34b9a
Signed-off-by: William Kurkian <wkurkian@cisco.com>
diff --git a/python/adapters/extensions/kpi/README.md b/python/adapters/extensions/kpi/README.md
new file mode 100644
index 0000000..a55c7d2
--- /dev/null
+++ b/python/adapters/extensions/kpi/README.md
@@ -0,0 +1,286 @@
+# VOLTHA Performance Monitoring/KPI Library
+
+This directory provides a common library for the creation of Performance Monitoring groups
+within VOLTHA and should be used to insure that KPI information from different adapters use
+the same format.
+
+The original KpiEvent protobuf message is still supported for adapters that wish to use theprevious format but device adapter developers are encouraged to support the new format and
+make use of this shared library. 
+
+**Also**, please read the **Remaining Work Item** sections of each README.md file. Some additional
+work items as well as existing/related JIRA items are highlighted in this section. 
+
+## KPI Manager Creation
+
+Currently, each device adapter is required to follow the following steps to create and
+register PM Metric manager. This is typically performed in the device handler's
+'activate' method (called in response to the device handler first being enabled)
+
+1. Create an instance of a derived **AdapterPmMetrics** manager object. This is currently an
+   **OltPmMetrics** object for an _OLT_ adapter, or an **OnuPmMetrics** adapter for an
+   _ONU_ adapter. If you have additional device specific metrics to report, you can
+   derive your own manager object from one of these two derived classes. In order to
+   inherit (or modify) the metrics defined in those classes as well as support any new
+   metrics specific to your device.
+   
+   This call takes a number of device adapter specific arguments and these are detailed
+   in the pydoc headers for the appropriate  **AdapterPmMetrics** _\_\_init___() method.
+   
+2. Create the ProtoBuf message for your metrics by calling the newly created _manager's_
+   **_make_proto_**() method. 
+   
+3. Register the ProtoBuf message configuration with the adapter agent via the 
+   _update_device_pm_config_() method with the optional init parameter set to **True**.
+   
+4. Request the manager to schedule the first PM collection interval by calling the
+   manager's _start_collector_() method. You may wish to do this after a short pause
+   depending on how your adapter is designed.
+   
+**NOTE:** Currently there is only a single collection frequency for all metrics for
+a given device adapter. In the future, individual collection intervals on a per-metric/metric-group
+will be supported by the shared library.
+   
+The next two subsections provides examples of these steps for both an OLT and an ONU
+device adapter  
+
+### OLT Device Adapters PM Manager setup
+
+```python
+    # Create the OLT PM Manager object
+    kwargs = {
+        'nni-ports': self.northbound_ports.values(),
+        'pon-ports': self.southbound_ports.values()
+    }
+    self.pm_metrics = OltPmMetrics(self.adapter_agent, self.device_id, self.logical_device_id,
+                                   grouped=True, freq_override=False,
+                                   **kwargs)
+
+    # Create the protobuf message configuration
+    pm_config = self.pm_metrics.make_proto()
+    self.log.debug("initial-pm-config", pm_config=pm_config)
+    
+    # Create the PM information in the adapter agent
+    self.adapter_agent.update_device_pm_config(pm_config, init=True)
+        
+    # Start collecting stats from the device after a brief pause
+    reactor.callLater(10, self.pm_metrics.start_collector)
+```
+
+### ONU Device Adapters PM Manager Setup
+
+For ONU devices, if you wish to include OpenOMCI 15-minute historical interval
+intervals, you will need to register the PM Metrics OpenOMCI Interval PM class
+with OpenOMCI.  This ties in the OpenOMCI PM Interval State Machine with the KPI
+shared library.
+
+```python
+
+    # Create the OLT PM Manager object
+    kwargs = {
+        'heartbeat': self.heartbeat,
+        'omci-cc': self.openomci.omci_cc
+    }
+    self.pm_metrics = OnuPmMetrics(self.adapter_agent, self.device_id, self.logical_device_id,
+                                   grouped=True, freq_override=False,
+                                   **kwargs)
+                                   
+    # Create the protobuf message configuration
+    pm_config = self.pm_metrics.make_proto()
+    
+    # Register the OMCI history intervals with OpenOMCI
+    self.openomci.set_pm_config(self.pm_metrics.omci_pm.openomci_interval_pm)
+    
+    # Create the PM information in the adapter agent
+    self.adapter_agent.update_device_pm_config(pm_config, init=True)
+    
+    # Start collecting stats from the device after a brief pause
+    reactor.callLater(30, self.pm_metrics.start_collector)
+```
+
+### How metrics are currently collected
+
+Currently, the default behaviour is to collect KPI information on a single periodic 
+interval that can be adjusted via the NBI/CLI of VOLTHA.  It collects data by extracting
+it from an object provided during the collection request and this object should either
+provide attributes or a property method that matches the metric to be collected.
+For instance, assume that you have an NNI metric called 'tx_packets'.  You would pass
+an object during collection that should have one of the two following;
+
+- a _tx_packets_ attribute/member name defined for the object that has the requested
+  value already set (via background poll)
+  
+- a _tx_packets_ **property** method that accesses an internal variable with the value
+  already set (via background poll) or that calculates/extracts the value without blockin
+  the call.
+
+### Known Issues in collection
+
+Note that a future story will be created to allow for collection to be requested for
+a metric/metric-group on demand so that background polling of KPI information is not
+required for all reported metrics.
+
+Note that a future story will be created to allow KPI information to be collected on
+per-group/metric intervals.
+
+# Basic KPI Format (**KpiEvent2**)
+
+The KPI information is published on the kafka bus under the _voltha.kpi_ topic. For 
+VOLTHA PM information, the kafka key is empty and the value is a JSON message composed
+of the following key-value pairs.
+
+| key        | value  | Notes |
+| :--------: | :----- | :---- |
+| type       | string | "slice" or "ts". A "slice" is a set of path/metric data for the same time-stamp. A "ts" is a time-series: array of data for same metric |
+| ts         | float  | UTC time-stamp of when the KpiEvent2 was created (seconds since the epoch of January 1, 1970) |
+| slice_data | list   | One or more sets of metrics composed of a _metadata_ section and a _metrics_ section. |
+
+**NOTE**: Time-series metrics and corresponding protobuf messages have not been defined.
+
+## Slice Data Format
+
+For KPI slice KPI messages, the _slice_data_ portion of the **KpiEvent2** is composed of a _metadata_
+section and a _metrics_ section.
+
+### _metadata_ Section Format
+
+The metadata section is used to:
+ - Define which metric/metric-group is being reported (The _title_ field)
+ - Provide some common fields required by all metrics (_title_, _timestamp_, _device ID_, ...)
+ - Provide metric/metric-group specific context (the _context_ fields)
+
+| key        | value  | Notes |
+| :--------: | :----- | :---- |
+| title       | string | "slice" or "ts". A "slice" is a set of path/metric data for the same time-stamp. A "ts" is a time-series: array of data for same metric |
+| ts         | float | UTC time-stamp of data at the time of collection (seconds since the epoch of January 1, 1970) |
+| logical_device_id | string | The logical ID that the device belongs to. This is equivalent to the DPID reported in ONOS for the VOLTHA logical device with the 'of:' prefix removed. |
+| device_id | string | The physical device ID that is reporting the metric. |
+| serial_number | string | The reported serial number for the physical device reporting the metric. |
+| context | map | A key-value map of metric/metric-group specific information.|
+
+The context map is composed of key-value pairs where the key (string) is the label for the context
+specific value and the value (string) is the corresponding context value. While most values may be
+better represented as a float/integer, there may be some that are better represented as text. For
+this reason, values are always represented as strings to allow the ProtoBuf message format to be
+as simple as possible.
+
+Here is an JSON _example_ of a current KPI published on the kafka bus under the 
+_voltha.kpi_ topic. 
+
+```json
+{
+  "type": "slice",
+  "ts": 1534440704.0,
+  "slice_data": [
+    {
+      "metadata": {
+        "title": "Ethernet",
+        "ts": 1534440704.0,
+        "logical_device_id": "000000139521a269",
+        "device_id": "000115929ed71696",
+        "serial_no": "dummy_sn2209199",
+        "context": {
+          "port_no": "1"
+        }
+      },
+      "metrics": {
+        "tx_dropped": 0.0,    # A COUNTER
+        "rx_packets": 0.0,
+        "rx_bytes": 0.0,
+        "rx_mcast_packets": 0.0,
+        "tx_mcast_packets": 16.0,
+        "rx_bcast_packets": 0.0,
+        "oper_status": 4.0,   # A STATE
+        "admin_state": 3.0,
+        "rx_errors": 0.0,
+        "tx_bytes": 1436.0,
+        "rx_dropped": 0.0,
+        "tx_packets": 16.0,
+        "tx_bcast": 0.0
+      }
+    },
+    {
+      "metadata": {
+        "title": "PON",
+        "logical_device_id": "000000139521a269",
+        "device_id": "000115929ed71696",
+        "serial_no": "dummy_sn2209199",
+        "ts": 1534440704.0,
+        "context": {
+          "port_no": "5",
+          "pon_id": "0"
+        },
+      },
+      "metrics": {
+        "rx_packets": 0.0,
+        "in_service_onus": 0.0,     # A GAUGE
+        "rx_bytes": 0.0,
+        "closest_onu_distance": -1.0,
+        "tx_bip_errors": 0.0,
+        "oper_status": 4.0,
+        "admin_state": 3.0,
+        "tx_bytes": 0.0,
+        "tx_packets": 0.0
+      }
+    },
+    ...
+}
+
+```
+
+For OpenOMCI historical intervals, the name is derived from the Managed Entity class:
+
+```json
+{
+  "type": "slice",
+  "ts": 1532372864.0,
+  "prefixes": {
+    "voltha.adtran_onu.0001b8c505090b5b.EthernetFrameExtendedPerformanceMonitoring": {
+      "metrics": {
+        "entity_id": 2.0,
+        "class_id": 334.0,
+        "packets": 0.0,
+        "octets": 0.0,
+        "interval_end_time": 0.0,
+        "crc_errored_packets": 0.0,
+        "broadcast_packets": 0.0,
+        "64_octets": 0.0,
+        "65_to_127_octets": 0.0,
+        "128_to_255_octets": 0.0,
+        "256_to_511_octets": 0.0,
+        "undersize_packets": 0.0,
+        "drop_events": 0.0,
+        "multicast_packets": 0.0,
+        "oversize_packets": 0.0
+      }
+    }
+  }
+}
+```
+More information on the OpenOMCI ONU Historical Intervals is detailed in the _IntervalMetrics.md_
+file in the _onu/_ subdirectory.
+
+# Remaining Work Items
+
+This initial code is only a preliminary sample. The following tasks need to be
+added to the VOLTHA JIRA or performed in the SEBA group:
+    
+- Get feedback from other OLT/ONU developers on any needed changes
+
+- Allow PM groups to have different collection times
+
+- Support calling a 'get-data' method before collect the metrics.  Currently metrics are collected
+  in a device adapter independent way and the PM just updates what the attributes happen to have.
+  This would provide an asynchronous request and upon successful completion, the KPI metric/group
+  would be published on the Kafka bus.
+
+- [VOL-931](https://jira.opencord.org/browse/VOL-931) Support for retrieval of PM measurements
+  on-demaind. Would be best implemented after the previous async (get-data) work item.
+
+- For statistics groups that have more than one instance, do we need to be able to
+  enable/disable specific instances? Major refactor of code if so (database work, ...)
+
+- [VOL-930](https://jira.opencord.org/browse/VOL-930) PM Collection Format. This format may
+  fit better with the time-series KPI collection as it requests ability for start/stop times.
+  It could possibly be done at a higher layer but the intent may be to have a greater number
+  of samples on a specific metric instance for a defined period of time. Need clarification
+  from the JIRA author.
diff --git a/python/adapters/extensions/kpi/__init__.py b/python/adapters/extensions/kpi/__init__.py
new file mode 100644
index 0000000..b0fb0b2
--- /dev/null
+++ b/python/adapters/extensions/kpi/__init__.py
@@ -0,0 +1,13 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
diff --git a/python/adapters/extensions/kpi/adapter_pm_metrics.py b/python/adapters/extensions/kpi/adapter_pm_metrics.py
new file mode 100644
index 0000000..fe14fee
--- /dev/null
+++ b/python/adapters/extensions/kpi/adapter_pm_metrics.py
@@ -0,0 +1,217 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+import structlog
+import arrow
+from twisted.internet.task import LoopingCall
+from voltha.protos.events_pb2 import KpiEvent2, KpiEventType, MetricInformation, MetricMetaData
+from voltha.protos.device_pb2 import PmConfig
+
+
+class AdapterPmMetrics(object):
+    """
+    Base class for Device Adapter PM Metrics Manager
+
+    Device specific (OLT, ONU, OpenOMCI, ...) will derive groups of PM information
+    and this base class is primarily used to provide a consistent interface to configure,
+    start, and stop statistics collection.
+    """
+    DEFAULT_FREQUENCY_KEY = 'default-collection-frequency'
+    DEFAULT_COLLECTION_FREQUENCY = 15 * 10      # 1/10ths of a second
+
+    # If the collection object has a property of the following name, it will be used
+    # to retrieve the UTC Collection Timestamp (UTC seconds since epoch). If the collection
+    # object does not support this attribute, the current time will be used. If the attribute
+    # is supported, but returns None, this signals that no metrics are currently available
+    # for collection.
+    TIMESTAMP_ATTRIBUTE = 'timestamp'
+
+    def __init__(self, adapter_agent, device_id, logical_device_id,
+                 grouped=False, freq_override=False, **kwargs):
+        """
+        Initializer for shared Device Adapter PM metrics manager
+
+        :param adapter_agent: (AdapterAgent) Adapter agent for the device
+        :param device_id: (str) Device ID
+        :param logical_device_id: (str) VOLTHA Logical Device ID
+        :param grouped: (bool) Flag indicating if statistics are managed as a group
+        :param freq_override: (bool) Flag indicating if frequency collection can be specified
+                                     on a per group basis
+        :param kwargs: (dict) Device Adapter specific values
+        """
+        self.log = structlog.get_logger(device_id=device_id)
+        self.device_id = device_id
+        self.adapter_agent = adapter_agent
+        self.name = adapter_agent.adapter_name
+        # Sanitize the vcore ID in the logical device ID
+        self.logical_device_id = '0000' + logical_device_id[4:]
+        device = self.adapter_agent.get_device(self.device_id)
+        self.serial_number = device.serial_number
+
+        self.default_freq = kwargs.get(AdapterPmMetrics.DEFAULT_FREQUENCY_KEY,
+                                       AdapterPmMetrics.DEFAULT_COLLECTION_FREQUENCY)
+        self.grouped = grouped
+        self.freq_override = grouped and freq_override
+        self.lc = None
+        self.pm_group_metrics = dict()      # name -> PmGroupConfig
+
+    def update(self, pm_config):
+        # TODO: Move any common steps into base class
+        raise NotImplementedError('Your derived class should override this method')
+
+    def make_proto(self, pm_config=None):
+        raise NotImplementedError('Your derived class should override this method')
+
+    def start_collector(self, callback=None):
+        """
+        Start the collection loop for an adapter if the frequency > 0
+
+        :param callback: (callable) Function to call to collect PM data
+        """
+        self.log.info("starting-pm-collection", device_name=self.name)
+        if callback is None:
+            callback = self.collect_and_publish_metrics
+
+        if self.lc is None:
+            self.lc = LoopingCall(callback)
+
+        if self.default_freq > 0:
+            self.lc.start(interval=self.default_freq / 10)
+
+    def stop_collector(self):
+        """ Stop the collection loop"""
+        if self.lc is not None and self.default_freq > 0:
+            self.lc.stop()
+
+    def collect_group_metrics(self, group_name, group, names, config):
+        """
+        Collect the metrics for a specific PM group.
+
+        This common collection method expects that the 'group object' provide as the second
+        parameter supports an attribute or property with the name of the value to
+        retrieve.
+
+        :param group_name: (str) The unique collection name. The name should not contain spaces.
+        :param group: (object) The object to query for the value of various attributes (PM names)
+        :param names: (set) A collection of PM names that, if implemented as a property in the object,
+                            will return a value to store in the returned PM dictionary
+        :param config: (PMConfig) PM Configuration settings. The enabled flag is examined to determine
+                                  if the data associated with a PM Name will be collected.
+
+        :return: (MetricInformation) collected metrics
+        """
+        assert ' ' not in group_name,  'Spaces are not allowed in metric titles, use an underscore'
+
+        if group is None:
+            return None
+
+        metrics = dict()
+        context = dict()
+
+        now = getattr(group, AdapterPmMetrics.TIMESTAMP_ATTRIBUTE) \
+            if hasattr(group, AdapterPmMetrics.TIMESTAMP_ATTRIBUTE) \
+            else arrow.utcnow().float_timestamp
+
+        if now is None:
+            return None     # No metrics available at this time for collection
+
+        for (metric, t) in names:
+            if config[metric].type == PmConfig.CONTEXT and hasattr(group, metric):
+                context[metric] = str(getattr(group, metric))
+
+            elif config[metric].type in (PmConfig.COUNTER, PmConfig.GAUGE, PmConfig.STATE):
+                if config[metric].enabled and hasattr(group, metric):
+                    metrics[metric] = getattr(group, metric)
+
+        # Check length of metric data. Will be zero if if/when individual group
+        # metrics can be disabled and all are (or or not supported by the
+        # underlying adapter)
+        if len(metrics) == 0:
+            return None
+
+        return MetricInformation(metadata=MetricMetaData(title=group_name,
+                                                         ts=now,
+                                                         logical_device_id=self.logical_device_id,
+                                                         serial_no=self.serial_number,
+                                                         device_id=self.device_id,
+                                                         context=context),
+                                 metrics=metrics)
+
+    def collect_metrics(self, data=None):
+        """
+        Collect metrics for this adapter.
+
+        The adapter type (OLT, ONU, ..) should provide a derived class where this
+        method iterates through all metrics and collects them up in a dictionary with
+        the group/metric name as the key, and the metric values as the contents.
+
+        The data collected (or passed in) is a list of pairs/tuples.  Each
+        pair is composed of a MetricMetaData metadata-portion and list of MetricValuePairs
+        that contains a single individual metric or list of metrics if this is a
+        group metric.
+
+        This method is called for each adapter at a fixed frequency.
+        TODO: Currently all group metrics are collected on a single timer tick.
+              This needs to be fixed as independent group or instance collection is
+              desirable.
+
+        :param data: (list) Existing list of collected metrics (MetricInformation).
+                            This is provided to allow derived classes to call into
+                            further encapsulated classes.
+
+        :return: (list) metadata and metrics pairs - see description above
+        """
+        raise NotImplementedError('Your derived class should override this method')
+
+    def collect_and_publish_metrics(self):
+        """ Request collection of all enabled metrics and publish them """
+        try:
+            data = self.collect_metrics()
+            self.publish_metrics(data)
+
+        except Exception as e:
+            self.log.exception('failed-to-collect-kpis', e=e)
+
+    def publish_metrics(self, data):
+        """
+        Publish the metrics during a collection.
+
+        The data collected (or passed in) is a list of dictionary pairs/tuple.  Each
+        pair is composed of a metadata-portion and a metrics-portion that contains
+        information for a specific instance of an individual metric or metric group.
+
+        :param data: (list) Existing list of collected metrics (MetricInformation)
+                            to convert to a KPIEvent and publish
+        """
+        self.log.debug('publish-metrics')
+
+        if len(data):
+            try:
+                # TODO: Existing adapters use the KpiEvent, if/when all existing
+                #       adapters use the shared KPI library, we may want to
+                #       deprecate the KPIEvent
+                kpi_event = KpiEvent2(
+                    type=KpiEventType.slice,
+                    ts=arrow.utcnow().float_timestamp,
+                    slice_data=data
+                )
+                self.adapter_agent.submit_kpis(kpi_event)
+
+            except Exception as e:
+                self.log.exception('failed-to-submit-kpis', e=e)
+
+    # TODO: Need to support on-demand counter update if provided by the PM 'group'.
+    #       Currently we expect PM data to be periodically polled by a separate
+    #       mechanism. The on-demand counter update should be optional in case the
+    #       particular device adapter group of data is polled anyway for other reasons.
diff --git a/python/adapters/extensions/kpi/olt/README.md b/python/adapters/extensions/kpi/olt/README.md
new file mode 100644
index 0000000..e0012b8
--- /dev/null
+++ b/python/adapters/extensions/kpi/olt/README.md
@@ -0,0 +1,179 @@
+# OLT PM Metrics
+
+
+**THESE ARE PRELIMINARY METRIC GROUPS**, Work is needed by the VOLTHA community to reach a consensus on the
+actual metrics that will be provided. **Also**, please read the **Remaining Work Item** sections of each
+README file.
+
+
+
+This document outlines the metrics reported by VOLTHA OLTs.  These are currently collected
+from OLT Device Adapter which is responsible for polling the hardware for information. A future
+version of the Performance Monitoring Library will allow for collection on-demand.
+
+## Format on the Kafka bus
+
+The format of the OLT KPI Events is detailed in the [Basic KPI Format (**KpiEvent2**)](../README.md)
+section of this documents parent directory for wire format on the bus. This document primarily provides
+the group metric information for OLT PKIs and associated metadata context information.
+
+**All** metric values reported by the library are reported as *float*s. The context and metric tables
+listed in the sections below report the type as initially collected by the OLT Device Adapters.
+
+#OLT PM Metric Groups
+
+The following sections outline the KPI metrics gathered by most OLT Device adapters. If an OLT does not
+support a specific metric in a group, it will not report that metric. This is preferred to reporting a
+metric and it always having a value of 0.0 (which could be misleading).
+
+## Admin and Oper State/status
+
+Various interfaces will provide a numeric (integer) value for the current Admin State and Operation
+Status of the interface.  These map to the following states:
+
+**Admin State**
+
+| State             | Value | Notes |
+| ----------------: | :---: | :---- |
+| UNKNOWN           |   0   | The administrative state of the device is unknown |
+| DISABLED          |   2   | The device is disabled and shall not perform its intended forwarding functions other than being available for re-activation. |
+| PREPROVISIONED    |   1   | The device is pre-provisioned into Voltha, but not contacted by it |
+| ENABLED           |   3   | The device is enabled for activation and operation |
+| DOWNLOADING_IMAGE |   4   | The device is in the state of image download |
+
+**Operational Status**
+
+| State      | Value | Notes |
+| ---------: | :---: | :---- |
+| UNKNOWN    |   0   | The status of the device is unknown at this point |
+| DISCOVERED |   1   | The device has been discovered, but not yet activated |
+| ACTIVATING |   2   | The device is being activated (booted, rebooted, upgraded, etc.) |
+| TESTING    |   3   | Service impacting tests are being conducted |
+| ACTIVE     |   4   | The device is up and active |
+| FAILED     |   5   | The device has failed and cannot fulfill its intended role |
+
+## NNI KPI Metrics
+
+This metric provides metrics for a specific NNI Port of an OLT
+
+**Metadata Context items**
+
+| key         | value   | Notes |
+| :---------: | :------ | :---- |
+| intf_id     | integer | Physical device interface port number for this NNI port |
+
+**Metrics**
+
+| key              | type / size  | Notes |
+| :--------------: | :----------- | :---- |
+| admin_state      | state        | See _Admin State_ section above |
+| oper_status      | state        | See _Operational Status_ section above |
+| rx_bytes         | int, 64-bits | TODO: add definition here... |
+| rx_packets       | int, 64-bits | TODO: add definition here... |
+| rx_ucast_packets | int, 64-bits | TODO: add definition here... |
+| rx_mcast_packets | int, 64-bits | TODO: add definition here... |
+| rx_bcast_packets | int, 64-bits | TODO: add definition here... |
+| rx_error_packets | int, 64-bits | TODO: add definition here... | 
+| tx_bytes         | int, 64-bits | TODO: add definition here... |
+| tx_packets       | int, 64-bits | TODO: add definition here... |
+| tx_ucast_packets | int, 64-bits | TODO: add definition here... |
+| tx_mcast_packets | int, 64-bits | TODO: add definition here... |
+| tx_bcast_packets | int, 64-bits | TODO: add definition here... |
+| tx_error_packets | int, 64-bits | TODO: add definition here... |
+| rx_crc_errors    | int, 64-bits | TODO: add definition here... |
+| bip_errors       | int, 64-bits | TODO: add definition here... |
+
+## PON KPI Metrics
+
+The OLT PON Port metrics
+
+**Metadata Context items**
+
+| key         | value   | Notes |
+| :---------: | :------ | :---- |
+| intf_id     | integer | Physical device interface port number for this NNI port |
+| pon_id      | integer | PON ID (0..n) |
+
+**Metrics**
+
+| key                  | type / size  | Notes |
+| :------------------: | :----------- | :---- |
+| admin_state          | state        | See _Admin State_ section above |
+| oper_status          | state        | See _Operational Status_ section above |
+| rx_packets           | int, 64-bits | Sum of all the RX Packets of GEM ports that are not base TCONT's |
+| rx_bytes             | int, 64-bits | Sum of all the RX Octets of GEM ports that are not base TCONT's |
+| tx_packets           | int, 64-bits | Sum of all the TX Packets of GEM ports that are not base TCONT's |
+| tx_bytes             | int, 64-bits | Sum of all the TX Octets of GEM ports that are not base TCONT's |
+| tx_bip_errors        | int, 32-bits | Sum of all the TX ONU bip errors to get TX BIP's per PON |
+| in_service_onus      | int          | The number of activated ONUs on this pon |
+| closest_onu_distance | float        | Distance to the closest ONU, units=kM w/granularity in the thousandths |
+
+## ONU KPI Metrics
+
+The OLT metrics for each activated ONUs
+
+**Metadata Context items**
+
+| key         | value   | Notes |
+| :---------: | :------ | :---- |
+| intf_id     | integer | Physical device interface port number for this NNI port |
+| pon_id      | integer | PON ID (0..n) |
+| onu_id      | integer | ONU ID |
+
+**Metrics**
+
+| key                | type / size  | Notes |
+| :----------------: | :----------- | :---- |
+| fiber_length       | float        | Distance to ONU, units=kM w/granularity in the thousandths |
+| equalization_delay | int, 32-bits | Equalization delay |
+| rssi               | int, 32-bits | The received signal strength indication of the ONU. |
+
+**TODO**: How about the following as well?
+ - rx_packets - int, 32-bits - Rx packets received on all GEM ports
+ - rx_bytes   - int, 64-bits - Rx octets received on all GEM ports
+ - tx_packets - int, 32-bits - Tx packets transmitted on all GEM ports
+ - tx_bytes   - int, 64-bits - Rx packets transmitted on all GEM ports
+ - tx_bip_errors - int, 32-bits - Sum of all the TX ONU bip errors to get TX BIP's on all GEM ports
+
+## GEM Port KPI Metrics
+
+The GEM Port metrics for each activated ONUs
+
+**Metadata Context items**
+
+| key         | value   | Notes |
+| :---------: | :------ | :---- |
+| intf_id     | integer | Physical device interface port number for this NNI port |
+| pon_id      | integer | PON ID (0..n) |
+| onu_id      | integer | ONU ID |
+| gem_id      | integer | GEM Port ID |
+
+**Metrics**
+
+| key         | type / size  | Notes |
+| :---------: | :----------- | :---- |
+| alloc_id    | int, 16-bits | TODO: add definition here... |
+| rx_packets  | int, 32-bits | Rx packets received |
+| rx_bytes    | int, 64-bits | Rx octets received |
+| tx_packets  | int, 32-bits | Tx packets transmitted |
+| tx_bytes    | int, 64-bits | Rx packets transmitted |
+
+# Remaining Work Items
+
+This initial code is only a preliminary work. See the [Remaining Work Items](../README.md)
+section of this document's parent directory for a list of remaining tasks. 
+  
+- [VOL-932](https://jira.opencord.org/browse/VOL-932) PM Interval collection on the OLT. Need
+  to consult OLT device adapter vendors and operators for which KPIs would best fit in the
+  interval groups. Intervals differ from other metric groups as they are defined to collect on
+  a specific interval (15-minutes most common) and at the start of the interval, the counters
+  should be set to zero so that the accumulation during the interval is what is reported. See
+  also [VOL-933](https://jira.opencord.org/browse/VOL-932),
+       [VOL-934](https://jira.opencord.org/browse/VOL-934),
+       [VOL-935](https://jira.opencord.org/browse/VOL-935),
+       [VOL-938](https://jira.opencord.org/browse/VOL-938),
+       [VOL-939](https://jira.opencord.org/browse/VOL-939),
+       [VOL-940](https://jira.opencord.org/browse/VOL-940).
+       **NOTE**: A couple of the ones above are for the ONU
+
+TODO: For each group, list if the default is enabled/disabled
\ No newline at end of file
diff --git a/python/adapters/extensions/kpi/olt/__init__.py b/python/adapters/extensions/kpi/olt/__init__.py
new file mode 100644
index 0000000..b0fb0b2
--- /dev/null
+++ b/python/adapters/extensions/kpi/olt/__init__.py
@@ -0,0 +1,13 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
diff --git a/python/adapters/extensions/kpi/olt/olt_pm_metrics.py b/python/adapters/extensions/kpi/olt/olt_pm_metrics.py
new file mode 100644
index 0000000..ea2e0c8
--- /dev/null
+++ b/python/adapters/extensions/kpi/olt/olt_pm_metrics.py
@@ -0,0 +1,300 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from voltha.protos.device_pb2 import PmConfig, PmConfigs, PmGroupConfig
+from voltha.extensions.kpi.adapter_pm_metrics import AdapterPmMetrics
+
+
+class OltPmMetrics(AdapterPmMetrics):
+    """
+    Shared OL Device Adapter PM Metrics Manager
+
+    This class specifically addresses ONU general PM (health, ...) area
+    specific PM (OMCI, PON, UNI) is supported in encapsulated classes accessible
+    from this object
+    """
+    def __init__(self, adapter_agent, device_id, logical_device_id,
+                 grouped=False, freq_override=False, **kwargs):
+        """
+        Initializer for shared ONU Device Adapter PM metrics
+
+        :param adapter_agent: (AdapterAgent) Adapter agent for the device
+        :param device_id: (str) Device ID
+        :param logical_device_id: (str) VOLTHA Logical Device ID
+        :param grouped: (bool) Flag indicating if statistics are managed as a group
+        :param freq_override: (bool) Flag indicating if frequency collection can be specified
+                                     on a per group basis
+        :param kwargs: (dict) Device Adapter specific values. For an ONU Device adapter, the
+                              expected key-value pairs are listed below. If not provided, the
+                              associated PM statistics are not gathered:
+
+                              'nni-ports': List of objects that provide NNI (northbound) port statistics
+                              'pon-ports': List of objects that provide PON port statistics
+        """
+        super(OltPmMetrics, self).__init__(adapter_agent, device_id, logical_device_id,
+                                           grouped=grouped, freq_override=freq_override,
+                                           **kwargs)
+
+        # PM Config Types are COUNTER, GAUGE, and STATE
+        self.nni_pm_names = {
+            ('intf_id', PmConfig.CONTEXT),      # Physical device interface ID/Port number
+
+            ('admin_state', PmConfig.STATE),
+            ('oper_status', PmConfig.STATE),
+
+            ('rx_bytes', PmConfig.COUNTER),
+            ('rx_packets', PmConfig.COUNTER),
+            ('rx_ucast_packets', PmConfig.COUNTER),
+            ('rx_mcast_packets', PmConfig.COUNTER),
+            ('rx_bcast_packets', PmConfig.COUNTER),
+            ('rx_error_packets', PmConfig.COUNTER),
+
+            ('tx_bytes', PmConfig.COUNTER),
+            ('tx_packets', PmConfig.COUNTER),
+            ('tx_ucast_packets', PmConfig.COUNTER),
+            ('tx_mcast_packets', PmConfig.COUNTER),
+            ('tx_bcast_packets', PmConfig.COUNTER),
+            ('tx_error_packets', PmConfig.COUNTER),
+            ('rx_crc_errors', PmConfig.COUNTER),
+            ('bip_errors', PmConfig.COUNTER),
+        }
+        self.pon_pm_names = {
+            ('intf_id', PmConfig.CONTEXT),        # Physical device port number (PON)
+            ('pon_id', PmConfig.CONTEXT),         # PON ID (0..n)
+
+            ('admin_state', PmConfig.STATE),
+            ('oper_status', PmConfig.STATE),
+            ('rx_packets', PmConfig.COUNTER),
+            ('rx_bytes', PmConfig.COUNTER),
+            ('tx_packets', PmConfig.COUNTER),
+            ('tx_bytes', PmConfig.COUNTER),
+            ('tx_bip_errors', PmConfig.COUNTER),
+            ('in_service_onus', PmConfig.GAUGE),
+            ('closest_onu_distance', PmConfig.GAUGE)
+        }
+        self.onu_pm_names = {
+            ('intf_id', PmConfig.CONTEXT),        # Physical device port number (PON)
+            ('pon_id', PmConfig.CONTEXT),
+            ('onu_id', PmConfig.CONTEXT),
+
+            ('fiber_length', PmConfig.GAUGE),
+            ('equalization_delay', PmConfig.GAUGE),
+            ('rssi', PmConfig.GAUGE),
+        }
+        self.gem_pm_names = {
+            ('intf_id', PmConfig.CONTEXT),        # Physical device port number (PON)
+            ('pon_id', PmConfig.CONTEXT),
+            ('onu_id', PmConfig.CONTEXT),
+            ('gem_id', PmConfig.CONTEXT),
+
+            ('alloc_id', PmConfig.GAUGE),
+            ('rx_packets', PmConfig.COUNTER),
+            ('rx_bytes', PmConfig.COUNTER),
+            ('tx_packets', PmConfig.COUNTER),
+            ('tx_bytes', PmConfig.COUNTER),
+        }
+        self.nni_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                   for (m, t) in self.nni_pm_names}
+        self.pon_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                   for (m, t) in self.pon_pm_names}
+        self.onu_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                   for (m, t) in self.onu_pm_names}
+        self.gem_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                   for (m, t) in self.gem_pm_names}
+
+        self._nni_ports = kwargs.pop('nni-ports', None)
+        self._pon_ports = kwargs.pop('pon-ports', None)
+
+    def update(self, pm_config):
+        try:
+            # TODO: Test frequency override capability for a particular group
+            if self.default_freq != pm_config.default_freq:
+                # Update the callback to the new frequency.
+                self.default_freq = pm_config.default_freq
+                self.lc.stop()
+                self.lc.start(interval=self.default_freq / 10)
+
+            if pm_config.grouped:
+                for group in pm_config.groups:
+                    group_config = self.pm_group_metrics.get(group.group_name)
+                    if group_config is not None:
+                        group_config.enabled = group.enabled
+            else:
+                msg = 'There are no independent OLT metrics, only group metrics at this time'
+                raise NotImplemented(msg)
+
+        except Exception as e:
+            self.log.exception('update-failure', e=e)
+            raise
+
+    def make_proto(self, pm_config=None):
+        if pm_config is None:
+            pm_config = PmConfigs(id=self.device_id, default_freq=self.default_freq,
+                                  grouped=self.grouped,
+                                  freq_override=self.freq_override)
+        metrics = set()
+        have_nni = self._nni_ports is not None and len(self._nni_ports) > 0
+        have_pon = self._pon_ports is not None and len(self._pon_ports) > 0
+
+        if self.grouped:
+            if have_nni:
+                pm_ether_stats = PmGroupConfig(group_name='Ethernet',
+                                               group_freq=self.default_freq,
+                                               enabled=True)
+                self.pm_group_metrics[pm_ether_stats.group_name] = pm_ether_stats
+
+            else:
+                pm_ether_stats = None
+
+            if have_pon:
+                pm_pon_stats = PmGroupConfig(group_name='PON',
+                                             group_freq=self.default_freq,
+                                             enabled=True)
+
+                pm_onu_stats = PmGroupConfig(group_name='ONU',
+                                             group_freq=self.default_freq,
+                                             enabled=True)
+
+                pm_gem_stats = PmGroupConfig(group_name='GEM',
+                                             group_freq=self.default_freq,
+                                             enabled=True)
+
+                self.pm_group_metrics[pm_pon_stats.group_name] = pm_pon_stats
+                self.pm_group_metrics[pm_onu_stats.group_name] = pm_onu_stats
+                self.pm_group_metrics[pm_gem_stats.group_name] = pm_gem_stats
+            else:
+                pm_pon_stats = None
+                pm_onu_stats = None
+                pm_gem_stats = None
+
+        else:
+            pm_ether_stats = pm_config if have_nni else None
+            pm_pon_stats = pm_config if have_pon else None
+            pm_onu_stats = pm_config if have_pon else None
+            pm_gem_stats = pm_config if have_pon else None
+
+        if have_nni:
+            for m in sorted(self.nni_metrics_config):
+                pm = self.nni_metrics_config[m]
+                if not self.grouped:
+                    if pm.name in metrics:
+                        continue
+                    metrics.add(pm.name)
+                pm_ether_stats.metrics.extend([PmConfig(name=pm.name,
+                                                        type=pm.type,
+                                                        enabled=pm.enabled)])
+        if have_pon:
+            for m in sorted(self.pon_metrics_config):
+                pm = self.pon_metrics_config[m]
+                if not self.grouped:
+                    if pm.name in metrics:
+                        continue
+                    metrics.add(pm.name)
+                pm_pon_stats.metrics.extend([PmConfig(name=pm.name,
+                                                      type=pm.type,
+                                                      enabled=pm.enabled)])
+
+            for m in sorted(self.onu_metrics_config):
+                pm = self.onu_metrics_config[m]
+                if not self.grouped:
+                    if pm.name in metrics:
+                        continue
+                    metrics.add(pm.name)
+                pm_onu_stats.metrics.extend([PmConfig(name=pm.name,
+                                                      type=pm.type,
+                                                      enabled=pm.enabled)])
+
+            for m in sorted(self.gem_metrics_config):
+                pm = self.gem_metrics_config[m]
+                if not self.grouped:
+                    if pm.name in metrics:
+                        continue
+                    metrics.add(pm.name)
+                pm_gem_stats.metrics.extend([PmConfig(name=pm.name,
+                                                      type=pm.type,
+                                                      enabled=pm.enabled)])
+        if self.grouped:
+            pm_config.groups.extend([stats for stats in
+                                     self.pm_group_metrics.itervalues()])
+
+        return pm_config
+
+    def collect_metrics(self, data=None):
+        """
+        Collect metrics for this adapter.
+
+        The data collected (or passed in) is a list of pairs/tuples.  Each
+        pair is composed of a MetricMetaData metadata-portion and list of MetricValuePairs
+        that contains a single individual metric or list of metrics if this is a
+        group metric.
+
+        This method is called for each adapter at a fixed frequency.
+        TODO: Currently all group metrics are collected on a single timer tick.
+              This needs to be fixed as independent group or instance collection is
+              desirable.
+
+        :param data: (list) Existing list of collected metrics (MetricInformation).
+                            This is provided to allow derived classes to call into
+                            further encapsulated classes.
+
+        :return: (list) metadata and metrics pairs - see description above
+        """
+        if data is None:
+            data = list()
+
+        group_name = 'Ethernet'
+        if self.pm_group_metrics[group_name].enabled:
+            for port in self._nni_ports:
+                group_data = self.collect_group_metrics(group_name,
+                                                        port,
+                                                        self.nni_pm_names,
+                                                        self.nni_metrics_config)
+                if group_data is not None:
+                    data.append(group_data)
+
+        for port in self._pon_ports:
+            group_name = 'PON'
+            if self.pm_group_metrics[group_name].enabled:
+                group_data = self.collect_group_metrics(group_name,
+                                                        port,
+                                                        self.pon_pm_names,
+                                                        self.pon_metrics_config)
+                if group_data is not None:
+                    data.append(group_data)
+
+            for onu_id in port.onu_ids:
+                onu = port.onu(onu_id)
+                if onu is not None:
+                    group_name = 'ONU'
+                    if self.pm_group_metrics[group_name].enabled:
+                        group_data = self.collect_group_metrics(group_name,
+                                                                onu,
+                                                                self.onu_pm_names,
+                                                                self.onu_metrics_config)
+                        if group_data is not None:
+                            data.append(group_data)
+
+                    group_name = 'GEM'
+                    if self.pm_group_metrics[group_name].enabled:
+                        for gem in onu.gem_ports:
+                            if not gem.multicast:
+                                group_data = self.collect_group_metrics(group_name,
+                                                                        onu,
+                                                                        self.gem_pm_names,
+                                                                        self.gem_metrics_config)
+                                if group_data is not None:
+                                    data.append(group_data)
+
+                            # TODO: Do any multicast GEM PORT metrics here...
+        return data
diff --git a/python/adapters/extensions/kpi/onu/IntervalMetrics.md b/python/adapters/extensions/kpi/onu/IntervalMetrics.md
new file mode 100644
index 0000000..31169ae
--- /dev/null
+++ b/python/adapters/extensions/kpi/onu/IntervalMetrics.md
@@ -0,0 +1,302 @@
+# ONU OMCI Historical Interval PM Groups
+
+This document outlines the 15-minute interval groups currently supported by the
+**OnuPmIntervalMetrics** _onu_pm_interval_metrics.py_ file.  These groups
+cover a 15-minute interval.
+
+## Performance Interval State Machine
+
+At OpenOMCI startup within an ONU Device Adapter, as soon as the OpenOMCI ME database has
+been declared to be in-sync ONU's ME Database, the Performance Interval State Machine is
+started for the ONU. The first task it performs is to synchronize the ONU's (hardware) time with
+the ONU Device Handler's (Linux container) so that a 15-minute interval is established.
+
+The OpenOMCI PM interval state machine then examines managed elements created by the
+ONU autonomously or created by OpenOMCI in response to a OMCI request from an ONU
+adapter to determine if an appropriate 15-Minute historical PM ME needs to be attached. The
+state machine also registers for notification of any create/delete requests at that
+point so that it can add/remove 15-minute historical PM MEs as services are applied or
+removed. 
+
+Before adding a 15-minute historical PM ME, the ME capabilities of the ONU is
+examined to insure that it can support that particular ME. This is important as the
+Ethernet Frame historical intervals are actually supported by up to 4 different MEs
+reporting the basically the same data. This is detailed below in the _Ethernet Frame
+Performance Monitoring MEs_ section.
+
+## Timezone
+
+The ONU will be synchronized to the Linux Container running the ONU Device handler's
+time in UTC upon startup. Not all ONUs have the capability to set their calendar 
+date (month, day, year) to that of the ONU's Device Handler, but it will set the
+clock to that date. For reporting of 15-minute intervals, only an accurate 15-minute
+boundary is really of any great importance.
+
+## Interval Reporting
+
+After the ONU time synchronization request is made, the first reported interval is
+schedule to occur in the next 15-minute boundry.  For example, if the OpenOMCI
+state machine synchronizes the ONU's time at 
+
+## Common Elements for All Reported MEs
+
+In addition to counter elements (attributes) reported in each ME, every reported 
+historical interval report the following Elements as context values in the KPI
+Event metadata field.  Each value is reported as a _string_ per the Protobuf structure 
+but are actually integer/floats.
+
+| Label               | Type         | Description |
+| ------------------: | :----------: | :---------- |
+| class_id            | int, 16-bits | The ME Class ID of the PM Interval ME |
+| entity_id           | int, 16-bits | The OMCI Entity Instance of the particular PM Interval ME |
+| interval_end_time   | int, 8-bits  | Identifies the most recently finished 15 minute. This attribute is set to zero when a synchronize time request is performed by OpenOMCI.  This counter rolls over from 255 to 0 upon saturation. | 
+| interval_start_time | int, 64-bits | The UTC timestamp (seconds since epoch) rounded down to the start time of the specific interval |
+
+# Supported 15-Minute Historical Performance Monitoring MEs
+
+The following 15-minute historical performance monitoring MEs currently supported are detailed
+in the sections below
+
+## Ethernet Frame Performance Monitoring MEs
+
+The OMCI Ethernet PM supported by OpenOMCI includes 4 possible MEs.  These MEs are attached to
+the MAC Bridge Port Configuration MEs for the ONU. For downstream data, the ME is placed on the
+MAC Bridge Port Configuration ME closest to the ANI Port. For upstream data, the ME is placed
+on the MAC Bridge Port Configuration ME closest to the associated UNI.
+
+The OpenOMCI will first attempt to use the Extended Performance Monitoring MEs if they are
+supported by the ONU.  First the 64-bit counter version will be attempted and then the 32-bit
+counters as a fallback. If of the Extended Performance Monitoring MEs are supported, the
+appropriate Upstream or DownStream Monitoring ME will be used.
+
+### ME Information
+
+The table below describes the four Ethernet Frame Performance Monitoring MEs and provides their
+counter width (in bytes) and ME Class ID.
+
+| ME Name                                                     | Class ID | Counter Width |
+| ----------------------------------------------------------: | :------: | :---:   |
+| Ethernet Frame Extended Performance Monitoring64Bit         |   426    |  64-bit |
+| Ethernet Frame Extended Performance Monitoring              |   334    |  32-bit |
+| Ethernet Frame Upstream Performance MonitoringHistoryData   |   322    |  32-bit |
+| Ethernet Frame Downstream Performance MonitoringHistoryData |   321    |  32-bit |
+
+**Metric Group Name**: Ethernet_Bridge_Port_History  
+**Default Collection**: True  
+**Default Interval**:  15 minutes & aligned to wall-clock. Read-Only
+
+### Counter Information
+
+Each of the Ethernet Frame PM MEs contain the following counters
+
+| Attribute Name      | Description |
+| ------------------: | :-----------|
+| drop_events         | The total number of events in which packets were dropped due to a lack of resources. This is not necessarily the number of packets dropped; it is the number of times this event was detected. |
+| octets              | The total number of upstream octets received, including those in bad packets, excluding framing bits, but including FCS. |
+| packets             | The total number of upstream packets received, including bad packets, broadcast packets and multicast packets. |
+| broadcast_packets   | The total number of upstream good packets received that were directed to the broadcast address. This does not include multicast packets. |
+| multicast_packets   | The total number of upstream good packets received that were directed to a multicast address. This does not include broadcast packets. |
+| crc_errored_packets | The total number of upstream packets received that had a length (excluding framing bits, but including FCS octets) of between 64 octets and 1518 octets, inclusive, but had either a bad FCS with an integral number of octets (FCS error) or a bad FCS with a non-integral number of octets (alignment error). |
+| undersize_packets   | The total number of upstream packets received that were less than 64 octets long, but were otherwise well formed (excluding framing bits, but including FCS). |
+| oversize_packets    | The total number of upstream packets received that were longer than 1518 octets (excluding framing bits, but including FCS) and were otherwise well formed. NOTE 2 – If 2 000 byte Ethernet frames are supported, counts in this performance parameter are not necessarily errors. |
+| 64_octets           | The total number of upstream received packets (including bad packets) that were 64 octets long, excluding framing bits but including FCS. |
+| 65_to_127_octets    | The total number of upstream received packets (including bad packets) that were 65..127 octets long, excluding framing bits but including FCS. |
+| 128_to_255_octets   | The total number of upstream packets (including bad packets) received that were 128..255 octets long, excluding framing bits but including FCS. |
+| 256_to_511_octets   | The total number of upstream packets (including bad packets) received that were 256..511 octets long, excluding framing bits but including FCS. |
+| 512_to_1023_octets  | The total number of upstream packets (including bad packets) received that were 512..1 023 octets long, excluding framing bits but including FCS. |
+| 1024_to_1518_octets | The total number of upstream packets (including bad packets) received that were 1024..1518 octets long, excluding framing bits, but including FCS. |
+
+## Ethernet PM Monitoring History Data (Class ID 24)
+
+This managed entity collects some of the performance monitoring data for a physical
+Ethernet interface. Instances of this managed entity are created and deleted by the OLT.
+
+**Metric Group Name**: Ethernet_UNI_History  
+**Default Collection**: True  
+**Default Interval**:  15 minutes & aligned to wall-clock. Read-Only
+
+### Application
+
+For performance monitoring of Ethernet UNI.
+
+### Relationships
+
+An instance of this managed entity is associated with an instance of the physical path
+termination point Ethernet UNI.                 
+
+### Attributes
+All counters are 32-bits wide.
+
+| Attribute Name      | Description |
+| ------------------: | :-----------|
+| fcs_errors                        | This attribute counts frames received on a particular interface that were an integral number of octets in length but failed the frame check sequence (FCS) check. The count is incremented when the MAC service returns the frameCheckError status to the link layer control (LLC) or other MAC user. Received frames for which multiple error conditions are obtained are counted according to the error status presented to the LLC. |
+| excessive_collision_counter       | This attribute counts frames whose transmission failed due to excessive collisions. |
+| late_collision_counter            | This attribute counts the number of times that a collision was detected later than 512 bit times into the transmission of a packet. |
+| frames_too_long                   | This attribute counts received frames that exceeded the maximum permitted frame size. The count is incremented when the MAC service returns the frameTooLong status to the LLC. |
+| buffer_overflows_on_rx            | This attribute counts the number of times that the receive buffer overflowed. |
+| buffer_overflows_on_tx            | This attribute counts the number of times that the transmit buffer overflowed. |
+| single_collision_frame_counter    | This attribute counts successfully transmitted frames whose transmission was delayed by exactly one collision. |
+| multiple_collisions_frame_counter | This attribute counts successfully transmitted frames whose transmission was delayed by more than one collision. |
+| sqe_counter                       | This attribute counts the number of times that the SQE test error message was generated by the PLS sublayer. |
+| deferred_tx_counter               | This attribute counts frames whose first transmission attempt was delayed because the medium was busy. The count does not include frames involved in collisions. |
+| internal_mac_tx_error_counter     | This attribute counts frames whose transmission failed due to an internal MAC sublayer transmit error. |
+| carrier_sense_error_counter       | This attribute counts the number of times that carrier sense was lost or never asserted when attempting to transmit a frame. |
+| alignment_error_counter           | This attribute counts received frames that were not an integral number of octets in length and did not pass the FCS check. |
+| internal_mac_rx_error_counter     | This attribute counts frames whose reception failed due to an internal MAC sublayer receive error. |
+
+## FEC Performance Monitoring History Data (Class ID 312)
+
+This managed entity collects performance monitoring data associated with PON downstream FEC
+counters. Instances of this managed entity are created and deleted by the OLT.
+
+**Metric Group Name**: FEC_History  
+**Default Collection**: True  
+**Default Interval**:  15 minutes & aligned to wall-clock. Read-Only
+
+### Application
+This managed entity collects performance monitoring data associated with PON downstream FEC
+counters.
+
+### Relationships
+An instance of this managed entity is associated with an instance of the ANI-G managed entity.
+
+### Attributes
+
+| Attribute Name           | Counter Width | Description |
+| -----------------------: | :-----: | :-----------|
+| corrected_bytes          | 32-bits | This attribute counts the number of bytes that were corrected by the FEC function. |
+| corrected_code_words     | 32-bits | This attribute counts the code words that were corrected by the FEC function. |
+| uncorrectable_code_words | 32-bits | This attribute counts errored code words that could not be corrected by the FEC function. |
+| total_code_words         | 32-bits | This attribute counts the total received code words. |
+| fec_seconds              | 16-bits | This attribute counts seconds during which there was a forward error correction anomaly. |
+
+
+## GEM Port Network CTP Monitoring History Data (Class ID 341)
+
+This managed entity collects GEM frame performance monitoring data associated with a GEM port
+network CTP. Instances of this managed entity are created and deleted by the OLT.
+
+Note 1: One might expect to find some form of impaired or discarded frame count associated with
+a GEM port. However, the only impairment that might be detected at the GEM frame level would be
+a corrupted GEM frame header. In this case, no part of the header could be considered reliable
+including the port ID. For this reason, there is no impaired or discarded frame count in this ME.
+
+Note 2: This managed entity replaces the GEM port performance history data managed entity and
+is preferred for new implementations.
+
+**Metric Group Name**: GEM_Port_History  
+**Default Collection**: False  
+**Default Interval**:  15 minutes & aligned to wall-clock. Read-Only
+
+### Relationships
+
+An instance of this managed entity is associated with an instance of the GEM port network CTP
+managed entity.                
+
+### Attributes
+
+| Attribute Name            | Counter Width | Description |
+| ------------------------: | :-----: | :-----------|
+| transmitted_gem_frames    | 32-bits | This attribute counts GEM frames transmitted on the monitored GEM port. |
+| received_gem_frames       | 32-bits | This attribute counts GEM frames received correctly on the monitored GEM port. A correctly received GEM frame is one that does not contain uncorrectable errors and has a valid HEC. |
+| received_payload_bytes    | 64-bits | This attribute counts user payload bytes received on the monitored GEM port. |
+| transmitted_payload_bytes | 64-bits | This attribute counts user payload bytes transmitted on the monitored GEM port. |
+| encryption_key_errors     | 32-bits | This attribute is defined in ITU-T G.987 systems only. It counts GEM frames with erroneous encryption key indexes. If the GEM port is not encrypted, this attribute counts any frame with a key index not equal to 0. If the GEM port is encrypted, this attribute counts any frame whose key index specifies a key that is not known to the ONU. |
+
+Note 3: GEM PM ignores idle GEM frames.
+
+Note 4: GEM PM counts each non-idle GEM frame, whether it contains an entire user frame or only
+a fragment of a user frame.
+
+## XgPon TC Performance Monitoring History Data (Class ID 344)
+
+This managed entity collects performance monitoring data associated with the XG-PON
+transmission convergence layer, as defined in ITU-T G.987.3.
+
+**Metric Group Name**: xgPON_TC_History  
+**Default Collection**: False  
+**Default Interval**:  15 minutes & aligned to wall-clock. Read-Only
+
+### Relationships
+An instance of this managed entity is associated with an ANI-G.
+
+### Attributes
+
+All counters are 32-bits wide.
+
+| Attribute Name            | Description |
+| ------------------------: | :-----------|
+| psbd_hec_error_count      | This attribute counts HEC errors in any of the fields of the downstream physical sync block. |
+| xgtc_hec_error_count      | This attribute counts HEC errors detected in the XGTC header. |
+| unknown_profile_count     | This attribute counts the number of grants received whose specified profile was not known to the ONU. |
+| transmitted_xgem_frames   | This attribute counts the number of non-idle XGEM frames transmitted. If an SDU is fragmented, each fragment is an XGEM frame and is counted as such. |
+| fragment_xgem_frames      | This attribute counts the number of XGEM frames that represent fragmented SDUs, as indicated by the LF bit = 0. |
+| xgem_hec_lost_words_count | This attribute counts the number of four-byte words lost because of an XGEM frame HEC error. In general, all XGTC payload following the error is lost, until the next PSBd event. |
+| xgem_key_errors           | This attribute counts the number of downstream XGEM frames received with an invalid key specification. The key may be invalid for several reasons. |
+| xgem_hec_error_count      | This attribute counts the number of instances of an XGEM frame HEC error. |
+
+## XgPon Downstream Performance Monitoring History Data (Class ID 345)
+
+This managed entity collects performance monitoring data associated with the XG-PON
+transmission convergence layer, as defined in ITU-T G.987.3. It collects counters associated with
+downstream PLOAM and OMCI messages.
+
+**Metric Group Name**: xgPON_Downstream_History  
+**Default Collection**: False  
+**Default Interval**:  15 minutes & aligned to wall-clock. Read-Only
+
+### Relationships
+
+An instance of this managed entity is associated with an ANI-G.           
+
+### Attributes
+     
+All counters are 32-bits wide.
+
+| Attribute Name                          | Description |
+| --------------------------------------: | :-----------|
+| ploam_mic_error_count                   | This attribute counts MIC errors detected in downstream PLOAM messages, either directed to this ONU or broadcast to all ONUs. |
+| downstream_ploam_messages_count         | This attribute counts PLOAM messages received, either directed to this ONU or broadcast to all ONUs. |
+| profile_messages_received               | This attribute counts the number of profile messages received, either directed to this ONU or broadcast to all ONUs. |
+| ranging_time_messages_received          | This attribute counts the number of ranging_time messages received, either directed to this ONU or broadcast to all ONUs. |
+| deactivate_onu_id_messages_received     | This attribute counts the number of deactivate_ONU-ID messages received, either directed to this ONU or broadcast to all ONUs. Deactivate_ONU-ID messages do not reset this counter. |
+| disable_serial_number_messages_received | This attribute counts the number of disable_serial_number messages received, whose serial number specified this ONU. |
+| request_registration_messages_received  | This attribute counts the number request_registration messages received. |
+| assign_alloc_id_messages_received       | This attribute counts the number of assign_alloc-ID messages received. |
+| key_control_messages_received           | This attribute counts the number of key_control messages received, either directed to this ONU or broadcast to all ONUs. |
+| sleep_allow_messages_received           | This attribute counts the number of sleep_allow messages received, either directed to this ONU or broadcast to all ONUs. |
+| baseline_omci_messages_received_count   | This attribute counts the number of OMCI messages received in the baseline message format. |
+| extended_omci_messages_received_count   | This attribute counts the number of OMCI messages received in the extended message format. |
+| assign_onu_id_messages_received         | This attribute counts the number of assign_ONU-ID messages received since the last re-boot. |
+| omci_mic_error_count                    | This attribute counts MIC errors detected in OMCI messages directed to this ONU. |
+
+## XgPon Upstream Performance Monitoring History Data (Class ID 346)
+
+This managed entity collects performance monitoring data associated with the XG-PON
+transmission convergence layer, as defined in ITU-T G.987.3. It counts upstream PLOAM
+messages transmitted by the ONU.
+
+**Metric Group Name**: xgPON_Upstream_History  
+**Default Collection**: False  
+**Default Interval**:  15 minutes & aligned to wall-clock. Read-Only
+
+###Relationships
+
+An instance of this managed entity is associated with an ANI-G.          
+
+### Attributes
+
+All counters are 32-bits wide.
+
+| Attribute Name                  | Description |
+| ------------------------------: | :-----------|
+| upstream_ploam_message_count    | This attribute counts PLOAM messages transmitted upstream, excluding acknowledge messages. |
+| serial_number_onu_message_count | This attribute counts Serial_number_ONU PLOAM messages transmitted. |
+| registration_message_count      | This attribute counts registration PLOAM messages transmitted. |
+| key_report_message_count        | This attribute counts key_report PLOAM messages transmitted. |
+| acknowledge_message_count       | This attribute counts acknowledge PLOAM messages transmitted. It includes all forms of acknowledgement, including those transmitted in response to a PLOAM grant when the ONU has nothing to send. |
+| sleep_request_message_count     | This attribute counts sleep_request PLOAM messages transmitted. |
+
+# Remaining Work Items
+
+- The enable/disable of a PM group (CLI/NBI) should control whether or not a PM interval ME is created and collected.
\ No newline at end of file
diff --git a/python/adapters/extensions/kpi/onu/README.md b/python/adapters/extensions/kpi/onu/README.md
new file mode 100644
index 0000000..4b9798e
--- /dev/null
+++ b/python/adapters/extensions/kpi/onu/README.md
@@ -0,0 +1,123 @@
+# ONU PM Metrics
+
+
+**THESE ARE PRELIMINARY METRIC GROUPS**, Work is needed by the VOLTHA community to reach a consensus on the
+actual metrics that will be provided.  **Also**, please read the **Remaining Work Item** sections of each
+README file.
+
+
+This document outlines the non-interval metrics collected for the ONU by the OpenOMCI code.  These
+are primarily collected from one of the many OMCI Managed Entities
+
+## Format on the Kafka bus
+
+The format of the ONU KPI Events is detailed in the [Basic KPI Format (**KpiEvent2**)](../README.md)
+section of this documents parent directory for wire format on the bus. This document primarily provides
+the group metric information for OLT PKIs and associated metadata context information.
+
+**All** metric values reported by the library are reported as *float*s. The context and metric tables
+listed in the sections below report the type as initially collected by the OLT Device Adapters.
+
+#ONU PM Metric Groups
+
+The following sections outline the KPI metrics gathered by OpenOMCI on behalf of the ONU. If an ONU
+does not support a specific metric in a group, it will not report that metric. This is preferred to
+reporting a metric and it always having a value of 0.0 (which could be misleading).
+
+**Note**: Currently all metric groups are collected and reported at one time (only one collection timer)
+and this value is controlled by the VOLTHA shared kpi library's PM_Config default_freq value and will
+be set to _60 seconds_. This single-collection deficiency will be corrected in the near future. 
+
+## ANI Optical KPI Metrics
+
+This group reports the ONU's Optical Power metrics for each PON physical port as reported by the
+OMCI Managed Entity ANI-G (_Class ID #263_).
+
+**Metric Group Name**: PON_Optical 
+**Default Collection**: True  
+**Default Interval**:  15 minutes
+
+**Metadata Context items**
+
+| key     | value   | Notes |
+| :-----: | :------ | :---- |
+| intf_id | integer | Physical device interface port ID for this PON/ANI port |
+
+The port ID is extracted from the lower 8-bits of the ANI-G Managed Entity ID and indicates
+the physical position of the PON interface.
+
+**Metrics**
+
+| key                     | type / size  | Notes |
+| :---------------------: | :----------- | :---- |
+| transmit_power          | int, 16-bits | This attribute reports the current measurement of mean optical launch power. Its value is a 2s complement integer referred to 1 mW (i.e., dBm), with 0.002 dB granularity |
+| receive_power           | int, 16-bits | This attribute reports the current measurement of the total downstream optical signal level. Its value is a 2s complement integer referred to 1 mW (i.e., dBm), with 0.002 dB granularity. |
+
+**NOTE**: The following metrics were also requested for the PON interface in
+[VOL-935](https://jira.opencord.org/browse/VOL-935) but they are not available through
+the OpenOMCI set of Managed Entities. However there are alarms available that relate to
+these items available through the ANI-G ME:
+
+ - ONT Optical module/transceiver temperature
+ - ONT Optical module/transceiver voltage
+ - ONT Laser bias current
+ 
+TR-287 does reference mechanisms to perform OLT and ONU Optical Link monitoring to cover these
+three items but interfaces are not yet available in VOLTHA and retrieval of these values from
+an ONU may be difficult as the only defined interface to retrieve data is OMCI.
+
+## UNI KPI Metrics
+
+This group reports metrics associated with the customer facing UNI port of the ONU
+and is collected from OMCI Physical Path Termination Point Ethernet UNI (_Class ID #11_)
+and the UNI-G (_Class iD #264_).
+
+**Metric Group Name**: UNI_Status
+**Default Collection**: True  
+**Default Interval**:  15 minutes
+
+**Metadata Context items**
+
+| key     | value   | Notes |
+| :-----: | :------ | :---- |
+| intf_id | integer | Physical device interface port ID for this UNI port |
+
+The port ID is extracted from the UNI-G Managed Entity ID and indicates the 
+physical position of the UNI interface.  This ID is implicitly linked to the
+associated PPTP Ethernet UNI ME.
+
+**Metrics**
+
+| key              | type / size | From  | Notes |
+| :--------------: | :---------- | :---- | :---- |
+| ethernet_type    | int, gauge  | PPTP  | This attribute represents the sensed interface type as defined in the table below |
+| oper_status      | boolean     | PPTP  | Link status/Operational Status: Link up (1), Link down (0) |
+| pptp_admin_state | boolean     | PPTP  | Administrative state: Locked/disabled (1), Unlocked/enabled (0) |
+| uni_admin_state  | boolean     | UNI-G | Administrative state: Locked/disabled (1), Unlocked/enabled (0) |
+
+**Sensed Ethernet Type Table**
+
+| value | Rate             | Duplex |
+| ----: | :--------------: | :--- |
+|  0x00 | Unknown          | n/a  |
+|  0x01 | 10BASE-T         | full |
+|  0x02 | 100BASE-T        | full |
+|  0x03 | Gigabit Ethernet | full |
+|  0x04 | 10Gb/s Ethernet  | full |
+|  0x05 | 2.5Gb/s Ethernet | full |
+|  0x06 | 5Gb/s Ethernet   | full |
+|  0x07 | 25Gb/s Ethernet  | full |
+|  0x08 | 40Gb/s  Ethernet | full |
+|  0x11 | 10BASE-T         | half |
+|  0x12 | 100BASE-T        | half |
+|  0x13 | Gigabit Ethernet | half |
+
+# Remaining Work Items
+
+This initial code is only a preliminary work. See the [Remaining Work Items](../README.md)
+section of this document's parent directory for a list of remaining tasks. In addition to these
+work items, the interval statistics [README](./IntervalMetrics.md) may have additional work
+items remaining.
+
+
+TODO: For each group, list if the default is enabled/disabled
\ No newline at end of file
diff --git a/python/adapters/extensions/kpi/onu/__init__.py b/python/adapters/extensions/kpi/onu/__init__.py
new file mode 100644
index 0000000..b0fb0b2
--- /dev/null
+++ b/python/adapters/extensions/kpi/onu/__init__.py
@@ -0,0 +1,13 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
diff --git a/python/adapters/extensions/kpi/onu/onu_omci_pm.py b/python/adapters/extensions/kpi/onu/onu_omci_pm.py
new file mode 100644
index 0000000..c186bbb
--- /dev/null
+++ b/python/adapters/extensions/kpi/onu/onu_omci_pm.py
@@ -0,0 +1,317 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+import arrow
+from voltha.protos.device_pb2 import PmConfig, PmGroupConfig
+from voltha.protos.events_pb2 import MetricInformation, MetricMetaData
+from voltha.extensions.kpi.adapter_pm_metrics import AdapterPmMetrics
+from voltha.extensions.kpi.onu.onu_pm_interval_metrics import OnuPmIntervalMetrics
+from voltha.extensions.omci.omci_entities import UniG
+from voltha.extensions.omci.omci_entities import PptpEthernetUni
+
+
+class OnuOmciPmMetrics(AdapterPmMetrics):
+    """ ONU OMCI related metrics """
+
+    # Metric default settings
+    #
+    #  Frequency values are in 1/10ths of a second
+    #
+    OMCI_DEV_KEY = 'omci-onu-dev'
+    OMCI_CC_GROUP_NAME = 'OMCI_CC'
+    DEFAULT_OMCI_CC_ENABLED = False
+    DEFAULT_OMCI_CC_FREQUENCY = (2 * 60) * 10
+
+    OPTICAL_GROUP_NAME = 'PON_Optical'
+    DEFAULT_OPTICAL_ENABLED = True
+    DEFAULT_OPTICAL_FREQUENCY = (15 * 60 * 10)
+
+    UNI_STATUS_GROUP_NAME = 'UNI_Status'
+    DEFAULT_UNI_STATUS_ENABLED = True
+    DEFAULT_UNI_STATUS_FREQUENCY = (15 * 60 * 10)
+
+    def __init__(self, adapter_agent, device_id, logical_device_id,
+                 grouped=False, freq_override=False, **kwargs):
+        """
+        Initializer for shared ONU Device Adapter OMCI CC PM metrics
+
+        :param adapter_agent: (AdapterAgent) Adapter agent for the device
+        :param device_id: (str) Device ID
+        :param logical_device_id: (str) VOLTHA Logical Device ID
+        :param grouped: (bool) Flag indicating if statistics are managed as a group
+        :param freq_override: (bool) Flag indicating if frequency collection can be specified
+                                     on a per group basis
+        :param kwargs: (dict) Device Adapter specific values. For an ONU Device adapter, the
+                              expected key-value pairs are listed below. If not provided, the
+                              associated PM statistics are not gathered:
+
+                              'omci-onu-dev': Reference to the OMCI OnuDeviceEtnry object for
+                                         retrieval of OpenOMCI Communications channel statistics
+                                         and retrieval of polled statistics.
+        """
+        super(OnuOmciPmMetrics, self).__init__(adapter_agent, device_id, logical_device_id,
+                                               grouped=grouped, freq_override=freq_override,
+                                               **kwargs)
+
+        self._omci_onu_device = kwargs.pop(OnuOmciPmMetrics.OMCI_DEV_KEY, None)
+        self._omci_cc = self._omci_onu_device.omci_cc if self._omci_onu_device is not None else None
+
+        self.omci_cc_pm_names = {
+            ('tx_frames', PmConfig.COUNTER),
+            ('tx_errors', PmConfig.COUNTER),
+            ('rx_frames', PmConfig.COUNTER),
+            ('rx_unknown_tid', PmConfig.COUNTER),
+            ('rx_onu_frames', PmConfig.COUNTER),        # Rx ONU autonomous messages
+            ('rx_unknown_me', PmConfig.COUNTER),        # Managed Entities without a decode definition
+            ('rx_timeouts', PmConfig.COUNTER),
+            ('rx_late', PmConfig.COUNTER),
+            ('consecutive_errors', PmConfig.COUNTER),
+            ('reply_min', PmConfig.GAUGE),      # Milliseconds
+            ('reply_max', PmConfig.GAUGE),      # Milliseconds
+            ('reply_average', PmConfig.GAUGE),  # Milliseconds
+            ('hp_tx_queue_len', PmConfig.GAUGE),
+            ('lp_tx_queue_len', PmConfig.GAUGE),
+            ('max_hp_tx_queue', PmConfig.GAUGE),
+            ('max_lp_tx_queue', PmConfig.GAUGE),
+        }
+        self.omci_cc_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                       for (m, t) in self.omci_cc_pm_names}
+
+        self.omci_optical_pm_names = {
+            ('intf_id', PmConfig.CONTEXT),
+
+            ('transmit_power', PmConfig.GAUGE),
+            ('receive_power', PmConfig.GAUGE),
+        }
+        self.omci_optical_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                            for (m, t) in self.omci_optical_pm_names}
+
+        self.omci_uni_pm_names = {
+            ('intf_id', PmConfig.CONTEXT),
+
+            ('ethernet_type', PmConfig.GAUGE),     # PPTP Ethernet ME
+            ('oper_status', PmConfig.GAUGE),       # PPTP Ethernet ME
+            ('pptp_admin_state', PmConfig.GAUGE),  # PPTP Ethernet ME
+            ('uni_admin_state', PmConfig.GAUGE),   # UNI-G ME
+        }
+        self.omci_uni_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                        for (m, t) in self.omci_uni_pm_names}
+
+        self.openomci_interval_pm = OnuPmIntervalMetrics(adapter_agent, device_id, logical_device_id)
+
+    def update(self, pm_config):
+        # TODO: Test frequency override capability for a particular group
+        if self.default_freq != pm_config.default_freq:
+            # Update the callback to the new frequency.
+            self.default_freq = pm_config.default_freq
+            self.lc.stop()
+            self.lc.start(interval=self.default_freq / 10)
+
+        if pm_config.grouped:
+            for group in pm_config.groups:
+                group_config = self.pm_group_metrics.get(group.group_name)
+                if group_config is not None:
+                    group_config.enabled = group.enabled
+        else:
+            msg = 'There are on independent OMCI metrics, only group metrics at this time'
+            raise NotImplemented(msg)
+
+        self.openomci_interval_pm.update(pm_config)
+
+    def make_proto(self, pm_config=None):
+        assert pm_config is not None
+
+        # OMCI only supports grouped metrics
+        if self._omci_onu_device is None or not self.grouped:
+            return pm_config
+
+        pm_omci_cc_stats = PmGroupConfig(group_name=OnuOmciPmMetrics.OMCI_CC_GROUP_NAME,
+                                         group_freq=OnuOmciPmMetrics.DEFAULT_OMCI_CC_FREQUENCY,
+                                         enabled=OnuOmciPmMetrics.DEFAULT_OMCI_CC_ENABLED)
+        self.pm_group_metrics[pm_omci_cc_stats.group_name] = pm_omci_cc_stats
+
+        pm_omci_optical_stats = PmGroupConfig(group_name=OnuOmciPmMetrics.OPTICAL_GROUP_NAME,
+                                              group_freq=OnuOmciPmMetrics.DEFAULT_OPTICAL_FREQUENCY,
+                                              enabled=OnuOmciPmMetrics.DEFAULT_OPTICAL_ENABLED)
+        self.pm_group_metrics[pm_omci_optical_stats.group_name] = pm_omci_optical_stats
+
+        pm_omci_uni_stats = PmGroupConfig(group_name=OnuOmciPmMetrics.UNI_STATUS_GROUP_NAME,
+                                          group_freq=OnuOmciPmMetrics.DEFAULT_UNI_STATUS_FREQUENCY,
+                                          enabled=OnuOmciPmMetrics.DEFAULT_UNI_STATUS_ENABLED)
+        self.pm_group_metrics[pm_omci_uni_stats.group_name] = pm_omci_uni_stats
+
+        stats_and_config = [(pm_omci_cc_stats, self.omci_cc_metrics_config),
+                            (pm_omci_optical_stats, self.omci_optical_metrics_config),
+                            (pm_omci_uni_stats, self.omci_cc_metrics_config)]
+
+        for stats, config in stats_and_config:
+            for m in sorted(config):
+                pm = config[m]
+                stats.metrics.extend([PmConfig(name=pm.name,
+                                               type=pm.type,
+                                               enabled=pm.enabled)])
+            pm_config.groups.extend([stats])
+
+        # Also create OMCI Interval PM configs
+        return self.openomci_interval_pm.make_proto(pm_config)
+
+    def collect_metrics(self, data=None):
+        """
+        Collect metrics for this adapter.
+
+        The data collected (or passed in) is a list of pairs/tuples.  Each
+        pair is composed of a MetricMetaData metadata-portion and list of MetricValuePairs
+        that contains a single individual metric or list of metrics if this is a
+        group metric.
+
+        This method is called for each adapter at a fixed frequency.
+        TODO: Currently all group metrics are collected on a single timer tick.
+              This needs to be fixed as independent group or instance collection is
+              desirable.
+
+        :param data: (list) Existing list of collected metrics (MetricInformation).
+                            This is provided to allow derived classes to call into
+                            further encapsulated classes.
+
+        :return: (list) metadata and metrics pairs - see description above
+        """
+        if data is None:
+            data = list()
+
+        # Note: Interval PM is collection done autonomously, not through this method
+
+        if self._omci_cc is not None:
+            group_name = OnuOmciPmMetrics.OMCI_CC_GROUP_NAME
+            if self.pm_group_metrics[group_name].enabled:
+                group_data = self.collect_group_metrics(group_name,
+                                                        self._omci_cc,
+                                                        self.omci_cc_pm_names,
+                                                        self.omci_cc_metrics_config)
+                if group_data is not None:
+                    data.append(group_data)
+
+            # Optical and UNI data is collected on a per-port basis
+            data.extend(self.collect_optical_metrics())
+            data.extend(self.collect_uni_status_metrics())
+
+        return data
+
+    def collect_optical_metrics(self):
+        """
+        Collect the metrics for optical information from all ANI/PONs
+
+        :return: (list) collected metrics (MetricInformation)
+        """
+        now = self._omci_onu_device.timestamp
+
+        group_name = OnuOmciPmMetrics.OPTICAL_GROUP_NAME
+        if now is None or not self.pm_group_metrics[group_name].enabled:
+            return []
+
+        # Scan all ANI-G ports
+        ani_g_entities = self._omci_onu_device.configuration.ani_g_entities
+        ani_g_entities_ids = ani_g_entities.keys() if ani_g_entities is not None else None
+        metrics_info = []
+
+        if ani_g_entities_ids is not None and len(ani_g_entities_ids):
+            from voltha.extensions.omci.omci_entities import AniG
+            ani_g_items = ['optical_signal_level', 'transmit_optical_level']
+
+            for entity_id in ani_g_entities_ids:
+                metrics = dict()
+                data = self._omci_onu_device.query_mib(class_id=AniG.class_id,
+                                                       instance_id=entity_id,
+                                                       attributes=ani_g_items)
+                if len(data):
+                    if 'optical_signal_level' in data:
+                        metrics['receive_power'] = data['optical_signal_level']
+
+                    if 'transmit_optical_level' in data:
+                        metrics['transmit_power'] = data['transmit_optical_level']
+
+                if len(metrics):
+                    metric_data = MetricInformation(metadata=MetricMetaData(title=group_name,
+                                                                            ts=now,
+                                                                            logical_device_id=self.logical_device_id,
+                                                                            serial_no=self.serial_number,
+                                                                            device_id=self.device_id,
+                                                                            context={
+                                                                                'intf_id': str(entity_id)
+                                                                            }),
+                                                    metrics=metrics)
+                    metrics_info.append(metric_data)
+
+        return metrics_info
+
+    def collect_uni_status_metrics(self):
+        """
+        Collect the metrics for optical information from all ANI/PONs
+
+        :return: (list) collected metrics (MetricInformation)
+        """
+        now = self._omci_onu_device.timestamp
+
+        group_name = OnuOmciPmMetrics.UNI_STATUS_GROUP_NAME
+        if now is None or not self.pm_group_metrics[group_name].enabled:
+            return []
+
+        # Scan all UNI-G and PPTP ports
+        uni_g_entities = self._omci_onu_device.configuration.uni_g_entities
+        uni_g_entities_ids = uni_g_entities.keys() if uni_g_entities is not None else None
+        pptp_entities = self._omci_onu_device.configuration.pptp_entities
+        pptp_entities_ids = pptp_entities.keys() if pptp_entities is not None else None
+
+        metrics_info = []
+
+        if uni_g_entities_ids and pptp_entities_ids and len(uni_g_entities_ids) and \
+                len(uni_g_entities_ids) <= len(pptp_entities_ids):
+
+            uni_g_items = ['administrative_state']
+            pptp_items = ['administrative_state', 'operational_state', 'sensed_type']
+
+            for entity_id in pptp_entities_ids:
+                metrics = dict()
+                data = self._omci_onu_device.query_mib(class_id=UniG.class_id,
+                                                       instance_id=entity_id,
+                                                       attributes=uni_g_items)
+                if len(data):
+                    if 'administrative_state' in data:
+                        metrics['uni_admin_state'] = data['administrative_state']
+
+                data = self._omci_onu_device.query_mib(class_id=PptpEthernetUni.class_id,
+                                                       instance_id=entity_id,
+                                                       attributes=pptp_items)
+                if len(data):
+                    if 'administrative_state' in data:
+                        metrics['pptp_admin_state'] = data['administrative_state']
+
+                    if 'operational_state' in data:
+                        metrics['oper_status'] = data['operational_state']
+
+                    if 'sensed_type' in data:
+                        metrics['ethernet_type'] = data['sensed_type']
+
+                if len(metrics):
+                    metric_data = MetricInformation(metadata=MetricMetaData(title=group_name,
+                                                                            ts=now,
+                                                                            logical_device_id=self.logical_device_id,
+                                                                            serial_no=self.serial_number,
+                                                                            device_id=self.device_id,
+                                                                            context={
+                                                                                'intf_id': str(entity_id & 0xFF)
+                                                                            }),
+                                                    metrics=metrics)
+                    metrics_info.append(metric_data)
+
+        return metrics_info
diff --git a/python/adapters/extensions/kpi/onu/onu_pm_interval_metrics.py b/python/adapters/extensions/kpi/onu/onu_pm_interval_metrics.py
new file mode 100644
index 0000000..5629e7c
--- /dev/null
+++ b/python/adapters/extensions/kpi/onu/onu_pm_interval_metrics.py
@@ -0,0 +1,383 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+import arrow
+from voltha.protos.device_pb2 import PmConfig, PmGroupConfig
+from voltha.protos.events_pb2 import KpiEvent2, MetricInformation, MetricMetaData, KpiEventType
+from voltha.extensions.kpi.adapter_pm_metrics import AdapterPmMetrics
+from voltha.extensions.omci.omci_entities import \
+    EthernetFrameUpstreamPerformanceMonitoringHistoryData, \
+    EthernetFrameDownstreamPerformanceMonitoringHistoryData, \
+    EthernetFrameExtendedPerformanceMonitoring, \
+    EthernetFrameExtendedPerformanceMonitoring64Bit, \
+    EthernetPMMonitoringHistoryData, FecPerformanceMonitoringHistoryData, \
+    GemPortNetworkCtpMonitoringHistoryData, XgPonTcPerformanceMonitoringHistoryData, \
+    XgPonDownstreamPerformanceMonitoringHistoryData, \
+    XgPonUpstreamPerformanceMonitoringHistoryData
+
+
+class OnuPmIntervalMetrics(AdapterPmMetrics):
+    """
+    ONU OMCI PM Interval metrics
+
+    These differ from other PM Metrics as they are collected and generated as a
+    result of receipt of OMCI get responses on various PM History MEs.  They are
+    also always managed as a group with a fixed frequency of 15 minutes.
+    """
+    ME_ID_INFO = {
+        EthernetFrameUpstreamPerformanceMonitoringHistoryData.class_id: 'Ethernet_Bridge_Port_History',
+        EthernetFrameDownstreamPerformanceMonitoringHistoryData.class_id: 'Ethernet_Bridge_Port_History',
+        EthernetFrameExtendedPerformanceMonitoring.class_id: 'Ethernet_Bridge_Port_History',
+        EthernetFrameExtendedPerformanceMonitoring64Bit.class_id: 'Ethernet_Bridge_Port_History',
+        EthernetPMMonitoringHistoryData.class_id: 'Ethernet_UNI_History',
+        FecPerformanceMonitoringHistoryData.class_id: 'FEC_History',
+        GemPortNetworkCtpMonitoringHistoryData.class_id: 'GEM_Port_History',
+        XgPonTcPerformanceMonitoringHistoryData.class_id: 'xgPON_TC_History',
+        XgPonDownstreamPerformanceMonitoringHistoryData.class_id: 'xgPON_Downstream_History',
+        XgPonUpstreamPerformanceMonitoringHistoryData.class_id: 'xgPON_Upstream_History'
+    }
+    ETHERNET_BRIDGE_HISTORY_ENABLED = True
+    ETHERNET_UNI_HISTORY_ENABLED = True
+    FEC_HISTORY_ENABLED = True
+    GEM_PORT_HISTORY_ENABLED = False
+    TRANS_CONV_HISTORY_ENABLED = False
+    XGPON_DOWNSTREAM_HISTORY = False
+    XGPON_UPSTREAM_HISTORY = False
+
+    def __init__(self, adapter_agent, device_id, logical_device_id, **kwargs):
+        super(OnuPmIntervalMetrics, self).__init__(adapter_agent, device_id, logical_device_id,
+                                                   grouped=True, freq_override=False,
+                                                   **kwargs)
+        ethernet_bridge_history = {
+            ('class_id', PmConfig.CONTEXT),
+            ('entity_id', PmConfig.CONTEXT),
+            ("interval_end_time", PmConfig.CONTEXT),
+            ('parent_class_id', PmConfig.CONTEXT),
+            ('parent_entity_id', PmConfig.CONTEXT),
+            ('upstream', PmConfig.CONTEXT),
+
+            ("drop_events", PmConfig.COUNTER),
+            ("octets", PmConfig.COUNTER),
+            ("packets", PmConfig.COUNTER),
+            ("broadcast_packets", PmConfig.COUNTER),
+            ("multicast_packets", PmConfig.COUNTER),
+            ("crc_errored_packets", PmConfig.COUNTER),
+            ("undersize_packets", PmConfig.COUNTER),
+            ("oversize_packets", PmConfig.COUNTER),
+            ("64_octets", PmConfig.COUNTER),
+            ("65_to_127_octets", PmConfig.COUNTER),
+            ("128_to_255_octets", PmConfig.COUNTER),
+            ("256_to_511_octets", PmConfig.COUNTER),
+            ("512_to_1023_octets", PmConfig.COUNTER),
+            ("1024_to_1518_octets", PmConfig.COUNTER)
+        }
+        self._ethernet_bridge_history_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                                for (m, t) in ethernet_bridge_history}
+
+        ethernet_uni_history = {   # Ethernet History Data (Class ID 24)
+            ('class_id', PmConfig.CONTEXT),
+            ('entity_id', PmConfig.CONTEXT),
+            ("interval_end_time", PmConfig.CONTEXT),
+
+            ("fcs_errors", PmConfig.COUNTER),
+            ("excessive_collision_counter", PmConfig.COUNTER),
+            ("late_collision_counter", PmConfig.COUNTER),
+            ("frames_too_long", PmConfig.COUNTER),
+            ("buffer_overflows_on_rx", PmConfig.COUNTER),
+            ("buffer_overflows_on_tx", PmConfig.COUNTER),
+            ("single_collision_frame_counter", PmConfig.COUNTER),
+            ("multiple_collisions_frame_counter", PmConfig.COUNTER),
+            ("sqe_counter", PmConfig.COUNTER),
+            ("deferred_tx_counter", PmConfig.COUNTER),
+            ("internal_mac_tx_error_counter", PmConfig.COUNTER),
+            ("carrier_sense_error_counter", PmConfig.COUNTER),
+            ("alignment_error_counter", PmConfig.COUNTER),
+            ("internal_mac_rx_error_counter", PmConfig.COUNTER),
+        }
+        self._ethernet_uni_history_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                             for (m, t) in ethernet_uni_history}
+
+        fec_history = {   # FEC History Data (Class ID 312)
+            ('class_id', PmConfig.CONTEXT),
+            ('entity_id', PmConfig.CONTEXT),
+            ("interval_end_time", PmConfig.CONTEXT),
+
+            ("corrected_bytes", PmConfig.COUNTER),
+            ("corrected_code_words", PmConfig.COUNTER),
+            ("uncorrectable_code_words", PmConfig.COUNTER),
+            ("total_code_words", PmConfig.COUNTER),
+            ("fec_seconds", PmConfig.COUNTER),
+        }
+        self._fec_history_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                    for (m, t) in fec_history}
+
+        gem_port_history = {  # GEM Port Network CTP History Data (Class ID 341)
+            ('class_id', PmConfig.CONTEXT),
+            ('entity_id', PmConfig.CONTEXT),
+            ("interval_end_time", PmConfig.CONTEXT),
+
+            ("transmitted_gem_frames", PmConfig.COUNTER),
+            ("received_gem_frames", PmConfig.COUNTER),
+            ("received_payload_bytes", PmConfig.COUNTER),
+            ("transmitted_payload_bytes", PmConfig.COUNTER),
+            ("encryption_key_errors", PmConfig.COUNTER),
+        }
+        self._gem_port_history_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                         for (m, t) in gem_port_history}
+
+        xgpon_tc_history = {  # XgPon TC History Data (Class ID 344)
+            ('class_id', PmConfig.CONTEXT),
+            ('entity_id', PmConfig.CONTEXT),
+            ("interval_end_time", PmConfig.CONTEXT),
+
+            ("psbd_hec_error_count", PmConfig.COUNTER),
+            ("xgtc_hec_error_count", PmConfig.COUNTER),
+            ("unknown_profile_count", PmConfig.COUNTER),
+            ("transmitted_xgem_frames", PmConfig.COUNTER),
+            ("fragment_xgem_frames", PmConfig.COUNTER),
+            ("xgem_hec_lost_words_count", PmConfig.COUNTER),
+            ("xgem_key_errors", PmConfig.COUNTER),
+            ("xgem_hec_error_count", PmConfig.COUNTER),
+        }
+        self._xgpon_tc_history_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                         for (m, t) in xgpon_tc_history}
+
+        xgpon_downstream_history = {  # XgPon Downstream History Data (Class ID 345)
+            ('class_id', PmConfig.CONTEXT),
+            ('entity_id', PmConfig.CONTEXT),
+            ("interval_end_time", PmConfig.CONTEXT),
+
+            ("ploam_mic_error_count", PmConfig.COUNTER),
+            ("downstream_ploam_messages_count", PmConfig.COUNTER),
+            ("profile_messages_received", PmConfig.COUNTER),
+            ("ranging_time_messages_received", PmConfig.COUNTER),
+            ("deactivate_onu_id_messages_received", PmConfig.COUNTER),
+            ("disable_serial_number_messages_received", PmConfig.COUNTER),
+            ("request_registration_messages_received", PmConfig.COUNTER),
+            ("assign_alloc_id_messages_received", PmConfig.COUNTER),
+            ("key_control_messages_received", PmConfig.COUNTER),
+            ("sleep_allow_messages_received", PmConfig.COUNTER),
+            ("baseline_omci_messages_received_count", PmConfig.COUNTER),
+            ("extended_omci_messages_received_count", PmConfig.COUNTER),
+            ("assign_onu_id_messages_received", PmConfig.COUNTER),
+            ("omci_mic_error_count", PmConfig.COUNTER),
+        }
+        self._xgpon_downstream_history_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                                 for (m, t) in xgpon_downstream_history}
+
+        xgpon_upstream_history = {  # XgPon Upstream History Data (Class ID 346)
+            ('class_id', PmConfig.CONTEXT),
+            ('entity_id', PmConfig.CONTEXT),
+            ("interval_end_time", PmConfig.CONTEXT),
+
+            ("upstream_ploam_message_count", PmConfig.COUNTER),
+            ("serial_number_onu_message_count", PmConfig.COUNTER),
+            ("registration_message_count", PmConfig.COUNTER),
+            ("key_report_message_count", PmConfig.COUNTER),
+            ("acknowledge_message_count", PmConfig.COUNTER),
+            ("sleep_request_message_count", PmConfig.COUNTER),
+        }
+        self._xgpon_upstream_history_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                               for (m, t) in xgpon_upstream_history}
+        self._configs = {
+            EthernetFrameUpstreamPerformanceMonitoringHistoryData.class_id: self._ethernet_bridge_history_config,
+            EthernetFrameDownstreamPerformanceMonitoringHistoryData.class_id: self._ethernet_bridge_history_config,
+            EthernetFrameExtendedPerformanceMonitoring.class_id: self._ethernet_bridge_history_config,
+            EthernetFrameExtendedPerformanceMonitoring64Bit.class_id: self._ethernet_bridge_history_config,
+            EthernetPMMonitoringHistoryData.class_id: self._ethernet_uni_history_config,
+            FecPerformanceMonitoringHistoryData.class_id: self._fec_history_config,
+            GemPortNetworkCtpMonitoringHistoryData.class_id: self._gem_port_history_config,
+            XgPonTcPerformanceMonitoringHistoryData.class_id: self._xgpon_tc_history_config,
+            XgPonDownstreamPerformanceMonitoringHistoryData.class_id: self._xgpon_downstream_history_config,
+            XgPonUpstreamPerformanceMonitoringHistoryData.class_id: self._xgpon_upstream_history_config
+        }
+
+    def update(self, pm_config):
+        """
+        Update the PM Configuration.
+
+        For historical PM Intervals, the frequency always zero since the actual collection
+        and publishing is provided by the OpenOMCI library
+
+        :param pm_config:
+        """
+        self.log.debug('update')
+
+        try:
+            if pm_config.grouped:
+                for group in pm_config.groups:
+                    group_config = self.pm_group_metrics.get(group.group_name)
+                    if group_config is not None and group_config.enabled != group.enabled:
+                        group_config.enabled = group.enabled
+                        # TODO: For OMCI PM Metrics, tie this into add/remove of the PM Interval ME itself
+            else:
+                msg = 'There are on independent OMCI Interval metrics, only group metrics at this time'
+                raise NotImplemented(msg)
+
+        except Exception as e:
+            self.log.exception('update-failure', e=e)
+            raise
+
+    def make_proto(self, pm_config=None):
+        """
+        From the PM Configurations defined in this class's initializer, create
+        the PMConfigs protobuf message that defines our PM configuration and
+        data.
+
+        All ONU PM Interval metrics are grouped metrics that are generated autonmouslly
+        from the OpenOMCI Performance Intervals state machine.
+
+        :param pm_config (PMConfigs) PM Configuration message to add OpenOMCI config items too
+        :return: (PmConfigs) PM Configuration Protobuf message
+        """
+        assert pm_config is not None
+
+        pm_ethernet_bridge_history = PmGroupConfig(group_name=OnuPmIntervalMetrics.ME_ID_INFO[EthernetFrameUpstreamPerformanceMonitoringHistoryData.class_id],
+                                                   group_freq=0,
+                                                   enabled=OnuPmIntervalMetrics.ETHERNET_BRIDGE_HISTORY_ENABLED)
+        self.pm_group_metrics[pm_ethernet_bridge_history.group_name] = pm_ethernet_bridge_history
+
+        for m in sorted(self._ethernet_bridge_history_config):
+            pm = self._ethernet_bridge_history_config[m]
+            pm_ethernet_bridge_history.metrics.extend([PmConfig(name=pm.name,
+                                                                type=pm.type,
+                                                                enabled=pm.enabled)])
+
+        pm_ethernet_uni_history = PmGroupConfig(group_name=OnuPmIntervalMetrics.ME_ID_INFO[EthernetPMMonitoringHistoryData.class_id],
+                                                group_freq=0,
+                                                enabled=OnuPmIntervalMetrics.ETHERNET_UNI_HISTORY_ENABLED)
+        self.pm_group_metrics[pm_ethernet_uni_history.group_name] = pm_ethernet_uni_history
+
+        for m in sorted(self._ethernet_uni_history_config):
+            pm = self._ethernet_uni_history_config[m]
+            pm_ethernet_uni_history.metrics.extend([PmConfig(name=pm.name,
+                                                             type=pm.type,
+                                                             enabled=pm.enabled)])
+
+        pm_fec_history = PmGroupConfig(group_name=OnuPmIntervalMetrics.ME_ID_INFO[FecPerformanceMonitoringHistoryData.class_id],
+                                       group_freq=0,
+                                       enabled=OnuPmIntervalMetrics.FEC_HISTORY_ENABLED)
+        self.pm_group_metrics[pm_fec_history.group_name] = pm_fec_history
+
+        for m in sorted(self._fec_history_config):
+            pm = self._fec_history_config[m]
+            pm_fec_history.metrics.extend([PmConfig(name=pm.name,
+                                                    type=pm.type,
+                                                    enabled=pm.enabled)])
+
+        pm_gem_port_history = PmGroupConfig(group_name=OnuPmIntervalMetrics.ME_ID_INFO[GemPortNetworkCtpMonitoringHistoryData.class_id],
+                                            group_freq=0,
+                                            enabled=OnuPmIntervalMetrics.GEM_PORT_HISTORY_ENABLED)
+        self.pm_group_metrics[pm_gem_port_history.group_name] = pm_gem_port_history
+
+        for m in sorted(self._gem_port_history_config):
+            pm = self._gem_port_history_config[m]
+            pm_gem_port_history.metrics.extend([PmConfig(name=pm.name,
+                                                         type=pm.type,
+                                                         enabled=pm.enabled)])
+
+        pm_xgpon_tc_history = PmGroupConfig(group_name=OnuPmIntervalMetrics.ME_ID_INFO[XgPonTcPerformanceMonitoringHistoryData.class_id],
+                                            group_freq=0,
+                                            enabled=OnuPmIntervalMetrics.TRANS_CONV_HISTORY_ENABLED)
+        self.pm_group_metrics[pm_xgpon_tc_history.group_name] = pm_xgpon_tc_history
+
+        for m in sorted(self._xgpon_tc_history_config):
+            pm = self._xgpon_tc_history_config[m]
+            pm_xgpon_tc_history.metrics.extend([PmConfig(name=pm.name,
+                                                         type=pm.type,
+                                                         enabled=pm.enabled)])
+
+        pm_xgpon_downstream_history = PmGroupConfig(group_name=OnuPmIntervalMetrics.ME_ID_INFO[XgPonDownstreamPerformanceMonitoringHistoryData.class_id],
+                                                    group_freq=0,
+                                                    enabled=OnuPmIntervalMetrics.XGPON_DOWNSTREAM_HISTORY)
+        self.pm_group_metrics[pm_xgpon_downstream_history.group_name] = pm_xgpon_downstream_history
+
+        for m in sorted(self._xgpon_downstream_history_config):
+            pm = self._xgpon_downstream_history_config[m]
+            pm_xgpon_downstream_history.metrics.extend([PmConfig(name=pm.name,
+                                                                 type=pm.type,
+                                                                 enabled=pm.enabled)])
+
+        pm_xgpon_upstream_history = PmGroupConfig(group_name=OnuPmIntervalMetrics.ME_ID_INFO[XgPonUpstreamPerformanceMonitoringHistoryData.class_id],
+                                                  group_freq=0,
+                                                  enabled=OnuPmIntervalMetrics.XGPON_UPSTREAM_HISTORY)
+        self.pm_group_metrics[pm_xgpon_upstream_history.group_name] = pm_xgpon_upstream_history
+
+        for m in sorted(self._xgpon_upstream_history_config):
+            pm = self._xgpon_upstream_history_config[m]
+            pm_xgpon_upstream_history.metrics.extend([PmConfig(name=pm.name,
+                                                               type=pm.type,
+                                                               enabled=pm.enabled)])
+
+        pm_config.groups.extend([stats for stats in self.pm_group_metrics.itervalues()])
+
+        return pm_config
+
+    def publish_metrics(self, interval_data):
+        """
+        Collect the metrics for this ONU PM Interval
+
+        :param interval_data: (dict) PM interval dictionary with structure of
+                    {
+                        'class_id': self._class_id,
+                        'entity_id': self._entity_id,
+                        'me_name': self._entity.__name__,   # Mostly for debugging...
+                        'interval_utc_time': None,
+                        # Counters added here as they are retrieved
+                    }
+
+        :return: (dict) Key/Value of metric data
+        """
+        self.log.debug('publish-metrics')
+
+        try:
+            # Locate config
+            now = arrow.utcnow()
+            class_id = interval_data['class_id']
+            config = self._configs.get(class_id)
+            group = self.pm_group_metrics.get(OnuPmIntervalMetrics.ME_ID_INFO.get(class_id, ''))
+
+            if config is not None and group is not None and group.enabled:
+                # Extract only the metrics we need to publish
+                metrics = dict()
+                context = {
+                    'interval_start_time': str(now.replace(minute=int(now.minute / 15) * 15,
+                                                           second=0,
+                                                           microsecond=0).timestamp)
+                }
+                for metric, config_item in config.items():
+                    if config_item.type == PmConfig.CONTEXT and metric in interval_data:
+                        context[metric] = str(interval_data[metric])
+
+                    elif (config_item.type in (PmConfig.COUNTER, PmConfig.GAUGE, PmConfig.STATE) and
+                          metric in interval_data and
+                          config_item.enabled):
+                        metrics[metric] = interval_data[metric]
+
+                if len(metrics):
+                    metadata = MetricMetaData(title=group.group_name,
+                                              ts=now.float_timestamp,
+                                              logical_device_id=self.logical_device_id,
+                                              serial_no=self.serial_number,
+                                              device_id=self.device_id,
+                                              context=context)
+                    slice_data = [MetricInformation(metadata=metadata, metrics=metrics)]
+
+                    kpi_event = KpiEvent2(type=KpiEventType.slice,
+                                          ts=now.float_timestamp,
+                                          slice_data=slice_data)
+                    self.adapter_agent.submit_kpis(kpi_event)
+
+        except Exception as e:
+            self.log.exception('failed-to-submit-kpis', e=e)
diff --git a/python/adapters/extensions/kpi/onu/onu_pm_metrics.py b/python/adapters/extensions/kpi/onu/onu_pm_metrics.py
new file mode 100644
index 0000000..c94136a
--- /dev/null
+++ b/python/adapters/extensions/kpi/onu/onu_pm_metrics.py
@@ -0,0 +1,171 @@
+# Copyright 2017-present Open Networking Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from voltha.protos.device_pb2 import PmConfig, PmConfigs, PmGroupConfig
+from voltha.extensions.kpi.adapter_pm_metrics import AdapterPmMetrics
+from voltha.extensions.kpi.onu.onu_omci_pm import OnuOmciPmMetrics
+
+
+class OnuPmMetrics(AdapterPmMetrics):
+    """
+    Shared ONU Device Adapter PM Metrics Manager
+
+    This class specifically addresses ONU general PM (health, ...) area
+    specific PM (OMCI, PON, UNI) is supported in encapsulated classes accessible
+    from this object
+    """
+
+    # Metric default settings
+    DEFAULT_HEARTBEAT_ENABLED = False
+    DEFAULT_HEARTBEAT_FREQUENCY = 1200  # 1/10ths of a second
+    #
+    # Currently only a single KPI metrics collection occurs (individual group
+    # frequency not supported). The next value defines this single frequency until
+    # the KPI shared library supports individual collection.
+    DEFAULT_ONU_COLLECTION_FREQUENCY = 60 * 10      # 1 minute
+
+    def __init__(self, adapter_agent, device_id, logical_device_id,
+                 grouped=False, freq_override=False, **kwargs):
+        """
+        Initializer for shared ONU Device Adapter PM metrics
+
+        :param adapter_agent: (AdapterAgent) Adapter agent for the device
+        :param device_id: (str) Device ID
+        :param logical_device_id: (str) VOLTHA Logical Device ID
+        :param grouped: (bool) Flag indicating if statistics are managed as a group
+        :param freq_override: (bool) Flag indicating if frequency collection can be specified
+                                     on a per group basis
+        :param kwargs: (dict) Device Adapter specific values. For an ONU Device adapter, the
+                              expected key-value pairs are listed below. If not provided, the
+                              associated PMv statistics are not gathered:
+
+                              'heartbeat': Reference to the a class that provides an ONU heartbeat
+                                           statistics.   TODO: This should be standardized across adapters
+        """
+        super(OnuPmMetrics, self).__init__(adapter_agent, device_id, logical_device_id,
+                                           grouped=grouped, freq_override=freq_override,
+                                           **kwargs)
+
+        # The following HeartBeat PM is only an example. We may want to have a common heartbeat
+        # object for OLT and ONU DAs that work the same.  If so, it could also provide PM information
+        #
+        # TODO: In the actual 'collection' of PM data, I have the heartbeat stats disabled since
+        #       there is not yet a common 'heartbeat' object
+        #
+        self.health_pm_names = {
+            ('alarm_active', PmConfig.STATE),
+            ('heartbeat_count', PmConfig.COUNTER),
+            ('heartbeat_miss', PmConfig.COUNTER),
+            ('alarms_raised_count', PmConfig.COUNTER),
+            ('heartbeat_failed_limit', PmConfig.COUNTER),
+            ('heartbeat_interval', PmConfig.COUNTER),
+        }
+        # TODO Add PON Port pollable PM as a separate class and include like OMCI
+        # TODO Add UNI Port pollable PM as a separate class and include like OMCI
+        self._heartbeat = kwargs.pop('heartbeat', None)
+        self.health_metrics_config = {m: PmConfig(name=m, type=t, enabled=True)
+                                      for (m, t) in self.health_pm_names}
+
+        self.omci_pm = OnuOmciPmMetrics(adapter_agent, device_id, logical_device_id,
+                                        grouped=grouped, freq_override=freq_override,
+                                        **kwargs)
+
+    def update(self, pm_config):
+        try:
+            # TODO: Test frequency override capability for a particular group
+            if self.default_freq != pm_config.default_freq:
+                # Update the callback to the new frequency.
+                self.default_freq = pm_config.default_freq
+                self.lc.stop()
+                self.lc.start(interval=self.default_freq / 10)
+
+            if pm_config.grouped:
+                for group in pm_config.groups:
+                    group_config = self.pm_group_metrics.get(group.group_name)
+                    if group_config is not None:
+                        group_config.enabled = group.enabled
+            else:
+                msg = 'There are no independent ONU metrics, only group metrics at this time'
+                raise NotImplemented(msg)
+
+        except Exception as e:
+            self.log.exception('update-failure', e=e)
+            raise
+
+        self.omci_pm.update(pm_config)
+
+    def make_proto(self, pm_config=None):
+        if pm_config is None:
+            pm_config = PmConfigs(id=self.device_id,
+                                  default_freq=self.default_freq,
+                                  grouped=self.grouped,
+                                  freq_override=self.freq_override)
+        metrics = set()
+
+        if self._heartbeat is not None:
+            if self.grouped:
+                pm_health_stats = PmGroupConfig(group_name='Heartbeat',
+                                                group_freq=OnuPmMetrics.DEFAULT_HEARTBEAT_FREQUENCY,
+                                                enabled=OnuPmMetrics.DEFAULT_HEARTBEAT_ENABLED)
+                self.pm_group_metrics[pm_health_stats.group_name] = pm_health_stats
+            else:
+                pm_health_stats = pm_config
+
+            # Add metrics to the PM Group (or as individual metrics_
+            for m in sorted(self.health_metrics_config):
+                pm = self.health_metrics_config[m]
+                if not self.grouped:
+                    if pm.name in metrics:
+                        continue
+                    metrics.add(pm.name)
+
+                pm_health_stats.metrics.extend([PmConfig(name=pm.name,
+                                                         type=pm.type,
+                                                         enabled=pm.enabled)])
+            if self.grouped:
+                pm_config.groups.extend([pm_health_stats])
+
+        # TODO Add PON Port PM
+        # TODO Add UNI Port PM
+        pm_config = self.omci_pm.make_proto(pm_config)
+        return pm_config
+
+    def collect_metrics(self, data=None):
+        """
+        Collect metrics for this adapter.
+
+        The data collected (or passed in) is a list of pairs/tuples.  Each
+        pair is composed of a MetricMetaData metadata-portion and list of MetricValuePairs
+        that contains a single individual metric or list of metrics if this is a
+        group metric.
+
+        This method is called for each adapter at a fixed frequency.
+        TODO: Currently all group metrics are collected on a single timer tick.
+              This needs to be fixed as independent group or instance collection is
+              desirable.
+
+        :param data: (list) Existing list of collected metrics (MetricInformation).
+                            This is provided to allow derived classes to call into
+                            further encapsulated classes.
+
+        :return: (list) metadata and metrics pairs - see description above
+        """
+        if data is None:
+            data = list()
+
+        # TODO: Heartbeat stats disabled since it is not a common item on all ONUs (or OLTs)
+        # if self._heartbeat is not None:
+        #     data.extend(self.collect_metrics(self._heartbeat, self.health_pm_names,
+        #                                      self.health_metrics_config))
+        return self.omci_pm.collect_metrics(data=data)