VOL-1451 Openonu now runs using pyvoltha completely

Remove code that now exists in the library.  Modify include paths
Update requirements to more closely reflect pyvoltha requirements.txt

Change-Id: I94c619de82d8ee14cfeb302b3b82be3a26619301
diff --git a/python/adapters/brcm_openomci_onu/brcm_openomci_onu.py b/python/adapters/brcm_openomci_onu/brcm_openomci_onu.py
index ad89dc8..d5cfb6f 100644
--- a/python/adapters/brcm_openomci_onu/brcm_openomci_onu.py
+++ b/python/adapters/brcm_openomci_onu/brcm_openomci_onu.py
@@ -23,21 +23,20 @@
 from twisted.internet import reactor, task
 from zope.interface import implementer
 
-from voltha.adapters.brcm_openomci_onu.brcm_openomci_onu_handler import BrcmOpenomciOnuHandler
-from voltha.adapters.interface import IAdapterInterface
-from voltha.protos import third_party
-from voltha.protos.adapter_pb2 import Adapter
-from voltha.protos.adapter_pb2 import AdapterConfig
-from voltha.protos.common_pb2 import LogLevel
-from voltha.protos.device_pb2 import DeviceType, DeviceTypes, Port, Image
-from voltha.protos.health_pb2 import HealthStatus
+from pyvoltha.adapters.interface import IAdapterInterface
+from pyvoltha.protos import third_party
+from pyvoltha.protos.adapter_pb2 import Adapter
+from pyvoltha.protos.adapter_pb2 import AdapterConfig
+from pyvoltha.protos.common_pb2 import LogLevel
+from pyvoltha.protos.device_pb2 import DeviceType, DeviceTypes, Port, Image
+from pyvoltha.protos.health_pb2 import HealthStatus
 
-from common.frameio.frameio import hexify
-from voltha.extensions.omci.openomci_agent import OpenOMCIAgent, OpenOmciAgentDefaults
-from voltha.extensions.omci.omci_me import *
-from voltha.extensions.omci.database.mib_db_dict import MibDbVolatileDict
+from pyvoltha.adapters.common.frameio.frameio import hexify
+from pyvoltha.adapters.extensions.omci.openomci_agent import OpenOMCIAgent, OpenOmciAgentDefaults
+from pyvoltha.adapters.extensions.omci.omci_me import *
+
+from brcm_openomci_onu_handler import BrcmOpenomciOnuHandler
 from omci.brcm_capabilities_task import BrcmCapabilitiesTask
-from omci.brcm_get_mds_task import BrcmGetMdsTask
 from omci.brcm_mib_sync import BrcmMibSynchronizer
 from copy import deepcopy
 
diff --git a/python/adapters/brcm_openomci_onu/brcm_openomci_onu_handler.py b/python/adapters/brcm_openomci_onu/brcm_openomci_onu_handler.py
index 42dd0a9..f5dc8bf 100644
--- a/python/adapters/brcm_openomci_onu/brcm_openomci_onu_handler.py
+++ b/python/adapters/brcm_openomci_onu/brcm_openomci_onu_handler.py
@@ -28,33 +28,30 @@
 from twisted.internet.defer import DeferredQueue, inlineCallbacks, returnValue, TimeoutError
 
 from heartbeat import HeartBeat
-from voltha.extensions.kpi.onu.onu_pm_metrics import OnuPmMetrics
-from voltha.extensions.kpi.onu.onu_omci_pm import OnuOmciPmMetrics
-from voltha.extensions.alarms.adapter_alarms import AdapterAlarms
+from pyvoltha.adapters.extensions.kpi.onu.onu_pm_metrics import OnuPmMetrics
+from pyvoltha.adapters.extensions.kpi.onu.onu_omci_pm import OnuOmciPmMetrics
+from pyvoltha.adapters.extensions.alarms.adapter_alarms import AdapterAlarms
 
-from common.utils.indexpool import IndexPool
-import voltha.core.flow_decomposer as fd
-from voltha.registry import registry
-from voltha.core.config.config_backend import ConsulStore
-from voltha.core.config.config_backend import EtcdStore
-from voltha.protos import third_party
-from voltha.protos.common_pb2 import OperStatus, ConnectStatus, AdminState
-from voltha.protos.openflow_13_pb2 import OFPXMC_OPENFLOW_BASIC, ofp_port
-from voltha.protos.bbf_fiber_tcont_body_pb2 import TcontsConfigData
-from voltha.protos.bbf_fiber_gemport_body_pb2 import GemportsConfigData
-from voltha.extensions.omci.onu_configuration import OMCCVersion
-from voltha.extensions.omci.onu_device_entry import OnuDeviceEvents, \
+import pyvoltha.common.openflow.utils as fd
+from pyvoltha.common.utils.registry import registry
+from pyvoltha.common.config.config_backend import ConsulStore
+from pyvoltha.common.config.config_backend import EtcdStore
+from pyvoltha.protos import third_party
+from pyvoltha.protos.common_pb2 import OperStatus, ConnectStatus, AdminState
+from pyvoltha.protos.openflow_13_pb2 import OFPXMC_OPENFLOW_BASIC, ofp_port
+from pyvoltha.adapters.extensions.omci.onu_configuration import OMCCVersion
+from pyvoltha.adapters.extensions.omci.onu_device_entry import OnuDeviceEvents, \
     OnuDeviceEntry, IN_SYNC_KEY
-from voltha.adapters.brcm_openomci_onu.omci.brcm_mib_download_task import BrcmMibDownloadTask
-from voltha.adapters.brcm_openomci_onu.omci.brcm_tp_service_specific_task import BrcmTpServiceSpecificTask
-from voltha.adapters.brcm_openomci_onu.omci.brcm_uni_lock_task import BrcmUniLockTask
-from voltha.adapters.brcm_openomci_onu.omci.brcm_vlan_filter_task import BrcmVlanFilterTask
-from voltha.adapters.brcm_openomci_onu.onu_gem_port import *
-from voltha.adapters.brcm_openomci_onu.onu_tcont import *
-from voltha.adapters.brcm_openomci_onu.pon_port import *
-from voltha.adapters.brcm_openomci_onu.uni_port import *
-from voltha.adapters.brcm_openomci_onu.onu_traffic_descriptor import *
-from common.tech_profile.tech_profile import TechProfile
+from omci.brcm_mib_download_task import BrcmMibDownloadTask
+from omci.brcm_tp_service_specific_task import BrcmTpServiceSpecificTask
+from omci.brcm_uni_lock_task import BrcmUniLockTask
+from omci.brcm_vlan_filter_task import BrcmVlanFilterTask
+from onu_gem_port import *
+from onu_tcont import *
+from pon_port import *
+from uni_port import *
+from onu_traffic_descriptor import *
+from pyvoltha.common.tech_profile.tech_profile import TechProfile
 
 OP = EntityOperations
 RC = ReasonCodes
