[VOL-4291] Rw-core updates for gRPC migration

Change-Id: I8d5a554409115b29318089671ca4e1ab3fa98810
diff --git a/rw_core/core/adapter/agent.go b/rw_core/core/adapter/agent.go
index a6d8186..c2d45de 100644
--- a/rw_core/core/adapter/agent.go
+++ b/rw_core/core/adapter/agent.go
@@ -18,44 +18,108 @@
 
 import (
 	"context"
-	"github.com/golang/protobuf/ptypes"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"errors"
 	"sync"
 	"time"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc"
 )
 
 // agent represents adapter agent
 type agent struct {
-	adapter *voltha.Adapter
-	lock    sync.RWMutex
+	adapter            *voltha.Adapter
+	lock               sync.RWMutex
+	adapterAPIEndPoint string
+	vClient            *vgrpc.Client
+	adapterLock        sync.RWMutex
+	onAdapterRestart   vgrpc.RestartedHandler
+	liveProbeInterval  time.Duration
 }
 
-func newAdapterAgent(adapter *voltha.Adapter) *agent {
+func setAndTestAdapterServiceHandler(ctx context.Context, conn *grpc.ClientConn) interface{} {
+	svc := adapter_services.NewAdapterServiceClient(conn)
+	if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+		logger.Debugw(ctx, "connection-not-ready", log.Fields{"error": err, "health": h})
+		return nil
+	}
+	return svc
+}
+
+func newAdapterAgent(adapter *voltha.Adapter, onAdapterRestart vgrpc.RestartedHandler, liveProbeInterval time.Duration) *agent {
 	return &agent{
-		adapter: adapter,
+		adapter:            adapter,
+		onAdapterRestart:   onAdapterRestart,
+		adapterAPIEndPoint: adapter.Endpoint,
+		liveProbeInterval:  liveProbeInterval,
+	}
+}
+
+func (aa *agent) start(ctx context.Context) error {
+	// Establish grpc connection to Core
+	var err error
+	if aa.vClient, err = vgrpc.NewClient(aa.adapterAPIEndPoint,
+		aa.onAdapterRestart,
+		vgrpc.ActivityCheck(true)); err != nil {
+		return err
+	}
+
+	// Add a liveness communication update
+	aa.vClient.SubscribeForLiveness(aa.updateCommunicationTime)
+
+	go aa.vClient.Start(ctx, setAndTestAdapterServiceHandler)
+	return nil
+}
+
+func (aa *agent) stop(ctx context.Context) {
+	// Close the client
+	if aa.vClient != nil {
+		aa.vClient.Stop(ctx)
 	}
 }
 
 func (aa *agent) getAdapter(ctx context.Context) *voltha.Adapter {
-	aa.lock.RLock()
-	defer aa.lock.RUnlock()
-	logger.Debugw(ctx, "getAdapter", log.Fields{"adapter": aa.adapter})
+	aa.adapterLock.RLock()
+	defer aa.adapterLock.RUnlock()
 	return aa.adapter
 }
 
+func (aa *agent) getClient() (adapter_services.AdapterServiceClient, error) {
+	client, err := aa.vClient.GetClient()
+	if err != nil {
+		return nil, err
+	}
+	c, ok := client.(adapter_services.AdapterServiceClient)
+	if ok {
+		return c, nil
+	}
+	return nil, errors.New("invalid client returned")
+}
+
+func (aa *agent) resetConnection(ctx context.Context) {
+	if aa.vClient != nil {
+		aa.vClient.Reset(ctx)
+	}
+}
+
 // updateCommunicationTime updates the message to the specified time.
 // No attempt is made to save the time to the db, so only recent times are guaranteed to be accurate.
 func (aa *agent) updateCommunicationTime(new time.Time) {
 	// only update if new time is not in the future, and either the old time is invalid or new time > old time
 	aa.lock.Lock()
 	defer aa.lock.Unlock()
-	if last, err := ptypes.Timestamp(aa.adapter.LastCommunication); !new.After(time.Now()) && (err != nil || new.After(last)) {
-		timestamp, err := ptypes.TimestampProto(new)
-		if err != nil {
-			return // if the new time cannot be encoded, just ignore it
-		}
-
-		aa.adapter.LastCommunication = timestamp
+	timestamp := time.Unix(aa.adapter.LastCommunication, 0)
+	if !new.After(time.Now()) && new.After(timestamp) {
+		timestamp = new
+		aa.adapter.LastCommunication = timestamp.Unix()
 	}
 }
+
+func (aa *agent) IsConnectionUp() bool {
+	_, err := aa.getClient()
+	return err == nil
+}
diff --git a/rw_core/core/adapter/common.go b/rw_core/core/adapter/common.go
index 0c6b2c3..354ccc9 100644
--- a/rw_core/core/adapter/common.go
+++ b/rw_core/core/adapter/common.go
@@ -18,7 +18,7 @@
 package adapter
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/adapter/endpoint_manager.go b/rw_core/core/adapter/endpoint_manager.go
new file mode 100644
index 0000000..dac137c
--- /dev/null
+++ b/rw_core/core/adapter/endpoint_manager.go
@@ -0,0 +1,470 @@
+/*
+ * Copyright 2021-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.
+ */
+
+package adapter
+
+import (
+	"context"
+	"fmt"
+	"sync"
+
+	"github.com/buraksezer/consistent"
+	"github.com/cespare/xxhash"
+	"github.com/golang/protobuf/proto"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+const (
+	// All the values below can be tuned to get optimal data distribution.  The numbers below seems to work well when
+	// supporting 1000-10000 devices and 1 - 20 replicas of an adapter
+
+	// Keys are distributed among partitions. Prime numbers are good to distribute keys uniformly.
+	DefaultPartitionCount = 1117
+
+	// Represents how many times a node is replicated on the consistent ring.
+	DefaultReplicationFactor = 117
+
+	// Load is used to calculate average load.
+	DefaultLoad = 1.1
+)
+
+type Endpoint string // The gRPC endpoint of an adapter instance
+type ReplicaID int32 // The replication ID of an adapter instance
+
+type EndpointManager interface {
+
+	// Registers an adapter
+	RegisterAdapter(ctx context.Context, adapter *voltha.Adapter, deviceTypes *voltha.DeviceTypes) error
+
+	// GetEndpoint is called to get the endpoint to communicate with for a specific device and device type.
+	GetEndpoint(ctx context.Context, deviceID string, deviceType string) (Endpoint, error)
+
+	// IsDeviceOwnedByAdapter is invoked when a specific adapter (adapter type + replicaNumber) is restarted and
+	// devices owned by that adapter need to be reconciled
+	IsDeviceOwnedByAdapter(ctx context.Context, deviceID string, adapterType string, replicaNumber int32) (bool, error)
+
+	// GetReplicaAssignment returns the replica number of the adapter that owns the deviceID.  This is used by the
+	// test only
+	GetReplicaAssignment(ctx context.Context, deviceID string, adapterType string) (ReplicaID, error)
+}
+
+type adapterService struct {
+	adapterType    string // Type of the adapter.  The same type applies for all replicas of that adapter
+	totalReplicas  int32
+	replicas       map[ReplicaID]Endpoint
+	consistentRing *consistent.Consistent
+}
+
+type endpointManager struct {
+	partitionCount                    int
+	replicationFactor                 int
+	load                              float64
+	backend                           *db.Backend
+	adapterServices                   map[string]*adapterService
+	adapterServicesLock               sync.RWMutex
+	deviceTypeToAdapterServiceMap     map[string]string
+	deviceTypeToAdapterServiceMapLock sync.RWMutex
+}
+
+type EndpointManagerOption func(*endpointManager)
+
+func PartitionCount(count int) EndpointManagerOption {
+	return func(args *endpointManager) {
+		args.partitionCount = count
+	}
+}
+
+func ReplicationFactor(replicas int) EndpointManagerOption {
+	return func(args *endpointManager) {
+		args.replicationFactor = replicas
+	}
+}
+
+func Load(load float64) EndpointManagerOption {
+	return func(args *endpointManager) {
+		args.load = load
+	}
+}
+
+func newEndpointManager(backend *db.Backend, opts ...EndpointManagerOption) EndpointManager {
+	tm := &endpointManager{
+		partitionCount:                DefaultPartitionCount,
+		replicationFactor:             DefaultReplicationFactor,
+		load:                          DefaultLoad,
+		backend:                       backend,
+		adapterServices:               make(map[string]*adapterService),
+		deviceTypeToAdapterServiceMap: make(map[string]string),
+	}
+
+	for _, option := range opts {
+		option(tm)
+	}
+	return tm
+}
+
+func NewEndpointManager(backend *db.Backend, opts ...EndpointManagerOption) EndpointManager {
+	return newEndpointManager(backend, opts...)
+}
+
+func (ep *endpointManager) GetEndpoint(ctx context.Context, deviceID string, deviceType string) (Endpoint, error) {
+	logger.Debugw(ctx, "getting-endpoint", log.Fields{"device-id": deviceID, "device-type": deviceType})
+	owner, err := ep.getOwnerByDeviceType(ctx, deviceID, deviceType)
+	if err != nil {
+		return "", err
+	}
+	m, ok := owner.(Member)
+	if !ok {
+		return "", status.Errorf(codes.Aborted, "invalid-member-%v", owner)
+	}
+	endpoint := m.getEndPoint()
+	if endpoint == "" {
+		return "", status.Errorf(codes.Unavailable, "endpoint-not-set-%s", deviceType)
+	}
+	logger.Debugw(ctx, "returning-endpoint", log.Fields{"device-id": deviceID, "device-type": deviceType, "endpoint": endpoint})
+	return endpoint, nil
+}
+
+func (ep *endpointManager) IsDeviceOwnedByAdapter(ctx context.Context, deviceID string, adapterType string, replicaNumber int32) (bool, error) {
+	logger.Debugw(ctx, "device-ownership", log.Fields{"device-id": deviceID, "adapter-type": adapterType, "replica-number": replicaNumber})
+
+	serv, err := ep.getOwnerByAdapterType(ctx, deviceID, adapterType)
+	if err != nil {
+		return false, err
+	}
+	m, ok := serv.(Member)
+	if !ok {
+		return false, status.Errorf(codes.Aborted, "invalid-member-%v", serv)
+	}
+	return m.getReplica() == ReplicaID(replicaNumber), nil
+}
+
+func (ep *endpointManager) GetReplicaAssignment(ctx context.Context, deviceID string, adapterType string) (ReplicaID, error) {
+	owner, err := ep.getOwnerByAdapterType(ctx, deviceID, adapterType)
+	if err != nil {
+		return 0, nil
+	}
+	m, ok := owner.(Member)
+	if !ok {
+		return 0, status.Errorf(codes.Aborted, "invalid-member-%v", owner)
+	}
+	return m.getReplica(), nil
+}
+
+func (ep *endpointManager) getOwnerByDeviceType(ctx context.Context, deviceID string, deviceType string) (consistent.Member, error) {
+	serv, err := ep.getAdapterService(ctx, deviceType)
+	if err != nil {
+		return nil, err
+	}
+	key := ep.makeKey(deviceID, deviceType, serv.adapterType)
+	return serv.consistentRing.LocateKey(key), nil
+}
+
+func (ep *endpointManager) getOwnerByAdapterType(ctx context.Context, deviceID string, adapterType string) (consistent.Member, error) {
+	// Check whether the adapter exist
+	ep.adapterServicesLock.RLock()
+	serv, adapterExist := ep.adapterServices[adapterType]
+	ep.adapterServicesLock.RUnlock()
+
+	if !adapterExist {
+		// Sync from the dB
+		if err := ep.loadAdapterServices(ctx); err != nil {
+			return nil, err
+		}
+		// Check again
+		ep.adapterServicesLock.RLock()
+		serv, adapterExist = ep.adapterServices[adapterType]
+		ep.adapterServicesLock.RUnlock()
+		if !adapterExist {
+			return nil, fmt.Errorf("adapter-type-not-exist-%s", adapterType)
+		}
+	}
+
+	// Get the device type
+	deviceType := ""
+	ep.deviceTypeToAdapterServiceMapLock.RLock()
+	for dType, aType := range ep.deviceTypeToAdapterServiceMap {
+		if aType == adapterType {
+			deviceType = dType
+			break
+		}
+	}
+	ep.deviceTypeToAdapterServiceMapLock.RUnlock()
+
+	if deviceType == "" {
+		return nil, fmt.Errorf("device-type-not-exist-for-adapter-type-%s", adapterType)
+	}
+
+	owner := serv.consistentRing.LocateKey(ep.makeKey(deviceID, deviceType, serv.adapterType))
+	m, ok := owner.(Member)
+	if !ok {
+		return nil, status.Errorf(codes.Aborted, "invalid-member-%v", owner)
+	}
+	return m, nil
+}
+
+func (ep *endpointManager) getAdapterService(ctx context.Context, deviceType string) (*adapterService, error) {
+	// First get the adapter type for that device type
+	adapterType := ""
+	ep.deviceTypeToAdapterServiceMapLock.RLock()
+	for dType, aType := range ep.deviceTypeToAdapterServiceMap {
+		if dType == deviceType {
+			adapterType = aType
+			break
+		}
+	}
+	ep.deviceTypeToAdapterServiceMapLock.RUnlock()
+
+	// Check whether the adapter exist
+	adapterExist := false
+	var aServ *adapterService
+	if adapterType != "" {
+		ep.adapterServicesLock.RLock()
+		aServ, adapterExist = ep.adapterServices[adapterType]
+		ep.adapterServicesLock.RUnlock()
+	}
+
+	// Load the service and device types if not found, i.e. sync up with the dB
+	if !adapterExist || aServ == nil || int(aServ.totalReplicas) != len(aServ.consistentRing.GetMembers()) {
+		if err := ep.loadAdapterServices(ctx); err != nil {
+			return nil, err
+		}
+
+		// Get the adapter type if it was empty before
+		if adapterType == "" {
+			ep.deviceTypeToAdapterServiceMapLock.RLock()
+			for dType, aType := range ep.deviceTypeToAdapterServiceMap {
+				if dType == deviceType {
+					adapterType = aType
+					break
+				}
+			}
+			ep.deviceTypeToAdapterServiceMapLock.RUnlock()
+		}
+		// Error put if the adapter type is not set
+		if adapterType == "" {
+			return nil, fmt.Errorf("adapter-service-not-found-for-device-type-%s", deviceType)
+		}
+
+		// Get the service
+		ep.adapterServicesLock.RLock()
+		aServ, adapterExist = ep.adapterServices[adapterType]
+		ep.adapterServicesLock.RUnlock()
+	}
+
+	// Sanity check
+	if !adapterExist || aServ == nil || int(aServ.totalReplicas) != len(aServ.consistentRing.GetMembers()) {
+		return nil, fmt.Errorf("adapter-service-not-found-for-device-type-%s", deviceType)
+	}
+
+	return aServ, nil
+}
+
+func (ep *endpointManager) getConsistentConfig() consistent.Config {
+	return consistent.Config{
+		PartitionCount:    ep.partitionCount,
+		ReplicationFactor: ep.replicationFactor,
+		Load:              ep.load,
+		Hasher:            hasher{},
+	}
+}
+
+// loadAdapterServices loads the services (adapters) and device types in memory. Because of the small size of the data and
+// the data format in the dB being binary protobuf then it is better to load all the data if inconsistency is detected,
+// instead of watching for updates in the dB and acting on it.
+func (ep *endpointManager) loadAdapterServices(ctx context.Context) error {
+	ep.adapterServicesLock.Lock()
+	defer ep.adapterServicesLock.Unlock()
+	ep.deviceTypeToAdapterServiceMapLock.Lock()
+	defer ep.deviceTypeToAdapterServiceMapLock.Unlock()
+
+	if ep.backend == nil {
+		return status.Error(codes.Aborted, "backend-not-set")
+	}
+
+	ep.adapterServices = make(map[string]*adapterService)
+	ep.deviceTypeToAdapterServiceMap = make(map[string]string)
+
+	// Load the adapters
+	blobs, err := ep.backend.List(log.WithSpanFromContext(context.Background(), ctx), "adapters")
+	if err != nil {
+		return err
+	}
+
+	// Data is marshalled as proto bytes in the data store
+	for _, blob := range blobs {
+		data := blob.Value.([]byte)
+		adapter := &voltha.Adapter{}
+		if err := proto.Unmarshal(data, adapter); err != nil {
+			return err
+		}
+		// A valid adapter should have the vendorID set
+		if err := ep.setupAdapterWithLock(ctx, adapter); err != nil {
+			logger.Errorw(ctx, "missing vendor id", log.Fields{"adapter": adapter})
+		}
+	}
+	// Load the device types
+	blobs, err = ep.backend.List(log.WithSpanFromContext(context.Background(), ctx), "device_types")
+	if err != nil {
+		return err
+	}
+	for _, blob := range blobs {
+		data := blob.Value.([]byte)
+		deviceType := &voltha.DeviceType{}
+		if err := proto.Unmarshal(data, deviceType); err != nil {
+			return err
+		}
+		ep.addDeviceTypeWithLock(deviceType)
+	}
+
+	ep.printServices(ctx)
+	return nil
+}
+
+func (ep *endpointManager) printServices(ctx context.Context) {
+	if logger.V(log.DebugLevel) {
+		for key, val := range ep.adapterServices {
+			members := val.consistentRing.GetMembers()
+			logger.Debugw(ctx, "adapter-service", log.Fields{"service": key, "expected-replica": val.totalReplicas, "replicas": len(val.consistentRing.GetMembers())})
+			for _, m := range members {
+				n := m.(Member)
+				logger.Debugw(ctx, "adapter-instance-registered", log.Fields{"service-id": n.getID(), "adapter-type": n.getAdapterType(), "replica": n.getReplica(), "endpoint": n.getEndPoint()})
+			}
+		}
+		logger.Debugw(ctx, "device-types", log.Fields{"device-types": ep.deviceTypeToAdapterServiceMap})
+	}
+}
+
+func (ep *endpointManager) RegisterAdapter(ctx context.Context, adapter *voltha.Adapter, deviceTypes *voltha.DeviceTypes) error {
+	ep.adapterServicesLock.Lock()
+	defer ep.adapterServicesLock.Unlock()
+	ep.deviceTypeToAdapterServiceMapLock.Lock()
+	defer ep.deviceTypeToAdapterServiceMapLock.Unlock()
+
+	if err := ep.setupAdapterWithLock(ctx, adapter); err != nil {
+		return err
+	}
+	ep.addDeviceTypesWithLock(deviceTypes)
+	ep.printServices(ctx)
+	return nil
+}
+
+func (ep *endpointManager) setupAdapterWithLock(ctx context.Context, adapter *voltha.Adapter) error {
+	// Build the consistent ring for that adapter
+	if adapter.Vendor != "" {
+		if _, ok := ep.adapterServices[adapter.Type]; !ok {
+			ep.adapterServices[adapter.Type] = &adapterService{
+				adapterType:    adapter.Type,
+				totalReplicas:  adapter.TotalReplicas,
+				replicas:       make(map[ReplicaID]Endpoint),
+				consistentRing: consistent.New(nil, ep.getConsistentConfig()),
+			}
+
+		}
+		currentReplica := ReplicaID(adapter.CurrentReplica)
+		endpoint := Endpoint(adapter.Endpoint)
+		ep.adapterServices[adapter.Type].replicas[currentReplica] = endpoint
+		ep.adapterServices[adapter.Type].consistentRing.Add(newMember(adapter.Id, adapter.Type, adapter.Vendor, endpoint, adapter.Version, currentReplica))
+	} else {
+		logger.Errorw(ctx, "missing-vendor-id", log.Fields{"adapter": adapter})
+		return fmt.Errorf("missing vendor id for %s adapter", adapter.Id)
+	}
+	return nil
+}
+
+func (ep *endpointManager) addDeviceTypesWithLock(deviceTypes *voltha.DeviceTypes) {
+	// Update the device types
+	for _, deviceType := range deviceTypes.Items {
+		if _, ok := ep.deviceTypeToAdapterServiceMap[deviceType.Id]; !ok {
+			ep.deviceTypeToAdapterServiceMap[deviceType.Id] = deviceType.AdapterType
+		}
+	}
+}
+
+func (ep *endpointManager) addDeviceTypeWithLock(deviceType *voltha.DeviceType) {
+	if _, ok := ep.deviceTypeToAdapterServiceMap[deviceType.Id]; !ok {
+		ep.deviceTypeToAdapterServiceMap[deviceType.Id] = deviceType.AdapterType
+	}
+}
+
+// makeKey creates the string that the hash function uses to create the hash
+// In most cases, a deviceType is the same as a serviceType.  It is being differentiated here to allow a
+// serviceType to support multiple device types
+func (ep *endpointManager) makeKey(deviceID string, deviceType string, serviceType string) []byte {
+	return []byte(fmt.Sprintf("%s_%s_%s", serviceType, deviceType, deviceID))
+}
+
+// The consistent package requires a hasher function
+type hasher struct{}
+
+// Sum64 provides the hasher function.  Based upon numerous testing scenarios, the xxhash package seems to provide the
+// best distribution compare to other hash packages
+func (h hasher) Sum64(data []byte) uint64 {
+	return xxhash.Sum64(data)
+}
+
+// Member represents a member on the consistent ring
+type Member interface {
+	String() string
+	getReplica() ReplicaID
+	getEndPoint() Endpoint
+	getID() string
+	getAdapterType() string
+}
+
+// member implements the Member interface
+type member struct {
+	id          string
+	adapterType string
+	vendor      string
+	version     string
+	replica     ReplicaID
+	endpoint    Endpoint
+}
+
+func newMember(id string, adapterType string, vendor string, endPoint Endpoint, version string, replica ReplicaID) Member {
+	return &member{
+		id:          id,
+		adapterType: adapterType,
+		vendor:      vendor,
+		version:     version,
+		replica:     replica,
+		endpoint:    endPoint,
+	}
+}
+
+func (m *member) String() string {
+	return string(m.endpoint)
+}
+
+func (m *member) getReplica() ReplicaID {
+	return m.replica
+}
+
+func (m *member) getEndPoint() Endpoint {
+	return m.endpoint
+}
+
+func (m *member) getID() string {
+	return m.id
+}
+
+func (m *member) getAdapterType() string {
+	return m.adapterType
+}
diff --git a/rw_core/core/adapter/endpoint_manager_test.go b/rw_core/core/adapter/endpoint_manager_test.go
new file mode 100644
index 0000000..78635d1
--- /dev/null
+++ b/rw_core/core/adapter/endpoint_manager_test.go
@@ -0,0 +1,258 @@
+/*
+ * Copyright 2021-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.
+ */
+package adapter
+
+import (
+	"context"
+	"fmt"
+	"math"
+	"strconv"
+	"testing"
+	"time"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/google/uuid"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/mocks/etcd"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"github.com/phayes/freeport"
+	"github.com/stretchr/testify/assert"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+type EPTest struct {
+	etcdServer  *etcd.EtcdServer
+	backend     *db.Backend
+	maxReplicas int
+	minReplicas int
+}
+
+func newEPTest(minReplicas, maxReplicas int) *EPTest {
+	ctx := context.Background()
+	test := &EPTest{
+		minReplicas: minReplicas,
+		maxReplicas: maxReplicas,
+	}
+
+	// Create backend
+	if err := test.initBackend(); err != nil {
+		logger.Fatalw(ctx, "setting-backend-failed", log.Fields{"error": err})
+	}
+
+	// Populate backend with data
+	if err := test.populateBackend(); err != nil {
+		logger.Fatalw(ctx, "populating-db-failed", log.Fields{"error": err})
+	}
+	return test
+}
+
+func (ep *EPTest) initBackend() error {
+	ctx := context.Background()
+	configName := "voltha-go.adapter.ep.test"
+	storageDir := "voltha-go.adapter.ep.etcd"
+	logLevel := "error"
+	timeout := 5 * time.Second
+
+	kvClientPort, err := freeport.GetFreePort()
+	if err != nil {
+		return err
+	}
+	peerPort, err := freeport.GetFreePort()
+	if err != nil {
+		return err
+	}
+	ep.etcdServer = etcd.StartEtcdServer(ctx, etcd.MKConfig(ctx, configName, kvClientPort, peerPort, storageDir, logLevel))
+	if ep.etcdServer == nil {
+		return status.Error(codes.Internal, "Embedded server failed to start")
+	}
+
+	ep.backend = db.NewBackend(ctx, "etcd", "127.0.0.1"+":"+strconv.Itoa(kvClientPort), timeout, "service/voltha")
+	return nil
+}
+
+func (ep *EPTest) stopAll() {
+	if ep.etcdServer != nil {
+		ep.etcdServer.Stop(context.Background())
+	}
+}
+
+func (ep *EPTest) populateBackend() error {
+	// Add an adapter with multiple replicas
+	adapterPrefix := "adapter_brcm_openomci_onu"
+	numReplicas := ep.maxReplicas
+	for i := 0; i < numReplicas; i++ {
+		adapter := &voltha.Adapter{
+			Id:             fmt.Sprintf("%s_%d", adapterPrefix, i),
+			Vendor:         "VOLTHA OpenONU",
+			Version:        "2.4.0-dev0",
+			Type:           adapterPrefix,
+			CurrentReplica: int32(i),
+			TotalReplicas:  int32(numReplicas),
+			Endpoint:       fmt.Sprintf("%s_%d", adapterPrefix, i),
+		}
+		adapterKVKey := fmt.Sprintf("%s/%d", adapterPrefix, i)
+		blob, err := proto.Marshal(adapter)
+		if err != nil {
+			return err
+		}
+		if err := ep.backend.Put(context.Background(), "adapters/"+adapterKVKey, blob); err != nil {
+			return err
+		}
+	}
+
+	// Add an adapter with minreplicas
+	adapterPrefix = "adapter_openolt"
+	numReplicas = ep.minReplicas
+	for i := 0; i < numReplicas; i++ {
+		adapter := &voltha.Adapter{
+			Id:             fmt.Sprintf("%s_%d", adapterPrefix, i),
+			Vendor:         "VOLTHA OpenOLT",
+			Version:        "2.3.1-dev",
+			Type:           adapterPrefix,
+			CurrentReplica: int32(i),
+			TotalReplicas:  int32(numReplicas),
+			Endpoint:       fmt.Sprintf("%s_%d", adapterPrefix, i),
+		}
+		adapterKVKey := fmt.Sprintf("%s/%d", adapterPrefix, i)
+		blob, err := proto.Marshal(adapter)
+		if err != nil {
+			return err
+		}
+		if err := ep.backend.Put(context.Background(), "adapters/"+adapterKVKey, blob); err != nil {
+			return err
+		}
+	}
+
+	// Add the brcm_openomci_onu device type
+	dType := "brcm_openomci_onu"
+	adapterName := "adapter_brcm_openomci_onu"
+	deviceType := &voltha.DeviceType{
+		Id:                          dType,
+		VendorIds:                   []string{"OPEN", "ALCL", "BRCM", "TWSH", "ALPH", "ISKT", "SFAA", "BBSM", "SCOM", "ARPX", "DACM", "ERSN", "HWTC", "CIGG"},
+		AdapterType:                 adapterName,
+		AcceptsAddRemoveFlowUpdates: true,
+	}
+	blob, err := proto.Marshal(deviceType)
+	if err != nil {
+		return err
+	}
+	if err := ep.backend.Put(context.Background(), "device_types/"+deviceType.Id, blob); err != nil {
+		return err
+	}
+
+	// Add the openolt device type
+	dType = "openolt"
+	adapterName = "adapter_openolt"
+	deviceType = &voltha.DeviceType{
+		Id:                          dType,
+		AdapterType:                 adapterName,
+		AcceptsAddRemoveFlowUpdates: true,
+	}
+	blob, err = proto.Marshal(deviceType)
+	if err != nil {
+		return err
+	}
+	if err := ep.backend.Put(context.Background(), "device_types/"+deviceType.Id, blob); err != nil {
+		return err
+	}
+	return nil
+}
+
+func getMeanAndStdDeviation(val []int, replicas int) (float64, float64) {
+	var sum, mean, sd float64
+	for i := 0; i < replicas; i++ {
+		sum += float64(val[i])
+	}
+	mean = sum / float64(replicas)
+
+	for j := 0; j < replicas; j++ {
+		sd += math.Pow(float64(val[j])-mean, 2)
+	}
+	sd = math.Sqrt(sd / float64(replicas))
+	return mean, sd
+}
+
+func (ep *EPTest) testEndpointManagerAPIs(t *testing.T, tm EndpointManager, adapterType string, deviceType string, replicas int) {
+	ctx := context.Background()
+	// Map of device ids to topic
+	deviceIDs := make(map[string]Endpoint)
+	numDevices := 1000
+	total := make([]int, replicas)
+	for i := 0; i < numDevices; i++ {
+		deviceID := uuid.New().String()
+		endpoint, err := tm.GetEndpoint(ctx, deviceID, deviceType)
+		if err != nil {
+			logger.Fatalw(ctx, "error-getting-endpoint", log.Fields{"error": err})
+		}
+		deviceIDs[deviceID] = endpoint
+		replicaID, err := tm.GetReplicaAssignment(ctx, deviceID, adapterType)
+		if err != nil {
+			logger.Fatalw(ctx, "error-getting-endpoint", log.Fields{"error": err})
+		}
+		total[replicaID]++
+	}
+
+	mean, sdtDev := getMeanAndStdDeviation(total, replicas)
+	fmt.Printf("Device distributions => devices:%d adapter_replicas:%d mean:%d standard_deviation:%d, distributions:%v\n", numDevices, replicas, int(mean), int(sdtDev), total)
+
+	// Verify that we get the same topic for a given device ID, irrespective of the number of iterations
+	numIterations := 10
+	for i := 0; i < numIterations; i++ {
+		for deviceID, expectedEndpoint := range deviceIDs {
+			endpointByAdapterType, err := tm.GetEndpoint(ctx, deviceID, deviceType)
+			if err != nil {
+				logger.Fatalw(ctx, "error-getting-endpoint", log.Fields{"error": err})
+			}
+			assert.Equal(t, expectedEndpoint, endpointByAdapterType)
+		}
+	}
+
+	// Verify that a device belong to the correct node
+	for deviceID := range deviceIDs {
+		replicaID, err := tm.GetReplicaAssignment(ctx, deviceID, adapterType)
+		if err != nil {
+			logger.Fatalw(ctx, "error-getting-topic", log.Fields{"error": err})
+		}
+		for k := 0; k < replicas; k++ {
+			owned, err := tm.IsDeviceOwnedByAdapter(ctx, deviceID, adapterType, int32(k))
+			if err != nil {
+				logger.Fatalw(ctx, "error-verifying-device-ownership", log.Fields{"error": err})
+			}
+			assert.Equal(t, ReplicaID(k) == replicaID, owned)
+		}
+	}
+}
+
+func TestEndpointManagerSuite(t *testing.T) {
+	tmt := newEPTest(1, 10)
+	assert.NotNil(t, tmt)
+
+	tm := NewEndpointManager(
+		tmt.backend,
+		PartitionCount(1117),
+		ReplicationFactor(200),
+		Load(1.1))
+
+	defer tmt.stopAll()
+
+	//1. Test APIs with multiple replicas
+	tmt.testEndpointManagerAPIs(t, tm, "adapter_brcm_openomci_onu", "brcm_openomci_onu", tmt.maxReplicas)
+
+	//2. Test APIs with single replica
+	tmt.testEndpointManagerAPIs(t, tm, "adapter_openolt", "openolt", tmt.minReplicas)
+}
diff --git a/rw_core/core/adapter/manager.go b/rw_core/core/adapter/manager.go
index 772ff75..b592842 100644
--- a/rw_core/core/adapter/manager.go
+++ b/rw_core/core/adapter/manager.go
@@ -18,129 +18,149 @@
 
 import (
 	"context"
+	"errors"
 	"fmt"
 	"sync"
 	"time"
 
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+
 	"github.com/gogo/protobuf/proto"
 	"github.com/golang/protobuf/ptypes/empty"
 	"github.com/opencord/voltha-go/db/model"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-lib-go/v5/pkg/probe"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
 
 // Manager represents adapter manager attributes
 type Manager struct {
-	adapterAgents               map[string]*agent
-	deviceTypes                 map[string]*voltha.DeviceType
-	adapterProxy                *model.Proxy
-	deviceTypeProxy             *model.Proxy
-	onAdapterRestart            adapterRestartedHandler
-	coreInstanceID              string
-	lockAdaptersMap             sync.RWMutex
-	lockdDeviceTypeToAdapterMap sync.RWMutex
+	adapterAgents           map[string]*agent
+	adapterEndpoints        map[Endpoint]*agent
+	deviceTypes             map[string]*voltha.DeviceType
+	adapterDbProxy          *model.Proxy
+	deviceTypeDbProxy       *model.Proxy
+	onAdapterRestart        vgrpc.RestartedHandler
+	endpointMgr             EndpointManager
+	lockAdapterAgentsMap    sync.RWMutex
+	lockDeviceTypesMap      sync.RWMutex
+	lockAdapterEndPointsMap sync.RWMutex
+	liveProbeInterval       time.Duration
 }
 
-func NewAdapterManager(ctx context.Context, dbPath *model.Path, coreInstanceID string, kafkaClient kafka.Client) *Manager {
-	aMgr := &Manager{
-		coreInstanceID:  coreInstanceID,
-		adapterProxy:    dbPath.Proxy("adapters"),
-		deviceTypeProxy: dbPath.Proxy("device_types"),
-		deviceTypes:     make(map[string]*voltha.DeviceType),
-		adapterAgents:   make(map[string]*agent),
-	}
-	kafkaClient.SubscribeForMetadata(ctx, aMgr.updateLastAdapterCommunication)
-	return aMgr
-}
-
-// an interface type for callbacks
-// if more than one callback is required, this should be converted to a proper interface
-type adapterRestartedHandler func(ctx context.Context, adapter *voltha.Adapter) error
-
-func (aMgr *Manager) SetAdapterRestartedCallback(onAdapterRestart adapterRestartedHandler) {
+// SetAdapterRestartedCallback is used to set the callback that needs to be invoked on an adapter restart
+func (aMgr *Manager) SetAdapterRestartedCallback(onAdapterRestart vgrpc.RestartedHandler) {
 	aMgr.onAdapterRestart = onAdapterRestart
 }
 
-func (aMgr *Manager) Start(ctx context.Context) {
-	probe.UpdateStatusFromContext(ctx, "adapter-manager", probe.ServiceStatusPreparing)
-	logger.Info(ctx, "starting-adapter-manager")
+func NewAdapterManager(
+	dbPath *model.Path,
+	coreInstanceID string,
+	backend *db.Backend,
+	liveProbeInterval time.Duration,
+) *Manager {
+	return &Manager{
+		adapterDbProxy:    dbPath.Proxy("adapters"),
+		deviceTypeDbProxy: dbPath.Proxy("device_types"),
+		deviceTypes:       make(map[string]*voltha.DeviceType),
+		adapterAgents:     make(map[string]*agent),
+		adapterEndpoints:  make(map[Endpoint]*agent),
+		endpointMgr:       NewEndpointManager(backend),
+		liveProbeInterval: liveProbeInterval,
+	}
+}
+
+func (aMgr *Manager) Start(ctx context.Context, serviceName string) {
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusPreparing)
+	logger.Infow(ctx, "starting-service", log.Fields{"service": serviceName})
 
 	// Load the existing adapterAgents and device types - this will also ensure the correct paths have been
 	// created if there are no data in the dB to start
 	err := aMgr.loadAdaptersAndDevicetypesInMemory(ctx)
 	if err != nil {
-		logger.Fatalf(ctx, "failed-to-load-adapters-and-device-types-in-memory: %s", err)
+		logger.Fatalw(ctx, "failed-to-load-adapters-and-device-types-in-memory", log.Fields{"service": serviceName, "error": err})
 	}
 
-	probe.UpdateStatusFromContext(ctx, "adapter-manager", probe.ServiceStatusRunning)
-	logger.Info(ctx, "adapter-manager-started")
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+	logger.Infow(ctx, "service-started", log.Fields{"service": serviceName})
 }
 
-//loadAdaptersAndDevicetypesInMemory loads the existing set of adapters and device types in memory
-func (aMgr *Manager) loadAdaptersAndDevicetypesInMemory(ctx context.Context) error {
-	// Load the adapters
-	var adapters []*voltha.Adapter
-	if err := aMgr.adapterProxy.List(log.WithSpanFromContext(context.Background(), ctx), &adapters); err != nil {
-		logger.Errorw(ctx, "Failed-to-list-adapters-from-cluster-data-proxy", log.Fields{"error": err})
-		return err
+func (aMgr *Manager) Stop(ctx context.Context) {
+	//	Stop all adapters
+	aMgr.lockAdapterAgentsMap.RLock()
+	defer aMgr.lockAdapterAgentsMap.RUnlock()
+	for _, adapterAgent := range aMgr.adapterAgents {
+		adapterAgent.stop(ctx)
 	}
-	if len(adapters) != 0 {
-		for _, adapter := range adapters {
-			if err := aMgr.addAdapter(ctx, adapter, false); err != nil {
-				logger.Errorw(ctx, "failed to add adapter", log.Fields{"adapterId": adapter.Id})
-			} else {
-				logger.Debugw(ctx, "adapter added successfully", log.Fields{"adapterId": adapter.Id})
-			}
-		}
-	}
-
-	// Load the device types
-	var deviceTypes []*voltha.DeviceType
-	if err := aMgr.deviceTypeProxy.List(log.WithSpanFromContext(context.Background(), ctx), &deviceTypes); err != nil {
-		logger.Errorw(ctx, "Failed-to-list-device-types-from-cluster-data-proxy", log.Fields{"error": err})
-		return err
-	}
-	if len(deviceTypes) != 0 {
-		dTypes := &voltha.DeviceTypes{Items: []*voltha.DeviceType{}}
-		for _, dType := range deviceTypes {
-			logger.Debugw(ctx, "found-existing-device-types", log.Fields{"deviceTypes": dTypes})
-			dTypes.Items = append(dTypes.Items, dType)
-		}
-		return aMgr.addDeviceTypes(ctx, dTypes, false)
-	}
-
-	logger.Debug(ctx, "no-existing-device-type-found")
-
-	return nil
 }
 
-func (aMgr *Manager) updateLastAdapterCommunication(adapterID string, timestamp time.Time) {
-	aMgr.lockAdaptersMap.RLock()
-	adapterAgent, have := aMgr.adapterAgents[adapterID]
-	aMgr.lockAdaptersMap.RUnlock()
+func (aMgr *Manager) GetAdapterEndpoint(ctx context.Context, deviceID string, deviceType string) (string, error) {
+	endPoint, err := aMgr.endpointMgr.GetEndpoint(ctx, deviceID, deviceType)
+	if err != nil {
+		return "", err
+	}
+	return string(endPoint), nil
+}
+
+func (aMgr *Manager) GetAdapterWithEndpoint(ctx context.Context, endPoint string) (*voltha.Adapter, error) {
+	aMgr.lockAdapterEndPointsMap.RLock()
+	agent, have := aMgr.adapterEndpoints[Endpoint(endPoint)]
+	aMgr.lockAdapterEndPointsMap.RUnlock()
 
 	if have {
-		adapterAgent.updateCommunicationTime(timestamp)
+		return agent.getAdapter(ctx), nil
 	}
+
+	return nil, errors.New("Not found")
+}
+
+func (aMgr *Manager) GetAdapterNameWithEndpoint(ctx context.Context, endPoint string) (string, error) {
+	aMgr.lockAdapterEndPointsMap.RLock()
+	agent, have := aMgr.adapterEndpoints[Endpoint(endPoint)]
+	aMgr.lockAdapterEndPointsMap.RUnlock()
+
+	if have {
+		return agent.adapter.Id, nil
+	}
+
+	return "", errors.New("Not found")
+}
+
+func (aMgr *Manager) GetAdapterClient(_ context.Context, endpoint string) (adapter_services.AdapterServiceClient, error) {
+	if endpoint == "" {
+		return nil, errors.New("endpoint-cannot-be-empty")
+	}
+	aMgr.lockAdapterEndPointsMap.RLock()
+	defer aMgr.lockAdapterEndPointsMap.RUnlock()
+
+	if agent, have := aMgr.adapterEndpoints[Endpoint(endpoint)]; have {
+		return agent.getClient()
+	}
+
+	return nil, fmt.Errorf("Endpoint-not-found-%s", endpoint)
 }
 
 func (aMgr *Manager) addAdapter(ctx context.Context, adapter *voltha.Adapter, saveToDb bool) error {
-	aMgr.lockAdaptersMap.Lock()
-	defer aMgr.lockAdaptersMap.Unlock()
+	aMgr.lockAdapterAgentsMap.Lock()
+	aMgr.lockAdapterEndPointsMap.Lock()
+	defer aMgr.lockAdapterEndPointsMap.Unlock()
+	defer aMgr.lockAdapterAgentsMap.Unlock()
 	logger.Debugw(ctx, "adding-adapter", log.Fields{"adapterId": adapter.Id, "vendor": adapter.Vendor,
 		"currentReplica": adapter.CurrentReplica, "totalReplicas": adapter.TotalReplicas, "endpoint": adapter.Endpoint})
 	if _, exist := aMgr.adapterAgents[adapter.Id]; !exist {
 		if saveToDb {
 			// Save the adapter to the KV store - first check if it already exist
-			if have, err := aMgr.adapterProxy.Get(log.WithSpanFromContext(context.Background(), ctx), adapter.Id, &voltha.Adapter{}); err != nil {
+			if have, err := aMgr.adapterDbProxy.Get(log.WithSpanFromContext(context.Background(), ctx), adapter.Id, &voltha.Adapter{}); err != nil {
 				logger.Errorw(ctx, "failed-to-get-adapters-from-cluster-proxy", log.Fields{"error": err})
 				return err
 			} else if !have {
-				if err := aMgr.adapterProxy.Set(log.WithSpanFromContext(context.Background(), ctx), adapter.Id, adapter); err != nil {
+				if err := aMgr.adapterDbProxy.Set(log.WithSpanFromContext(context.Background(), ctx), adapter.Id, adapter); err != nil {
 					logger.Errorw(ctx, "failed-to-save-adapter", log.Fields{"adapterId": adapter.Id, "vendor": adapter.Vendor,
 						"currentReplica": adapter.CurrentReplica, "totalReplicas": adapter.TotalReplicas, "endpoint": adapter.Endpoint, "replica": adapter.CurrentReplica, "total": adapter.TotalReplicas})
 					return err
@@ -155,7 +175,11 @@
 			}
 		}
 		clonedAdapter := (proto.Clone(adapter)).(*voltha.Adapter)
-		aMgr.adapterAgents[adapter.Id] = newAdapterAgent(clonedAdapter)
+		// Use a muted adapter restart handler which is invoked by the corresponding gRPC client on an adapter restart.
+		// This handler just log the restart event.  The actual action taken following an adapter restart
+		// will be done when an adapter re-registers itself.
+		aMgr.adapterAgents[adapter.Id] = newAdapterAgent(clonedAdapter, aMgr.mutedAdapterRestartedHandler, aMgr.liveProbeInterval)
+		aMgr.adapterEndpoints[Endpoint(adapter.Endpoint)] = aMgr.adapterAgents[adapter.Id]
 	}
 	return nil
 }
@@ -165,10 +189,10 @@
 		return fmt.Errorf("no-device-type")
 	}
 	logger.Debugw(ctx, "adding-device-types", log.Fields{"deviceTypes": deviceTypes})
-	aMgr.lockAdaptersMap.Lock()
-	defer aMgr.lockAdaptersMap.Unlock()
-	aMgr.lockdDeviceTypeToAdapterMap.Lock()
-	defer aMgr.lockdDeviceTypeToAdapterMap.Unlock()
+	aMgr.lockAdapterAgentsMap.Lock()
+	defer aMgr.lockAdapterAgentsMap.Unlock()
+	aMgr.lockDeviceTypesMap.Lock()
+	defer aMgr.lockDeviceTypesMap.Unlock()
 
 	// create an in memory map to fetch the entire voltha.DeviceType from a device.Type string
 	for _, deviceType := range deviceTypes.Items {
@@ -178,13 +202,13 @@
 	if saveToDb {
 		// Save the device types to the KV store
 		for _, deviceType := range deviceTypes.Items {
-			if have, err := aMgr.deviceTypeProxy.Get(log.WithSpanFromContext(context.Background(), ctx), deviceType.Id, &voltha.DeviceType{}); err != nil {
+			if have, err := aMgr.deviceTypeDbProxy.Get(log.WithSpanFromContext(context.Background(), ctx), deviceType.Id, &voltha.DeviceType{}); err != nil {
 				logger.Errorw(ctx, "Failed-to--device-types-from-cluster-data-proxy", log.Fields{"error": err})
 				return err
 			} else if !have {
 				//	Does not exist - save it
 				clonedDType := (proto.Clone(deviceType)).(*voltha.DeviceType)
-				if err := aMgr.deviceTypeProxy.Set(log.WithSpanFromContext(context.Background(), ctx), deviceType.Id, clonedDType); err != nil {
+				if err := aMgr.deviceTypeDbProxy.Set(log.WithSpanFromContext(context.Background(), ctx), deviceType.Id, clonedDType); err != nil {
 					logger.Errorw(ctx, "Failed-to-add-device-types-to-cluster-data-proxy", log.Fields{"error": err})
 					return err
 				}
@@ -192,35 +216,71 @@
 			}
 		}
 	}
-
 	return nil
 }
 
-// ListAdapters returns the contents of all adapters known to the system
-func (aMgr *Manager) ListAdapters(ctx context.Context, _ *empty.Empty) (*voltha.Adapters, error) {
-	result := &voltha.Adapters{Items: []*voltha.Adapter{}}
-	aMgr.lockAdaptersMap.RLock()
-	defer aMgr.lockAdaptersMap.RUnlock()
-	for _, adapterAgent := range aMgr.adapterAgents {
-		if a := adapterAgent.getAdapter(ctx); a != nil {
-			result.Items = append(result.Items, (proto.Clone(a)).(*voltha.Adapter))
+//loadAdaptersAndDevicetypesInMemory loads the existing set of adapters and device types in memory
+func (aMgr *Manager) loadAdaptersAndDevicetypesInMemory(ctx context.Context) error {
+	// Load the adapters
+	var adapters []*voltha.Adapter
+	if err := aMgr.adapterDbProxy.List(log.WithSpanFromContext(context.Background(), ctx), &adapters); err != nil {
+		logger.Errorw(ctx, "Failed-to-list-adapters-from-cluster-data-proxy", log.Fields{"error": err})
+		return err
+	}
+
+	logger.Debugw(ctx, "retrieved-adapters", log.Fields{"count": len(adapters)})
+
+	if len(adapters) != 0 {
+		for _, adapter := range adapters {
+			if err := aMgr.addAdapter(ctx, adapter, false); err != nil {
+				logger.Errorw(ctx, "failed-to-add-adapter", log.Fields{"adapterId": adapter.Id})
+			} else {
+				logger.Debugw(ctx, "adapter-added-successfully", log.Fields{"adapterId": adapter.Id})
+			}
 		}
 	}
-	return result, nil
-}
 
-func (aMgr *Manager) getAdapter(ctx context.Context, adapterID string) *voltha.Adapter {
-	aMgr.lockAdaptersMap.RLock()
-	defer aMgr.lockAdaptersMap.RUnlock()
-	if adapterAgent, ok := aMgr.adapterAgents[adapterID]; ok {
-		return adapterAgent.getAdapter(ctx)
+	// Load the device types
+	var deviceTypes []*voltha.DeviceType
+	if err := aMgr.deviceTypeDbProxy.List(log.WithSpanFromContext(context.Background(), ctx), &deviceTypes); err != nil {
+		logger.Errorw(ctx, "Failed-to-list-device-types-from-cluster-data-proxy", log.Fields{"error": err})
+		return err
 	}
+
+	logger.Debugw(ctx, "retrieved-devicetypes", log.Fields{"count": len(deviceTypes)})
+
+	if len(deviceTypes) != 0 {
+		dTypes := &voltha.DeviceTypes{Items: []*voltha.DeviceType{}}
+		for _, dType := range deviceTypes {
+			logger.Debugw(ctx, "found-existing-device-types", log.Fields{"deviceTypes": deviceTypes})
+			dTypes.Items = append(dTypes.Items, dType)
+		}
+		if err := aMgr.addDeviceTypes(ctx, dTypes, false); err != nil {
+			logger.Errorw(ctx, "failed-to-add-device-type", log.Fields{"deviceTypes": deviceTypes})
+		} else {
+			logger.Debugw(ctx, "device-type-added-successfully", log.Fields{"deviceTypes": deviceTypes})
+		}
+	}
+
+	// Start the adapter agents - this will trigger the connection to the adapter
+	aMgr.lockAdapterAgentsMap.RLock()
+	defer aMgr.lockAdapterAgentsMap.RUnlock()
+	for _, adapterAgent := range aMgr.adapterAgents {
+		subCtx := log.WithSpanFromContext(context.Background(), ctx)
+		if err := adapterAgent.start(subCtx); err != nil {
+			logger.Errorw(ctx, "failed-to-start-adapter", log.Fields{"adapter-endpoint": adapterAgent.adapterAPIEndPoint})
+		}
+	}
+
+	logger.Debug(ctx, "no-existing-device-type-found")
+
 	return nil
 }
 
-func (aMgr *Manager) RegisterAdapter(ctx context.Context, adapter *voltha.Adapter, deviceTypes *voltha.DeviceTypes) (*voltha.CoreInstance, error) {
-	logger.Debugw(ctx, "RegisterAdapter", log.Fields{"adapterId": adapter.Id, "vendor": adapter.Vendor,
-		"currentReplica": adapter.CurrentReplica, "totalReplicas": adapter.TotalReplicas, "endpoint": adapter.Endpoint, "deviceTypes": deviceTypes.Items})
+func (aMgr *Manager) RegisterAdapter(ctx context.Context, registration *ic.AdapterRegistration) (*empty.Empty, error) {
+	adapter := registration.Adapter
+	deviceTypes := registration.DTypes
+	logger.Infow(ctx, "RegisterAdapter", log.Fields{"adapter": adapter, "deviceTypes": deviceTypes.Items})
 
 	if adapter.Type == "" {
 		logger.Errorw(ctx, "adapter-not-specifying-type", log.Fields{
@@ -231,15 +291,25 @@
 		return nil, status.Error(codes.InvalidArgument, "adapter-not-specifying-type")
 	}
 
-	if aMgr.getAdapter(ctx, adapter.Id) != nil {
+	if adpt, _ := aMgr.getAdapter(ctx, adapter.Id); adpt != nil {
 		//	Already registered - Adapter may have restarted.  Trigger the reconcile process for that adapter
+		logger.Warnw(ctx, "adapter-restarted", log.Fields{"adapter": adpt.Id, "endpoint": adpt.Endpoint})
+
+		// First reset the adapter connection
+		agt, err := aMgr.getAgent(ctx, adpt.Id)
+		if err != nil {
+			logger.Errorw(ctx, "no-adapter-agent", log.Fields{"error": err})
+			return nil, err
+		}
+		agt.resetConnection(ctx)
+
 		go func() {
-			err := aMgr.onAdapterRestart(log.WithSpanFromContext(context.Background(), ctx), adapter)
+			err := aMgr.onAdapterRestart(log.WithSpanFromContext(context.Background(), ctx), adpt.Endpoint)
 			if err != nil {
 				logger.Errorw(ctx, "unable-to-restart-adapter", log.Fields{"error": err})
 			}
 		}()
-		return &voltha.CoreInstance{InstanceId: aMgr.coreInstanceID}, nil
+		return &empty.Empty{}, nil
 	}
 	// Save the adapter and the device types
 	if err := aMgr.addAdapter(ctx, adapter, true); err != nil {
@@ -254,16 +324,45 @@
 	logger.Debugw(ctx, "adapter-registered", log.Fields{"adapterId": adapter.Id, "vendor": adapter.Vendor,
 		"currentReplica": adapter.CurrentReplica, "totalReplicas": adapter.TotalReplicas, "endpoint": adapter.Endpoint})
 
-	return &voltha.CoreInstance{InstanceId: aMgr.coreInstanceID}, nil
+	// Setup the endpoints for this adapter
+	if err := aMgr.endpointMgr.RegisterAdapter(ctx, adapter, deviceTypes); err != nil {
+		logger.Errorw(ctx, "failed-to-register-adapter", log.Fields{"error": err})
+	}
+
+	// Start adapter instance - this will trigger the connection to the adapter
+	if agent, err := aMgr.getAgent(ctx, adapter.Id); agent != nil {
+		subCtx := log.WithSpanFromContext(context.Background(), ctx)
+		if err := agent.start(subCtx); err != nil {
+			logger.Errorw(ctx, "failed-to-start-adapter", log.Fields{"error": err})
+			return nil, err
+		}
+	} else {
+		logger.Fatalw(ctx, "adapter-absent", log.Fields{"error": err, "adapter": adapter.Id})
+	}
+
+	return &empty.Empty{}, nil
 }
 
-// GetAdapterType returns the name of the device adapter that service this device type
+func (aMgr *Manager) GetAdapterTypeByVendorID(vendorID string) (string, error) {
+	aMgr.lockDeviceTypesMap.RLock()
+	defer aMgr.lockDeviceTypesMap.RUnlock()
+	for _, dType := range aMgr.deviceTypes {
+		for _, v := range dType.VendorIds {
+			if v == vendorID {
+				return dType.AdapterType, nil
+			}
+		}
+	}
+	return "", fmt.Errorf("vendor id %s not found", vendorID)
+}
+
+// GetAdapterType returns the name of the device adapter that services this device type
 func (aMgr *Manager) GetAdapterType(deviceType string) (string, error) {
-	aMgr.lockdDeviceTypeToAdapterMap.Lock()
-	defer aMgr.lockdDeviceTypeToAdapterMap.Unlock()
-	for _, adapterAgent := range aMgr.adapterAgents {
-		if deviceType == adapterAgent.adapter.Type {
-			return adapterAgent.adapter.Type, nil
+	aMgr.lockDeviceTypesMap.Lock()
+	defer aMgr.lockDeviceTypesMap.Unlock()
+	for _, dt := range aMgr.deviceTypes {
+		if deviceType == dt.Id {
+			return dt.AdapterType, nil
 		}
 	}
 	return "", fmt.Errorf("adapter-not-registered-for-device-type %s", deviceType)
@@ -272,8 +371,8 @@
 // ListDeviceTypes returns all the device types known to the system
 func (aMgr *Manager) ListDeviceTypes(ctx context.Context, _ *empty.Empty) (*voltha.DeviceTypes, error) {
 	logger.Debug(ctx, "ListDeviceTypes")
-	aMgr.lockdDeviceTypeToAdapterMap.Lock()
-	defer aMgr.lockdDeviceTypeToAdapterMap.Unlock()
+	aMgr.lockDeviceTypesMap.Lock()
+	defer aMgr.lockDeviceTypesMap.Unlock()
 
 	deviceTypes := make([]*voltha.DeviceType, 0, len(aMgr.deviceTypes))
 	for _, deviceType := range aMgr.deviceTypes {
@@ -283,10 +382,10 @@
 }
 
 // GetDeviceType returns the device type proto definition given the name of the device type
-func (aMgr *Manager) GetDeviceType(ctx context.Context, deviceType *voltha.ID) (*voltha.DeviceType, error) {
+func (aMgr *Manager) GetDeviceType(ctx context.Context, deviceType *common.ID) (*voltha.DeviceType, error) {
 	logger.Debugw(ctx, "GetDeviceType", log.Fields{"typeid": deviceType.Id})
-	aMgr.lockdDeviceTypeToAdapterMap.Lock()
-	defer aMgr.lockdDeviceTypeToAdapterMap.Unlock()
+	aMgr.lockDeviceTypesMap.Lock()
+	defer aMgr.lockDeviceTypesMap.Unlock()
 
 	dType, exist := aMgr.deviceTypes[deviceType.Id]
 	if !exist {
@@ -294,3 +393,81 @@
 	}
 	return dType, nil
 }
+
+// ListAdapters returns the contents of all adapters known to the system
+func (aMgr *Manager) ListAdapters(ctx context.Context, _ *empty.Empty) (*voltha.Adapters, error) {
+	logger.Debug(ctx, "Listing adapters")
+	result := &voltha.Adapters{Items: []*voltha.Adapter{}}
+	aMgr.lockAdapterAgentsMap.RLock()
+	defer aMgr.lockAdapterAgentsMap.RUnlock()
+	for _, adapterAgent := range aMgr.adapterAgents {
+		if a := adapterAgent.getAdapter(ctx); a != nil {
+			result.Items = append(result.Items, (proto.Clone(a)).(*voltha.Adapter))
+		}
+	}
+	logger.Debugw(ctx, "Listing adapters", log.Fields{"result": result})
+	return result, nil
+}
+
+func (aMgr *Manager) getAgent(ctx context.Context, adapterID string) (*agent, error) {
+	aMgr.lockAdapterAgentsMap.RLock()
+	defer aMgr.lockAdapterAgentsMap.RUnlock()
+	if adapterAgent, ok := aMgr.adapterAgents[adapterID]; ok {
+		return adapterAgent, nil
+	}
+	return nil, errors.New("Not found")
+}
+
+func (aMgr *Manager) getAdapter(ctx context.Context, adapterID string) (*voltha.Adapter, error) {
+	aMgr.lockAdapterAgentsMap.RLock()
+	defer aMgr.lockAdapterAgentsMap.RUnlock()
+	if adapterAgent, ok := aMgr.adapterAgents[adapterID]; ok {
+		return adapterAgent.getAdapter(ctx), nil
+	}
+	return nil, errors.New("Not found")
+}
+
+// mutedAdapterRestartedHandler will be invoked by the grpc client on an adapter restart.
+// Since the Adapter will re-register itself and that will trigger the reconcile process,
+// therefore this handler does nothing, other than logging the event.
+func (aMgr *Manager) mutedAdapterRestartedHandler(ctx context.Context, endpoint string) error {
+	logger.Infow(ctx, "muted-adapter-restarted", log.Fields{"endpoint": endpoint})
+	return nil
+}
+
+func (aMgr *Manager) WaitUntilConnectionsToAdaptersAreUp(ctx context.Context, connectionRetryInterval time.Duration) error {
+	logger.Infow(ctx, "waiting-for-adapters-to-be-up", log.Fields{"retry-interval": connectionRetryInterval})
+	for {
+		aMgr.lockAdapterAgentsMap.Lock()
+		numAdapters := len(aMgr.adapterAgents)
+		if numAdapters == 0 {
+			// No adapter registered yet
+			aMgr.lockAdapterAgentsMap.Unlock()
+			logger.Info(ctx, "no-adapter-registered")
+			return nil
+		}
+		// A case of Core restart
+		agentsUp := true
+	adapterloop:
+		for _, agt := range aMgr.adapterAgents {
+			agentsUp = agentsUp && agt.IsConnectionUp()
+			if !agentsUp {
+				break adapterloop
+			}
+		}
+		aMgr.lockAdapterAgentsMap.Unlock()
+		if agentsUp {
+			logger.Infow(ctx, "adapter-connections-ready", log.Fields{"adapter-count": numAdapters})
+			return nil
+		}
+		logger.Warnw(ctx, "adapter-connections-not-ready", log.Fields{"adapter-count": numAdapters})
+		select {
+		case <-time.After(connectionRetryInterval):
+			logger.Infow(ctx, "retrying-adapter-connections", log.Fields{"adapter-count": numAdapters})
+			continue
+		case <-ctx.Done():
+			logger.Errorw(ctx, "context-timeout", log.Fields{"adapter-count": numAdapters, "err": ctx.Err()})
+			return ctx.Err()
+		}
+	}
+}
diff --git a/rw_core/core/api/adapter_request_handler.go b/rw_core/core/api/adapter_request_handler.go
deleted file mode 100644
index 890ac8b..0000000
--- a/rw_core/core/api/adapter_request_handler.go
+++ /dev/null
@@ -1,979 +0,0 @@
-/*
- * Copyright 2018-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.
- */
-
-package api
-
-import (
-	"context"
-	"errors"
-	"github.com/golang/protobuf/ptypes"
-	"github.com/golang/protobuf/ptypes/empty"
-	"github.com/opencord/voltha-go/rw_core/core/adapter"
-	"github.com/opencord/voltha-go/rw_core/core/device"
-	"github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-// AdapterRequestHandlerProxy represent adapter request handler proxy attributes
-type AdapterRequestHandlerProxy struct {
-	deviceMgr  *device.Manager
-	adapterMgr *adapter.Manager
-}
-
-// NewAdapterRequestHandlerProxy assigns values for adapter request handler proxy attributes and returns the new instance
-func NewAdapterRequestHandlerProxy(dMgr *device.Manager, aMgr *adapter.Manager) *AdapterRequestHandlerProxy {
-	return &AdapterRequestHandlerProxy{
-		deviceMgr:  dMgr,
-		adapterMgr: aMgr,
-	}
-}
-
-func (rhp *AdapterRequestHandlerProxy) Register(ctx context.Context, args []*ic.Argument) (*voltha.CoreInstance, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	adapter := &voltha.Adapter{}
-	deviceTypes := &voltha.DeviceTypes{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "adapter":
-			if err := ptypes.UnmarshalAny(arg.Value, adapter); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-adapter", log.Fields{"error": err})
-				return nil, err
-			}
-		case "deviceTypes":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceTypes); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-types", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "register", log.Fields{"adapter": *adapter, "device-types": deviceTypes, "transaction-id": transactionID.Val})
-
-	return rhp.adapterMgr.RegisterAdapter(ctx, adapter, deviceTypes)
-}
-
-// GetDevice returns device info
-func (rhp *AdapterRequestHandlerProxy) GetDevice(ctx context.Context, args []*ic.Argument) (*voltha.Device, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	pID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, pID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "get-device", log.Fields{"device-id": pID.Id, "transaction-id": transactionID.Val})
-
-	// Get the device via the device manager
-	device, err := rhp.deviceMgr.GetDevice(log.WithSpanFromContext(context.TODO(), ctx), pID)
-	if err != nil {
-		logger.Debugw(ctx, "get-device-failed", log.Fields{"device-id": pID.Id, "error": err})
-	}
-	return device, err
-}
-
-// DeviceUpdate updates device using adapter data
-func (rhp *AdapterRequestHandlerProxy) DeviceUpdate(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "device-update", log.Fields{"device-id": device.Id, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "DeviceUpdate")
-	if err := rhp.deviceMgr.UpdateDeviceUsingAdapterData(rpcCtx, device); err != nil {
-		logger.Debugw(ctx, "unable-to-update-device-using-adapter-data", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// GetChildDevice returns details of child device
-func (rhp *AdapterRequestHandlerProxy) GetChildDevice(ctx context.Context, args []*ic.Argument) (*voltha.Device, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	pID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	serialNumber := &ic.StrType{}
-	onuID := &ic.IntType{}
-	parentPortNo := &ic.IntType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, pID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "serial_number":
-			if err := ptypes.UnmarshalAny(arg.Value, serialNumber); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "onu_id":
-			if err := ptypes.UnmarshalAny(arg.Value, onuID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "parent_port_no":
-			if err := ptypes.UnmarshalAny(arg.Value, parentPortNo); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "get-child-device", log.Fields{"parent-device-id": pID.Id, "args": args, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "GetChildDevice")
-	return rhp.deviceMgr.GetChildDevice(rpcCtx, pID.Id, serialNumber.Val, onuID.Val, parentPortNo.Val)
-}
-
-// GetChildDeviceWithProxyAddress returns details of child device with proxy address
-func (rhp *AdapterRequestHandlerProxy) GetChildDeviceWithProxyAddress(ctx context.Context, args []*ic.Argument) (*voltha.Device, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	proxyAddress := &voltha.Device_ProxyAddress{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "proxy_address":
-			if err := ptypes.UnmarshalAny(arg.Value, proxyAddress); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-proxy-address", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "get-child-device-with-proxy-address", log.Fields{"proxy-address": proxyAddress, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "GetChildDeviceWithProxyAddress")
-	return rhp.deviceMgr.GetChildDeviceWithProxyAddress(rpcCtx, proxyAddress)
-}
-
-// GetPorts returns the ports information of the device based on the port type.
-func (rhp *AdapterRequestHandlerProxy) GetPorts(ctx context.Context, args []*ic.Argument) (*voltha.Ports, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	pt := &ic.IntType{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "port_type":
-			if err := ptypes.UnmarshalAny(arg.Value, pt); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-porttype", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "get-ports", log.Fields{"device-id": deviceID.Id, "port-type": pt.Val, "transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "GetPorts")
-	return rhp.deviceMgr.GetPorts(rpcCtx, deviceID.Id, voltha.Port_PortType(pt.Val))
-}
-
-// GetChildDevices gets all the child device IDs from the device passed as parameter
-func (rhp *AdapterRequestHandlerProxy) GetChildDevices(ctx context.Context, args []*ic.Argument) (*voltha.Devices, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	pID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, pID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "get-child-devices", log.Fields{"device-id": pID.Id, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "GetChildDevices")
-
-	return rhp.deviceMgr.GetAllChildDevices(rpcCtx, pID.Id)
-}
-
-// ChildDeviceDetected is invoked when a child device is detected.  The following parameters are expected:
-// {parent_device_id, parent_port_no, child_device_type, channel_id, vendor_id, serial_number)
-func (rhp *AdapterRequestHandlerProxy) ChildDeviceDetected(ctx context.Context, args []*ic.Argument) (*voltha.Device, error) {
-	if len(args) < 5 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	pID := &voltha.ID{}
-	portNo := &ic.IntType{}
-	dt := &ic.StrType{}
-	chnlID := &ic.IntType{}
-	transactionID := &ic.StrType{}
-	serialNumber := &ic.StrType{}
-	vendorID := &ic.StrType{}
-	onuID := &ic.IntType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "parent_device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, pID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-parent-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "parent_port_no":
-			if err := ptypes.UnmarshalAny(arg.Value, portNo); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-parent-port", log.Fields{"error": err})
-				return nil, err
-			}
-		case "child_device_type":
-			if err := ptypes.UnmarshalAny(arg.Value, dt); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-child-device-type", log.Fields{"error": err})
-				return nil, err
-			}
-		case "channel_id":
-			if err := ptypes.UnmarshalAny(arg.Value, chnlID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-channel-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "vendor_id":
-			if err := ptypes.UnmarshalAny(arg.Value, vendorID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-vendor-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "serial_number":
-			if err := ptypes.UnmarshalAny(arg.Value, serialNumber); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-serial-number", log.Fields{"error": err})
-				return nil, err
-			}
-		case "onu_id":
-			if err := ptypes.UnmarshalAny(arg.Value, onuID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-onu-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "fromTopic":
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-fromTopic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "child-device-detected", log.Fields{"parent-device-id": pID.Id, "parent-port-no": portNo.Val,
-		"device-type": dt.Val, "channel-id": chnlID.Val, "serial-number": serialNumber.Val,
-		"vendor-id": vendorID.Val, "onu-id": onuID.Val, "transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "ChildDeviceDetected")
-	fromTopicContext := utils.WithFromTopicMetadataContext(rpcCtx, fromTopic.Val)
-	device, err := rhp.deviceMgr.ChildDeviceDetected(fromTopicContext, pID.Id, portNo.Val, dt.Val, chnlID.Val, vendorID.Val, serialNumber.Val, onuID.Val)
-	if err != nil {
-		logger.Debugw(ctx, "child-detection-failed", log.Fields{"parent-device-id": pID.Id, "onu-id": onuID.Val, "error": err})
-	}
-	return device, err
-}
-
-// DeviceStateUpdate updates device status
-func (rhp *AdapterRequestHandlerProxy) DeviceStateUpdate(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	operStatus := &ic.IntType{}
-	connStatus := &ic.IntType{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "oper_status":
-			if err := ptypes.UnmarshalAny(arg.Value, operStatus); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-operStatus", log.Fields{"error": err})
-				return nil, err
-			}
-		case "connect_status":
-			if err := ptypes.UnmarshalAny(arg.Value, connStatus); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-connStatus", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "device-state-update", log.Fields{"device-id": deviceID.Id, "oper-status": operStatus,
-		"conn-status": connStatus, "transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "DeviceStateUpdate")
-
-	if err := rhp.deviceMgr.UpdateDeviceStatus(rpcCtx, deviceID.Id, voltha.OperStatus_Types(operStatus.Val),
-		voltha.ConnectStatus_Types(connStatus.Val)); err != nil {
-		logger.Debugw(ctx, "unable-to-update-device-status", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// ChildrenStateUpdate updates child device status
-func (rhp *AdapterRequestHandlerProxy) ChildrenStateUpdate(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	operStatus := &ic.IntType{}
-	connStatus := &ic.IntType{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "oper_status":
-			if err := ptypes.UnmarshalAny(arg.Value, operStatus); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-operStatus", log.Fields{"error": err})
-				return nil, err
-			}
-		case "connect_status":
-			if err := ptypes.UnmarshalAny(arg.Value, connStatus); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-connStatus", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "children-state-update", log.Fields{"device-id": deviceID.Id, "oper-status": operStatus,
-		"conn-status": connStatus, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "ChildrenStateUpdate")
-	// When the enum is not set (i.e. -1), Go still convert to the Enum type with the value being -1
-	if err := rhp.deviceMgr.UpdateChildrenStatus(rpcCtx, deviceID.Id, voltha.OperStatus_Types(operStatus.Val),
-		voltha.ConnectStatus_Types(connStatus.Val)); err != nil {
-		logger.Debugw(ctx, "unable-to-update-children-status", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// PortsStateUpdate updates the ports state related to the device
-func (rhp *AdapterRequestHandlerProxy) PortsStateUpdate(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	portTypeFilter := &ic.IntType{}
-	operStatus := &ic.IntType{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "port_type_filter":
-			if err := ptypes.UnmarshalAny(arg.Value, portTypeFilter); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "oper_status":
-			if err := ptypes.UnmarshalAny(arg.Value, operStatus); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-operStatus", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "ports-state-update", log.Fields{"device-id": deviceID.Id, "oper-status": operStatus, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "PortsStateUpdate")
-	if err := rhp.deviceMgr.UpdatePortsState(rpcCtx, deviceID.Id, uint32(portTypeFilter.Val), voltha.OperStatus_Types(operStatus.Val)); err != nil {
-		logger.Debugw(ctx, "unable-to-update-ports-state", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// PortStateUpdate updates the port state of the device
-func (rhp *AdapterRequestHandlerProxy) PortStateUpdate(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	portType := &ic.IntType{}
-	portNo := &ic.IntType{}
-	operStatus := &ic.IntType{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "oper_status":
-			if err := ptypes.UnmarshalAny(arg.Value, operStatus); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-operStatus", log.Fields{"error": err})
-				return nil, err
-			}
-		case "port_type":
-			if err := ptypes.UnmarshalAny(arg.Value, portType); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-porttype", log.Fields{"error": err})
-				return nil, err
-			}
-		case "port_no":
-			if err := ptypes.UnmarshalAny(arg.Value, portNo); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-portno", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "port-state-update", log.Fields{"device-id": deviceID.Id, "oper-status": operStatus,
-		"port-type": portType, "port-no": portNo, "transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "PortStateUpdate")
-
-	if err := rhp.deviceMgr.UpdatePortState(rpcCtx, deviceID.Id, voltha.Port_PortType(portType.Val), uint32(portNo.Val),
-		voltha.OperStatus_Types(operStatus.Val)); err != nil {
-		// If the error doesn't change behavior and is essentially ignored, it is not an error, it is a
-		// warning.
-		// TODO: VOL-2707
-		logger.Debugw(ctx, "unable-to-update-port-state", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// DeleteAllPorts deletes all ports of device
-func (rhp *AdapterRequestHandlerProxy) DeleteAllPorts(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "delete-all-ports", log.Fields{"device-id": deviceID.Id, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "DeleteAllPorts")
-	if err := rhp.deviceMgr.DeleteAllPorts(rpcCtx, deviceID.Id); err != nil {
-		logger.Debugw(ctx, "unable-to-delete-ports", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// GetDevicePort returns a single port
-func (rhp *AdapterRequestHandlerProxy) GetDevicePort(ctx context.Context, args []*ic.Argument) (*voltha.Port, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	portNo := &ic.IntType{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "port_no":
-			if err := ptypes.UnmarshalAny(arg.Value, portNo); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-port-no", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "get-device-port", log.Fields{"device-id": deviceID.Id, "port-no": portNo.Val, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "GetDevicePort")
-
-	return rhp.deviceMgr.GetDevicePort(rpcCtx, deviceID.Id, uint32(portNo.Val))
-}
-
-// ListDevicePorts returns all ports belonging to the device
-func (rhp *AdapterRequestHandlerProxy) ListDevicePorts(ctx context.Context, args []*ic.Argument) (*voltha.Ports, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "list-device-ports", log.Fields{"device-id": deviceID.Id, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "ListDevicePorts")
-	return rhp.deviceMgr.ListDevicePorts(rpcCtx, deviceID)
-}
-
-// ChildDevicesLost indicates that a parent device is in a state (Disabled) where it cannot manage the child devices.
-// This will trigger the Core to disable all the child devices.
-func (rhp *AdapterRequestHandlerProxy) ChildDevicesLost(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	parentDeviceID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "parent_device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, parentDeviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "child-devices-lost", log.Fields{"device-id": parentDeviceID.Id, "transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "ChildDevicesLost")
-
-	if err := rhp.deviceMgr.ChildDevicesLost(rpcCtx, parentDeviceID.Id); err != nil {
-		logger.Debugw(ctx, "unable-to-disable-child-devices", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// ChildDevicesDetected invoked by an adapter when child devices are found, typically after after a disable/enable sequence.
-// This will trigger the Core to Enable all the child devices of that parent.
-func (rhp *AdapterRequestHandlerProxy) ChildDevicesDetected(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	parentDeviceID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "parent_device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, parentDeviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "child-devices-detected", log.Fields{"parent-device-id": parentDeviceID.Id, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "ChildDevicesDetected")
-
-	if err := rhp.deviceMgr.ChildDevicesDetected(rpcCtx, parentDeviceID.Id); err != nil {
-		logger.Debugw(ctx, "child-devices-detection-failed", log.Fields{"parent-device-id": parentDeviceID.Id, "error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// PortCreated adds port to device
-func (rhp *AdapterRequestHandlerProxy) PortCreated(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	port := &voltha.Port{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "port":
-			if err := ptypes.UnmarshalAny(arg.Value, port); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-port", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "fromTopic":
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-fromTopic", log.Fields{"error": err})
-				return nil, err
-			}
-			//log.EnrichSpan(ctx,log.Fields{"fromTopic": fromTopic})
-		}
-	}
-	logger.Debugw(ctx, "port-created", log.Fields{"device-id": deviceID.Id, "port": port, "transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "PortCreated")
-	fromTopicContext := utils.WithFromTopicMetadataContext(rpcCtx, fromTopic.Val)
-
-	if err := rhp.deviceMgr.AddPort(fromTopicContext, deviceID.Id, port); err != nil {
-		logger.Debugw(ctx, "unable-to-add-port", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// DevicePMConfigUpdate initializes the pm configs as defined by the adapter.
-func (rhp *AdapterRequestHandlerProxy) DevicePMConfigUpdate(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	pmConfigs := &voltha.PmConfigs{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_pm_config":
-			if err := ptypes.UnmarshalAny(arg.Value, pmConfigs); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-pm-config", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "device-pm-config-update", log.Fields{"device-id": pmConfigs.Id, "configs": pmConfigs,
-		"transaction-id": transactionID.Val})
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "DevicePMConfigUpdate")
-
-	if err := rhp.deviceMgr.InitPmConfigs(rpcCtx, pmConfigs.Id, pmConfigs); err != nil {
-		logger.Debugw(ctx, "unable-to-initialize-pm-configs", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// PacketIn sends the incoming packet of device
-func (rhp *AdapterRequestHandlerProxy) PacketIn(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 4 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	portNo := &ic.IntType{}
-	packet := &ic.Packet{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "port":
-			if err := ptypes.UnmarshalAny(arg.Value, portNo); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-port-no", log.Fields{"error": err})
-				return nil, err
-			}
-		case "packet":
-			if err := ptypes.UnmarshalAny(arg.Value, packet); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-packet", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "packet-in", log.Fields{"device-id": deviceID.Id, "port": portNo.Val, "packet": packet,
-		"transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "PacketIn")
-
-	if err := rhp.deviceMgr.PacketIn(rpcCtx, deviceID.Id, uint32(portNo.Val), transactionID.Val, packet.Payload); err != nil {
-		logger.Debugw(ctx, "unable-to-receive-packet-from-adapter", log.Fields{"error": err})
-		return nil, err
-
-	}
-	return &empty.Empty{}, nil
-}
-
-// UpdateImageDownload updates image download
-func (rhp *AdapterRequestHandlerProxy) UpdateImageDownload(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	img := &voltha.ImageDownload{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "image_download":
-			if err := ptypes.UnmarshalAny(arg.Value, img); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-imgaeDownload", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "update-image-download", log.Fields{"device-id": deviceID.Id, "image-download": img,
-		"transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "UpdateImageDownload")
-
-	if err := rhp.deviceMgr.UpdateImageDownload(rpcCtx, deviceID.Id, img); err != nil {
-		logger.Debugw(ctx, "unable-to-update-image-download", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// ReconcileChildDevices reconciles child devices
-func (rhp *AdapterRequestHandlerProxy) ReconcileChildDevices(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	parentDeviceID := &voltha.ID{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "parent_device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, parentDeviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "reconcile-child-devices", log.Fields{"parent-device-id": parentDeviceID.Id, "transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "ReconcileChildDevices")
-
-	if err := rhp.deviceMgr.ReconcileChildDevices(rpcCtx, parentDeviceID.Id); err != nil {
-		logger.Debugw(ctx, "unable-to-reconcile-child-devices", log.Fields{"error": err})
-		return nil, err
-	}
-	return &empty.Empty{}, nil
-}
-
-// DeviceReasonUpdate updates device reason
-func (rhp *AdapterRequestHandlerProxy) DeviceReasonUpdate(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "device-reason-update-invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("DeviceReasonUpdate: invalid-number-of-args")
-		return nil, err
-	}
-	deviceID := &voltha.ID{}
-	reason := &ic.StrType{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device_id":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "device_reason":
-			if err := ptypes.UnmarshalAny(arg.Value, reason); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-reason", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "fromTopic":
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-fromTopic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "device-reason-update", log.Fields{"device-id": deviceID.Id, "reason": reason.Val,
-		"transaction-id": transactionID.Val})
-
-	rpcCtx := utils.WithRPCMetadataContext(log.WithSpanFromContext(context.TODO(), ctx), "DeviceReasonUpdate")
-	fromTopicContext := utils.WithFromTopicMetadataContext(rpcCtx, fromTopic.Val)
-
-	if err := rhp.deviceMgr.UpdateDeviceReason(fromTopicContext, deviceID.Id, reason.Val); err != nil {
-		logger.Debugw(ctx, "unable-to-update-device-reason", log.Fields{"error": err})
-		return nil, err
-
-	}
-	return &empty.Empty{}, nil
-}
diff --git a/rw_core/core/api/common.go b/rw_core/core/api/common.go
index 0cf2888..13294eb 100644
--- a/rw_core/core/api/common.go
+++ b/rw_core/core/api/common.go
@@ -18,7 +18,7 @@
 package api
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/api/common_test.go b/rw_core/core/api/common_test.go
deleted file mode 100644
index 3a4fe8e..0000000
--- a/rw_core/core/api/common_test.go
+++ /dev/null
@@ -1,280 +0,0 @@
-/*
- * Copyright 2019-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.
- */
-package api
-
-import (
-	"context"
-	"fmt"
-	"time"
-
-	"github.com/golang/protobuf/ptypes/empty"
-	"github.com/google/uuid"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-	"google.golang.org/grpc/metadata"
-)
-
-const (
-	volthaSerialNumberKey = "voltha_serial_number"
-	retryInterval         = 50 * time.Millisecond
-)
-
-var (
-	coreInCompeteMode bool
-)
-
-type isLogicalDeviceConditionSatisfied func(ld *voltha.LogicalDevice) bool
-type isLogicalDevicePortsConditionSatisfied func(ports []*voltha.LogicalPort) bool
-type isDeviceConditionSatisfied func(ld *voltha.Device) bool
-type isDevicePortsConditionSatisfied func(ports *voltha.Ports) bool
-type isDevicesConditionSatisfied func(ds *voltha.Devices) bool
-type isLogicalDevicesConditionSatisfied func(lds *voltha.LogicalDevices) bool
-type isConditionSatisfied func() bool
-
-func init() {
-	//Default mode is two rw-core running in a pair of competing cores
-	coreInCompeteMode = true
-}
-
-func setCoreCompeteMode(mode bool) {
-	coreInCompeteMode = mode
-}
-
-func getContext() context.Context {
-	if coreInCompeteMode {
-		return metadata.NewIncomingContext(context.Background(), metadata.Pairs(volthaSerialNumberKey, uuid.New().String()))
-	}
-	return context.Background()
-}
-
-func waitUntilDeviceReadiness(deviceID string,
-	timeout time.Duration,
-	verificationFunction isDeviceConditionSatisfied,
-	nbi *NBIHandler) error {
-	ch := make(chan int, 1)
-	done := false
-	go func() {
-		for {
-			device, _ := nbi.GetDevice(getContext(), &voltha.ID{Id: deviceID})
-			if verificationFunction(device) {
-				ch <- 1
-				break
-			}
-			if done {
-				break
-			}
-			time.Sleep(retryInterval)
-		}
-	}()
-	timer := time.NewTimer(timeout)
-	defer timer.Stop()
-	select {
-	case <-ch:
-		return nil
-	case <-timer.C:
-		done = true
-		return fmt.Errorf("expected-states-not-reached-for-device%s", deviceID)
-	}
-}
-
-func waitUntilDevicePortsReadiness(deviceID string,
-	timeout time.Duration,
-	verificationFunction isDevicePortsConditionSatisfied,
-	nbi *NBIHandler) error {
-	ch := make(chan int, 1)
-	done := false
-	go func() {
-		for {
-			ports, _ := nbi.ListDevicePorts(getContext(), &voltha.ID{Id: deviceID})
-			if verificationFunction(ports) {
-				ch <- 1
-				break
-			}
-			if done {
-				break
-			}
-			time.Sleep(retryInterval)
-		}
-	}()
-	timer := time.NewTimer(timeout)
-	defer timer.Stop()
-	select {
-	case <-ch:
-		return nil
-	case <-timer.C:
-		done = true
-		return fmt.Errorf("expected-states-not-reached-for-device%s", deviceID)
-	}
-}
-
-func waitUntilLogicalDeviceReadiness(oltDeviceID string,
-	timeout time.Duration,
-	nbi *NBIHandler,
-	verificationFunction isLogicalDeviceConditionSatisfied,
-) error {
-	ch := make(chan int, 1)
-	done := false
-	go func() {
-		for {
-			// Get the logical device from the olt device
-			d, _ := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDeviceID})
-			if d != nil && d.ParentId != "" {
-				ld, _ := nbi.GetLogicalDevice(getContext(), &voltha.ID{Id: d.ParentId})
-				if verificationFunction(ld) {
-					ch <- 1
-					break
-				}
-				if done {
-					break
-				}
-			} else if d != nil && d.ParentId == "" { // case where logical device deleted
-				if verificationFunction(nil) {
-					ch <- 1
-					break
-				}
-				if done {
-					break
-				}
-			}
-			time.Sleep(retryInterval)
-		}
-	}()
-	timer := time.NewTimer(timeout)
-	defer timer.Stop()
-	select {
-	case <-ch:
-		return nil
-	case <-timer.C:
-		done = true
-		return fmt.Errorf("timeout-waiting-for-logical-device-readiness%s", oltDeviceID)
-	}
-}
-
-func waitUntilLogicalDevicePortsReadiness(oltDeviceID string,
-	timeout time.Duration,
-	nbi *NBIHandler,
-	verificationFunction isLogicalDevicePortsConditionSatisfied,
-) error {
-	ch := make(chan int, 1)
-	done := false
-	go func() {
-		for {
-			// Get the logical device from the olt device
-			d, _ := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDeviceID})
-			if d != nil && d.ParentId != "" {
-				ports, err := nbi.ListLogicalDevicePorts(getContext(), &voltha.ID{Id: d.ParentId})
-				if err == nil && verificationFunction(ports.Items) {
-					ch <- 1
-					break
-				}
-				if done {
-					break
-				}
-			}
-			time.Sleep(retryInterval)
-		}
-	}()
-	timer := time.NewTimer(timeout)
-	defer timer.Stop()
-	select {
-	case <-ch:
-		return nil
-	case <-timer.C:
-		done = true
-		return fmt.Errorf("timeout-waiting-for-logical-device-readiness%s", oltDeviceID)
-	}
-}
-
-func waitUntilConditionForDevices(timeout time.Duration, nbi *NBIHandler, verificationFunction isDevicesConditionSatisfied) error {
-	ch := make(chan int, 1)
-	done := false
-	go func() {
-		for {
-			devices, _ := nbi.ListDevices(getContext(), &empty.Empty{})
-			if verificationFunction(devices) {
-				ch <- 1
-				break
-			}
-			if done {
-				break
-			}
-
-			time.Sleep(retryInterval)
-		}
-	}()
-	timer := time.NewTimer(timeout)
-	defer timer.Stop()
-	select {
-	case <-ch:
-		return nil
-	case <-timer.C:
-		done = true
-		return fmt.Errorf("timeout-waiting-devices")
-	}
-}
-
-func waitUntilConditionForLogicalDevices(timeout time.Duration, nbi *NBIHandler, verificationFunction isLogicalDevicesConditionSatisfied) error {
-	ch := make(chan int, 1)
-	done := false
-	go func() {
-		for {
-			lDevices, _ := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
-			if verificationFunction(lDevices) {
-				ch <- 1
-				break
-			}
-			if done {
-				break
-			}
-
-			time.Sleep(retryInterval)
-		}
-	}()
-	timer := time.NewTimer(timeout)
-	defer timer.Stop()
-	select {
-	case <-ch:
-		return nil
-	case <-timer.C:
-		done = true
-		return fmt.Errorf("timeout-waiting-logical-devices")
-	}
-}
-
-func waitUntilCondition(timeout time.Duration, nbi *NBIHandler, verificationFunction isConditionSatisfied) error {
-	ch := make(chan int, 1)
-	done := false
-	go func() {
-		for {
-			if verificationFunction() {
-				ch <- 1
-				break
-			}
-			if done {
-				break
-			}
-			time.Sleep(retryInterval)
-		}
-	}()
-	timer := time.NewTimer(timeout)
-	defer timer.Stop()
-	select {
-	case <-ch:
-		return nil
-	case <-timer.C:
-		done = true
-		return fmt.Errorf("timeout-waiting-for-condition")
-	}
-}
diff --git a/rw_core/core/api/grpc_nbi_handler.go b/rw_core/core/api/grpc_nbi_handler.go
index e034da5..6defba3 100755
--- a/rw_core/core/api/grpc_nbi_handler.go
+++ b/rw_core/core/api/grpc_nbi_handler.go
@@ -20,17 +20,18 @@
 	"context"
 	"encoding/json"
 	"errors"
+
 	"github.com/golang/protobuf/ptypes/empty"
 	"github.com/opencord/voltha-go/rw_core/core/adapter"
 	"github.com/opencord/voltha-go/rw_core/core/device"
-	"github.com/opencord/voltha-lib-go/v5/pkg/version"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	"github.com/opencord/voltha-protos/v4/go/omci"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/version"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/omci"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
-// NBIHandler combines the partial API implementations in various components into a complete voltha implementation
-type NBIHandler struct {
+// APIHandler combines the partial API implementations in various components into a complete voltha implementation
+type APIHandler struct {
 	*device.Manager
 	*device.LogicalManager
 	adapterManager // *adapter.Manager
@@ -39,17 +40,21 @@
 // avoid having multiple embedded types with the same name (`<package>.Manager`s conflict)
 type adapterManager struct{ *adapter.Manager }
 
-// NewNBIHandler creates API handler instance
-func NewNBIHandler(deviceMgr *device.Manager, logicalDeviceMgr *device.LogicalManager, adapterMgr *adapter.Manager) *NBIHandler {
-	return &NBIHandler{
+// NewAPIHandler creates API handler instance
+func NewAPIHandler(deviceMgr *device.Manager, logicalDeviceMgr *device.LogicalManager, adapterMgr *adapter.Manager) *APIHandler {
+	return &APIHandler{
 		Manager:        deviceMgr,
 		LogicalManager: logicalDeviceMgr,
 		adapterManager: adapterManager{adapterMgr},
 	}
 }
 
+func (handler *APIHandler) GetHealthStatus(ctx context.Context, empty *empty.Empty) (*voltha.HealthStatus, error) {
+	return &voltha.HealthStatus{State: voltha.HealthStatus_HEALTHY}, nil
+}
+
 // GetVoltha currently just returns version information
-func (handler *NBIHandler) GetVoltha(ctx context.Context, _ *empty.Empty) (*voltha.Voltha, error) {
+func (handler *APIHandler) GetVoltha(ctx context.Context, _ *empty.Empty) (*voltha.Voltha, error) {
 	logger.Debug(ctx, "GetVoltha")
 	/*
 	 * For now, encode all the version information into a JSON object and
@@ -67,48 +72,48 @@
 
 var errUnimplemented = errors.New("unimplemented")
 
-func (handler *NBIHandler) ListCoreInstances(context.Context, *empty.Empty) (*voltha.CoreInstances, error) {
+func (handler *APIHandler) ListCoreInstances(context.Context, *empty.Empty) (*voltha.CoreInstances, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) GetCoreInstance(context.Context, *voltha.ID) (*voltha.CoreInstance, error) {
+func (handler *APIHandler) GetCoreInstance(context.Context, *voltha.ID) (*voltha.CoreInstance, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) ListDeviceGroups(context.Context, *empty.Empty) (*voltha.DeviceGroups, error) {
+func (handler *APIHandler) ListDeviceGroups(context.Context, *empty.Empty) (*voltha.DeviceGroups, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) GetDeviceGroup(context.Context, *voltha.ID) (*voltha.DeviceGroup, error) {
+func (handler *APIHandler) GetDeviceGroup(context.Context, *voltha.ID) (*voltha.DeviceGroup, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) CreateEventFilter(context.Context, *voltha.EventFilter) (*voltha.EventFilter, error) {
+func (handler *APIHandler) CreateEventFilter(context.Context, *voltha.EventFilter) (*voltha.EventFilter, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) UpdateEventFilter(context.Context, *voltha.EventFilter) (*voltha.EventFilter, error) {
+func (handler *APIHandler) UpdateEventFilter(context.Context, *voltha.EventFilter) (*voltha.EventFilter, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) DeleteEventFilter(context.Context, *voltha.EventFilter) (*empty.Empty, error) {
+func (handler *APIHandler) DeleteEventFilter(context.Context, *voltha.EventFilter) (*empty.Empty, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) GetEventFilter(context.Context, *voltha.ID) (*voltha.EventFilters, error) {
+func (handler *APIHandler) GetEventFilter(context.Context, *voltha.ID) (*voltha.EventFilters, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) ListEventFilters(context.Context, *empty.Empty) (*voltha.EventFilters, error) {
+func (handler *APIHandler) ListEventFilters(context.Context, *empty.Empty) (*voltha.EventFilters, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) SelfTest(context.Context, *voltha.ID) (*voltha.SelfTestResponse, error) {
+func (handler *APIHandler) SelfTest(context.Context, *voltha.ID) (*voltha.SelfTestResponse, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) Subscribe(context.Context, *voltha.OfAgentSubscriber) (*voltha.OfAgentSubscriber, error) {
+func (handler *APIHandler) Subscribe(context.Context, *voltha.OfAgentSubscriber) (*voltha.OfAgentSubscriber, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) GetAlarmDeviceData(context.Context, *common.ID) (*omci.AlarmDeviceData, error) {
+func (handler *APIHandler) GetAlarmDeviceData(context.Context, *common.ID) (*omci.AlarmDeviceData, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) GetMibDeviceData(context.Context, *common.ID) (*omci.MibDeviceData, error) {
+func (handler *APIHandler) GetMibDeviceData(context.Context, *common.ID) (*omci.MibDeviceData, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) GetMembership(context.Context, *empty.Empty) (*voltha.Membership, error) {
+func (handler *APIHandler) GetMembership(context.Context, *empty.Empty) (*voltha.Membership, error) {
 	return nil, errUnimplemented
 }
-func (handler *NBIHandler) UpdateMembership(context.Context, *voltha.Membership) (*empty.Empty, error) {
+func (handler *APIHandler) UpdateMembership(context.Context, *voltha.Membership) (*empty.Empty, error) {
 	return nil, errUnimplemented
 }
diff --git a/rw_core/core/api/grpc_nbi_handler_test.go b/rw_core/core/api/grpc_nbi_handler_test.go
deleted file mode 100755
index 5241a95..0000000
--- a/rw_core/core/api/grpc_nbi_handler_test.go
+++ /dev/null
@@ -1,1830 +0,0 @@
-/*
- * Copyright 2019-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.
- */
-
-package api
-
-import (
-	"context"
-	"errors"
-	"fmt"
-	"math/rand"
-	"os"
-	"runtime"
-	"runtime/pprof"
-	"strconv"
-	"strings"
-	"sync"
-	"testing"
-	"time"
-
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-
-	"github.com/golang/protobuf/jsonpb"
-
-	"github.com/golang/protobuf/ptypes/empty"
-	"github.com/opencord/voltha-go/db/model"
-	"github.com/opencord/voltha-go/rw_core/config"
-	"github.com/opencord/voltha-go/rw_core/core/adapter"
-	"github.com/opencord/voltha-go/rw_core/core/device"
-	cm "github.com/opencord/voltha-go/rw_core/mocks"
-	tst "github.com/opencord/voltha-go/rw_core/test"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events"
-	"github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	mock_etcd "github.com/opencord/voltha-lib-go/v5/pkg/mocks/etcd"
-	mock_kafka "github.com/opencord/voltha-lib-go/v5/pkg/mocks/kafka"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-	"github.com/phayes/freeport"
-	"github.com/stretchr/testify/assert"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
-)
-
-const numTrapOnNNIFlows = 4
-
-type NBTest struct {
-	etcdServer        *mock_etcd.EtcdServer
-	deviceMgr         *device.Manager
-	logicalDeviceMgr  *device.LogicalManager
-	adapterMgr        *adapter.Manager
-	kmp               kafka.InterContainerProxy
-	kClient           kafka.Client
-	kEventClient      kafka.Client
-	kvClientPort      int
-	numONUPerOLT      int
-	startingUNIPortNo int
-	oltAdapter        *cm.OLTAdapter
-	onuAdapter        *cm.ONUAdapter
-	oltAdapterName    string
-	onuAdapterName    string
-	coreInstanceID    string
-	defaultTimeout    time.Duration
-	maxTimeout        time.Duration
-}
-
-var testLogger log.CLogger
-
-func init() {
-	var err error
-	testLogger, err = log.RegisterPackage(log.JSON, log.InfoLevel, log.Fields{"nbi-handler-test": true})
-	if err != nil {
-		panic(err)
-	}
-
-	if err = log.SetLogLevel(log.InfoLevel); err != nil {
-		panic(err)
-	}
-}
-
-func newNBTest(ctx context.Context) *NBTest {
-	test := &NBTest{}
-	// Start the embedded etcd server
-	var err error
-	test.etcdServer, test.kvClientPort, err = tst.StartEmbeddedEtcdServer(ctx, "voltha.rwcore.nb.test", "voltha.rwcore.nb.etcd", "error")
-	if err != nil {
-		logger.Fatal(ctx, err)
-	}
-	// Create the kafka client
-	test.kClient = mock_kafka.NewKafkaClient()
-	test.kEventClient = mock_kafka.NewKafkaClient()
-	test.oltAdapterName = "olt_adapter_mock"
-	test.onuAdapterName = "onu_adapter_mock"
-	test.coreInstanceID = "rw-nbi-test"
-	test.defaultTimeout = 10 * time.Second
-	test.maxTimeout = 20 * time.Second
-	return test
-}
-
-func (nb *NBTest) startCore(inCompeteMode bool) {
-	ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
-	defer cancel()
-	cfg := &config.RWCoreFlags{}
-	cfg.ParseCommandArguments([]string{}) // sets defaults
-	cfg.CoreTopic = "rw_core"
-	cfg.EventTopic = "voltha.events"
-	cfg.DefaultRequestTimeout = nb.defaultTimeout
-	cfg.DefaultCoreTimeout = nb.defaultTimeout
-	cfg.KVStoreAddress = "127.0.0.1" + ":" + strconv.Itoa(nb.kvClientPort)
-	grpcPort, err := freeport.GetFreePort()
-	if err != nil {
-		logger.Fatal(ctx, "Cannot get a freeport for grpc")
-	}
-	cfg.GrpcAddress = "127.0.0.1" + ":" + strconv.Itoa(grpcPort)
-	setCoreCompeteMode(inCompeteMode)
-	client := tst.SetupKVClient(ctx, cfg, nb.coreInstanceID)
-	backend := &db.Backend{
-		Client:                  client,
-		StoreType:               cfg.KVStoreType,
-		Address:                 cfg.KVStoreAddress,
-		Timeout:                 cfg.KVStoreTimeout,
-		LivenessChannelInterval: cfg.LiveProbeInterval / 2}
-	nb.kmp = kafka.NewInterContainerProxy(
-		kafka.InterContainerAddress(cfg.KafkaAdapterAddress),
-		kafka.MsgClient(nb.kClient),
-		kafka.DefaultTopic(&kafka.Topic{Name: cfg.CoreTopic}))
-
-	endpointMgr := kafka.NewEndpointManager(backend)
-	proxy := model.NewDBPath(backend)
-	nb.adapterMgr = adapter.NewAdapterManager(ctx, proxy, nb.coreInstanceID, nb.kClient)
-	eventProxy := events.NewEventProxy(events.MsgClient(nb.kEventClient), events.MsgTopic(kafka.Topic{Name: cfg.EventTopic}))
-	nb.deviceMgr, nb.logicalDeviceMgr = device.NewManagers(proxy, nb.adapterMgr, nb.kmp, endpointMgr, cfg, nb.coreInstanceID, eventProxy)
-	nb.adapterMgr.Start(ctx)
-
-	if err := nb.kmp.Start(ctx); err != nil {
-		logger.Fatalf(ctx, "Cannot start InterContainerProxy: %s", err)
-	}
-	requestProxy := NewAdapterRequestHandlerProxy(nb.deviceMgr, nb.adapterMgr)
-	if err := nb.kmp.SubscribeWithRequestHandlerInterface(ctx, kafka.Topic{Name: cfg.CoreTopic}, requestProxy); err != nil {
-		logger.Fatalf(ctx, "Cannot add request handler: %s", err)
-	}
-}
-
-func (nb *NBTest) stopAll(ctx context.Context) {
-	if nb.kClient != nil {
-		nb.kClient.Stop(ctx)
-	}
-	if nb.kmp != nil {
-		nb.kmp.Stop(ctx)
-	}
-	if nb.etcdServer != nil {
-		tst.StopEmbeddedEtcdServer(ctx, nb.etcdServer)
-	}
-	if nb.kEventClient != nil {
-		nb.kEventClient.Stop(ctx)
-	}
-}
-
-func (nb *NBTest) verifyLogicalDevices(t *testing.T, oltDevice *voltha.Device, nbi *NBIHandler) {
-	// Get the latest set of logical devices
-	logicalDevices, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, logicalDevices)
-	assert.Equal(t, 1, len(logicalDevices.Items))
-
-	ld := logicalDevices.Items[0]
-	ports, err := nbi.ListLogicalDevicePorts(getContext(), &voltha.ID{Id: ld.Id})
-	assert.Nil(t, err)
-
-	assert.NotEqual(t, "", ld.Id)
-	assert.NotEqual(t, uint64(0), ld.DatapathId)
-	assert.Equal(t, "olt_adapter_mock", ld.Desc.HwDesc)
-	assert.Equal(t, "olt_adapter_mock", ld.Desc.SwDesc)
-	assert.NotEqual(t, "", ld.RootDeviceId)
-	assert.NotEqual(t, "", ld.Desc.SerialNum)
-	assert.Equal(t, uint32(256), ld.SwitchFeatures.NBuffers)
-	assert.Equal(t, uint32(2), ld.SwitchFeatures.NTables)
-	assert.Equal(t, uint32(15), ld.SwitchFeatures.Capabilities)
-	assert.Equal(t, 1+nb.numONUPerOLT, len(ports.Items))
-	assert.Equal(t, oltDevice.ParentId, ld.Id)
-	//Expected port no
-	expectedPortNo := make(map[uint32]bool)
-	expectedPortNo[uint32(2)] = false
-	for i := 0; i < nb.numONUPerOLT; i++ {
-		expectedPortNo[uint32(i+100)] = false
-	}
-	for _, p := range ports.Items {
-		assert.Equal(t, p.OfpPort.PortNo, p.DevicePortNo)
-		assert.Equal(t, uint32(4), p.OfpPort.State)
-		expectedPortNo[p.OfpPort.PortNo] = true
-		if strings.HasPrefix(p.Id, "nni") {
-			assert.Equal(t, true, p.RootPort)
-			//assert.Equal(t, uint32(2), p.OfpPort.PortNo)
-			assert.Equal(t, p.Id, fmt.Sprintf("nni-%d", p.DevicePortNo))
-		} else {
-			assert.Equal(t, p.Id, fmt.Sprintf("uni-%d", p.DevicePortNo))
-			assert.Equal(t, false, p.RootPort)
-		}
-	}
-}
-
-func (nb *NBTest) verifyDevices(t *testing.T, nbi *NBIHandler) {
-	// Get the latest set of devices
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, devices)
-
-	// A device is ready to be examined when its ADMIN state is ENABLED and OPERATIONAL state is ACTIVE
-	var vFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
-		return device.AdminState == voltha.AdminState_ENABLED && device.OperStatus == voltha.OperStatus_ACTIVE
-	}
-
-	var wg sync.WaitGroup
-	for _, device := range devices.Items {
-		wg.Add(1)
-		go func(wg *sync.WaitGroup, device *voltha.Device) {
-			// Wait until the device is in the right state
-			err := waitUntilDeviceReadiness(device.Id, nb.maxTimeout, vFunction, nbi)
-			assert.Nil(t, err)
-
-			// Now, verify the details of the device.  First get the latest update
-			d, err := nbi.GetDevice(getContext(), &voltha.ID{Id: device.Id})
-			assert.Nil(t, err)
-			dPorts, err := nbi.ListDevicePorts(getContext(), &voltha.ID{Id: device.Id})
-			assert.Nil(t, err)
-			assert.Equal(t, voltha.AdminState_ENABLED, d.AdminState)
-			assert.Equal(t, voltha.ConnectStatus_REACHABLE, d.ConnectStatus)
-			assert.Equal(t, voltha.OperStatus_ACTIVE, d.OperStatus)
-			assert.Equal(t, d.Type, d.Adapter)
-			assert.NotEqual(t, "", d.MacAddress)
-			assert.NotEqual(t, "", d.SerialNumber)
-
-			if d.Type == "olt_adapter_mock" {
-				assert.Equal(t, true, d.Root)
-				assert.NotEqual(t, "", d.Id)
-				assert.NotEqual(t, "", d.ParentId)
-				assert.Nil(t, d.ProxyAddress)
-			} else if d.Type == "onu_adapter_mock" {
-				assert.Equal(t, false, d.Root)
-				assert.NotEqual(t, uint32(0), d.Vlan)
-				assert.NotEqual(t, "", d.Id)
-				assert.NotEqual(t, "", d.ParentId)
-				assert.NotEqual(t, "", d.ProxyAddress.DeviceId)
-				assert.Equal(t, "olt_adapter_mock", d.ProxyAddress.DeviceType)
-			} else {
-				assert.Error(t, errors.New("invalid-device-type"))
-			}
-			assert.Equal(t, 2, len(dPorts.Items))
-			for _, p := range dPorts.Items {
-				assert.Equal(t, voltha.AdminState_ENABLED, p.AdminState)
-				assert.Equal(t, voltha.OperStatus_ACTIVE, p.OperStatus)
-				if p.Type == voltha.Port_ETHERNET_NNI || p.Type == voltha.Port_ETHERNET_UNI {
-					assert.Equal(t, 0, len(p.Peers))
-				} else if p.Type == voltha.Port_PON_OLT {
-					assert.Equal(t, nb.numONUPerOLT, len(p.Peers))
-					assert.Equal(t, uint32(1), p.PortNo)
-				} else if p.Type == voltha.Port_PON_ONU {
-					assert.Equal(t, 1, len(p.Peers))
-					assert.Equal(t, uint32(1), p.PortNo)
-				} else {
-					assert.Error(t, errors.New("invalid-port"))
-				}
-			}
-			wg.Done()
-		}(&wg, device)
-	}
-	wg.Wait()
-}
-
-func (nb *NBTest) getADevice(rootDevice bool, nbi *NBIHandler) (*voltha.Device, error) {
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	if err != nil {
-		return nil, err
-	}
-	for _, d := range devices.Items {
-		if d.Root == rootDevice {
-			return d, nil
-		}
-	}
-	return nil, status.Errorf(codes.NotFound, "%v device not found", rootDevice)
-}
-
-func (nb *NBTest) testCoreWithoutData(t *testing.T, nbi *NBIHandler) {
-	lds, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, lds)
-	assert.Equal(t, 0, len(lds.Items))
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, devices)
-	assert.Equal(t, 0, len(devices.Items))
-	adapters, err := nbi.ListAdapters(getContext(), &empty.Empty{})
-	assert.Equal(t, 0, len(adapters.Items))
-	assert.Nil(t, err)
-	assert.NotNil(t, adapters)
-}
-
-func (nb *NBTest) testAdapterRegistration(t *testing.T, nbi *NBIHandler) {
-	ctx := context.Background()
-	adapters, err := nbi.ListAdapters(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, adapters)
-	assert.Equal(t, 2, len(adapters.Items))
-	for _, a := range adapters.Items {
-		switch a.Id {
-		case nb.oltAdapterName:
-			assert.Equal(t, "Voltha-olt", a.Vendor)
-		case nb.onuAdapterName:
-			assert.Equal(t, "Voltha-onu", a.Vendor)
-		default:
-			logger.Fatal(ctx, "unregistered-adapter", a.Id)
-		}
-	}
-	deviceTypes, err := nbi.ListDeviceTypes(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, deviceTypes)
-	assert.Equal(t, 2, len(deviceTypes.Items))
-	for _, dt := range deviceTypes.Items {
-		switch dt.Id {
-		case nb.oltAdapterName:
-			assert.Equal(t, nb.oltAdapterName, dt.Adapter)
-			assert.Equal(t, false, dt.AcceptsBulkFlowUpdate)
-			assert.Equal(t, true, dt.AcceptsAddRemoveFlowUpdates)
-		case nb.onuAdapterName:
-			assert.Equal(t, nb.onuAdapterName, dt.Adapter)
-			assert.Equal(t, false, dt.AcceptsBulkFlowUpdate)
-			assert.Equal(t, true, dt.AcceptsAddRemoveFlowUpdates)
-		default:
-			logger.Fatal(ctx, "invalid-device-type", dt.Id)
-		}
-	}
-}
-
-func (nb *NBTest) testCreateDevice(t *testing.T, nbi *NBIHandler) {
-	//	Create a valid device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, device)
-	assert.Equal(t, oltDevice.String(), device.String())
-
-	// Try to create the same device
-	_, err = nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.NotNil(t, err)
-	assert.Equal(t, "device is already pre-provisioned", err.Error())
-
-	// Try to create a device with invalid data
-	_, err = nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName})
-	assert.NotNil(t, err)
-	assert.Equal(t, "no-device-info-present; MAC or HOSTIP&PORT", err.Error())
-
-	// Ensure we only have 1 device in the Core
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, devices)
-	assert.Equal(t, 1, len(devices.Items))
-	assert.Equal(t, oltDevice.String(), devices.Items[0].String())
-
-	//Remove the device
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-}
-func (nb *NBTest) enableDevice(t *testing.T, nbi *NBIHandler, oltDevice *voltha.Device) {
-	// Create a logical device monitor will automatically send trap and eapol flows to the devices being enables
-	var wg sync.WaitGroup
-	wg.Add(1)
-	go nb.monitorLogicalDevice(t, nbi, 1, nb.numONUPerOLT, &wg, false, false)
-
-	// Enable the oltDevice
-	_, err := nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the logical device to be in the ready state
-	var vldFunction = func(ports []*voltha.LogicalPort) bool {
-		return len(ports) == nb.numONUPerOLT+1
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
-	assert.Nil(t, err)
-
-	// Verify that the devices have been setup correctly
-	nb.verifyDevices(t, nbi)
-
-	// Get latest oltDevice data
-	oltDevice, err = nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Verify that the logical device has been setup correctly
-	nb.verifyLogicalDevices(t, oltDevice, nbi)
-
-	// Wait until all flows has been sent to the devices successfully
-	wg.Wait()
-
-}
-func (nb *NBTest) testForceDeletePreProvDevice(t *testing.T, nbi *NBIHandler) {
-	//	Create a valid device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, device)
-	assert.Equal(t, oltDevice.String(), device.String())
-
-	// Ensure we only have 1 device in the Core
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, devices)
-	assert.Equal(t, 1, len(devices.Items))
-	assert.Equal(t, oltDevice.String(), devices.Items[0].String())
-
-	//Remove the device forcefully
-	_, err = nbi.ForceDeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) testForceDeleteEnabledDevice(t *testing.T, nbi *NBIHandler) {
-	//	Create a valid device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, device)
-	assert.Equal(t, oltDevice.String(), device.String())
-
-	nb.enableDevice(t, nbi, oltDevice)
-
-	//Remove the device forcefully
-	_, err = nbi.ForceDeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) testDeletePreProvDevice(t *testing.T, nbi *NBIHandler) {
-	//	Create a valid device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, device)
-	assert.Equal(t, oltDevice.String(), device.String())
-
-	// Ensure we only have 1 device in the Core
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, devices)
-	assert.Equal(t, 1, len(devices.Items))
-	assert.Equal(t, oltDevice.String(), devices.Items[0].String())
-
-	//Remove the device forcefully
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) testDeleteEnabledDevice(t *testing.T, nbi *NBIHandler) {
-	//	Create a valid device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, device)
-	assert.Equal(t, oltDevice.String(), device.String())
-
-	nb.enableDevice(t, nbi, oltDevice)
-
-	//Remove the device
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) testForceDeleteDeviceFailure(t *testing.T, nbi *NBIHandler) {
-	//	Create a valid device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, device)
-	assert.Equal(t, oltDevice.String(), device.String())
-
-	nb.enableDevice(t, nbi, oltDevice)
-	nb.oltAdapter.SetDeleteAction(true)
-	//Remove the device
-	_, err = nbi.ForceDeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core although delete was failed - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-
-}
-
-func (nb *NBTest) testDeleteDeviceFailure(t *testing.T, nbi *NBIHandler) {
-	//	Create a valid device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, device)
-	assert.Equal(t, oltDevice.String(), device.String())
-
-	nb.enableDevice(t, nbi, oltDevice)
-
-	nb.oltAdapter.SetDeleteAction(true)
-	//Remove the device
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are devices in the Core as delete was failed - wait until condition satisfied or timeout
-	var vFunction1 isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		state, err := nbi.GetTransientState(getContext(), oltDevice.Id)
-		if err != nil {
-			return false
-		}
-		return devices != nil && len(devices.Items) == (nb.numONUPerOLT+1) &&
-			state == voltha.DeviceTransientState_DELETE_FAILED
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction1)
-	assert.Nil(t, err)
-
-	nb.oltAdapter.SetDeleteAction(false)
-
-	// Now Force Delete this device
-	_, err = nbi.ForceDeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are devices in the Core as delete was failed - wait until condition satisfied or timeout
-	var vFunction2 isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction2)
-	assert.Nil(t, err)
-
-}
-
-func (nb *NBTest) testEnableDevice(t *testing.T, nbi *NBIHandler) {
-	// Create a device that has no adapter registered
-	oltDeviceNoAdapter, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: "noAdapterRegistered", MacAddress: "aa:bb:cc:cc:ee:ff"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDeviceNoAdapter)
-
-	// Try to enable the oltDevice and check the error message
-	_, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDeviceNoAdapter.Id})
-	assert.NotNil(t, err)
-	assert.Equal(t, "adapter-not-registered-for-device-type noAdapterRegistered", err.Error())
-
-	//Remove the device
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDeviceNoAdapter.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vdFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vdFunction)
-	assert.Nil(t, err)
-
-	// Create a logical device monitor will automatically send trap and eapol flows to the devices being enables
-	var wg sync.WaitGroup
-	wg.Add(1)
-	go nb.monitorLogicalDevice(t, nbi, 1, nb.numONUPerOLT, &wg, false, false)
-
-	//	Create the device with valid data
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Verify oltDevice exist in the core
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.Equal(t, 1, len(devices.Items))
-	assert.Equal(t, oltDevice.Id, devices.Items[0].Id)
-
-	// Enable the oltDevice
-	_, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the logical device to be in the ready state
-	var vldFunction = func(ports []*voltha.LogicalPort) bool {
-		return len(ports) == nb.numONUPerOLT+1
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
-	assert.Nil(t, err)
-
-	// Verify that the devices have been setup correctly
-	nb.verifyDevices(t, nbi)
-
-	// Get latest oltDevice data
-	oltDevice, err = nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Verify that the logical device has been setup correctly
-	nb.verifyLogicalDevices(t, oltDevice, nbi)
-
-	// Wait until all flows has been sent to the devices successfully
-	wg.Wait()
-}
-
-func (nb *NBTest) testDisableAndReEnableRootDevice(t *testing.T, nbi *NBIHandler) {
-	//Get an OLT device
-	oltDevice, err := nb.getADevice(true, nbi)
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Disable the oltDevice
-	_, err = nbi.DisableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the old device to be disabled
-	var vdFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
-		return device.AdminState == voltha.AdminState_DISABLED && device.OperStatus == voltha.OperStatus_UNKNOWN
-	}
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
-	assert.Nil(t, err)
-
-	// Verify that all onu devices are disabled as well
-	onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
-	assert.Nil(t, err)
-	for _, onu := range onuDevices.Items {
-		err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
-		assert.Nil(t, err)
-	}
-
-	// Wait for the logical device to satisfy the expected condition
-	var vlFunction = func(ports []*voltha.LogicalPort) bool {
-		for _, lp := range ports {
-			if (lp.OfpPort.Config&uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
-				lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LINK_DOWN) {
-				return false
-			}
-		}
-		return true
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-
-	// Reenable the oltDevice
-	_, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the old device to be enabled
-	vdFunction = func(device *voltha.Device) bool {
-		return device.AdminState == voltha.AdminState_ENABLED && device.OperStatus == voltha.OperStatus_ACTIVE
-	}
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
-	assert.Nil(t, err)
-
-	// Verify that all onu devices are enabled as well
-	onuDevices, err = nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
-	assert.Nil(t, err)
-	for _, onu := range onuDevices.Items {
-		err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
-		assert.Nil(t, err)
-	}
-
-	// Wait for the logical device to satisfy the expected condition
-	vlFunction = func(ports []*voltha.LogicalPort) bool {
-		for _, lp := range ports {
-			if (lp.OfpPort.Config&^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
-				lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LIVE) {
-				return false
-			}
-		}
-		return true
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) testDisableAndDeleteAllDevice(t *testing.T, nbi *NBIHandler) {
-	//Get an OLT device
-	oltDevice, err := nb.getADevice(true, nbi)
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Disable the oltDevice
-	_, err = nbi.DisableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the olt device to be disabled
-	var vdFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
-		return device.AdminState == voltha.AdminState_DISABLED && device.OperStatus == voltha.OperStatus_UNKNOWN
-	}
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
-	assert.Nil(t, err)
-
-	// Verify that all onu devices are disabled as well
-	onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
-	assert.Nil(t, err)
-	for _, onu := range onuDevices.Items {
-		err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
-		assert.Nil(t, err)
-	}
-
-	// Delete the oltDevice
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-
-	// Wait for absence of logical device
-	var vlFunction isLogicalDevicesConditionSatisfied = func(lds *voltha.LogicalDevices) bool {
-		return lds != nil && len(lds.Items) == 0
-	}
-
-	err = waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) deleteAllDevices(t *testing.T, nbi *NBIHandler) {
-	devices, _ := nbi.ListDevices(getContext(), &empty.Empty{})
-	if len(devices.Items) == 0 {
-		// Nothing to do
-		return
-	}
-	//Get an OLT device
-	oltDevice, err := nb.getADevice(true, nbi)
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Delete the oltDevice
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for all devices to be deleted
-	vFunction := func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-
-	// Wait for absence of logical device
-	vlFunction := func(lds *voltha.LogicalDevices) bool {
-		return lds != nil && len(lds.Items) == 0
-	}
-
-	err = waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) testEnableAndDeleteAllDevice(t *testing.T, nbi *NBIHandler) {
-	//Create the device with valid data
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	//Get an OLT device
-	oltDevice, err = nb.getADevice(true, nbi)
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Enable the oltDevice
-	_, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the logical device to be in the ready state
-	var vldFunction = func(ports []*voltha.LogicalPort) bool {
-		return len(ports) == nb.numONUPerOLT+1
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
-	assert.Nil(t, err)
-
-	//Get all child devices
-	onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
-	assert.Nil(t, err)
-
-	// Wait for the all onu devices to be enabled
-	var vdFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
-		return device.AdminState == voltha.AdminState_ENABLED
-	}
-	for _, onu := range onuDevices.Items {
-		err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
-		assert.Nil(t, err)
-	}
-	// Wait for each onu device to get deleted
-	var vdFunc isDeviceConditionSatisfied = func(device *voltha.Device) bool {
-		return device == nil
-	}
-
-	// Delete the onuDevice
-	for _, onu := range onuDevices.Items {
-		_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: onu.Id})
-		assert.Nil(t, err)
-		err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunc, nbi)
-		assert.Nil(t, err)
-	}
-
-	// Disable the oltDevice
-	_, err = nbi.DisableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the olt device to be disabled
-	var vFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
-		return device.AdminState == voltha.AdminState_DISABLED && device.OperStatus == voltha.OperStatus_UNKNOWN
-	}
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vFunction, nbi)
-	assert.Nil(t, err)
-
-	// Delete the oltDevice
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	var vFunc isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunc)
-	assert.Nil(t, err)
-}
-func (nb *NBTest) testDisableAndEnablePort(t *testing.T, nbi *NBIHandler) {
-	//Get an OLT device
-	var cp *voltha.Port
-	oltDevice, err := nb.getADevice(true, nbi)
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	oltPorts, err := nbi.ListDevicePorts(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	for _, cp = range oltPorts.Items {
-		if cp.Type == voltha.Port_PON_OLT {
-			break
-		}
-
-	}
-	assert.NotNil(t, cp)
-	cp.DeviceId = oltDevice.Id
-
-	// Disable the NW Port of oltDevice
-	_, err = nbi.DisablePort(getContext(), cp)
-	assert.Nil(t, err)
-	// Wait for the olt device Port  to be disabled
-	var vdFunction isDevicePortsConditionSatisfied = func(ports *voltha.Ports) bool {
-		for _, port := range ports.Items {
-			if port.PortNo == cp.PortNo {
-				return port.AdminState == voltha.AdminState_DISABLED
-			}
-		}
-		return false
-	}
-	err = waitUntilDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
-	assert.Nil(t, err)
-	// Wait for the logical device to satisfy the expected condition
-	var vlFunction = func(ports []*voltha.LogicalPort) bool {
-		for _, lp := range ports {
-			if (lp.OfpPort.Config&^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
-				lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LIVE) {
-				return false
-			}
-		}
-		return true
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-
-	// Enable the NW Port of oltDevice
-	_, err = nbi.EnablePort(getContext(), cp)
-	assert.Nil(t, err)
-
-	// Wait for the olt device Port to be enabled
-	vdFunction = func(ports *voltha.Ports) bool {
-		for _, port := range ports.Items {
-			if port.PortNo == cp.PortNo {
-				return port.AdminState == voltha.AdminState_ENABLED
-			}
-		}
-		return false
-	}
-	err = waitUntilDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
-	assert.Nil(t, err)
-	// Wait for the logical device to satisfy the expected condition
-	vlFunction = func(ports []*voltha.LogicalPort) bool {
-		for _, lp := range ports {
-			if (lp.OfpPort.Config&^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
-				lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LIVE) {
-				return false
-			}
-		}
-		return true
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-
-	// Disable a non-PON port
-	for _, cp = range oltPorts.Items {
-		if cp.Type != voltha.Port_PON_OLT {
-			break
-		}
-
-	}
-	assert.NotNil(t, cp)
-	cp.DeviceId = oltDevice.Id
-
-	// Disable the NW Port of oltDevice
-	_, err = nbi.DisablePort(getContext(), cp)
-	assert.NotNil(t, err)
-
-}
-
-func (nb *NBTest) testDeviceRebootWhenOltIsEnabled(t *testing.T, nbi *NBIHandler) {
-	//Get an OLT device
-	oltDevice, err := nb.getADevice(true, nbi)
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	assert.Equal(t, oltDevice.ConnectStatus, voltha.ConnectStatus_REACHABLE)
-	assert.Equal(t, oltDevice.AdminState, voltha.AdminState_ENABLED)
-
-	// Verify that we have one or more ONUs to start with
-	onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
-	assert.Nil(t, err)
-	assert.NotNil(t, onuDevices)
-	assert.Greater(t, len(onuDevices.Items), 0)
-
-	// Reboot the OLT and very that Connection Status goes to UNREACHABLE and operation status to UNKNOWN
-	_, err = nbi.RebootDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	var vlFunction0 = func(d *voltha.Device) bool {
-		return d.ConnectStatus == voltha.ConnectStatus_UNREACHABLE && d.OperStatus == voltha.OperStatus_UNKNOWN
-	}
-
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vlFunction0, nbi)
-	assert.Nil(t, err)
-
-	// Wait for the logical device to satisfy the expected condition
-	var vlFunction1 = func(ld *voltha.LogicalDevice) bool {
-		return ld == nil
-	}
-
-	err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction1)
-	assert.Nil(t, err)
-
-	// Wait for the device to satisfy the expected condition (device does not have flows)
-	var vlFunction2 = func(d *voltha.Device) bool {
-		var deviceFlows *ofp.Flows
-		var err error
-		if deviceFlows, err = nbi.ListDeviceFlows(getContext(), &voltha.ID{Id: d.Id}); err != nil {
-			return false
-		}
-		return len(deviceFlows.Items) == 0
-	}
-
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vlFunction2, nbi)
-	assert.Nil(t, err)
-
-	// Wait for the device to satisfy the expected condition (there are no child devices)
-	var vlFunction3 = func(d *voltha.Device) bool {
-		var devices *voltha.Devices
-		var err error
-		if devices, err = nbi.ListDevices(getContext(), nil); err != nil {
-			return false
-		}
-		for _, device := range devices.Items {
-			if device.ParentId == d.Id {
-				// We have a child device still left
-				return false
-			}
-		}
-		return true
-	}
-
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vlFunction3, nbi)
-	assert.Nil(t, err)
-
-	// Update the OLT Connection Status to REACHABLE and operation status to ACTIVE
-	// Normally, in a real adapter this happens after connection regain via a heartbeat mechanism with real hardware
-	err = nbi.UpdateDeviceStatus(getContext(), oltDevice.Id, voltha.OperStatus_ACTIVE, voltha.ConnectStatus_REACHABLE)
-	assert.Nil(t, err)
-
-	// Verify the device connection and operation states
-	oltDevice, err = nb.getADevice(true, nbi)
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	assert.Equal(t, oltDevice.ConnectStatus, voltha.ConnectStatus_REACHABLE)
-	assert.Equal(t, oltDevice.AdminState, voltha.AdminState_ENABLED)
-
-	// Wait for the logical device to satisfy the expected condition
-	var vlFunction4 = func(ld *voltha.LogicalDevice) bool {
-		return ld != nil
-	}
-	err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction4)
-	assert.Nil(t, err)
-
-	// Verify that logical device is created again
-	logicalDevices, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, logicalDevices)
-	assert.Equal(t, 1, len(logicalDevices.Items))
-
-	// Verify that we have no ONUs left
-	onuDevices, err = nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
-	assert.Nil(t, err)
-	assert.NotNil(t, onuDevices)
-	assert.Equal(t, 0, len(onuDevices.Items))
-}
-
-func (nb *NBTest) testStartOmciTestAction(t *testing.T, nbi *NBIHandler) {
-	// -----------------------------------------------------------------------
-	// SubTest 1: Omci test action should fail due to nonexistent device id
-
-	request := &voltha.OmciTestRequest{Id: "123", Uuid: "456"}
-	_, err := nbi.StartOmciTestAction(getContext(), request)
-	assert.NotNil(t, err)
-	assert.Equal(t, "rpc error: code = NotFound desc = 123", err.Error())
-
-	// -----------------------------------------------------------------------
-	// SubTest 2: Error should be returned for device with no adapter registered
-
-	// Create a device that has no adapter registered
-	deviceNoAdapter, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: "noAdapterRegisteredOmciTest", MacAddress: "aa:bb:cc:cc:ee:01"})
-	assert.Nil(t, err)
-	assert.NotNil(t, deviceNoAdapter)
-
-	// Omci test action should fail due to nonexistent adapter
-	request = &voltha.OmciTestRequest{Id: deviceNoAdapter.Id, Uuid: "456"}
-	_, err = nbi.StartOmciTestAction(getContext(), request)
-	assert.NotNil(t, err)
-	assert.Equal(t, "adapter-not-registered-for-device-type noAdapterRegisteredOmciTest", err.Error())
-
-	//Remove the device
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: deviceNoAdapter.Id})
-	assert.Nil(t, err)
-
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-
-	// -----------------------------------------------------------------------
-	// SubTest 3: Omci test action should succeed on valid ONU
-
-	//	Create the device with valid data
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Verify oltDevice exist in the core
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.Equal(t, 1, len(devices.Items))
-	assert.Equal(t, oltDevice.Id, devices.Items[0].Id)
-
-	// Enable the oltDevice
-	_, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the logical device to be in the ready state
-	var vldFunction = func(ports []*voltha.LogicalPort) bool {
-		return len(ports) == nb.numONUPerOLT+1
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
-	assert.Nil(t, err)
-
-	// Wait for the olt device to be enabled
-	vdFunction := func(device *voltha.Device) bool {
-		return device.AdminState == voltha.AdminState_ENABLED && device.OperStatus == voltha.OperStatus_ACTIVE
-	}
-	err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
-	assert.Nil(t, err)
-
-	onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
-	assert.Nil(t, err)
-	assert.Greater(t, len(onuDevices.Items), 0)
-
-	onuDevice := onuDevices.Items[0]
-
-	// Omci test action should succeed
-	request = &voltha.OmciTestRequest{Id: onuDevice.Id, Uuid: "456"}
-	resp, err := nbi.StartOmciTestAction(getContext(), request)
-	assert.Nil(t, err)
-	assert.Equal(t, resp.Result, voltha.TestResponse_SUCCESS)
-
-	//Remove the device
-	_, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	//Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.Nil(t, err)
-}
-
-func makeSimpleFlowMod(fa *flows.FlowArgs) *ofp.OfpFlowMod {
-	matchFields := make([]*ofp.OfpOxmField, 0)
-	for _, val := range fa.MatchFields {
-		matchFields = append(matchFields, &ofp.OfpOxmField{Field: &ofp.OfpOxmField_OfbField{OfbField: val}})
-	}
-	return flows.MkSimpleFlowMod(matchFields, fa.Actions, fa.Command, fa.KV)
-}
-
-func createMetadata(cTag int, techProfile int, port int) uint64 {
-	md := 0
-	md = (md | (cTag & 0xFFFF)) << 16
-	md = (md | (techProfile & 0xFFFF)) << 32
-	return uint64(md | (port & 0xFFFFFFFF))
-}
-
-func (nb *NBTest) verifyLogicalDeviceFlowCount(t *testing.T, nbi *NBIHandler, numNNIPorts int, numUNIPorts int, flowAddFail bool) {
-	expectedNumFlows := numNNIPorts*numTrapOnNNIFlows + numNNIPorts*numUNIPorts
-	if flowAddFail {
-		expectedNumFlows = 0
-	}
-	// Wait for logical device to have the flows (or none
-	var vlFunction isLogicalDevicesConditionSatisfied = func(lds *voltha.LogicalDevices) bool {
-		id := ""
-		if lds != nil {
-			id = lds.Items[0].Id
-		}
-		flws, _ := nbi.ListLogicalDeviceFlows(getContext(), &voltha.ID{Id: id})
-		return lds != nil && len(lds.Items) == 1 && len(flws.Items) == expectedNumFlows
-	}
-	// No timeout implies a success
-	err := waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) sendTrapFlows(t *testing.T, nbi *NBIHandler, logicalDeviceID string, ports []*voltha.LogicalPort, meterID uint64, startingVlan int) (numNNIPorts, numUNIPorts int) {
-	// Send flows for the parent device
-	var nniPorts []*voltha.LogicalPort
-	var uniPorts []*voltha.LogicalPort
-	for _, p := range ports {
-		if p.RootPort {
-			nniPorts = append(nniPorts, p)
-		} else {
-			uniPorts = append(uniPorts, p)
-		}
-	}
-	assert.Equal(t, 1, len(nniPorts))
-	//assert.Greater(t, len(uniPorts), 1 )
-	nniPort := nniPorts[0].OfpPort.PortNo
-	maxInt32 := uint64(0xFFFFFFFF)
-	controllerPortMask := uint32(4294967293) // will result in 4294967293&0x7fffffff => 2147483645 which is the actual controller port
-	var fa *flows.FlowArgs
-	fa = &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(nniPort),
-			flows.EthType(35020),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.Output(controllerPortMask),
-		},
-	}
-	flowLLDP := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDeviceID}
-	_, err := nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowLLDP)
-	assert.Nil(t, err)
-
-	fa = &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(nniPort),
-			flows.EthType(2048),
-			flows.IpProto(17),
-			flows.UdpSrc(67),
-			flows.UdpDst(68),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.Output(controllerPortMask),
-		},
-	}
-	flowIPV4 := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDeviceID}
-	_, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowIPV4)
-	assert.Nil(t, err)
-
-	fa = &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(nniPort),
-			flows.EthType(34525),
-			flows.IpProto(17),
-			flows.UdpSrc(546),
-			flows.UdpDst(547),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.Output(controllerPortMask),
-		},
-	}
-	flowIPV6 := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDeviceID}
-	_, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowIPV6)
-	assert.Nil(t, err)
-
-	fa = &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(nniPort),
-			flows.EthType(34915),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.Output(controllerPortMask),
-		},
-	}
-	flowPPPoEP := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDeviceID}
-	_, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowPPPoEP)
-	assert.Nil(t, err)
-
-	return len(nniPorts), len(uniPorts)
-}
-
-func (nb *NBTest) sendEAPFlows(t *testing.T, nbi *NBIHandler, logicalDeviceID string, port *ofp.OfpPort, vlan int, meterID uint64) {
-	maxInt32 := uint64(0xFFFFFFFF)
-	controllerPortMask := uint32(4294967293) // will result in 4294967293&0x7fffffff => 2147483645 which is the actual controller port
-	fa := &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1, "write_metadata": createMetadata(vlan, 64, 0), "meter_id": meterID},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(port.PortNo),
-			flows.EthType(34958),
-			flows.VlanVid(8187),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.Output(controllerPortMask),
-		},
-	}
-	flowEAP := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDeviceID}
-	maxTries := 3
-	var err error
-	for {
-		if _, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowEAP); err == nil {
-			if maxTries < 3 {
-				t.Log("Re-sending EAPOL flow succeeded for port:", port)
-			}
-			break
-		}
-		t.Log("Sending EAPOL flows fail:", err)
-		time.Sleep(50 * time.Millisecond)
-		maxTries--
-		if maxTries == 0 {
-			break
-		}
-	}
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) monitorLogicalDevice(t *testing.T, nbi *NBIHandler, numNNIPorts int, numUNIPorts int, wg *sync.WaitGroup, flowAddFail bool, flowDeleteFail bool) {
-	defer wg.Done()
-
-	// Clear any existing flows on the adapters
-	nb.oltAdapter.ClearFlows()
-	nb.onuAdapter.ClearFlows()
-
-	// Set the adapter actions on flow addition/deletion
-	nb.oltAdapter.SetFlowAction(flowAddFail, flowDeleteFail)
-	nb.onuAdapter.SetFlowAction(flowAddFail, flowDeleteFail)
-
-	// Wait until a logical device is ready
-	var vlFunction isLogicalDevicesConditionSatisfied = func(lds *voltha.LogicalDevices) bool {
-		if lds == nil || len(lds.Items) != 1 || lds.Items[0] == nil {
-			return false
-		}
-		// Ensure there are both NNI ports and at least one UNI port on the logical device
-		ld := lds.Items[0]
-		ports, err := nbi.ListLogicalDevicePorts(getContext(), &voltha.ID{Id: ld.Id})
-		if err != nil {
-			return false
-		}
-		nniPort := false
-		uniPort := false
-		for _, p := range ports.Items {
-			nniPort = nniPort || p.RootPort == true
-			uniPort = uniPort || p.RootPort == false
-			if nniPort && uniPort {
-				return true
-			}
-		}
-		return false
-	}
-	err := waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
-	assert.Nil(t, err)
-
-	logicalDevices, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.NotNil(t, logicalDevices)
-	assert.Equal(t, 1, len(logicalDevices.Items))
-
-	logicalDeviceID := logicalDevices.Items[0].Id
-	meterID := rand.Uint32()
-
-	// Add a meter to the logical device
-	meterMod := &ofp.OfpMeterMod{
-		Command: ofp.OfpMeterModCommand_OFPMC_ADD,
-		Flags:   rand.Uint32(),
-		MeterId: meterID,
-		Bands: []*ofp.OfpMeterBandHeader{
-			{Type: ofp.OfpMeterBandType_OFPMBT_EXPERIMENTER,
-				Rate:      rand.Uint32(),
-				BurstSize: rand.Uint32(),
-				Data:      nil,
-			},
-		},
-	}
-	_, err = nbi.UpdateLogicalDeviceMeterTable(getContext(), &ofp.MeterModUpdate{Id: logicalDeviceID, MeterMod: meterMod})
-	assert.Nil(t, err)
-
-	ports, err := nbi.ListLogicalDevicePorts(getContext(), &voltha.ID{Id: logicalDeviceID})
-	assert.Nil(t, err)
-
-	// Send initial set of Trap flows
-	startingVlan := 4091
-	nb.sendTrapFlows(t, nbi, logicalDeviceID, ports.Items, uint64(meterID), startingVlan)
-
-	// Listen for port events
-	start := time.Now()
-	processedNniLogicalPorts := 0
-	processedUniLogicalPorts := 0
-
-	for event := range nbi.GetChangeEventsQueueForTest() {
-		startingVlan++
-		if portStatus, ok := (event.Event).(*ofp.ChangeEvent_PortStatus); ok {
-			ps := portStatus.PortStatus
-			if ps.Reason == ofp.OfpPortReason_OFPPR_ADD {
-				if ps.Desc.PortNo >= uint32(nb.startingUNIPortNo) {
-					processedUniLogicalPorts++
-					nb.sendEAPFlows(t, nbi, logicalDeviceID, ps.Desc, startingVlan, uint64(meterID))
-				} else {
-					processedNniLogicalPorts++
-				}
-			}
-		}
-
-		if processedNniLogicalPorts >= numNNIPorts && processedUniLogicalPorts >= numUNIPorts {
-			fmt.Println("Total time to send all flows:", time.Since(start))
-			break
-		}
-	}
-	//Verify the flow count on the logical device
-	nb.verifyLogicalDeviceFlowCount(t, nbi, numNNIPorts, numUNIPorts, flowAddFail)
-
-	// Wait until all flows have been sent to the OLT adapters (or all failed)
-	expectedFlowCount := (numNNIPorts * numTrapOnNNIFlows) + numNNIPorts*numUNIPorts
-	if flowAddFail {
-		expectedFlowCount = 0
-	}
-	var oltVFunc isConditionSatisfied = func() bool {
-		return nb.oltAdapter.GetFlowCount() >= expectedFlowCount
-	}
-	err = waitUntilCondition(nb.maxTimeout, nbi, oltVFunc)
-	assert.Nil(t, err)
-
-	// Wait until all flows have been sent to the ONU adapters (or all failed)
-	expectedFlowCount = numUNIPorts
-	if flowAddFail {
-		expectedFlowCount = 0
-	}
-	var onuVFunc isConditionSatisfied = func() bool {
-		return nb.onuAdapter.GetFlowCount() == expectedFlowCount
-	}
-	err = waitUntilCondition(nb.maxTimeout, nbi, onuVFunc)
-	assert.Nil(t, err)
-}
-
-func (nb *NBTest) testFlowAddFailure(t *testing.T, nbi *NBIHandler) {
-
-	// Create a logical device monitor will automatically send trap and eapol flows to the devices being enables
-	var wg sync.WaitGroup
-	wg.Add(1)
-	go nb.monitorLogicalDevice(t, nbi, 1, nb.numONUPerOLT, &wg, true, false)
-
-	//	Create the device with valid data
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Verify oltDevice exist in the core
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.Equal(t, 1, len(devices.Items))
-	assert.Equal(t, oltDevice.Id, devices.Items[0].Id)
-
-	// Enable the oltDevice
-	_, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the logical device to be in the ready state
-	var vldFunction = func(ports []*voltha.LogicalPort) bool {
-		return len(ports) == nb.numONUPerOLT+1
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
-	assert.Nil(t, err)
-
-	// Verify that the devices have been setup correctly
-	nb.verifyDevices(t, nbi)
-
-	// Get latest oltDevice data
-	oltDevice, err = nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Verify that the logical device has been setup correctly
-	nb.verifyLogicalDevices(t, oltDevice, nbi)
-
-	// Wait until all flows has been sent to the devices successfully
-	wg.Wait()
-}
-
-func (nb *NBTest) testMPLSFlowsAddition(t *testing.T, nbi *NBIHandler) {
-	// Check whether Device already exist
-	devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.NoError(t, err)
-	testLogger.Infow(getContext(), "device-list", log.Fields{"devices": devices})
-	for _, dev := range devices.GetItems() {
-		// Delete the found device for fresh start
-		testLogger.Warnf(getContext(), "deleting-existing-device", dev.GetId())
-		_, err := nbi.DeleteDevice(context.Background(), &voltha.ID{
-			Id: dev.GetId(),
-		})
-		assert.NoError(t, err)
-	}
-
-	// Ensure there are no devices in the Core now - wait until condition satisfied or timeout
-	var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
-		return devices != nil && len(devices.Items) == 0
-	}
-	err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
-	assert.NoError(t, err)
-
-	// Get list of devices, to make sure the above operation deleted all the devices
-	devices, err = nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.NoError(t, err)
-	assert.Equal(t, 0, len(devices.Items))
-
-	// Create device
-	oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ff"})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-
-	// Verify oltDevice exist in the core
-	devices, err = nbi.ListDevices(getContext(), &empty.Empty{})
-	assert.Nil(t, err)
-	assert.Equal(t, 1, len(devices.Items))
-	assert.Equal(t, oltDevice.Id, devices.Items[0].Id)
-
-	// Enable the oltDevice
-	_, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-
-	// Wait for the logical device to be in the ready state
-	var vldFunction = func(ports []*voltha.LogicalPort) bool {
-		return len(ports) == nb.numONUPerOLT+1
-	}
-	err = waitUntilLogicalDevicePortsReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
-	assert.Nil(t, err)
-
-	// Verify that the devices have been setup correctly
-	nb.verifyDevices(t, nbi)
-
-	// Get latest oltDevice data
-	oltDevice, err = nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
-	assert.Nil(t, err)
-	assert.NotNil(t, oltDevice)
-	testLogger.Infow(getContext(), "olt-device-created-and-verified", log.Fields{"device-id": oltDevice.GetId()})
-
-	// Verify that the logical device has been setup correctly
-	nb.verifyLogicalDevices(t, oltDevice, nbi)
-
-	logicalDevices, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
-	assert.NoError(t, err)
-
-	testLogger.Infow(getContext(), "list-logical-devices", log.Fields{"logical-device": logicalDevices.GetItems()[0]})
-	// Add a meter to the logical device, which the flow can refer to
-	meterMod := &ofp.OfpMeterMod{
-		Command: ofp.OfpMeterModCommand_OFPMC_ADD,
-		Flags:   rand.Uint32(),
-		MeterId: 1,
-		Bands: []*ofp.OfpMeterBandHeader{
-			{Type: ofp.OfpMeterBandType_OFPMBT_EXPERIMENTER,
-				Rate:      rand.Uint32(),
-				BurstSize: rand.Uint32(),
-				Data:      nil,
-			},
-		},
-	}
-	_, err = nbi.UpdateLogicalDeviceMeterTable(getContext(), &ofp.MeterModUpdate{
-		Id:       logicalDevices.GetItems()[0].GetId(),
-		MeterMod: meterMod,
-	})
-	assert.NoError(t, err)
-
-	meters, err := nbi.ListLogicalDeviceMeters(getContext(), &voltha.ID{Id: logicalDevices.GetItems()[0].GetId()})
-	assert.NoError(t, err)
-
-	for _, item := range meters.GetItems() {
-		testLogger.Infow(getContext(), "list-logical-device-meters", log.Fields{"meter-config": item.GetConfig()})
-	}
-
-	logicalPorts, err := nbi.ListLogicalDevicePorts(context.Background(), &voltha.ID{Id: logicalDevices.GetItems()[0].GetId()})
-	assert.NoError(t, err)
-	m := jsonpb.Marshaler{}
-	logicalPortsJson, err := m.MarshalToString(logicalPorts)
-	assert.NoError(t, err)
-
-	testLogger.Infow(getContext(), "list-logical-ports", log.Fields{"ports": logicalPortsJson})
-
-	callables := []func() *ofp.OfpFlowMod{getOnuUpstreamRules, getOltUpstreamRules, getOLTDownstreamMplsSingleTagRules,
-		getOLTDownstreamMplsDoubleTagRules, getOLTDownstreamRules, getOnuDownstreamRules}
-
-	for _, callable := range callables {
-		_, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &ofp.FlowTableUpdate{Id: logicalDevices.GetItems()[0].GetId(), FlowMod: callable()})
-		assert.NoError(t, err)
-	}
-}
-
-func getOnuUpstreamRules() (flowMod *ofp.OfpFlowMod) {
-	fa := &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 1000, "table_id": 1, "meter_id": 1, "write_metadata": 4100100000},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(103),
-			flows.VlanVid(4096),
-		},
-		Actions: []*ofp.OfpAction{},
-	}
-
-	flowMod = makeSimpleFlowMod(fa)
-	flowMod.TableId = 0
-	m := jsonpb.Marshaler{}
-	flowModJson, _ := m.MarshalToString(flowMod)
-	testLogger.Infow(getContext(), "onu-upstream-flow", log.Fields{"flow-mod": flowModJson})
-	return
-}
-
-func getOltUpstreamRules() (flowMod *ofp.OfpFlowMod) {
-	fa := &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 1000, "table_id": 1, "meter_id": 1, "write_metadata": 4100000000},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(103),
-			flows.VlanVid(4096),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.PushVlan(0x8100),
-			flows.SetField(flows.VlanVid(2)),
-			flows.SetField(flows.EthSrc(1111)),
-			flows.SetField(flows.EthDst(2222)),
-			flows.PushVlan(0x8847),
-			flows.SetField(flows.MplsLabel(100)),
-			flows.SetField(flows.MplsBos(1)),
-			flows.PushVlan(0x8847),
-			flows.SetField(flows.MplsLabel(200)),
-			flows.MplsTtl(64),
-			flows.Output(2),
-		},
-	}
-	flowMod = makeSimpleFlowMod(fa)
-	flowMod.TableId = 1
-	m := jsonpb.Marshaler{}
-	flowModJson, _ := m.MarshalToString(flowMod)
-	testLogger.Infow(getContext(), "olt-upstream-flow", log.Fields{"flow-mod": flowModJson})
-	return
-}
-
-func getOLTDownstreamMplsSingleTagRules() (flowMod *ofp.OfpFlowMod) {
-	fa := &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 1000, "table_id": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(2),
-			flows.Metadata_ofp((1000 << 32) | 1),
-			flows.EthType(0x8847),
-			flows.MplsBos(1),
-			flows.EthSrc(2222),
-		},
-		Actions: []*ofp.OfpAction{
-			{Type: ofp.OfpActionType_OFPAT_DEC_MPLS_TTL, Action: &ofp.OfpAction_MplsTtl{MplsTtl: &ofp.OfpActionMplsTtl{MplsTtl: 62}}},
-			flows.PopMpls(0x8847),
-		},
-	}
-	flowMod = makeSimpleFlowMod(fa)
-	flowMod.TableId = 0
-	m := jsonpb.Marshaler{}
-	flowModJson, _ := m.MarshalToString(flowMod)
-	testLogger.Infow(getContext(), "olt-mpls-downstream-single-tag-flow", log.Fields{"flow-mod": flowModJson})
-	return
-}
-
-func getOLTDownstreamMplsDoubleTagRules() (flowMod *ofp.OfpFlowMod) {
-	fa := &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 1000, "table_id": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(2),
-			flows.EthType(0x8847),
-			flows.EthSrc(2222),
-		},
-		Actions: []*ofp.OfpAction{
-			{Type: ofp.OfpActionType_OFPAT_DEC_MPLS_TTL, Action: &ofp.OfpAction_MplsTtl{MplsTtl: &ofp.OfpActionMplsTtl{MplsTtl: 62}}},
-			flows.PopMpls(0x8847),
-			flows.PopMpls(0x8847),
-		},
-	}
-	flowMod = makeSimpleFlowMod(fa)
-	flowMod.TableId = 0
-	m := jsonpb.Marshaler{}
-	flowModJson, _ := m.MarshalToString(flowMod)
-	testLogger.Infow(getContext(), "olt-mpls-downstream-double-tagged-flow", log.Fields{"flow-mod": flowModJson})
-	return
-}
-
-func getOLTDownstreamRules() (flowMod *ofp.OfpFlowMod) {
-	fa := &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 1000, "table_id": 2, "meter_id": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(2),
-			flows.VlanVid(2),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.PopVlan(),
-		},
-	}
-	flowMod = makeSimpleFlowMod(fa)
-	flowMod.TableId = 1
-	m := jsonpb.Marshaler{}
-	flowModJson, _ := m.MarshalToString(flowMod)
-	testLogger.Infow(getContext(), "olt-downstream-flow", log.Fields{"flow-mod": flowModJson})
-	return
-}
-
-func getOnuDownstreamRules() (flowMod *ofp.OfpFlowMod) {
-	fa := &flows.FlowArgs{
-		KV: flows.OfpFlowModArgs{"priority": 1000, "meter_id": 1},
-		MatchFields: []*ofp.OfpOxmOfbField{
-			flows.InPort(2),
-			flows.Metadata_ofp((1000 << 32) | 1),
-			flows.VlanVid(4096),
-		},
-		Actions: []*ofp.OfpAction{
-			flows.Output(103),
-		},
-	}
-	flowMod = makeSimpleFlowMod(fa)
-	flowMod.TableId = 2
-	m := jsonpb.Marshaler{}
-	flowModJson, _ := m.MarshalToString(flowMod)
-	testLogger.Infow(getContext(), "onu-downstream-flow", log.Fields{"flow-mod": flowModJson})
-	return
-}
-
-func TestSuiteNbiApiHandler(t *testing.T) {
-	ctx := context.Background()
-	f, err := os.Create("../../../tests/results/profile.cpu")
-	if err != nil {
-		logger.Fatalf(ctx, "could not create CPU profile: %v\n ", err)
-	}
-	defer func() {
-		err = f.Close()
-		if err != nil {
-			logger.Errorf(ctx, "failed to close file: %v\n", err)
-		}
-	}()
-	runtime.SetBlockProfileRate(1)
-	runtime.SetMutexProfileFraction(-1)
-	if err := pprof.StartCPUProfile(f); err != nil {
-		logger.Fatalf(ctx, "could not start CPU profile: %v\n", err)
-	}
-	defer pprof.StopCPUProfile()
-
-	//log.SetPackageLogLevel("github.com/opencord/voltha-go/rw_core/core", log.DebugLevel)
-
-	nb := newNBTest(ctx)
-	assert.NotNil(t, nb)
-
-	defer nb.stopAll(ctx)
-
-	// Start the Core
-	nb.startCore(false)
-
-	// Set the grpc API interface - no grpc server is running in unit test
-	nbi := NewNBIHandler(nb.deviceMgr, nb.logicalDeviceMgr, nb.adapterMgr)
-
-	// 1. Basic test with no data in Core
-	nb.testCoreWithoutData(t, nbi)
-
-	// Create/register the adapters
-	nb.oltAdapter, nb.onuAdapter = tst.CreateAndregisterAdapters(ctx, t, nb.kClient, nb.coreInstanceID, nb.oltAdapterName, nb.onuAdapterName, nb.adapterMgr)
-	nb.numONUPerOLT = nb.oltAdapter.GetNumONUPerOLT()
-	nb.startingUNIPortNo = nb.oltAdapter.GetStartingUNIPortNo()
-
-	// 2. Test adapter registration
-	nb.testAdapterRegistration(t, nbi)
-
-	numberOfTestRuns := 2
-	for i := 1; i <= numberOfTestRuns; i++ {
-
-		// 3. Test create device
-		nb.testCreateDevice(t, nbi)
-
-		// 4. Test Delete Device Scenarios
-		nb.testForceDeletePreProvDevice(t, nbi)
-		nb.testDeletePreProvDevice(t, nbi)
-		nb.testForceDeleteEnabledDevice(t, nbi)
-		nb.testDeleteEnabledDevice(t, nbi)
-		nb.testForceDeleteDeviceFailure(t, nbi)
-		nb.testDeleteDeviceFailure(t, nbi)
-
-		// 5. Test Enable a device
-		nb.testEnableDevice(t, nbi)
-
-		// 6. Test disable and ReEnable a root device
-		nb.testDisableAndReEnableRootDevice(t, nbi)
-
-		// 7. Test disable and Enable pon port of OLT device
-		nb.testDisableAndEnablePort(t, nbi)
-
-		// 8.Test Device unreachable when OLT is enabled
-		nb.testDeviceRebootWhenOltIsEnabled(t, nbi)
-
-		// 9. Test disable and delete all devices
-		nb.testDisableAndDeleteAllDevice(t, nbi)
-
-		// 10. Test enable and delete all devices
-		nb.testEnableAndDeleteAllDevice(t, nbi)
-
-		// 11. Test omci test
-		nb.testStartOmciTestAction(t, nbi)
-
-		// 12. Remove all devices from tests above
-		nb.deleteAllDevices(t, nbi)
-
-		// 13. Test flow add failure
-		nb.testFlowAddFailure(t, nbi)
-
-		// 14. Clean up
-		nb.deleteAllDevices(t, nbi)
-	}
-}
-
-func TestFlowAddition(t *testing.T) {
-	ctx := context.Background()
-	nb := newNBTest(ctx)
-	assert.NotNil(t, nb)
-
-	defer nb.stopAll(ctx)
-
-	// Start the Core
-	nb.startCore(false)
-
-	// Set the grpc API interface - no grpc server is running in unit test
-	nbi := NewNBIHandler(nb.deviceMgr, nb.logicalDeviceMgr, nb.adapterMgr)
-
-	// Create/register the adapters
-	nb.oltAdapter, nb.onuAdapter = tst.CreateAndregisterAdapters(ctx, t, nb.kClient, nb.coreInstanceID, nb.oltAdapterName, nb.onuAdapterName, nb.adapterMgr)
-	nb.numONUPerOLT = nb.oltAdapter.GetNumONUPerOLT()
-	nb.startingUNIPortNo = nb.oltAdapter.GetStartingUNIPortNo()
-
-	// 2. Test adapter registration
-	nb.testAdapterRegistration(t, nbi)
-
-	// 3. Test MPLS flows addition where:
-	/*
-		Upstream
-		ONU
-		ADDED, bytes=0, packets=0, table=0, priority=1000, selector=[IN_PORT:32, VLAN_VID:ANY], treatment=[immediate=[],
-		transition=TABLE:1, meter=METER:1, metadata=METADATA:4100010000/0]
-		OLT
-		ADDED, bytes=0, packets=0, table=1, priority=1000, selector=[IN_PORT:32, VLAN_VID:ANY], treatment=[immediate=[VLAN_PUSH:vlan,
-		VLAN_ID:2, MPLS_PUSH:mpls_unicast, MPLS_LABEL:YYY,MPLS_BOS:true, MPLS_PUSH:mpls_unicast ,MPLS_LABEL:XXX, MPLS_BOS:false,
-		EXTENSION:of:0000000000000227/VolthaPushL2Header{​​​​​​​}​​​​​​​, ETH_SRC:OLT_MAC, ETH_DST:LEAF_MAC,  TTL:64, OUTPUT:65536],
-		meter=METER:1, metadata=METADATA:4100000000/0]
-
-		Downstream
-		OLT
-		//Below flow rule to pop L2 Ethernet headers from packets which have a single MPLS label
-		ADDED, bytes=0, packets=0, table=0, priority=1000, selector=[IN_PORT:65536, ETH_TYPE:mpls_unicast, MPLS_BOS:true, ETH_SRC:LEAF_MAC],
-		treatment=[DefaultTrafficTreatment{immediate=[DEC_MPLS_TTL, TTL_IN, MPLS_POP:mpls_unicast, EXTENSION:of:0000000000000227/VolthaPopL2Header{},
-		transition=TABLE:1]
-
-		//Below flow rule to pop L2 Ethernet headers from packets which have two MPLS label
-		ADDED, bytes=0, packets=0, table=0, priority=1000, selector=[IN_PORT:65536, ETH_TYPE:mpls_unicast, MPLS_BOS:false, ETH_SRC:LEAF_MAC],
-		treatment=[DefaultTrafficTreatment{immediate=[DEC_MPLS_TTL, TTL_IN, MPLS_POP:mpls_unicast, MPLS_POP:mpls_unicast ,
-		EXTENSION:of:0000000000000227/VolthaPopL2Header{}, transition=TABLE:1]
-
-		//Below flow rules are unchanged from the current implementations except for the table numbers
-		ADDED, bytes=0, packets=0, table=1, priority=1000, selector=[IN_PORT:65536, VLAN_VID:2], treatment=[immediate=[VLAN_POP], transition=TABLE:2,
-		meter=METER:2, metadata=METADATA:1000004100000020/0]
-		ONU
-		ADDED, bytes=0, packets=0, table=2, priority=1000, selector=[IN_PORT:65536, METADATA:20 VLAN_VID:ANY], treatment=[immediate=[OUTPUT:32],
-		meter=METER:2, metadata=METADATA:4100000000/0]
-	*/
-	nb.testMPLSFlowsAddition(t, nbi)
-}
diff --git a/rw_core/core/common.go b/rw_core/core/common.go
index 57d0119..b101d8a 100644
--- a/rw_core/core/common.go
+++ b/rw_core/core/common.go
@@ -18,7 +18,7 @@
 package core
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/core.go b/rw_core/core/core.go
index c52f170..3fa68ef 100644
--- a/rw_core/core/core.go
+++ b/rw_core/core/core.go
@@ -25,65 +25,75 @@
 	"github.com/opencord/voltha-go/rw_core/core/adapter"
 	"github.com/opencord/voltha-go/rw_core/core/api"
 	"github.com/opencord/voltha-go/rw_core/core/device"
-	conf "github.com/opencord/voltha-lib-go/v5/pkg/config"
-	grpcserver "github.com/opencord/voltha-lib-go/v5/pkg/grpc"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-lib-go/v5/pkg/probe"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	conf "github.com/opencord/voltha-lib-go/v7/pkg/config"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events"
+	grpcserver "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"github.com/opencord/voltha-protos/v5/go/extension"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc"
 )
 
 // Core represent read,write core attributes
 type Core struct {
-	shutdown context.CancelFunc
-	stopped  chan struct{}
+	Shutdown    context.CancelFunc
+	Stopped     chan struct{}
+	KafkaClient kafka.Client
 }
 
 const (
-	adapterMessageBus = "adapter-message-bus"
-	clusterMessageBus = "cluster-message-bus"
+	clusterMessagingService = "cluster-message-service"
+	grpcNBIService          = "grpc-nbi-service"
+	grpcSBIService          = "grpc-sbi-service"
+	adapterService          = "adapter-service"
+	kvService               = "kv-service"
+	deviceService           = "device-service"
+	logicalDeviceService    = "logical-device-service"
 )
 
 // NewCore creates instance of rw core
-func NewCore(ctx context.Context, id string, cf *config.RWCoreFlags) *Core {
+func NewCore(ctx context.Context, id string, cf *config.RWCoreFlags) (*Core, context.Context) {
 	// If the context has a probe then fetch it and register our services
 	if p := probe.GetProbeFromContext(ctx); p != nil {
 		p.RegisterService(
 			ctx,
-			adapterMessageBus,
-			"kv-store",
-			"adapter-manager",
-			"device-manager",
-			"logical-device-manager",
-			"grpc-service",
-			"adapter-request-handler",
+			kvService,
+			adapterService,
+			grpcSBIService,
+			clusterMessagingService,
+			deviceService,
+			logicalDeviceService,
 		)
-
-		if cf.KafkaAdapterAddress != cf.KafkaClusterAddress {
-			p.RegisterService(
-				ctx,
-				clusterMessageBus,
-			)
-		}
 	}
 
+	// create kafka client for events
+	KafkaClient := kafka.NewSaramaClient(
+		kafka.Address(cf.KafkaClusterAddress),
+		kafka.ProducerReturnOnErrors(true),
+		kafka.ProducerReturnOnSuccess(true),
+		kafka.ProducerMaxRetries(6),
+		kafka.ProducerRetryBackoff(time.Millisecond*30),
+		kafka.AutoCreateTopic(true),
+		kafka.MetadatMaxRetries(15),
+	)
+
 	// new threads will be given a new cancelable context, so that they can be aborted later when Stop() is called
 	shutdownCtx, cancelCtx := context.WithCancel(ctx)
 
-	core := &Core{shutdown: cancelCtx, stopped: make(chan struct{})}
-	go core.start(shutdownCtx, id, cf)
-	return core
+	rwCore := &Core{Shutdown: cancelCtx, Stopped: make(chan struct{}), KafkaClient: KafkaClient}
+	return rwCore, shutdownCtx
 }
 
-func (core *Core) start(ctx context.Context, id string, cf *config.RWCoreFlags) {
+func (core *Core) Start(ctx context.Context, id string, cf *config.RWCoreFlags) {
 	logger.Info(ctx, "starting-core-services", log.Fields{"coreId": id})
 
 	// deferred functions are used to run cleanup
 	// failing partway will stop anything that's been started
-	defer close(core.stopped)
-	defer core.shutdown()
+	defer close(core.Stopped)
+	defer core.Shutdown()
 
 	logger.Info(ctx, "starting-rw-core-components")
 
@@ -104,91 +114,69 @@
 	backend.LivenessChannelInterval = cf.LiveProbeInterval / 2
 
 	// wait until connection to KV Store is up
-	if err := waitUntilKVStoreReachableOrMaxTries(ctx, kvClient, cf.MaxConnectionRetries, cf.ConnectionRetryInterval); err != nil {
+	if err := waitUntilKVStoreReachableOrMaxTries(ctx, kvClient, cf.MaxConnectionRetries, cf.ConnectionRetryInterval, kvService); err != nil {
 		logger.Fatal(ctx, "unable-to-connect-to-kv-store")
 	}
-	go monitorKVStoreLiveness(ctx, backend, cf.LiveProbeInterval, cf.NotLiveProbeInterval)
+	go monitorKVStoreLiveness(ctx, backend, kvService, cf.LiveProbeInterval, cf.NotLiveProbeInterval)
 
-	// create kafka client
-	kafkaClient := kafka.NewSaramaClient(
-		kafka.Address(cf.KafkaAdapterAddress),
-		kafka.ConsumerType(kafka.GroupCustomer),
-		kafka.ProducerReturnOnErrors(true),
-		kafka.ProducerReturnOnSuccess(true),
-		kafka.ProducerMaxRetries(6),
-		kafka.NumPartitions(3),
-		kafka.ConsumerGroupName(id),
-		kafka.ConsumerGroupPrefix(id),
-		kafka.AutoCreateTopic(true),
-		kafka.ProducerFlushFrequency(5),
-		kafka.ProducerRetryBackoff(time.Millisecond*30),
-		kafka.LivenessChannelInterval(cf.LiveProbeInterval/2),
-	)
-
-	// create kafka client for events
-	kafkaClientEvent := kafka.NewSaramaClient(
-		kafka.Address(cf.KafkaClusterAddress),
-		kafka.ProducerReturnOnErrors(true),
-		kafka.ProducerReturnOnSuccess(true),
-		kafka.ProducerMaxRetries(6),
-		kafka.ProducerRetryBackoff(time.Millisecond*30),
-		kafka.AutoCreateTopic(true),
-		kafka.MetadatMaxRetries(15),
-	)
-
-	// create event proxy
-	updateProbeClusterService := cf.KafkaAdapterAddress != cf.KafkaClusterAddress
-	eventProxy, err := startEventProxy(ctx, kafkaClientEvent, cf.EventTopic, cf.ConnectionRetryInterval, updateProbeClusterService)
-	if err != nil {
-		logger.Warn(ctx, "failed-to-setup-kafka-event-proxy-connection")
-		return
+	// Start kafka communications and artefacts
+	if err := kafka.StartAndWaitUntilKafkaConnectionIsUp(ctx, core.KafkaClient, cf.ConnectionRetryInterval, clusterMessagingService); err != nil {
+		logger.Fatal(ctx, "unable-to-connect-to-kafka")
 	}
-	if cf.KafkaAdapterAddress != cf.KafkaClusterAddress {
-		// if we're using a single kafka cluster we don't need two liveliness probes on the same cluster
-		go monitorKafkaLiveness(ctx, eventProxy, cf.LiveProbeInterval, cf.NotLiveProbeInterval, clusterMessageBus)
-	}
+	defer core.KafkaClient.Stop(ctx)
 
-	defer stopEventProxy(ctx, kafkaClientEvent, eventProxy)
+	// Create the event proxy to post events to KAFKA
+	eventProxy := events.NewEventProxy(events.MsgClient(core.KafkaClient), events.MsgTopic(kafka.Topic{Name: cf.EventTopic}))
+	go func() {
+		if err := eventProxy.Start(); err != nil {
+			logger.Fatalw(ctx, "event-proxy-cannot-start", log.Fields{"error": err})
+		}
+	}()
+	defer eventProxy.Stop()
+
+	// Start the kafka monitoring routine
+	go kafka.MonitorKafkaReadiness(ctx, core.KafkaClient, cf.LiveProbeInterval, cf.NotLiveProbeInterval, clusterMessagingService)
 
 	// create kv path
 	dbPath := model.NewDBPath(backend)
 
 	// load adapters & device types while other things are starting
-	adapterMgr := adapter.NewAdapterManager(ctx, dbPath, id, kafkaClient)
-	go adapterMgr.Start(ctx)
-
-	// connect to kafka, then wait until reachable and publisher/consumer created
-	// core.kmp must be created before deviceMgr and adapterMgr
-	kmp, err := startKafkInterContainerProxy(ctx, kafkaClient, cf.KafkaAdapterAddress, cf.CoreTopic, cf.ConnectionRetryInterval)
-	if err != nil {
-		logger.Warn(ctx, "failed-to-setup-kafka-adapter-proxy-connection")
-		return
-	}
-	defer kmp.Stop(ctx)
-	go monitorKafkaLiveness(ctx, kmp, cf.LiveProbeInterval, cf.NotLiveProbeInterval, adapterMessageBus)
+	adapterMgr := adapter.NewAdapterManager(dbPath, id, backend, cf.LiveProbeInterval)
+	adapterMgr.Start(ctx, adapterService)
 
 	// create the core of the system, the device managers
-	endpointMgr := kafka.NewEndpointManager(backend)
-	deviceMgr, logicalDeviceMgr := device.NewManagers(dbPath, adapterMgr, kmp, endpointMgr, cf, id, eventProxy)
+	deviceMgr, logicalDeviceMgr := device.NewManagers(dbPath, adapterMgr, cf, id, eventProxy)
 
 	// Start the device manager to load the devices. Wait until it is completed to prevent multiple loading happening
 	// triggered by logicalDeviceMgr.Start(Ctx)
-	deviceMgr.Start(ctx)
+	err = deviceMgr.Start(ctx, deviceService)
+	if err != nil {
+		logger.Fatalw(ctx, "failure-starting-device-manager", log.Fields{"error": err})
+	}
 
 	// Start the logical device manager to load the logical devices.
-	logicalDeviceMgr.Start(ctx)
+	logicalDeviceMgr.Start(ctx, logicalDeviceService)
 
-	// register kafka RPC handler
-	registerAdapterRequestHandlers(ctx, kmp, deviceMgr, adapterMgr, cf, "adapter-request-handler")
+	// Create and start the SBI gRPC service
+	grpcSBIServer := grpcserver.NewGrpcServer(cf.GrpcSBIAddress, nil, false, probe.GetProbeFromContext(ctx))
+	go startGrpcSbiService(ctx, grpcSBIServer, grpcSBIService, api.NewAPIHandler(deviceMgr, nil, adapterMgr))
+	defer grpcSBIServer.Stop()
 
-	// start gRPC handler
-	grpcServer := grpcserver.NewGrpcServer(cf.GrpcAddress, nil, false, probe.GetProbeFromContext(ctx))
+	// In the case of a restart, let's wait until all the registered adapters are connected to the Core
+	// before starting the grpc server that handles NBI requests.
+	err = adapterMgr.WaitUntilConnectionsToAdaptersAreUp(ctx, cf.ConnectionRetryInterval)
+	if err != nil {
+		logger.Fatalw(ctx, "failure-connecting-to-adapters", log.Fields{"error": err})
+	}
+
+	// Create the NBI gRPC server
+	grpcNBIServer := grpcserver.NewGrpcServer(cf.GrpcNBIAddress, nil, false, probe.GetProbeFromContext(ctx))
 
 	//Register the 'Extension' service on this gRPC server
-	addGRPCExtensionService(ctx, grpcServer, device.GetNewExtensionManager(deviceMgr))
+	addGRPCExtensionService(ctx, grpcNBIServer, device.GetNewExtensionManager(deviceMgr))
 
-	go startGRPCService(ctx, grpcServer, api.NewNBIHandler(deviceMgr, logicalDeviceMgr, adapterMgr))
-	defer grpcServer.Stop()
+	go startGrpcNbiService(ctx, grpcNBIServer, grpcNBIService, api.NewAPIHandler(deviceMgr, logicalDeviceMgr, adapterMgr))
+	defer grpcNBIServer.Stop()
 
 	// wait for core to be stopped, via Stop() or context cancellation, before running deferred functions
 	<-ctx.Done()
@@ -196,23 +184,33 @@
 
 // Stop brings down core services
 func (core *Core) Stop() {
-	core.shutdown()
-	<-core.stopped
+	core.Shutdown()
+	<-core.Stopped
 }
 
-// startGRPCService creates the grpc service handlers, registers it to the grpc server and starts the server
-func startGRPCService(ctx context.Context, server *grpcserver.GrpcServer, handler voltha.VolthaServiceServer) {
-	logger.Info(ctx, "grpc-server-created")
+// startGrpcSbiService creates the grpc core service handlers, registers it to the grpc server and starts the server
+func startGrpcSbiService(ctx context.Context, server *grpcserver.GrpcServer, serviceName string, handler core.CoreServiceServer) {
+	logger.Infow(ctx, "starting-grpc-sbi-service", log.Fields{"service": serviceName})
+
+	server.AddService(func(server *grpc.Server) { core.RegisterCoreServiceServer(server, handler) })
+	logger.Infow(ctx, "grpc-sbi-service-added", log.Fields{"service": serviceName})
+
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+	logger.Infow(ctx, "grpc-sbi-server-started", log.Fields{"service": serviceName})
+	server.Start(ctx)
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusStopped)
+}
+
+// startGrpcNbiService creates the grpc NBI service handlers, registers it to the grpc server and starts the server
+func startGrpcNbiService(ctx context.Context, server *grpcserver.GrpcServer, serviceName string, handler voltha.VolthaServiceServer) {
+	logger.Infow(ctx, "starting-grpc-nbi-service", log.Fields{"service": serviceName})
 
 	server.AddService(func(gs *grpc.Server) { voltha.RegisterVolthaServiceServer(gs, handler) })
-	logger.Info(ctx, "grpc-service-added")
+	logger.Infow(ctx, "grpc-nbi-service-added-and-started", log.Fields{"service": serviceName})
 
-	probe.UpdateStatusFromContext(ctx, "grpc-service", probe.ServiceStatusRunning)
-	logger.Info(ctx, "grpc-server-started")
 	// Note that there is a small window here in which the core could return its status as ready,
 	// when it really isn't.  This is unlikely to cause issues, as the delay is incredibly short.
 	server.Start(ctx)
-	probe.UpdateStatusFromContext(ctx, "grpc-service", probe.ServiceStatusStopped)
 }
 
 func addGRPCExtensionService(ctx context.Context, server *grpcserver.GrpcServer, handler extension.ExtensionServer) {
@@ -221,5 +219,4 @@
 	server.AddService(func(server *grpc.Server) {
 		extension.RegisterExtensionServer(server, handler)
 	})
-
 }
diff --git a/rw_core/core/device/agent.go b/rw_core/core/device/agent.go
index cd879c6..1a83cb2 100755
--- a/rw_core/core/device/agent.go
+++ b/rw_core/core/device/agent.go
@@ -25,9 +25,11 @@
 	"sync"
 	"time"
 
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
+	"github.com/opencord/voltha-protos/v5/go/core"
+
 	"github.com/cenkalti/backoff/v3"
 	"github.com/gogo/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
 	"github.com/golang/protobuf/ptypes/empty"
 	"github.com/opencord/voltha-go/rw_core/config"
 	"google.golang.org/grpc/codes"
@@ -38,32 +40,34 @@
 	"github.com/opencord/voltha-go/rw_core/core/device/flow"
 	"github.com/opencord/voltha-go/rw_core/core/device/group"
 	"github.com/opencord/voltha-go/rw_core/core/device/port"
-	"github.com/opencord/voltha-go/rw_core/core/device/remote"
 	"github.com/opencord/voltha-go/rw_core/core/device/transientstate"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/extension"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
+var errReconcileAborted = errors.New("reconcile aborted")
+var errContextExpired = errors.New("context expired")
+
 // Agent represents device agent attributes
 type Agent struct {
 	deviceID             string
 	parentID             string
 	deviceType           string
+	adapterEndpoint      string
 	isRootDevice         bool
-	adapterProxy         *remote.AdapterProxy
 	adapterMgr           *adapter.Manager
 	deviceMgr            *Manager
 	dbProxy              *model.Proxy
 	exitChannel          chan int
 	device               *voltha.Device
 	requestQueue         *coreutils.RequestQueue
-	defaultTimeout       time.Duration
+	internalTimeout      time.Duration
+	rpcTimeout           time.Duration
 	startOnce            sync.Once
 	stopOnce             sync.Once
 	stopped              bool
@@ -78,7 +82,7 @@
 }
 
 //newAgent creates a new device agent. The device will be initialized when start() is called.
-func newAgent(ap *remote.AdapterProxy, device *voltha.Device, deviceMgr *Manager, dbPath *model.Path, deviceProxy *model.Proxy, timeout time.Duration) *Agent {
+func newAgent(device *voltha.Device, deviceMgr *Manager, dbPath *model.Path, deviceProxy *model.Proxy, internalTimeout, rpcTimeout time.Duration) *Agent {
 	deviceID := device.Id
 	if deviceID == "" {
 		deviceID = coreutils.CreateDeviceID()
@@ -86,15 +90,16 @@
 
 	return &Agent{
 		deviceID:             deviceID,
-		adapterProxy:         ap,
 		isRootDevice:         device.Root,
 		parentID:             device.ParentId,
 		deviceType:           device.Type,
+		adapterEndpoint:      device.AdapterEndpoint,
 		deviceMgr:            deviceMgr,
 		adapterMgr:           deviceMgr.adapterMgr,
 		exitChannel:          make(chan int, 1),
 		dbProxy:              deviceProxy,
-		defaultTimeout:       timeout,
+		internalTimeout:      internalTimeout,
+		rpcTimeout:           rpcTimeout,
 		device:               proto.Clone(device).(*voltha.Device),
 		requestQueue:         coreutils.NewRequestQueue(),
 		config:               deviceMgr.config,
@@ -132,22 +137,23 @@
 			} else if !have {
 				return nil, status.Errorf(codes.NotFound, "device-%s", agent.deviceID)
 			}
+			logger.Infow(ctx, "device-loaded-from-db", log.Fields{"device-id": agent.deviceID, "adapter-endpoint": device.AdapterEndpoint, "type": device.Type})
 		}
-		agent.deviceType = device.Adapter
+		agent.deviceType = device.Type
+		agent.adapterEndpoint = device.AdapterEndpoint
 		agent.device = proto.Clone(device).(*voltha.Device)
 		// load the ports from KV to cache
 		agent.portLoader.Load(ctx)
 		agent.transientStateLoader.Load(ctx)
-
-		logger.Infow(ctx, "device-loaded-from-db", log.Fields{"device-id": agent.deviceID})
 	} else {
 		// Create a new device
 		var desc string
+		var err error
 		prevState := common.AdminState_UNKNOWN
 		currState := common.AdminState_UNKNOWN
-		operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+		requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
 
-		defer agent.logDeviceUpdate(ctx, "createDevice", &prevState, &currState, operStatus, &desc)
+		defer func() { agent.logDeviceUpdate(ctx, &prevState, &currState, requestStatus, err, desc) }()
 
 		// Assumption is that AdminState, FlowGroups, and Flows are uninitialized since this
 		// is a new device, so populate them here before passing the device to ldProxy.Set.
@@ -162,13 +168,13 @@
 			device.Vlan = deviceToCreate.ProxyAddress.ChannelId
 		}
 
-		// Add the initial device to the local model
-		if err := agent.dbProxy.Set(ctx, agent.deviceID, device); err != nil {
-			desc = fmt.Sprintf("failed-adding-device-%s: %s", agent.deviceID, err.Error())
-			return nil, status.Errorf(codes.Aborted, "failed-adding-device-%s: %s", agent.deviceID, err)
+		// Save the device to the model
+		if err = agent.dbProxy.Set(ctx, agent.deviceID, device); err != nil {
+			err = status.Errorf(codes.Aborted, "failed-adding-device-%s: %s", agent.deviceID, err)
+			return nil, err
 		}
 		_ = agent.deviceMgr.Agent.SendDeviceStateChangeEvent(ctx, device.OperStatus, device.ConnectStatus, prevState, device, time.Now().Unix())
-		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		agent.device = device
 	}
 	startSucceeded = true
@@ -225,8 +231,9 @@
 		return // not found in kv
 	}
 
-	agent.deviceType = device.Adapter
+	agent.deviceType = device.Type
 	agent.device = device
+	agent.adapterEndpoint = device.AdapterEndpoint
 	agent.portLoader.Load(ctx)
 	agent.transientStateLoader.Load(ctx)
 
@@ -234,139 +241,68 @@
 }
 
 // onSuccess is a common callback for scenarios where we receive a nil response following a request to an adapter
-// and the only action required is to publish a successful result on kafka
-func (agent *Agent) onSuccess(ctx context.Context, rpc string, response interface{}, reqArgs ...interface{}) {
-	logger.Debugw(ctx, "response-successful", log.Fields{"rpc": rpc, "device-id": agent.deviceID})
-	// TODO: Post success message onto kafka
+func (agent *Agent) onSuccess(ctx context.Context, prevState, currState *common.AdminState_Types, deviceUpdateLog bool) {
+	if deviceUpdateLog {
+		requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_SUCCESS}
+		desc := "adapter-response"
+		agent.logDeviceUpdate(ctx, prevState, currState, requestStatus, nil, desc)
+		return
+	}
+	logger.Debugw(ctx, "successful-operation", log.Fields{"device-id": agent.deviceID, "rpc": coreutils.GetRPCMetadataFromContext(ctx)})
 }
 
 // onFailure is a common callback for scenarios where we receive an error response following a request to an adapter
 // and the only action required is to publish the failed result on kafka
-func (agent *Agent) onFailure(ctx context.Context, rpc string, response interface{}, reqArgs ...interface{}) {
-	if res, ok := response.(error); ok {
-		logger.Errorw(ctx, "rpc-failed", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": res, "args": reqArgs})
-	} else {
-		logger.Errorw(ctx, "rpc-failed-invalid-error", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
-	}
-	// TODO: Post failure message onto kafka
-}
+func (agent *Agent) onFailure(ctx context.Context, err error, prevState, currState *common.AdminState_Types, deviceUpdateLog bool) {
+	// Send an event on kafka
+	rpce := agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
+	go agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
+		voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
 
-func (agent *Agent) waitForAdapterForceDeleteResponse(ctx context.Context, cancel context.CancelFunc, rpc string, ch chan *kafka.RpcResponse,
-	onSuccess coreutils.ResponseCallback, onFailure coreutils.ResponseCallback, reqArgs ...interface{}) {
-	defer cancel()
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			onFailure(ctx, rpc, status.Errorf(codes.Aborted, "channel-closed"), reqArgs)
-		} else if rpcResponse.Err != nil {
-			onFailure(ctx, rpc, rpcResponse.Err, reqArgs)
-		} else {
-			onSuccess(ctx, rpc, rpcResponse.Reply, reqArgs)
-		}
-	case <-ctx.Done():
-		onFailure(ctx, rpc, ctx.Err(), reqArgs)
+	// Log the device update event
+	if deviceUpdateLog {
+		requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+		desc := "adapter-response"
+		agent.logDeviceUpdate(ctx, prevState, currState, requestStatus, err, desc)
+		return
 	}
+	logger.Errorw(ctx, "failed-operation", log.Fields{"error": err, "device-id": agent.deviceID, "rpc": coreutils.GetRPCMetadataFromContext(ctx)})
 }
 
 // onDeleteSuccess is a common callback for scenarios where we receive a nil response following a delete request
 // to an adapter.
-func (agent *Agent) onDeleteSuccess(ctx context.Context, rpc string, response interface{}, reqArgs ...interface{}) {
-	logger.Debugw(ctx, "response-successful", log.Fields{"rpc": rpc, "device-id": agent.deviceID})
+func (agent *Agent) onDeleteSuccess(ctx context.Context, prevState, currState *common.AdminState_Types) {
 	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-		logger.Errorw(ctx, "delete-device-failure", log.Fields{"device-id": agent.deviceID, "error": err, "args": reqArgs})
+		logger.Errorw(ctx, "delete-device-failure", log.Fields{"device-id": agent.deviceID, "error": err})
 	}
 	previousDeviceTransientState := agent.getTransientState()
 	newDevice := agent.cloneDeviceWithoutLock()
 	if err := agent.updateDeviceWithTransientStateAndReleaseLock(ctx, newDevice,
-		voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE, previousDeviceTransientState); err != nil {
-		logger.Errorw(ctx, "delete-device-failure", log.Fields{"device-id": agent.deviceID, "error": err, "args": reqArgs})
+		core.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE, previousDeviceTransientState); err != nil {
+		logger.Errorw(ctx, "delete-device-failure", log.Fields{"device-id": agent.deviceID, "error": err})
 	}
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_SUCCESS}
+	desc := "adapter-response"
+	agent.logDeviceUpdate(ctx, prevState, currState, requestStatus, nil, desc)
 }
 
 // onDeleteFailure is a common callback for scenarios where we receive an error response following a delete request
 //  to an adapter and the only action required is to return the error response.
-func (agent *Agent) onDeleteFailure(ctx context.Context, rpc string, response interface{}, reqArgs ...interface{}) {
-	if res, ok := response.(error); ok {
-		logger.Errorw(ctx, "rpc-failed", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": res, "args": reqArgs})
-	} else {
-		logger.Errorw(ctx, "rpc-failed-invalid-error", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
-	}
+func (agent *Agent) onDeleteFailure(ctx context.Context, err error, prevState, currState *common.AdminState_Types) {
+	logger.Errorw(ctx, "rpc-failed", log.Fields{"rpc": coreutils.GetRPCMetadataFromContext(ctx), "device-id": agent.deviceID, "error": err})
+
 	//Only updating of transient state is required, no transition.
-	if err := agent.updateTransientState(ctx, voltha.DeviceTransientState_DELETE_FAILED); err != nil {
-		logger.Errorw(ctx, "failed-to-update-transient-state-as-delete-failed", log.Fields{"device-id": agent.deviceID})
+	if er := agent.updateTransientState(ctx, core.DeviceTransientState_DELETE_FAILED); er != nil {
+		logger.Errorw(ctx, "failed-to-update-transient-state-as-delete-failed", log.Fields{"device-id": agent.deviceID, "error": er})
 	}
+	rpce := agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
+	go agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
+		voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
 
-}
-
-func (agent *Agent) waitForAdapterResponse(ctx context.Context, cancel context.CancelFunc, rpc string, ch chan *kafka.RpcResponse,
-	onSuccess coreutils.ResponseCallback, onFailure coreutils.ResponseCallback, reqArgs ...interface{}) {
-	defer cancel()
-	var rpce *voltha.RPCEvent
-	defer func() {
-		if rpce != nil {
-			agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
-				voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
-		}
-	}()
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, "Response Channel Closed", nil)
-			onFailure(ctx, rpc, status.Errorf(codes.Aborted, "channel-closed"), reqArgs)
-			//add failure
-		} else if rpcResponse.Err != nil {
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, rpcResponse.Err.Error(), nil)
-			onFailure(ctx, rpc, rpcResponse.Err, reqArgs)
-			//add failure
-		} else {
-			onSuccess(ctx, rpc, rpcResponse.Reply, reqArgs)
-		}
-	case <-ctx.Done():
-		rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, ctx.Err().Error(), nil)
-		onFailure(ctx, rpc, ctx.Err(), reqArgs)
-	}
-}
-
-func (agent *Agent) waitForAdapterResponseAndLogDeviceUpdate(ctx context.Context, cancel context.CancelFunc, rpc string, ch chan *kafka.RpcResponse,
-	onSuccess coreutils.ResponseCallback, onFailure coreutils.ResponseCallback, prevState *common.AdminState_Types, reqArgs ...interface{}) {
-	defer cancel()
-	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-	defer func() {
-		currAdminState := prevState
-		if d, _ := agent.getDeviceReadOnly(ctx); d != nil {
-			currAdminState = &d.AdminState
-		}
-		agent.logDeviceUpdate(ctx, rpc, prevState, currAdminState, operStatus, &desc)
-	}()
-	var rpce *voltha.RPCEvent
-	defer func() {
-		if rpce != nil {
-			agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
-				voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
-		}
-	}()
-
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, "Response Channel Closed", nil)
-			onFailure(ctx, rpc, status.Errorf(codes.Aborted, "channel-closed"), reqArgs)
-			//add failure
-		} else if rpcResponse.Err != nil {
-			desc = rpcResponse.Err.Error()
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, desc, nil)
-			onFailure(ctx, rpc, rpcResponse.Err, reqArgs)
-			//add failure
-		} else {
-			operStatus.Code = common.OperationResp_OPERATION_SUCCESS
-			onSuccess(ctx, rpc, rpcResponse.Reply, reqArgs)
-		}
-	case <-ctx.Done():
-		desc = ctx.Err().Error()
-		rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, desc, nil)
-		onFailure(ctx, rpc, ctx.Err(), reqArgs)
-	}
+	// Log the device update event
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	desc := "adapter-response"
+	agent.logDeviceUpdate(ctx, prevState, currState, requestStatus, err, desc)
 }
 
 // getDeviceReadOnly returns a device which MUST NOT be modified, but is safe to keep forever.
@@ -390,117 +326,128 @@
 	return proto.Clone(agent.device).(*voltha.Device)
 }
 
+func (agent *Agent) updateDeviceTypeAndEndpoint(ctx context.Context) error {
+	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		return err
+	}
+	changed := false
+	cloned := agent.cloneDeviceWithoutLock()
+	if cloned.Type == "" {
+		adapterType, err := agent.adapterMgr.GetAdapterType(cloned.Type)
+		if err != nil {
+			agent.requestQueue.RequestComplete()
+			return err
+		}
+		cloned.Type = adapterType
+		changed = true
+	}
+
+	if cloned.AdapterEndpoint == "" {
+		var err error
+		if cloned.AdapterEndpoint, err = agent.adapterMgr.GetAdapterEndpoint(ctx, cloned.Id, cloned.Type); err != nil {
+			agent.requestQueue.RequestComplete()
+			return err
+		}
+		agent.adapterEndpoint = cloned.AdapterEndpoint
+		changed = true
+	}
+
+	if changed {
+		return agent.updateDeviceAndReleaseLock(ctx, cloned)
+	}
+	agent.requestQueue.RequestComplete()
+	return nil
+}
+
 // enableDevice activates a preprovisioned or a disable device
 func (agent *Agent) enableDevice(ctx context.Context) error {
 	//To preserve and use oldDevice state as prev state in new device
+	var err error
 	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	var prevAdminState, currAdminState common.AdminState_Types
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
 
-	defer agent.logDeviceUpdate(ctx, "enableDevice", nil, nil, operStatus, &desc)
+	defer func() { agent.logDeviceUpdate(ctx, &prevAdminState, &currAdminState, requestStatus, err, desc) }()
 
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
 	logger.Debugw(ctx, "enable-device", log.Fields{"device-id": agent.deviceID})
 
-	prevDeviceState := agent.device.AdminState
-
 	oldDevice := agent.getDeviceReadOnlyWithoutLock()
+	prevAdminState = oldDevice.AdminState
 
 	if !agent.proceedWithRequest(oldDevice) {
 		agent.requestQueue.RequestComplete()
-
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconcile is in progress/failed.", agent.deviceID)
-		return status.Error(codes.FailedPrecondition, desc)
+		err = status.Errorf(codes.FailedPrecondition, "cannot complete operation as device deletion is in progress or reconciling is in progress/failed: %s", agent.deviceID)
+		return err
 	}
 
 	if oldDevice.AdminState == voltha.AdminState_ENABLED {
 		logger.Warnw(ctx, "device-already-enabled", log.Fields{"device-id": agent.deviceID})
 		agent.requestQueue.RequestComplete()
-		desc = fmt.Sprintf("cannot-enable-an-already-enabled-device: %s", oldDevice.Id)
-		return status.Error(codes.FailedPrecondition, desc)
-	}
-
-	// First figure out which adapter will handle this device type.  We do it at this stage as allow devices to be
-	// pre-provisioned with the required adapter not registered.   At this stage, since we need to communicate
-	// with the adapter then we need to know the adapter that will handle this request
-	adapterName, err := agent.adapterMgr.GetAdapterType(oldDevice.Type)
-	if err != nil {
-		agent.requestQueue.RequestComplete()
-		desc = err.Error()
+		err = status.Errorf(codes.FailedPrecondition, fmt.Sprintf("cannot-enable-an-already-enabled-device: %s", oldDevice.Id))
 		return err
 	}
 
+	// Verify whether there is a device type that supports this device type
+	_, err = agent.adapterMgr.GetAdapterType(oldDevice.Type)
+	if err != nil {
+		agent.requestQueue.RequestComplete()
+		return err
+	}
+
+	// Update device adapter endpoint if not set.  This is set once by the Core and use as is by the adapters.  E.g if this is a
+	// child device then the parent adapter will use this device's adapter endpoint (set here) to communicate with it.
 	newDevice := agent.cloneDeviceWithoutLock()
-	newDevice.Adapter = adapterName
+	if newDevice.AdapterEndpoint == "" {
+		if newDevice.AdapterEndpoint, err = agent.adapterMgr.GetAdapterEndpoint(ctx, newDevice.Id, newDevice.Type); err != nil {
+			agent.requestQueue.RequestComplete()
+			return err
+		}
+		agent.adapterEndpoint = newDevice.AdapterEndpoint
+	}
 
 	// Update the Admin State and set the operational state to activating before sending the request to the Adapters
 	newDevice.AdminState = voltha.AdminState_ENABLED
 	newDevice.OperStatus = voltha.OperStatus_ACTIVATING
 
-	if err := agent.updateDeviceAndReleaseLock(ctx, newDevice); err != nil {
-		desc = err.Error()
-		return err
-	}
-
 	// Adopt the device if it was in pre-provision state.  In all other cases, try to re-enable it.
-	var ch chan *kafka.RpcResponse
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-	subCtx = coreutils.WithFromTopicMetadataFromContext(subCtx, ctx)
-
-	if oldDevice.AdminState == voltha.AdminState_PREPROVISIONED {
-		ch, err = agent.adapterProxy.AdoptDevice(subCtx, newDevice)
-	} else {
-		ch, err = agent.adapterProxy.ReEnableDevice(subCtx, newDevice)
-	}
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		cancel()
-		desc = err.Error()
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": newDevice.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return err
 	}
-
-	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-
-	// Wait for response
-	go agent.waitForAdapterResponseAndLogDeviceUpdate(subCtx, cancel, "enableDevice", ch, agent.onSuccess, agent.onFailure, &prevDeviceState)
-	return nil
-}
-
-func (agent *Agent) waitForAdapterFlowResponse(ctx context.Context, cancel context.CancelFunc, rpc string, ch chan *kafka.RpcResponse, response coreutils.Response) {
-	defer cancel()
-	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-	defer agent.logDeviceUpdate(ctx, rpc, nil, nil, operStatus, &desc)
-
-	var rpce *voltha.RPCEvent
-	defer func() {
-		if rpce != nil {
-			agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
-				voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+	requestStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		var err error
+		if oldDevice.AdminState == voltha.AdminState_PREPROVISIONED {
+			_, err = client.AdoptDevice(subCtx, newDevice)
+		} else {
+			_, err = client.ReEnableDevice(subCtx, newDevice)
+		}
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, true)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, true)
 		}
 	}()
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			//add failure
-			desc = "Response Channel Closed"
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, "Response Channel Closed", nil)
-			response.Error(status.Errorf(codes.Aborted, "channel-closed"))
-		} else if rpcResponse.Err != nil {
-			//add failure
-			desc = rpcResponse.Err.Error()
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, desc, nil)
-			response.Error(rpcResponse.Err)
-		} else {
-			operStatus.Code = common.OperationResp_OPERATION_SUCCESS
-			response.Done()
-		}
-	case <-ctx.Done():
-		desc = ctx.Err().Error()
-		rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, desc, nil)
-		response.Error(ctx.Err())
+
+	// Update device
+	if err = agent.updateDeviceAndReleaseLock(ctx, newDevice); err != nil {
+		return err
 	}
+	currAdminState = newDevice.AdminState
+	return nil
 }
 
 //addFlowsAndGroups adds the "newFlows" and "newGroups" from the existing flows/groups and sends the update to the
@@ -516,8 +463,8 @@
 	if grpResponse, err = agent.addGroupsToAdapter(ctx, newGroups, flowMetadata); err != nil {
 		return err
 	}
-	if errs := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, flwResponse, grpResponse); errs != nil {
-		logger.Warnw(ctx, "no-adapter-response", log.Fields{"device-id": agent.deviceID, "result": errs})
+	if errs := coreutils.WaitForNilOrErrorResponses(agent.rpcTimeout, flwResponse, grpResponse); errs != nil {
+		logger.Warnw(ctx, "adapter-response", log.Fields{"device-id": agent.deviceID, "result": errs})
 		return status.Errorf(codes.Aborted, "flow-failure-device-%s", agent.deviceID)
 	}
 	return nil
@@ -535,7 +482,7 @@
 		return err
 	}
 
-	if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, flwResponse, grpResponse); res != nil {
+	if res := coreutils.WaitForNilOrErrorResponses(agent.rpcTimeout, flwResponse, grpResponse); res != nil {
 		return status.Errorf(codes.Aborted, "errors-%s", res)
 	}
 	return nil
@@ -553,7 +500,7 @@
 		return err
 	}
 
-	if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, flwResponse, grpResponse); res != nil {
+	if res := coreutils.WaitForNilOrErrorResponses(agent.rpcTimeout, flwResponse, grpResponse); res != nil {
 		return status.Errorf(codes.Aborted, "errors-%s", res)
 	}
 	return nil
@@ -561,73 +508,81 @@
 
 //disableDevice disable a device
 func (agent *Agent) disableDevice(ctx context.Context) error {
+	var err error
 	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	var prevAdminState, currAdminState common.AdminState_Types
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, &prevAdminState, &currAdminState, requestStatus, err, desc) }()
 
-	prevDeviceState := agent.device.AdminState
-
-	defer agent.logDeviceUpdate(ctx, "disableDevice", nil, nil, operStatus, &desc)
-
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-		desc = err.Error()
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
 	logger.Debugw(ctx, "disable-device", log.Fields{"device-id": agent.deviceID})
 
 	cloned := agent.cloneDeviceWithoutLock()
+	prevAdminState = agent.device.AdminState
 
 	if !agent.proceedWithRequest(cloned) {
+		err = status.Errorf(codes.FailedPrecondition, "cannot complete operation as device deletion is in progress or reconciling is in progress/failed: %s", agent.deviceID)
 		agent.requestQueue.RequestComplete()
-		desc = fmt.Sprintf("deviceId:%s,Cannot complete operation as device deletion is in progress or reconciling is in progress/failed.", agent.deviceID)
-		return status.Errorf(codes.FailedPrecondition, desc)
+		return err
 	}
 
 	if cloned.AdminState == voltha.AdminState_DISABLED {
 		desc = "device-already-disabled"
-		logger.Debugw(ctx, "device-already-disabled", log.Fields{"device-id": agent.deviceID})
 		agent.requestQueue.RequestComplete()
 		return nil
 	}
 	if cloned.AdminState == voltha.AdminState_PREPROVISIONED {
 		agent.requestQueue.RequestComplete()
-		desc = fmt.Sprintf("deviceId:%s, invalid-admin-state:%s", agent.deviceID, cloned.AdminState)
-		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, invalid-admin-state:%s", agent.deviceID, cloned.AdminState)
+		err = status.Errorf(codes.FailedPrecondition, "deviceId:%s, invalid-admin-state:%s", agent.deviceID, cloned.AdminState)
+		return err
 	}
 
 	// Update the Admin State and operational state before sending the request out
 	cloned.AdminState = voltha.AdminState_DISABLED
 	cloned.OperStatus = voltha.OperStatus_UNKNOWN
 
-	if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
-		return err
-	}
-
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.DisableDevice(subCtx, cloned)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		cancel()
-		desc = err.Error()
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return err
 	}
-	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+	requestStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		_, err := client.DisableDevice(subCtx, cloned)
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, true)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, true)
+		}
+	}()
 
-	// Wait for response
-	go agent.waitForAdapterResponseAndLogDeviceUpdate(subCtx, cancel, "disableDevice", ch, agent.onSuccess, agent.onFailure, &prevDeviceState)
+	// Update device
+	if err = agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
+		return err
+	}
+	currAdminState = cloned.AdminState
 
 	return nil
 }
 
 func (agent *Agent) rebootDevice(ctx context.Context) error {
 	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	var err error
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
 
-	prevDeviceState := agent.device.AdminState
-
-	defer agent.logDeviceUpdate(ctx, "rebootDevice", nil, nil, operStatus, &desc)
-
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		desc = err.Error()
 		return err
 	}
@@ -637,22 +592,32 @@
 	device := agent.getDeviceReadOnlyWithoutLock()
 
 	if !agent.proceedWithRequest(device) {
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed.", agent.deviceID)
-		return status.Errorf(codes.FailedPrecondition, desc)
-	}
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.RebootDevice(subCtx, device)
-	if err != nil {
-		cancel()
-		desc = err.Error()
+		err = status.Errorf(codes.FailedPrecondition, "cannot complete operation as device deletion is in progress or reconciling is in progress/failed:%s", agent.deviceID)
 		return err
 	}
-	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
 
-	// Wait for response
-	go agent.waitForAdapterResponseAndLogDeviceUpdate(subCtx, cancel, "rebootDevice", ch, agent.onSuccess, agent.onFailure, &prevDeviceState)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+	requestStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		_, err := client.RebootDevice(subCtx, device)
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, true)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, true)
+		}
+	}()
 	return nil
 }
 
@@ -660,50 +625,54 @@
 	logger.Debugw(ctx, "delete-device-force", log.Fields{"device-id": agent.deviceID})
 
 	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	var err error
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
 
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-		desc = err.Error()
-		agent.logDeviceUpdate(ctx, "deleteDeviceForce", nil, nil, operStatus, &desc)
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
 	// Get the device Transient state, return err if it is DELETING
 	previousDeviceTransientState := agent.getTransientState()
-
-	if agent.isStateDeleting(previousDeviceTransientState) {
-		agent.requestQueue.RequestComplete()
-		desc = fmt.Sprintf("deviceId:%s, Device Deletion is in progress",
-			agent.deviceID)
-		agent.logDeviceUpdate(ctx, "deleteDeviceForce", nil, nil, operStatus, &desc)
-		return status.Error(codes.FailedPrecondition, desc)
-	}
-
-	//Send stop Reconcile if in progress
-	agent.stopReconcile()
-
 	device := agent.cloneDeviceWithoutLock()
-	if err := agent.updateDeviceWithTransientStateAndReleaseLock(ctx, device,
-		voltha.DeviceTransientState_FORCE_DELETING, previousDeviceTransientState); err != nil {
+	if !agent.isForceDeletingAllowed(previousDeviceTransientState, device) {
+		agent.requestQueue.RequestComplete()
+		err = status.Error(codes.FailedPrecondition, fmt.Sprintf("deviceId:%s, force deletion is in progress", agent.deviceID))
 		return err
 	}
-	previousAdminState := device.AdminState
-	if previousAdminState != ic.AdminState_PREPROVISIONED {
-		subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-		subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 
-		ch, err := agent.adapterProxy.DeleteDevice(subCtx, device)
+	previousAdminState := device.AdminState
+	if previousAdminState != common.AdminState_PREPROVISIONED {
+		var client adapter_services.AdapterServiceClient
+		client, err = agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "deleteDeviceForce", nil, nil, operStatus, &desc)
+			logger.Errorw(ctx, "grpc-client-nil",
+				log.Fields{
+					"error":            err,
+					"device-id":        agent.deviceID,
+					"device-type":      agent.deviceType,
+					"adapter-endpoint": device.AdapterEndpoint,
+				})
+			agent.requestQueue.RequestComplete()
 			return err
 		}
-		// As force delete will not be dependent over the response of adapter, marking this operation as success
-		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
-		agent.logDeviceUpdate(ctx, "deleteDeviceForce", nil, nil, operStatus, &desc)
-		// Since it is a case of force delete, nothing needs to be done on adapter responses.
-		go agent.waitForAdapterForceDeleteResponse(subCtx, cancel, "deleteDeviceForce", ch, agent.onSuccess,
-			agent.onFailure)
+		subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+		requestStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+		go func() {
+			defer cancel()
+			_, err := client.DeleteDevice(subCtx, device)
+			if err == nil {
+				agent.onSuccess(subCtx, nil, nil, true)
+			} else {
+				agent.onFailure(subCtx, err, nil, nil, true)
+			}
+		}()
+	}
+
+	// Update device
+	if err = agent.updateDeviceWithTransientStateAndReleaseLock(ctx, device,
+		core.DeviceTransientState_FORCE_DELETING, previousDeviceTransientState); err != nil {
+		return err
 	}
 	return nil
 }
@@ -712,12 +681,11 @@
 	logger.Debugw(ctx, "delete-device", log.Fields{"device-id": agent.deviceID})
 
 	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-	prevState := agent.device.AdminState
+	var err error
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
 
-	defer agent.logDeviceUpdate(ctx, "deleteDevice", nil, nil, operStatus, &desc)
-
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		desc = err.Error()
 		return err
 	}
@@ -726,8 +694,8 @@
 
 	if !agent.proceedWithRequest(device) {
 		agent.requestQueue.RequestComplete()
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", agent.deviceID)
-		return status.Error(codes.FailedPrecondition, desc)
+		err = status.Errorf(codes.FailedPrecondition, "cannot complete operation as device deletion is in progress or reconciling is in progress/failed: %s", agent.deviceID)
+		return err
 	}
 
 	// Get the device Transient state, return err if it is DELETING
@@ -735,38 +703,48 @@
 
 	previousAdminState := device.AdminState
 	// Change the device transient state to DELETING_FROM_ADAPTER  state till the device is removed from adapters.
-	currentDeviceTransientState := voltha.DeviceTransientState_DELETING_FROM_ADAPTER
+	currentDeviceTransientState := core.DeviceTransientState_DELETING_FROM_ADAPTER
 
-	if previousAdminState == ic.AdminState_PREPROVISIONED {
+	if previousAdminState == common.AdminState_PREPROVISIONED {
 		// Change the state to DELETING POST ADAPTER RESPONSE directly as adapters have no info of the device.
-		currentDeviceTransientState = voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE
+		currentDeviceTransientState = core.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE
 	}
-	if err := agent.updateDeviceWithTransientStateAndReleaseLock(ctx, device,
+	// If the device was in pre-prov state (only parent device are in that state) then do not send the request to the
+	// adapter
+	if previousAdminState != common.AdminState_PREPROVISIONED {
+		var client adapter_services.AdapterServiceClient
+		client, err = agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+		if err != nil {
+			logger.Errorw(ctx, "grpc-client-nil",
+				log.Fields{
+					"error":            err,
+					"device-id":        agent.deviceID,
+					"device-type":      agent.deviceType,
+					"adapter-endpoint": device.AdapterEndpoint,
+				})
+			agent.requestQueue.RequestComplete()
+			return err
+		}
+		subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+		requestStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+		go func() {
+			defer cancel()
+			_, err := client.DeleteDevice(subCtx, device)
+			if err == nil {
+				agent.onDeleteSuccess(subCtx, nil, nil)
+			} else {
+				agent.onDeleteFailure(subCtx, err, nil, nil)
+			}
+		}()
+	}
+
+	// Update device and release lock
+	if err = agent.updateDeviceWithTransientStateAndReleaseLock(ctx, device,
 		currentDeviceTransientState, previousDeviceTransientState); err != nil {
 		desc = err.Error()
 		return err
 	}
-	// If the device was in pre-prov state (only parent device are in that state) then do not send the request to the
-	// adapter
-	if previousAdminState != ic.AdminState_PREPROVISIONED {
-		subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-		subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 
-		ch, err := agent.adapterProxy.DeleteDevice(subCtx, device)
-		if err != nil {
-			cancel()
-			//updating of transient state is required in error
-			if err := agent.updateTransientState(ctx, voltha.DeviceTransientState_DELETE_FAILED); err != nil {
-				logger.Errorw(ctx, "failed-to-update-transient-state-as-delete-failed", log.Fields{"device-id": agent.deviceID})
-			}
-			desc = err.Error()
-			return err
-		}
-
-		operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-		go agent.waitForAdapterResponseAndLogDeviceUpdate(subCtx, cancel, "deleteDevice", ch, agent.onDeleteSuccess,
-			agent.onDeleteFailure, &prevState)
-	}
 	return nil
 }
 
@@ -789,67 +767,66 @@
 	if err != nil {
 		return nil, err
 	}
-	ch, err := agent.adapterProxy.GetOfpDeviceInfo(ctx, device)
+
+	// Get the gRPC client
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
 		return nil, err
 	}
 
-	// Wait for adapter response
-	rpcResponse, ok := <-ch
-	if !ok {
-		return nil, status.Errorf(codes.Aborted, "channel-closed")
-	}
-	if rpcResponse.Err != nil {
-		return nil, rpcResponse.Err
-	}
-	// Successful response
-	switchCap := &ic.SwitchCapability{}
-	if err := ptypes.UnmarshalAny(rpcResponse.Reply, switchCap); err != nil {
-		return nil, err
-	}
-	return switchCap, nil
+	return client.GetOfpDeviceInfo(ctx, device)
 }
 
-func (agent *Agent) onPacketFailure(ctx context.Context, rpc string, response interface{}, args ...interface{}) {
-	// packet data is encoded in the args param as the first parameter
-	var packet []byte
-	if len(args) >= 1 {
-		if pkt, ok := args[0].([]byte); ok {
-			packet = pkt
-		}
-	}
-	var errResp error
-	if err, ok := response.(error); ok {
-		errResp = err
-	}
-	logger.Warnw(ctx, "packet-out-error", log.Fields{
+func (agent *Agent) onPacketFailure(ctx context.Context, err error, packet *ofp.OfpPacketOut) {
+	logger.Errorw(ctx, "packet-out-error", log.Fields{
 		"device-id": agent.deviceID,
-		"error":     errResp,
-		"packet":    hex.EncodeToString(packet),
+		"error":     err.Error(),
+		"packet":    hex.EncodeToString(packet.Data),
 	})
+	rpce := agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
+	go agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
+		voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
 }
 
 func (agent *Agent) packetOut(ctx context.Context, outPort uint32, packet *ofp.OfpPacketOut) error {
-	// If deviceType=="" then we must have taken ownership of this device.
-	// Fixes VOL-2226 where a core would take ownership and have stale data
 	if agent.deviceType == "" {
 		agent.reconcileWithKVStore(ctx)
 	}
 	//	Send packet to adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.PacketOut(subCtx, agent.deviceType, agent.deviceID, outPort, packet)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		cancel()
-		return nil
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":       err,
+				"device-id":   agent.deviceID,
+				"device-type": agent.deviceType,
+			})
+		return err
 	}
-	go agent.waitForAdapterResponse(subCtx, cancel, "packetOut", ch, agent.onSuccess, agent.onPacketFailure, packet.Data)
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+	go func() {
+		defer cancel()
+		_, err := client.SendPacketOut(subCtx, &ic.PacketOut{
+			DeviceId:     agent.deviceID,
+			EgressPortNo: outPort,
+			Packet:       packet,
+		})
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, false)
+		} else {
+			agent.onPacketFailure(subCtx, err, packet)
+		}
+	}()
 	return nil
 }
 
 func (agent *Agent) updateDeviceUsingAdapterData(ctx context.Context, device *voltha.Device) error {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
 	logger.Debugw(ctx, "update-device-using-adapter-data", log.Fields{"device-id": device.Id})
@@ -863,27 +840,40 @@
 	cloned.Vlan = device.Vlan
 	cloned.Reason = device.Reason
 	cloned.ImageDownloads = device.ImageDownloads
-	return agent.updateDeviceAndReleaseLock(ctx, cloned)
+	cloned.OperStatus = device.OperStatus
+	cloned.ConnectStatus = device.ConnectStatus
+	if err = agent.updateDeviceAndReleaseLock(ctx, cloned); err == nil {
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
+	}
+	return err
 }
 
 func (agent *Agent) updateDeviceStatus(ctx context.Context, operStatus voltha.OperStatus_Types, connStatus voltha.ConnectStatus_Types) error {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	opStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, opStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
 
 	cloned := agent.cloneDeviceWithoutLock()
 	// Ensure the enums passed in are valid - they will be invalid if they are not set when this function is invoked
 	if s, ok := voltha.ConnectStatus_Types_name[int32(connStatus)]; ok {
-		logger.Debugw(ctx, "update-device-status-conn", log.Fields{"ok": ok, "val": s})
+		logger.Debugw(ctx, "update-device-conn-status", log.Fields{"ok": ok, "val": s})
 		cloned.ConnectStatus = connStatus
 	}
 	if s, ok := voltha.OperStatus_Types_name[int32(operStatus)]; ok {
-		logger.Debugw(ctx, "update-device-status-conn", log.Fields{"ok": ok, "val": s})
+		logger.Debugw(ctx, "update-device-oper-status", log.Fields{"ok": ok, "val": s})
 		cloned.OperStatus = operStatus
 	}
 	logger.Debugw(ctx, "update-device-status", log.Fields{"device-id": cloned.Id, "oper-status": cloned.OperStatus, "connect-status": cloned.ConnectStatus})
 	// Store the device
-	return agent.updateDeviceAndReleaseLock(ctx, cloned)
+	if err = agent.updateDeviceAndReleaseLock(ctx, cloned); err == nil {
+		opStatus.Code = common.OperationResp_OPERATION_SUCCESS
+	}
+	return err
 }
 
 // TODO: A generic device update by attribute
@@ -926,7 +916,12 @@
 }
 
 func (agent *Agent) simulateAlarm(ctx context.Context, simulateReq *voltha.SimulateAlarmRequest) error {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
 	defer agent.requestQueue.RequestComplete()
@@ -934,15 +929,28 @@
 
 	device := agent.getDeviceReadOnlyWithoutLock()
 
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.SimulateAlarm(subCtx, device, simulateReq)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		cancel()
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
 		return err
 	}
-	go agent.waitForAdapterResponse(subCtx, cancel, "simulateAlarm", ch, agent.onSuccess, agent.onFailure)
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+	requestStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		_, err := client.SimulateAlarm(subCtx, &ic.SimulateAlarmMessage{Device: device, Request: simulateReq})
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, false)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, false)
+		}
+	}()
 	return nil
 }
 
@@ -990,7 +998,7 @@
 // This function updates the device transient in the DB through loader, releases the device lock, and runs any state transitions.
 // The calling function MUST hold the device lock.  The caller MUST NOT modify the device after this is called.
 func (agent *Agent) updateDeviceWithTransientStateAndReleaseLock(ctx context.Context, device *voltha.Device,
-	transientState, prevTransientState voltha.DeviceTransientState_Types) error {
+	transientState, prevTransientState core.DeviceTransientState_Types) error {
 	// fail early if this agent is no longer valid
 	if agent.stopped {
 		agent.requestQueue.RequestComplete()
@@ -1004,9 +1012,10 @@
 	// update in db
 	if err := agent.dbProxy.Set(ctx, agent.deviceID, device); err != nil {
 		//Reverting TransientState update
-		err := agent.updateTransientState(ctx, prevTransientState)
-		logger.Errorw(ctx, "failed-to-revert-transient-state-update-on-error", log.Fields{"device-id": device.Id,
-			"previous-transient-state": prevTransientState, "current-transient-state": transientState})
+		if errTransient := agent.updateTransientState(ctx, prevTransientState); errTransient != nil {
+			logger.Errorw(ctx, "failed-to-revert-transient-state-update-on-error", log.Fields{"device-id": device.Id,
+				"previous-transient-state": prevTransientState, "current-transient-state": transientState, "error": errTransient})
+		}
 		agent.requestQueue.RequestComplete()
 		return status.Errorf(codes.Internal, "failed-update-device:%s: %s", agent.deviceID, err)
 	}
@@ -1023,44 +1032,47 @@
 
 	// release lock before processing transition
 	agent.requestQueue.RequestComplete()
-	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
-	if err := agent.deviceMgr.stateTransitions.ProcessTransition(subCtx,
-		device, prevDevice, transientState, prevTransientState); err != nil {
-		logger.Errorw(ctx, "failed-process-transition", log.Fields{"device-id": device.Id, "previous-admin-state": prevDevice.AdminState, "current-admin-state": device.AdminState})
-		// Sending RPC EVENT here
-		rpce := agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
-		agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce, voltha.EventCategory_COMMUNICATION,
-			nil, time.Now().Unix())
-	}
+	go func() {
+		subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
+		if err := agent.deviceMgr.stateTransitions.ProcessTransition(subCtx,
+			device, prevDevice, transientState, prevTransientState); err != nil {
+			logger.Errorw(ctx, "failed-process-transition", log.Fields{"device-id": device.Id, "previous-admin-state": prevDevice.AdminState, "current-admin-state": device.AdminState})
+			// Sending RPC EVENT here
+			rpce := agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
+			agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce, voltha.EventCategory_COMMUNICATION,
+				nil, time.Now().Unix())
+		}
+	}()
 	return nil
 }
 func (agent *Agent) updateDeviceReason(ctx context.Context, reason string) error {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	logger.Debugw(ctx, "update-device-reason", log.Fields{"device-id": agent.deviceID, "reason": reason})
+
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
 
-	logger.Debugw(ctx, "update-device-reason", log.Fields{"device-id": agent.deviceID, "reason": reason})
-
-	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-
-	defer agent.logDeviceUpdate(ctx, "updateDeviceReason", nil, nil, operStatus, &desc)
-
 	cloned := agent.cloneDeviceWithoutLock()
 	cloned.Reason = reason
-	retErr := agent.updateDeviceAndReleaseLock(ctx, cloned)
-	if retErr != nil {
-		desc = retErr.Error()
-	} else {
-		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
-		desc = reason
+	if err = agent.updateDeviceAndReleaseLock(ctx, cloned); err == nil {
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
 	}
-	return retErr
+	return err
 }
 
 func (agent *Agent) ChildDeviceLost(ctx context.Context, device *voltha.Device) error {
 	logger.Debugw(ctx, "child-device-lost", log.Fields{"child-device-id": device.Id, "parent-device-id": agent.deviceID})
 
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
 	// Remove the associated peer ports on the parent device
 	for portID := range agent.portLoader.ListIDs() {
 		if portHandle, have := agent.portLoader.Lock(portID); have {
@@ -1082,154 +1094,199 @@
 	}
 
 	//send request to adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.ChildDeviceLost(ctx, agent.deviceType, device)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		cancel()
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
 		return err
 	}
-	go agent.waitForAdapterResponse(subCtx, cancel, "childDeviceLost", ch, agent.onSuccess, agent.onFailure)
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
+	requestStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		_, err := client.ChildDeviceLost(subCtx, device)
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, true)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, true)
+		}
+	}()
 	return nil
 }
 
 func (agent *Agent) startOmciTest(ctx context.Context, omcitestrequest *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-		return nil, err
-	}
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
 
-	cloned := agent.cloneDeviceWithoutLock()
-
-	if cloned.Adapter == "" {
-		adapterName, err := agent.adapterMgr.GetAdapterType(cloned.Type)
-		if err != nil {
-			agent.requestQueue.RequestComplete()
-			return nil, err
-		}
-		cloned.Adapter = adapterName
-	}
-
-	// Send request to the adapter
-	ch, err := agent.adapterProxy.StartOmciTest(ctx, cloned, omcitestrequest)
-	agent.requestQueue.RequestComplete()
+	// OMCI test may be performed on a pre-provisioned device.  If a device is in that state both its device type and endpoint
+	// may not have been set yet.
+	// First check if we need to update the type or endpoint
+	cloned, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
 		return nil, err
 	}
-
-	// Wait for the adapter response
-	rpcResponse, ok := <-ch
-	if !ok {
-		return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
-	}
-	if rpcResponse.Err != nil {
-		return nil, rpcResponse.Err
+	if cloned.Type == "" || cloned.AdapterEndpoint == "" {
+		if err = agent.updateDeviceTypeAndEndpoint(ctx); err != nil {
+			return nil, err
+		}
+		cloned, err = agent.getDeviceReadOnly(ctx)
+		if err != nil {
+			return nil, err
+		}
 	}
 
-	// Unmarshal and return the response
-	testResp := &voltha.TestResponse{}
-	if err := ptypes.UnmarshalAny(rpcResponse.Reply, testResp); err != nil {
-		return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
+	// Send request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		return nil, err
 	}
-	logger.Debugw(ctx, "omci_test_request-success-device-agent", log.Fields{"test-resp": testResp})
-	return testResp, nil
+
+	res, err := client.StartOmciTest(ctx, &ic.OMCITest{
+		Device:  cloned,
+		Request: omcitestrequest,
+	})
+	if err == nil {
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
+	}
+	return res, err
 }
 
 func (agent *Agent) getExtValue(ctx context.Context, pdevice *voltha.Device, cdevice *voltha.Device, valueparam *voltha.ValueSpecifier) (*voltha.ReturnValues, error) {
 	logger.Debugw(ctx, "get-ext-value", log.Fields{"device-id": agent.deviceID, "onu-id": valueparam.Id, "value-type": valueparam.Value})
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
 
-	//send request to adapter
-	ch, err := agent.adapterProxy.GetExtValue(ctx, pdevice, cdevice, valueparam.Id, valueparam.Value)
-	agent.requestQueue.RequestComplete()
+	//send request to adapter synchronously
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, pdevice.AdapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": pdevice.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
 
-	// Wait for the adapter response
-	rpcResponse, ok := <-ch
-	if !ok {
-		return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
-	}
-	if rpcResponse.Err != nil {
-		return nil, rpcResponse.Err
-	}
+	// Release lock before sending to adapter
+	agent.requestQueue.RequestComplete()
 
-	// Unmarshal and return the response
-	Resp := &voltha.ReturnValues{}
-	if err := ptypes.UnmarshalAny(rpcResponse.Reply, Resp); err != nil {
-		return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
+	retVal, err := client.GetExtValue(ctx, &ic.GetExtValueMessage{
+		ParentDevice: pdevice,
+		ChildDevice:  cdevice,
+		ValueType:    valueparam.Value,
+	})
+	if err == nil {
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
 	}
-	logger.Debugw(ctx, "get-ext-value-success-device-agent", log.Fields{"Resp": Resp})
-	return Resp, nil
+	return retVal, err
 }
 
 func (agent *Agent) setExtValue(ctx context.Context, device *voltha.Device, value *voltha.ValueSet) (*empty.Empty, error) {
 	logger.Debugw(ctx, "set-ext-value", log.Fields{"device-id": value.Id})
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
 
 	//send request to adapter
-	ch, err := agent.adapterProxy.SetExtValue(ctx, device, value)
-	agent.requestQueue.RequestComplete()
+	//send request to adapter synchronously
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
+	// Release lock before sending request to adapter
+	agent.requestQueue.RequestComplete()
 
-	// Wait for the adapter response
-	rpcResponse, ok := <-ch
-	if !ok {
-		return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
+	retVal, err := client.SetExtValue(ctx, &ic.SetExtValueMessage{
+		Device: device,
+		Value:  value,
+	})
+	if err == nil {
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
 	}
-	if rpcResponse.Err != nil {
-		return nil, rpcResponse.Err
-	}
-
-	// Unmarshal and return the response
-	logger.Debug(ctx, "set-ext-value-success-device-agent")
-	return &empty.Empty{}, nil
+	return retVal, err
 }
 
 func (agent *Agent) getSingleValue(ctx context.Context, request *extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
 	logger.Debugw(ctx, "get-single-value", log.Fields{"device-id": request.TargetId})
 
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
 
 	cloned := agent.cloneDeviceWithoutLock()
 
 	//send request to adapter
-	ch, err := agent.adapterProxy.GetSingleValue(ctx, cloned.Adapter, request)
-	agent.requestQueue.RequestComplete()
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        cloned.Id,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
+	// Release lock before sending request to adapter
+	agent.requestQueue.RequestComplete()
 
-	// Wait for the adapter response
-	rpcResponse, ok := <-ch
-	if !ok {
-		return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
+	resp, err := client.GetSingleValue(ctx, request)
+	if err == nil {
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
 	}
-
-	if rpcResponse.Err != nil {
-		return nil, rpcResponse.Err
-	}
-
-	resp := &extension.SingleGetValueResponse{}
-	if err := ptypes.UnmarshalAny(rpcResponse.Reply, resp); err != nil {
-		return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
-	}
-
-	return resp, nil
+	return resp, err
 }
 
 func (agent *Agent) setSingleValue(ctx context.Context, request *extension.SingleSetValueRequest) (*extension.SingleSetValueResponse, error) {
 	logger.Debugw(ctx, "set-single-value", log.Fields{"device-id": request.TargetId})
 
+	var err error
+	var desc string
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc) }()
+
 	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
@@ -1237,28 +1294,26 @@
 	cloned := agent.cloneDeviceWithoutLock()
 
 	//send request to adapter
-	ch, err := agent.adapterProxy.SetSingleValue(ctx, cloned.Adapter, request)
-	agent.requestQueue.RequestComplete()
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
+	// Release lock before sending request to adapter
+	agent.requestQueue.RequestComplete()
 
-	// Wait for the adapter response
-	rpcResponse, ok := <-ch
-	if !ok {
-		return nil, status.Errorf(codes.Aborted, "channel-closed-cloned-id-%s", agent.deviceID)
+	resp, err := client.SetSingleValue(ctx, request)
+	if err == nil {
+		requestStatus.Code = common.OperationResp_OPERATION_SUCCESS
 	}
-
-	if rpcResponse.Err != nil {
-		return nil, rpcResponse.Err
-	}
-
-	resp := &extension.SingleSetValueResponse{}
-	if err := ptypes.UnmarshalAny(rpcResponse.Reply, resp); err != nil {
-		return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
-	}
-
-	return resp, nil
+	return resp, err
 }
 
 func (agent *Agent) proceedWithRequest(device *voltha.Device) bool {
@@ -1273,32 +1328,66 @@
 	agent.stopReconcilingMutex.Unlock()
 }
 
-func (agent *Agent) ReconcileDevice(ctx context.Context, device *voltha.Device) {
+// abortAllProcessing is invoked when an adapter managing this device is restarted
+func (agent *Agent) abortAllProcessing(ctx context.Context) error {
+	logger.Infow(ctx, "aborting-current-running-requests", log.Fields{"device-id": agent.deviceID})
+	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		return err
+	}
+	defer agent.requestQueue.RequestComplete()
+
+	// If any reconciling is in progress just abort it. The adapter is gone.
+	agent.stopReconcile()
+
+	// Update the Core device transient state accordingly
+	var updatedState core.DeviceTransientState_Types
+	switch agent.getTransientState() {
+	case core.DeviceTransientState_RECONCILE_IN_PROGRESS:
+		updatedState = core.DeviceTransientState_NONE
+	case core.DeviceTransientState_FORCE_DELETING:
+		updatedState = core.DeviceTransientState_DELETE_FAILED
+	case core.DeviceTransientState_DELETING_FROM_ADAPTER:
+		updatedState = core.DeviceTransientState_DELETE_FAILED
+	default:
+		updatedState = core.DeviceTransientState_NONE
+	}
+	if err := agent.updateTransientState(ctx, updatedState); err != nil {
+		logger.Errorf(ctx, "transient-state-update-failed", log.Fields{"error": err})
+		return err
+	}
+	return nil
+}
+
+func (agent *Agent) ReconcileDevice(ctx context.Context) {
+	requestStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
 	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
 
 	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-		desc = err.Error()
-		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc)
+		return
+	}
+
+	device := agent.getDeviceReadOnlyWithoutLock()
+	if device.AdminState == voltha.AdminState_PREPROVISIONED {
+		agent.requestQueue.RequestComplete()
+		logger.Debugw(ctx, "device-in-preprovisioning-state-reconcile-not-needed", log.Fields{"device-id": device.Id})
 		return
 	}
 
 	if !agent.proceedWithRequest(device) {
 		agent.requestQueue.RequestComplete()
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", device.Id)
-		logger.Errorf(ctx, desc)
-		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		err := fmt.Errorf("cannot complete operation as device deletion/reconciling is in progress or reconcile failed for device : %s", device.Id)
+		logger.Errorw(ctx, "reconcile-failed", log.Fields{"error": err})
+		agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc)
 		return
 	}
 
 	//set transient state to RECONCILE IN PROGRESS
-	err := agent.updateTransientState(ctx, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	err := agent.updateTransientState(ctx, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	if err != nil {
 		agent.requestQueue.RequestComplete()
-		desc = fmt.Sprintf("Not able to set device transient state to Reconcile in progress."+
-			"Err: %s", err.Error())
-		logger.Errorf(ctx, desc)
-		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		logger.Errorw(ctx, "setting-transient-state-failed", log.Fields{"error": err})
+		agent.logDeviceUpdate(ctx, nil, nil, requestStatus, nil, desc)
 		return
 	}
 
@@ -1343,43 +1432,35 @@
 
 		backoffTimer = time.NewTimer(duration)
 
-		logger.Debugw(ctx, "retrying-reconciling", log.Fields{"deviceID": device.Id})
-		// Send a reconcile request to the adapter.
-		ch, err := agent.adapterProxy.ReconcileDevice(ctx, agent.device)
-		//release lock before moving further
+		logger.Debugw(ctx, "retrying-reconciling", log.Fields{"deviceID": device.Id, "endpoint": device.AdapterEndpoint})
+		// Release lock before sending request to adapter
 		agent.requestQueue.RequestComplete()
+
+		// Send a reconcile request to the adapter.
+		err := agent.sendReconcileRequestToAdapter(ctx, device)
+		if errors.Is(err, errContextExpired) || errors.Is(err, errReconcileAborted) {
+			logger.Errorw(ctx, "reconcile-aborted", log.Fields{"error": err})
+			requestStatus = &common.OperationResp{Code: common.OperationResp_OperationReturnCode(common.OperStatus_FAILED)}
+			desc = "aborted"
+			agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc)
+			break retry
+		}
 		if err != nil {
-			desc := fmt.Sprintf("Failed reconciling from adapter side. Err: %s", err.Error())
-			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+			agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc)
 			<-backoffTimer.C
 			// backoffTimer expired continue
 			// Take lock back before retrying
 			if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-				desc = err.Error()
-				agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+				agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc)
 				break retry
 			}
 			continue
 		}
-
-		// if return err retry if not then break loop and quit retrying reconcile
-		if err = agent.waitForReconcileResponse(backoffTimer, ch); err != nil {
-			desc = err.Error()
-			logger.Errorf(ctx, desc)
-			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
-			<-backoffTimer.C
-		} else {
-			operStatus = &common.OperationResp{Code: common.OperationResp_OPERATION_IN_PROGRESS}
-			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
-			break retry
-		}
-
-		// Take lock back before retrying
-		if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
-			break retry
-		}
+		// Success
+		requestStatus = &common.OperationResp{Code: common.OperationResp_OPERATION_SUCCESS}
+		desc = "adapter-response"
+		agent.logDeviceUpdate(ctx, nil, nil, requestStatus, err, desc)
+		break retry
 	}
 
 	// Retry loop is broken, so stop any timers and drain the channel
@@ -1400,18 +1481,23 @@
 	}
 }
 
-func (agent *Agent) waitForReconcileResponse(backoffTimer *time.Timer, ch chan *kafka.RpcResponse) error {
+func (agent *Agent) sendReconcileRequestToAdapter(ctx context.Context, device *voltha.Device) error {
+	logger.Debugw(ctx, "sending-reconcile-to-adapter", log.Fields{"device-id": device.Id, "endpoint": agent.adapterEndpoint})
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		return err
+	}
+	adapterResponse := make(chan error)
+	go func() {
+		_, err := client.ReconcileDevice(ctx, device)
+		adapterResponse <- err
+	}()
 	select {
 	// wait for response
-	case resp, ok := <-ch:
-		if !ok {
-			//channel-closed
-			return errors.New("channel on which reconcile response is awaited is closed")
-		} else if resp.Err != nil {
-			//error encountered
-			return fmt.Errorf("error encountered while retrying reconcile. Err: %s", resp.Err.Error())
+	case err := <-adapterResponse:
+		if err != nil {
+			return err
 		}
-
 		//In case of success quit retrying and wait for adapter to reset operation state of device
 		agent.stopReconcilingMutex.Lock()
 		agent.stopReconciling = nil
@@ -1425,33 +1511,47 @@
 		agent.stopReconcilingMutex.Unlock()
 		if !ok {
 			//channel-closed
-			return errors.New("channel used to notify to stop reconcile is closed")
+			return fmt.Errorf("reconcile channel closed:%w", errReconcileAborted)
 		}
-		return nil
-	//continue if timer expired
-	case <-backoffTimer.C:
+		return fmt.Errorf("reconciling aborted:%w", errReconcileAborted)
+	// Context expired
+	case <-ctx.Done():
+		return fmt.Errorf("context expired:%s :%w", ctx.Err(), errContextExpired)
 	}
-	return nil
 }
 
 func (agent *Agent) reconcilingCleanup(ctx context.Context) error {
 	var desc string
+	var err error
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-		desc = err.Error()
-		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		desc = "reconcile-cleanup-failed"
 		return err
 	}
 	defer agent.requestQueue.RequestComplete()
-	err := agent.updateTransientState(ctx, voltha.DeviceTransientState_NONE)
+	err = agent.updateTransientState(ctx, core.DeviceTransientState_NONE)
 	if err != nil {
-		desc = fmt.Sprintf("Not able to clear device transient state from Reconcile in progress."+
-			"Err: %s", err.Error())
-		logger.Errorf(ctx, desc)
-		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		logger.Errorf(ctx, "transient-state-update-failed", log.Fields{"error": err})
 		return err
 	}
-	operStatus = &common.OperationResp{Code: common.OperationResp_OPERATION_SUCCESS}
-	agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+	operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 	return nil
 }
+
+func (agent *Agent) isAdapterConnectionUp(ctx context.Context) bool {
+	c, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	return c != nil && err == nil
+}
+
+func (agent *Agent) canDeviceRequestProceed(ctx context.Context) error {
+	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		return err
+	}
+	defer agent.requestQueue.RequestComplete()
+	if agent.proceedWithRequest(agent.device) {
+		return nil
+	}
+	return fmt.Errorf("device-cannot-process-request-%s", agent.deviceID)
+}
diff --git a/rw_core/core/device/agent_device_update.go b/rw_core/core/device/agent_device_update.go
index a4f2986..599c2cf 100644
--- a/rw_core/core/device/agent_device_update.go
+++ b/rw_core/core/device/agent_device_update.go
@@ -19,23 +19,31 @@
 import (
 	"context"
 	"fmt"
+
 	"github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/common"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
 )
 
-func (agent *Agent) logDeviceUpdate(ctx context.Context, operation string, prevState *common.AdminState_Types, currState *common.AdminState_Types, status *common.OperationResp, desc *string) {
-	logger.Debugw(ctx, "addDeviceUpdate", log.Fields{"device-id": agent.deviceID})
-
-	requestedBy := utils.GetFromTopicMetadataFromContext(ctx)
+func (agent *Agent) logDeviceUpdate(ctx context.Context, prevState, currState *common.AdminState_Types, status *common.OperationResp, err error, desc string) {
+	requestedBy := utils.GetEndpointMetadataFromContext(ctx)
 
 	if requestedBy == "" {
 		requestedBy = "NB"
 	}
 
-	logger.Infow(ctx, "logDeviceUpdate", log.Fields{"device-update": operation, "device-update-id": agent.deviceID,
+	rpc := utils.GetRPCMetadataFromContext(ctx)
+
+	fields := log.Fields{"rpc": rpc, "device-id": agent.deviceID,
 		"requested-by": requestedBy, "state-change": agent.stateChangeString(prevState, currState),
-		"status": status.GetCode().String(), "description": desc})
+		"status": status.GetCode().String(), "description": desc, "error": err}
+
+	if err != nil {
+		logger.Errorw(ctx, "logDeviceUpdate-failed", fields)
+		return
+	}
+
+	logger.Infow(ctx, "logDeviceUpdate-success", fields)
 }
 
 func (agent *Agent) stateChangeString(prevState *common.AdminState_Types, currState *common.AdminState_Types) string {
diff --git a/rw_core/core/device/agent_flow.go b/rw_core/core/device/agent_flow.go
index f13003b..6ad4488 100644
--- a/rw_core/core/device/agent_flow.go
+++ b/rw_core/core/device/agent_flow.go
@@ -20,13 +20,15 @@
 	"context"
 	"fmt"
 
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+
 	"github.com/gogo/protobuf/proto"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -47,8 +49,10 @@
 func (agent *Agent) addFlowsToAdapter(ctx context.Context, newFlows []*ofp.OfpFlowStats, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
 	logger.Debugw(ctx, "add-flows-to-adapters", log.Fields{"device-id": agent.deviceID, "flows": newFlows, "flow-metadata": flowMetadata})
 
+	var err error
 	var desc string
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	if (len(newFlows)) == 0 {
 		logger.Debugw(ctx, "nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": newFlows})
@@ -56,22 +60,17 @@
 	}
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		desc = err.Error()
-		agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
 		return coreutils.DoneResponse(), status.Errorf(codes.Aborted, "%s", err)
 	}
 
 	if !agent.proceedWithRequest(device) {
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", agent.deviceID)
-		agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "%s", desc)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return coreutils.DoneResponse(), err
 	}
 
 	dType, err := agent.adapterMgr.GetDeviceType(ctx, &voltha.ID{Id: device.Type})
 	if err != nil {
-		desc = fmt.Sprintf("non-existent-device-type-%s", device.Type)
-		agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+		return coreutils.DoneResponse(), err
 	}
 
 	flowsToAdd := make([]*ofp.OfpFlowStats, 0)
@@ -80,7 +79,6 @@
 		flowHandle, created, err := agent.flowCache.LockOrCreate(ctx, flow)
 		if err != nil {
 			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
 			return coreutils.DoneResponse(), err
 		}
 		if created {
@@ -91,9 +89,7 @@
 				//Flow needs to be updated.
 				if err := flowHandle.Update(ctx, flow); err != nil {
 					flowHandle.Unlock()
-					desc = fmt.Sprintf("failure-updating-flow-%d-to-device-%s", flow.Id, agent.deviceID)
-					agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
-					return coreutils.DoneResponse(), status.Errorf(codes.Internal, "failure-updating-flow-%d-to-device-%s", flow.Id, agent.deviceID)
+					return coreutils.DoneResponse(), err
 				}
 				flowsToDelete = append(flowsToDelete, flowToReplace)
 				flowsToAdd = append(flowsToAdd, flow)
@@ -108,25 +104,21 @@
 	// Sanity check
 	if (len(flowsToAdd)) == 0 {
 		logger.Debugw(ctx, "no-flows-to-update", log.Fields{"device-id": agent.deviceID, "flows": newFlows})
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return coreutils.DoneResponse(), nil
 	}
 
 	// Send update to adapters
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
 	response := coreutils.NewResponse()
+	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
 	if !dType.AcceptsAddRemoveFlowUpdates {
-
 		updatedAllFlows := agent.listDeviceFlows()
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, updatedAllFlows, nil, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
+		ctr, flowSlice := 0, make([]*ofp.OfpFlowStats, len(updatedAllFlows))
+		for _, flow := range updatedAllFlows {
+			flowSlice[ctr] = flow
+			ctr++
 		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "addFlowsToAdapter", rpcResponse, response)
+		go agent.sendBulkFlows(subCtx, device, &voltha.Flows{Items: flowSlice}, nil, flowMetadata, response)
 	} else {
 		flowChanges := &ofp.FlowChanges{
 			ToAdd:    &voltha.Flows{Items: flowsToAdd},
@@ -137,55 +129,128 @@
 			ToRemove: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 			ToUpdate: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 		}
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
-		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "addFlowsToAdapter", rpcResponse, response)
+		go agent.sendIncrementalFlows(subCtx, device, flowChanges, groupChanges, flowMetadata, response)
 	}
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-	agent.logDeviceUpdate(ctx, "addFlowsToAdapter", nil, nil, operStatus, &desc)
 	return response, nil
 }
 
+func (agent *Agent) sendBulkFlows(
+	ctx context.Context,
+	device *voltha.Device,
+	flows *voltha.Flows,
+	groups *voltha.FlowGroups,
+	flowMetadata *voltha.FlowMetadata,
+	response coreutils.Response,
+) {
+	var err error
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
+	// Get a grpc client
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
+		response.Error(err)
+		return
+	}
+	subCtx, cancel := context.WithTimeout(ctx, agent.rpcTimeout)
+	defer cancel()
+
+	if _, err = client.UpdateFlowsBulk(subCtx, &ic.BulkFlows{
+		Device:       device,
+		Flows:        flows,
+		Groups:       groups,
+		FlowMetadata: flowMetadata,
+	}); err != nil {
+		response.Error(err)
+	} else {
+		response.Done()
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
+	}
+}
+
+func (agent *Agent) sendIncrementalFlows(
+	ctx context.Context,
+	device *voltha.Device,
+	flowChanges *ofp.FlowChanges,
+	groupChanges *ofp.FlowGroupChanges,
+	flowMetadata *voltha.FlowMetadata,
+	response coreutils.Response,
+) {
+	var err error
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
+	// Get a grpc client
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
+		response.Error(err)
+		return
+	}
+	subCtx, cancel := context.WithTimeout(ctx, agent.rpcTimeout)
+	defer cancel()
+	if _, err = client.UpdateFlowsIncrementally(subCtx, &ic.IncrementalFlows{
+		Device:       device,
+		Flows:        flowChanges,
+		Groups:       groupChanges,
+		FlowMetadata: flowMetadata,
+	}); err != nil {
+		response.Error(err)
+	} else {
+		response.Done()
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
+	}
+}
+
 func (agent *Agent) deleteFlowsFromAdapter(ctx context.Context, flowsToDel []*ofp.OfpFlowStats, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
 	logger.Debugw(ctx, "delete-flows-from-adapter", log.Fields{"device-id": agent.deviceID, "flows": flowsToDel})
 
 	var desc string
+	var err error
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	if (len(flowsToDel)) == 0 {
 		logger.Debugw(ctx, "nothing-to-delete", log.Fields{"device-id": agent.deviceID, "flows": flowsToDel})
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return coreutils.DoneResponse(), nil
 	}
 
-	defer agent.logDeviceUpdate(ctx, "deleteFlowsFromAdapter", nil, nil, operStatus, &desc)
-
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		desc = err.Error()
-		return coreutils.DoneResponse(), status.Errorf(codes.Aborted, "%s", err)
+		return coreutils.DoneResponse(), err
 	}
 
 	if !agent.proceedWithRequest(device) {
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", device.Id)
-		agent.logDeviceUpdate(ctx, "deleteFlowsFromAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "%s", desc)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return coreutils.DoneResponse(), err
 	}
 
 	dType, err := agent.adapterMgr.GetDeviceType(ctx, &voltha.ID{Id: device.Type})
 	if err != nil {
-		desc = fmt.Sprintf("non-existent-device-type-%s", device.Type)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+		return coreutils.DoneResponse(), err
 	}
 
 	for _, flow := range flowsToDel {
 		if flowHandle, have := agent.flowCache.Lock(flow.Id); have {
 			// Update the store and cache
-			if err := flowHandle.Delete(ctx); err != nil {
+			if err = flowHandle.Delete(ctx); err != nil {
 				flowHandle.Unlock()
 				desc = err.Error()
 				return coreutils.DoneResponse(), err
@@ -195,20 +260,16 @@
 	}
 
 	// Send update to adapters
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
 	response := coreutils.NewResponse()
+	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
 	if !dType.AcceptsAddRemoveFlowUpdates {
-
 		updatedAllFlows := agent.listDeviceFlows()
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, updatedAllFlows, nil, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			return coreutils.DoneResponse(), err
+		ctr, flowSlice := 0, make([]*ofp.OfpFlowStats, len(updatedAllFlows))
+		for _, flow := range updatedAllFlows {
+			flowSlice[ctr] = flow
+			ctr++
 		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "deleteFlowToAdapter", rpcResponse, response)
+		go agent.sendBulkFlows(subCtx, device, &voltha.Flows{Items: flowSlice}, nil, flowMetadata, response)
 	} else {
 		flowChanges := &ofp.FlowChanges{
 			ToAdd:    &voltha.Flows{Items: []*ofp.OfpFlowStats{}},
@@ -219,13 +280,7 @@
 			ToRemove: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 			ToUpdate: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 		}
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			return coreutils.DoneResponse(), err
-		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "deleteFlowToAdapter", rpcResponse, response)
+		go agent.sendIncrementalFlows(subCtx, device, flowChanges, groupChanges, flowMetadata, response)
 	}
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
 	return response, nil
@@ -234,37 +289,35 @@
 func (agent *Agent) updateFlowsToAdapter(ctx context.Context, updatedFlows []*ofp.OfpFlowStats, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
 	logger.Debugw(ctx, "update-flows-to-adapter", log.Fields{"device-id": agent.deviceID, "flows": updatedFlows})
 
+	var err error
 	var desc string
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	if (len(updatedFlows)) == 0 {
 		logger.Debugw(ctx, "nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": updatedFlows})
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return coreutils.DoneResponse(), nil
 	}
 
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		return coreutils.DoneResponse(), status.Errorf(codes.Aborted, "%s", err)
+		return coreutils.DoneResponse(), err
 	}
 
 	if !agent.proceedWithRequest(device) {
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", device.Id)
-		agent.logDeviceUpdate(ctx, "updateFlowsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "%s", desc)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return coreutils.DoneResponse(), err
 	}
 
 	if device.OperStatus != voltha.OperStatus_ACTIVE || device.ConnectStatus != voltha.ConnectStatus_REACHABLE || device.AdminState != voltha.AdminState_ENABLED {
-		desc = "invalid device states"
-		agent.logDeviceUpdate(ctx, "updateFlowsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "invalid device states")
+		err = status.Errorf(codes.FailedPrecondition, "invalid device states")
+		return coreutils.DoneResponse(), err
 	}
 
 	dType, err := agent.adapterMgr.GetDeviceType(ctx, &voltha.ID{Id: device.Type})
 	if err != nil {
-		desc = fmt.Sprintf("non-existent-device-type-%s", device.Type)
-		agent.logDeviceUpdate(ctx, "updateFlowsToAdapter", nil, nil, operStatus, &desc)
-
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+		return coreutils.DoneResponse(), err
 	}
 
 	flowsToAdd := make([]*ofp.OfpFlowStats, 0, len(updatedFlows))
@@ -273,10 +326,8 @@
 		if flowHandle, have := agent.flowCache.Lock(flow.Id); have {
 			flowToDelete := flowHandle.GetReadOnly()
 			// Update the store and cache
-			if err := flowHandle.Update(ctx, flow); err != nil {
+			if err = flowHandle.Update(ctx, flow); err != nil {
 				flowHandle.Unlock()
-				desc = err.Error()
-				agent.logDeviceUpdate(ctx, "updateFlowsToAdapter", nil, nil, operStatus, &desc)
 				return coreutils.DoneResponse(), err
 			}
 
@@ -286,21 +337,17 @@
 		}
 	}
 
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
 	response := coreutils.NewResponse()
+	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
 	// Process bulk flow update differently than incremental update
 	if !dType.AcceptsAddRemoveFlowUpdates {
 		updatedAllFlows := agent.listDeviceFlows()
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, updatedAllFlows, nil, nil)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "updateFlowsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
+		ctr, flowSlice := 0, make([]*ofp.OfpFlowStats, len(updatedAllFlows))
+		for _, flow := range updatedAllFlows {
+			flowSlice[ctr] = flow
+			ctr++
 		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "updateFlowToAdapter", rpcResponse, response)
+		go agent.sendBulkFlows(subCtx, device, &voltha.Flows{Items: flowSlice}, nil, flowMetadata, response)
 	} else {
 		logger.Debugw(ctx, "updating-flows-and-groups",
 			log.Fields{
@@ -311,7 +358,7 @@
 		// Sanity check
 		if (len(flowsToAdd) | len(flowsToDelete)) == 0 {
 			logger.Debugw(ctx, "nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": updatedFlows})
-			cancel()
+			operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 			return coreutils.DoneResponse(), nil
 		}
 
@@ -324,18 +371,9 @@
 			ToRemove: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 			ToUpdate: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 		}
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "updateFlowsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
-		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "updateFlowToAdapter", rpcResponse, response)
+		go agent.sendIncrementalFlows(subCtx, device, flowChanges, groupChanges, flowMetadata, response)
 	}
-
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-	agent.logDeviceUpdate(ctx, "updateFlowsToAdapter", nil, nil, operStatus, &desc)
 	return response, nil
 }
 
@@ -362,7 +400,7 @@
 	if err != nil {
 		return err
 	}
-	if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, response); res != nil {
+	if res := coreutils.WaitForNilOrErrorResponses(agent.rpcTimeout, response); res != nil {
 		return status.Errorf(codes.Aborted, "errors-%s", res)
 	}
 	return nil
@@ -372,30 +410,29 @@
 func (agent *Agent) deleteAllFlows(ctx context.Context) error {
 	logger.Debugw(ctx, "deleteAllFlows", log.Fields{"device-id": agent.deviceID})
 
-	var error string
+	var err error
+	var errFlows string
 	var desc string
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-
-	defer agent.logDeviceUpdate(ctx, "deleteAllFlows", nil, nil, operStatus, &desc)
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	for flowID := range agent.flowCache.ListIDs() {
 		if flowHandle, have := agent.flowCache.Lock(flowID); have {
 			// Update the store and cache
-			if err := flowHandle.Delete(ctx); err != nil {
+			if err = flowHandle.Delete(ctx); err != nil {
 				flowHandle.Unlock()
-				error += fmt.Sprintf("%v ", flowID)
-				logger.Errorw(ctx, "unable-to-delete-flow", log.Fields{"device-id": agent.deviceID, "flowID": flowID})
+				errFlows += fmt.Sprintf("%v ", flowID)
+				logger.Errorw(ctx, "unable-to-delete-flow", log.Fields{"device-id": agent.deviceID, "flowID": flowID, "error": err})
 				continue
 			}
 			flowHandle.Unlock()
 		}
 	}
 
-	if error != "" {
-		desc = fmt.Sprintf("Unable to delete flows : %s", error)
+	if errFlows != "" {
+		err = fmt.Errorf("unable to delete flows : %s", errFlows)
 	} else {
 		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 	}
-
-	return nil
+	return err
 }
diff --git a/rw_core/core/device/agent_group.go b/rw_core/core/device/agent_group.go
index ba58d2f..43a8929 100644
--- a/rw_core/core/device/agent_group.go
+++ b/rw_core/core/device/agent_group.go
@@ -18,15 +18,13 @@
 
 import (
 	"context"
-	"fmt"
-	"strconv"
 
 	"github.com/gogo/protobuf/proto"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -47,32 +45,30 @@
 func (agent *Agent) addGroupsToAdapter(ctx context.Context, newGroups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
 	logger.Debugw(ctx, "add-groups-to-adapters", log.Fields{"device-id": agent.deviceID, "groups": newGroups, "flow-metadata": flowMetadata})
 
+	var err error
 	var desc string
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	if (len(newGroups)) == 0 {
-		logger.Debugw(ctx, "nothing-to-update", log.Fields{"device-id": agent.deviceID, "groups": newGroups})
+		desc = "no new groups"
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return coreutils.DoneResponse(), nil
 	}
 
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		desc = err.Error()
-		agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.Aborted, "%s", err)
+		return coreutils.DoneResponse(), err
 	}
 
 	if !agent.proceedWithRequest(device) {
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", device.Id)
-		agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "%s", desc)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return coreutils.DoneResponse(), err
 	}
 
 	dType, err := agent.adapterMgr.GetDeviceType(ctx, &voltha.ID{Id: device.Type})
 	if err != nil {
-		desc = fmt.Sprintf("non-existent-device-type-%s", device.Type)
-		agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+		return coreutils.DoneResponse(), err
 	}
 
 	groupsToAdd := make([]*ofp.OfpGroupEntry, 0)
@@ -80,8 +76,6 @@
 	for _, group := range newGroups {
 		groupHandle, created, err := agent.groupCache.LockOrCreate(ctx, group)
 		if err != nil {
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
 			return coreutils.DoneResponse(), err
 		}
 
@@ -91,11 +85,9 @@
 			groupToChange := groupHandle.GetReadOnly()
 			if !proto.Equal(groupToChange, group) {
 				//Group needs to be updated.
-				if err := groupHandle.Update(ctx, group); err != nil {
+				if err = groupHandle.Update(ctx, group); err != nil {
 					groupHandle.Unlock()
-					desc = fmt.Sprintf("failure-updating-group-%s-to-device-%s", strconv.Itoa(int(group.Desc.GroupId)), agent.deviceID)
-					agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
-					return coreutils.DoneResponse(), status.Errorf(codes.Internal, "failure-updating-group-%s-to-device-%s", strconv.Itoa(int(group.Desc.GroupId)), agent.deviceID)
+					return coreutils.DoneResponse(), err
 				}
 				groupsToDelete = append(groupsToDelete, groupToChange)
 				groupsToAdd = append(groupsToAdd, group)
@@ -109,25 +101,22 @@
 	}
 	// Sanity check
 	if (len(groupsToAdd)) == 0 {
-		logger.Debugw(ctx, "no-groups-to-update", log.Fields{"device-id": agent.deviceID, "groups": newGroups})
+		desc = "no group to update"
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return coreutils.DoneResponse(), nil
 	}
 
 	// Send update to adapters
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
 	response := coreutils.NewResponse()
+	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
 	if !dType.AcceptsAddRemoveFlowUpdates {
 		updatedAllGroups := agent.listDeviceGroups()
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, nil, updatedAllGroups, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
+		ctr, groupSlice := 0, make([]*ofp.OfpGroupEntry, len(updatedAllGroups))
+		for _, group := range updatedAllGroups {
+			groupSlice[ctr] = group
+			ctr++
 		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "addGroupsToAdapter", rpcResponse, response)
+		go agent.sendBulkFlows(subCtx, device, nil, &voltha.FlowGroups{Items: groupSlice}, flowMetadata, response)
 	} else {
 		flowChanges := &ofp.FlowChanges{
 			ToAdd:    &voltha.Flows{Items: []*ofp.OfpFlowStats{}},
@@ -138,57 +127,46 @@
 			ToRemove: &voltha.FlowGroups{Items: groupsToDelete},
 			ToUpdate: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 		}
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
-		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "addGroupsToAdapter", rpcResponse, response)
+		go agent.sendIncrementalFlows(subCtx, device, flowChanges, groupChanges, flowMetadata, response)
 	}
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-	agent.logDeviceUpdate(ctx, "addGroupsToAdapter", nil, nil, operStatus, &desc)
 	return response, nil
 }
 
 func (agent *Agent) deleteGroupsFromAdapter(ctx context.Context, groupsToDel []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
 	logger.Debugw(ctx, "delete-groups-from-adapter", log.Fields{"device-id": agent.deviceID, "groups": groupsToDel})
 
+	var desc string
+	var err error
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
 	if (len(groupsToDel)) == 0 {
-		logger.Debugw(ctx, "nothing-to-delete", log.Fields{"device-id": agent.deviceID})
+		desc = "nothing to delete"
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return coreutils.DoneResponse(), nil
 	}
 
-	var desc string
-	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-
-	defer agent.logDeviceUpdate(ctx, "deleteGroupsFromAdapter", nil, nil, operStatus, &desc)
-
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		desc = err.Error()
-		return coreutils.DoneResponse(), status.Errorf(codes.Aborted, "%s", err)
+		return coreutils.DoneResponse(), err
 	}
 
 	if !agent.proceedWithRequest(device) {
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", device.Id)
-		agent.logDeviceUpdate(ctx, "deleteGroupsFromAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "%s", desc)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return coreutils.DoneResponse(), err
 	}
 
 	dType, err := agent.adapterMgr.GetDeviceType(ctx, &voltha.ID{Id: device.Type})
 	if err != nil {
-		desc = fmt.Sprintf("non-existent-device-type-%s", device.Type)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+		return coreutils.DoneResponse(), err
 	}
 
 	for _, group := range groupsToDel {
 		if groupHandle, have := agent.groupCache.Lock(group.Desc.GroupId); have {
 			// Update the store and cache
-			if err := groupHandle.Delete(ctx); err != nil {
+			if err = groupHandle.Delete(ctx); err != nil {
 				groupHandle.Unlock()
-				desc = err.Error()
 				return coreutils.DoneResponse(), err
 			}
 			groupHandle.Unlock()
@@ -196,19 +174,16 @@
 	}
 
 	// Send update to adapters
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
 	response := coreutils.NewResponse()
+	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
 	if !dType.AcceptsAddRemoveFlowUpdates {
 		updatedAllGroups := agent.listDeviceGroups()
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, nil, updatedAllGroups, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			return coreutils.DoneResponse(), err
+		ctr, groupSlice := 0, make([]*ofp.OfpGroupEntry, len(updatedAllGroups))
+		for _, group := range updatedAllGroups {
+			groupSlice[ctr] = group
+			ctr++
 		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "deleteGroupsFromAdapter", rpcResponse, response)
+		go agent.sendBulkFlows(subCtx, device, nil, &voltha.FlowGroups{Items: groupSlice}, flowMetadata, response)
 	} else {
 		flowChanges := &ofp.FlowChanges{
 			ToAdd:    &voltha.Flows{Items: []*ofp.OfpFlowStats{}},
@@ -219,13 +194,7 @@
 			ToRemove: &voltha.FlowGroups{Items: groupsToDel},
 			ToUpdate: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 		}
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			return coreutils.DoneResponse(), err
-		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "deleteGroupsFromAdapter", rpcResponse, response)
+		go agent.sendIncrementalFlows(subCtx, device, flowChanges, groupChanges, flowMetadata, response)
 	}
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
 	return response, nil
@@ -235,47 +204,43 @@
 	logger.Debugw(ctx, "update-groups-to-adapter", log.Fields{"device-id": agent.deviceID, "groups": updatedGroups})
 
 	var desc string
+	var err error
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	if (len(updatedGroups)) == 0 {
-		logger.Debugw(ctx, "nothing-to-update", log.Fields{"device-id": agent.deviceID, "groups": updatedGroups})
+		desc = "no groups to update"
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return coreutils.DoneResponse(), nil
 	}
 
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		desc = err.Error()
-		agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.Aborted, "%s", err)
+		return coreutils.DoneResponse(), err
 	}
 
 	if !agent.proceedWithRequest(device) {
-		desc = fmt.Sprintf("deviceId:%s, Cannot complete operation as device deletion is in progress or Reconciling is in progress/failed", device.Id)
-		agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "%s", desc)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return coreutils.DoneResponse(), err
 	}
 
 	if device.OperStatus != voltha.OperStatus_ACTIVE || device.ConnectStatus != voltha.ConnectStatus_REACHABLE || device.AdminState != voltha.AdminState_ENABLED {
-		desc = fmt.Sprintf("invalid device states-oper-%s-connect-%s-admin-%s", device.OperStatus, device.ConnectStatus, device.AdminState)
-		agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "invalid device states-oper-%s-connect-%s-admin-%s", device.OperStatus, device.ConnectStatus, device.AdminState)
+		err = status.Errorf(codes.FailedPrecondition, "invalid device states-oper-%s-connect-%s-admin-%s", device.OperStatus, device.ConnectStatus, device.AdminState)
+		return coreutils.DoneResponse(), err
 	}
 
 	dType, err := agent.adapterMgr.GetDeviceType(ctx, &voltha.ID{Id: device.Type})
 	if err != nil {
-		desc = fmt.Sprintf("non-existent-device-type-%s", device.Type)
-		agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
-		return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+		err = status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+		return coreutils.DoneResponse(), err
 	}
 
 	groupsToUpdate := make([]*ofp.OfpGroupEntry, 0)
 	for _, group := range updatedGroups {
 		if groupHandle, have := agent.groupCache.Lock(group.Desc.GroupId); have {
 			// Update the store and cache
-			if err := groupHandle.Update(ctx, group); err != nil {
+			if err = groupHandle.Update(ctx, group); err != nil {
 				groupHandle.Unlock()
-				desc = err.Error()
-				agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
 				return coreutils.DoneResponse(), err
 			}
 			groupsToUpdate = append(groupsToUpdate, group)
@@ -283,21 +248,17 @@
 		}
 	}
 
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
 	response := coreutils.NewResponse()
+	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
 	// Process bulk flow update differently than incremental update
 	if !dType.AcceptsAddRemoveFlowUpdates {
 		updatedAllGroups := agent.listDeviceGroups()
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, nil, updatedAllGroups, nil)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
+		ctr, groupSlice := 0, make([]*ofp.OfpGroupEntry, len(updatedAllGroups))
+		for _, group := range updatedAllGroups {
+			groupSlice[ctr] = group
+			ctr++
 		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "updateGroupsToAdapter", rpcResponse, response)
+		go agent.sendBulkFlows(subCtx, device, nil, &voltha.FlowGroups{Items: groupSlice}, flowMetadata, response)
 	} else {
 		logger.Debugw(ctx, "updating-groups",
 			log.Fields{
@@ -307,8 +268,8 @@
 
 		// Sanity check
 		if (len(groupsToUpdate)) == 0 {
-			logger.Debugw(ctx, "nothing-to-update", log.Fields{"device-id": agent.deviceID, "groups": groupsToUpdate})
-			cancel()
+			desc = "nothing to update"
+			operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 			return coreutils.DoneResponse(), nil
 		}
 
@@ -321,17 +282,9 @@
 			ToRemove: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
 			ToUpdate: &voltha.FlowGroups{Items: groupsToUpdate},
 		}
-		rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
-		if err != nil {
-			cancel()
-			desc = err.Error()
-			agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
-			return coreutils.DoneResponse(), err
-		}
-		go agent.waitForAdapterFlowResponse(subCtx, cancel, "updateGroupsToAdapter", rpcResponse, response)
+		go agent.sendIncrementalFlows(subCtx, device, flowChanges, groupChanges, flowMetadata, response)
 	}
 
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-	agent.logDeviceUpdate(ctx, "updateGroupsToAdapter", nil, nil, operStatus, &desc)
 	return response, nil
 }
diff --git a/rw_core/core/device/agent_image.go b/rw_core/core/device/agent_image.go
index e71af8a..0fda20e 100644
--- a/rw_core/core/device/agent_image.go
+++ b/rw_core/core/device/agent_image.go
@@ -18,20 +18,28 @@
 
 import (
 	"context"
+	"errors"
+	"time"
 
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-protos/v4/go/common"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+
+	"github.com/opencord/voltha-protos/v5/go/common"
 
 	"github.com/gogo/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
 
 func (agent *Agent) downloadImage(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
+	var err error
+	var desc string
+	var prevAdminState, currAdminState common.AdminState_Types
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, &prevAdminState, &currAdminState, operStatus, err, desc) }()
+
 	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
@@ -39,23 +47,26 @@
 
 	if agent.device.Root {
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, is an OLT. Image update "+
+		err = status.Errorf(codes.FailedPrecondition, "device-id:%s, is an OLT. Image update "+
 			"not supported by VOLTHA. Use Device Manager or other means", agent.deviceID)
+		return nil, err
 	}
 
 	device := agent.cloneDeviceWithoutLock()
 
 	if !agent.proceedWithRequest(device) {
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed.",
-			agent.deviceID)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return nil, err
 	}
 
 	if device.ImageDownloads != nil {
 		for _, image := range device.ImageDownloads {
 			if image.DownloadState == voltha.ImageDownload_DOWNLOAD_REQUESTED {
-				return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, already downloading image:%s",
+				err = status.Errorf(codes.FailedPrecondition, "device-id:%s, already downloading image:%s",
 					agent.deviceID, image.Name)
+				agent.requestQueue.RequestComplete()
+				return nil, err
 			}
 		}
 	}
@@ -71,22 +82,39 @@
 		cloned.ImageDownloads[index] = clonedImg
 	}
 
-	cloned.AdminState = voltha.AdminState_DOWNLOADING_IMAGE
-	if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
-		return nil, err
-	}
-
 	// Send the request to the adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.DownloadImage(subCtx, cloned, clonedImg)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		cancel()
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
-	go agent.waitForAdapterResponse(subCtx, cancel, "downloadImage", ch, agent.onImageSuccess, agent.onImageFailure)
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		response, err := client.DownloadImage(subCtx, &ic.ImageDownloadMessage{
+			Device: cloned,
+			Image:  clonedImg,
+		})
+		if err == nil {
+			agent.onImageSuccess(subCtx, response)
+		} else {
+			agent.onImageFailure(subCtx, err)
+		}
+	}()
 
+	cloned.AdminState = voltha.AdminState_DOWNLOADING_IMAGE
+	if err = agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
+		return nil, err
+	}
 	return &common.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
 }
 
@@ -102,7 +130,14 @@
 }
 
 func (agent *Agent) cancelImageDownload(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+
+	var err error
+	var desc string
+	var prevAdminState, currAdminState common.AdminState_Types
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, &prevAdminState, &currAdminState, operStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
 	logger.Debugw(ctx, "cancel-image-download", log.Fields{"device-id": agent.deviceID})
@@ -111,8 +146,8 @@
 
 	if !agent.proceedWithRequest(cloned) {
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed.",
-			agent.deviceID)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return nil, err
 	}
 
 	// Update image download state
@@ -121,33 +156,59 @@
 		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
-
+	prevAdminState = cloned.AdminState
 	cloned.ImageDownloads[index].DownloadState = voltha.ImageDownload_DOWNLOAD_CANCELLED
 
 	if cloned.AdminState != voltha.AdminState_DOWNLOADING_IMAGE {
+		err = status.Errorf(codes.Aborted, "device not in image download state %s", cloned.Id)
 		agent.requestQueue.RequestComplete()
-	} else {
-		// Set the device to Enabled
-		cloned.AdminState = voltha.AdminState_ENABLED
-		if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
-			return nil, err
-		}
-		subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-		subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-		ch, err := agent.adapterProxy.CancelImageDownload(subCtx, cloned, img)
-		if err != nil {
-			cancel()
-			return nil, err
-		}
-		go agent.waitForAdapterResponse(subCtx, cancel, "cancelImageDownload", ch, agent.onImageSuccess,
-			agent.onImageFailure)
+		return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_FAILURE}, err
 	}
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		response, err := client.CancelImageDownload(subCtx, &ic.ImageDownloadMessage{
+			Device: cloned,
+			Image:  img,
+		})
+		if err == nil {
+			agent.onImageSuccess(subCtx, response)
+		} else {
+			agent.onImageFailure(subCtx, err)
+		}
+	}()
+
+	// Set the device to Enabled
+	cloned.AdminState = voltha.AdminState_ENABLED
+	if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
+		return nil, err
+	}
+	currAdminState = cloned.AdminState
+
 	return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
 }
 
 func (agent *Agent) activateImage(ctx context.Context, img *voltha.ImageDownload) (*voltha.OperationResp, error) {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
 	logger.Debugw(ctx, "activate-image", log.Fields{"device-id": agent.deviceID})
@@ -155,9 +216,9 @@
 	cloned := agent.cloneDeviceWithoutLock()
 
 	if !agent.proceedWithRequest(cloned) {
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, Cannot complete operation as device deletion is in progress or reconcile is in progress/failed.",
-			agent.deviceID)
+		return nil, err
 	}
 
 	// Update image download state
@@ -169,72 +230,125 @@
 
 	if image.DownloadState != voltha.ImageDownload_DOWNLOAD_SUCCEEDED {
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, device-has-not-downloaded-image:%s", agent.deviceID, img.Name)
+		err = status.Errorf(codes.FailedPrecondition, "device-id:%s, device-has-not-downloaded-image:%s", agent.deviceID, img.Name)
+		return nil, err
 	}
 
 	//TODO does this need to be removed ?
 	if cloned.AdminState == voltha.AdminState_DOWNLOADING_IMAGE {
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, device-in-downloading-state:%s", agent.deviceID, img.Name)
+		err = status.Errorf(codes.FailedPrecondition, "device-id:%s, device-in-downloading-state:%s", agent.deviceID, img.Name)
+		return nil, err
 	}
 
-	// Save the image
+	// Update the image
 	cloned.ImageDownloads[index].ImageState = voltha.ImageDownload_IMAGE_ACTIVATING
-
 	cloned.AdminState = voltha.AdminState_DOWNLOADING_IMAGE
+
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		response, err := client.ActivateImageUpdate(subCtx, &ic.ImageDownloadMessage{
+			Device: cloned,
+			Image:  img,
+		})
+		if err == nil {
+			agent.onImageSuccess(subCtx, response)
+		} else {
+			agent.onImageFailure(subCtx, err)
+		}
+	}()
+
+	// Save the image
 	if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
 		return nil, err
 	}
 
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.ActivateImageUpdate(subCtx, cloned, img)
-	if err != nil {
-		cancel()
-		return nil, err
-	}
-	go agent.waitForAdapterResponse(subCtx, cancel, "activateImageUpdate", ch, agent.onImageSuccess, agent.onFailure)
-
 	// The status of the AdminState will be changed following the update_download_status response from the adapter
 	// The image name will also be removed from the device list
 	return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
 }
 
 func (agent *Agent) revertImage(ctx context.Context, img *voltha.ImageDownload) (*voltha.OperationResp, error) {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
 	logger.Debugw(ctx, "revert-image", log.Fields{"device-id": agent.deviceID})
 
-	// Update image download state
 	cloned := agent.cloneDeviceWithoutLock()
+	if !agent.proceedWithRequest(cloned) {
+		agent.requestQueue.RequestComplete()
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return nil, err
+	}
+
+	// Update image download state
 	_, index, err := getImage(img, cloned)
 	if err != nil {
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, image-not-registered:%s", agent.deviceID, img.Name)
+		err = status.Errorf(codes.FailedPrecondition, "deviceId:%s, image-not-registered:%s", agent.deviceID, img.Name)
+		return nil, err
 	}
 	if cloned.AdminState != voltha.AdminState_ENABLED {
 		agent.requestQueue.RequestComplete()
-		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, device-not-enabled-state:%s", agent.deviceID, img.Name)
+		err = status.Errorf(codes.FailedPrecondition, "deviceId:%s, device-not-enabled-state:%s", agent.deviceID, img.Name)
+		return nil, err
 	}
 
 	cloned.ImageDownloads[index].ImageState = voltha.ImageDownload_IMAGE_REVERTING
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
+	go func() {
+		defer cancel()
+		_, err := client.RevertImageUpdate(subCtx, &ic.ImageDownloadMessage{
+			Device: cloned,
+			Image:  img,
+		})
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, true)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, true)
+		}
+	}()
 
+	// Save data
 	if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
 		return nil, err
 	}
 
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, err := agent.adapterProxy.RevertImageUpdate(subCtx, cloned, img)
-	if err != nil {
-		cancel()
-		return nil, err
-	}
-	go agent.waitForAdapterResponse(subCtx, cancel, "revertImageUpdate", ch, agent.onSuccess, agent.onFailure)
-
 	return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
 }
 
@@ -244,40 +358,49 @@
 	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return nil, err
 	}
+	defer agent.requestQueue.RequestComplete()
+
 	device := agent.getDeviceReadOnlyWithoutLock()
-	ch, err := agent.adapterProxy.GetImageDownloadStatus(ctx, device, img)
-	agent.requestQueue.RequestComplete()
+	if !agent.proceedWithRequest(device) {
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+	}
+
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
 		return nil, err
 	}
-	// Wait for the adapter response
-	rpcResponse, ok := <-ch
-	if !ok {
-		return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
-	}
-	if rpcResponse.Err != nil {
-		return nil, rpcResponse.Err
-	}
-	// Successful response
-	imgDownload := &voltha.ImageDownload{}
-	if err := ptypes.UnmarshalAny(rpcResponse.Reply, imgDownload); err != nil {
-		return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
-	}
-	return imgDownload, nil
+	return client.GetImageDownloadStatus(ctx, &ic.ImageDownloadMessage{
+		Device: device,
+		Image:  img,
+	})
 }
 
 func (agent *Agent) updateImageDownload(ctx context.Context, img *voltha.ImageDownload) error {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+	var err error
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return err
 	}
+
 	logger.Debugw(ctx, "updating-image-download", log.Fields{"device-id": agent.deviceID, "img": img})
 
 	cloned := agent.cloneDeviceWithoutLock()
 
 	if !agent.proceedWithRequest(cloned) {
 		agent.requestQueue.RequestComplete()
-		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed.",
-			agent.deviceID)
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+		return err
 	}
 
 	// Update the image as well as remove it if the download was cancelled
@@ -326,14 +449,25 @@
 }
 
 // onImageFailure brings back the device to Enabled state and sets the image to image download_failed.
-func (agent *Agent) onImageFailure(ctx context.Context, rpc string, response interface{}, reqArgs ...interface{}) {
+func (agent *Agent) onImageFailure(ctx context.Context, imgErr error) {
 	// original context has failed due to timeout , let's open a new one
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.internalTimeout)
 	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 	defer cancel()
+	rpc := coreutils.GetRPCMetadataFromContext(subCtx)
+
+	defer func() {
+		eCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
+		rpce := agent.deviceMgr.NewRPCEvent(eCtx, agent.deviceID, imgErr.Error(), nil)
+		go agent.deviceMgr.SendRPCEvent(eCtx, "RPC_ERROR_RAISE_EVENT", rpce,
+			voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
+		operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+		desc := "adapter-response"
+		agent.logDeviceUpdate(ctx, nil, nil, operStatus, imgErr, desc)
+	}()
 
 	if err := agent.requestQueue.WaitForGreenLight(subCtx); err != nil {
-		logger.Errorw(subCtx, "can't obtain lock", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": err, "args": reqArgs})
+		logger.Errorw(subCtx, "can't obtain lock", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": err})
 		return
 	}
 
@@ -345,8 +479,8 @@
 			log.Fields{"rpc": rpc, "device-id": agent.deviceID})
 		return
 	}
-	if res, ok := response.(error); ok {
-		logger.Errorw(subCtx, "rpc-failed", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": res, "args": reqArgs})
+	if imgErr != nil {
+		logger.Errorw(subCtx, "rpc-failed", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": imgErr})
 		cloned := agent.cloneDeviceWithoutLock()
 		//TODO base this on IMAGE ID when created
 		var imageFailed *voltha.ImageDownload
@@ -362,7 +496,7 @@
 		}
 
 		if imageFailed == nil {
-			logger.Errorw(subCtx, "can't find image", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
+			logger.Errorw(subCtx, "can't find image", log.Fields{"rpc": rpc, "device-id": agent.deviceID})
 			return
 		}
 
@@ -376,19 +510,24 @@
 		cloned.AdminState = voltha.AdminState_ENABLED
 		if err := agent.updateDeviceAndReleaseLock(subCtx, cloned); err != nil {
 			logger.Errorw(subCtx, "failed-enable-device-after-image-failure",
-				log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": res, "args": reqArgs})
+				log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": err})
 		}
 	} else {
-		logger.Errorw(subCtx, "rpc-failed-invalid-error", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
+		logger.Errorw(subCtx, "rpc-failed-invalid-error", log.Fields{"rpc": rpc, "device-id": agent.deviceID})
 		return
 	}
-	// TODO: Post failure message onto kafka
 }
 
 // onImageSuccess brings back the device to Enabled state and sets the image to image download_failed.
-func (agent *Agent) onImageSuccess(ctx context.Context, rpc string, response interface{}, reqArgs ...interface{}) {
-	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-		logger.Errorw(ctx, "cannot-obtain-lock", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": err, "args": reqArgs})
+func (agent *Agent) onImageSuccess(ctx context.Context, response interface{}) {
+	rpc := coreutils.GetRPCMetadataFromContext(ctx)
+
+	var err error
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
+
+	if err = agent.requestQueue.WaitForGreenLight(ctx); err != nil {
 		return
 	}
 
@@ -396,12 +535,10 @@
 
 	if !agent.proceedWithRequest(cloned) {
 		agent.requestQueue.RequestComplete()
-		logger.Errorw(ctx, "Cannot complete operation as Device deletion is in progress or reconciling is in progress/failed.",
-			log.Fields{"rpc": rpc, "device-id": agent.deviceID})
+		err = status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
 		return
 	}
-	logger.Infow(ctx, "rpc-successful", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "response": response, "args": reqArgs})
-
+	logger.Infow(ctx, "rpc-successful", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "response": response})
 	//TODO base this on IMAGE ID when created
 	var imageSucceeded *voltha.ImageDownload
 	var index int
@@ -416,7 +553,7 @@
 	}
 
 	if imageSucceeded == nil {
-		logger.Errorw(ctx, "can't find image", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
+		err = errors.New("can't find image")
 		return
 	}
 	//update image state on success
@@ -427,11 +564,14 @@
 	}
 	//Enabled is the only state we can go back to.
 	cloned.AdminState = voltha.AdminState_ENABLED
-	if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
+	if err = agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
 		logger.Errorw(ctx, "failed-enable-device-after-image-download-success",
-			log.Fields{"rpc": rpc, "device-id": agent.deviceID, "response": response, "args": reqArgs})
+			log.Fields{"rpc": rpc, "device-id": agent.deviceID, "response": response})
 	}
-
+	// Update operation status
+	if err == nil {
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
+	}
 }
 
 func (agent *Agent) downloadImageToDevice(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
@@ -447,19 +587,30 @@
 	}
 
 	cloned := agent.cloneDeviceWithoutLock()
+	if !agent.proceedWithRequest(cloned) {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+	}
 
 	// Send the request to the adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
 	defer cancel()
 	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 
-	ch, err := agent.adapterProxy.DownloadImageToOnuDevice(subCtx, cloned, request)
 	agent.requestQueue.RequestComplete()
-	if err != nil {
-		return nil, err
-	}
-
-	return agent.getDeviceImageResponseFromAdapter(ctx, ch)
+	return client.DownloadOnuImage(subCtx, request)
 }
 
 func (agent *Agent) getImageStatus(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
@@ -467,21 +618,33 @@
 		return nil, err
 	}
 
-	cloned := agent.cloneDeviceWithoutLock()
-
-	// Send the request to the adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	defer cancel()
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 	logger.Debugw(ctx, "get-image-status", log.Fields{"device-id": agent.deviceID})
 
-	ch, err := agent.adapterProxy.GetOnuImageStatus(subCtx, cloned, request)
-	agent.requestQueue.RequestComplete()
+	cloned := agent.cloneDeviceWithoutLock()
+	if !agent.proceedWithRequest(cloned) {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+	}
+
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
 
-	return agent.getDeviceImageResponseFromAdapter(subCtx, ch)
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	defer cancel()
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+
+	return client.GetOnuImageStatus(subCtx, request)
 }
 
 func (agent *Agent) activateImageOnDevice(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
@@ -489,21 +652,35 @@
 		return nil, err
 	}
 
-	cloned := agent.cloneDeviceWithoutLock()
-
-	// Send the request to the adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	defer cancel()
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 	logger.Debugw(ctx, "activate-image-on-device", log.Fields{"device-id": agent.deviceID})
 
-	ch, err := agent.adapterProxy.ActivateOnuImage(subCtx, cloned, request)
-	agent.requestQueue.RequestComplete()
+	cloned := agent.cloneDeviceWithoutLock()
+
+	if !agent.proceedWithRequest(cloned) {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+	}
+
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
 
-	return agent.getDeviceImageResponseFromAdapter(subCtx, ch)
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	defer cancel()
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+
+	agent.requestQueue.RequestComplete()
+	return client.ActivateOnuImage(subCtx, request)
 }
 
 func (agent *Agent) abortImageUpgradeToDevice(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
@@ -511,21 +688,35 @@
 		return nil, err
 	}
 
-	cloned := agent.cloneDeviceWithoutLock()
-
-	// Send the request to the adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	defer cancel()
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 	logger.Debugw(ctx, "abort-image-on-device", log.Fields{"device-id": agent.deviceID})
 
-	ch, err := agent.adapterProxy.AbortImageUpgrade(subCtx, cloned, request)
-	agent.requestQueue.RequestComplete()
-	if err != nil {
-		return nil, err
+	cloned := agent.cloneDeviceWithoutLock()
+
+	if !agent.proceedWithRequest(cloned) {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
 	}
 
-	return agent.getDeviceImageResponseFromAdapter(ctx, ch)
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	defer cancel()
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+
+	agent.requestQueue.RequestComplete()
+
+	return client.AbortOnuImageUpgrade(subCtx, request)
 }
 
 func (agent *Agent) commitImage(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
@@ -533,21 +724,33 @@
 		return nil, err
 	}
 
-	cloned := agent.cloneDeviceWithoutLock()
-
-	// Send the request to the adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	defer cancel()
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 	logger.Debugw(ctx, "commit-image-on-device", log.Fields{"device-id": agent.deviceID})
 
-	ch, err := agent.adapterProxy.CommitImage(subCtx, cloned, request)
-	agent.requestQueue.RequestComplete()
-	if err != nil {
-		return nil, err
+	cloned := agent.cloneDeviceWithoutLock()
+
+	if !agent.proceedWithRequest(cloned) {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
 	}
 
-	return agent.getDeviceImageResponseFromAdapter(ctx, ch)
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.rpcTimeout)
+	defer cancel()
+	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+
+	return client.CommitOnuImage(subCtx, request)
 }
 
 func (agent *Agent) getOnuImages(ctx context.Context, id *common.ID) (*voltha.OnuImages, error) {
@@ -555,63 +758,29 @@
 		return nil, err
 	}
 
-	cloned := agent.cloneDeviceWithoutLock()
-
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	defer cancel()
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 	logger.Debug(ctx, "get-onu-images")
 
+	cloned := agent.cloneDeviceWithoutLock()
+
+	if !agent.proceedWithRequest(cloned) {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", "cannot complete operation as device deletion is in progress or reconciling is in progress/failed")
+	}
+
 	// Send the request to the adapter
-	ch, err := agent.adapterProxy.GetOnuImages(subCtx, cloned, id)
-	agent.requestQueue.RequestComplete()
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": cloned.AdapterEndpoint,
+			})
+		agent.requestQueue.RequestComplete()
 		return nil, err
 	}
 
-	//wait for adapter response
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
-		} else if rpcResponse.Err != nil {
-			// return error
-			return nil, status.Errorf(codes.Internal, "%s", rpcResponse.Err.Error())
-		} else {
-			resp := &voltha.OnuImages{}
-			if err := ptypes.UnmarshalAny(rpcResponse.Reply, resp); err != nil {
-				return nil, status.Errorf(codes.Internal, "%s", err.Error())
-			}
-
-			return resp, nil
-		}
-	case <-ctx.Done():
-		return nil, ctx.Err()
-	}
-}
-
-func (agent *Agent) getDeviceImageResponseFromAdapter(ctx context.Context, ch chan *kafka.RpcResponse) (*voltha.DeviceImageResponse, error) {
-	//wait for adapter response
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
-		} else if rpcResponse.Err != nil {
-			// return error
-			return nil, status.Errorf(codes.Internal, "%s", rpcResponse.Err.Error())
-		} else {
-			resp := &voltha.DeviceImageResponse{}
-			if err := ptypes.UnmarshalAny(rpcResponse.Reply, resp); err != nil {
-				return nil, status.Errorf(codes.Internal, "%s", err.Error())
-			}
-
-			if len(resp.DeviceImageStates) == 0 || resp.DeviceImageStates[0] == nil {
-				return nil, status.Errorf(codes.Internal, "invalid response from adapter")
-			}
-
-			return resp, nil
-		}
-	case <-ctx.Done():
-		return nil, ctx.Err()
-	}
+	agent.requestQueue.RequestComplete()
+	return client.GetOnuImages(ctx, id)
 }
diff --git a/rw_core/core/device/agent_pm_config.go b/rw_core/core/device/agent_pm_config.go
index 176a126..ba1005a 100644
--- a/rw_core/core/device/agent_pm_config.go
+++ b/rw_core/core/device/agent_pm_config.go
@@ -18,79 +18,73 @@
 
 import (
 	"context"
-	"fmt"
+	"time"
+
 	"github.com/gogo/protobuf/proto"
-	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
-	"time"
 )
 
 func (agent *Agent) updatePmConfigs(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
 	logger.Debugw(ctx, "update-pm-configs", log.Fields{"device-id": pmConfigs.Id})
 
-	cloned := agent.cloneDeviceWithoutLock()
+	var rpce *voltha.RPCEvent
+	defer func() {
+		if rpce != nil {
+			go agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
+				voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
+		}
+	}()
+
+	cloned, err := agent.getDeviceReadOnly(ctx)
+	if err != nil {
+		return err
+	}
 
 	if !agent.proceedWithRequest(cloned) {
 		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, Cannot complete operation as device deletion is in progress or reconciling is in progress/failed", cloned.Id)
 	}
 
-	cloned.PmConfigs = proto.Clone(pmConfigs).(*voltha.PmConfigs)
+	// We need to send the response for the PM Config Updates in a synchronous manner to the adapter.
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
+	if err != nil {
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": agent.adapterEndpoint,
+			})
+		return err
+	}
+	_, pmErr := client.UpdatePmConfig(ctx, &ic.PmConfigsInfo{
+		DeviceId:  agent.deviceID,
+		PmConfigs: pmConfigs,
+	})
 
-	// Send the request to the adapter
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	defer cancel()
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
-
-	ch, pmErr := agent.adapterProxy.UpdatePmConfigs(subCtx, cloned, pmConfigs)
 	if pmErr != nil {
+		rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, pmErr.Error(), nil)
 		return pmErr
 	}
-
-	var rpce *voltha.RPCEvent
-	defer func() {
-		if rpce != nil {
-			agent.deviceMgr.SendRPCEvent(ctx, "RPC_ERROR_RAISE_EVENT", rpce,
-				voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
-		}
-	}()
-	// We need to send the response for the PM Config Updates in a synchronous manner to the caller.
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			pmErr = fmt.Errorf("response-channel-closed")
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, pmErr.Error(), nil)
-		} else if rpcResponse.Err != nil {
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, rpcResponse.Err.Error(), nil)
-			pmErr = rpcResponse.Err
-		}
-	case <-ctx.Done():
-		rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, ctx.Err().Error(), nil)
-		pmErr = ctx.Err()
-	}
-
 	// In case of no error for PM Config update, commit the new PM Config to DB.
-	if pmErr == nil {
-		// acquire lock for update the device to DB
-		if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
-			return err
-		}
-		// the Device properties might have changed due to other concurrent transactions on the device, so get latest copy
-		cloned = agent.cloneDeviceWithoutLock()
-		// commit new pm config
-		cloned.PmConfigs = proto.Clone(pmConfigs).(*voltha.PmConfigs)
-
-		// Store back the device to DB and release lock
-		if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
-			logger.Errorw(ctx, "error-updating-device-context-to-db", log.Fields{"device-id": agent.deviceID})
-			rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
-			return err
-		}
+	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		return err
 	}
+	// the Device properties might have changed due to other concurrent transactions on the device, so get latest copy
+	cloned = agent.cloneDeviceWithoutLock()
+	// commit new pm config
+	cloned.PmConfigs = proto.Clone(pmConfigs).(*voltha.PmConfigs)
 
-	return pmErr
+	// Store back the device to DB and release lock
+	if err := agent.updateDeviceAndReleaseLock(ctx, cloned); err != nil {
+		logger.Errorw(ctx, "error-updating-device-context-to-db", log.Fields{"device-id": agent.deviceID})
+		rpce = agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
+		return err
+	}
+	return nil
 }
 
 func (agent *Agent) initPmConfigs(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
diff --git a/rw_core/core/device/agent_port.go b/rw_core/core/device/agent_port.go
index f9bf7de..e7bb371 100644
--- a/rw_core/core/device/agent_port.go
+++ b/rw_core/core/device/agent_port.go
@@ -19,13 +19,14 @@
 import (
 	"context"
 	"fmt"
-	"github.com/opencord/voltha-protos/v4/go/common"
+
+	"github.com/opencord/voltha-protos/v5/go/common"
 
 	"github.com/gogo/protobuf/proto"
 	"github.com/opencord/voltha-go/rw_core/core/device/port"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -149,9 +150,10 @@
 func (agent *Agent) addPort(ctx context.Context, port *voltha.Port) error {
 	logger.Debugw(ctx, "addPort", log.Fields{"device-id": agent.deviceID})
 	var desc string
+	var err error
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
 
-	defer agent.logDeviceUpdate(ctx, "addPort", nil, nil, operStatus, &desc)
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	port.AdminState = voltha.AdminState_ENABLED
 
@@ -171,6 +173,7 @@
 	if oldPort.Label != "" || oldPort.Type != voltha.Port_PON_OLT {
 		logger.Debugw(ctx, "port-already-exists", log.Fields{"port": port})
 		desc = fmt.Sprintf("port already exists, port : %s", port)
+		operStatus.Code = common.OperationResp_OPERATION_SUCCESS
 		return nil
 	}
 
@@ -237,97 +240,118 @@
 func (agent *Agent) disablePort(ctx context.Context, portID uint32) error {
 	logger.Debugw(ctx, "disable-port", log.Fields{"device-id": agent.deviceID, "port-no": portID})
 
+	var err error
 	var desc string
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-
-	defer agent.logDeviceUpdate(ctx, "disablePort", nil, nil, operStatus, &desc)
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	portHandle, have := agent.portLoader.Lock(portID)
 	if !have {
-		desc = fmt.Sprintf("Invalid argument portID: %v", portID)
-		return status.Errorf(codes.InvalidArgument, "%v", portID)
+		err = status.Errorf(codes.InvalidArgument, "%v", portID)
+		return err
 	}
 	defer portHandle.Unlock()
 
 	oldPort := portHandle.GetReadOnly()
 
 	if oldPort.Type != voltha.Port_PON_OLT {
-		desc = fmt.Sprintf("Disabling of Port Type %v unimplemented", oldPort.Type)
-		return status.Errorf(codes.InvalidArgument, "Disabling of Port Type %v unimplemented", oldPort.Type)
+		err = status.Errorf(codes.Unimplemented, "disabling of Port Type %v unimplemented", oldPort.Type)
+		return err
 	}
 
 	newPort := *oldPort
 	newPort.AdminState = voltha.AdminState_DISABLED
 	if err := portHandle.Update(ctx, &newPort); err != nil {
-		desc = err.Error()
 		return err
 	}
 
 	//send request to adapter
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		desc = err.Error()
 		return err
 	}
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 
-	ch, err := agent.adapterProxy.DisablePort(ctx, device, &newPort)
+	// Send the request to the adapter
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		desc = err.Error()
-		cancel()
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
 		return err
 	}
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-	go agent.waitForAdapterResponseAndLogDeviceUpdate(subCtx, cancel, "disablePort", ch, agent.onSuccess, agent.onFailure, nil)
+	go func() {
+		defer cancel()
+		_, err := client.DisablePort(subCtx, &newPort)
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, true)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, true)
+		}
+	}()
 	return nil
 }
 
 func (agent *Agent) enablePort(ctx context.Context, portID uint32) error {
 	logger.Debugw(ctx, "enable-port", log.Fields{"device-id": agent.deviceID, "port-no": portID})
 
+	var err error
 	var desc string
 	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
-
-	defer agent.logDeviceUpdate(ctx, "enablePort", nil, nil, operStatus, &desc)
+	defer func() { agent.logDeviceUpdate(ctx, nil, nil, operStatus, err, desc) }()
 
 	portHandle, have := agent.portLoader.Lock(portID)
 	if !have {
-		desc = fmt.Sprintf("Invalid Argument portID: %v", portID)
-		return status.Errorf(codes.InvalidArgument, "%v", portID)
+		err = status.Errorf(codes.InvalidArgument, "%v", portID)
+		return err
 	}
 	defer portHandle.Unlock()
 
 	oldPort := portHandle.GetReadOnly()
 
 	if oldPort.Type != voltha.Port_PON_OLT {
-		desc = fmt.Sprintf("Enabling of Port Type %v unimplemented", oldPort.Type)
-		return status.Errorf(codes.InvalidArgument, "Enabling of Port Type %v unimplemented", oldPort.Type)
+		err = status.Errorf(codes.Unimplemented, "enabling of Port Type %v unimplemented", oldPort.Type)
+		return err
 	}
 
 	newPort := *oldPort
 	newPort.AdminState = voltha.AdminState_ENABLED
-	if err := portHandle.Update(ctx, &newPort); err != nil {
-		desc = err.Error()
+	if err = portHandle.Update(ctx, &newPort); err != nil {
 		return err
 	}
 
 	//send request to adapter
 	device, err := agent.getDeviceReadOnly(ctx)
 	if err != nil {
-		desc = err.Error()
 		return err
 	}
-	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
-	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 
-	ch, err := agent.adapterProxy.EnablePort(ctx, device, &newPort)
+	client, err := agent.adapterMgr.GetAdapterClient(ctx, agent.adapterEndpoint)
 	if err != nil {
-		desc = err.Error()
-		cancel()
+		logger.Errorw(ctx, "grpc-client-nil",
+			log.Fields{
+				"error":            err,
+				"device-id":        agent.deviceID,
+				"device-type":      agent.deviceType,
+				"adapter-endpoint": device.AdapterEndpoint,
+			})
 		return err
 	}
+	subCtx, cancel := context.WithTimeout(coreutils.WithAllMetadataFromContext(ctx), agent.rpcTimeout)
 	operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-	go agent.waitForAdapterResponseAndLogDeviceUpdate(subCtx, cancel, "enablePort", ch, agent.onSuccess, agent.onFailure, nil)
+	go func() {
+		defer cancel()
+		_, err := client.EnablePort(subCtx, &newPort)
+		if err == nil {
+			agent.onSuccess(subCtx, nil, nil, true)
+		} else {
+			agent.onFailure(subCtx, err, nil, nil, true)
+		}
+	}()
 	return nil
 }
diff --git a/rw_core/core/device/agent_test.go b/rw_core/core/device/agent_test.go
index e3716a7..0577c1e 100755
--- a/rw_core/core/device/agent_test.go
+++ b/rw_core/core/device/agent_test.go
@@ -18,7 +18,6 @@
 
 import (
 	"context"
-	"fmt"
 	"math/rand"
 	"sort"
 	"strconv"
@@ -27,21 +26,23 @@
 	"testing"
 	"time"
 
+	ver "github.com/opencord/voltha-lib-go/v7/pkg/version"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+
 	"github.com/gogo/protobuf/proto"
 	"github.com/opencord/voltha-go/db/model"
 	"github.com/opencord/voltha-go/rw_core/config"
 	"github.com/opencord/voltha-go/rw_core/core/adapter"
-	cm "github.com/opencord/voltha-go/rw_core/mocks"
 	tst "github.com/opencord/voltha-go/rw_core/test"
-	com "github.com/opencord/voltha-lib-go/v5/pkg/adapters/common"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	mock_etcd "github.com/opencord/voltha-lib-go/v5/pkg/mocks/etcd"
-	mock_kafka "github.com/opencord/voltha-lib-go/v5/pkg/mocks/kafka"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	com "github.com/opencord/voltha-lib-go/v7/pkg/adapters/common"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events"
+	"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	mock_etcd "github.com/opencord/voltha-lib-go/v7/pkg/mocks/etcd"
+	mock_kafka "github.com/opencord/voltha-lib-go/v7/pkg/mocks/kafka"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"github.com/phayes/freeport"
 	"github.com/stretchr/testify/assert"
 )
@@ -51,17 +52,13 @@
 	deviceMgr        *Manager
 	logicalDeviceMgr *LogicalManager
 	adapterMgr       *adapter.Manager
-	kmp              kafka.InterContainerProxy
 	kClient          kafka.Client
 	kEventClient     kafka.Client
 	kvClientPort     int
-	oltAdapter       *cm.OLTAdapter
-	onuAdapter       *cm.ONUAdapter
 	oltAdapterName   string
 	onuAdapterName   string
 	coreInstanceID   string
-	defaultTimeout   time.Duration
-	maxTimeout       time.Duration
+	internalTimeout  time.Duration
 	device           *voltha.Device
 	devicePorts      map[uint32]*voltha.Port
 	done             chan int
@@ -78,24 +75,22 @@
 	// Create the kafka client
 	test.kClient = mock_kafka.NewKafkaClient()
 	test.kEventClient = mock_kafka.NewKafkaClient()
-	test.oltAdapterName = "olt_adapter_mock"
-	test.onuAdapterName = "onu_adapter_mock"
+	test.oltAdapterName = "olt-mock-adapter"
+	test.onuAdapterName = "onu-mock-adapter"
 	test.coreInstanceID = "rw-da-test"
-	test.defaultTimeout = 5 * time.Second
-	test.maxTimeout = 20 * time.Second
+	test.internalTimeout = 5 * time.Second
 	test.done = make(chan int)
 	parentID := com.GetRandomString(10)
 	test.device = &voltha.Device{
-		Type:         "onu_adapter_mock",
+		Type:         "onu-mock-device-type",
 		ParentId:     parentID,
 		ParentPortNo: 1,
-		VendorId:     "onu_adapter_mock",
-		Adapter:      "onu_adapter_mock",
+		VendorId:     "onu-mock-vendor",
 		Vlan:         100,
 		Address:      nil,
 		ProxyAddress: &voltha.Device_ProxyAddress{
 			DeviceId:           parentID,
-			DeviceType:         "olt_adapter_mock",
+			DeviceType:         "olt-mock-device-type",
 			ChannelId:          100,
 			ChannelGroupId:     0,
 			ChannelTermination: "",
@@ -119,15 +114,14 @@
 func (dat *DATest) startCore(ctx context.Context) {
 	cfg := &config.RWCoreFlags{}
 	cfg.ParseCommandArguments([]string{})
-	cfg.CoreTopic = "rw_core"
 	cfg.EventTopic = "voltha.events"
-	cfg.DefaultRequestTimeout = dat.defaultTimeout
+	cfg.InternalTimeout = dat.internalTimeout
 	cfg.KVStoreAddress = "127.0.0.1" + ":" + strconv.Itoa(dat.kvClientPort)
 	grpcPort, err := freeport.GetFreePort()
 	if err != nil {
 		logger.Fatal(ctx, "Cannot get a freeport for grpc")
 	}
-	cfg.GrpcAddress = "127.0.0.1" + ":" + strconv.Itoa(grpcPort)
+	cfg.GrpcNBIAddress = "127.0.0.1" + ":" + strconv.Itoa(grpcPort)
 	client := tst.SetupKVClient(ctx, cfg, dat.coreInstanceID)
 	backend := &db.Backend{
 		Client:                  client,
@@ -135,34 +129,20 @@
 		Address:                 cfg.KVStoreAddress,
 		Timeout:                 cfg.KVStoreTimeout,
 		LivenessChannelInterval: cfg.LiveProbeInterval / 2}
-	dat.kmp = kafka.NewInterContainerProxy(
-		kafka.InterContainerAddress(cfg.KafkaAdapterAddress),
-		kafka.MsgClient(dat.kClient),
-		kafka.DefaultTopic(&kafka.Topic{Name: cfg.CoreTopic}))
 
-	endpointMgr := kafka.NewEndpointManager(backend)
 	proxy := model.NewDBPath(backend)
-	dat.adapterMgr = adapter.NewAdapterManager(ctx, proxy, dat.coreInstanceID, dat.kClient)
+	dat.adapterMgr = adapter.NewAdapterManager(proxy, dat.coreInstanceID, backend, 5)
 	eventProxy := events.NewEventProxy(events.MsgClient(dat.kEventClient), events.MsgTopic(kafka.Topic{Name: cfg.EventTopic}))
-	dat.deviceMgr, dat.logicalDeviceMgr = NewManagers(proxy, dat.adapterMgr, dat.kmp, endpointMgr, cfg, dat.coreInstanceID, eventProxy)
-	dat.adapterMgr.Start(context.Background())
-	if err = dat.kmp.Start(ctx); err != nil {
-		logger.Fatal(ctx, "Cannot start InterContainerProxy")
-	}
-
-	if err := dat.kmp.SubscribeWithDefaultRequestHandler(ctx, kafka.Topic{Name: cfg.CoreTopic}, kafka.OffsetNewest); err != nil {
-		logger.Fatalf(ctx, "Cannot add default request handler: %s", err)
-	}
-
+	dat.deviceMgr, dat.logicalDeviceMgr = NewManagers(proxy, dat.adapterMgr, cfg, dat.coreInstanceID, eventProxy)
+	dat.adapterMgr.Start(context.Background(), "agent-test")
+	dat.registerAdapters(context.Background())
+	log.SetAllLogLevel(log.FatalLevel)
 }
 
 func (dat *DATest) stopAll(ctx context.Context) {
 	if dat.kClient != nil {
 		dat.kClient.Stop(ctx)
 	}
-	if dat.kmp != nil {
-		dat.kmp.Stop(ctx)
-	}
 	if dat.etcdServer != nil {
 		tst.StopEmbeddedEtcdServer(ctx, dat.etcdServer)
 	}
@@ -174,7 +154,7 @@
 func (dat *DATest) createDeviceAgent(t *testing.T) *Agent {
 	deviceMgr := dat.deviceMgr
 	clonedDevice := proto.Clone(dat.device).(*voltha.Device)
-	deviceAgent := newAgent(deviceMgr.adapterProxy, clonedDevice, deviceMgr, deviceMgr.dbPath, deviceMgr.dProxy, deviceMgr.defaultTimeout)
+	deviceAgent := newAgent(clonedDevice, deviceMgr, deviceMgr.dbPath, deviceMgr.dProxy, deviceMgr.internalTimeout, deviceMgr.rpcTimeout)
 	d, err := deviceAgent.start(context.TODO(), false, clonedDevice)
 	assert.Nil(t, err)
 	assert.NotNil(t, d)
@@ -215,19 +195,13 @@
 		deviceToUpdate.MacAddress = macAddress
 		deviceToUpdate.Vlan = vlan
 		deviceToUpdate.Reason = reason
+		deviceToUpdate.OperStatus = voltha.OperStatus_ACTIVE
+		deviceToUpdate.ConnectStatus = voltha.ConnectStatus_REACHABLE
 		err := da.updateDeviceUsingAdapterData(context.Background(), deviceToUpdate)
 		assert.Nil(t, err)
 		localWG.Done()
 	}()
 
-	// Update the device status routine
-	localWG.Add(1)
-	go func() {
-		err := da.updateDeviceStatus(context.Background(), voltha.OperStatus_ACTIVE, voltha.ConnectStatus_REACHABLE)
-		assert.Nil(t, err)
-		localWG.Done()
-	}()
-
 	// Add a port routine
 	localWG.Add(1)
 	go func() {
@@ -253,8 +227,6 @@
 	updatedDevice, _ := da.getDeviceReadOnly(context.Background())
 	updatedDevicePorts := da.listDevicePorts()
 	assert.NotNil(t, updatedDevice)
-	fmt.Printf("1 %+v\n", expectedChange)
-	fmt.Printf("2 %+v\n", updatedDevice)
 	assert.True(t, proto.Equal(expectedChange, updatedDevice))
 	assert.Equal(t, len(originalDevicePorts)+1, len(updatedDevicePorts))
 	assert.True(t, proto.Equal(updatedDevicePorts[portToAdd.PortNo], portToAdd))
@@ -268,7 +240,7 @@
 		da := newDATest(ctx)
 		assert.NotNil(t, da)
 		defer da.stopAll(ctx)
-		log.SetPackageLogLevel("github.com/opencord/voltha-go/rw_core/core", log.DebugLevel)
+
 		// Start the Core
 		da.startCore(ctx)
 
@@ -279,7 +251,6 @@
 			a := da.createDeviceAgent(t)
 			go da.updateDeviceConcurrently(t, a, &wg)
 		}
-
 		wg.Wait()
 	}
 }
@@ -292,8 +263,6 @@
 	log.SetPackageLogLevel("github.com/opencord/voltha-go/rw_core/core", log.DebugLevel)
 	// Start the Core
 	da.startCore(ctx)
-	da.oltAdapter, da.onuAdapter = tst.CreateAndregisterAdapters(ctx, t, da.kClient, da.coreInstanceID, da.oltAdapterName, da.onuAdapterName, da.adapterMgr)
-
 	a := da.createDeviceAgent(t)
 	err1 := a.requestQueue.WaitForGreenLight(ctx)
 	assert.Nil(t, err1)
@@ -312,7 +281,6 @@
 
 	// Start the Core
 	da.startCore(ctx)
-	da.oltAdapter, da.onuAdapter = tst.CreateAndregisterAdapters(ctx, t, da.kClient, da.coreInstanceID, da.oltAdapterName, da.onuAdapterName, da.adapterMgr)
 	a := da.createDeviceAgent(t)
 	err1 := a.requestQueue.WaitForGreenLight(ctx)
 	assert.Nil(t, err1)
@@ -380,7 +348,9 @@
 		{Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
 	}
 	err := da.addFlowsAndGroups(context.Background(), newFlows, []*ofp.OfpGroupEntry{}, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	// Expect specific error as adapter communication, for unit tests, are not set
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "flow-failure-device-"))
 	daFlows := changeToFlowList(da.listDeviceFlows())
 	assert.True(t, isFlowSliceEqual(newFlows, daFlows))
 
@@ -399,7 +369,8 @@
 	}
 
 	err = da.addFlowsAndGroups(context.Background(), newFlows, []*ofp.OfpGroupEntry{}, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "flow-failure-device-"))
 	daFlows = changeToFlowList(da.listDeviceFlows())
 	assert.True(t, isFlowSliceEqual(expectedFlows, daFlows))
 
@@ -420,7 +391,8 @@
 	}
 
 	err = da.addFlowsAndGroups(context.Background(), newFlows, []*ofp.OfpGroupEntry{}, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daFlows = changeToFlowList(da.listDeviceFlows())
 	assert.True(t, isFlowSliceEqual(expectedFlows, daFlows))
 
@@ -459,7 +431,8 @@
 	}
 
 	err = da.deleteFlowsAndGroups(context.Background(), flowsToDelete, []*ofp.OfpGroupEntry{}, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daFlows = changeToFlowList(da.listDeviceFlows())
 	assert.True(t, isFlowSliceEqual(expectedFlows, daFlows))
 	//Delete flows with an unexisting one
@@ -474,7 +447,8 @@
 	}
 
 	err = da.deleteFlowsAndGroups(context.Background(), flowsToDelete, []*ofp.OfpGroupEntry{}, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daFlows = changeToFlowList(da.listDeviceFlows())
 	assert.True(t, isFlowSliceEqual(expectedFlows, daFlows))
 }
@@ -486,7 +460,8 @@
 		{Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
 	}
 	err := da.addFlowsAndGroups(context.Background(), []*ofp.OfpFlowStats{}, newGroups, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "flow-failure-device-"))
 	daGroups := changeToGroupList(da.listDeviceGroups())
 	assert.True(t, isGroupSliceEqual(newGroups, daGroups))
 
@@ -502,7 +477,8 @@
 		{Desc: &ofp.OfpGroupDesc{Type: 4, GroupId: 40, Buckets: nil}},
 	}
 	err = da.addFlowsAndGroups(context.Background(), []*ofp.OfpFlowStats{}, newGroups, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daGroups = changeToGroupList(da.listDeviceGroups())
 	assert.True(t, isGroupSliceEqual(expectedGroups, daGroups))
 
@@ -520,7 +496,8 @@
 		{Desc: &ofp.OfpGroupDesc{Type: 5, GroupId: 50, Buckets: nil}},
 	}
 	err = da.addFlowsAndGroups(context.Background(), []*ofp.OfpFlowStats{}, newGroups, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daGroups = changeToGroupList(da.listDeviceGroups())
 	assert.True(t, isGroupSliceEqual(expectedGroups, daGroups))
 
@@ -536,7 +513,8 @@
 		{Desc: &ofp.OfpGroupDesc{Type: 5, GroupId: 50, Buckets: nil}},
 	}
 	err = da.updateFlowsAndGroups(context.Background(), []*ofp.OfpFlowStats{}, updtGroups, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daGroups = changeToGroupList(da.listDeviceGroups())
 	assert.True(t, isGroupSliceEqual(expectedGroups, daGroups))
 
@@ -551,7 +529,8 @@
 		{Desc: &ofp.OfpGroupDesc{Type: 5, GroupId: 50, Buckets: nil}},
 	}
 	err = da.deleteFlowsAndGroups(context.Background(), []*ofp.OfpFlowStats{}, delGroups, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daGroups = changeToGroupList(da.listDeviceGroups())
 	assert.True(t, isGroupSliceEqual(expectedGroups, daGroups))
 
@@ -565,7 +544,49 @@
 		{Desc: &ofp.OfpGroupDesc{Type: 5, GroupId: 50, Buckets: nil}},
 	}
 	err = da.deleteFlowsAndGroups(context.Background(), []*ofp.OfpFlowStats{}, delGroups, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
+	assert.NotNil(t, err)
+	assert.True(t, strings.Contains(err.Error(), "Aborted"))
 	daGroups = changeToGroupList(da.listDeviceGroups())
 	assert.True(t, isGroupSliceEqual(expectedGroups, daGroups))
 }
+
+// registerAdapters registers the ONU and OLT adapters
+func (dat *DATest) registerAdapters(ctx context.Context) {
+	oltAdapter := &voltha.Adapter{
+		Id:             "olt-mock-adapter-1",
+		Vendor:         "olt-mock-vendor",
+		Version:        ver.VersionInfo.Version,
+		Type:           "olt-mock-adapter-type",
+		CurrentReplica: 1,
+		TotalReplicas:  1,
+		Endpoint:       "mock-olt-endpoint",
+	}
+	types := []*voltha.DeviceType{{Id: "olt-mock-device-type", AdapterType: "olt-mock-adapter-type", AcceptsAddRemoveFlowUpdates: true}}
+	deviceTypes := &voltha.DeviceTypes{Items: types}
+	_, err := dat.adapterMgr.RegisterAdapter(ctx, &ic.AdapterRegistration{
+		Adapter: oltAdapter,
+		DTypes:  deviceTypes,
+	})
+	if err != nil {
+		logger.Fatalw(ctx, "olt registration failed", log.Fields{"error": err})
+	}
+
+	onuAdapter := &voltha.Adapter{
+		Id:             "onu-mock-adapter-1",
+		Vendor:         "onu-mock-vendor",
+		Version:        ver.VersionInfo.Version,
+		Type:           "onu-mock-adapter-type",
+		CurrentReplica: 1,
+		TotalReplicas:  1,
+		Endpoint:       "mock-onu-endpoint",
+	}
+	types = []*voltha.DeviceType{{Id: "onu-mock-device-type", AdapterType: "onu-mock-adapter-type", AcceptsAddRemoveFlowUpdates: true}}
+	deviceTypes = &voltha.DeviceTypes{Items: types}
+	_, err = dat.adapterMgr.RegisterAdapter(ctx, &ic.AdapterRegistration{
+		Adapter: onuAdapter,
+		DTypes:  deviceTypes,
+	})
+	if err != nil {
+		logger.Fatalw(ctx, "onu registration failed", log.Fields{"error": err})
+	}
+}
diff --git a/rw_core/core/device/agent_transient_state.go b/rw_core/core/device/agent_transient_state.go
index 0adf57d..ea105dd 100644
--- a/rw_core/core/device/agent_transient_state.go
+++ b/rw_core/core/device/agent_transient_state.go
@@ -18,26 +18,28 @@
 
 import (
 	"context"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
 
-func (agent *Agent) getTransientState() voltha.DeviceTransientState_Types {
+func (agent *Agent) getTransientState() core.DeviceTransientState_Types {
 	transientStateHandle := agent.transientStateLoader.Lock()
 	deviceTransientState := transientStateHandle.GetReadOnly()
 	transientStateHandle.UnLock()
 	return deviceTransientState
 }
 
-func (agent *Agent) matchTransientState(transientState voltha.DeviceTransientState_Types) bool {
+func (agent *Agent) matchTransientState(transientState core.DeviceTransientState_Types) bool {
 	transientStateHandle := agent.transientStateLoader.Lock()
 	defer transientStateHandle.UnLock()
 	return transientState == transientStateHandle.GetReadOnly()
 }
 
-func (agent *Agent) updateTransientState(ctx context.Context, transientState voltha.DeviceTransientState_Types) error {
+func (agent *Agent) updateTransientState(ctx context.Context, transientState core.DeviceTransientState_Types) error {
 	// Already in same transientState
 	if transientState == agent.getTransientState() {
 		return nil
@@ -54,16 +56,17 @@
 
 func (agent *Agent) isDeletionInProgress() bool {
 	deviceTransientState := agent.getTransientState()
-	return deviceTransientState == voltha.DeviceTransientState_FORCE_DELETING ||
-		deviceTransientState == voltha.DeviceTransientState_DELETING_FROM_ADAPTER ||
-		deviceTransientState == voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE
+	return deviceTransientState == core.DeviceTransientState_FORCE_DELETING ||
+		deviceTransientState == core.DeviceTransientState_DELETING_FROM_ADAPTER ||
+		deviceTransientState == core.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE
 }
 
-func (agent *Agent) isStateDeleting(deviceTransientState voltha.DeviceTransientState_Types) bool {
-	return deviceTransientState == voltha.DeviceTransientState_FORCE_DELETING ||
-		deviceTransientState == voltha.DeviceTransientState_DELETING_FROM_ADAPTER ||
-		deviceTransientState == voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE
+func (agent *Agent) isForceDeletingAllowed(deviceTransientState core.DeviceTransientState_Types, device *voltha.Device) bool {
+	return deviceTransientState != core.DeviceTransientState_FORCE_DELETING &&
+		(device.OperStatus != common.OperStatus_RECONCILING ||
+			!agent.matchTransientState(core.DeviceTransientState_RECONCILE_IN_PROGRESS))
 }
+
 func (agent *Agent) deleteTransientState(ctx context.Context) error {
 	transientStateHandle := agent.transientStateLoader.Lock()
 	if err := transientStateHandle.Delete(ctx); err != nil {
@@ -76,5 +79,5 @@
 
 func (agent *Agent) isInReconcileState(device *voltha.Device) bool {
 	return device.OperStatus == common.OperStatus_RECONCILING || device.OperStatus == common.OperStatus_RECONCILING_FAILED ||
-		agent.matchTransientState(voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+		agent.matchTransientState(core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 }
diff --git a/rw_core/core/device/common.go b/rw_core/core/device/common.go
index d781187..bad2067 100644
--- a/rw_core/core/device/common.go
+++ b/rw_core/core/device/common.go
@@ -18,7 +18,7 @@
 package device
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/event/common.go b/rw_core/core/device/event/common.go
index 72d5077..7a0ca19 100644
--- a/rw_core/core/device/event/common.go
+++ b/rw_core/core/device/event/common.go
@@ -18,7 +18,7 @@
 package event
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/event/event.go b/rw_core/core/device/event/event.go
index 2a134f1..d21995f 100644
--- a/rw_core/core/device/event/event.go
+++ b/rw_core/core/device/event/event.go
@@ -26,12 +26,12 @@
 
 	"github.com/golang/protobuf/ptypes/empty"
 	"github.com/opencord/voltha-go/rw_core/utils"
-	ev "github.com/opencord/voltha-lib-go/v5/pkg/events"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	ev "github.com/opencord/voltha-lib-go/v7/pkg/events"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"github.com/opentracing/opentracing-go"
 	jtracing "github.com/uber/jaeger-client-go"
 )
@@ -67,7 +67,7 @@
 		stackID:        stackID,
 	}
 }
-func (q *Manager) SendPacketIn(ctx context.Context, deviceID string, transationID string, packet *openflow_13.OfpPacketIn) {
+func (q *Manager) SendPacketIn(ctx context.Context, deviceID string, packet *openflow_13.OfpPacketIn) {
 	// TODO: Augment the OF PacketIn to include the transactionId
 	packetIn := openflow_13.PacketIn{Id: deviceID, PacketIn: packet}
 	logger.Debugw(ctx, "send-packet-in", log.Fields{"packet-in": packetIn})
@@ -273,7 +273,7 @@
 	//TODO Instead of directly sending to the kafka bus, queue the message and send it asynchronously
 	if rpcEvent.Rpc != "" {
 		if err := q.eventProxy.SendRPCEvent(ctx, id, rpcEvent, category, subCategory, raisedTs); err != nil {
-			logger.Errorw(ctx, "failed-to-send-rpc-event", log.Fields{"resource-id": id})
+			logger.Errorw(ctx, "failed-to-send-rpc-event", log.Fields{"resource-id": id, "error": err})
 		}
 	}
 }
@@ -283,7 +283,7 @@
 	rpcEvent := q.NewRPCEvent(ctx, resourceID, desc, context)
 	if rpcEvent.Rpc != "" {
 		if err := q.eventProxy.SendRPCEvent(ctx, id, rpcEvent, category, subCategory, raisedTs); err != nil {
-			logger.Errorw(ctx, "failed-to-send-rpc-event", log.Fields{"resource-id": id})
+			logger.Errorw(ctx, "failed-to-send-rpc-event", log.Fields{"resource-id": id, "error": err})
 		}
 	}
 }
diff --git a/rw_core/core/device/extension_manager.go b/rw_core/core/device/extension_manager.go
index 40c98aa..a895c34 100644
--- a/rw_core/core/device/extension_manager.go
+++ b/rw_core/core/device/extension_manager.go
@@ -18,8 +18,9 @@
 
 import (
 	"context"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
diff --git a/rw_core/core/device/flow/cache.go b/rw_core/core/device/flow/cache.go
index b4e358e..81efe0d 100644
--- a/rw_core/core/device/flow/cache.go
+++ b/rw_core/core/device/flow/cache.go
@@ -20,7 +20,7 @@
 	"context"
 	"sync"
 
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
 )
 
 // Cache hides all low-level locking & synchronization related to flow state updates
diff --git a/rw_core/core/device/flow/common.go b/rw_core/core/device/flow/common.go
index 6672e01..5765c5d 100644
--- a/rw_core/core/device/flow/common.go
+++ b/rw_core/core/device/flow/common.go
@@ -18,7 +18,7 @@
 package flow
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/group/cache.go b/rw_core/core/device/group/cache.go
index eb6a5a3..b182914 100644
--- a/rw_core/core/device/group/cache.go
+++ b/rw_core/core/device/group/cache.go
@@ -20,7 +20,7 @@
 	"context"
 	"sync"
 
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
 )
 
 // Cache hides all low-level locking & synchronization related to group state updates
diff --git a/rw_core/core/device/logical_agent.go b/rw_core/core/device/logical_agent.go
index b9ad95a..cbf2625 100644
--- a/rw_core/core/device/logical_agent.go
+++ b/rw_core/core/device/logical_agent.go
@@ -26,16 +26,16 @@
 	"github.com/opencord/voltha-go/db/model"
 	"github.com/opencord/voltha-go/rw_core/core/device/flow"
 	"github.com/opencord/voltha-go/rw_core/core/device/group"
-	"github.com/opencord/voltha-go/rw_core/core/device/logical_port"
+	lp "github.com/opencord/voltha-go/rw_core/core/device/logical_port"
 	"github.com/opencord/voltha-go/rw_core/core/device/meter"
 	fd "github.com/opencord/voltha-go/rw_core/flowdecomposition"
 	"github.com/opencord/voltha-go/rw_core/route"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -51,7 +51,7 @@
 	stopped         bool
 	deviceRoutes    *route.DeviceRoutes
 	flowDecomposer  *fd.FlowDecomposer
-	defaultTimeout  time.Duration
+	internalTimeout time.Duration
 	logicalDevice   *voltha.LogicalDevice
 	requestQueue    *coreutils.RequestQueue
 	orderedEvents   orderedEvents
@@ -61,11 +61,11 @@
 	flowCache   *flow.Cache
 	meterLoader *meter.Loader
 	groupCache  *group.Cache
-	portLoader  *port.Loader
+	portLoader  *lp.Loader
 }
 
 func newLogicalAgent(ctx context.Context, id string, sn string, deviceID string, ldeviceMgr *LogicalManager,
-	deviceMgr *Manager, dbProxy *model.Path, ldProxy *model.Proxy, defaultTimeout time.Duration) *LogicalAgent {
+	deviceMgr *Manager, dbProxy *model.Path, ldProxy *model.Proxy, internalTimeout time.Duration) *LogicalAgent {
 	return &LogicalAgent{
 		logicalDeviceID: id,
 		serialNumber:    sn,
@@ -75,13 +75,13 @@
 		ldeviceMgr:      ldeviceMgr,
 		deviceRoutes:    route.NewDeviceRoutes(id, deviceID, deviceMgr.listDevicePorts),
 		flowDecomposer:  fd.NewFlowDecomposer(deviceMgr.getDeviceReadOnly),
-		defaultTimeout:  defaultTimeout,
+		internalTimeout: internalTimeout,
 		requestQueue:    coreutils.NewRequestQueue(),
 
 		flowCache:   flow.NewCache(),
 		groupCache:  group.NewCache(),
 		meterLoader: meter.NewLoader(dbProxy.SubPath("logical_meters").Proxy(id)),
-		portLoader:  port.NewLoader(dbProxy.SubPath("logical_ports").Proxy(id)),
+		portLoader:  lp.NewLoader(dbProxy.SubPath("logical_ports").Proxy(id)),
 	}
 }
 
@@ -196,7 +196,7 @@
 			return
 		}
 		defer agent.requestQueue.RequestComplete()
-		subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
+		subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.internalTimeout)
 		// Before deletion of the logical agent, make sure all events for ldagent are sent to avoid race conditions
 		if err := agent.orderedEvents.waitForAllEventsToBeSent(subCtx, cancel); err != nil {
 			//Log the error here
@@ -236,7 +236,7 @@
 		response := coreutils.NewResponse()
 		responses = append(responses, response)
 		go func(deviceId string, value *fu.FlowsAndGroups) {
-			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
+			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.internalTimeout)
 			subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 			defer cancel()
 
@@ -272,7 +272,7 @@
 		response := coreutils.NewResponse()
 		responses = append(responses, response)
 		go func(deviceId string, value *fu.FlowsAndGroups) {
-			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
+			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.internalTimeout)
 			subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 			defer cancel()
 
@@ -306,7 +306,7 @@
 		response := coreutils.NewResponse()
 		responses = append(responses, response)
 		go func(deviceId string, value *fu.FlowsAndGroups) {
-			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
+			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.internalTimeout)
 			subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 
 			defer cancel()
@@ -342,7 +342,7 @@
 		}
 		logger.Debugw(ctx, "uni-port", log.Fields{"flows": flows, "uni-port": uniPort})
 		go func(uniPort uint32, metadata *voltha.FlowMetadata) {
-			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
+			subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.internalTimeout)
 			subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
 
 			defer cancel()
@@ -375,16 +375,15 @@
 	}
 }
 
-func (agent *LogicalAgent) packetIn(ctx context.Context, port uint32, transactionID string, packet []byte) {
+func (agent *LogicalAgent) packetIn(ctx context.Context, port uint32, packet []byte) {
 	if logger.V(log.InfoLevel) {
-		logger.Debugw(ctx, "packet-in", log.Fields{
-			"port":          port,
-			"packet":        hex.EncodeToString(packet),
-			"transactionId": transactionID,
+		logger.Infow(ctx, "packet-in", log.Fields{
+			"port":   port,
+			"packet": hex.EncodeToString(packet),
 		})
 	}
 
 	packetIn := fu.MkPacketIn(port, packet)
-	agent.ldeviceMgr.SendPacketIn(ctx, agent.logicalDeviceID, transactionID, packetIn)
+	agent.ldeviceMgr.SendPacketIn(ctx, agent.logicalDeviceID, packetIn)
 	logger.Debugw(ctx, "sending-packet-in", log.Fields{"packet": hex.EncodeToString(packetIn.Data)})
 }
diff --git a/rw_core/core/device/logical_agent_flow.go b/rw_core/core/device/logical_agent_flow.go
index 72b915f..85b78ef 100644
--- a/rw_core/core/device/logical_agent_flow.go
+++ b/rw_core/core/device/logical_agent_flow.go
@@ -24,12 +24,13 @@
 	"time"
 
 	"github.com/gogo/protobuf/proto"
+	"github.com/opencord/voltha-go/rw_core/core/device/flow"
 	"github.com/opencord/voltha-go/rw_core/route"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -104,7 +105,7 @@
 	var flowToReplace *ofp.OfpFlowStats
 
 	//if flow is not found in the map, create a new entry, otherwise get the existing one.
-	flowHandle, created, err := agent.flowCache.LockOrCreate(ctx, flow)
+	flowHandle, flowCreated, err := agent.flowCache.LockOrCreate(ctx, flow)
 	if err != nil {
 		return changed, updated, err
 	}
@@ -122,7 +123,7 @@
 			changed = true
 		}
 	} else {
-		if !created {
+		if !flowCreated {
 			flowToReplace = flowHandle.GetReadOnly()
 			if (mod.Flags & uint32(ofp.OfpFlowModFlags_OFPFF_RESET_COUNTS)) != 0 {
 				flow.ByteCount = flowToReplace.ByteCount
@@ -151,9 +152,30 @@
 
 		deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, updatedFlows, groups)
 		if err != nil {
+			if flowCreated {
+				if er := flowHandle.Delete(ctx); er != nil {
+					logger.Errorw(ctx, "deleting-flow-from-cache-failed", log.Fields{"error": er, "flow-id": flow.Id})
+				}
+			}
 			return changed, updated, err
 		}
 
+		// Verify whether the flow request can proceed, usually to multiple adapters
+		// This is an optimization to address the case where a decomposed set of flows need to
+		// be sent to multiple adapters.  One or more adapters may not be ready at this time.
+		// If one adapter is not ready this will result in flows being reverted from the
+		// other adapters, at times continuously as the OF controller will keep sending the
+		// flows until they are successfully added.
+		if err := agent.deviceMgr.canMultipleAdapterRequestProceed(ctx, deviceRules.Keys()); err != nil {
+			logger.Warnw(ctx, "adapters-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "flow-id": flow.Id, "error": err})
+			if flowCreated {
+				if er := flowHandle.Delete(ctx); er != nil {
+					logger.Errorw(ctx, "deleting-flow-from-cache-failed", log.Fields{"error": er, "flow-id": flow.Id})
+				}
+			}
+			return false, false, err
+		}
+
 		logger.Debugw(ctx, "rules", log.Fields{"rules": deviceRules.String()})
 		//	Update store and cache
 		if updated {
@@ -166,7 +188,7 @@
 		// Create the go routines to wait
 		go func() {
 			// Wait for completion
-			if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChannels...); res != nil {
+			if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, respChannels...); res != nil {
 				logger.Errorw(ctx, "failed-to-add-flow-will-attempt-deletion", log.Fields{
 					"errors":            res,
 					"logical-device-id": agent.logicalDeviceID,
@@ -237,7 +259,7 @@
 	// Wait for the responses
 	go func() {
 		// Since this action is taken following an add failure, we may also receive a failure for the revert
-		if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+		if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, respChnls...); res != nil {
 			logger.Warnw(ctx, "failure-reverting-added-flows", log.Fields{
 				"logical-device-id": agent.logicalDeviceID,
 				"flow-cookie":       mod.Cookie,
@@ -327,6 +349,18 @@
 			partialRoute = true
 		}
 
+		var devicesInFlows []string
+		if deviceRules != nil {
+			devicesInFlows = deviceRules.Keys()
+		} else {
+			devicesInFlows = []string{agent.rootDeviceID}
+		}
+
+		if err := agent.deviceMgr.canMultipleAdapterRequestProceed(ctx, devicesInFlows); err != nil {
+			logger.Warnw(ctx, "adapters-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "flow": toDelete, "error": err})
+			return err
+		}
+
 		// Update the devices
 		if partialRoute {
 			respChnls = agent.deleteFlowsFromParentDevice(ctx, toDelete, mod)
@@ -337,7 +371,7 @@
 		// Wait for the responses
 		go func() {
 			// Wait for completion
-			if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+			if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, respChnls...); res != nil {
 				logger.Errorw(ctx, "failure-updating-device-flows", log.Fields{"logical-device-id": agent.logicalDeviceID, "errors": res})
 				context := make(map[string]string)
 				context["rpc"] = coreutils.GetRPCMetadataFromContext(ctx)
@@ -363,6 +397,9 @@
 
 //flowDeleteStrict deletes a flow from the flow table of that logical device
 func (agent *LogicalAgent) flowDeleteStrict(ctx context.Context, flowUpdate *ofp.FlowTableUpdate) error {
+	var flowHandle *flow.Handle
+	var have bool
+
 	mod := flowUpdate.FlowMod
 	logger.Debugw(ctx, "flow-delete-strict", log.Fields{"mod": mod})
 	if mod == nil {
@@ -373,13 +410,18 @@
 	if err != nil {
 		return err
 	}
+
+	defer func() {
+		if flowHandle != nil {
+			flowHandle.Unlock()
+		}
+	}()
+
 	logger.Debugw(ctx, "flow-id-in-flow-delete-strict", log.Fields{"flow-id": flow.Id})
-	flowHandle, have := agent.flowCache.Lock(flow.Id)
+	flowHandle, have = agent.flowCache.Lock(flow.Id)
 	if !have {
-		logger.Debugw(ctx, "skipping-flow-delete-strict-request-no-flow-found", log.Fields{"flow-mod": mod})
-		return nil
+		logger.Debugw(ctx, "flow-delete-strict-request-no-flow-found-continuing", log.Fields{"flow-mod": mod})
 	}
-	defer flowHandle.Unlock()
 
 	groups := make(map[uint32]*ofp.OfpGroupEntry)
 	for groupID := range agent.groupCache.ListIDs() {
@@ -389,12 +431,11 @@
 		}
 	}
 
-	if changedMeter := agent.updateFlowCountOfMeterStats(ctx, mod, flow, false); !changedMeter {
-		return fmt.Errorf("Cannot delete flow - %s. Meter update failed", flow)
+	flowsToDelete := map[uint64]*ofp.OfpFlowStats{flow.Id: flow}
+	if flowHandle != nil {
+		flowsToDelete = map[uint64]*ofp.OfpFlowStats{flow.Id: flowHandle.GetReadOnly()}
 	}
 
-	flowsToDelete := map[uint64]*ofp.OfpFlowStats{flow.Id: flowHandle.GetReadOnly()}
-
 	var respChnls []coreutils.Response
 	var partialRoute bool
 	deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, flowsToDelete, groups)
@@ -408,10 +449,18 @@
 		partialRoute = true
 	}
 
-	// Update the model
-	if err := flowHandle.Delete(ctx); err != nil {
+	var devicesInFlows []string
+	if deviceRules != nil {
+		devicesInFlows = deviceRules.Keys()
+	} else {
+		devicesInFlows = []string{agent.rootDeviceID}
+	}
+
+	if err := agent.deviceMgr.canMultipleAdapterRequestProceed(ctx, devicesInFlows); err != nil {
+		logger.Warnw(ctx, "adapters-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "flow": flowsToDelete, "error": err})
 		return err
 	}
+
 	// Update the devices
 	if partialRoute {
 		respChnls = agent.deleteFlowsFromParentDevice(ctx, flowsToDelete, mod)
@@ -420,30 +469,39 @@
 	}
 
 	// Wait for completion
-	go func() {
-		if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
-			logger.Warnw(ctx, "failure-deleting-device-flows", log.Fields{
-				"flow-cookie":       mod.Cookie,
-				"logical-device-id": agent.logicalDeviceID,
-				"errors":            res,
-			})
-			// TODO: Revert flow changes
-			// send event, and allow any queued events to be sent as well
-			agent.ldeviceMgr.SendFlowChangeEvent(ctx, agent.logicalDeviceID, res, flowUpdate.Xid, flowUpdate.FlowMod.Cookie)
-			context := make(map[string]string)
-			context["rpc"] = coreutils.GetRPCMetadataFromContext(ctx)
-			context["flow-id"] = fmt.Sprintf("%v", flow.Id)
-			context["flow-cookie"] = fmt.Sprintf("%v", flowUpdate.FlowMod.Cookie)
-			context["logical-device-id"] = agent.logicalDeviceID
-			if deviceRules != nil {
-				context["device-rules"] = deviceRules.String()
-			}
-			// Create context and send extra information as part of it.
-			agent.ldeviceMgr.SendRPCEvent(ctx,
-				agent.logicalDeviceID, "failed-to-delete-device-flows", context, "RPC_ERROR_RAISE_EVENT",
-				voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
+	if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, respChnls...); res != nil {
+		logger.Warnw(ctx, "failure-deleting-device-flows", log.Fields{
+			"flow-cookie":       mod.Cookie,
+			"logical-device-id": agent.logicalDeviceID,
+			"errors":            res,
+		})
+		context := make(map[string]string)
+		context["rpc"] = coreutils.GetRPCMetadataFromContext(ctx)
+		context["flow-id"] = fmt.Sprintf("%v", flow.Id)
+		context["flow-cookie"] = fmt.Sprintf("%v", flowUpdate.FlowMod.Cookie)
+		context["logical-device-id"] = agent.logicalDeviceID
+		if deviceRules != nil {
+			context["device-rules"] = deviceRules.String()
 		}
-	}()
+		// Create context and send extra information as part of it.
+		agent.ldeviceMgr.SendRPCEvent(ctx,
+			agent.logicalDeviceID, "failed-to-delete-device-flows", context, "RPC_ERROR_RAISE_EVENT",
+			voltha.EventCategory_COMMUNICATION, nil, time.Now().Unix())
+
+		return status.Errorf(codes.Aborted, "failed deleting flows id:%d, errors:%v", flow.Id, res)
+	}
+
+	// Update meter count
+	if changedMeter := agent.updateFlowCountOfMeterStats(ctx, mod, flow, false); !changedMeter {
+		return fmt.Errorf("cannot delete flow - %s. Meter update failed", flow)
+	}
+
+	// Update the model
+	if flowHandle != nil {
+		if err := flowHandle.Delete(ctx); err != nil {
+			return err
+		}
+	}
 
 	return nil
 }
diff --git a/rw_core/core/device/logical_agent_group.go b/rw_core/core/device/logical_agent_group.go
index c778aa5..9a47d81 100644
--- a/rw_core/core/device/logical_agent_group.go
+++ b/rw_core/core/device/logical_agent_group.go
@@ -22,10 +22,10 @@
 	"time"
 
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -91,7 +91,7 @@
 
 	// Wait for completion
 	go func() {
-		if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+		if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, respChnls...); res != nil {
 			logger.Warnw(ctx, "failure-updating-device-flows-groups", log.Fields{"logical-device-id": agent.logicalDeviceID, "errors": res})
 			context := make(map[string]string)
 			context["rpc"] = coreutils.GetRPCMetadataFromContext(ctx)
@@ -163,6 +163,7 @@
 		deviceRules = fu.NewDeviceRules()
 		deviceRules.CreateEntryIfNotExist(agent.rootDeviceID)
 	}
+
 	//add groups to deviceRules
 	for _, groupEntry := range affectedGroups {
 		fg := fu.NewFlowsAndGroups()
@@ -171,12 +172,17 @@
 	}
 	logger.Debugw(ctx, "rules", log.Fields{"rules": deviceRules.String()})
 
+	if err := agent.deviceMgr.canMultipleAdapterRequestProceed(ctx, deviceRules.Keys()); err != nil {
+		logger.Warnw(ctx, "adapters-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+		return err
+	}
+
 	// delete groups and related flows, if any
 	respChnls := agent.deleteFlowsAndGroupsFromDevices(ctx, deviceRules, &ofp.OfpFlowMod{})
 
 	// Wait for completion
 	go func() {
-		if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+		if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, respChnls...); res != nil {
 			logger.Warnw(ctx, "failure-updating-device-flows-groups", log.Fields{"logical-device-id": agent.logicalDeviceID, "errors": res})
 			context := make(map[string]string)
 			context["rpc"] = coreutils.GetRPCMetadataFromContext(ctx)
@@ -229,7 +235,7 @@
 
 	// Wait for completion
 	go func() {
-		if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+		if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, respChnls...); res != nil {
 			logger.Warnw(ctx, "failure-updating-device-flows-groups", log.Fields{"logical-device-id": agent.logicalDeviceID, "errors": res})
 			context := make(map[string]string)
 			context["rpc"] = coreutils.GetRPCMetadataFromContext(ctx)
diff --git a/rw_core/core/device/logical_agent_meter.go b/rw_core/core/device/logical_agent_meter.go
index 7e824d2..f26a8bd 100644
--- a/rw_core/core/device/logical_agent_meter.go
+++ b/rw_core/core/device/logical_agent_meter.go
@@ -20,9 +20,9 @@
 	"context"
 	"fmt"
 
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
diff --git a/rw_core/core/device/logical_agent_meter_helpers.go b/rw_core/core/device/logical_agent_meter_helpers.go
index 19f7aad..16f7302 100644
--- a/rw_core/core/device/logical_agent_meter_helpers.go
+++ b/rw_core/core/device/logical_agent_meter_helpers.go
@@ -20,9 +20,9 @@
 	"context"
 	"fmt"
 
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
 )
 
 // GetMeterConfig returns meters which which are used by the given flows
diff --git a/rw_core/core/device/logical_agent_port.go b/rw_core/core/device/logical_agent_port.go
index 4672aca..35f18b9 100644
--- a/rw_core/core/device/logical_agent_port.go
+++ b/rw_core/core/device/logical_agent_port.go
@@ -22,10 +22,10 @@
 	"sync"
 
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -93,7 +93,7 @@
 	}
 
 	// Now, set up the UNI ports if needed.
-	children, err := agent.deviceMgr.GetAllChildDevices(ctx, agent.rootDeviceID)
+	children, err := agent.deviceMgr.getAllChildDevices(ctx, agent.rootDeviceID)
 	if err != nil {
 		logger.Errorw(ctx, "error-getting-child-devices", log.Fields{"error": err, "device-id": agent.rootDeviceID})
 		return err
@@ -120,7 +120,7 @@
 		}(subCtx, child)
 	}
 	// Wait for completion
-	if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, responses...); res != nil {
+	if res := coreutils.WaitForNilOrErrorResponses(agent.internalTimeout, responses...); res != nil {
 		return status.Errorf(codes.Aborted, "errors-%s", res)
 	}
 	return nil
@@ -220,8 +220,8 @@
 	// Reset the logical device routes
 	go func() {
 		subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
-		if err := agent.buildRoutes(subCtx); err != nil {
-			logger.Warnw(ctx, "device-routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+		if err := agent.removeRoutes(subCtx); err != nil {
+			logger.Warnw(ctx, "error-removing-routes", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
 		}
 	}()
 	return nil
@@ -513,16 +513,16 @@
 
 // getUNILogicalPortNo returns the UNI logical port number specified in the flow
 func (agent *LogicalAgent) getUNILogicalPortNo(flow *ofp.OfpFlowStats) (uint32, error) {
-	var uniPort uint32
 	inPortNo := fu.GetInPort(flow)
 	outPortNo := fu.GetOutPort(flow)
-	if agent.isNNIPort(inPortNo) {
-		uniPort = outPortNo
-	} else if agent.isNNIPort(outPortNo) {
-		uniPort = inPortNo
+	if inPortNo == 0 && outPortNo == 0 {
+		return 0, status.Errorf(codes.NotFound, "no-uni-port: %v", flow)
 	}
-	if uniPort != 0 {
-		return uniPort, nil
+	if inPortNo != 0 && !agent.isNNIPort(inPortNo) {
+		return inPortNo, nil
+	}
+	if outPortNo != 0 && !agent.isNNIPort(outPortNo) {
+		return outPortNo, nil
 	}
 	return 0, status.Errorf(codes.NotFound, "no-uni-port: %v", flow)
 }
diff --git a/rw_core/core/device/logical_agent_route.go b/rw_core/core/device/logical_agent_route.go
index 102397f..333efa8 100644
--- a/rw_core/core/device/logical_agent_route.go
+++ b/rw_core/core/device/logical_agent_route.go
@@ -21,9 +21,9 @@
 	"fmt"
 
 	"github.com/opencord/voltha-go/rw_core/route"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 // GetRoute returns a route
@@ -107,6 +107,17 @@
 	return nil
 }
 
+func (agent *LogicalAgent) removeRoutes(ctx context.Context) error {
+	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		return err
+	}
+	defer agent.requestQueue.RequestComplete()
+
+	agent.deviceRoutes.RemoveRoutes()
+
+	return nil
+}
+
 //updateRoutes updates the device routes
 func (agent *LogicalAgent) updateRoutes(ctx context.Context, deviceID string, devicePorts map[uint32]*voltha.Port, lp *voltha.LogicalPort, lps map[uint32]*voltha.LogicalPort) error {
 	logger.Debugw(ctx, "updateRoutes", log.Fields{"logical-device-id": agent.logicalDeviceID, "device-id": deviceID, "port:": lp})
diff --git a/rw_core/core/device/logical_agent_test.go b/rw_core/core/device/logical_agent_test.go
index 8114dd0..e611019 100644
--- a/rw_core/core/device/logical_agent_test.go
+++ b/rw_core/core/device/logical_agent_test.go
@@ -28,15 +28,15 @@
 	"github.com/opencord/voltha-go/rw_core/config"
 	"github.com/opencord/voltha-go/rw_core/core/adapter"
 	tst "github.com/opencord/voltha-go/rw_core/test"
-	com "github.com/opencord/voltha-lib-go/v5/pkg/adapters/common"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events"
-	fu "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	mock_etcd "github.com/opencord/voltha-lib-go/v5/pkg/mocks/etcd"
-	mock_kafka "github.com/opencord/voltha-lib-go/v5/pkg/mocks/kafka"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	com "github.com/opencord/voltha-lib-go/v7/pkg/adapters/common"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events"
+	fu "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	mock_etcd "github.com/opencord/voltha-lib-go/v7/pkg/mocks/etcd"
+	mock_kafka "github.com/opencord/voltha-lib-go/v7/pkg/mocks/kafka"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"github.com/phayes/freeport"
 	"github.com/stretchr/testify/assert"
 )
@@ -44,7 +44,6 @@
 type LDATest struct {
 	etcdServer       *mock_etcd.EtcdServer
 	deviceMgr        *Manager
-	kmp              kafka.InterContainerProxy
 	logicalDeviceMgr *LogicalManager
 	kClient          kafka.Client
 	kEventClient     kafka.Client
@@ -52,8 +51,8 @@
 	oltAdapterName   string
 	onuAdapterName   string
 	coreInstanceID   string
-	defaultTimeout   time.Duration
-	maxTimeout       time.Duration
+	internalTimeout  time.Duration
+	rpcTimeout       time.Duration
 	logicalDevice    *voltha.LogicalDevice
 	logicalPorts     map[uint32]*voltha.LogicalPort
 	deviceIds        []string
@@ -74,8 +73,8 @@
 	test.oltAdapterName = "olt_adapter_mock"
 	test.onuAdapterName = "onu_adapter_mock"
 	test.coreInstanceID = "rw-da-test"
-	test.defaultTimeout = 5 * time.Second
-	test.maxTimeout = 20 * time.Second
+	test.internalTimeout = 5 * time.Second
+	test.rpcTimeout = 20 * time.Second
 	test.done = make(chan int)
 	test.deviceIds = []string{com.GetRandomString(10), com.GetRandomString(10), com.GetRandomString(10)}
 	test.logicalDevice = &voltha.LogicalDevice{
@@ -138,15 +137,14 @@
 func (lda *LDATest) startCore(ctx context.Context, inCompeteMode bool) {
 	cfg := &config.RWCoreFlags{}
 	cfg.ParseCommandArguments([]string{})
-	cfg.CoreTopic = "rw_core"
 	cfg.EventTopic = "voltha.events"
-	cfg.DefaultRequestTimeout = lda.defaultTimeout
+	cfg.InternalTimeout = lda.internalTimeout
 	cfg.KVStoreAddress = "127.0.0.1" + ":" + strconv.Itoa(lda.kvClientPort)
 	grpcPort, err := freeport.GetFreePort()
 	if err != nil {
 		logger.Fatal(ctx, "Cannot get a freeport for grpc")
 	}
-	cfg.GrpcAddress = "127.0.0.1" + ":" + strconv.Itoa(grpcPort)
+	cfg.GrpcNBIAddress = "127.0.0.1" + ":" + strconv.Itoa(grpcPort)
 	client := tst.SetupKVClient(ctx, cfg, lda.coreInstanceID)
 	backend := &db.Backend{
 		Client:                  client,
@@ -154,29 +152,18 @@
 		Address:                 cfg.KVStoreAddress,
 		Timeout:                 cfg.KVStoreTimeout,
 		LivenessChannelInterval: cfg.LiveProbeInterval / 2}
-	lda.kmp = kafka.NewInterContainerProxy(
-		kafka.InterContainerAddress(cfg.KafkaAdapterAddress),
-		kafka.MsgClient(lda.kClient),
-		kafka.DefaultTopic(&kafka.Topic{Name: cfg.CoreTopic}))
 
-	endpointMgr := kafka.NewEndpointManager(backend)
 	proxy := model.NewDBPath(backend)
-	adapterMgr := adapter.NewAdapterManager(ctx, proxy, lda.coreInstanceID, lda.kClient)
+	adapterMgr := adapter.NewAdapterManager(proxy, lda.coreInstanceID, backend, 5)
 	eventProxy := events.NewEventProxy(events.MsgClient(lda.kEventClient), events.MsgTopic(kafka.Topic{Name: cfg.EventTopic}))
-	lda.deviceMgr, lda.logicalDeviceMgr = NewManagers(proxy, adapterMgr, lda.kmp, endpointMgr, cfg, lda.coreInstanceID, eventProxy)
-	if err = lda.kmp.Start(ctx); err != nil {
-		logger.Fatal(ctx, "Cannot start InterContainerProxy")
-	}
-	adapterMgr.Start(context.Background())
+	lda.deviceMgr, lda.logicalDeviceMgr = NewManagers(proxy, adapterMgr, cfg, lda.coreInstanceID, eventProxy)
+	adapterMgr.Start(context.Background(), "logical-test")
 }
 
 func (lda *LDATest) stopAll(ctx context.Context) {
 	if lda.kClient != nil {
 		lda.kClient.Stop(ctx)
 	}
-	if lda.kmp != nil {
-		lda.kmp.Stop(ctx)
-	}
 	if lda.etcdServer != nil {
 		tst.StopEmbeddedEtcdServer(ctx, lda.etcdServer)
 	}
@@ -191,7 +178,7 @@
 	clonedLD := proto.Clone(lda.logicalDevice).(*voltha.LogicalDevice)
 	clonedLD.Id = com.GetRandomString(10)
 	clonedLD.DatapathId = rand.Uint64()
-	lDeviceAgent := newLogicalAgent(context.Background(), clonedLD.Id, clonedLD.Id, clonedLD.RootDeviceId, lDeviceMgr, deviceMgr, lDeviceMgr.dbPath, lDeviceMgr.ldProxy, lDeviceMgr.defaultTimeout)
+	lDeviceAgent := newLogicalAgent(context.Background(), clonedLD.Id, clonedLD.Id, clonedLD.RootDeviceId, lDeviceMgr, deviceMgr, lDeviceMgr.dbPath, lDeviceMgr.ldProxy, lDeviceMgr.internalTimeout)
 	lDeviceAgent.logicalDevice = clonedLD
 	for _, port := range lda.logicalPorts {
 		clonedPort := proto.Clone(port).(*voltha.LogicalPort)
diff --git a/rw_core/core/device/logical_manager.go b/rw_core/core/device/logical_manager.go
index 26021ca..def6dc9 100644
--- a/rw_core/core/device/logical_manager.go
+++ b/rw_core/core/device/logical_manager.go
@@ -19,21 +19,21 @@
 import (
 	"context"
 	"errors"
-	"github.com/opencord/voltha-lib-go/v5/pkg/probe"
 	"io"
 	"strconv"
 	"strings"
 	"sync"
 	"time"
 
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+
 	"github.com/golang/protobuf/ptypes/empty"
 	"github.com/opencord/voltha-go/db/model"
 	"github.com/opencord/voltha-go/rw_core/core/device/event"
 	"github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -43,26 +43,25 @@
 	*event.Manager
 	logicalDeviceAgents            sync.Map
 	deviceMgr                      *Manager
-	kafkaICProxy                   kafka.InterContainerProxy
 	dbPath                         *model.Path
 	ldProxy                        *model.Proxy
-	defaultTimeout                 time.Duration
+	internalTimeout                time.Duration
 	logicalDevicesLoadingLock      sync.RWMutex
 	logicalDeviceLoadingInProgress map[string][]chan int
 }
 
-func (ldMgr *LogicalManager) Start(ctx context.Context) {
+func (ldMgr *LogicalManager) Start(ctx context.Context, serviceName string) {
 	logger.Info(ctx, "starting-logical-device-manager")
-	probe.UpdateStatusFromContext(ctx, "logical-device-manager", probe.ServiceStatusPreparing)
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusPreparing)
 
 	// Load all the logical devices from the dB
 	var logicalDevices []*voltha.LogicalDevice
 	if err := ldMgr.ldProxy.List(ctx, &logicalDevices); err != nil {
-		logger.Fatalw(ctx, "failed-to-list-logical-devices-from-cluster-proxy", log.Fields{"error": err})
+		logger.Fatalw(ctx, "failed-to-list-logical-devices-from-cluster-proxy", log.Fields{"error": err, "service-name": serviceName})
 	}
 	for _, lDevice := range logicalDevices {
 		// Create an agent for each device
-		agent := newLogicalAgent(ctx, lDevice.Id, "", "", ldMgr, ldMgr.deviceMgr, ldMgr.dbPath, ldMgr.ldProxy, ldMgr.defaultTimeout)
+		agent := newLogicalAgent(ctx, lDevice.Id, "", "", ldMgr, ldMgr.deviceMgr, ldMgr.dbPath, ldMgr.ldProxy, ldMgr.internalTimeout)
 		if err := agent.start(ctx, true, lDevice); err != nil {
 			logger.Warnw(ctx, "failure-starting-logical-agent", log.Fields{"logical-device-id": lDevice.Id})
 		} else {
@@ -70,7 +69,7 @@
 		}
 	}
 
-	probe.UpdateStatusFromContext(ctx, "logical-device-manager", probe.ServiceStatusRunning)
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
 	logger.Info(ctx, "logical-device-manager-started")
 }
 
@@ -128,7 +127,9 @@
 	var logicalDevices []*voltha.LogicalDevice
 	ldMgr.logicalDeviceAgents.Range(func(key, value interface{}) bool {
 		if ld, err := value.(*LogicalAgent).GetLogicalDeviceReadOnly(ctx); err == nil {
-			logicalDevices = append(logicalDevices, ld)
+			if ld != nil {
+				logicalDevices = append(logicalDevices, ld)
+			}
 		} else {
 			logger.Errorw(ctx, "unable-to-get-logical-device", log.Fields{"err": err})
 		}
@@ -159,7 +160,7 @@
 
 	logger.Debugw(ctx, "logical-device-id", log.Fields{"logical-device-id": id})
 
-	agent := newLogicalAgent(ctx, id, sn, device.Id, ldMgr, ldMgr.deviceMgr, ldMgr.dbPath, ldMgr.ldProxy, ldMgr.defaultTimeout)
+	agent := newLogicalAgent(ctx, id, sn, device.Id, ldMgr, ldMgr.deviceMgr, ldMgr.dbPath, ldMgr.ldProxy, ldMgr.internalTimeout)
 	ldMgr.addLogicalDeviceAgentToMap(agent)
 
 	// Update the root device with the logical device Id reference
@@ -232,7 +233,7 @@
 			ldMgr.logicalDevicesLoadingLock.Unlock()
 			if _, err := ldMgr.getLogicalDeviceFromModel(ctx, lDeviceID); err == nil {
 				logger.Debugw(ctx, "loading-logical-device", log.Fields{"lDeviceId": lDeviceID})
-				agent := newLogicalAgent(ctx, lDeviceID, "", "", ldMgr, ldMgr.deviceMgr, ldMgr.dbPath, ldMgr.ldProxy, ldMgr.defaultTimeout)
+				agent := newLogicalAgent(ctx, lDeviceID, "", "", ldMgr, ldMgr.deviceMgr, ldMgr.dbPath, ldMgr.ldProxy, ldMgr.internalTimeout)
 				if err := agent.start(ctx, true, nil); err != nil {
 					return err
 				}
@@ -562,10 +563,10 @@
 	return &empty.Empty{}, agent.disableLogicalPort(ctx, uint32(portNo))
 }
 
-func (ldMgr *LogicalManager) packetIn(ctx context.Context, logicalDeviceID string, port uint32, transactionID string, packet []byte) error {
+func (ldMgr *LogicalManager) packetIn(ctx context.Context, logicalDeviceID string, port uint32, packet []byte) error {
 	logger.Debugw(ctx, "packet-in", log.Fields{"logical-device-id": logicalDeviceID, "port": port})
 	if agent := ldMgr.getLogicalDeviceAgent(ctx, logicalDeviceID); agent != nil {
-		agent.packetIn(ctx, port, transactionID, packet)
+		agent.packetIn(ctx, port, packet)
 	} else {
 		logger.Error(ctx, "logical-device-not-exist", log.Fields{"logical-device-id": logicalDeviceID})
 	}
diff --git a/rw_core/core/device/logical_port/common.go b/rw_core/core/device/logical_port/common.go
index fede329..c8b3538 100644
--- a/rw_core/core/device/logical_port/common.go
+++ b/rw_core/core/device/logical_port/common.go
@@ -18,7 +18,7 @@
 package port
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/logical_port/loader.go b/rw_core/core/device/logical_port/loader.go
index 3117743..c6f272d 100644
--- a/rw_core/core/device/logical_port/loader.go
+++ b/rw_core/core/device/logical_port/loader.go
@@ -22,8 +22,8 @@
 	"sync"
 
 	"github.com/opencord/voltha-go/db/model"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
diff --git a/rw_core/core/device/manager.go b/rw_core/core/device/manager.go
index 0b34bfb..5519cab 100755
--- a/rw_core/core/device/manager.go
+++ b/rw_core/core/device/manager.go
@@ -18,27 +18,24 @@
 
 import (
 	"context"
-	"errors"
-	"github.com/opencord/voltha-go/rw_core/config"
-	"github.com/opencord/voltha-lib-go/v5/pkg/probe"
+	"fmt"
 	"sync"
 	"time"
 
-	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-go/rw_core/config"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-protos/v5/go/core"
+
 	"github.com/opencord/voltha-go/db/model"
 	"github.com/opencord/voltha-go/rw_core/core/adapter"
 	"github.com/opencord/voltha-go/rw_core/core/device/event"
-	"github.com/opencord/voltha-go/rw_core/core/device/remote"
 	"github.com/opencord/voltha-go/rw_core/core/device/state"
 	"github.com/opencord/voltha-go/rw_core/utils"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -48,32 +45,30 @@
 	deviceAgents      sync.Map
 	rootDevices       map[string]bool
 	lockRootDeviceMap sync.RWMutex
-	adapterProxy      *remote.AdapterProxy
 	*event.Agent
 	adapterMgr              *adapter.Manager
 	logicalDeviceMgr        *LogicalManager
-	kafkaICProxy            kafka.InterContainerProxy
 	stateTransitions        *state.TransitionMap
 	dbPath                  *model.Path
 	dProxy                  *model.Proxy
 	coreInstanceID          string
-	defaultTimeout          time.Duration
+	internalTimeout         time.Duration
+	rpcTimeout              time.Duration
 	devicesLoadingLock      sync.RWMutex
 	deviceLoadingInProgress map[string][]chan int
 	config                  *config.RWCoreFlags
 }
 
 //NewManagers creates the Manager and the Logical Manager.
-func NewManagers(dbPath *model.Path, adapterMgr *adapter.Manager, kmp kafka.InterContainerProxy, endpointMgr kafka.EndpointManager, cf *config.RWCoreFlags, coreInstanceID string, eventProxy *events.EventProxy) (*Manager, *LogicalManager) {
+func NewManagers(dbPath *model.Path, adapterMgr *adapter.Manager, cf *config.RWCoreFlags, coreInstanceID string, eventProxy *events.EventProxy) (*Manager, *LogicalManager) {
 	deviceMgr := &Manager{
 		rootDevices:             make(map[string]bool),
-		kafkaICProxy:            kmp,
-		adapterProxy:            remote.NewAdapterProxy(kmp, cf.CoreTopic, endpointMgr),
 		coreInstanceID:          coreInstanceID,
 		dbPath:                  dbPath,
 		dProxy:                  dbPath.Proxy("devices"),
 		adapterMgr:              adapterMgr,
-		defaultTimeout:          cf.DefaultCoreTimeout,
+		internalTimeout:         cf.InternalTimeout,
+		rpcTimeout:              cf.RPCTimeout,
 		Agent:                   event.NewAgent(eventProxy, coreInstanceID, cf.VolthaStackID),
 		deviceLoadingInProgress: make(map[string][]chan int),
 		config:                  cf,
@@ -83,33 +78,40 @@
 	logicalDeviceMgr := &LogicalManager{
 		Manager:                        event.NewManager(eventProxy, coreInstanceID, cf.VolthaStackID),
 		deviceMgr:                      deviceMgr,
-		kafkaICProxy:                   kmp,
 		dbPath:                         dbPath,
 		ldProxy:                        dbPath.Proxy("logical_devices"),
-		defaultTimeout:                 cf.DefaultCoreTimeout,
+		internalTimeout:                cf.InternalTimeout,
 		logicalDeviceLoadingInProgress: make(map[string][]chan int),
 	}
 	deviceMgr.logicalDeviceMgr = logicalDeviceMgr
 
-	adapterMgr.SetAdapterRestartedCallback(deviceMgr.adapterRestarted)
+	adapterMgr.SetAdapterRestartedCallback(deviceMgr.adapterRestartedHandler)
 
 	return deviceMgr, logicalDeviceMgr
 }
 
-func (dMgr *Manager) Start(ctx context.Context) {
+func (dMgr *Manager) Start(ctx context.Context, serviceName string) error {
 	logger.Info(ctx, "starting-device-manager")
-	probe.UpdateStatusFromContext(ctx, "device-manager", probe.ServiceStatusPreparing)
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusPreparing)
 
 	// Load all the devices from the dB
 	var devices []*voltha.Device
 	if err := dMgr.dProxy.List(ctx, &devices); err != nil {
 		// Any error from the dB means if we proceed we may end up with corrupted data
-		logger.Fatalw(ctx, "failed-to-list-devices-from-KV", log.Fields{"error": err})
+		logger.Errorw(ctx, "failed-to-list-devices-from-KV", log.Fields{"error": err, "service-name": serviceName})
+		return err
+	}
+
+	defer probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+
+	if len(devices) == 0 {
+		logger.Info(ctx, "no-device-to-load")
+		return nil
 	}
 
 	for _, device := range devices {
 		// Create an agent for each device
-		agent := newAgent(dMgr.adapterProxy, device, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.defaultTimeout)
+		agent := newAgent(device, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.internalTimeout, dMgr.rpcTimeout)
 		if _, err := agent.start(ctx, true, device); err != nil {
 			logger.Warnw(ctx, "failure-starting-agent", log.Fields{"device-id": device.Id})
 		} else {
@@ -119,8 +121,9 @@
 
 	// TODO: Need to trigger a reconcile at this point
 
-	probe.UpdateStatusFromContext(ctx, "device-manager", probe.ServiceStatusRunning)
 	logger.Info(ctx, "device-manager-started")
+
+	return nil
 }
 
 func (dMgr *Manager) addDeviceAgentToMap(agent *Agent) {
@@ -172,165 +175,6 @@
 	return result
 }
 
-// CreateDevice creates a new parent device in the data model
-func (dMgr *Manager) CreateDevice(ctx context.Context, device *voltha.Device) (*voltha.Device, error) {
-	if device.MacAddress == "" && device.GetHostAndPort() == "" {
-		logger.Errorf(ctx, "no-device-info-present")
-		return &voltha.Device{}, errors.New("no-device-info-present; MAC or HOSTIP&PORT")
-	}
-	ctx = utils.WithRPCMetadataContext(ctx, "CreateDevice")
-	logger.Debugw(ctx, "create-device", log.Fields{"device": *device})
-
-	deviceExist, err := dMgr.isParentDeviceExist(ctx, device)
-	if err != nil {
-		logger.Errorf(ctx, "failed-to-fetch-parent-device-info")
-		return nil, err
-	}
-	if deviceExist {
-		logger.Errorf(ctx, "device-is-pre-provisioned-already-with-same-ip-port-or-mac-address")
-		return nil, errors.New("device is already pre-provisioned")
-	}
-	logger.Debugw(ctx, "create-device", log.Fields{"device": device, "aproxy": dMgr.adapterProxy})
-
-	// Ensure this device is set as root
-	device.Root = true
-	// Create and start a device agent for that device
-	agent := newAgent(dMgr.adapterProxy, device, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.defaultTimeout)
-	device, err = agent.start(ctx, false, device)
-	if err != nil {
-		logger.Errorw(ctx, "fail-to-start-device", log.Fields{"device-id": agent.deviceID, "error": err})
-		return nil, err
-	}
-	dMgr.addDeviceAgentToMap(agent)
-	return device, nil
-}
-
-// EnableDevice activates a device by invoking the adopt_device API on the appropriate adapter
-func (dMgr *Manager) EnableDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "EnableDevice")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "enable-device", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-	return &empty.Empty{}, agent.enableDevice(ctx)
-}
-
-// DisableDevice disables a device along with any child device it may have
-func (dMgr *Manager) DisableDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "DisableDevice")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "disable-device", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-	return &empty.Empty{}, agent.disableDevice(ctx)
-}
-
-//RebootDevice invoked the reboot API to the corresponding adapter
-func (dMgr *Manager) RebootDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "RebootDevice")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "reboot-device", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-	return &empty.Empty{}, agent.rebootDevice(ctx)
-}
-
-// DeleteDevice removes a device from the data model
-func (dMgr *Manager) DeleteDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "DeleteDevice")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "delete-device", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-	return &empty.Empty{}, agent.deleteDevice(ctx)
-}
-
-// ForceDeleteDevice removes a device from the data model forcefully without successfully waiting for the adapters.
-func (dMgr *Manager) ForceDeleteDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ForceDeleteDevice")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "force-delete-device", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-	return &empty.Empty{}, agent.deleteDeviceForce(ctx)
-}
-
-// GetDevicePort returns the port details for a specific device port entry
-func (dMgr *Manager) GetDevicePort(ctx context.Context, deviceID string, portID uint32) (*voltha.Port, error) {
-	logger.Debugw(ctx, "get-device-port", log.Fields{"device-id": deviceID})
-	agent := dMgr.getDeviceAgent(ctx, deviceID)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "device-%s", deviceID)
-	}
-	return agent.getDevicePort(portID)
-}
-
-// ListDevicePorts returns the ports details for a specific device entry
-func (dMgr *Manager) ListDevicePorts(ctx context.Context, id *voltha.ID) (*voltha.Ports, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ListDevicePorts")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "list-device-ports", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "device-%s", id.Id)
-	}
-
-	ports := agent.listDevicePorts()
-	ctr, ret := 0, make([]*voltha.Port, len(ports))
-	for _, port := range ports {
-		ret[ctr] = port
-		ctr++
-	}
-	return &voltha.Ports{Items: ret}, nil
-}
-
-// ListDeviceFlows returns the flow details for a specific device entry
-func (dMgr *Manager) ListDeviceFlows(ctx context.Context, id *voltha.ID) (*ofp.Flows, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ListDeviceFlows")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "list-device-flows", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "device-%s", id.Id)
-	}
-
-	flows := agent.listDeviceFlows()
-	ctr, ret := 0, make([]*ofp.OfpFlowStats, len(flows))
-	for _, flow := range flows {
-		ret[ctr] = flow
-		ctr++
-	}
-	return &openflow_13.Flows{Items: ret}, nil
-}
-
-// ListDeviceFlowGroups returns the flow group details for a specific device entry
-func (dMgr *Manager) ListDeviceFlowGroups(ctx context.Context, id *voltha.ID) (*voltha.FlowGroups, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ListDeviceFlowGroups")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "list-device-flow-groups", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "device-%s", id.Id)
-	}
-	groups := agent.listDeviceGroups()
-	ctr, ret := 0, make([]*openflow_13.OfpGroupEntry, len(groups))
-	for _, group := range groups {
-		ret[ctr] = group
-		ctr++
-	}
-	return &voltha.FlowGroups{Items: ret}, nil
-}
-
 // stopManagingDevice stops the management of the device as well as any of its reference device and logical device.
 // This function is called only in the Core that does not own this device.  In the Core that owns this device then a
 // deletion deletion also includes removal of any reference of this device.
@@ -350,21 +194,6 @@
 	}
 }
 
-// RunPostDeviceDelete removes any reference of this device
-func (dMgr *Manager) RunPostDeviceDelete(ctx context.Context, cDevice *voltha.Device) error {
-	logger.Infow(ctx, "run-post-device-delete", log.Fields{"device-id": cDevice.Id})
-	dMgr.stopManagingDevice(ctx, cDevice.Id)
-	return nil
-}
-
-// GetDevice exists primarily to implement the gRPC interface.
-// Internal functions should call getDeviceReadOnly instead.
-func (dMgr *Manager) GetDevice(ctx context.Context, id *voltha.ID) (*voltha.Device, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "GetDevice")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	return dMgr.getDeviceReadOnly(ctx, id.Id)
-}
-
 // getDeviceReadOnly will returns a device, either from memory or from the dB, if present
 func (dMgr *Manager) getDeviceReadOnly(ctx context.Context, id string) (*voltha.Device, error) {
 	logger.Debugw(ctx, "get-device-read-only", log.Fields{"device-id": id})
@@ -383,119 +212,12 @@
 	return agent.listDevicePorts(), nil
 }
 
-// GetChildDevice will return a device, either from memory or from the dB, if present
-func (dMgr *Manager) GetChildDevice(ctx context.Context, parentDeviceID string, serialNumber string, onuID int64, parentPortNo int64) (*voltha.Device, error) {
-	logger.Debugw(ctx, "get-child-device", log.Fields{"parent-device-id": parentDeviceID, "serialNumber": serialNumber,
-		"parent-port-no": parentPortNo, "onu-id": onuID})
-
-	parentDevicePorts, err := dMgr.listDevicePorts(ctx, parentDeviceID)
-	if err != nil {
-		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
-	}
-	childDeviceIds := dMgr.getAllChildDeviceIds(ctx, parentDevicePorts)
-	if len(childDeviceIds) == 0 {
-		logger.Debugw(ctx, "no-child-devices", log.Fields{"parent-device-id": parentDeviceID, "serial-number": serialNumber, "onu-id": onuID})
-		return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
-	}
-
-	var foundChildDevice *voltha.Device
-	for childDeviceID := range childDeviceIds {
-		var found bool
-		if searchDevice, err := dMgr.getDeviceReadOnly(ctx, childDeviceID); err == nil {
-
-			foundOnuID := false
-			if searchDevice.ProxyAddress.OnuId == uint32(onuID) {
-				if searchDevice.ParentPortNo == uint32(parentPortNo) {
-					logger.Debugw(ctx, "found-child-by-onuid", log.Fields{"parent-device-id": parentDeviceID, "onu-id": onuID})
-					foundOnuID = true
-				}
-			}
-
-			foundSerialNumber := false
-			if searchDevice.SerialNumber == serialNumber {
-				logger.Debugw(ctx, "found-child-by-serial-number", log.Fields{"parent-device-id": parentDeviceID, "serial-number": serialNumber})
-				foundSerialNumber = true
-			}
-
-			// if both onuId and serialNumber are provided both must be true for the device to be found
-			// otherwise whichever one found a match is good enough
-			if onuID > 0 && serialNumber != "" {
-				found = foundOnuID && foundSerialNumber
-			} else {
-				found = foundOnuID || foundSerialNumber
-			}
-
-			if found {
-				foundChildDevice = searchDevice
-				break
-			}
-		}
-	}
-
-	if foundChildDevice != nil {
-		logger.Debugw(ctx, "child-device-found", log.Fields{"parent-device-id": parentDeviceID, "found-child-device": foundChildDevice})
-		return foundChildDevice, nil
-	}
-
-	logger.Debugw(ctx, "child-device-not-found", log.Fields{"parent-device-id": parentDeviceID,
-		"serial-number": serialNumber, "onu-id": onuID, "parent-port-no": parentPortNo})
-	return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
-}
-
-// GetChildDeviceWithProxyAddress will return a device based on proxy address
-func (dMgr *Manager) GetChildDeviceWithProxyAddress(ctx context.Context, proxyAddress *voltha.Device_ProxyAddress) (*voltha.Device, error) {
-	logger.Debugw(ctx, "get-child-device-with-proxy-address", log.Fields{"proxy-address": proxyAddress})
-
-	parentDevicePorts, err := dMgr.listDevicePorts(ctx, proxyAddress.DeviceId)
-	if err != nil {
-		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
-	}
-	childDeviceIds := dMgr.getAllChildDeviceIds(ctx, parentDevicePorts)
-	if len(childDeviceIds) == 0 {
-		logger.Debugw(ctx, "no-child-devices", log.Fields{"parent-device-id": proxyAddress.DeviceId})
-		return nil, status.Errorf(codes.NotFound, "%s", proxyAddress)
-	}
-
-	var foundChildDevice *voltha.Device
-	for childDeviceID := range childDeviceIds {
-		if searchDevice, err := dMgr.getDeviceReadOnly(ctx, childDeviceID); err == nil {
-			if searchDevice.ProxyAddress == proxyAddress {
-				foundChildDevice = searchDevice
-				break
-			}
-		}
-	}
-
-	if foundChildDevice != nil {
-		logger.Debugw(ctx, "child-device-found", log.Fields{"proxy-address": proxyAddress})
-		return foundChildDevice, nil
-	}
-
-	logger.Warnw(ctx, "child-device-not-found", log.Fields{"proxy-address": proxyAddress})
-	return nil, status.Errorf(codes.NotFound, "%s", proxyAddress)
-}
-
 // IsDeviceInCache returns true if device is found in the map
 func (dMgr *Manager) IsDeviceInCache(id string) bool {
 	_, exist := dMgr.deviceAgents.Load(id)
 	return exist
 }
 
-// ListDevices retrieves the latest devices from the data model
-func (dMgr *Manager) ListDevices(ctx context.Context, _ *empty.Empty) (*voltha.Devices, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ListDevices")
-	logger.Debug(ctx, "list-devices")
-	result := &voltha.Devices{}
-
-	dMgr.deviceAgents.Range(func(key, value interface{}) bool {
-		result.Items = append(result.Items, value.(*Agent).device)
-		return true
-	})
-
-	logger.Debugw(ctx, "list-devices-end", log.Fields{"len": len(result.Items)})
-	return result, nil
-}
-
 //isParentDeviceExist checks whether device is already preprovisioned.
 func (dMgr *Manager) isParentDeviceExist(ctx context.Context, newDevice *voltha.Device) (bool, error) {
 	hostPort := newDevice.GetHostAndPort()
@@ -547,7 +269,7 @@
 			// Proceed with the loading only if the device exist in the Model (could have been deleted)
 			if device, err = dMgr.getDeviceFromModel(ctx, deviceID); err == nil {
 				logger.Debugw(ctx, "loading-device", log.Fields{"device-id": deviceID})
-				agent := newAgent(dMgr.adapterProxy, device, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.defaultTimeout)
+				agent := newAgent(device, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.internalTimeout, dMgr.rpcTimeout)
 				if _, err = agent.start(ctx, true, device); err != nil {
 					logger.Warnw(ctx, "failure-loading-device", log.Fields{"device-id": deviceID, "error": err})
 				} else {
@@ -648,132 +370,34 @@
 	return nil
 }
 
-// ListDeviceIds retrieves the latest device IDs information from the data model (memory data only)
-func (dMgr *Manager) ListDeviceIds(ctx context.Context, _ *empty.Empty) (*voltha.IDs, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ListDeviceIds")
-	logger.Debug(ctx, "list-device-ids")
-	// Report only device IDs that are in the device agent map
-	return dMgr.listDeviceIdsFromMap(), nil
-}
-
-// ReconcileDevices is a request to a voltha core to update its list of managed devices.  This will
-// trigger loading the devices along with their children and parent in memory
-func (dMgr *Manager) ReconcileDevices(ctx context.Context, ids *voltha.IDs) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ReconcileDevices")
-
-	numDevices := 0
-	if ids != nil {
-		numDevices = len(ids.Items)
-	}
-
-	logger.Debugw(ctx, "reconcile-devices", log.Fields{"num-devices": numDevices})
-	if ids != nil && len(ids.Items) != 0 {
-		toReconcile := len(ids.Items)
-		reconciled := 0
-		var err error
-		for _, id := range ids.Items {
-			if err = dMgr.load(ctx, id.Id); err != nil {
-				logger.Warnw(ctx, "failure-reconciling-device", log.Fields{"device-id": id.Id, "error": err})
-			} else {
-				reconciled++
-			}
-		}
-		if toReconcile != reconciled {
-			return nil, status.Errorf(codes.DataLoss, "less-device-reconciled-than-requested:%d/%d", reconciled, toReconcile)
-		}
-	} else {
-		return nil, status.Errorf(codes.InvalidArgument, "empty-list-of-ids")
-	}
-	return &empty.Empty{}, nil
-}
-
 // adapterRestarted is invoked whenever an adapter is restarted
 func (dMgr *Manager) adapterRestarted(ctx context.Context, adapter *voltha.Adapter) error {
 	logger.Debugw(ctx, "adapter-restarted", log.Fields{"adapter-id": adapter.Id, "vendor": adapter.Vendor,
-		"current-replica": adapter.CurrentReplica, "total-replicas": adapter.TotalReplicas, "endpoint": adapter.Endpoint})
+		"current-replica": adapter.CurrentReplica, "total-replicas": adapter.TotalReplicas, "restarted-endpoint": adapter.Endpoint})
 
-	// Let's reconcile the device managed by this Core only
-	if len(dMgr.rootDevices) == 0 {
-		logger.Debugw(ctx, "nothing-to-reconcile", log.Fields{"adapter-id": adapter.Id})
-		return nil
-	}
-
-	if len(dMgr.rootDevices) == 0 {
-		logger.Debugw(ctx, "no-managed-device-to-reconcile", log.Fields{"adapter-id": adapter.Id})
-		return nil
-	}
-
-	for rootDeviceID := range dMgr.rootDevices {
-		dAgent := dMgr.getDeviceAgent(ctx, rootDeviceID)
-		if dAgent == nil {
-			continue
-		}
-		logger.Debugw(ctx, "checking-adapter-type", log.Fields{"agentType": dAgent.deviceType, "adapter-type": adapter.Type})
-		if dAgent.deviceType == adapter.Type {
-			rootDevice, _ := dAgent.getDeviceReadOnly(ctx)
-			if rootDevice == nil {
-				continue
-			}
-			isDeviceOwnedByService, err := dMgr.adapterProxy.IsDeviceOwnedByService(ctx, rootDeviceID, adapter.Type, adapter.CurrentReplica)
-			if err != nil {
-				logger.Warnw(ctx, "is-device-owned-by-service", log.Fields{"error": err, "root-device-id": rootDeviceID, "adapter-type": adapter.Type, "replica-number": adapter.CurrentReplica})
-				continue
-			}
-			if isDeviceOwnedByService {
-				if rootDevice.AdminState != voltha.AdminState_PREPROVISIONED {
-					logger.Debugw(ctx, "reconciling-root-device", log.Fields{"rootId": rootDevice.Id})
-					go dAgent.ReconcileDevice(ctx, rootDevice)
-				} else {
-					logger.Debugw(ctx, "not-reconciling-root-device", log.Fields{"rootId": rootDevice.Id, "state": rootDevice.AdminState})
-				}
-			} else { // Should we be reconciling the root's children instead?
-				rootDevicePorts, _ := dMgr.listDevicePorts(ctx, rootDeviceID)
-			childManagedByAdapter:
-				for _, port := range rootDevicePorts {
-					for _, peer := range port.Peers {
-						if childDevice, _ := dMgr.getDeviceFromModel(ctx, peer.DeviceId); childDevice != nil {
-							isDeviceOwnedByService, err := dMgr.adapterProxy.IsDeviceOwnedByService(ctx, childDevice.Id, adapter.Type, adapter.CurrentReplica)
-							if err != nil {
-								logger.Warnw(ctx, "is-device-owned-by-service", log.Fields{"error": err, "child-device-id": childDevice.Id, "adapter-type": adapter.Type, "replica-number": adapter.CurrentReplica})
-							}
-							if isDeviceOwnedByService {
-								if childDevice.AdminState != voltha.AdminState_PREPROVISIONED {
-									logger.Debugw(ctx, "reconciling-child-device", log.Fields{"child-device-id": childDevice.Id})
-									go dAgent.ReconcileDevice(ctx, childDevice)
-								} else {
-									logger.Debugw(ctx, "not-reconciling-child-device", log.Fields{"child-device-id": childDevice.Id, "state": childDevice.AdminState})
-								}
-							} else {
-								// All child devices under a parent device are typically managed by the same adapter type.
-								// Therefore we only need to check whether the first device we retrieved is managed by that adapter
-								break childManagedByAdapter
-							}
-						}
-					}
-				}
+	numberOfDevicesToReconcile := 0
+	dMgr.deviceAgents.Range(func(key, value interface{}) bool {
+		deviceAgent, ok := value.(*Agent)
+		if ok && deviceAgent.adapterEndpoint == adapter.Endpoint {
+			// Before reconciling, abort in-process request
+			if err := deviceAgent.abortAllProcessing(utils.WithNewSpanAndRPCMetadataContext(ctx, "AbortProcessingOnRestart")); err == nil {
+				logger.Debugw(ctx, "reconciling-device",
+					log.Fields{
+						"device-id":          deviceAgent.deviceID,
+						"root-device":        deviceAgent.isRootDevice,
+						"restarted-endpoint": adapter.Endpoint,
+						"device-type":        deviceAgent.deviceType,
+						"adapter-type":       adapter.Type,
+					})
+				go deviceAgent.ReconcileDevice(utils.WithNewSpanAndRPCMetadataContext(ctx, "ReconcileDevice"))
+				numberOfDevicesToReconcile++
+			} else {
+				logger.Errorw(ctx, "failed-aborting-exisiting-processing", log.Fields{"error": err})
 			}
 		}
-	}
-	logger.Debugw(ctx, "Reconciling for device on adapter restart is initiated", log.Fields{"adapter-id": adapter.Id})
-
-	return nil
-}
-
-func (dMgr *Manager) ReconcileChildDevices(ctx context.Context, parentDeviceID string) error {
-	dAgent := dMgr.getDeviceAgent(ctx, parentDeviceID)
-	if dAgent == nil {
-		return status.Errorf(codes.NotFound, "error-unable to get agent from device")
-	}
-	if parentDevicePorts, err := dMgr.listDevicePorts(ctx, parentDeviceID); err == nil {
-		for _, port := range parentDevicePorts {
-			for _, peer := range port.Peers {
-				if childDevice, err := dMgr.getDeviceFromModel(ctx, peer.DeviceId); err == nil {
-					go dAgent.ReconcileDevice(ctx, childDevice)
-				}
-			}
-		}
-		logger.Debugw(ctx, "Reconcile initiated for child devices", log.Fields{"parent-device-id": parentDeviceID})
-	}
+		return true
+	})
+	logger.Debugw(ctx, "reconciling-on-adapter-restart-initiated", log.Fields{"adapter-endpoint": adapter.Endpoint, "number-of-devices-to-reconcile": numberOfDevicesToReconcile})
 	return nil
 }
 
@@ -831,6 +455,28 @@
 	return status.Errorf(codes.NotFound, "%s", deviceID)
 }
 
+func (dMgr *Manager) canMultipleAdapterRequestProceed(ctx context.Context, deviceIDs []string) error {
+	ready := len(deviceIDs) > 0
+	for _, deviceID := range deviceIDs {
+		agent := dMgr.getDeviceAgent(ctx, deviceID)
+		if agent == nil {
+			logger.Errorw(ctx, "adapter-nil", log.Fields{"device-id": deviceID})
+			return status.Errorf(codes.Unavailable, "adapter-nil-for-%s", deviceID)
+		}
+		ready = ready && agent.isAdapterConnectionUp(ctx)
+		if !ready {
+			return status.Errorf(codes.Unavailable, "adapter-connection-down-for-%s", deviceID)
+		}
+		if err := agent.canDeviceRequestProceed(ctx); err != nil {
+			return err
+		}
+	}
+	if !ready {
+		return status.Error(codes.Unavailable, "adapter(s)-not-ready")
+	}
+	return nil
+}
+
 func (dMgr *Manager) addFlowsAndGroups(ctx context.Context, deviceID string, flows []*ofp.OfpFlowStats, groups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) error {
 	logger.Debugw(ctx, "add-flows-and-groups", log.Fields{"device-id": deviceID, "groups:": groups, "flow-metadata": flowMetadata})
 	if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
@@ -867,21 +513,6 @@
 	return status.Errorf(codes.NotFound, "%s", deviceID)
 }
 
-// UpdateDevicePmConfigs updates the PM configs.  This is executed when the northbound gRPC API is invoked, typically
-// following a user action
-func (dMgr *Manager) UpdateDevicePmConfigs(ctx context.Context, configs *voltha.PmConfigs) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "UpdateDevicePmConfigs")
-	log.EnrichSpan(ctx, log.Fields{"device-id": configs.Id})
-	if configs.Id == "" {
-		return nil, status.Error(codes.FailedPrecondition, "invalid-device-Id")
-	}
-	agent := dMgr.getDeviceAgent(ctx, configs.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", configs.Id)
-	}
-	return &empty.Empty{}, agent.updatePmConfigs(ctx, configs)
-}
-
 // InitPmConfigs initialize the pm configs as defined by the adapter.
 func (dMgr *Manager) InitPmConfigs(ctx context.Context, deviceID string, pmConfigs *voltha.PmConfigs) error {
 	if pmConfigs.Id == "" {
@@ -893,17 +524,6 @@
 	return status.Errorf(codes.NotFound, "%s", deviceID)
 }
 
-// ListDevicePmConfigs returns pm configs of device
-func (dMgr *Manager) ListDevicePmConfigs(ctx context.Context, id *voltha.ID) (*voltha.PmConfigs, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ListDevicePmConfigs")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-	return agent.listPmConfigs(ctx)
-}
-
 func (dMgr *Manager) getSwitchCapability(ctx context.Context, deviceID string) (*ic.SwitchCapability, error) {
 	logger.Debugw(ctx, "get-switch-capability", log.Fields{"device-id": deviceID})
 	if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
@@ -912,15 +532,6 @@
 	return nil, status.Errorf(codes.NotFound, "%s", deviceID)
 }
 
-func (dMgr *Manager) GetPorts(ctx context.Context, deviceID string, portType voltha.Port_PortType) (*voltha.Ports, error) {
-	logger.Debugw(ctx, "get-ports", log.Fields{"device-id": deviceID, "port-type": portType})
-	agent := dMgr.getDeviceAgent(ctx, deviceID)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", deviceID)
-	}
-	return agent.getPorts(ctx, portType), nil
-}
-
 func (dMgr *Manager) UpdateDeviceStatus(ctx context.Context, deviceID string, operStatus voltha.OperStatus_Types, connStatus voltha.ConnectStatus_Types) error {
 	logger.Debugw(ctx, "update-device-status", log.Fields{"device-id": deviceID, "oper-status": operStatus, "conn-status": connStatus})
 	if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
@@ -974,31 +585,6 @@
 	return status.Errorf(codes.NotFound, "%s", deviceID)
 }
 
-func (dMgr *Manager) DeleteAllPorts(ctx context.Context, deviceID string) error {
-	logger.Debugw(ctx, "delete-all-ports", log.Fields{"device-id": deviceID})
-	if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
-		if err := agent.deleteAllPorts(ctx); err != nil {
-			return err
-		}
-		// Notify the logical device manager to remove all logical ports, if needed.
-		// At this stage the device itself may gave been deleted already at a DeleteAllPorts
-		// typically is part of a device deletion phase.
-		if device, err := dMgr.getDeviceReadOnly(ctx, deviceID); err == nil {
-			go func() {
-				subCtx := utils.WithSpanAndRPCMetadataFromContext(ctx)
-				if err := dMgr.logicalDeviceMgr.deleteAllLogicalPorts(subCtx, device); err != nil {
-					logger.Errorw(ctx, "unable-to-delete-logical-ports", log.Fields{"error": err})
-				}
-			}()
-		} else {
-			logger.Warnw(ctx, "failed-to-retrieve-device", log.Fields{"device-id": deviceID})
-			return err
-		}
-		return nil
-	}
-	return status.Errorf(codes.NotFound, "%s", deviceID)
-}
-
 //UpdatePortsState updates all ports on the device
 func (dMgr *Manager) UpdatePortsState(ctx context.Context, deviceID string, portTypeFilter uint32, state voltha.OperStatus_Types) error {
 	logger.Debugw(ctx, "update-ports-state", log.Fields{"device-id": deviceID})
@@ -1016,80 +602,6 @@
 	return nil
 }
 
-func (dMgr *Manager) ChildDeviceDetected(ctx context.Context, parentDeviceID string, parentPortNo int64, deviceType string,
-	channelID int64, vendorID string, serialNumber string, onuID int64) (*voltha.Device, error) {
-	logger.Debugw(ctx, "child-device-detected", log.Fields{"parent-device-id": parentDeviceID, "parent-port-no": parentPortNo, "device-type": deviceType, "channel-id": channelID, "vendor-id": vendorID, "serial-number": serialNumber, "onu-id": onuID})
-
-	if deviceType == "" && vendorID != "" {
-		logger.Debug(ctx, "device-type-is-nil-fetching-device-type")
-		deviceTypes, err := dMgr.adapterMgr.ListDeviceTypes(ctx, nil)
-		if err != nil {
-			return nil, err
-		}
-	OLoop:
-		for _, dType := range deviceTypes.Items {
-			for _, v := range dType.VendorIds {
-				if v == vendorID {
-					deviceType = dType.Adapter
-					break OLoop
-				}
-			}
-		}
-	}
-	//if no match found for the vendorid,report adapter with the custom error message
-	if deviceType == "" {
-		logger.Errorw(ctx, "failed-to-fetch-adapter-name ", log.Fields{"vendor-id": vendorID})
-		return nil, status.Errorf(codes.NotFound, "%s", vendorID)
-	}
-
-	// Create the ONU device
-	childDevice := &voltha.Device{}
-	childDevice.Type = deviceType
-	childDevice.ParentId = parentDeviceID
-	childDevice.ParentPortNo = uint32(parentPortNo)
-	childDevice.VendorId = vendorID
-	childDevice.SerialNumber = serialNumber
-	childDevice.Root = false
-
-	// Get parent device type
-	pAgent := dMgr.getDeviceAgent(ctx, parentDeviceID)
-	if pAgent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
-	}
-	if pAgent.deviceType == "" {
-		return nil, status.Errorf(codes.FailedPrecondition, "device Type not set %s", parentDeviceID)
-	}
-
-	if device, err := dMgr.GetChildDevice(ctx, parentDeviceID, serialNumber, onuID, parentPortNo); err == nil {
-		logger.Warnw(ctx, "child-device-exists", log.Fields{"parent-device-id": parentDeviceID, "serial-number": serialNumber})
-		return device, status.Errorf(codes.AlreadyExists, "%s", serialNumber)
-	}
-
-	childDevice.ProxyAddress = &voltha.Device_ProxyAddress{DeviceId: parentDeviceID, DeviceType: pAgent.deviceType, ChannelId: uint32(channelID), OnuId: uint32(onuID)}
-
-	// Create and start a device agent for that device
-	agent := newAgent(dMgr.adapterProxy, childDevice, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.defaultTimeout)
-	insertedChildDevice, err := agent.start(ctx, false, childDevice)
-	if err != nil {
-		logger.Errorw(ctx, "error-starting-child-device", log.Fields{"parent-device-id": childDevice.ParentId, "child-device-id": agent.deviceID, "error": err})
-		return nil, err
-	}
-	dMgr.addDeviceAgentToMap(agent)
-
-	// Activate the child device
-	if agent = dMgr.getDeviceAgent(ctx, agent.deviceID); agent != nil {
-		go func() {
-			subCtx := utils.WithFromTopicMetadataFromContext(utils.WithSpanAndRPCMetadataFromContext(ctx), ctx)
-			err := agent.enableDevice(subCtx)
-			if err != nil {
-				logger.Errorw(ctx, "unable-to-enable-device", log.Fields{"error": err})
-			}
-		}()
-	}
-
-	return insertedChildDevice, nil
-}
-
 func (dMgr *Manager) packetOut(ctx context.Context, deviceID string, outPort uint32, packet *ofp.OfpPacketOut) error {
 	logger.Debugw(ctx, "packet-out", log.Fields{"device-id": deviceID, "out-port": outPort})
 	if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
@@ -1113,7 +625,7 @@
 		return status.Errorf(codes.FailedPrecondition, "%s", deviceID)
 	}
 
-	if err := dMgr.logicalDeviceMgr.packetIn(ctx, device.ParentId, port, transactionID, packet); err != nil {
+	if err := dMgr.logicalDeviceMgr.packetIn(ctx, device.ParentId, port, packet); err != nil {
 		return err
 	}
 	return nil
@@ -1127,47 +639,6 @@
 	return status.Errorf(codes.NotFound, "%s", device.Id)
 }
 
-//
-//CreateLogicalDevice creates logical device in core
-func (dMgr *Manager) CreateLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
-	logger.Info(ctx, "create-logical-device")
-	// Verify whether the logical device has already been created
-	if cDevice.ParentId != "" {
-		logger.Debugw(ctx, "parent-device-already-exist", log.Fields{"device-id": cDevice.Id, "logical-device-id": cDevice.Id})
-		return nil
-	}
-	var err error
-	if _, err = dMgr.logicalDeviceMgr.createLogicalDevice(ctx, cDevice); err != nil {
-		logger.Warnw(ctx, "create-logical-device-error", log.Fields{"device": cDevice})
-		return err
-	}
-	return nil
-}
-
-// DeleteLogicalDevice deletes logical device from core
-func (dMgr *Manager) DeleteLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
-	logger.Info(ctx, "delete-logical-device")
-	var err error
-	if err = dMgr.logicalDeviceMgr.deleteLogicalDevice(ctx, cDevice); err != nil {
-		logger.Warnw(ctx, "delete-logical-device-error", log.Fields{"device-id": cDevice.Id})
-		return err
-	}
-	// Remove the logical device Id from the parent device
-	logicalID := ""
-	dMgr.UpdateDeviceAttribute(ctx, cDevice.Id, "ParentId", logicalID)
-	return nil
-}
-
-// DeleteLogicalPorts removes the logical ports associated with that deviceId
-func (dMgr *Manager) DeleteLogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
-	logger.Debugw(ctx, "delete-all-logical-ports", log.Fields{"device-id": cDevice.Id})
-	if err := dMgr.logicalDeviceMgr.deleteLogicalPorts(ctx, cDevice.Id); err != nil {
-		// Just log the error.   The logical device or port may already have been deleted before this callback is invoked.
-		logger.Warnw(ctx, "delete-logical-ports-error", log.Fields{"device-id": cDevice.Id, "error": err})
-	}
-	return nil
-}
-
 func (dMgr *Manager) getParentDevice(ctx context.Context, childDevice *voltha.Device) *voltha.Device {
 	//	Sanity check
 	if childDevice.Root {
@@ -1178,54 +649,6 @@
 	return parentDevice
 }
 
-//ChildDevicesLost is invoked by an adapter to indicate that a parent device is in a state (Disabled) where it
-//cannot manage the child devices.  This will trigger the Core to disable all the child devices.
-func (dMgr *Manager) ChildDevicesLost(ctx context.Context, parentDeviceID string) error {
-	logger.Debug(ctx, "child-devices-lost")
-	parentDevice, err := dMgr.getDeviceReadOnly(ctx, parentDeviceID)
-	if err != nil {
-		logger.Warnw(ctx, "failed-getting-device", log.Fields{"parent-device-id": parentDeviceID, "error": err})
-		return err
-	}
-	return dMgr.DisableAllChildDevices(ctx, parentDevice)
-}
-
-//ChildDevicesDetected is invoked by an adapter when child devices are found, typically after after a
-// disable/enable sequence.  This will trigger the Core to Enable all the child devices of that parent.
-func (dMgr *Manager) ChildDevicesDetected(ctx context.Context, parentDeviceID string) error {
-	logger.Debug(ctx, "child-devices-detected")
-	parentDevicePorts, err := dMgr.listDevicePorts(ctx, parentDeviceID)
-	if err != nil {
-		logger.Warnw(ctx, "failed-getting-device", log.Fields{"device-id": parentDeviceID, "error": err})
-		return err
-	}
-	childDeviceIds := dMgr.getAllChildDeviceIds(ctx, parentDevicePorts)
-	if len(childDeviceIds) == 0 {
-		logger.Debugw(ctx, "no-child-device", log.Fields{"parent-device-id": parentDeviceID})
-	}
-	allChildEnableRequestSent := true
-	for childDeviceID := range childDeviceIds {
-		if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
-			subCtx := utils.WithSpanAndRPCMetadataFromContext(ctx)
-			// Run the children re-registration in its own routine
-			go func(ctx context.Context) {
-				err = agent.enableDevice(ctx)
-				if err != nil {
-					logger.Errorw(ctx, "unable-to-enable-device", log.Fields{"error": err})
-				}
-			}(subCtx)
-		} else {
-			err = status.Errorf(codes.Unavailable, "no agent for child device %s", childDeviceID)
-			logger.Errorw(ctx, "no-child-device-agent", log.Fields{"parent-device-id": parentDeviceID, "child-id": childDeviceID})
-			allChildEnableRequestSent = false
-		}
-	}
-	if !allChildEnableRequestSent {
-		return err
-	}
-	return nil
-}
-
 /*
 All the functions below are callback functions where they are invoked with the latest and previous data.  We can
 therefore use the data as is without trying to get the latest from the model.
@@ -1246,63 +669,6 @@
 	return nil
 }
 
-//DeleteAllChildDevices is invoked as a callback when the parent device is deleted
-func (dMgr *Manager) DeleteAllChildDevices(ctx context.Context, parentCurrDevice *voltha.Device) error {
-	logger.Debug(ctx, "delete-all-child-devices")
-	force := false
-	// Get the parent device Transient state, if its FORCE_DELETED(go for force delete for child devices)
-	// So in cases when this handler is getting called other than DELETE operation, no force option would be used.
-	agent := dMgr.getDeviceAgent(ctx, parentCurrDevice.Id)
-	if agent == nil {
-		return status.Errorf(codes.NotFound, "%s", parentCurrDevice.Id)
-	}
-
-	force = agent.getTransientState() == voltha.DeviceTransientState_FORCE_DELETING
-
-	ports, _ := dMgr.listDevicePorts(ctx, parentCurrDevice.Id)
-	for childDeviceID := range dMgr.getAllChildDeviceIds(ctx, ports) {
-		if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
-			if force {
-				if err := agent.deleteDeviceForce(ctx); err != nil {
-					logger.Warnw(ctx, "failure-delete-device-force", log.Fields{"device-id": childDeviceID,
-						"error": err.Error()})
-				}
-			} else {
-				if err := agent.deleteDevice(ctx); err != nil {
-					logger.Warnw(ctx, "failure-delete-device", log.Fields{"device-id": childDeviceID,
-						"error": err.Error()})
-				}
-			}
-			// No further action is required here.  The deleteDevice will change the device state where the resulting
-			// callback will take care of cleaning the child device agent.
-		}
-	}
-	return nil
-}
-
-//DeleteAllLogicalPorts is invoked as a callback when the parent device's connection status moves to UNREACHABLE
-func (dMgr *Manager) DeleteAllLogicalPorts(ctx context.Context, parentDevice *voltha.Device) error {
-	logger.Debugw(ctx, "delete-all-logical-ports", log.Fields{"parent-device-id": parentDevice.Id})
-	if err := dMgr.logicalDeviceMgr.deleteAllLogicalPorts(ctx, parentDevice); err != nil {
-		// Just log error as logical device may already have been deleted
-		logger.Warnw(ctx, "delete-all-logical-ports-fail", log.Fields{"parent-device-id": parentDevice.Id, "error": err})
-	}
-	return nil
-}
-
-//DeleteAllDeviceFlows is invoked as a callback when the parent device's connection status moves to UNREACHABLE
-func (dMgr *Manager) DeleteAllDeviceFlows(ctx context.Context, parentDevice *voltha.Device) error {
-	logger.Debugw(ctx, "delete-all-device-flows", log.Fields{"parent-device-id": parentDevice.Id})
-	if agent := dMgr.getDeviceAgent(ctx, parentDevice.Id); agent != nil {
-		if err := agent.deleteAllFlows(ctx); err != nil {
-			logger.Errorw(ctx, "error-deleting-all-device-flows", log.Fields{"parent-device-id": parentDevice.Id})
-			return err
-		}
-		return nil
-	}
-	return status.Errorf(codes.NotFound, "%s", parentDevice.Id)
-}
-
 //getAllChildDeviceIds is a helper method to get all the child device IDs from the device passed as parameter
 func (dMgr *Manager) getAllChildDeviceIds(ctx context.Context, parentDevicePorts map[uint32]*voltha.Port) map[string]struct{} {
 	logger.Debug(ctx, "get-all-child-device-ids")
@@ -1316,8 +682,8 @@
 	return childDeviceIds
 }
 
-//GetAllChildDevices is a helper method to get all the child device IDs from the device passed as parameter
-func (dMgr *Manager) GetAllChildDevices(ctx context.Context, parentDeviceID string) (*voltha.Devices, error) {
+//GgtAllChildDevices is a helper method to get all the child device IDs from the device passed as parameter
+func (dMgr *Manager) getAllChildDevices(ctx context.Context, parentDeviceID string) (*voltha.Devices, error) {
 	logger.Debugw(ctx, "get-all-child-devices", log.Fields{"parent-device-id": parentDeviceID})
 	if parentDevicePorts, err := dMgr.listDevicePorts(ctx, parentDeviceID); err == nil {
 		childDevices := make([]*voltha.Device, 0)
@@ -1331,165 +697,6 @@
 	return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
 }
 
-// SetupUNILogicalPorts creates UNI ports on the logical device that represents a child UNI interface
-func (dMgr *Manager) SetupUNILogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
-	logger.Info(ctx, "setup-uni-logical-ports")
-	cDevicePorts, err := dMgr.listDevicePorts(ctx, cDevice.Id)
-	if err != nil {
-		return err
-	}
-	if err := dMgr.logicalDeviceMgr.setupUNILogicalPorts(ctx, cDevice, cDevicePorts); err != nil {
-		logger.Warnw(ctx, "setup-uni-logical-ports-error", log.Fields{"device": cDevice, "err": err})
-		return err
-	}
-	return nil
-}
-
-// convenience to avoid redefining
-var operationFailureResp = &common.OperationResp{Code: voltha.OperationResp_OPERATION_FAILURE}
-
-// DownloadImage execute an image download request
-func (dMgr *Manager) DownloadImage(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "DownloadImage")
-	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
-	logger.Debugw(ctx, "download-image", log.Fields{"device-id": img.Id, "image-name": img.Name})
-	agent := dMgr.getDeviceAgent(ctx, img.Id)
-	if agent == nil {
-		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
-	}
-	resp, err := agent.downloadImage(ctx, img)
-	if err != nil {
-		return operationFailureResp, err
-	}
-	return resp, nil
-}
-
-// CancelImageDownload cancels image download request
-func (dMgr *Manager) CancelImageDownload(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "CancelImageDownload")
-	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
-	logger.Debugw(ctx, "cancel-image-download", log.Fields{"device-id": img.Id, "image-name": img.Name})
-	agent := dMgr.getDeviceAgent(ctx, img.Id)
-	if agent == nil {
-		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
-	}
-	resp, err := agent.cancelImageDownload(ctx, img)
-	if err != nil {
-		return operationFailureResp, err
-	}
-	return resp, nil
-}
-
-// ActivateImageUpdate activates image update request
-func (dMgr *Manager) ActivateImageUpdate(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ActivateImageUpdate")
-	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
-	logger.Debugw(ctx, "activate-image-update", log.Fields{"device-id": img.Id, "image-name": img.Name})
-	agent := dMgr.getDeviceAgent(ctx, img.Id)
-	if agent == nil {
-		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
-	}
-	resp, err := agent.activateImage(ctx, img)
-	if err != nil {
-		return operationFailureResp, err
-	}
-	return resp, nil
-}
-
-// RevertImageUpdate reverts image update
-func (dMgr *Manager) RevertImageUpdate(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "RevertImageUpdate")
-	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
-	logger.Debugw(ctx, "rever-image-update", log.Fields{"device-id": img.Id, "image-name": img.Name})
-	agent := dMgr.getDeviceAgent(ctx, img.Id)
-	if agent == nil {
-		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
-	}
-	resp, err := agent.revertImage(ctx, img)
-	if err != nil {
-		return operationFailureResp, err
-	}
-	return resp, nil
-}
-
-// convenience to avoid redefining
-var imageDownloadFailureResp = &voltha.ImageDownload{DownloadState: voltha.ImageDownload_DOWNLOAD_UNKNOWN}
-
-// GetImageDownloadStatus returns status of image download
-func (dMgr *Manager) GetImageDownloadStatus(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "GetImageDownloadStatus")
-	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
-	logger.Debugw(ctx, "get-image-download-status", log.Fields{"device-id": img.Id, "image-name": img.Name})
-	agent := dMgr.getDeviceAgent(ctx, img.Id)
-	if agent == nil {
-		return imageDownloadFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
-	}
-	resp, err := agent.getImageDownloadStatus(ctx, img)
-	if err != nil {
-		return imageDownloadFailureResp, err
-	}
-	return resp, nil
-}
-
-func (dMgr *Manager) UpdateImageDownload(ctx context.Context, deviceID string, img *voltha.ImageDownload) error {
-	ctx = utils.WithRPCMetadataContext(ctx, "UpdateImageDownload")
-	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
-	logger.Debugw(ctx, "update-image-download", log.Fields{"device-id": img.Id, "image-name": img.Name})
-	if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
-		if err := agent.updateImageDownload(ctx, img); err != nil {
-			logger.Debugw(ctx, "update-image-download-failed", log.Fields{"err": err, "image-name": img.Name})
-			return err
-		}
-	} else {
-		return status.Errorf(codes.NotFound, "%s", img.Id)
-	}
-	return nil
-}
-
-// GetImageDownload returns image download
-func (dMgr *Manager) GetImageDownload(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "GetImageDownload")
-	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
-	logger.Debugw(ctx, "get-image-download", log.Fields{"device-id": img.Id, "image-name": img.Name})
-	agent := dMgr.getDeviceAgent(ctx, img.Id)
-	if agent == nil {
-		return imageDownloadFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
-	}
-	resp, err := agent.getImageDownload(ctx, img)
-	if err != nil {
-		return imageDownloadFailureResp, err
-	}
-	return resp, nil
-}
-
-// ListImageDownloads returns image downloads
-func (dMgr *Manager) ListImageDownloads(ctx context.Context, id *voltha.ID) (*voltha.ImageDownloads, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "ListImageDownloads")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "list-image-downloads", log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return &voltha.ImageDownloads{Items: []*voltha.ImageDownload{imageDownloadFailureResp}}, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-	resp, err := agent.listImageDownloads(ctx, id.Id)
-	if err != nil {
-		return &voltha.ImageDownloads{Items: []*voltha.ImageDownload{imageDownloadFailureResp}}, err
-	}
-	return resp, nil
-}
-
-// GetImages returns all images for a specific device entry
-func (dMgr *Manager) GetImages(ctx context.Context, id *voltha.ID) (*voltha.Images, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "GetImages")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	logger.Debugw(ctx, "get-images", log.Fields{"device-id": id.Id})
-	device, err := dMgr.getDeviceReadOnly(ctx, id.Id)
-	if err != nil {
-		return nil, err
-	}
-	return device.Images, nil
-}
-
 func (dMgr *Manager) NotifyInvalidTransition(ctx context.Context, device *voltha.Device) error {
 	logger.Errorw(ctx, "notify-invalid-transition", log.Fields{
 		"device":           device.Id,
@@ -1517,21 +724,6 @@
 	return ""
 }
 
-func (dMgr *Manager) SimulateAlarm(ctx context.Context, simulateReq *voltha.SimulateAlarmRequest) (*common.OperationResp, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "SimulateAlarm")
-	logger.Debugw(ctx, "simulate-alarm", log.Fields{"id": simulateReq.Id, "indicator": simulateReq.Indicator, "intf-id": simulateReq.IntfId,
-		"port-type-name": simulateReq.PortTypeName, "onu-device-id": simulateReq.OnuDeviceId, "inverse-bit-error-rate": simulateReq.InverseBitErrorRate,
-		"drift": simulateReq.Drift, "new-eqd": simulateReq.NewEqd, "onu-serial-number": simulateReq.OnuSerialNumber, "operation": simulateReq.Operation})
-	agent := dMgr.getDeviceAgent(ctx, simulateReq.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", simulateReq.Id)
-	}
-	if err := agent.simulateAlarm(ctx, simulateReq); err != nil {
-		return nil, err
-	}
-	return &common.OperationResp{Code: common.OperationResp_OPERATION_SUCCESS}, nil
-}
-
 func (dMgr *Manager) UpdateDeviceReason(ctx context.Context, deviceID string, reason string) error {
 	logger.Debugw(ctx, "update-device-reason", log.Fields{"device-id": deviceID, "reason": reason})
 	if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
@@ -1540,368 +732,20 @@
 	return status.Errorf(codes.NotFound, "%s", deviceID)
 }
 
-func (dMgr *Manager) EnablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "EnablePort")
-	log.EnrichSpan(ctx, log.Fields{"device-id": port.DeviceId})
-	logger.Debugw(ctx, "enable-port", log.Fields{"device-id": port.DeviceId, "port-no": port.PortNo})
-	agent := dMgr.getDeviceAgent(ctx, port.DeviceId)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", port.DeviceId)
-	}
-	return &empty.Empty{}, agent.enablePort(ctx, port.PortNo)
-}
-
-func (dMgr *Manager) DisablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "DisablePort")
-	log.EnrichSpan(ctx, log.Fields{"device-id": port.DeviceId})
-	logger.Debugw(ctx, "disable-port", log.Fields{"device-id": port.DeviceId, "port-no": port.PortNo})
-	agent := dMgr.getDeviceAgent(ctx, port.DeviceId)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", port.DeviceId)
-	}
-	return &empty.Empty{}, agent.disablePort(ctx, port.PortNo)
-}
-
-// ChildDeviceLost  calls parent adapter to delete child device and all its references
-func (dMgr *Manager) ChildDeviceLost(ctx context.Context, curr *voltha.Device) error {
-	logger.Debugw(ctx, "child-device-lost", log.Fields{"child-device-id": curr.Id, "parent-device-id": curr.ParentId})
-	if parentAgent := dMgr.getDeviceAgent(ctx, curr.ParentId); parentAgent != nil {
-		if err := parentAgent.ChildDeviceLost(ctx, curr); err != nil {
-			// Just log the message and let the remaining pipeline proceed.
-			logger.Warnw(ctx, "childDeviceLost", log.Fields{"child-device-id": curr.Id, "parent-device-id": curr.ParentId, "error": err})
-		}
-	}
-	// Do not return an error as parent device may also have been deleted.  Let the remaining pipeline proceed.
-	return nil
-}
-
-func (dMgr *Manager) StartOmciTestAction(ctx context.Context, request *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "StartOmciTestAction")
-	log.EnrichSpan(ctx, log.Fields{"device-id": request.Id})
-	logger.Debugw(ctx, "start-omci-test-action", log.Fields{"device-id": request.Id, "uuid": request.Uuid})
-	agent := dMgr.getDeviceAgent(ctx, request.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", request.Id)
-	}
-	return agent.startOmciTest(ctx, request)
-}
-
-func (dMgr *Manager) GetExtValue(ctx context.Context, value *voltha.ValueSpecifier) (*voltha.ReturnValues, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "GetExtValue")
-	log.EnrichSpan(ctx, log.Fields{"device-id": value.Id})
-	logger.Debugw(ctx, "get-ext-value", log.Fields{"onu-id": value.Id})
-	cDevice, err := dMgr.getDeviceReadOnly(ctx, value.Id)
-	if err != nil {
-		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
-	}
-	pDevice, err := dMgr.getDeviceReadOnly(ctx, cDevice.ParentId)
-	if err != nil {
-		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
-	}
-	if agent := dMgr.getDeviceAgent(ctx, cDevice.ParentId); agent != nil {
-		resp, err := agent.getExtValue(ctx, pDevice, cDevice, value)
-		if err != nil {
-			return nil, err
-		}
-		logger.Debugw(ctx, "get-ext-value-result", log.Fields{"result": resp})
-		return resp, nil
-	}
-	return nil, status.Errorf(codes.NotFound, "%s", value.Id)
-
-}
-
-// SetExtValue  set some given configs or value
-func (dMgr *Manager) SetExtValue(ctx context.Context, value *voltha.ValueSet) (*empty.Empty, error) {
-	ctx = utils.WithRPCMetadataContext(ctx, "SetExtValue")
-	logger.Debugw(ctx, "set-ext-value", log.Fields{"onu-id": value.Id})
-	device, err := dMgr.getDeviceReadOnly(ctx, value.Id)
-	if err != nil {
-		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
-	}
-	if agent := dMgr.getDeviceAgent(ctx, device.Id); agent != nil {
-		resp, err := agent.setExtValue(ctx, device, value)
-		if err != nil {
-			return nil, err
-		}
-		logger.Debugw(ctx, "set-ext-value-result", log.Fields{"result": resp})
-		return resp, nil
-	}
-	return nil, status.Errorf(codes.NotFound, "%s", value.Id)
-
-}
-
 func (dMgr *Manager) SendRPCEvent(ctx context.Context, id string, rpcEvent *voltha.RPCEvent,
 	category voltha.EventCategory_Types, subCategory *voltha.EventSubCategory_Types, raisedTs int64) {
 	//TODO Instead of directly sending to the kafka bus, queue the message and send it asynchronously
 	dMgr.Agent.SendRPCEvent(ctx, id, rpcEvent, category, subCategory, raisedTs)
 }
 
-func (dMgr *Manager) GetTransientState(ctx context.Context, id string) (voltha.DeviceTransientState_Types, error) {
+func (dMgr *Manager) GetTransientState(ctx context.Context, id string) (core.DeviceTransientState_Types, error) {
 	agent := dMgr.getDeviceAgent(ctx, id)
 	if agent == nil {
-		return voltha.DeviceTransientState_NONE, status.Errorf(codes.NotFound, "%s", id)
+		return core.DeviceTransientState_NONE, status.Errorf(codes.NotFound, "%s", id)
 	}
 	return agent.getTransientState(), nil
 }
 
-func (dMgr *Manager) DownloadImageToDevice(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
-	if err := dMgr.validateImageDownloadRequest(request); err != nil {
-		return nil, err
-	}
-
-	ctx = utils.WithRPCMetadataContext(ctx, "DownloadImageToDevice")
-	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
-
-	for index, deviceID := range request.DeviceId {
-		// Create download request per device
-		downloadReq := &voltha.DeviceImageDownloadRequest{
-			Image:             request.Image,
-			ActivateOnSuccess: request.ActivateOnSuccess,
-			CommitOnSuccess:   request.CommitOnSuccess,
-		}
-
-		//slice-out only single deviceID from the request
-		downloadReq.DeviceId = request.DeviceId[index : index+1]
-
-		go func(deviceID string, req *voltha.DeviceImageDownloadRequest, ch chan []*voltha.DeviceImageState) {
-			agent := dMgr.getDeviceAgent(ctx, deviceID)
-			if agent == nil {
-				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
-				ch <- nil
-				return
-			}
-
-			resp, err := agent.downloadImageToDevice(ctx, req)
-			if err != nil {
-				logger.Errorw(ctx, "download-image-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-
-			err = dMgr.validateDeviceImageResponse(resp)
-			if err != nil {
-				logger.Errorw(ctx, "download-image-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-			ch <- resp.GetDeviceImageStates()
-		}(deviceID.GetId(), downloadReq, respCh)
-
-	}
-
-	return dMgr.waitForAllResponses(ctx, "download-image-to-device", respCh, len(request.GetDeviceId()))
-}
-
-func (dMgr *Manager) GetImageStatus(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
-	if err := dMgr.validateImageRequest(request); err != nil {
-		return nil, err
-	}
-
-	ctx = utils.WithRPCMetadataContext(ctx, "GetImageStatus")
-
-	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
-	for index, deviceID := range request.DeviceId {
-		// Create status request per device
-		imageStatusReq := &voltha.DeviceImageRequest{
-			Version:         request.Version,
-			CommitOnSuccess: request.CommitOnSuccess,
-		}
-
-		//slice-out only single deviceID from the request
-		imageStatusReq.DeviceId = request.DeviceId[index : index+1]
-
-		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
-			agent := dMgr.getDeviceAgent(ctx, deviceID)
-			if agent == nil {
-				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
-				ch <- nil
-				return
-			}
-
-			resp, err := agent.getImageStatus(ctx, req)
-			if err != nil {
-				logger.Errorw(ctx, "get-image-status-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-
-			err = dMgr.validateDeviceImageResponse(resp)
-			if err != nil {
-				logger.Errorw(ctx, "get-image-status-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-			ch <- resp.GetDeviceImageStates()
-		}(deviceID.GetId(), imageStatusReq, respCh)
-
-	}
-
-	return dMgr.waitForAllResponses(ctx, "get-image-status", respCh, len(request.GetDeviceId()))
-}
-
-func (dMgr *Manager) AbortImageUpgradeToDevice(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
-	if err := dMgr.validateImageRequest(request); err != nil {
-		return nil, err
-	}
-
-	ctx = utils.WithRPCMetadataContext(ctx, "AbortImageUpgradeToDevice")
-	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
-
-	for index, deviceID := range request.DeviceId {
-		// Create abort request per device
-		abortImageReq := &voltha.DeviceImageRequest{
-			Version:         request.Version,
-			CommitOnSuccess: request.CommitOnSuccess,
-		}
-
-		//slice-out only single deviceID from the request
-		abortImageReq.DeviceId = request.DeviceId[index : index+1]
-
-		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
-			agent := dMgr.getDeviceAgent(ctx, deviceID)
-			if agent == nil {
-				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
-				ch <- nil
-				return
-			}
-
-			resp, err := agent.abortImageUpgradeToDevice(ctx, req)
-			if err != nil {
-				logger.Errorw(ctx, "abort-image-upgrade-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-
-			err = dMgr.validateDeviceImageResponse(resp)
-			if err != nil {
-				logger.Errorw(ctx, "abort-image-upgrade-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-			ch <- resp.GetDeviceImageStates()
-		}(deviceID.GetId(), abortImageReq, respCh)
-
-	}
-
-	return dMgr.waitForAllResponses(ctx, "abort-image-upgrade-to-device", respCh, len(request.GetDeviceId()))
-}
-
-func (dMgr *Manager) GetOnuImages(ctx context.Context, id *common.ID) (*voltha.OnuImages, error) {
-	if id == nil || id.Id == "" {
-		return nil, status.Errorf(codes.InvalidArgument, "empty device id")
-	}
-
-	ctx = utils.WithRPCMetadataContext(ctx, "GetOnuImages")
-	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
-	agent := dMgr.getDeviceAgent(ctx, id.Id)
-	if agent == nil {
-		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
-	}
-
-	resp, err := agent.getOnuImages(ctx, id)
-	if err != nil {
-		return nil, err
-	}
-
-	logger.Debugw(ctx, "get-onu-images-result", log.Fields{"onu-image": resp.Items})
-
-	return resp, nil
-}
-
-func (dMgr *Manager) ActivateImage(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
-	if err := dMgr.validateImageRequest(request); err != nil {
-		return nil, err
-	}
-
-	ctx = utils.WithRPCMetadataContext(ctx, "ActivateImage")
-	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
-
-	for index, deviceID := range request.DeviceId {
-		// Create activate request per device
-		activateImageReq := &voltha.DeviceImageRequest{
-			Version:         request.Version,
-			CommitOnSuccess: request.CommitOnSuccess,
-		}
-
-		//slice-out only single deviceID from the request
-		activateImageReq.DeviceId = request.DeviceId[index : index+1]
-
-		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
-			agent := dMgr.getDeviceAgent(ctx, deviceID)
-			if agent == nil {
-				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
-				ch <- nil
-				return
-			}
-
-			resp, err := agent.activateImageOnDevice(ctx, req)
-			if err != nil {
-				logger.Errorw(ctx, "activate-image-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-
-			err = dMgr.validateDeviceImageResponse(resp)
-			if err != nil {
-				logger.Errorw(ctx, "activate-image-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-
-			ch <- resp.GetDeviceImageStates()
-		}(deviceID.GetId(), activateImageReq, respCh)
-
-	}
-
-	return dMgr.waitForAllResponses(ctx, "activate-image", respCh, len(request.GetDeviceId()))
-}
-
-func (dMgr *Manager) CommitImage(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
-	if err := dMgr.validateImageRequest(request); err != nil {
-		return nil, err
-	}
-
-	ctx = utils.WithRPCMetadataContext(ctx, "CommitImage")
-	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
-
-	for index, deviceID := range request.DeviceId {
-		// Create commit request per device
-		commitImageReq := &voltha.DeviceImageRequest{
-			Version:         request.Version,
-			CommitOnSuccess: request.CommitOnSuccess,
-		}
-		//slice-out only single deviceID from the request
-		commitImageReq.DeviceId = request.DeviceId[index : index+1]
-
-		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
-			agent := dMgr.getDeviceAgent(ctx, deviceID)
-			if agent == nil {
-				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
-				ch <- nil
-				return
-			}
-
-			resp, err := agent.commitImage(ctx, req)
-			if err != nil {
-				logger.Errorw(ctx, "commit-image-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-
-			err = dMgr.validateDeviceImageResponse(resp)
-			if err != nil {
-				logger.Errorf(ctx, "commit-image-failed", log.Fields{"device-id": deviceID, "error": err})
-				ch <- nil
-				return
-			}
-			ch <- resp.GetDeviceImageStates()
-		}(deviceID.GetId(), commitImageReq, respCh)
-
-	}
-
-	return dMgr.waitForAllResponses(ctx, "commit-image", respCh, len(request.GetDeviceId()))
-}
-
 func (dMgr *Manager) validateImageDownloadRequest(request *voltha.DeviceImageDownloadRequest) error {
 	if request == nil || request.Image == nil || len(request.DeviceId) == 0 {
 		return status.Errorf(codes.InvalidArgument, "invalid argument")
@@ -1979,3 +823,12 @@
 	}
 	return nil
 }
+
+func (dMgr *Manager) adapterRestartedHandler(ctx context.Context, endpoint string) error {
+	// Get the adapter corresponding to that endpoint
+	if a, _ := dMgr.adapterMgr.GetAdapterWithEndpoint(ctx, endpoint); a != nil {
+		return dMgr.adapterRestarted(ctx, a)
+	}
+	logger.Errorw(ctx, "restarted-adapter-not-found", log.Fields{"endpoint": endpoint})
+	return fmt.Errorf("restarted adapter at endpoint %s not found", endpoint)
+}
diff --git a/rw_core/core/device/manager_nbi.go b/rw_core/core/device/manager_nbi.go
new file mode 100644
index 0000000..9a16812
--- /dev/null
+++ b/rw_core/core/device/manager_nbi.go
@@ -0,0 +1,772 @@
+/*
+ * Copyright 2021-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.
+ */
+package device
+
+import (
+	"context"
+	"errors"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-go/rw_core/utils"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+// CreateDevice creates a new parent device in the data model
+func (dMgr *Manager) CreateDevice(ctx context.Context, device *voltha.Device) (*voltha.Device, error) {
+	if device.MacAddress == "" && device.GetHostAndPort() == "" {
+		logger.Errorf(ctx, "no-device-info-present")
+		return &voltha.Device{}, errors.New("no-device-info-present; MAC or HOSTIP&PORT")
+	}
+	ctx = utils.WithRPCMetadataContext(ctx, "CreateDevice")
+	logger.Debugw(ctx, "create-device", log.Fields{"device": *device})
+
+	deviceExist, err := dMgr.isParentDeviceExist(ctx, device)
+	if err != nil {
+		logger.Errorf(ctx, "failed-to-fetch-parent-device-info")
+		return nil, err
+	}
+	if deviceExist {
+		logger.Errorf(ctx, "device-is-pre-provisioned-already-with-same-ip-port-or-mac-address")
+		return nil, errors.New("device is already pre-provisioned")
+	}
+	logger.Debugw(ctx, "create-device", log.Fields{"device": device})
+
+	// Ensure this device is set as root
+	device.Root = true
+	// Create and start a device agent for that device
+	agent := newAgent(device, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.internalTimeout, dMgr.rpcTimeout)
+	device, err = agent.start(ctx, false, device)
+	if err != nil {
+		logger.Errorw(ctx, "fail-to-start-device", log.Fields{"device-id": agent.deviceID, "error": err})
+		return nil, err
+	}
+	dMgr.addDeviceAgentToMap(agent)
+	return device, nil
+}
+
+// EnableDevice activates a device by invoking the adopt_device API on the appropriate adapter
+func (dMgr *Manager) EnableDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "EnableDevice")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "enable-device", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+	return &empty.Empty{}, agent.enableDevice(ctx)
+}
+
+// DisableDevice disables a device along with any child device it may have
+func (dMgr *Manager) DisableDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "DisableDevice")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "disable-device", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+	return &empty.Empty{}, agent.disableDevice(ctx)
+}
+
+//RebootDevice invoked the reboot API to the corresponding adapter
+func (dMgr *Manager) RebootDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "RebootDevice")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "reboot-device", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+	return &empty.Empty{}, agent.rebootDevice(ctx)
+}
+
+// DeleteDevice removes a device from the data model
+func (dMgr *Manager) DeleteDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "DeleteDevice")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "delete-device", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+	return &empty.Empty{}, agent.deleteDevice(ctx)
+}
+
+// ForceDeleteDevice removes a device from the data model forcefully without successfully waiting for the adapters.
+func (dMgr *Manager) ForceDeleteDevice(ctx context.Context, id *voltha.ID) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ForceDeleteDevice")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "force-delete-device", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+	return &empty.Empty{}, agent.deleteDeviceForce(ctx)
+}
+
+// ListDevices retrieves the latest devices from the data model
+func (dMgr *Manager) ListDevices(ctx context.Context, _ *empty.Empty) (*voltha.Devices, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ListDevices")
+
+	logger.Debug(ctx, "list-devices")
+	result := &voltha.Devices{}
+
+	dMgr.deviceAgents.Range(func(key, value interface{}) bool {
+		result.Items = append(result.Items, value.(*Agent).device)
+		return true
+	})
+
+	logger.Debugw(ctx, "list-devices-end", log.Fields{"len": len(result.Items)})
+	return result, nil
+}
+
+// ListDeviceIds retrieves the latest device IDs information from the data model (memory data only)
+func (dMgr *Manager) ListDeviceIds(ctx context.Context, _ *empty.Empty) (*voltha.IDs, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ListDeviceIds")
+
+	logger.Debug(ctx, "list-device-ids")
+	// Report only device IDs that are in the device agent map
+	return dMgr.listDeviceIdsFromMap(), nil
+}
+
+// ReconcileDevices is a request to a voltha core to update its list of managed devices.  This will
+// trigger loading the devices along with their children and parent in memory
+func (dMgr *Manager) ReconcileDevices(ctx context.Context, ids *voltha.IDs) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ReconcileDevices")
+
+	numDevices := 0
+	if ids != nil {
+		numDevices = len(ids.Items)
+	}
+
+	logger.Debugw(ctx, "reconcile-devices", log.Fields{"num-devices": numDevices})
+	if ids != nil && len(ids.Items) != 0 {
+		toReconcile := len(ids.Items)
+		reconciled := 0
+		var err error
+		for _, id := range ids.Items {
+			if err = dMgr.load(ctx, id.Id); err != nil {
+				logger.Warnw(ctx, "failure-reconciling-device", log.Fields{"device-id": id.Id, "error": err})
+			} else {
+				reconciled++
+			}
+		}
+		if toReconcile != reconciled {
+			return nil, status.Errorf(codes.DataLoss, "less-device-reconciled-than-requested:%d/%d", reconciled, toReconcile)
+		}
+	} else {
+		return nil, status.Errorf(codes.InvalidArgument, "empty-list-of-ids")
+	}
+	return &empty.Empty{}, nil
+}
+
+// GetDevice exists primarily to implement the gRPC interface.
+// Internal functions should call getDeviceReadOnly instead.
+func (dMgr *Manager) GetDevice(ctx context.Context, id *voltha.ID) (*voltha.Device, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "GetDevice")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	return dMgr.getDeviceReadOnly(ctx, id.Id)
+}
+
+// convenience to avoid redefining
+var operationFailureResp = &common.OperationResp{Code: voltha.OperationResp_OPERATION_FAILURE}
+
+// DownloadImage execute an image download request
+func (dMgr *Manager) DownloadImage(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "DownloadImage")
+	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
+
+	logger.Debugw(ctx, "download-image", log.Fields{"device-id": img.Id, "image-name": img.Name})
+	agent := dMgr.getDeviceAgent(ctx, img.Id)
+	if agent == nil {
+		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
+	}
+	resp, err := agent.downloadImage(ctx, img)
+	if err != nil {
+		return operationFailureResp, err
+	}
+	return resp, nil
+}
+
+// CancelImageDownload cancels image download request
+func (dMgr *Manager) CancelImageDownload(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "CancelImageDownload")
+	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
+
+	logger.Debugw(ctx, "cancel-image-download", log.Fields{"device-id": img.Id, "image-name": img.Name})
+	agent := dMgr.getDeviceAgent(ctx, img.Id)
+	if agent == nil {
+		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
+	}
+	resp, err := agent.cancelImageDownload(ctx, img)
+	if err != nil {
+		return operationFailureResp, err
+	}
+	return resp, nil
+}
+
+// ActivateImageUpdate activates image update request
+func (dMgr *Manager) ActivateImageUpdate(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ActivateImageUpdate")
+	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
+
+	logger.Debugw(ctx, "activate-image-update", log.Fields{"device-id": img.Id, "image-name": img.Name})
+	agent := dMgr.getDeviceAgent(ctx, img.Id)
+	if agent == nil {
+		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
+	}
+	resp, err := agent.activateImage(ctx, img)
+	if err != nil {
+		return operationFailureResp, err
+	}
+	return resp, nil
+}
+
+// RevertImageUpdate reverts image update
+func (dMgr *Manager) RevertImageUpdate(ctx context.Context, img *voltha.ImageDownload) (*common.OperationResp, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "RevertImageUpdate")
+	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
+
+	logger.Debugw(ctx, "rever-image-update", log.Fields{"device-id": img.Id, "image-name": img.Name})
+	agent := dMgr.getDeviceAgent(ctx, img.Id)
+	if agent == nil {
+		return operationFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
+	}
+	resp, err := agent.revertImage(ctx, img)
+	if err != nil {
+		return operationFailureResp, err
+	}
+	return resp, nil
+}
+
+func (dMgr *Manager) DownloadImageToDevice(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
+	if err := dMgr.validateImageDownloadRequest(request); err != nil {
+		return nil, err
+	}
+
+	ctx = utils.WithRPCMetadataContext(ctx, "DownloadImageToDevice")
+	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
+
+	for index, deviceID := range request.DeviceId {
+		// Create download request per device
+		downloadReq := &voltha.DeviceImageDownloadRequest{
+			Image:             request.Image,
+			ActivateOnSuccess: request.ActivateOnSuccess,
+			CommitOnSuccess:   request.CommitOnSuccess,
+		}
+
+		//slice-out only single deviceID from the request
+		downloadReq.DeviceId = request.DeviceId[index : index+1]
+
+		go func(deviceID string, req *voltha.DeviceImageDownloadRequest, ch chan []*voltha.DeviceImageState) {
+			agent := dMgr.getDeviceAgent(ctx, deviceID)
+			if agent == nil {
+				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
+				ch <- nil
+				return
+			}
+
+			resp, err := agent.downloadImageToDevice(ctx, req)
+			if err != nil {
+				logger.Errorw(ctx, "download-image-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+
+			err = dMgr.validateDeviceImageResponse(resp)
+			if err != nil {
+				logger.Errorw(ctx, "download-image-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+			ch <- resp.GetDeviceImageStates()
+		}(deviceID.GetId(), downloadReq, respCh)
+
+	}
+
+	return dMgr.waitForAllResponses(ctx, "download-image-to-device", respCh, len(request.GetDeviceId()))
+}
+
+func (dMgr *Manager) GetImageStatus(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	if err := dMgr.validateImageRequest(request); err != nil {
+		return nil, err
+	}
+
+	ctx = utils.WithRPCMetadataContext(ctx, "GetImageStatus")
+
+	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
+	for index, deviceID := range request.DeviceId {
+		// Create status request per device
+		imageStatusReq := &voltha.DeviceImageRequest{
+			Version:         request.Version,
+			CommitOnSuccess: request.CommitOnSuccess,
+		}
+
+		//slice-out only single deviceID from the request
+		imageStatusReq.DeviceId = request.DeviceId[index : index+1]
+
+		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
+			agent := dMgr.getDeviceAgent(ctx, deviceID)
+			if agent == nil {
+				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
+				ch <- nil
+				return
+			}
+
+			resp, err := agent.getImageStatus(ctx, req)
+			if err != nil {
+				logger.Errorw(ctx, "get-image-status-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+
+			err = dMgr.validateDeviceImageResponse(resp)
+			if err != nil {
+				logger.Errorw(ctx, "get-image-status-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+			ch <- resp.GetDeviceImageStates()
+		}(deviceID.GetId(), imageStatusReq, respCh)
+
+	}
+
+	return dMgr.waitForAllResponses(ctx, "get-image-status", respCh, len(request.GetDeviceId()))
+}
+
+func (dMgr *Manager) AbortImageUpgradeToDevice(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	if err := dMgr.validateImageRequest(request); err != nil {
+		return nil, err
+	}
+
+	ctx = utils.WithRPCMetadataContext(ctx, "AbortImageUpgradeToDevice")
+	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
+
+	for index, deviceID := range request.DeviceId {
+		// Create abort request per device
+		abortImageReq := &voltha.DeviceImageRequest{
+			Version:         request.Version,
+			CommitOnSuccess: request.CommitOnSuccess,
+		}
+
+		//slice-out only single deviceID from the request
+		abortImageReq.DeviceId = request.DeviceId[index : index+1]
+
+		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
+			agent := dMgr.getDeviceAgent(ctx, deviceID)
+			if agent == nil {
+				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
+				ch <- nil
+				return
+			}
+
+			resp, err := agent.abortImageUpgradeToDevice(ctx, req)
+			if err != nil {
+				logger.Errorw(ctx, "abort-image-upgrade-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+
+			err = dMgr.validateDeviceImageResponse(resp)
+			if err != nil {
+				logger.Errorw(ctx, "abort-image-upgrade-to-device-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+			ch <- resp.GetDeviceImageStates()
+		}(deviceID.GetId(), abortImageReq, respCh)
+
+	}
+
+	return dMgr.waitForAllResponses(ctx, "abort-image-upgrade-to-device", respCh, len(request.GetDeviceId()))
+}
+
+func (dMgr *Manager) GetOnuImages(ctx context.Context, id *common.ID) (*voltha.OnuImages, error) {
+	if id == nil || id.Id == "" {
+		return nil, status.Errorf(codes.InvalidArgument, "empty device id")
+	}
+
+	ctx = utils.WithRPCMetadataContext(ctx, "GetOnuImages")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+
+	resp, err := agent.getOnuImages(ctx, id)
+	if err != nil {
+		return nil, err
+	}
+
+	logger.Debugw(ctx, "get-onu-images-result", log.Fields{"onu-image": resp.Items})
+
+	return resp, nil
+}
+
+func (dMgr *Manager) ActivateImage(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	if err := dMgr.validateImageRequest(request); err != nil {
+		return nil, err
+	}
+
+	ctx = utils.WithRPCMetadataContext(ctx, "ActivateImage")
+	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
+
+	for index, deviceID := range request.DeviceId {
+		// Create activate request per device
+		activateImageReq := &voltha.DeviceImageRequest{
+			Version:         request.Version,
+			CommitOnSuccess: request.CommitOnSuccess,
+		}
+
+		//slice-out only single deviceID from the request
+		activateImageReq.DeviceId = request.DeviceId[index : index+1]
+
+		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
+			agent := dMgr.getDeviceAgent(ctx, deviceID)
+			if agent == nil {
+				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
+				ch <- nil
+				return
+			}
+
+			resp, err := agent.activateImageOnDevice(ctx, req)
+			if err != nil {
+				logger.Errorw(ctx, "activate-image-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+
+			err = dMgr.validateDeviceImageResponse(resp)
+			if err != nil {
+				logger.Errorw(ctx, "activate-image-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+
+			ch <- resp.GetDeviceImageStates()
+		}(deviceID.GetId(), activateImageReq, respCh)
+
+	}
+
+	return dMgr.waitForAllResponses(ctx, "activate-image", respCh, len(request.GetDeviceId()))
+}
+
+func (dMgr *Manager) CommitImage(ctx context.Context, request *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	if err := dMgr.validateImageRequest(request); err != nil {
+		return nil, err
+	}
+
+	ctx = utils.WithRPCMetadataContext(ctx, "CommitImage")
+	respCh := make(chan []*voltha.DeviceImageState, len(request.GetDeviceId()))
+
+	for index, deviceID := range request.DeviceId {
+		// Create commit request per device
+		commitImageReq := &voltha.DeviceImageRequest{
+			Version:         request.Version,
+			CommitOnSuccess: request.CommitOnSuccess,
+		}
+		//slice-out only single deviceID from the request
+		commitImageReq.DeviceId = request.DeviceId[index : index+1]
+
+		go func(deviceID string, req *voltha.DeviceImageRequest, ch chan []*voltha.DeviceImageState) {
+			agent := dMgr.getDeviceAgent(ctx, deviceID)
+			if agent == nil {
+				logger.Errorw(ctx, "Not-found", log.Fields{"device-id": deviceID})
+				ch <- nil
+				return
+			}
+
+			resp, err := agent.commitImage(ctx, req)
+			if err != nil {
+				logger.Errorw(ctx, "commit-image-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+
+			err = dMgr.validateDeviceImageResponse(resp)
+			if err != nil {
+				logger.Errorf(ctx, "commit-image-failed", log.Fields{"device-id": deviceID, "error": err})
+				ch <- nil
+				return
+			}
+			ch <- resp.GetDeviceImageStates()
+		}(deviceID.GetId(), commitImageReq, respCh)
+
+	}
+
+	return dMgr.waitForAllResponses(ctx, "commit-image", respCh, len(request.GetDeviceId()))
+}
+
+// convenience to avoid redefining
+var imageDownloadFailureResp = &voltha.ImageDownload{DownloadState: voltha.ImageDownload_DOWNLOAD_UNKNOWN}
+
+// GetImageDownloadStatus returns status of image download
+func (dMgr *Manager) GetImageDownloadStatus(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "GetImageDownloadStatus")
+	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
+
+	logger.Debugw(ctx, "get-image-download-status", log.Fields{"device-id": img.Id, "image-name": img.Name})
+	agent := dMgr.getDeviceAgent(ctx, img.Id)
+	if agent == nil {
+		return imageDownloadFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
+	}
+	resp, err := agent.getImageDownloadStatus(ctx, img)
+	if err != nil {
+		return imageDownloadFailureResp, err
+	}
+	return resp, nil
+}
+
+// GetImageDownload returns image download
+func (dMgr *Manager) GetImageDownload(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "GetImageDownload")
+	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
+
+	logger.Debugw(ctx, "get-image-download", log.Fields{"device-id": img.Id, "image-name": img.Name})
+	agent := dMgr.getDeviceAgent(ctx, img.Id)
+	if agent == nil {
+		return imageDownloadFailureResp, status.Errorf(codes.NotFound, "%s", img.Id)
+	}
+	resp, err := agent.getImageDownload(ctx, img)
+	if err != nil {
+		return imageDownloadFailureResp, err
+	}
+	return resp, nil
+}
+
+// ListImageDownloads returns image downloads
+func (dMgr *Manager) ListImageDownloads(ctx context.Context, id *voltha.ID) (*voltha.ImageDownloads, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ListImageDownloads")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "list-image-downloads", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return &voltha.ImageDownloads{Items: []*voltha.ImageDownload{imageDownloadFailureResp}}, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+	resp, err := agent.listImageDownloads(ctx, id.Id)
+	if err != nil {
+		return &voltha.ImageDownloads{Items: []*voltha.ImageDownload{imageDownloadFailureResp}}, err
+	}
+	return resp, nil
+}
+
+// GetImages returns all images for a specific device entry
+func (dMgr *Manager) GetImages(ctx context.Context, id *voltha.ID) (*voltha.Images, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "GetImages")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "get-images", log.Fields{"device-id": id.Id})
+	device, err := dMgr.getDeviceReadOnly(ctx, id.Id)
+	if err != nil {
+		return nil, err
+	}
+	return device.Images, nil
+}
+
+// ListDevicePorts returns the ports details for a specific device entry
+func (dMgr *Manager) ListDevicePorts(ctx context.Context, id *voltha.ID) (*voltha.Ports, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ListDevicePorts")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	logger.Debugw(ctx, "list-device-ports", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "device-%s", id.Id)
+	}
+
+	ports := agent.listDevicePorts()
+	ctr, ret := 0, make([]*voltha.Port, len(ports))
+	for _, port := range ports {
+		ret[ctr] = port
+		ctr++
+	}
+	return &voltha.Ports{Items: ret}, nil
+}
+
+// ListDevicePmConfigs returns pm configs of device
+func (dMgr *Manager) ListDevicePmConfigs(ctx context.Context, id *voltha.ID) (*voltha.PmConfigs, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ListDevicePmConfigs")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", id.Id)
+	}
+	return agent.listPmConfigs(ctx)
+}
+
+// UpdateDevicePmConfigs updates the PM configs.  This is executed when the northbound gRPC API is invoked, typically
+// following a user action
+func (dMgr *Manager) UpdateDevicePmConfigs(ctx context.Context, configs *voltha.PmConfigs) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "UpdateDevicePmConfigs")
+	log.EnrichSpan(ctx, log.Fields{"device-id": configs.Id})
+
+	if configs.Id == "" {
+		return nil, status.Error(codes.FailedPrecondition, "invalid-device-Id")
+	}
+	agent := dMgr.getDeviceAgent(ctx, configs.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", configs.Id)
+	}
+	return &empty.Empty{}, agent.updatePmConfigs(ctx, configs)
+}
+
+// ListDeviceFlows returns the flow details for a specific device entry
+func (dMgr *Manager) ListDeviceFlows(ctx context.Context, id *voltha.ID) (*ofp.Flows, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ListDeviceFlows")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+	logger.Debugw(ctx, "list-device-flows", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "device-%s", id.Id)
+	}
+
+	flows := agent.listDeviceFlows()
+	ctr, ret := 0, make([]*ofp.OfpFlowStats, len(flows))
+	for _, flow := range flows {
+		ret[ctr] = flow
+		ctr++
+	}
+	return &ofp.Flows{Items: ret}, nil
+}
+
+// ListDeviceFlowGroups returns the flow group details for a specific device entry
+func (dMgr *Manager) ListDeviceFlowGroups(ctx context.Context, id *voltha.ID) (*voltha.FlowGroups, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "ListDeviceFlowGroups")
+	log.EnrichSpan(ctx, log.Fields{"device-id": id.Id})
+	logger.Debugw(ctx, "list-device-flow-groups", log.Fields{"device-id": id.Id})
+	agent := dMgr.getDeviceAgent(ctx, id.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "device-%s", id.Id)
+	}
+	groups := agent.listDeviceGroups()
+	ctr, ret := 0, make([]*ofp.OfpGroupEntry, len(groups))
+	for _, group := range groups {
+		ret[ctr] = group
+		ctr++
+	}
+	return &voltha.FlowGroups{Items: ret}, nil
+}
+
+func (dMgr *Manager) EnablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "EnablePort")
+	log.EnrichSpan(ctx, log.Fields{"device-id": port.DeviceId})
+
+	logger.Debugw(ctx, "enable-port", log.Fields{"device-id": port.DeviceId, "port-no": port.PortNo})
+	agent := dMgr.getDeviceAgent(ctx, port.DeviceId)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", port.DeviceId)
+	}
+	return &empty.Empty{}, agent.enablePort(ctx, port.PortNo)
+}
+
+func (dMgr *Manager) DisablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "DisablePort")
+	log.EnrichSpan(ctx, log.Fields{"device-id": port.DeviceId})
+
+	logger.Debugw(ctx, "disable-port", log.Fields{"device-id": port.DeviceId, "port-no": port.PortNo})
+	agent := dMgr.getDeviceAgent(ctx, port.DeviceId)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", port.DeviceId)
+	}
+	return &empty.Empty{}, agent.disablePort(ctx, port.PortNo)
+}
+
+func (dMgr *Manager) GetExtValue(ctx context.Context, value *voltha.ValueSpecifier) (*voltha.ReturnValues, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "GetExtValue")
+	log.EnrichSpan(ctx, log.Fields{"device-id": value.Id})
+
+	logger.Debugw(ctx, "get-ext-value", log.Fields{"onu-id": value.Id})
+	cDevice, err := dMgr.getDeviceReadOnly(ctx, value.Id)
+	if err != nil {
+		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+	}
+	pDevice, err := dMgr.getDeviceReadOnly(ctx, cDevice.ParentId)
+	if err != nil {
+		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+	}
+	if agent := dMgr.getDeviceAgent(ctx, cDevice.ParentId); agent != nil {
+		resp, err := agent.getExtValue(ctx, pDevice, cDevice, value)
+		if err != nil {
+			return nil, err
+		}
+		logger.Debugw(ctx, "get-ext-value-result", log.Fields{"result": resp})
+		return resp, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", value.Id)
+
+}
+
+// SetExtValue  set some given configs or value
+func (dMgr *Manager) SetExtValue(ctx context.Context, value *voltha.ValueSet) (*empty.Empty, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "SetExtValue")
+	logger.Debugw(ctx, "set-ext-value", log.Fields{"onu-id": value.Id})
+
+	device, err := dMgr.getDeviceReadOnly(ctx, value.Id)
+	if err != nil {
+		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+	}
+	if agent := dMgr.getDeviceAgent(ctx, device.Id); agent != nil {
+		resp, err := agent.setExtValue(ctx, device, value)
+		if err != nil {
+			return nil, err
+		}
+		logger.Debugw(ctx, "set-ext-value-result", log.Fields{"result": resp})
+		return resp, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", value.Id)
+
+}
+
+func (dMgr *Manager) StartOmciTestAction(ctx context.Context, request *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "StartOmciTestAction")
+	log.EnrichSpan(ctx, log.Fields{"device-id": request.Id})
+
+	logger.Debugw(ctx, "start-omci-test-action", log.Fields{"device-id": request.Id, "uuid": request.Uuid})
+	agent := dMgr.getDeviceAgent(ctx, request.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", request.Id)
+	}
+	return agent.startOmciTest(ctx, request)
+}
+
+func (dMgr *Manager) SimulateAlarm(ctx context.Context, simulateReq *voltha.SimulateAlarmRequest) (*common.OperationResp, error) {
+	ctx = utils.WithRPCMetadataContext(ctx, "SimulateAlarm")
+
+	logger.Debugw(ctx, "simulate-alarm", log.Fields{"id": simulateReq.Id, "indicator": simulateReq.Indicator, "intf-id": simulateReq.IntfId,
+		"port-type-name": simulateReq.PortTypeName, "onu-device-id": simulateReq.OnuDeviceId, "inverse-bit-error-rate": simulateReq.InverseBitErrorRate,
+		"drift": simulateReq.Drift, "new-eqd": simulateReq.NewEqd, "onu-serial-number": simulateReq.OnuSerialNumber, "operation": simulateReq.Operation})
+	agent := dMgr.getDeviceAgent(ctx, simulateReq.Id)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", simulateReq.Id)
+	}
+	if err := agent.simulateAlarm(ctx, simulateReq); err != nil {
+		return nil, err
+	}
+	return &common.OperationResp{Code: common.OperationResp_OPERATION_SUCCESS}, nil
+}
diff --git a/rw_core/core/device/manager_sbi.go b/rw_core/core/device/manager_sbi.go
new file mode 100644
index 0000000..53d2a24
--- /dev/null
+++ b/rw_core/core/device/manager_sbi.go
@@ -0,0 +1,529 @@
+/*
+ * Copyright 2021-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.
+ */
+package device
+
+import (
+	"context"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-go/rw_core/utils"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+func (dMgr *Manager) PortCreated(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "PortCreated")
+
+	logger.Debugw(ctx, "port-created", log.Fields{"port": port})
+
+	agent := dMgr.getDeviceAgent(ctx, port.DeviceId)
+	if agent != nil {
+		if err := agent.addPort(ctx, port); err != nil {
+			return nil, err
+		}
+		//	Setup peer ports in its own routine
+		go func() {
+			if err := dMgr.addPeerPort(log.WithSpanFromContext(context.Background(), ctx), port.DeviceId, port); err != nil {
+				logger.Errorw(ctx, "unable-to-add-peer-port", log.Fields{"error": err, "device-id": port.DeviceId})
+			}
+		}()
+		return &empty.Empty{}, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", port.DeviceId)
+}
+
+func (dMgr *Manager) DeviceUpdate(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "DeviceUpdate")
+	logger.Debugw(ctx, "device-update", log.Fields{"device-id": device.Id, "device": device})
+
+	if agent := dMgr.getDeviceAgent(ctx, device.Id); agent != nil {
+		if err := agent.updateDeviceUsingAdapterData(ctx, device); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", device.Id)
+}
+
+func (dMgr *Manager) DeviceStateUpdate(ctx context.Context, ds *ic.DeviceStateFilter) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "DeviceStateUpdate")
+	logger.Debugw(ctx, "device-state-update", log.Fields{"device-id": ds.DeviceId, "operStatus": ds.OperStatus, "connStatus": ds.ConnStatus})
+
+	if agent := dMgr.getDeviceAgent(ctx, ds.DeviceId); agent != nil {
+		if err := agent.updateDeviceStatus(ctx, ds.OperStatus, ds.ConnStatus); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", ds.DeviceId)
+}
+
+func (dMgr *Manager) ChildDeviceDetected(ctx context.Context, dd *ic.DeviceDiscovery) (*voltha.Device, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "ChildDeviceDetected")
+	logger.Debugw(ctx, "child-device-detected",
+		log.Fields{
+			"parent-device-id": dd.ParentId,
+			"parentPortNo":     dd.ParentPortNo,
+			"deviceType":       dd.ChildDeviceType,
+			"channelId":        dd.ChannelId,
+			"vendorId":         dd.VendorId,
+			"serialNumber":     dd.SerialNumber,
+			"onuId":            dd.OnuId,
+		})
+
+	var err error
+	if dd.ChildDeviceType == "" && dd.VendorId != "" {
+		logger.Debug(ctx, "device-type-is-nil-fetching-device-type")
+		if dd.ChildDeviceType, err = dMgr.adapterMgr.GetAdapterTypeByVendorID(dd.VendorId); err != nil {
+			return nil, err
+		}
+	}
+	//if no match found for the vendorid,report adapter with the custom error message
+	if dd.ChildDeviceType == "" {
+		logger.Errorw(ctx, "failed-to-fetch-adapter-name ", log.Fields{"vendorId": dd.VendorId})
+		return nil, status.Errorf(codes.NotFound, "%s", dd.VendorId)
+	}
+
+	// Create the ONU device
+	childDevice := &voltha.Device{}
+	childDevice.Type = dd.ChildDeviceType
+	childDevice.ParentId = dd.ParentId
+	childDevice.ParentPortNo = uint32(dd.ParentPortNo)
+	childDevice.VendorId = dd.VendorId
+	childDevice.SerialNumber = dd.SerialNumber
+	childDevice.Root = false
+
+	// Get parent device type
+	pAgent := dMgr.getDeviceAgent(ctx, dd.ParentId)
+	if pAgent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", dd.ParentId)
+	}
+	if pAgent.deviceType == "" {
+		pDevice, err := pAgent.getDeviceReadOnly(ctx)
+		logger.Errorw(ctx, "device-type-not-set", log.Fields{"parent-device": pDevice, "error": err})
+		return nil, status.Errorf(codes.FailedPrecondition, "device Type not set %s", dd.ParentId)
+	}
+
+	if device, err := dMgr.GetChildDevice(ctx, &ic.ChildDeviceFilter{
+		ParentId:     dd.ParentId,
+		SerialNumber: dd.SerialNumber,
+		OnuId:        dd.OnuId,
+		ParentPortNo: dd.ParentPortNo}); err == nil {
+		logger.Warnw(ctx, "child-device-exists", log.Fields{"parent-device-id": dd.ParentId, "serialNumber": dd.SerialNumber})
+		return device, status.Errorf(codes.AlreadyExists, "%s", dd.SerialNumber)
+	}
+
+	//Get parent endpoint
+	pEndPoint, err := dMgr.adapterMgr.GetAdapterEndpoint(ctx, pAgent.deviceID, pAgent.deviceType)
+	if err != nil {
+		logger.Errorw(ctx, "endpoint-error", log.Fields{"error": err, "parent-id": pAgent.deviceID, "parent-device-type": pAgent.deviceType})
+		return nil, status.Errorf(codes.NotFound, "parent-endpoint-%s", dd.ParentId)
+	}
+
+	childDevice.ProxyAddress = &voltha.Device_ProxyAddress{DeviceId: dd.ParentId, DeviceType: pAgent.deviceType, ChannelId: dd.ChannelId, OnuId: dd.OnuId, AdapterEndpoint: pEndPoint}
+
+	// Set child device ID -- needed to get the device endpoint
+	childDevice.Id = utils.CreateDeviceID()
+
+	// Set the child adapter endpoint
+	childDevice.AdapterEndpoint, err = dMgr.adapterMgr.GetAdapterEndpoint(ctx, childDevice.Id, childDevice.Type)
+	if err != nil {
+		return nil, status.Errorf(codes.NotFound, "child-endpoint-%s", childDevice.Id)
+	}
+
+	// Create and start a device agent for that device
+	agent := newAgent(childDevice, dMgr, dMgr.dbPath, dMgr.dProxy, dMgr.internalTimeout, dMgr.rpcTimeout)
+	insertedChildDevice, err := agent.start(ctx, false, childDevice)
+	if err != nil {
+		logger.Errorw(ctx, "error-starting-child-device", log.Fields{"parent-device-id": childDevice.ParentId, "child-device-id": agent.deviceID, "error": err})
+		return nil, err
+	}
+	dMgr.addDeviceAgentToMap(agent)
+
+	// Activate the child device
+	if agent = dMgr.getDeviceAgent(ctx, agent.deviceID); agent != nil {
+		go func() {
+			err := agent.enableDevice(utils.WithSpanAndRPCMetadataFromContext(ctx))
+			if err != nil {
+				logger.Errorw(ctx, "unable-to-enable-device", log.Fields{"error": err, "device-id": agent.deviceID})
+			}
+		}()
+	}
+
+	return insertedChildDevice, nil
+}
+
+func (dMgr *Manager) GetChildDevice(ctx context.Context, df *ic.ChildDeviceFilter) (*voltha.Device, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "GetChildDevice")
+	logger.Debugw(ctx, "get-child-device", log.Fields{"filter": df})
+
+	parentDevicePorts, err := dMgr.listDevicePorts(ctx, df.ParentId)
+	if err != nil {
+		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+	}
+	childDeviceIds := dMgr.getAllChildDeviceIds(ctx, parentDevicePorts)
+	if len(childDeviceIds) == 0 {
+		logger.Debugw(ctx, "no-child-devices", log.Fields{"parent-device-id": df.ParentId, "serial-number": df.SerialNumber, "onu-id": df.OnuId})
+		return nil, status.Errorf(codes.NotFound, "%s", df.ParentId)
+	}
+
+	var foundChildDevice *voltha.Device
+	for childDeviceID := range childDeviceIds {
+		var found bool
+		if searchDevice, err := dMgr.getDeviceReadOnly(ctx, childDeviceID); err == nil {
+
+			foundOnuID := false
+			if searchDevice.ProxyAddress.OnuId == uint32(df.OnuId) {
+				if searchDevice.ParentPortNo == uint32(df.ParentPortNo) {
+					logger.Debugw(ctx, "found-child-by-onu-id", log.Fields{"parent-device-id": df.ParentId, "onuId": df.OnuId})
+					foundOnuID = true
+				}
+			}
+
+			foundSerialNumber := false
+			if searchDevice.SerialNumber == df.SerialNumber {
+				logger.Debugw(ctx, "found-child-by-serial-number", log.Fields{"parent-device-id": df.ParentId, "serialNumber": df.SerialNumber})
+				foundSerialNumber = true
+			}
+
+			// if both onuId and serialNumber are provided both must be true for the device to be found
+			// otherwise whichever one found a match is good enough
+			if df.OnuId > 0 && df.SerialNumber != "" {
+				found = foundOnuID && foundSerialNumber
+			} else {
+				found = foundOnuID || foundSerialNumber
+			}
+
+			if found {
+				foundChildDevice = searchDevice
+				break
+			}
+		}
+	}
+
+	if foundChildDevice != nil {
+		logger.Debugw(ctx, "child-device-found", log.Fields{"parent-device-id": df.ParentId, "foundChildDevice": foundChildDevice})
+		return foundChildDevice, nil
+	}
+
+	logger.Debugw(ctx, "child-device-not-found", log.Fields{"parent-device-id": df.ParentId,
+		"serialNumber": df.SerialNumber, "onuId": df.OnuId, "parentPortNo": df.ParentPortNo})
+	return nil, status.Errorf(codes.NotFound, "%s", df.ParentId)
+}
+
+// PortsStateUpdate updates the operational status of all ports on the device
+func (dMgr *Manager) PortsStateUpdate(ctx context.Context, ps *ic.PortStateFilter) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "PortsStateUpdate")
+	logger.Debugw(ctx, "ports-state-update", log.Fields{"device-id": ps.DeviceId})
+
+	agent := dMgr.getDeviceAgent(ctx, ps.DeviceId)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", ps.DeviceId)
+	}
+	if ps.OperStatus != voltha.OperStatus_ACTIVE && ps.OperStatus != voltha.OperStatus_UNKNOWN {
+		return nil, status.Error(codes.Unimplemented, "state-change-not-implemented")
+	}
+	if err := agent.updatePortsOperState(ctx, ps.PortTypeFilter, ps.OperStatus); err != nil {
+		logger.Warnw(ctx, "ports-state-update-failed", log.Fields{"device-id": ps.DeviceId, "error": err})
+		return nil, err
+	}
+	return &empty.Empty{}, nil
+}
+
+//ChildDevicesLost is invoked by an adapter to indicate that a parent device is in a state (Disabled) where it
+//cannot manage the child devices.  This will trigger the Core to disable all the child devices.
+func (dMgr *Manager) ChildDevicesLost(ctx context.Context, parentID *common.ID) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "ChildDevicesLost")
+	logger.Debugw(ctx, "child-devices-lost", log.Fields{"parent-id": parentID.Id})
+
+	parentDevice, err := dMgr.getDeviceReadOnly(ctx, parentID.Id)
+	if err != nil {
+		logger.Warnw(ctx, "failed-getting-device", log.Fields{"parent-device-id": parentID.Id, "error": err})
+		return nil, err
+	}
+	if err = dMgr.DisableAllChildDevices(ctx, parentDevice); err != nil {
+		return nil, err
+	}
+	return &empty.Empty{}, nil
+}
+
+//ChildDevicesDetected is invoked by an adapter when child devices are found, typically after after a
+// disable/enable sequence.  This will trigger the Core to Enable all the child devices of that parent.
+func (dMgr *Manager) ChildDevicesDetected(ctx context.Context, parentDeviceID *common.ID) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "ChildDevicesDetected")
+	logger.Debugw(ctx, "child-devices-detected", log.Fields{"parent-device-id": parentDeviceID})
+
+	parentDevicePorts, err := dMgr.listDevicePorts(ctx, parentDeviceID.Id)
+	if err != nil {
+		logger.Warnw(ctx, "failed-getting-device", log.Fields{"device-id": parentDeviceID.Id, "error": err})
+		return nil, err
+	}
+	childDeviceIds := dMgr.getAllChildDeviceIds(ctx, parentDevicePorts)
+	if len(childDeviceIds) == 0 {
+		logger.Debugw(ctx, "no-child-device", log.Fields{"parent-device-id": parentDeviceID.Id})
+	}
+	allChildEnableRequestSent := true
+	for childDeviceID := range childDeviceIds {
+		if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
+			// Run the children re-registration in its own routine
+			go func(ctx context.Context) {
+				err = agent.enableDevice(ctx)
+				if err != nil {
+					logger.Errorw(ctx, "unable-to-enable-device", log.Fields{"error": err})
+				}
+			}(log.WithSpanFromContext(context.Background(), ctx))
+		} else {
+			err = status.Errorf(codes.Unavailable, "no agent for child device %s", childDeviceID)
+			logger.Errorw(ctx, "no-child-device-agent", log.Fields{"parent-device-id": parentDeviceID.Id, "childId": childDeviceID})
+			allChildEnableRequestSent = false
+		}
+	}
+	if !allChildEnableRequestSent {
+		return nil, err
+	}
+	return &empty.Empty{}, nil
+}
+
+// GetChildDeviceWithProxyAddress will return a device based on proxy address
+func (dMgr *Manager) GetChildDeviceWithProxyAddress(ctx context.Context, proxyAddress *voltha.Device_ProxyAddress) (*voltha.Device, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "GetChildDeviceWithProxyAddress")
+
+	logger.Debugw(ctx, "get-child-device-with-proxy-address", log.Fields{"proxyAddress": proxyAddress})
+
+	parentDevicePorts, err := dMgr.listDevicePorts(ctx, proxyAddress.DeviceId)
+	if err != nil {
+		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+	}
+	childDeviceIds := dMgr.getAllChildDeviceIds(ctx, parentDevicePorts)
+	if len(childDeviceIds) == 0 {
+		logger.Debugw(ctx, "no-child-devices", log.Fields{"parent-device-id": proxyAddress.DeviceId})
+		return nil, status.Errorf(codes.NotFound, "%s", proxyAddress)
+	}
+
+	var foundChildDevice *voltha.Device
+	for childDeviceID := range childDeviceIds {
+		if searchDevice, err := dMgr.getDeviceReadOnly(ctx, childDeviceID); err == nil {
+			if searchDevice.ProxyAddress == proxyAddress {
+				foundChildDevice = searchDevice
+				break
+			}
+		}
+	}
+
+	if foundChildDevice != nil {
+		logger.Debugw(ctx, "child-device-found", log.Fields{"proxyAddress": proxyAddress})
+		return foundChildDevice, nil
+	}
+
+	logger.Warnw(ctx, "child-device-not-found", log.Fields{"proxyAddress": proxyAddress})
+	return nil, status.Errorf(codes.NotFound, "%s", proxyAddress)
+}
+
+func (dMgr *Manager) GetPorts(ctx context.Context, pf *ic.PortFilter) (*voltha.Ports, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "GetPorts")
+	logger.Debugw(ctx, "get-ports", log.Fields{"device-id": pf.DeviceId, "portType": pf.PortType})
+
+	agent := dMgr.getDeviceAgent(ctx, pf.DeviceId)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "%s", pf.DeviceId)
+	}
+	return agent.getPorts(ctx, pf.PortType), nil
+}
+
+func (dMgr *Manager) GetChildDevices(ctx context.Context, parentDeviceID *common.ID) (*voltha.Devices, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "GetChildDevices")
+
+	logger.Debugw(ctx, "get-child-devices", log.Fields{"parent-device-id": parentDeviceID.Id})
+	return dMgr.getAllChildDevices(ctx, parentDeviceID.Id)
+}
+
+func (dMgr *Manager) ChildrenStateUpdate(ctx context.Context, ds *ic.DeviceStateFilter) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "ChildrenStateUpdate")
+	logger.Debugw(ctx, "children-state-update", log.Fields{"parent-device-id": ds.ParentDeviceId, "operStatus": ds.OperStatus, "connStatus": ds.ConnStatus})
+
+	parentDevicePorts, err := dMgr.listDevicePorts(ctx, ds.ParentDeviceId)
+	if err != nil {
+		return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+	}
+	for childDeviceID := range dMgr.getAllChildDeviceIds(ctx, parentDevicePorts) {
+		if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
+			if err = agent.updateDeviceStatus(ctx, ds.OperStatus, ds.ConnStatus); err != nil {
+				return nil, status.Errorf(codes.Aborted, "childDevice:%s, error:%s", childDeviceID, err.Error())
+			}
+		}
+	}
+	return &empty.Empty{}, nil
+}
+
+func (dMgr *Manager) PortStateUpdate(ctx context.Context, ps *ic.PortState) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "PortStateUpdate")
+	logger.Debugw(ctx, "port-state-update", log.Fields{"device-id": ps.DeviceId, "portType": ps.PortType, "portNo": ps.PortNo, "operStatus": ps.OperStatus})
+
+	if agent := dMgr.getDeviceAgent(ctx, ps.DeviceId); agent != nil {
+		if err := agent.updatePortState(ctx, ps.PortType, ps.PortNo, ps.OperStatus); err != nil {
+			logger.Errorw(ctx, "updating-port-state-failed", log.Fields{"device-id": ps.DeviceId, "portNo": ps.PortNo, "error": err})
+			return nil, err
+		}
+		// Notify the logical device manager to change the port state
+		// Do this for NNI and UNIs only. PON ports are not known by logical device
+		if ps.PortType == voltha.Port_ETHERNET_NNI || ps.PortType == voltha.Port_ETHERNET_UNI {
+			go func() {
+				err := dMgr.logicalDeviceMgr.updatePortState(log.WithSpanFromContext(context.Background(), ctx), ps.DeviceId, ps.PortNo, ps.OperStatus)
+				if err != nil {
+					// While we want to handle (catch) and log when
+					// an update to a port was not able to be
+					// propagated to the logical port, we can report
+					// it as a warning and not an error because it
+					// doesn't stop or modify processing.
+					// TODO: VOL-2707
+					logger.Warnw(ctx, "unable-to-update-logical-port-state", log.Fields{"error": err})
+				}
+			}()
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", ps.DeviceId)
+}
+
+func (dMgr *Manager) DeleteAllPorts(ctx context.Context, deviceID *common.ID) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "DeleteAllPorts")
+	logger.Debugw(ctx, "delete-all-ports", log.Fields{"device-id": deviceID.Id})
+
+	if agent := dMgr.getDeviceAgent(ctx, deviceID.Id); agent != nil {
+		if err := agent.deleteAllPorts(ctx); err != nil {
+			return nil, err
+		}
+		// Notify the logical device manager to remove all logical ports, if needed.
+		// At this stage the device itself may gave been deleted already at a DeleteAllPorts
+		// typically is part of a device deletion phase.
+		if device, err := dMgr.getDeviceReadOnly(ctx, deviceID.Id); err == nil {
+			go func() {
+				subCtx := utils.WithSpanAndRPCMetadataFromContext(ctx)
+				if err := dMgr.logicalDeviceMgr.deleteAllLogicalPorts(subCtx, device); err != nil {
+					logger.Errorw(ctx, "unable-to-delete-logical-ports", log.Fields{"error": err})
+				}
+			}()
+		} else {
+			logger.Warnw(ctx, "failed-to-retrieve-device", log.Fields{"device-id": deviceID.Id})
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", deviceID.Id)
+}
+
+// GetDevicePort returns the port details for a specific device port entry
+func (dMgr *Manager) GetDevicePort(ctx context.Context, pf *ic.PortFilter) (*voltha.Port, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "GetDevicePort")
+	logger.Debugw(ctx, "get-device-port", log.Fields{"device-id": pf.DeviceId})
+
+	agent := dMgr.getDeviceAgent(ctx, pf.DeviceId)
+	if agent == nil {
+		return nil, status.Errorf(codes.NotFound, "device-%s", pf.DeviceId)
+	}
+	return agent.getDevicePort(pf.Port)
+}
+
+// DevicePMConfigUpdate updates the pm configs as defined by the adapter.
+func (dMgr *Manager) DevicePMConfigUpdate(ctx context.Context, pc *voltha.PmConfigs) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "DevicePMConfigUpdate")
+	logger.Debugw(ctx, "device-pm-config-update", log.Fields{"device-id": pc.Id})
+
+	if pc.Id == "" {
+		return nil, status.Errorf(codes.FailedPrecondition, "invalid-device-Id")
+	}
+	if agent := dMgr.getDeviceAgent(ctx, pc.Id); agent != nil {
+		if err := agent.initPmConfigs(ctx, pc); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", pc.Id)
+}
+
+// SendPacketIn receives packetIn request from adapter
+func (dMgr *Manager) SendPacketIn(ctx context.Context, pi *ic.PacketIn) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "SendPacketIn")
+	logger.Debugw(ctx, "packet-in", log.Fields{"device-id": pi.DeviceId, "port": pi.Port})
+
+	// Get the logical device Id based on the deviceId
+	var device *voltha.Device
+	var err error
+	if device, err = dMgr.getDeviceReadOnly(ctx, pi.DeviceId); err != nil {
+		logger.Errorw(ctx, "device-not-found", log.Fields{"device-id": pi.DeviceId})
+		return nil, err
+	}
+	if !device.Root {
+		logger.Errorw(ctx, "device-not-root", log.Fields{"device-id": pi.DeviceId})
+		return nil, status.Errorf(codes.FailedPrecondition, "%s", pi.DeviceId)
+	}
+
+	if err := dMgr.logicalDeviceMgr.packetIn(ctx, device.ParentId, pi.Port, pi.Packet); err != nil {
+		return nil, err
+	}
+	return &empty.Empty{}, nil
+}
+
+func (dMgr *Manager) DeviceReasonUpdate(ctx context.Context, dr *ic.DeviceReason) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "DeviceReasonUpdate")
+	logger.Debugw(ctx, "update-device-reason", log.Fields{"device-id": dr.DeviceId, "reason": dr.Reason})
+
+	if agent := dMgr.getDeviceAgent(ctx, dr.DeviceId); agent != nil {
+		if err := agent.updateDeviceReason(ctx, dr.Reason); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, status.Errorf(codes.NotFound, "%s", dr.DeviceId)
+}
+
+func (dMgr *Manager) ReconcileChildDevices(ctx context.Context, parentDeviceID *common.ID) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "ReconcileChildDevices")
+	logger.Debugw(ctx, "reconcile-child-devices", log.Fields{"device-id": parentDeviceID.Id})
+
+	numberOfDevicesToReconcile := 0
+	dMgr.deviceAgents.Range(func(key, value interface{}) bool {
+		deviceAgent, ok := value.(*Agent)
+		if ok && deviceAgent.parentID == parentDeviceID.Id {
+			go deviceAgent.ReconcileDevice(utils.WithNewSpanAndRPCMetadataContext(ctx, "ReconcileChildDevices"))
+			numberOfDevicesToReconcile++
+		}
+		return true
+	})
+	logger.Debugw(ctx, "reconciling-child-devices-initiated", log.Fields{"parent-device-id": parentDeviceID.Id, "number-of-child-devices-to-reconcile": numberOfDevicesToReconcile})
+	return &empty.Empty{}, nil
+}
+
+func (dMgr *Manager) UpdateImageDownload(ctx context.Context, img *voltha.ImageDownload) (*empty.Empty, error) {
+	ctx = utils.WithNewSpanAndRPCMetadataContext(ctx, "UpdateImageDownload")
+	log.EnrichSpan(ctx, log.Fields{"device-id": img.Id})
+
+	logger.Debugw(ctx, "update-image-download", log.Fields{"device-id": img.Id, "image-name": img.Name})
+
+	if agent := dMgr.getDeviceAgent(ctx, img.Id); agent != nil {
+		if err := agent.updateImageDownload(ctx, img); err != nil {
+			logger.Debugw(ctx, "update-image-download-failed", log.Fields{"err": err, "image-name": img.Name})
+			return nil, err
+		}
+	} else {
+		return nil, status.Errorf(codes.NotFound, "%s", img.Id)
+	}
+	return &empty.Empty{}, nil
+}
diff --git a/rw_core/core/device/manager_state_callback.go b/rw_core/core/device/manager_state_callback.go
new file mode 100644
index 0000000..6bd42eb
--- /dev/null
+++ b/rw_core/core/device/manager_state_callback.go
@@ -0,0 +1,159 @@
+/*
+ * Copyright 2021-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.
+ */
+package device
+
+import (
+	"context"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+//
+//CreateLogicalDevice creates logical device in core
+func (dMgr *Manager) CreateLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
+	logger.Info(ctx, "create-logical-device")
+	// Verify whether the logical device has already been created
+	if cDevice.ParentId != "" {
+		logger.Debugw(ctx, "parent-device-already-exist", log.Fields{"device-id": cDevice.Id, "logical-device-id": cDevice.Id})
+		return nil
+	}
+	var err error
+	if _, err = dMgr.logicalDeviceMgr.createLogicalDevice(ctx, cDevice); err != nil {
+		logger.Warnw(ctx, "create-logical-device-error", log.Fields{"device": cDevice})
+		return err
+	}
+	return nil
+}
+
+// DeleteLogicalDevice deletes logical device from core
+func (dMgr *Manager) DeleteLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
+	logger.Info(ctx, "delete-logical-device")
+	var err error
+	if err = dMgr.logicalDeviceMgr.deleteLogicalDevice(ctx, cDevice); err != nil {
+		logger.Warnw(ctx, "delete-logical-device-error", log.Fields{"device-id": cDevice.Id})
+		return err
+	}
+	// Remove the logical device Id from the parent device
+	logicalID := ""
+	dMgr.UpdateDeviceAttribute(ctx, cDevice.Id, "ParentId", logicalID)
+	return nil
+}
+
+// DeleteLogicalPorts removes the logical ports associated with that deviceId
+func (dMgr *Manager) DeleteLogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
+	logger.Debugw(ctx, "delete-all-logical-ports", log.Fields{"device-id": cDevice.Id})
+	if err := dMgr.logicalDeviceMgr.deleteLogicalPorts(ctx, cDevice.Id); err != nil {
+		// Just log the error.   The logical device or port may already have been deleted before this callback is invoked.
+		logger.Warnw(ctx, "delete-logical-ports-error", log.Fields{"device-id": cDevice.Id, "error": err})
+	}
+	return nil
+}
+
+// SetupUNILogicalPorts creates UNI ports on the logical device that represents a child UNI interface
+func (dMgr *Manager) SetupUNILogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
+	logger.Info(ctx, "setup-uni-logical-ports")
+	cDevicePorts, err := dMgr.listDevicePorts(ctx, cDevice.Id)
+	if err != nil {
+		return err
+	}
+	if err := dMgr.logicalDeviceMgr.setupUNILogicalPorts(ctx, cDevice, cDevicePorts); err != nil {
+		logger.Warnw(ctx, "setup-uni-logical-ports-error", log.Fields{"device": cDevice, "err": err})
+		return err
+	}
+	return nil
+}
+
+// RunPostDeviceDelete removes any reference of this device
+func (dMgr *Manager) RunPostDeviceDelete(ctx context.Context, cDevice *voltha.Device) error {
+	logger.Infow(ctx, "run-post-device-delete", log.Fields{"device-id": cDevice.Id})
+	dMgr.stopManagingDevice(ctx, cDevice.Id)
+	return nil
+}
+
+//DeleteAllLogicalPorts is invoked as a callback when the parent device's connection status moves to UNREACHABLE
+func (dMgr *Manager) DeleteAllLogicalPorts(ctx context.Context, parentDevice *voltha.Device) error {
+	logger.Debugw(ctx, "delete-all-logical-ports", log.Fields{"parent-device-id": parentDevice.Id})
+	if err := dMgr.logicalDeviceMgr.deleteAllLogicalPorts(ctx, parentDevice); err != nil {
+		// Just log error as logical device may already have been deleted
+		logger.Warnw(ctx, "delete-all-logical-ports-fail", log.Fields{"parent-device-id": parentDevice.Id, "error": err})
+	}
+	return nil
+}
+
+//DeleteAllChildDevices is invoked as a callback when the parent device is deleted
+func (dMgr *Manager) DeleteAllChildDevices(ctx context.Context, parentCurrDevice *voltha.Device) error {
+	logger.Debugw(ctx, "delete-all-child-devices", log.Fields{"parent-device-id": parentCurrDevice.Id})
+	force := false
+	// Get the parent device Transient state, if its FORCE_DELETED(go for force delete for child devices)
+	// So in cases when this handler is getting called other than DELETE operation, no force option would be used.
+	agent := dMgr.getDeviceAgent(ctx, parentCurrDevice.Id)
+	if agent == nil {
+		return status.Errorf(codes.NotFound, "%s", parentCurrDevice.Id)
+	}
+
+	force = agent.getTransientState() == core.DeviceTransientState_FORCE_DELETING
+
+	ports, _ := dMgr.listDevicePorts(ctx, parentCurrDevice.Id)
+	for childDeviceID := range dMgr.getAllChildDeviceIds(ctx, ports) {
+		if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
+			logger.Debugw(ctx, "invoking-delete-device", log.Fields{"device-id": childDeviceID, "force-delete": force})
+			if force {
+				if err := agent.deleteDeviceForce(ctx); err != nil {
+					logger.Warnw(ctx, "failure-delete-device-force", log.Fields{"device-id": childDeviceID,
+						"error": err.Error()})
+				}
+			} else {
+				if err := agent.deleteDevice(ctx); err != nil {
+					logger.Warnw(ctx, "failure-delete-device", log.Fields{"device-id": childDeviceID,
+						"error": err.Error()})
+				}
+			}
+			// No further action is required here.  The deleteDevice will change the device state where the resulting
+			// callback will take care of cleaning the child device agent.
+		}
+	}
+	return nil
+}
+
+//DeleteAllDeviceFlows is invoked as a callback when the parent device's connection status moves to UNREACHABLE
+func (dMgr *Manager) DeleteAllDeviceFlows(ctx context.Context, parentDevice *voltha.Device) error {
+	logger.Debugw(ctx, "delete-all-device-flows", log.Fields{"parent-device-id": parentDevice.Id})
+	if agent := dMgr.getDeviceAgent(ctx, parentDevice.Id); agent != nil {
+		if err := agent.deleteAllFlows(ctx); err != nil {
+			logger.Errorw(ctx, "error-deleting-all-device-flows", log.Fields{"parent-device-id": parentDevice.Id})
+			return err
+		}
+		return nil
+	}
+	return status.Errorf(codes.NotFound, "%s", parentDevice.Id)
+}
+
+// ChildDeviceLost  calls parent adapter to delete child device and all its references
+func (dMgr *Manager) ChildDeviceLost(ctx context.Context, curr *voltha.Device) error {
+	logger.Debugw(ctx, "child-device-lost", log.Fields{"child-device-id": curr.Id, "parent-device-id": curr.ParentId})
+	if parentAgent := dMgr.getDeviceAgent(ctx, curr.ParentId); parentAgent != nil {
+		if err := parentAgent.ChildDeviceLost(ctx, curr); err != nil {
+			// Just log the message and let the remaining pipeline proceed.
+			logger.Warnw(ctx, "childDeviceLost", log.Fields{"child-device-id": curr.Id, "parent-device-id": curr.ParentId, "error": err})
+		}
+	}
+	// Do not return an error as parent device may also have been deleted.  Let the remaining pipeline proceed.
+	return nil
+}
diff --git a/rw_core/core/device/manager_test.go b/rw_core/core/device/manager_test.go_todo
similarity index 98%
rename from rw_core/core/device/manager_test.go
rename to rw_core/core/device/manager_test.go_todo
index 1594b6e..e0156bb 100644
--- a/rw_core/core/device/manager_test.go
+++ b/rw_core/core/device/manager_test.go_todo
@@ -29,11 +29,11 @@
 	"github.com/opencord/voltha-go/rw_core/config"
 	"github.com/opencord/voltha-go/rw_core/core/adapter"
 	tst "github.com/opencord/voltha-go/rw_core/test"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-protos/v4/go/common"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events"
+	"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"github.com/phayes/freeport"
 	"github.com/stretchr/testify/assert"
 )
diff --git a/rw_core/core/device/meter/common.go b/rw_core/core/device/meter/common.go
index 773c0b7..7ecb52c 100644
--- a/rw_core/core/device/meter/common.go
+++ b/rw_core/core/device/meter/common.go
@@ -18,7 +18,7 @@
 package meter
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/meter/loader.go b/rw_core/core/device/meter/loader.go
index 42f4264..52cdb9a 100644
--- a/rw_core/core/device/meter/loader.go
+++ b/rw_core/core/device/meter/loader.go
@@ -15,12 +15,13 @@
 import (
 	"context"
 	"fmt"
+	"sync"
+
 	"github.com/opencord/voltha-go/db/model"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
-	"sync"
 )
 
 // Loader hides all low-level locking & synchronization related to meter state updates
diff --git a/rw_core/core/device/mock_kafka.go b/rw_core/core/device/mock_kafka.go
index 54ee427..68c8aa5 100644
--- a/rw_core/core/device/mock_kafka.go
+++ b/rw_core/core/device/mock_kafka.go
@@ -15,7 +15,7 @@
  */
 
 // Code generated by MockGen. DO NOT EDIT.
-// Source: github.com/opencord/voltha-lib-go/v5/pkg/kafka (interfaces: InterContainerProxy)
+// Source: github.com/opencord/voltha-lib-go/v7/pkg/kafka (interfaces: InterContainerProxy)
 
 // Package device is a generated GoMock package.
 package device
@@ -26,7 +26,7 @@
 
 	gomock "github.com/golang/mock/gomock"
 	any "github.com/golang/protobuf/ptypes/any"
-	kafka "github.com/opencord/voltha-lib-go/v5/pkg/kafka"
+	kafka "github.com/opencord/voltha-lib-go/v7/pkg/kafka"
 )
 
 // MockInterContainerProxy is a mock of InterContainerProxy interface.
diff --git a/rw_core/core/device/port/common.go b/rw_core/core/device/port/common.go
index fede329..c8b3538 100644
--- a/rw_core/core/device/port/common.go
+++ b/rw_core/core/device/port/common.go
@@ -18,7 +18,7 @@
 package port
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/port/loader.go b/rw_core/core/device/port/loader.go
index e1017bd..823d1dd 100644
--- a/rw_core/core/device/port/loader.go
+++ b/rw_core/core/device/port/loader.go
@@ -22,8 +22,8 @@
 	"sync"
 
 	"github.com/opencord/voltha-go/db/model"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
diff --git a/rw_core/core/device/remote/adapter_proxy.go b/rw_core/core/device/remote/adapter_proxy.go
deleted file mode 100755
index 77b9d1c..0000000
--- a/rw_core/core/device/remote/adapter_proxy.go
+++ /dev/null
@@ -1,604 +0,0 @@
-/*
- * Copyright 2018-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.
- */
-
-package remote
-
-import (
-	"context"
-
-	"github.com/opencord/voltha-protos/v4/go/common"
-
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-// AdapterProxy represents adapter proxy attributes
-type AdapterProxy struct {
-	kafka.EndpointManager
-	coreTopic    string
-	kafkaICProxy kafka.InterContainerProxy
-}
-
-// NewAdapterProxy will return adapter proxy instance
-func NewAdapterProxy(kafkaProxy kafka.InterContainerProxy, coreTopic string, endpointManager kafka.EndpointManager) *AdapterProxy {
-	return &AdapterProxy{
-		EndpointManager: endpointManager,
-		kafkaICProxy:    kafkaProxy,
-		coreTopic:       coreTopic,
-	}
-}
-
-func (ap *AdapterProxy) getCoreTopic() kafka.Topic {
-	return kafka.Topic{Name: ap.coreTopic}
-}
-
-func (ap *AdapterProxy) getAdapterTopic(ctx context.Context, deviceID string, adapterType string) (*kafka.Topic, error) {
-
-	endpoint, err := ap.GetEndpoint(ctx, deviceID, adapterType)
-	if err != nil {
-		return nil, err
-	}
-
-	return &kafka.Topic{Name: string(endpoint)}, nil
-}
-
-func (ap *AdapterProxy) sendRPC(ctx context.Context, rpc string, toTopic *kafka.Topic, replyToTopic *kafka.Topic,
-	waitForResponse bool, deviceID string, kvArgs ...*kafka.KVArg) (chan *kafka.RpcResponse, error) {
-
-	// Sent the request to kafka
-	respChnl := ap.kafkaICProxy.InvokeAsyncRPC(ctx, rpc, toTopic, replyToTopic, waitForResponse, deviceID, kvArgs...)
-
-	// Wait for first response which would indicate whether the request was successfully sent to kafka.
-	firstResponse, ok := <-respChnl
-	if !ok || firstResponse.MType != kafka.RpcSent {
-		logger.Errorw(ctx, "failure to request to kafka", log.Fields{"rpc": rpc, "device-id": deviceID, "error": firstResponse.Err})
-		return nil, firstResponse.Err
-	}
-	// return the kafka channel for the caller to wait for the response of the RPC call
-	return respChnl, nil
-}
-
-// AdoptDevice invokes adopt device rpc
-func (ap *AdapterProxy) AdoptDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "AdoptDevice", log.Fields{"device-id": device.Id})
-	rpc := "adopt_device"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-	}
-	replyToTopic := ap.getCoreTopic()
-	logger.Debugw(ctx, "adoptDevice-send-request", log.Fields{"device-id": device.Id, "deviceType": device.Type, "serialNumber": device.SerialNumber})
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// DisableDevice invokes disable device rpc
-func (ap *AdapterProxy) DisableDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "DisableDevice", log.Fields{"device-id": device.Id})
-	rpc := "disable_device"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// ReEnableDevice invokes reenable device rpc
-func (ap *AdapterProxy) ReEnableDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "ReEnableDevice", log.Fields{"device-id": device.Id})
-	rpc := "reenable_device"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// RebootDevice invokes reboot device rpc
-func (ap *AdapterProxy) RebootDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "RebootDevice", log.Fields{"device-id": device.Id})
-	rpc := "reboot_device"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// DeleteDevice invokes delete device rpc
-func (ap *AdapterProxy) DeleteDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "DeleteDevice", log.Fields{"device-id": device.Id})
-	rpc := "delete_device"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// GetOfpDeviceInfo invokes get ofp device info rpc
-func (ap *AdapterProxy) GetOfpDeviceInfo(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "GetOfpDeviceInfo", log.Fields{"device-id": device.Id})
-	rpc := "get_ofp_device_info"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// ReconcileDevice invokes reconcile device rpc
-func (ap *AdapterProxy) ReconcileDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "ReconcileDevice", log.Fields{"device-id": device.Id})
-	rpc := "reconcile_device"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// DownloadImage invokes download image rpc
-func (ap *AdapterProxy) DownloadImage(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "DownloadImage", log.Fields{"device-id": device.Id, "image": download.Name})
-	rpc := "download_image"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "request", Value: download},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// GetImageDownloadStatus invokes get image download status rpc
-func (ap *AdapterProxy) GetImageDownloadStatus(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "GetImageDownloadStatus", log.Fields{"device-id": device.Id, "image": download.Name})
-	rpc := "get_image_download_status"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "request", Value: download},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// CancelImageDownload invokes cancel image download rpc
-func (ap *AdapterProxy) CancelImageDownload(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "CancelImageDownload", log.Fields{"device-id": device.Id, "image": download.Name})
-	rpc := "cancel_image_download"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "request", Value: download},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// ActivateImageUpdate invokes activate image update rpc
-func (ap *AdapterProxy) ActivateImageUpdate(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "ActivateImageUpdate", log.Fields{"device-id": device.Id, "image": download.Name})
-	rpc := "activate_image_update"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "request", Value: download},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// RevertImageUpdate invokes revert image update rpc
-func (ap *AdapterProxy) RevertImageUpdate(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "RevertImageUpdate", log.Fields{"device-id": device.Id, "image": download.Name})
-	rpc := "revert_image_update"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "request", Value: download},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) PacketOut(ctx context.Context, deviceType string, deviceID string, outPort uint32, packet *ofp.OfpPacketOut) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "PacketOut", log.Fields{"device-id": deviceID, "device-type": deviceType, "out-port": outPort})
-	toTopic, err := ap.getAdapterTopic(ctx, deviceID, deviceType)
-	if err != nil {
-		return nil, err
-	}
-	rpc := "receive_packet_out"
-	args := []*kafka.KVArg{
-		{Key: "deviceId", Value: &ic.StrType{Val: deviceID}},
-		{Key: "outPort", Value: &ic.IntType{Val: int64(outPort)}},
-		{Key: "packet", Value: packet},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, deviceID, args...)
-}
-
-// UpdateFlowsBulk invokes update flows bulk rpc
-func (ap *AdapterProxy) UpdateFlowsBulk(ctx context.Context, device *voltha.Device, flows map[uint64]*ofp.OfpFlowStats, groups map[uint32]*voltha.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "UpdateFlowsBulk", log.Fields{"device-id": device.Id, "flow-count": len(flows), "group-count": len(groups), "flow-metadata": flowMetadata})
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	rpc := "update_flows_bulk"
-
-	ctr, flowSlice := 0, make([]*ofp.OfpFlowStats, len(flows))
-	for _, flow := range flows {
-		flowSlice[ctr] = flow
-		ctr++
-	}
-	ctr, groupSlice := 0, make([]*ofp.OfpGroupEntry, len(groups))
-	for _, group := range groups {
-		groupSlice[ctr] = group
-		ctr++
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "flows", Value: &voltha.Flows{Items: flowSlice}},
-		{Key: "groups", Value: &voltha.FlowGroups{Items: groupSlice}},
-		{Key: "flow_metadata", Value: flowMetadata},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(log.WithSpanFromContext(context.TODO(), ctx), rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// UpdateFlowsIncremental invokes update flows incremental rpc
-func (ap *AdapterProxy) UpdateFlowsIncremental(ctx context.Context, device *voltha.Device, flowChanges *ofp.FlowChanges, groupChanges *ofp.FlowGroupChanges, flowMetadata *voltha.FlowMetadata) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "UpdateFlowsIncremental",
-		log.Fields{
-			"device-id":             device.Id,
-			"flow-to-add-count":     len(flowChanges.ToAdd.Items),
-			"flow-to-delete-count":  len(flowChanges.ToRemove.Items),
-			"group-to-add-count":    len(groupChanges.ToAdd.Items),
-			"group-to-delete-count": len(groupChanges.ToRemove.Items),
-			"group-to-update-count": len(groupChanges.ToUpdate.Items),
-		})
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	rpc := "update_flows_incrementally"
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "flow_changes", Value: flowChanges},
-		{Key: "group_changes", Value: groupChanges},
-		{Key: "flow_metadata", Value: flowMetadata},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(log.WithSpanFromContext(context.TODO(), ctx), rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// UpdatePmConfigs invokes update pm configs rpc
-func (ap *AdapterProxy) UpdatePmConfigs(ctx context.Context, device *voltha.Device, pmConfigs *voltha.PmConfigs) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "UpdatePmConfigs", log.Fields{"device-id": device.Id, "pm-configs-id": pmConfigs.Id})
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	rpc := "update_pm_config"
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "pm_configs", Value: pmConfigs},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// SimulateAlarm invokes simulate alarm rpc
-func (ap *AdapterProxy) SimulateAlarm(ctx context.Context, device *voltha.Device, simulateReq *voltha.SimulateAlarmRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "SimulateAlarm", log.Fields{"device-id": device.Id, "simulate-req-id": simulateReq.Id})
-	rpc := "simulate_alarm"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "device", Value: device},
-		{Key: "request", Value: simulateReq},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) DisablePort(ctx context.Context, device *voltha.Device, port *voltha.Port) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "DisablePort", log.Fields{"device-id": device.Id, "port-no": port.PortNo})
-	rpc := "disable_port"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceId", Value: &ic.StrType{Val: device.Id}},
-		{Key: "port", Value: port},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) EnablePort(ctx context.Context, device *voltha.Device, port *voltha.Port) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "EnablePort", log.Fields{"device-id": device.Id, "port-no": port.PortNo})
-	rpc := "enable_port"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceId", Value: &ic.StrType{Val: device.Id}},
-		{Key: "port", Value: port},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-// ChildDeviceLost invokes child device_lost rpc
-func (ap *AdapterProxy) ChildDeviceLost(ctx context.Context, deviceType string, childDevice *voltha.Device) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "ChildDeviceLost",
-		log.Fields{"device-id": childDevice.ParentId, "parent-port-no": childDevice.ParentPortNo,
-			"onu-id": childDevice.ProxyAddress.OnuId, "serial-number": childDevice.SerialNumber})
-	rpc := "child_device_lost"
-	toTopic, err := ap.getAdapterTopic(ctx, childDevice.ParentId, deviceType)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "childDevice", Value: childDevice},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, childDevice.ParentId, args...)
-}
-
-func (ap *AdapterProxy) StartOmciTest(ctx context.Context, device *voltha.Device, omcitestrequest *voltha.OmciTestRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "Omci_test_Request_adapter_proxy", log.Fields{"device": device, "omciTestRequest": omcitestrequest})
-	rpc := "start_omci_test"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	// Use a device specific topic as we are the only core handling requests for this device
-	replyToTopic := ap.getCoreTopic()
-	// TODO: Perhaps this should have used omcitestrequest.uuid as the second argument rather
-	//   than including the whole request, which is (deviceid, uuid)
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id,
-		&kafka.KVArg{Key: "device", Value: device},
-		&kafka.KVArg{Key: "omcitestrequest", Value: omcitestrequest})
-}
-
-func (ap *AdapterProxy) GetExtValue(ctx context.Context, pdevice *voltha.Device, cdevice *voltha.Device, id string, valuetype voltha.ValueType_Type) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "GetExtValue", log.Fields{"device-id": pdevice.Id, "onuid": id})
-	rpc := "get_ext_value"
-	toTopic, err := ap.getAdapterTopic(ctx, pdevice.Id, pdevice.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	args := []*kafka.KVArg{
-		{
-			Key:   "pDeviceId",
-			Value: &ic.StrType{Val: pdevice.Id},
-		},
-		{
-			Key:   "device",
-			Value: cdevice,
-		},
-		{
-			Key:   "valuetype",
-			Value: &ic.IntType{Val: int64(valuetype)},
-		}}
-
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, pdevice.Id, args...)
-}
-
-// SetExtValue  set some given configs or value
-func (ap *AdapterProxy) SetExtValue(ctx context.Context, device *voltha.Device, value *voltha.ValueSet) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "SetExtValue", log.Fields{"device-id": value.Id})
-	rpc := "set_ext_value"
-	toTopic, err := ap.getAdapterTopic(ctx, value.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	args := []*kafka.KVArg{
-		{
-			Key:   "value",
-			Value: value,
-		},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, value.Id, args...)
-}
-
-// GetSingleValue get a value from the adapter, based on the request type
-func (ap *AdapterProxy) GetSingleValue(ctx context.Context, adapterType string, request *extension.SingleGetValueRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "GetSingleValue", log.Fields{"device-id": request.TargetId})
-	rpc := "single_get_value_request"
-	toTopic, err := ap.getAdapterTopic(ctx, request.TargetId, adapterType)
-	if err != nil {
-		return nil, err
-	}
-
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	args := []*kafka.KVArg{
-		{
-			Key:   "request",
-			Value: request,
-		},
-	}
-
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, request.TargetId, args...)
-}
-
-// SetSingleValue set a single value on the adapter, based on the request type
-func (ap *AdapterProxy) SetSingleValue(ctx context.Context, adapterType string, request *extension.SingleSetValueRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "SetSingleValue", log.Fields{"device-id": request.TargetId})
-	rpc := "single_set_value_request"
-	toTopic, err := ap.getAdapterTopic(ctx, request.TargetId, adapterType)
-	if err != nil {
-		return nil, err
-	}
-
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	args := []*kafka.KVArg{
-		{
-			Key:   "request",
-			Value: request,
-		},
-	}
-
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, request.TargetId, args...)
-}
-
-// DownloadImageToOnuDevice invokes download image rpc
-func (ap *AdapterProxy) DownloadImageToOnuDevice(ctx context.Context, device *voltha.Device, downloadRequest *voltha.DeviceImageDownloadRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "download-image-to-device", log.Fields{"device-id": device.Id, "image": downloadRequest.Image.Name})
-	rpc := "Download_onu_image"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceImageDownloadReq", Value: downloadRequest},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) GetOnuImageStatus(ctx context.Context, device *voltha.Device, request *voltha.DeviceImageRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "get-image-status", log.Fields{"device-id": device.Id})
-	rpc := "Get_onu_image_status"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceImageReq", Value: request},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) ActivateOnuImage(ctx context.Context, device *voltha.Device, request *voltha.DeviceImageRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "activate-onu-image", log.Fields{"device-id": device.Id})
-	rpc := "Activate_onu_image"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceImageReq", Value: request},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) AbortImageUpgrade(ctx context.Context, device *voltha.Device, request *voltha.DeviceImageRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "abort-image-upgrade", log.Fields{"device-id": device.Id})
-	rpc := "Abort_onu_image_upgrade"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceImageReq", Value: request},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) CommitImage(ctx context.Context, device *voltha.Device, request *voltha.DeviceImageRequest) (chan *kafka.RpcResponse, error) {
-	logger.Debugw(ctx, "commit-image", log.Fields{"device-id": device.Id})
-	rpc := "Commit_onu_image"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceImageReq", Value: request},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
-
-func (ap *AdapterProxy) GetOnuImages(ctx context.Context, device *voltha.Device, id *common.ID) (chan *kafka.RpcResponse, error) {
-	logger.Debug(ctx, "get-onu-images")
-	rpc := "Get_onu_images"
-	toTopic, err := ap.getAdapterTopic(ctx, device.Id, device.Adapter)
-	if err != nil {
-		return nil, err
-	}
-	args := []*kafka.KVArg{
-		{Key: "deviceId", Value: &ic.StrType{Val: id.Id}},
-	}
-	replyToTopic := ap.getCoreTopic()
-	return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
-}
diff --git a/rw_core/core/device/remote/adapter_proxy_test.go b/rw_core/core/device/remote/adapter_proxy_test.go
deleted file mode 100755
index ad8cd31..0000000
--- a/rw_core/core/device/remote/adapter_proxy_test.go
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Copyright 2019-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.
- */
-package remote
-
-import (
-	"context"
-	"crypto/rand"
-	"strings"
-	"testing"
-	"time"
-
-	"github.com/golang/protobuf/ptypes"
-	any2 "github.com/golang/protobuf/ptypes/any"
-	cm "github.com/opencord/voltha-go/rw_core/mocks"
-	com "github.com/opencord/voltha-lib-go/v5/pkg/adapters/common"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	mock_kafka "github.com/opencord/voltha-lib-go/v5/pkg/mocks/kafka"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	of "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-	"github.com/stretchr/testify/assert"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
-)
-
-const (
-	coreName       = "rw_core"
-	adapterName    = "adapter_mock"
-	coreInstanceID = "1000"
-)
-
-var (
-	coreKafkaICProxy    kafka.InterContainerProxy
-	adapterKafkaICProxy kafka.InterContainerProxy
-	kc                  kafka.Client
-	adapterReqHandler   *com.RequestHandlerProxy
-	adapter             *cm.Adapter
-)
-
-func init() {
-	ctx := context.Background()
-	if _, err := log.SetDefaultLogger(log.JSON, 0, log.Fields{"instanceId": coreInstanceID}); err != nil {
-		logger.With(log.Fields{"error": err}).Fatal(ctx, "Cannot setup logging")
-	}
-	// Set the log level to Warning
-	log.SetAllLogLevel(log.WarnLevel)
-
-	var err error
-
-	// Create the KV client
-	kc = mock_kafka.NewKafkaClient()
-
-	// Setup core inter-container proxy and core request handler
-	coreKafkaICProxy = kafka.NewInterContainerProxy(
-		kafka.MsgClient(kc),
-		kafka.DefaultTopic(&kafka.Topic{Name: coreName}))
-
-	if err = coreKafkaICProxy.Start(ctx); err != nil {
-		logger.Fatalw(ctx, "Failure-starting-core-kafka-intercontainerProxy", log.Fields{"error": err})
-	}
-	if err = coreKafkaICProxy.SubscribeWithDefaultRequestHandler(ctx, kafka.Topic{Name: coreName}, 0); err != nil {
-		logger.Fatalw(ctx, "Failure-subscribing-core-request-handler", log.Fields{"error": err})
-	}
-
-	// Setup adapter inter-container proxy and adapter request handler
-	adapterCoreProxy := com.NewCoreProxy(ctx, nil, adapterName, coreName)
-	adapter = cm.NewAdapter(adapterCoreProxy)
-	adapterReqHandler = com.NewRequestHandlerProxy(coreInstanceID, adapter, adapterCoreProxy)
-	adapterKafkaICProxy = kafka.NewInterContainerProxy(
-		kafka.MsgClient(kc),
-		kafka.DefaultTopic(&kafka.Topic{Name: adapterName}),
-		kafka.RequestHandlerInterface(adapterReqHandler))
-
-	if err = adapterKafkaICProxy.Start(ctx); err != nil {
-		logger.Fatalw(ctx, "Failure-starting-adapter-kafka-intercontainerProxy", log.Fields{"error": err})
-	}
-	if err = adapterKafkaICProxy.SubscribeWithDefaultRequestHandler(ctx, kafka.Topic{Name: adapterName}, 0); err != nil {
-		logger.Fatalw(ctx, "Failure-subscribing-adapter-request-handler", log.Fields{"error": err})
-	}
-}
-
-func getRandomBytes(size int) (bytes []byte, err error) {
-	bytes = make([]byte, size)
-	_, err = rand.Read(bytes)
-	return
-}
-
-func TestCreateAdapterProxy(t *testing.T) {
-	ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
-	assert.NotNil(t, ap)
-}
-
-func waitForResponse(ctx context.Context, ch chan *kafka.RpcResponse) (*any2.Any, error) {
-	select {
-	case rpcResponse, ok := <-ch:
-		if !ok {
-			return nil, status.Error(codes.Aborted, "channel-closed")
-		} else if rpcResponse.Err != nil {
-			return nil, rpcResponse.Err
-		} else {
-			return rpcResponse.Reply, nil
-		}
-	case <-ctx.Done():
-		return nil, ctx.Err()
-	}
-}
-
-func testSimpleRequests(t *testing.T) {
-	type simpleRequest func(context.Context, *voltha.Device) (chan *kafka.RpcResponse, error)
-	ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
-	simpleRequests := []simpleRequest{
-		ap.AdoptDevice,
-		ap.DisableDevice,
-		ap.RebootDevice,
-		ap.DeleteDevice,
-		ap.ReconcileDevice,
-		ap.ReEnableDevice,
-	}
-	for _, f := range simpleRequests {
-		// Success
-		d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
-		ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
-		rpcResponse, err := f(ctx, d)
-		assert.Nil(t, err)
-		_, err = waitForResponse(ctx, rpcResponse)
-		assert.Nil(t, err)
-		cancel()
-
-		//	Failure - invalid adapter
-		expectedError := "context deadline exceeded"
-		d = &voltha.Device{Id: "deviceId", Adapter: "adapter_mock_1"}
-		ctx, cancel = context.WithTimeout(context.Background(), 20*time.Millisecond)
-		rpcResponse, err = f(ctx, d)
-		assert.Nil(t, err)
-		_, err = waitForResponse(ctx, rpcResponse)
-		cancel()
-		assert.NotNil(t, err)
-		assert.True(t, strings.Contains(err.Error(), expectedError))
-
-		// Failure -  timeout
-		d = &voltha.Device{Id: "deviceId", Adapter: adapterName}
-		ctx, cancel = context.WithTimeout(context.Background(), 100*time.Nanosecond)
-		rpcResponse, err = f(ctx, d)
-		assert.Nil(t, err)
-		_, err = waitForResponse(ctx, rpcResponse)
-		cancel()
-		assert.NotNil(t, err)
-		assert.True(t, strings.Contains(err.Error(), expectedError))
-	}
-}
-
-func testGetSwitchCapabilityFromAdapter(t *testing.T) {
-	ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
-	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
-	ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
-	defer cancel()
-	rpcResponse, err := ap.GetOfpDeviceInfo(ctx, d)
-	assert.Nil(t, err)
-	response, err := waitForResponse(ctx, rpcResponse)
-	assert.Nil(t, err)
-	switchCap := &ic.SwitchCapability{}
-	err = ptypes.UnmarshalAny(response, switchCap)
-	assert.Nil(t, err)
-	assert.NotNil(t, switchCap)
-	expectedCap, _ := adapter.Get_ofp_device_info(ctx, d)
-	assert.Equal(t, switchCap.String(), expectedCap.String())
-}
-
-func testPacketOut(t *testing.T) {
-	ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
-	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
-	outPort := uint32(1)
-	packet, err := getRandomBytes(50)
-	assert.Nil(t, err)
-	ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
-	defer cancel()
-	rpcResponse, err := ap.PacketOut(ctx, adapterName, d.Id, outPort, &of.OfpPacketOut{Data: packet})
-	assert.Nil(t, err)
-	_, err = waitForResponse(ctx, rpcResponse)
-	assert.Nil(t, err)
-}
-
-func testFlowUpdates(t *testing.T) {
-	ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
-	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
-	_, err := ap.UpdateFlowsBulk(context.Background(), d, nil, nil, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
-	flowChanges := &voltha.FlowChanges{ToAdd: &voltha.Flows{Items: nil}, ToRemove: &voltha.Flows{Items: nil}}
-	groupChanges := &voltha.FlowGroupChanges{ToAdd: &voltha.FlowGroups{Items: nil}, ToRemove: &voltha.FlowGroups{Items: nil}, ToUpdate: &voltha.FlowGroups{Items: nil}}
-	ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
-	defer cancel()
-	rpcResponse, err := ap.UpdateFlowsIncremental(ctx, d, flowChanges, groupChanges, &voltha.FlowMetadata{})
-	assert.Nil(t, err)
-	_, err = waitForResponse(ctx, rpcResponse)
-	assert.Nil(t, err)
-}
-
-func testPmUpdates(t *testing.T) {
-	ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
-	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
-	ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
-	defer cancel()
-	rpcResponse, err := ap.UpdatePmConfigs(ctx, d, &voltha.PmConfigs{})
-	assert.Nil(t, err)
-	_, err = waitForResponse(ctx, rpcResponse)
-	assert.Nil(t, err)
-}
-
-func TestSuiteAdapterProxy(t *testing.T) {
-	//1. Test the simple requests first
-	testSimpleRequests(t)
-
-	//2.  Test get switch capability
-	testGetSwitchCapabilityFromAdapter(t)
-
-	//3. Test PacketOut
-	testPacketOut(t)
-
-	//4. Test flow updates
-	testFlowUpdates(t)
-
-	//5. Pm configs
-	testPmUpdates(t)
-}
diff --git a/rw_core/core/device/remote/common.go b/rw_core/core/device/remote/common.go
deleted file mode 100644
index 2e44b29..0000000
--- a/rw_core/core/device/remote/common.go
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Copyright 2020-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.
- */
-
-// Package core Common Logger initialization
-package remote
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/rw_core/core/device/state/common.go b/rw_core/core/device/state/common.go
index 346841a..43d6419 100644
--- a/rw_core/core/device/state/common.go
+++ b/rw_core/core/device/state/common.go
@@ -18,7 +18,7 @@
 package state
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/state/transitions.go b/rw_core/core/device/state/transitions.go
index c3b98e7..80ac28e 100644
--- a/rw_core/core/device/state/transitions.go
+++ b/rw_core/core/device/state/transitions.go
@@ -18,10 +18,12 @@
 
 import (
 	"context"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
 	"reflect"
 	"runtime"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 // deviceType mentions type of device like parent, child
@@ -68,7 +70,7 @@
 	Admin       voltha.AdminState_Types
 	Connection  voltha.ConnectStatus_Types
 	Operational voltha.OperStatus_Types
-	Transient   voltha.DeviceTransientState_Types
+	Transient   core.DeviceTransientState_Types
 }
 
 // transitionHandler function type which takes the current and previous device info as input parameter
@@ -112,242 +114,242 @@
 		transitionMap.transitions,
 		transition{
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.CreateLogicalDevice}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.SetupUNILogicalPorts}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.SetupUNILogicalPorts}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ //DELETE PRE PROVISIONED State device forcefully
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_FORCE_DELETING},
+			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_FORCE_DELETING},
 			handlers:      []transitionHandler{dMgr.RunPostDeviceDelete}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ // DELETE PRE PROVISIONED State device no force option set
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE},
+			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE},
 			handlers:      []transitionHandler{dMgr.RunPostDeviceDelete}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ //DELETE device forcefully
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_FORCE_DELETING},
+			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_FORCE_DELETING},
 			handlers:      []transitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.RunPostDeviceDelete}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ //DELETE device after adapter response
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE},
+			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE},
 			handlers:      []transitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.RunPostDeviceDelete}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ //DELETE no operation transition
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_DELETING_FROM_ADAPTER},
+			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_DELETING_FROM_ADAPTER},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.DeleteAllDeviceFlows}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.DeleteAllDeviceFlows}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.CreateLogicalDevice}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.CreateLogicalDevice}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ //DELETE force case
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_FORCE_DELETING},
+			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_FORCE_DELETING},
 			handlers:      []transitionHandler{dMgr.ChildDeviceLost, dMgr.DeleteLogicalPorts, dMgr.RunPostDeviceDelete}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ //DELETE after adapter response case
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE},
+			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE},
 			handlers:      []transitionHandler{dMgr.ChildDeviceLost, dMgr.DeleteLogicalPorts, dMgr.RunPostDeviceDelete}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{ //DELETE wait for adapter response(no operation)
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_ANY},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_DELETING_FROM_ADAPTER},
+			previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_ANY},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_DELETING_FROM_ADAPTER},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    parent,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    child,
-			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
-			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
+			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING_FAILED, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING_FAILED, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING_FAILED, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING_FAILED, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 	transitionMap.transitions = append(transitionMap.transitions,
 		transition{
 			deviceType:    any,
-			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
-			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING_FAILED, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING_FAILED, Transient: core.DeviceTransientState_RECONCILE_IN_PROGRESS},
 			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 
 	return &transitionMap
 }
 
-func getDeviceStates(device *voltha.Device, transientState voltha.DeviceTransientState_Types) deviceState {
+func getDeviceStates(device *voltha.Device, transientState core.DeviceTransientState_Types) deviceState {
 	return deviceState{Admin: device.AdminState, Connection: device.ConnectStatus, Operational: device.OperStatus,
 		Transient: transientState}
 }
@@ -362,14 +364,14 @@
 			transient: currPrevStateMatch}
 	}
 	// Do we have transient state match?
-	if current.Transient == transition.currentState.Transient && transition.currentState.Transient != voltha.DeviceTransientState_ANY {
-		if previous.Transient == transition.previousState.Transient || transition.previousState.Transient == voltha.DeviceTransientState_ANY {
+	if current.Transient == transition.currentState.Transient && transition.currentState.Transient != core.DeviceTransientState_ANY {
+		if previous.Transient == transition.previousState.Transient || transition.previousState.Transient == core.DeviceTransientState_ANY {
 			m.transient = currPrevStateMatch
 		} else {
 			m.transient = currStateOnlyMatch
 		}
-	} else if current.Transient == transition.currentState.Transient && transition.currentState.Transient == voltha.DeviceTransientState_ANY {
-		if previous.Transient == transition.previousState.Transient || transition.previousState.Transient == voltha.DeviceTransientState_ANY {
+	} else if current.Transient == transition.currentState.Transient && transition.currentState.Transient == core.DeviceTransientState_ANY {
+		if previous.Transient == transition.previousState.Transient || transition.previousState.Transient == core.DeviceTransientState_ANY {
 			m.transient = currWildcardMatch
 		}
 	}
@@ -426,7 +428,7 @@
 
 // getTransitionHandler returns transition handler & a flag that's set if the transition is invalid
 func (tMap *TransitionMap) getTransitionHandler(ctx context.Context, cDevice, pDevice *voltha.Device,
-	cTransientState, pTransientState voltha.DeviceTransientState_Types) []transitionHandler {
+	cTransientState, pTransientState core.DeviceTransientState_Types) []transitionHandler {
 	//1. Get the previous and current set of states
 	cState := getDeviceStates(cDevice, cTransientState)
 	pState := getDeviceStates(pDevice, pTransientState)
@@ -468,7 +470,7 @@
 }
 
 func (tMap *TransitionMap) ProcessTransition(ctx context.Context, device, prevDevice *voltha.Device,
-	deviceTransientState, prevDeviceTransientState voltha.DeviceTransientState_Types) error {
+	deviceTransientState, prevDeviceTransientState core.DeviceTransientState_Types) error {
 	// This will be triggered on every state update
 	logger.Debugw(ctx, "state-transition", log.Fields{
 		"device":               device.Id,
diff --git a/rw_core/core/device/state/transitions_test.go b/rw_core/core/device/state/transitions_test.go
index ad53526..5065a60 100644
--- a/rw_core/core/device/state/transitions_test.go
+++ b/rw_core/core/device/state/transitions_test.go
@@ -21,7 +21,9 @@
 	"reflect"
 	"testing"
 
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-protos/v5/go/core"
+
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 	"github.com/stretchr/testify/assert"
 )
 
@@ -52,13 +54,13 @@
 }
 
 func assertInvalidTransition(t *testing.T, device, prevDevice *voltha.Device) {
-	handlers := transitionMap.getTransitionHandler(context.Background(), device, prevDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers := transitionMap.getTransitionHandler(context.Background(), device, prevDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.NotifyInvalidTransition).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 }
 
-func assertNoOpTransition(t *testing.T, device, prevDevice *voltha.Device, transientState voltha.DeviceTransientState_Types) {
+func assertNoOpTransition(t *testing.T, device, prevDevice *voltha.Device, transientState core.DeviceTransientState_Types) {
 	handlers := transitionMap.getTransitionHandler(context.Background(), device, prevDevice, transientState,
 		transientState)
 	assert.Equal(t, 0, len(handlers))
@@ -69,134 +71,134 @@
 
 	previousDevice := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
-	handlers := transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers := transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_DISCOVERED)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_FORCE_DELETING,
-		voltha.DeviceTransientState_ANY)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_FORCE_DELETING,
+		core.DeviceTransientState_ANY)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE,
-		voltha.DeviceTransientState_ANY)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE,
+		core.DeviceTransientState_ANY)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_FORCE_DELETING,
-		voltha.DeviceTransientState_ANY)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_FORCE_DELETING,
+		core.DeviceTransientState_ANY)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_FAILED)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_FORCE_DELETING,
-		voltha.DeviceTransientState_ANY)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_FORCE_DELETING,
+		core.DeviceTransientState_ANY)
 	assert.Equal(t, 3, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ChildDeviceLost).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 	assert.True(t, reflect.ValueOf(tdm.DeleteLogicalPorts).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
@@ -204,8 +206,8 @@
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 4, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.DeleteAllLogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 	assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
@@ -214,15 +216,15 @@
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 4, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.DeleteAllLogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 	assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
@@ -231,78 +233,78 @@
 
 	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_NONE,
+		core.DeviceTransientState_NONE)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING_FAILED)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING_FAILED)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING_FAILED)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING_FAILED)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_RECONCILING_FAILED)
-	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
-		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
@@ -344,8 +346,8 @@
 		for _, device := range deleteDeviceTest.devices {
 			device.Root = true
 			t.Run(testName, func(t *testing.T) {
-				handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_FORCE_DELETING,
-					voltha.DeviceTransientState_ANY)
+				handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_FORCE_DELETING,
+					core.DeviceTransientState_ANY)
 				assert.Equal(t, 4, len(handlers))
 				for idx, expHandler := range deleteDeviceTest.expectedParentHandlers {
 					assert.True(t, reflect.ValueOf(expHandler).Pointer() == reflect.ValueOf(handlers[idx]).Pointer())
@@ -359,8 +361,8 @@
 		for _, device := range deleteDeviceTest.devices {
 			device.Root = false
 			t.Run(testName, func(t *testing.T) {
-				handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_FORCE_DELETING,
-					voltha.DeviceTransientState_ANY)
+				handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, core.DeviceTransientState_FORCE_DELETING,
+					core.DeviceTransientState_ANY)
 				assert.Equal(t, 3, len(handlers))
 				for idx, expHandler := range deleteDeviceTest.expectedChildHandlers {
 					assert.True(t, reflect.ValueOf(expHandler).Pointer() == reflect.ValueOf(handlers[idx]).Pointer())
@@ -397,71 +399,71 @@
 
 	previousDevice := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
 	device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
 	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
 	device = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
 	device = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 
 	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
 	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
-	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assertNoOpTransition(t, device, previousDevice, core.DeviceTransientState_RECONCILE_IN_PROGRESS)
 }
 
 func TestMatch(t *testing.T) {
diff --git a/rw_core/core/device/transientstate/common.go b/rw_core/core/device/transientstate/common.go
index feabb19..920c567 100644
--- a/rw_core/core/device/transientstate/common.go
+++ b/rw_core/core/device/transientstate/common.go
@@ -18,7 +18,7 @@
 package transientstate
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/rw_core/core/device/transientstate/loader.go b/rw_core/core/device/transientstate/loader.go
index 9a95bd9..e953b91 100644
--- a/rw_core/core/device/transientstate/loader.go
+++ b/rw_core/core/device/transientstate/loader.go
@@ -21,9 +21,10 @@
 	"fmt"
 	"sync"
 
+	"github.com/opencord/voltha-protos/v5/go/core"
+
 	"github.com/opencord/voltha-go/db/model"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -37,7 +38,7 @@
 }
 
 type data struct {
-	transientState voltha.DeviceTransientState_Types
+	transientState core.DeviceTransientState_Types
 	deviceID       string
 }
 
@@ -45,7 +46,7 @@
 	return &Loader{
 		dbProxy: dbProxy,
 		deviceTransientState: &data{
-			transientState: voltha.DeviceTransientState_NONE,
+			transientState: core.DeviceTransientState_NONE,
 			deviceID:       deviceID,
 		},
 	}
@@ -57,7 +58,7 @@
 	loader.lock.Lock()
 	defer loader.lock.Unlock()
 
-	var deviceTransientState voltha.DeviceTransientState
+	var deviceTransientState core.DeviceTransientState
 	have, err := loader.dbProxy.Get(ctx, loader.deviceTransientState.deviceID, &deviceTransientState)
 	if err != nil {
 		logger.Errorw(ctx, "failed-to-get-device-transient-state-from-cluster-data-proxy", log.Fields{"error": err})
@@ -67,7 +68,7 @@
 		loader.deviceTransientState.transientState = deviceTransientState.TransientState
 		return
 	}
-	loader.deviceTransientState.transientState = voltha.DeviceTransientState_NONE
+	loader.deviceTransientState.transientState = core.DeviceTransientState_NONE
 }
 
 // Lock acquires the lock for deviceTransientStateLoader, and returns a handle
@@ -88,14 +89,14 @@
 }
 
 // GetReadOnly returns device transient which MUST NOT be modified externally, but which is safe to keep indefinitely
-func (h *Handle) GetReadOnly() voltha.DeviceTransientState_Types {
+func (h *Handle) GetReadOnly() core.DeviceTransientState_Types {
 	return h.data.transientState
 }
 
 // Update updates device transient state in KV store
 // The provided device transient state must not be modified afterwards.
-func (h *Handle) Update(ctx context.Context, state voltha.DeviceTransientState_Types) error {
-	var tState voltha.DeviceTransientState
+func (h *Handle) Update(ctx context.Context, state core.DeviceTransientState_Types) error {
+	var tState core.DeviceTransientState
 	tState.TransientState = state
 	if err := h.loader.dbProxy.Set(ctx, fmt.Sprint(h.data.deviceID), &tState); err != nil {
 		return status.Errorf(codes.Internal, "failed-to-update-device-%v-transient-state: %s", h.data.deviceID, err)
diff --git a/rw_core/core/kafka.go b/rw_core/core/kafka.go
deleted file mode 100644
index 6ba8bed..0000000
--- a/rw_core/core/kafka.go
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Copyright 2018-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.
- */
-
-package core
-
-import (
-	"context"
-	"github.com/cenkalti/backoff/v3"
-	"github.com/opencord/voltha-go/rw_core/config"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events"
-
-	"time"
-
-	"github.com/opencord/voltha-go/rw_core/core/adapter"
-	"github.com/opencord/voltha-go/rw_core/core/api"
-	"github.com/opencord/voltha-go/rw_core/core/device"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-lib-go/v5/pkg/probe"
-)
-
-// startKafkInterContainerProxy is responsible for starting the Kafka Interadapter Proxy
-func startKafkInterContainerProxy(ctx context.Context, kafkaClient kafka.Client, address string, coreTopic string, connectionRetryInterval time.Duration) (kafka.InterContainerProxy, error) {
-	logger.Infow(ctx, "initialize-kafka-manager", log.Fields{"address": address, "topic": coreTopic})
-
-	probe.UpdateStatusFromContext(ctx, adapterMessageBus, probe.ServiceStatusPreparing)
-
-	// create the kafka RPC proxy
-	kmp := kafka.NewInterContainerProxy(
-		kafka.InterContainerAddress(address),
-		kafka.MsgClient(kafkaClient),
-		kafka.DefaultTopic(&kafka.Topic{Name: coreTopic}))
-
-	probe.UpdateStatusFromContext(ctx, adapterMessageBus, probe.ServiceStatusPrepared)
-
-	// wait for connectivity
-	logger.Infow(ctx, "starting-kafka-manager", log.Fields{"address": address, "topic": coreTopic})
-
-	for {
-		// If we haven't started yet, then try to start
-		logger.Infow(ctx, "starting-kafka-proxy", log.Fields{})
-		if err := kmp.Start(ctx); err != nil {
-			// We failed to start. Delay and then try again later.
-			// Don't worry about liveness, as we can't be live until we've started.
-			probe.UpdateStatusFromContext(ctx, adapterMessageBus, probe.ServiceStatusNotReady)
-			logger.Warnw(ctx, "error-starting-kafka-messaging-proxy", log.Fields{"error": err})
-			select {
-			case <-time.After(connectionRetryInterval):
-			case <-ctx.Done():
-				return nil, ctx.Err()
-			}
-			continue
-		}
-		// We started. We only need to do this once.
-		// Next we'll fall through and start checking liveness.
-		logger.Infow(ctx, "started-kafka-proxy", log.Fields{})
-		break
-	}
-	return kmp, nil
-}
-
-func startEventProxy(ctx context.Context, kafkaClient kafka.Client, eventTopic string, connectionRetryInterval time.Duration, updateProbeService bool) (*events.EventProxy, error) {
-	ep := events.NewEventProxy(events.MsgClient(kafkaClient), events.MsgTopic(kafka.Topic{Name: eventTopic}))
-	for {
-		if err := kafkaClient.Start(ctx); err != nil {
-			if updateProbeService {
-				probe.UpdateStatusFromContext(ctx, clusterMessageBus, probe.ServiceStatusNotReady)
-			}
-			logger.Warnw(ctx, "failed-to-setup-kafka-connection-on-kafka-cluster-address", log.Fields{"error": err})
-			select {
-			case <-time.After(connectionRetryInterval):
-				continue
-			case <-ctx.Done():
-				return nil, ctx.Err()
-			}
-		}
-		go ep.Start()
-		if updateProbeService {
-			probe.UpdateStatusFromContext(ctx, clusterMessageBus, probe.ServiceStatusRunning)
-		}
-		logger.Info(ctx, "started-connection-on-kafka-cluster-address")
-		break
-	}
-	return ep, nil
-}
-
-func stopEventProxy(ctx context.Context, kafkaClient kafka.Client, ep *events.EventProxy) {
-	defer kafkaClient.Stop(ctx)
-	ep.Stop()
-}
-
-// Interface that is valid for both EventProxy and InterContainerProxy
-type KafkaProxy interface {
-	EnableLivenessChannel(ctx context.Context, enable bool) chan bool
-	SendLiveness(ctx context.Context) error
-}
-
-/*
- * monitorKafkaLiveness is responsible for monitoring the Kafka Interadapter Proxy connectivity state
- *
- * Any producer that fails to send will cause KafkaInterContainerProxy to
- * post a false event on its liveness channel. Any producer that succeeds in sending
- * will cause KafkaInterContainerProxy to post a true event on its liveness
- * channel. Group receivers also update liveness state, and a receiver will typically
- * indicate a loss of liveness within 3-5 seconds of Kafka going down. Receivers
- * only indicate restoration of liveness if a message is received. During normal
- * operation, messages will be routinely produced and received, automatically
- * indicating liveness state. These routine liveness indications are rate-limited
- * inside sarama_client.
- *
- * This thread monitors the status of KafkaInterContainerProxy's liveness and pushes
- * that state to the core's readiness probes. If no liveness event has been seen
- * within a timeout, then the thread will make an attempt to produce a "liveness"
- * message, which will in turn trigger a liveness event on the liveness channel, true
- * or false depending on whether the attempt succeeded.
- *
- * The gRPC server in turn monitors the state of the readiness probe and will
- * start issuing UNAVAILABLE response while the probe is not ready.
- *
- * startupRetryInterval -- interval between attempts to start
- * liveProbeInterval -- interval between liveness checks when in a live state
- * notLiveProbeInterval -- interval between liveness checks when in a notLive state
- *
- * liveProbeInterval and notLiveProbeInterval can be configured separately,
- * though the current default is that both are set to 60 seconds.
- */
-func monitorKafkaLiveness(ctx context.Context, kmp KafkaProxy, liveProbeInterval time.Duration, notLiveProbeInterval time.Duration, serviceName string) {
-	logger.Infow(ctx, "started-kafka-message-proxy", log.Fields{"service": serviceName})
-
-	livenessChannel := kmp.EnableLivenessChannel(ctx, true)
-
-	logger.Infow(ctx, "enabled-kafka-liveness-channel", log.Fields{"service": serviceName})
-
-	timeout := liveProbeInterval
-	for {
-		timeoutTimer := time.NewTimer(timeout)
-		select {
-		case liveness := <-livenessChannel:
-			logger.Infow(ctx, "kafka-manager-thread-liveness-event", log.Fields{"liveness": liveness, "service": serviceName})
-			// there was a state change in Kafka liveness
-			if !liveness {
-				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusNotReady)
-				logger.Infow(ctx, "kafka-manager-thread-set-server-notready", log.Fields{"service": serviceName})
-
-				// retry frequently while life is bad
-				timeout = notLiveProbeInterval
-			} else {
-				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
-				logger.Infow(ctx, "kafka-manager-thread-set-server-ready", log.Fields{"service": serviceName})
-
-				// retry infrequently while life is good
-				timeout = liveProbeInterval
-			}
-			if !timeoutTimer.Stop() {
-				<-timeoutTimer.C
-			}
-		case <-timeoutTimer.C:
-			logger.Infow(ctx, "kafka-proxy-liveness-recheck", log.Fields{"service": serviceName})
-			// send the liveness probe in a goroutine; we don't want to deadlock ourselves as
-			// the liveness probe may wait (and block) writing to our channel.
-			go func() {
-				err := kmp.SendLiveness(ctx)
-				if err != nil {
-					// Catch possible error case if sending liveness after Sarama has been stopped.
-					logger.Warnw(ctx, "error-kafka-send-liveness", log.Fields{"error": err, "service": serviceName})
-				}
-			}()
-		case <-ctx.Done():
-			return // just exit
-		}
-	}
-}
-
-func registerAdapterRequestHandlers(ctx context.Context, kmp kafka.InterContainerProxy, dMgr *device.Manager,
-	aMgr *adapter.Manager, cf *config.RWCoreFlags, serviceName string) {
-	logger.Infow(ctx, "registering-request-handler", log.Fields{"topic": cf.CoreTopic})
-
-	// Set the exponential backoff params
-	kafkaRetryBackoff := backoff.NewExponentialBackOff()
-	kafkaRetryBackoff.InitialInterval = cf.BackoffRetryInitialInterval
-	kafkaRetryBackoff.MaxElapsedTime = cf.BackoffRetryMaxElapsedTime
-	kafkaRetryBackoff.MaxInterval = cf.BackoffRetryMaxInterval
-
-	//For initial request, do not wait
-	backoffTimer := time.NewTimer(0)
-
-	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusNotReady)
-	requestProxy := api.NewAdapterRequestHandlerProxy(dMgr, aMgr)
-	for {
-		select {
-		case <-backoffTimer.C:
-			// Register the broadcast topic to handle any core-bound broadcast requests
-			err := kmp.SubscribeWithRequestHandlerInterface(ctx, kafka.Topic{Name: cf.CoreTopic}, requestProxy)
-			if err == nil {
-				logger.Infow(ctx, "request-handler-registered", log.Fields{"topic": cf.CoreTopic})
-				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
-				return
-			}
-			logger.Errorw(ctx, "failed-registering-broadcast-handler-retrying", log.Fields{"topic": cf.CoreTopic})
-			duration := kafkaRetryBackoff.NextBackOff()
-			//This case should never occur(by default) as max elapsed time for backoff is 0(by default) , so it will never return stop
-			if duration == backoff.Stop {
-				// If we reach a maximum then warn and reset the backoff timer and keep attempting.
-				logger.Warnw(ctx, "maximum-kafka-retry-backoff-reached-resetting",
-					log.Fields{"max-kafka-retry-backoff": kafkaRetryBackoff.MaxElapsedTime})
-				kafkaRetryBackoff.Reset()
-				duration = kafkaRetryBackoff.NextBackOff()
-			}
-			backoffTimer = time.NewTimer(duration)
-		case <-ctx.Done():
-			logger.Infow(ctx, "context-closed", log.Fields{"topic": cf.CoreTopic})
-			return
-		}
-	}
-}
diff --git a/rw_core/core/kv.go b/rw_core/core/kv.go
index 7302321..3a3e346 100644
--- a/rw_core/core/kv.go
+++ b/rw_core/core/kv.go
@@ -21,10 +21,10 @@
 	"errors"
 	"time"
 
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-lib-go/v5/pkg/probe"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 )
@@ -47,7 +47,7 @@
 }
 
 // waitUntilKVStoreReachableOrMaxTries will wait until it can connect to a KV store or until maxtries has been reached
-func waitUntilKVStoreReachableOrMaxTries(ctx context.Context, kvClient kvstore.Client, maxRetries int, retryInterval time.Duration) error {
+func waitUntilKVStoreReachableOrMaxTries(ctx context.Context, kvClient kvstore.Client, maxRetries int, retryInterval time.Duration, serviceName string) error {
 	logger.Infow(ctx, "verifying-KV-store-connectivity", log.Fields{"retries": maxRetries, "retryInterval": retryInterval})
 	count := 0
 	for {
@@ -72,7 +72,7 @@
 			break
 		}
 	}
-	probe.UpdateStatusFromContext(ctx, "kv-store", probe.ServiceStatusRunning)
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
 	logger.Info(ctx, "KV-store-reachable")
 	return nil
 }
@@ -91,13 +91,13 @@
  * The gRPC server in turn monitors the state of the readiness probe and will
  * start issuing UNAVAILABLE response while the probe is not ready.
  */
-func monitorKVStoreLiveness(ctx context.Context, backend *db.Backend, liveProbeInterval, notLiveProbeInterval time.Duration) {
+func monitorKVStoreLiveness(ctx context.Context, backend *db.Backend, serviceName string, liveProbeInterval, notLiveProbeInterval time.Duration) {
 	logger.Info(ctx, "start-monitoring-kvstore-liveness")
 
 	// Instruct backend to create Liveness channel for transporting state updates
 	livenessChannel := backend.EnableLivenessChannel(ctx)
 
-	logger.Debug(ctx, "enabled-kvstore-liveness-channel")
+	logger.Infow(ctx, "enabled-liveness-channel", log.Fields{"service-name": serviceName})
 
 	// Default state for kvstore is alive for rw_core
 	timeout := liveProbeInterval
@@ -107,17 +107,17 @@
 		select {
 
 		case liveness := <-livenessChannel:
-			logger.Debugw(ctx, "received-liveness-change-notification", log.Fields{"liveness": liveness})
+			logger.Debugw(ctx, "received-liveness-change-notification", log.Fields{"liveness": liveness, "service-name": serviceName})
 
 			if !liveness {
-				probe.UpdateStatusFromContext(ctx, "kv-store", probe.ServiceStatusNotReady)
-				logger.Info(ctx, "kvstore-set-server-notready")
+				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusNotReady)
+				logger.Infow(ctx, "service-not-ready", log.Fields{"service-name": serviceName})
 
 				timeout = notLiveProbeInterval
 
 			} else {
-				probe.UpdateStatusFromContext(ctx, "kv-store", probe.ServiceStatusRunning)
-				logger.Info(ctx, "kvstore-set-server-ready")
+				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+				logger.Infow(ctx, "service-ready", log.Fields{"service-name": serviceName})
 
 				timeout = liveProbeInterval
 			}
@@ -130,7 +130,7 @@
 			break loop
 
 		case <-timeoutTimer.C:
-			logger.Info(ctx, "kvstore-perform-liveness-check-on-timeout")
+			logger.Infow(ctx, "perform-liveness-check-on-timeout", log.Fields{"service-name": serviceName})
 
 			// Trigger Liveness check if no liveness update received within the timeout period.
 			// The Liveness check will push Live state to same channel which this routine is