@@ -701,8 +698,6 @@
     # Not currently called.  Would be called presumably from the olt handler
     def remove_gemport(self, data):
         self.log.debug('remove-gemport', data=data)
-        gem_port = GemportsConfigData()
-        gem_port.CopyFrom(data)
         device = self.adapter_agent.get_device(self.device_id)
         if device.connect_status != ConnectStatus.REACHABLE:
             self.log.error('device-unreachable')
diff --git a/python/adapters/brcm_openomci_onu/heartbeat.py b/python/adapters/brcm_openomci_onu/heartbeat.py
index 5242caa..f47883c 100644
--- a/python/adapters/brcm_openomci_onu/heartbeat.py
+++ b/python/adapters/brcm_openomci_onu/heartbeat.py
@@ -14,8 +14,8 @@
 
 import structlog
 from twisted.internet import reactor
-from voltha.protos.common_pb2 import OperStatus, ConnectStatus
-from voltha.extensions.omci.omci_me import OntGFrame
+from pyvoltha.protos.common_pb2 import OperStatus, ConnectStatus
+from pyvoltha.adapters.extensions.omci.omci_me import OntGFrame
 
 
 class HeartBeat(object):
@@ -145,7 +145,7 @@
         device = self._handler.adapter_agent.get_device(self._device_id)
 
         try:
-            from voltha.extensions.alarms.heartbeat_alarm import HeartbeatAlarm
+            from pyvoltha.adapters.extensions.alarms.heartbeat_alarm import HeartbeatAlarm
 
             if self._heartbeat_miss >= self.heartbeat_failed_limit:
                 if device.connect_status == ConnectStatus.REACHABLE:
diff --git a/python/adapters/brcm_openomci_onu/main.py b/python/adapters/brcm_openomci_onu/main.py
index ed1d15f..a88d8bc 100755
--- a/python/adapters/brcm_openomci_onu/main.py
+++ b/python/adapters/brcm_openomci_onu/main.py
@@ -29,22 +29,23 @@
 from twisted.internet.task import LoopingCall
 from zope.interface import implementer
 
-from common.structlog_setup import setup_logging, update_logging
-from common.utils.asleep import asleep
-from common.utils.deferred_utils import TimeOutError
-from common.utils.dockerhelpers import get_my_containers_name
-from common.utils.nethelpers import get_my_primary_local_ipv4, \
+from pyvoltha.common.structlog_setup import setup_logging, update_logging
+from pyvoltha.common.utils.asleep import asleep
+from pyvoltha.common.utils.deferred_utils import TimeOutError
+from pyvoltha.common.utils.dockerhelpers import get_my_containers_name
+from pyvoltha.common.utils.nethelpers import get_my_primary_local_ipv4, \
     get_my_primary_interface
-from voltha.core.registry import registry, IComponent
-from kafka.adapter_proxy import AdapterProxy
-from kafka.adapter_request_facade import AdapterRequestFacade
-from kafka.core_proxy import CoreProxy
-from kafka.kafka_inter_container_library import IKafkaMessagingProxy, \
+from pyvoltha.common.utils.registry import registry, IComponent
+from pyvoltha.adapters.kafka.adapter_proxy import AdapterProxy
+from pyvoltha.adapters.kafka.adapter_request_facade import AdapterRequestFacade
+from pyvoltha.adapters.kafka.core_proxy import CoreProxy
+from pyvoltha.adapters.kafka.kafka_inter_container_library import IKafkaMessagingProxy, \
     get_messaging_proxy
-from kafka.kafka_proxy import KafkaProxy, get_kafka_proxy
+from pyvoltha.adapters.kafka.kafka_proxy import KafkaProxy, get_kafka_proxy
+from pyvoltha.protos import third_party
+from pyvoltha.protos.adapter_pb2 import AdapterConfig
+
 from brcm_openomci_onu import BrcmOpenomciOnuAdapter
-from voltha.protos import third_party
-from voltha.protos.adapter_pb2 import AdapterConfig
 
 _ = third_party
 
diff --git a/python/adapters/brcm_openomci_onu/omci/brcm_capabilities_task.py b/python/adapters/brcm_openomci_onu/omci/brcm_capabilities_task.py
index 6bf5b93..d56a1d0 100644
--- a/python/adapters/brcm_openomci_onu/omci/brcm_capabilities_task.py
+++ b/python/adapters/brcm_openomci_onu/omci/brcm_capabilities_task.py
@@ -14,7 +14,7 @@
 # limitations under the License.
 
 import structlog
-from voltha.extensions.omci.tasks.onu_capabilities_task import OnuCapabilitiesTask
+from pyvoltha.adapters.extensions.omci.tasks.onu_capabilities_task import OnuCapabilitiesTask
 from twisted.internet.defer import failure
 
 
@@ -88,7 +88,7 @@
             return super(BrcmCapabilitiesTask, self).supported_message_types
 
         # TODO: figure out why broadcom wont answer for ME 287 to get this.  otherwise manually fill in
-        from voltha.extensions.omci.omci_entities import EntityOperations
+        from pyvoltha.adapters.extensions.omci.omci_entities import EntityOperations
         op_11287800f1 = [
             EntityOperations.Create,
             EntityOperations.CreateComplete,
diff --git a/python/adapters/brcm_openomci_onu/omci/brcm_get_mds_task.py b/python/adapters/brcm_openomci_onu/omci/brcm_get_mds_task.py
index eabf356..16a8783 100644
--- a/python/adapters/brcm_openomci_onu/omci/brcm_get_mds_task.py
+++ b/python/adapters/brcm_openomci_onu/omci/brcm_get_mds_task.py
@@ -14,7 +14,7 @@
 # limitations under the License.
 
 import structlog
-from voltha.extensions.omci.tasks.get_mds_task import GetMdsTask
+from pyvoltha.adapters.extensions.omci.tasks.get_mds_task import GetMdsTask
 
 
 class BrcmGetMdsTask(GetMdsTask):
diff --git a/python/adapters/brcm_openomci_onu/omci/brcm_mib_download_task.py b/python/adapters/brcm_openomci_onu/omci/brcm_mib_download_task.py
index 3341219..2d58fe0 100644
--- a/python/adapters/brcm_openomci_onu/omci/brcm_mib_download_task.py
+++ b/python/adapters/brcm_openomci_onu/omci/brcm_mib_download_task.py
@@ -14,14 +14,13 @@
 # limitations under the License.
 
 import structlog
-from common.frameio.frameio import hexify
 from twisted.internet import reactor
 from twisted.internet.defer import inlineCallbacks, returnValue, TimeoutError, failure
-from voltha.extensions.omci.omci_me import *
-from voltha.extensions.omci.tasks.task import Task
-from voltha.extensions.omci.omci_defs import *
-from voltha.adapters.brcm_openomci_onu.uni_port import *
-from voltha.adapters.brcm_openomci_onu.pon_port \
+from pyvoltha.adapters.extensions.omci.omci_me import *
+from pyvoltha.adapters.extensions.omci.tasks.task import Task
+from pyvoltha.adapters.extensions.omci.omci_defs import *
+from adapters.brcm_openomci_onu.uni_port import *
+from adapters.brcm_openomci_onu.pon_port \
     import BRDCM_DEFAULT_VLAN, TASK_PRIORITY, DEFAULT_TPID, DEFAULT_GEM_PAYLOAD
 
 OP = EntityOperations
diff --git a/python/adapters/brcm_openomci_onu/omci/brcm_mib_sync.py b/python/adapters/brcm_openomci_onu/omci/brcm_mib_sync.py
index 1898c52..e330be9 100644
--- a/python/adapters/brcm_openomci_onu/omci/brcm_mib_sync.py
+++ b/python/adapters/brcm_openomci_onu/omci/brcm_mib_sync.py
@@ -15,7 +15,7 @@
 
 import structlog
 from twisted.internet import reactor
-from voltha.extensions.omci.state_machines.mib_sync import MibSynchronizer
+from pyvoltha.adapters.extensions.omci.state_machines.mib_sync import MibSynchronizer
 
 log = structlog.get_logger()
 
diff --git a/python/adapters/brcm_openomci_onu/omci/brcm_tp_service_specific_task.py b/python/adapters/brcm_openomci_onu/omci/brcm_tp_service_specific_task.py
index ff0bd30..1a9eb1e 100644
--- a/python/adapters/brcm_openomci_onu/omci/brcm_tp_service_specific_task.py
+++ b/python/adapters/brcm_openomci_onu/omci/brcm_tp_service_specific_task.py
@@ -14,14 +14,13 @@
 # limitations under the License.
 
 import structlog
-from common.frameio.frameio import hexify
 from twisted.internet import reactor
 from twisted.internet.defer import inlineCallbacks, returnValue, TimeoutError, failure
-from voltha.extensions.omci.omci_me import *
-from voltha.extensions.omci.tasks.task import Task
-from voltha.extensions.omci.omci_defs import *
-from voltha.adapters.brcm_openomci_onu.uni_port import *
-from voltha.adapters.brcm_openomci_onu.pon_port \
+from pyvoltha.adapters.extensions.omci.omci_me import *
+from pyvoltha.adapters.extensions.omci.tasks.task import Task
+from pyvoltha.adapters.extensions.omci.omci_defs import *
+from adapters.brcm_openomci_onu.uni_port import *
+from adapters.brcm_openomci_onu.pon_port \
     import BRDCM_DEFAULT_VLAN, TASK_PRIORITY, DEFAULT_TPID, DEFAULT_GEM_PAYLOAD
 
 OP = EntityOperations
diff --git a/python/adapters/brcm_openomci_onu/omci/brcm_uni_lock_task.py b/python/adapters/brcm_openomci_onu/omci/brcm_uni_lock_task.py
index c304a27..f062075 100644
--- a/python/adapters/brcm_openomci_onu/omci/brcm_uni_lock_task.py
+++ b/python/adapters/brcm_openomci_onu/omci/brcm_uni_lock_task.py
@@ -13,12 +13,12 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-from voltha.extensions.omci.tasks.task import Task
+from pyvoltha.adapters.extensions.omci.tasks.task import Task
 from twisted.internet import reactor
 from twisted.internet.defer import inlineCallbacks, failure, returnValue
-from voltha.extensions.omci.omci_defs import ReasonCodes, EntityOperations
-from voltha.extensions.omci.omci_me import OntGFrame
-from voltha.extensions.omci.omci_me import PptpEthernetUniFrame, VeipUniFrame
+from pyvoltha.adapters.extensions.omci.omci_defs import ReasonCodes, EntityOperations
+from pyvoltha.adapters.extensions.omci.omci_me import OntGFrame
+from pyvoltha.adapters.extensions.omci.omci_me import PptpEthernetUniFrame, VeipUniFrame
 
 RC = ReasonCodes
 OP = EntityOperations
diff --git a/python/adapters/brcm_openomci_onu/omci/brcm_vlan_filter_task.py b/python/adapters/brcm_openomci_onu/omci/brcm_vlan_filter_task.py
index 6c665c7..1afc671 100644
--- a/python/adapters/brcm_openomci_onu/omci/brcm_vlan_filter_task.py
+++ b/python/adapters/brcm_openomci_onu/omci/brcm_vlan_filter_task.py
@@ -13,11 +13,11 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-from voltha.extensions.omci.tasks.task import Task
+from pyvoltha.adapters.extensions.omci.tasks.task import Task
 from twisted.internet import reactor
 from twisted.internet.defer import inlineCallbacks, failure, returnValue
-from voltha.extensions.omci.omci_defs import ReasonCodes, EntityOperations
-from voltha.extensions.omci.omci_me import *
+from pyvoltha.adapters.extensions.omci.omci_defs import ReasonCodes, EntityOperations
+from pyvoltha.adapters.extensions.omci.omci_me import *
 
 RC = ReasonCodes
 OP = EntityOperations
diff --git a/python/adapters/brcm_openomci_onu/onu_gem_port.py b/python/adapters/brcm_openomci_onu/onu_gem_port.py
index b388030..8ecd192 100644
--- a/python/adapters/brcm_openomci_onu/onu_gem_port.py
+++ b/python/adapters/brcm_openomci_onu/onu_gem_port.py
@@ -15,8 +15,8 @@
 
 import structlog
 from twisted.internet.defer import inlineCallbacks, returnValue
-from voltha.extensions.omci.omci_me import *
-from voltha.extensions.omci.omci_defs import *
+from pyvoltha.adapters.extensions.omci.omci_me import *
+from pyvoltha.adapters.extensions.omci.omci_defs import *
 
 RC = ReasonCodes
 
diff --git a/python/adapters/brcm_openomci_onu/onu_tcont.py b/python/adapters/brcm_openomci_onu/onu_tcont.py
index c5414ee..1f5ad12 100644
--- a/python/adapters/brcm_openomci_onu/onu_tcont.py
+++ b/python/adapters/brcm_openomci_onu/onu_tcont.py
@@ -14,10 +14,9 @@
 # limitations under the License.
 
 import structlog
-from common.frameio.frameio import hexify
 from twisted.internet.defer import  inlineCallbacks, returnValue, succeed
-from voltha.extensions.omci.omci_me import *
-from voltha.extensions.omci.omci_defs import *
+from pyvoltha.adapters.extensions.omci.omci_me import *
+from pyvoltha.adapters.extensions.omci.omci_defs import *
 
 RC = ReasonCodes
 
diff --git a/python/adapters/brcm_openomci_onu/pon_port.py b/python/adapters/brcm_openomci_onu/pon_port.py
index db1daa8..4d93567 100644
--- a/python/adapters/brcm_openomci_onu/pon_port.py
+++ b/python/adapters/brcm_openomci_onu/pon_port.py
@@ -15,9 +15,9 @@
 
 import structlog
 from twisted.internet.defer import inlineCallbacks, returnValue
-from voltha.protos.common_pb2 import AdminState, OperStatus
-from voltha.protos.device_pb2 import Port
-from voltha.extensions.omci.tasks.task import Task
+from pyvoltha.protos.common_pb2 import AdminState, OperStatus
+from pyvoltha.protos.device_pb2 import Port
+from pyvoltha.adapters.extensions.omci.tasks.task import Task
 
 BRDCM_DEFAULT_VLAN = 4091
 TASK_PRIORITY = Task.DEFAULT_PRIORITY + 10
diff --git a/python/adapters/brcm_openomci_onu/uni_port.py b/python/adapters/brcm_openomci_onu/uni_port.py
index 2ee307b..fb3e06c 100644
--- a/python/adapters/brcm_openomci_onu/uni_port.py
+++ b/python/adapters/brcm_openomci_onu/uni_port.py
@@ -15,13 +15,13 @@
 
 import structlog
 from enum import Enum
-from voltha.protos.common_pb2 import OperStatus, AdminState
-from voltha.protos.device_pb2 import Port
-from voltha.protos.openflow_13_pb2 import OFPPF_10GB_FD
-from voltha.core.logical_device_agent import mac_str_to_tuple
-from voltha.protos.logical_device_pb2 import LogicalPort
-from voltha.protos.openflow_13_pb2 import OFPPS_LIVE, OFPPF_FIBER, OFPPS_LINK_DOWN
-from voltha.protos.openflow_13_pb2 import ofp_port
+from pyvoltha.protos.common_pb2 import OperStatus, AdminState
+from pyvoltha.protos.device_pb2 import Port
+from pyvoltha.protos.openflow_13_pb2 import OFPPF_10GB_FD
+from pyvoltha.common.utils.nethelpers import mac_str_to_tuple
+from pyvoltha.protos.logical_device_pb2 import LogicalPort
+from pyvoltha.protos.openflow_13_pb2 import OFPPS_LIVE, OFPPF_FIBER, OFPPS_LINK_DOWN
+from pyvoltha.protos.openflow_13_pb2 import ofp_port
 
 class UniType(Enum):
     """
diff --git a/python/adapters/iadapter.py b/python/adapters/iadapter.py
deleted file mode 100644
index 31c5d7a..0000000
--- a/python/adapters/iadapter.py
+++ /dev/null
@@ -1,358 +0,0 @@
-#
-# Copyright 2018 the original author or authors.
-#
-# 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.
-#
-
-"""
-Adapter abstract base class
-"""
-
-import structlog
-from twisted.internet import reactor
-from zope.interface import implementer
-
-from interface import IAdapterInterface
-from voltha.protos.adapter_pb2 import Adapter
-from voltha.protos.adapter_pb2 import AdapterConfig
-from voltha.protos.common_pb2 import AdminState
-from voltha.protos.common_pb2 import LogLevel
-from voltha.protos.device_pb2 import DeviceType, DeviceTypes
-from voltha.protos.health_pb2 import HealthStatus
-
-
-log = structlog.get_logger()
-
-
-@implementer(IAdapterInterface)
-class IAdapter(object):
-    def __init__(self,
-                 core_proxy,
-                 adapter_proxy,
-                 config,
-                 device_handler_class,
-                 name,
-                 vendor,
-                 version,
-                 device_type, vendor_id,
-                 accepts_bulk_flow_update=True,
-                 accepts_add_remove_flow_updates=False):
-        log.debug(
-            'Initializing adapter: {} {} {}'.format(vendor, name, version))
-        self.core_proxy = core_proxy
-        self.adapter_proxy = adapter_proxy
-        self.config = config
-        self.name = name
-        self.supported_device_types = [
-            DeviceType(
-                id=device_type,
-                vendor_id=vendor_id,
-                adapter=name,
-                accepts_bulk_flow_update=accepts_bulk_flow_update,
-                accepts_add_remove_flow_updates=accepts_add_remove_flow_updates
-            )
-        ]
-        self.descriptor = Adapter(
-            id=self.name,
-            vendor=vendor,
-            version=version,
-            config=AdapterConfig(log_level=LogLevel.INFO)
-        )
-        self.devices_handlers = dict()  # device_id -> Olt/OnuHandler()
-        self.device_handler_class = device_handler_class
-
-    def start(self):
-        log.info('Starting adapter: {}'.format(self.name))
-
-    def stop(self):
-        log.info('Stopping adapter: {}'.format(self.name))
-
-    def adapter_descriptor(self):
-        return self.descriptor
-
-    def device_types(self):
-        return DeviceTypes(items=self.supported_device_types)
-
-    def health(self):
-        # return HealthStatus(state=HealthStatus.HealthState.HEALTHY)
-        return HealthStatus(state=HealthStatus.HEALTHY)
-
-    def change_master_state(self, master):
-        raise NotImplementedError()
-
-    def get_ofp_device_info(self, device):
-        log.debug('get_ofp_device_info_start', device_id=device.id)
-        ofp_device_info = self.devices_handlers[device.id].get_ofp_device_info(
-            device)
-        log.debug('get_ofp_device_info_ends', device_id=device.id)
-        return ofp_device_info
-
-    def get_ofp_port_info(self, device, port_no):
-        log.debug('get_ofp_port_info_start', device_id=device.id,
-                  port_no=port_no)
-        ofp_port_info = self.devices_handlers[device.id].get_ofp_port_info(
-            device, port_no)
-        log.debug('get_ofp_port_info_ends', device_id=device.id,
-                  port_no=port_no)
-        return ofp_port_info
-
-    def adopt_device(self, device):
-        log.debug('adopt_device', device_id=device.id)
-        self.devices_handlers[device.id] = self.device_handler_class(self,
-                                                                     device.id)
-        reactor.callLater(0, self.devices_handlers[device.id].activate, device)
-        log.debug('adopt_device_done', device_id=device.id)
-        return device
-
-    def reconcile_device(self, device):
-        raise NotImplementedError()
-
-    def abandon_device(self, device):
-        raise NotImplementedError()
-
-    def disable_device(self, device):
-        log.info('disable-device', device_id=device.id)
-        reactor.callLater(0, self.devices_handlers[device.id].disable)
-        log.debug('disable-device-done', device_id=device.id)
-        return device
-
-    def reenable_device(self, device):
-        log.info('reenable-device', device_id=device.id)
-        reactor.callLater(0, self.devices_handlers[device.id].reenable)
-        log.info('reenable-device-done', device_id=device.id)
-        return device
-
-    def reboot_device(self, device):
-        log.info('reboot-device', device_id=device.id)
-        reactor.callLater(0, self.devices_handlers[device.id].reboot)
-        log.info('reboot-device-done', device_id=device.id)
-        return device
-
-    def download_image(self, device, request):
-        raise NotImplementedError()
-
-    def get_image_download_status(self, device, request):
-        raise NotImplementedError()
-
-    def cancel_image_download(self, device, request):
-        raise NotImplementedError()
-
-    def activate_image_update(self, device, request):
-        raise NotImplementedError()
-
-    def revert_image_update(self, device, request):
-        raise NotImplementedError()
-
-    def self_test_device(self, device):
-        log.info('self-test', device_id=device.id)
-        result = reactor.callLater(0, self.devices_handlers[
-            device.id].self_test_device)
-        log.info('self-test-done', device_id=device.id)
-        return result
-
-    def delete_device(self, device):
-        log.info('delete-device', device_id=device.id)
-        reactor.callLater(0, self.devices_handlers[device.id].delete)
-        log.info('delete-device-done', device_id=device.id)
-        return device
-
-    def get_device_details(self, device):
-        raise NotImplementedError()
-
-    def update_flows_bulk(self, device, flows, groups):
-        log.info('bulk-flow-update', device_id=device.id,
-                 flows=flows, groups=groups)
-        assert len(groups.items) == 0
-        reactor.callLater(0, self.devices_handlers[device.id].update_flow_table,
-                          flows.items)
-        return device
-
-    def update_flows_incrementally(self, device, flow_changes, group_changes):
-        log.info('incremental-flow-update', device_id=device.id,
-                 flows=flow_changes, groups=group_changes)
-        # For now, there is no support for group changes
-        assert len(group_changes.to_add.items) == 0
-        assert len(group_changes.to_remove.items) == 0
-
-        handler = self.devices_handlers[device.id]
-        # Remove flows
-        if len(flow_changes.to_remove.items) != 0:
-            reactor.callLater(0, handler.remove_from_flow_table,
-                              flow_changes.to_remove.items)
-
-        # Add flows
-        if len(flow_changes.to_add.items) != 0:
-            reactor.callLater(0, handler.add_to_flow_table,
-                              flow_changes.to_add.items)
-        return device
-
-    def update_pm_config(self, device, pm_config):
-        log.info("adapter-update-pm-config", device=device,
-                 pm_config=pm_config)
-        handler = self.devices_handlers[device.id]
-        if handler:
-            reactor.callLater(0, handler.update_pm_config, device, pm_config)
-
-    def process_inter_adapter_message(self, msg):
-        raise NotImplementedError()
-
-    def receive_packet_out(self, device_id, egress_port_no, msg):
-        raise NotImplementedError()
-
-    def suppress_alarm(self, filter):
-        raise NotImplementedError()
-
-    def unsuppress_alarm(self, filter):
-        raise NotImplementedError()
-
-    def _get_handler(self, device):
-        if device.id in self.devices_handlers:
-            handler = self.devices_handlers[device.id]
-            if handler is not None:
-                return handler
-            return None
-
-
-"""
-OLT Adapter base class
-"""
-
-
-class OltAdapter(IAdapter):
-    def __init__(self,
-                 core_proxy,
-                 adapter_proxy,
-                 config,
-                 device_handler_class,
-                 name,
-                 vendor,
-                 version, device_type,
-                 accepts_bulk_flow_update=True,
-                 accepts_add_remove_flow_updates=False):
-        super(OltAdapter, self).__init__(core_proxy=core_proxy,
-                                         adapter_proxy=adapter_proxy,
-                                         config=config,
-                                         device_handler_class=device_handler_class,
-                                         name=name,
-                                         vendor=vendor,
-                                         version=version,
-                                         device_type=device_type,
-                                         vendor_id=None,
-                                         accepts_bulk_flow_update=accepts_bulk_flow_update,
-                                         accepts_add_remove_flow_updates=accepts_add_remove_flow_updates)
-        self.logical_device_id_to_root_device_id = dict()
-
-    def reconcile_device(self, device):
-        try:
-            self.devices_handlers[device.id] = self.device_handler_class(self,
-                                                                         device.id)
-            # Work only required for devices that are in ENABLED state
-            if device.admin_state == AdminState.ENABLED:
-                reactor.callLater(0,
-                                  self.devices_handlers[device.id].reconcile,
-                                  device)
-            else:
-                # Invoke the children reconciliation which would setup the
-                # basic children data structures
-                self.core_proxy.reconcile_child_devices(device.id)
-            return device
-        except Exception, e:
-            log.exception('Exception', e=e)
-
-    def send_proxied_message(self, proxy_address, msg):
-        log.info('send-proxied-message', proxy_address=proxy_address, msg=msg)
-        handler = self.devices_handlers[proxy_address.device_id]
-        handler.send_proxied_message(proxy_address, msg)
-
-    def process_inter_adapter_message(self, msg):
-        log.debug('process-inter-adapter-message', msg=msg)
-        # Unpack the header to know which device needs to handle this message
-        handler = None
-        if msg.header.proxy_device_id:
-            # typical request
-            handler = self.devices_handlers[msg.header.proxy_device_id]
-        elif msg.header.to_device_id and \
-                msg.header.to_device_id in self.devices_handlers:
-            # typical response
-            handler = self.devices_handlers[msg.header.to_device_id]
-        if handler:
-            reactor.callLater(0, handler.process_inter_adapter_message, msg)
-
-    def receive_packet_out(self, device_id, egress_port_no, msg):
-        try:
-            log.info('receive_packet_out', device_id=device_id,
-                     egress_port=egress_port_no, msg=msg)
-            handler = self.devices_handlers[device_id]
-            if handler:
-                reactor.callLater(0, handler.packet_out, egress_port_no, msg.data)
-        except Exception, e:
-            log.exception('packet-out-failure', e=e)
-
-
-"""
-ONU Adapter base class
-"""
-
-
-class OnuAdapter(IAdapter):
-    def __init__(self,
-                 core_proxy,
-                 adapter_proxy,
-                 config,
-                 device_handler_class,
-                 name,
-                 vendor,
-                 version,
-                 device_type,
-                 vendor_id,
-                 accepts_bulk_flow_update=True,
-                 accepts_add_remove_flow_updates=False):
-        super(OnuAdapter, self).__init__(core_proxy=core_proxy,
-                                         adapter_proxy=adapter_proxy,
-                                         config=config,
-                                         device_handler_class=device_handler_class,
-                                         name=name,
-                                         vendor=vendor,
-                                         version=version,
-                                         device_type=device_type,
-                                         vendor_id=vendor_id,
-                                         accepts_bulk_flow_update=accepts_bulk_flow_update,
-                                         accepts_add_remove_flow_updates=accepts_add_remove_flow_updates)
-
-    def reconcile_device(self, device):
-        self.devices_handlers[device.id] = self.device_handler_class(self,
-                                                                     device.id)
-        # Reconcile only if state was ENABLED
-        if device.admin_state == AdminState.ENABLED:
-            reactor.callLater(0,
-                              self.devices_handlers[device.id].reconcile,
-                              device)
-        return device
-
-    def receive_proxied_message(self, proxy_address, msg):
-        log.info('receive-proxied-message', proxy_address=proxy_address,
-                 device_id=proxy_address.device_id, msg=msg)
-        # Device_id from the proxy_address is the olt device id. We need to
-        # get the onu device id using the port number in the proxy_address
-        device = self.core_proxy. \
-            get_child_device_with_proxy_address(proxy_address)
-        if device:
-            handler = self.devices_handlers[device.id]
-            handler.receive_message(msg)
-
-    def process_inter_adapter_message(self, msg):
-        log.info('process-inter-adapter-message', msg=msg)
-        # Unpack the header to know which device needs to handle this message
-        if msg.header:
-            handler = self.devices_handlers[msg.header.to_device_id]
-            handler.process_inter_adapter_message(msg)
diff --git a/python/adapters/interface.py b/python/adapters/interface.py
deleted file mode 100644
index b0390d8..0000000
--- a/python/adapters/interface.py
+++ /dev/null
@@ -1,459 +0,0 @@
-#
-# Copyright 2018 the original author or authors.
-#
-# 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.
-#
-
-"""
-Interface definition for Voltha Adapters
-"""
-from zope.interface import Interface
-
-
-class IAdapterInterface(Interface):
-    """
-    A Voltha adapter.  This interface is used by the Voltha Core to initiate
-    requests towards a voltha adapter.
-    """
-
-    def adapter_descriptor():
-        """
-        Return the adapter descriptor object for this adapter.
-        :return: voltha.Adapter grpc object (see voltha/protos/adapter.proto),
-        with adapter-specific information and config extensions.
-        """
-
-    def device_types():
-        """
-        Return list of device types supported by the adapter.
-        :return: voltha.DeviceTypes protobuf object, with optional type
-        specific extensions.
-        """
-
-    def health():
-        """
-        Return a 3-state health status using the voltha.HealthStatus message.
-        :return: Deferred or direct return with voltha.HealthStatus message
-        """
-
-    def adopt_device(device):
-        """
-        Make sure the adapter looks after given device. Called when a device
-        is provisioned top-down and needs to be activated by the adapter.
-        :param device: A voltha.Device object, with possible device-type
-        specific extensions. Such extensions shall be described as part of
-        the device type specification returned by device_types().
-        :return: (Deferred) Shall be fired to acknowledge device ownership.
-        """
-
-    def reconcile_device(device):
-        """
-        Make sure the adapter looks after given device. Called when this
-        device has changed ownership from another Voltha instance to
-        this one (typically, this occurs when the previous voltha
-        instance went down).
-        :param device: A voltha.Device object, with possible device-type
-        specific extensions. Such extensions shall be described as part of
-        the device type specification returned by device_types().
-        :return: (Deferred) Shall be fired to acknowledge device ownership.
-        """
-
-    def abandon_device(device):
-        """
-        Make sur ethe adapter no longer looks after device. This is called
-        if device ownership is taken over by another Voltha instance.
-        :param device: A Voltha.Device object.
-        :return: (Deferred) Shall be fired to acknowledge abandonment.
-        """
-
-    def disable_device(device):
-        """
-        This is called when a previously enabled device needs to be disabled
-        based on a NBI call.
-        :param device: A Voltha.Device object.
-        :return: (Deferred) Shall be fired to acknowledge disabling the device.
-        """
-
-    def reenable_device(device):
-        """
-        This is called when a previously disabled device needs to be enabled
-        based on a NBI call.
-        :param device: A Voltha.Device object.
-        :return: (Deferred) Shall be fired to acknowledge re-enabling the
-        device.
-        """
-
-    def reboot_device(device):
-        """
-        This is called to reboot a device based on a NBI call.  The admin
-        state of the device will not change after the reboot
-        :param device: A Voltha.Device object.
-        :return: (Deferred) Shall be fired to acknowledge the reboot.
-        """
-
-    def download_image(device, request):
-        """
-        This is called to request downloading a specified image into
-        the standby partition of a device based on a NBI call.
-        This call is expected to be non-blocking.
-        :param device: A Voltha.Device object.
-                       A Voltha.ImageDownload object.
-        :return: (Deferred) Shall be fired to acknowledge the download.
-        """
-
-    def get_image_download_status(device, request):
-        """
-        This is called to inquire about a requested image download
-        status based on a NBI call.
-        The adapter is expected to update the DownloadImage DB object
-        with the query result
-        :param device: A Voltha.Device object.
-                       A Voltha.ImageDownload object.
-        :return: (Deferred) Shall be fired to acknowledge
-        """
-
-    def cancel_image_download(device, request):
-        """
-        This is called to cancel a requested image download
-        based on a NBI call.  The admin state of the device will not
-        change after the download.
-        :param device: A Voltha.Device object.
-                       A Voltha.ImageDownload object.
-        :return: (Deferred) Shall be fired to acknowledge
-        """
-
-    def activate_image_update(device, request):
-        """
-        This is called to activate a downloaded image from
-        a standby partition into active partition.
-        Depending on the device implementation, this call
-        may or may not cause device reboot.
-        If no reboot, then a reboot is required to make the
-        activated image running on device
-        This call is expected to be non-blocking.
-        :param device: A Voltha.Device object.
-                       A Voltha.ImageDownload object.
-        :return: (Deferred) OperationResponse object.
-        """
-
-    def revert_image_update(device, request):
-        """
-        This is called to deactivate the specified image at
-        active partition, and revert to previous image at
-        standby partition.
-        Depending on the device implementation, this call
-        may or may not cause device reboot.
-        If no reboot, then a reboot is required to make the
-        previous image running on device
-        This call is expected to be non-blocking.
-        :param device: A Voltha.Device object.
-                       A Voltha.ImageDownload object.
-        :return: (Deferred) OperationResponse object.
-        """
-
-    def self_test_device(device):
-        """
-        This is called to Self a device based on a NBI call.
-        :param device: A Voltha.Device object.
-        :return: Will return result of self test
-        """
-
-    def delete_device(device):
-        """
-        This is called to delete a device from the PON based on a NBI call.
-        If the device is an OLT then the whole PON will be deleted.
-        :param device: A Voltha.Device object.
-        :return: (Deferred) Shall be fired to acknowledge the deletion.
-        """
-
-    def get_device_details(device):
-        """
-        This is called to get additional device details based on a NBI call.
-        :param device: A Voltha.Device object.
-        :return: (Deferred) Shall be fired to acknowledge the retrieval of
-        additional details.
-        """
-
-    def update_flows_bulk(device, flows, groups):
-        """
-        Called after any flow table change, but only if the device supports
-        bulk mode, which is expressed by the 'accepts_bulk_flow_update'
-        capability attribute of the device type.
-        :param device: A Voltha.Device object.
-        :param flows: An openflow_v13.Flows object
-        :param groups: An  openflow_v13.Flows object
-        :return: (Deferred or None)
-        """
-
-    def update_flows_incrementally(device, flow_changes, group_changes):
-        """
-        Called after a flow table update, but only if the device supports
-        non-bulk mode, which is expressed by the 'accepts_add_remove_flow_updates'
-        capability attribute of the device type.
-        :param device: A Voltha.Device object.
-        :param flow_changes: An openflow_v13.FlowChanges object
-        :param group_changes: An openflow_v13.FlowGroupChanges object
-        :return: (Deferred or None)
-        """
-
-    def update_pm_config(device, pm_configs):
-        """
-        Called every time a request is made to change pm collection behavior
-        :param device: A Voltha.Device object
-        :param pm_collection_config: A Pms
-        """
-
-    def receive_packet_out(device_id, egress_port_no, msg):
-        """
-        Pass a packet_out message content to adapter so that it can forward
-        it out to the device. This is only called on root devices.
-        :param device_id: device ID
-        :param egress_port: egress logical port number
-         :param msg: actual message
-        :return: None
-        """
-
-    def suppress_alarm(filter):
-        """
-        Inform an adapter that all incoming alarms should be suppressed
-        :param filter: A Voltha.AlarmFilter object.
-        :return: (Deferred) Shall be fired to acknowledge the suppression.
-        """
-
-    def unsuppress_alarm(filter):
-        """
-        Inform an adapter that all incoming alarms should resume
-        :param filter: A Voltha.AlarmFilter object.
-        :return: (Deferred) Shall be fired to acknowledge the unsuppression.
-        """
-
-    def get_ofp_device_info(device):
-        """
-        Retrieve the OLT device info. This includes the ofp_desc and
-        ofp_switch_features. The existing ofp structures can be used,
-        or all the attributes get added to the Device definition or a new proto
-        definition gets created. This API will allow the Core to create a
-        LogicalDevice associated with this device (OLT only).
-        :param device: device
-        :return: Proto Message (TBD)
-        """
-
-    def get_ofp_port_info(device, port_no):
-        """
-        Retrieve the port info. This includes the ofp_port. The existing ofp
-        structure can be used, or all the attributes get added to the Port
-        definitions or a new proto definition gets created.  This API will allow
-        the Core to create a LogicalPort associated with this device.
-        :param device: device
-        :param port_no: port number
-        :return: Proto Message (TBD)
-        """
-
-    def process_inter_adapter_message(msg):
-        """
-        Called when the adapter receives a message that was sent to it directly
-        from another adapter. An adapter is automatically registered for these
-        messages when creating the inter-container kafka proxy. Note that it is
-        the responsibility of the sending and receiving adapters to properly encode
-        and decode the message.
-        :param msg: Proto Message (any)
-        :return: Proto Message Response
-        """
-
-
-class ICoreSouthBoundInterface(Interface):
-    """
-    Represents a Voltha Core. This is used by an adapter to initiate async
-    calls towards Voltha Core.
-    """
-
-    def get_device(device_id):
-        """
-        Retrieve a device using its ID.
-        :param device_id: a device ID
-        :return: Device Object or None
-        """
-
-    def get_child_device(parent_device_id, **kwargs):
-        """
-        Retrieve a child device object belonging to the specified parent
-        device based on some match criteria. The first child device that
-        matches the provided criteria is returned.
-        :param parent_device_id: parent's device protobuf ID
-        :param **kwargs: arbitrary list of match criteria where the Value
-        in each key-value pair must be a protobuf type
-        :return: Child Device Object or None
-        """
-
-    def get_ports(device_id, port_type):
-        """
-        Retrieve all the ports of a given type of a Device.
-        :param device_id: a device ID
-        :param port_type: type of port
-        :return Ports object
-        """
-
-    def get_child_devices(parent_device_id):
-        """
-        Get all child devices given a parent device id
-        :param parent_device_id: The parent device ID
-        :return: Devices object
-        """
-
-    def get_child_device_with_proxy_address(proxy_address):
-        """
-        Get a child device based on its proxy address. Proxy address is
-        defined as {parent id, channel_id}
-        :param proxy_address: A Device.ProxyAddress object
-        :return: Device object or None
-        """
-
-    def device_state_update(device_id,
-                            oper_status=None,
-                            connect_status=None):
-        """
-        Update a device state.
-        :param device_id: The device ID
-        :param oper_state: Operational state of device
-        :param conn_state: Connection state of device
-        :return: None
-        """
-
-    def child_device_detected(parent_device_id,
-                              parent_port_no,
-                              child_device_type,
-                              channel_id,
-                              **kw):
-        """
-        A child device has been detected.  Core will create the device along
-        with its unique ID.
-        :param parent_device_id: The parent device ID
-        :param parent_port_no: The parent port number
-        :param device_type: The child device type
-        :param channel_id: A unique identifier for that child device within
-        the parent device (e.g. vlan_id)
-        :param kw: A list of key-value pair where the value is a protobuf
-        message
-        :return: None
-        """
-
-    def device_update(device):
-        """
-        Event corresponding to a device update.
-        :param device: Device Object
-        :return: None
-        """
-
-    def child_device_removed(parent_device_id, child_device_id):
-        """
-        Event indicating a child device has been removed from a parent.
-        :param parent_device_id: Device ID of the parent
-        :param child_device_id: Device ID of the child
-        :return: None
-        """
-
-    def child_devices_state_update(parent_device_id,
-                                   oper_status=None,
-                                   connect_status=None,
-                                   admin_status=None):
-        """
-        Event indicating the status of all child devices have been changed.
-        :param parent_device_id: Device ID of the parent
-        :param oper_status: Operational status
-        :param connect_status: Connection status
-        :param admin_status: Admin status
-        :return: None
-        """
-
-    def child_devices_removed(parent_device_id):
-        """
-        Event indicating all child devices have been removed from a parent.
-        :param parent_device_id: Device ID of the parent device
-        :return: None
-        """
-
-    def device_pm_config_update(device_pm_config, init=False):
-        """
-        Event corresponding to a PM config update of a device.
-        :param device_pm_config: a PmConfigs object
-        :param init: True indicates initializing stage
-        :return: None
-        """
-
-    def port_created(device_id, port):
-        """
-        A port has been created and needs to be added to a device.
-        :param device_id: a device ID
-        :param port: Port object
-        :return None
-        """
-
-    def port_removed(device_id, port):
-        """
-        A port has been removed and it needs to be removed from a Device.
-        :param device_id: a device ID
-        :param port: a Port object
-        :return None
-        """
-
-    def ports_enabled(device_id):
-        """
-        All ports on that device have been re-enabled. The Core will change
-        the admin state to ENABLED and operational state to ACTIVE for all
-        ports on that device.
-        :param device_id: a device ID
-        :return: None
-        """
-
-    def ports_disabled(device_id):
-        """
-        All ports on that device have been disabled. The Core will change the
-        admin status to DISABLED and operational state to UNKNOWN for all
-        ports on that device.
-        :param device_id: a device ID
-        :return: None
-        """
-
-    def ports_oper_status_update(device_id, oper_status):
-        """
-        The operational status of all ports of a Device has been changed.
-        The Core will update the operational status for all ports on the
-        device.
-        :param device_id: a device ID
-        :param oper_status: operational Status
-        :return None
-        """
-
-    def image_download_update(img_dnld):
-        """
-        Event corresponding to an image download update.
-        :param img_dnld: a ImageDownload object
-        :return: None
-        """
-
-    def image_download_deleted(img_dnld):
-        """
-        Event corresponding to the deletion of a downloaded image. The
-        references of this image needs to be removed from the Core.
-        :param img_dnld: a ImageDownload object
-        :return: None
-        """
-
-    def packet_in(device_id, egress_port_no, packet):
-        """
-        Sends a packet to the SDN controller via voltha Core
-        :param device_id: The OLT device ID
-        :param egress_port_no: The port number representing the ONU (cvid)
-        :param packet: The actual packet
-         :return: None
-        """