[VOL-4293] OpenONU Adapter update for gRPC migration

Change-Id: I05300d3b95b878f44576a99a05f53f52fdc0cda1
diff --git a/internal/pkg/config/config.go b/internal/pkg/config/config.go
index b61a909..aff1de3 100644
--- a/internal/pkg/config/config.go
+++ b/internal/pkg/config/config.go
@@ -19,73 +19,24 @@
 
 import (
 	"flag"
-	"fmt"
 	"os"
 	"time"
 )
 
 // Open ONU default constants
 const (
-	etcdStoreName               = "etcd"
-	defaultInstanceid           = "openonu"
-	defaultKafkaadapteraddress  = "127.0.0.1:9092"
-	defaultKafkaclusteraddress  = "127.0.0.1:9092"
-	defaultKvstoretype          = etcdStoreName
-	defaultKvstoretimeout       = 5 * time.Second
-	defaultKvstoreaddress       = "127.0.0.1:2379"
-	defaultLoglevel             = "WARN"
-	defaultBanner               = false
-	defaultDisplayVersionOnly   = false
-	defaultAccIncrEvto          = false
-	defaultTopic                = "openonu"
-	defaultCoreTopic            = "rwcore"
-	defaultEventTopic           = "voltha.events"
-	defaultOnunumber            = 1
-	defaultProbeHost            = ""
-	defaultProbePort            = 8080
-	defaultLiveProbeInterval    = 60 * time.Second
-	defaultNotLiveProbeInterval = 5 * time.Second // Probe more frequently when not alive
-	//defaultHearbeatFailReportInterval is the time in seconds the adapter will keep checking the hardware for heartbeat.
-	defaultHearbeatCheckInterval = 30 * time.Second
-	// defaultHearbeatFailReportInterval is the time adapter will wait before updating the state to the core.
-	defaultHearbeatFailReportInterval = 180 * time.Second
-	//defaultKafkaReconnectRetries -1: reconnect endlessly.
-	defaultKafkaReconnectRetries      = -1
-	defaultCurrentReplica             = 1
-	defaultTotalReplicas              = 1
-	defaultMaxTimeoutInterAdapterComm = 30 * time.Second
-	defaultMaxTimeoutReconciling      = 10 * time.Second
-	defaultOnuVendorIds               = "OPEN,ALCL,BRCM,TWSH,ALPH,ISKT,SFAA,BBSM,SCOM,ARPX,DACM,ERSN,HWTC,CIGG,ADTN,ARCA,AVMG"
-
-	// For Tracing
-	defaultTraceEnabled          = false
-	defaultTraceAgentAddress     = "127.0.0.1:6831"
-	defaultLogCorrelationEnabled = true
-
-	defaultMetricsEnabled     = false
-	defaultMibAuditInterval   = 0
-	defaultAlarmAuditInterval = 300 * time.Second
-
-	defaultOmciTimeout          = 3 * time.Second
-	defaultDlToAdapterTimeout   = 10 * time.Second
-	defaultDlToOnuTimeoutPer4MB = 60 * time.Minute //assumed for 4 MB of the image
-	//Mask to indicate which possibly active ONU UNI state  is really reported to the core
-	// compare python code - at the moment restrict active state to the first ONU UNI port
-	// check is limited to max 16 uni ports - cmp above UNI limit!!!
-	defaultUniPortMask = 0x0001
+	EtcdStoreName = "etcd"
+	OnuVendorIds  = "OPEN,ALCL,BRCM,TWSH,ALPH,ISKT,SFAA,BBSM,SCOM,ARPX,DACM,ERSN,HWTC,CIGG,ADTN,ARCA,AVMG"
 )
 
 // AdapterFlags represents the set of configurations used by the read-write adaptercore service
 type AdapterFlags struct {
 	// Command line parameters
 	InstanceID                  string
-	KafkaAdapterAddress         string
 	KafkaClusterAddress         string // NOTE this is unused across the adapter
 	KVStoreType                 string
 	KVStoreTimeout              time.Duration
 	KVStoreAddress              string
-	Topic                       string
-	CoreTopic                   string
 	EventTopic                  string
 	LogLevel                    string
 	OnuNumber                   int
@@ -114,169 +65,220 @@
 	DownloadToAdapterTimeout    time.Duration
 	DownloadToOnuTimeout4MB     time.Duration
 	UniPortMask                 int
-}
-
-// NewAdapterFlags returns a new RWCore config
-func NewAdapterFlags() *AdapterFlags {
-	var adapterFlags = AdapterFlags{ // Default values
-		InstanceID:                  defaultInstanceid,
-		KafkaAdapterAddress:         defaultKafkaadapteraddress,
-		KafkaClusterAddress:         defaultKafkaclusteraddress,
-		KVStoreType:                 defaultKvstoretype,
-		KVStoreTimeout:              defaultKvstoretimeout,
-		KVStoreAddress:              defaultKvstoreaddress,
-		Topic:                       defaultTopic,
-		CoreTopic:                   defaultCoreTopic,
-		EventTopic:                  defaultEventTopic,
-		LogLevel:                    defaultLoglevel,
-		OnuNumber:                   defaultOnunumber,
-		Banner:                      defaultBanner,
-		DisplayVersionOnly:          defaultDisplayVersionOnly,
-		AccIncrEvto:                 defaultAccIncrEvto,
-		ProbeHost:                   defaultProbeHost,
-		ProbePort:                   defaultProbePort,
-		LiveProbeInterval:           defaultLiveProbeInterval,
-		NotLiveProbeInterval:        defaultNotLiveProbeInterval,
-		HeartbeatCheckInterval:      defaultHearbeatCheckInterval,
-		HeartbeatFailReportInterval: defaultHearbeatFailReportInterval,
-		KafkaReconnectRetries:       defaultKafkaReconnectRetries,
-		CurrentReplica:              defaultCurrentReplica,
-		TotalReplicas:               defaultTotalReplicas,
-		MaxTimeoutInterAdapterComm:  defaultMaxTimeoutInterAdapterComm,
-		MaxTimeoutReconciling:       defaultMaxTimeoutReconciling,
-		TraceEnabled:                defaultTraceEnabled,
-		TraceAgentAddress:           defaultTraceAgentAddress,
-		LogCorrelationEnabled:       defaultLogCorrelationEnabled,
-		OnuVendorIds:                defaultOnuVendorIds,
-		MetricsEnabled:              defaultMetricsEnabled,
-		MibAuditInterval:            defaultMibAuditInterval,
-		AlarmAuditInterval:          defaultAlarmAuditInterval,
-		OmciTimeout:                 defaultOmciTimeout,
-		DownloadToAdapterTimeout:    defaultDlToAdapterTimeout,
-		DownloadToOnuTimeout4MB:     defaultDlToOnuTimeoutPer4MB,
-		UniPortMask:                 defaultUniPortMask,
-	}
-	return &adapterFlags
+	MinBackoffRetryDelay        time.Duration
+	MaxBackoffRetryDelay        time.Duration
+	AdapterEndpoint             string
+	GrpcAddress                 string
+	CoreEndpoint                string
+	RPCTimeout                  time.Duration
 }
 
 // ParseCommandArguments parses the arguments when running read-write adaptercore service
-func (so *AdapterFlags) ParseCommandArguments() {
+func (so *AdapterFlags) ParseCommandArguments(args []string) {
 
-	help := "Kafka - Adapter messaging address"
-	flag.StringVar(&(so.KafkaAdapterAddress), "kafka_adapter_address", defaultKafkaadapteraddress, help)
+	fs := flag.NewFlagSet(os.Args[0], flag.ExitOnError)
 
-	help = "Kafka - Cluster messaging address"
-	flag.StringVar(&(so.KafkaClusterAddress), "kafka_cluster_address", defaultKafkaclusteraddress, help)
+	fs.StringVar(&(so.KafkaClusterAddress),
+		"kafka_cluster_address",
+		"127.0.0.1:9092",
+		"Kafka - Cluster messaging address")
 
-	help = "Open ONU topic"
-	baseAdapterTopic := flag.String("adapter_topic", defaultTopic, help)
+	fs.StringVar(&(so.EventTopic),
+		"event_topic",
+		"voltha.events",
+		"Event topic")
 
-	help = "Core topic"
-	flag.StringVar(&(so.CoreTopic), "core_topic", defaultCoreTopic, help)
+	fs.StringVar(&(so.KVStoreType),
+		"kv_store_type",
+		EtcdStoreName,
+		"KV store type")
 
-	help = "Event topic"
-	flag.StringVar(&(so.EventTopic), "event_topic", defaultEventTopic, help)
+	fs.DurationVar(&(so.KVStoreTimeout),
+		"kv_store_request_timeout",
+		5*time.Second,
+		"The default timeout when making a kv store request")
 
-	help = "KV store type"
-	flag.StringVar(&(so.KVStoreType), "kv_store_type", defaultKvstoretype, help)
+	fs.StringVar(&(so.KVStoreAddress),
+		"kv_store_address",
+		"127.0.0.1:2379",
+		"KV store address")
 
-	help = "The default timeout when making a kv store request"
-	flag.DurationVar(&(so.KVStoreTimeout), "kv_store_request_timeout", defaultKvstoretimeout, help)
+	fs.StringVar(&(so.LogLevel),
+		"log_level",
+		"WARN",
+		"Log level")
 
-	help = "KV store address"
-	flag.StringVar(&(so.KVStoreAddress), "kv_store_address", defaultKvstoreaddress, help)
+	fs.IntVar(&(so.OnuNumber),
+		"onu_number",
+		1,
+		"Number of ONUs")
 
-	help = "Log level"
-	flag.StringVar(&(so.LogLevel), "log_level", defaultLoglevel, help)
+	fs.BoolVar(&(so.Banner),
+		"banner",
+		false,
+		"Show startup banner log lines")
 
-	help = "Number of ONUs"
-	flag.IntVar(&(so.OnuNumber), "onu_number", defaultOnunumber, help)
+	fs.BoolVar(&(so.DisplayVersionOnly),
+		"version",
+		false,
+		"Show version information and exit")
 
-	help = "Show startup banner log lines"
-	flag.BoolVar(&(so.Banner), "banner", defaultBanner, help)
+	fs.BoolVar(&(so.AccIncrEvto),
+		"accept_incr_evto",
+		false,
+		"Acceptance of incremental EVTOCD configuration")
 
-	help = "Show version information and exit"
-	flag.BoolVar(&(so.DisplayVersionOnly), "version", defaultDisplayVersionOnly, help)
+	fs.StringVar(&(so.ProbeHost),
+		"probe_host",
+		"",
+		"The address on which to listen to answer liveness and readiness probe queries over HTTP")
 
-	help = "Acceptance of incremental EVTOCD configuration"
-	flag.BoolVar(&(so.AccIncrEvto), "accept_incr_evto", defaultAccIncrEvto, help)
+	fs.IntVar(&(so.ProbePort),
+		"probe_port",
+		8080,
+		"The port on which to listen to answer liveness and readiness probe queries over HTTP")
 
-	help = "The address on which to listen to answer liveness and readiness probe queries over HTTP"
-	flag.StringVar(&(so.ProbeHost), "probe_host", defaultProbeHost, help)
+	fs.DurationVar(&(so.LiveProbeInterval),
+		"live_probe_interval",
+		60*time.Second,
+		"Number of seconds for the default liveliness check")
 
-	help = "The port on which to listen to answer liveness and readiness probe queries over HTTP"
-	flag.IntVar(&(so.ProbePort), "probe_port", defaultProbePort, help)
+	fs.DurationVar(&(so.NotLiveProbeInterval),
+		"not_live_probe_interval",
+		60*time.Second,
+		"Number of seconds for liveliness check if probe is not running")
 
-	help = "Number of seconds for the default liveliness check"
-	flag.DurationVar(&(so.LiveProbeInterval), "live_probe_interval", defaultLiveProbeInterval, help)
+	fs.DurationVar(&(so.HeartbeatCheckInterval),
+		"hearbeat_check_interval",
+		30*time.Second,
+		"Number of seconds for heartbeat check interval")
 
-	help = "Number of seconds for liveliness check if probe is not running"
-	flag.DurationVar(&(so.NotLiveProbeInterval), "not_live_probe_interval", defaultNotLiveProbeInterval, help)
+	fs.DurationVar(&(so.HeartbeatFailReportInterval),
+		"hearbeat_fail_interval",
+		30*time.Second,
+		"Number of seconds adapter has to wait before reporting core on the hearbeat check failure")
 
-	help = "Number of seconds for heartbeat check interval"
-	flag.DurationVar(&(so.HeartbeatCheckInterval), "hearbeat_check_interval", defaultHearbeatCheckInterval, help)
+	fs.IntVar(&(so.KafkaReconnectRetries),
+		"kafka_reconnect_retries",
+		-1,
+		"Number of retries to connect to Kafka")
 
-	help = "Number of seconds adapter has to wait before reporting core on the hearbeat check failure"
-	flag.DurationVar(&(so.HeartbeatFailReportInterval), "hearbeat_fail_interval", defaultHearbeatFailReportInterval, help)
+	fs.IntVar(&(so.CurrentReplica),
+		"current_replica",
+		1,
+		"Replica number of this particular instance")
 
-	help = "Number of retries to connect to Kafka"
-	flag.IntVar(&(so.KafkaReconnectRetries), "kafka_reconnect_retries", defaultKafkaReconnectRetries, help)
+	fs.IntVar(&(so.TotalReplicas),
+		"total_replica",
+		1,
+		"Total number of instances for this adapter")
 
-	help = "Replica number of this particular instance"
-	flag.IntVar(&(so.CurrentReplica), "current_replica", defaultCurrentReplica, help)
+	fs.DurationVar(&(so.MaxTimeoutInterAdapterComm),
+		"max_timeout_interadapter_comm",
+		30*time.Second,
+		"Maximum Number of seconds for the default interadapter communication timeout")
 
-	help = "Total number of instances for this adapter"
-	flag.IntVar(&(so.TotalReplicas), "total_replica", defaultTotalReplicas, help)
+	fs.DurationVar(&(so.MaxTimeoutReconciling),
+		"max_timeout_reconciling",
+		10*time.Second,
+		"Maximum Number of seconds for the default ONU reconciling timeout")
 
-	help = "Maximum Number of seconds for the default interadapter communication timeout"
-	flag.DurationVar(&(so.MaxTimeoutInterAdapterComm), "max_timeout_interadapter_comm",
-		defaultMaxTimeoutInterAdapterComm, help)
+	fs.BoolVar(&(so.TraceEnabled),
+		"trace_enabled",
+		false,
+		"Whether to send logs to tracing agent")
 
-	help = "Maximum Number of seconds for the default ONU reconciling timeout"
-	flag.DurationVar(&(so.MaxTimeoutReconciling), "max_timeout_reconciling",
-		defaultMaxTimeoutReconciling, help)
+	fs.StringVar(&(so.TraceAgentAddress),
+		"trace_agent_address",
+		"127.0.0.1:6831",
+		"The address of tracing agent to which span info should be sent")
 
-	help = "Whether to send logs to tracing agent"
-	flag.BoolVar(&(so.TraceEnabled), "trace_enabled", defaultTraceEnabled, help)
+	fs.BoolVar(&(so.LogCorrelationEnabled),
+		"log_correlation_enabled",
+		true,
+		"Whether to enrich log statements with fields denoting operation being executed for achieving correlation")
 
-	help = "The address of tracing agent to which span info should be sent"
-	flag.StringVar(&(so.TraceAgentAddress), "trace_agent_address", defaultTraceAgentAddress, help)
+	fs.StringVar(&(so.OnuVendorIds),
+		"allowed_onu_vendors",
+		OnuVendorIds,
+		"List of Allowed ONU Vendor Ids")
 
-	help = "Whether to enrich log statements with fields denoting operation being executed for achieving correlation"
-	flag.BoolVar(&(so.LogCorrelationEnabled), "log_correlation_enabled", defaultLogCorrelationEnabled, help)
+	fs.BoolVar(&(so.MetricsEnabled),
+		"metrics_enabled",
+		false,
+		"Whether to enable metrics collection")
 
-	help = "List of Allowed ONU Vendor Ids"
-	flag.StringVar(&(so.OnuVendorIds), "allowed_onu_vendors", defaultOnuVendorIds, help)
+	fs.DurationVar(&(so.MibAuditInterval),
+		"mib_audit_interval",
+		300*time.Second,
+		"Mib Audit Interval in seconds - the value zero will disable Mib Audit")
 
-	help = "Whether to enable metrics collection"
-	flag.BoolVar(&(so.MetricsEnabled), "metrics_enabled", defaultMetricsEnabled, help)
+	fs.DurationVar(&(so.OmciTimeout),
+		"omci_timeout",
+		3*time.Second,
+		"OMCI timeout duration - this timeout value is used on the OMCI channel for waiting on response from ONU")
 
-	help = "Mib Audit Interval in seconds - the value zero will disable Mib Audit"
-	flag.DurationVar(&(so.MibAuditInterval), "mib_audit_interval", defaultMibAuditInterval, help)
+	fs.DurationVar(&(so.AlarmAuditInterval),
+		"alarm_audit_interval",
+		300*time.Second,
+		"Alarm Audit Interval in seconds - the value zero will disable alarm audit")
 
-	help = "OMCI timeout duration - this timeout value is used on the OMCI channel for waiting on response from ONU"
-	flag.DurationVar(&(so.OmciTimeout), "omci_timeout", defaultOmciTimeout, help)
+	fs.DurationVar(&(so.DownloadToAdapterTimeout),
+		"download_to_adapter_timeout",
+		10*time.Second,
+		"File download to adapter timeout in seconds")
 
-	help = "Alarm Audit Interval in seconds - the value zero will disable alarm audit"
-	flag.DurationVar(&(so.AlarmAuditInterval), "alarm_audit_interval", defaultAlarmAuditInterval, help)
+	fs.DurationVar(&(so.DownloadToOnuTimeout4MB),
+		"download_to_onu_timeout_4MB",
+		60*time.Minute,
+		"File download to ONU timeout in minutes for a block of 4MB")
 
-	help = "File download to adapter timeout in seconds"
-	flag.DurationVar(&(so.DownloadToAdapterTimeout), "download_to_adapter_timeout", defaultDlToAdapterTimeout, help)
+	//Mask to indicate which possibly active ONU UNI state  is really reported to the core
+	// compare python code - at the moment restrict active state to the first ONU UNI port
+	// check is limited to max 16 uni ports - cmp above UNI limit!!!
+	fs.IntVar(&(so.UniPortMask),
+		"uni_port_mask",
+		0x0001,
+		"The bitmask to identify UNI ports that need to be enabled")
 
-	help = "File download to ONU timeout in minutes for a block of 4MB"
-	flag.DurationVar(&(so.DownloadToOnuTimeout4MB), "download_to_onu_timeout_4MB", defaultDlToOnuTimeoutPer4MB, help)
+	fs.StringVar(&(so.GrpcAddress),
+		"grpc_address",
+		":50060",
+		"Adapter GRPC Server address")
 
-	help = "The bitmask to identify UNI ports that need to be enabled"
-	flag.IntVar(&(so.UniPortMask), "uni_port_mask", defaultUniPortMask, help)
+	fs.StringVar(&(so.CoreEndpoint),
+		"core_endpoint",
+		":55555",
+		"Core endpoint")
 
-	flag.Parse()
+	fs.StringVar(&(so.AdapterEndpoint),
+		"adapter_endpoint",
+		"",
+		"Adapter Endpoint")
+
+	fs.DurationVar(&(so.RPCTimeout),
+		"rpc_timeout",
+		10*time.Second,
+		"The default timeout when making an RPC request")
+
+	fs.DurationVar(&(so.MinBackoffRetryDelay),
+		"min_retry_delay",
+		500*time.Millisecond,
+		"The minimum number of milliseconds to delay before a connection retry attempt")
+
+	fs.DurationVar(&(so.MaxBackoffRetryDelay),
+		"max_retry_delay",
+		10*time.Second,
+		"The maximum number of milliseconds to delay before a connection retry attempt")
+
+	_ = fs.Parse(args)
 	containerName := getContainerInfo()
 	if len(containerName) > 0 {
 		so.InstanceID = containerName
+	} else {
+		so.InstanceID = "openonu"
 	}
 
-	so.Topic = fmt.Sprintf("%s_%d", *baseAdapterTopic, int32(so.CurrentReplica))
-
 }
 
 func getContainerInfo() string {
diff --git a/internal/pkg/onuadaptercore/adapter_download_manager.go b/internal/pkg/onuadaptercore/adapter_download_manager.go
index 6803aed..6328f72 100644
--- a/internal/pkg/onuadaptercore/adapter_download_manager.go
+++ b/internal/pkg/onuadaptercore/adapter_download_manager.go
@@ -30,9 +30,9 @@
 	"sync"
 	"time"
 
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 // ### downloadToAdapter related definitions  ####
diff --git a/internal/pkg/onuadaptercore/alarm_manager.go b/internal/pkg/onuadaptercore/alarm_manager.go
index 0dbc5b8..b9d4d68 100644
--- a/internal/pkg/onuadaptercore/alarm_manager.go
+++ b/internal/pkg/onuadaptercore/alarm_manager.go
@@ -28,9 +28,9 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"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/voltha"
+	"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/voltha"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/common.go b/internal/pkg/onuadaptercore/common.go
index 0a39a7e..b0b10e3 100644
--- a/internal/pkg/onuadaptercore/common.go
+++ b/internal/pkg/onuadaptercore/common.go
@@ -18,7 +18,7 @@
 package adaptercoreonu
 
 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/internal/pkg/onuadaptercore/device_handler.go b/internal/pkg/onuadaptercore/device_handler.go
index cfe99c9..2c5a4c8 100644
--- a/internal/pkg/onuadaptercore/device_handler.go
+++ b/internal/pkg/onuadaptercore/device_handler.go
@@ -25,25 +25,23 @@
 	"sync"
 	"time"
 
-	"github.com/opencord/voltha-protos/v4/go/tech_profile"
+	"github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
+	"github.com/opencord/voltha-protos/v5/go/tech_profile"
 
 	"github.com/gogo/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
 	"github.com/looplab/fsm"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	flow "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	vc "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"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	of "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	oop "github.com/opencord/voltha-protos/v4/go/openolt"
-	"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/eventif"
+	flow "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	vc "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"
+	of "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	oop "github.com/opencord/voltha-protos/v5/go/openolt"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 // Constants for timeouts
@@ -178,11 +176,11 @@
 	parentID         string
 	ponPortNumber    uint32
 
-	coreProxy    adapterif.CoreProxy
-	AdapterProxy adapterif.AdapterProxy
-	EventProxy   eventif.EventProxy
+	coreClient *vgrpc.Client
+	EventProxy eventif.EventProxy
 
 	pmConfigs *voltha.PmConfigs
+	config    *config.AdapterFlags
 
 	pOpenOnuAc      *OpenONUAC
 	pDeviceStateFsm *fsm.FSM
@@ -237,11 +235,11 @@
 }
 
 //newDeviceHandler creates a new device handler
-func newDeviceHandler(ctx context.Context, cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep eventif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *deviceHandler {
+func newDeviceHandler(ctx context.Context, cc *vgrpc.Client, ep eventif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *deviceHandler {
 	var dh deviceHandler
-	dh.coreProxy = cp
-	dh.AdapterProxy = ap
+	dh.coreClient = cc
 	dh.EventProxy = ep
+	dh.config = adapter.config
 	cloned := (proto.Clone(device)).(*voltha.Device)
 	dh.deviceID = cloned.Id
 	dh.DeviceType = cloned.Type
@@ -327,7 +325,7 @@
 
 //adoptOrReconcileDevice adopts the ONU device
 func (dh *deviceHandler) adoptOrReconcileDevice(ctx context.Context, device *voltha.Device) {
-	logger.Debugw(ctx, "Adopt_or_reconcile_device", log.Fields{"device-id": device.Id, "Address": device.GetHostAndPort()})
+	logger.Debugw(ctx, "adopt_or_reconcile_device", log.Fields{"device-id": device.Id, "Address": device.GetHostAndPort()})
 
 	logger.Debugw(ctx, "Device FSM: ", log.Fields{"state": string(dh.pDeviceStateFsm.Current())})
 	if dh.pDeviceStateFsm.Is(devStNull) {
@@ -338,7 +336,7 @@
 		// device.PmConfigs is not nil in cases when adapter restarts. We should not re-set the core again.
 		if device.PmConfigs == nil {
 			// Now, set the initial PM configuration for that device
-			if err := dh.coreProxy.DevicePMConfigUpdate(ctx, dh.pmConfigs); err != nil {
+			if err := dh.updatePMConfigInCore(ctx, dh.pmConfigs); err != nil {
 				logger.Errorw(ctx, "error updating pm config to core", log.Fields{"device-id": dh.deviceID, "err": err})
 			}
 		}
@@ -348,36 +346,26 @@
 
 }
 
-func (dh *deviceHandler) processInterAdapterOMCIReceiveMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	msgBody := msg.GetBody()
-	omciMsg := &ic.InterAdapterOmciMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, omciMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-omci-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
+func (dh *deviceHandler) handleOMCIIndication(ctx context.Context, msg *ic.OmciMessage) error {
 	/* msg print moved symmetrically to omci_cc, if wanted here as additional debug, than perhaps only based on additional debug setting!
 	//assuming omci message content is hex coded!
 	// with restricted output of 16(?) bytes would be ...omciMsg.Message[:16]
 	logger.Debugw(ctx, "inter-adapter-recv-omci", log.Fields{
 		"device-id": dh.deviceID, "RxOmciMessage": hex.EncodeToString(omciMsg.Message)})
 	*/
+
 	pDevEntry := dh.getOnuDeviceEntry(ctx, true)
 	if pDevEntry != nil {
 		if pDevEntry.PDevOmciCC != nil {
-			return pDevEntry.PDevOmciCC.receiveMessage(log.WithSpanFromContext(context.TODO(), ctx), omciMsg.Message)
+			return pDevEntry.PDevOmciCC.receiveMessage(log.WithSpanFromContext(context.TODO(), ctx), msg.Message)
 		}
-		logger.Debugw(ctx, "omciCC not ready to receive omci messages - incoming omci message ignored", log.Fields{"rxMsg": omciMsg.Message})
+		logger.Debugw(ctx, "omciCC not ready to receive omci messages - incoming omci message ignored", log.Fields{"rxMsg": msg.Message})
 	}
 	logger.Errorw(ctx, "No valid OnuDevice -aborting", log.Fields{"device-id": dh.deviceID})
 	return fmt.Errorf("no valid OnuDevice: %s", dh.deviceID)
 }
 
-func (dh *deviceHandler) processInterAdapterTechProfileDownloadReqMessage(
-	ctx context.Context,
-	msg *ic.InterAdapterMessage) error {
-
+func (dh *deviceHandler) handleTechProfileDownloadRequest(ctx context.Context, techProfMsg *ic.TechProfileDownloadMessage) error {
 	logger.Infow(ctx, "tech-profile-download-request", log.Fields{"device-id": dh.deviceID})
 
 	pDevEntry := dh.getOnuDeviceEntry(ctx, true)
@@ -400,14 +388,6 @@
 	// at least 'mib-downloaded' should be reached for processing of this specific ONU configuration
 	//  if (dh.deviceReason == "stopping-openomci") || (dh.deviceReason == "omci-admin-lock")
 
-	msgBody := msg.GetBody()
-	techProfMsg := &ic.InterAdapterTechProfileDownloadMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, techProfMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-techprof-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
 	// we have to lock access to TechProfile processing based on different messageType calls or
 	// even to fast subsequent calls of the same messageType as well as OnuKVStore processing due
 	// to possible concurrent access by flow processing
@@ -429,7 +409,7 @@
 	if bTpModify := pDevEntry.updateOnuUniTpPath(ctx, uniID, uint8(tpID), techProfMsg.TpInstancePath); bTpModify {
 
 		switch tpInst := techProfMsg.TechTpInstance.(type) {
-		case *ic.InterAdapterTechProfileDownloadMessage_TpInstance:
+		case *ic.TechProfileDownloadMessage_TpInstance:
 			logger.Debugw(ctx, "onu-uni-tp-path-modified", log.Fields{"uniID": uniID, "tp-path": techProfMsg.TpInstancePath, "tpID": tpID})
 			//	if there has been some change for some uni TechProfilePath
 			//in order to allow concurrent calls to other dh instances we do not wait for execution here
@@ -475,9 +455,8 @@
 	return nil
 }
 
-func (dh *deviceHandler) processInterAdapterDeleteGemPortReqMessage(
-	ctx context.Context,
-	msg *ic.InterAdapterMessage) error {
+func (dh *deviceHandler) handleDeleteGemPortRequest(ctx context.Context, delGemPortMsg *ic.DeleteGemPortMessage) error {
+	logger.Infow(ctx, "delete-gem-port-request", log.Fields{"device-id": dh.deviceID})
 
 	if dh.pOnuTP == nil {
 		//should normally not happen ...
@@ -485,15 +464,6 @@
 			log.Fields{"device-id": dh.deviceID})
 		return fmt.Errorf("techProfile DelGem request while onuTechProf instance not setup: %s", dh.deviceID)
 	}
-
-	msgBody := msg.GetBody()
-	delGemPortMsg := &ic.InterAdapterDeleteGemPortMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, delGemPortMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-delete-gem-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
 	//compare TECH_PROFILE_DOWNLOAD_REQUEST
 	dh.pOnuTP.lockTpProcMutex()
 	defer dh.pOnuTP.unlockTpProcMutex()
@@ -516,9 +486,8 @@
 
 }
 
-func (dh *deviceHandler) processInterAdapterDeleteTcontReqMessage(
-	ctx context.Context,
-	msg *ic.InterAdapterMessage) error {
+func (dh *deviceHandler) handleDeleteTcontRequest(ctx context.Context, delTcontMsg *ic.DeleteTcontMessage) error {
+	logger.Infow(ctx, "delete-tcont-request", log.Fields{"device-id": dh.deviceID})
 
 	pDevEntry := dh.getOnuDeviceEntry(ctx, true)
 	if pDevEntry == nil {
@@ -532,14 +501,6 @@
 		return fmt.Errorf("techProfile DelTcont request while onuTechProf instance not setup: %s", dh.deviceID)
 	}
 
-	msgBody := msg.GetBody()
-	delTcontMsg := &ic.InterAdapterDeleteTcontMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, delTcontMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-delete-tcont-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
 	//compare TECH_PROFILE_DOWNLOAD_REQUEST
 	dh.pOnuTP.lockTpProcMutex()
 	defer dh.pOnuTP.unlockTpProcMutex()
@@ -616,52 +577,10 @@
 	return nil
 }
 
-//processInterAdapterMessage sends the proxied messages to the target device
-// If the proxy address is not found in the unmarshalled message, it first fetches the onu device for which the message
-// is meant, and then send the unmarshalled omci message to this onu
-func (dh *deviceHandler) processInterAdapterMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	msgID := msg.Header.Id
-	msgType := msg.Header.Type
-	fromTopic := msg.Header.FromTopic
-	toTopic := msg.Header.ToTopic
-	toDeviceID := msg.Header.ToDeviceId
-	proxyDeviceID := msg.Header.ProxyDeviceId
-	logger.Debugw(ctx, "InterAdapter message header", log.Fields{"msgID": msgID, "msgType": msgType,
-		"fromTopic": fromTopic, "toTopic": toTopic, "toDeviceID": toDeviceID, "proxyDeviceID": proxyDeviceID})
-
-	switch msgType {
-	// case ic.InterAdapterMessageType_ONU_IND_REQUEST: was handled by OpenONUAC already - see comments there
-	//OMCI_RESPONSE also accepted acc. to VOL-3756 (OMCI_REQUEST request was legacy code)
-	case ic.InterAdapterMessageType_OMCI_RESPONSE, ic.InterAdapterMessageType_OMCI_REQUEST:
-		{
-			return dh.processInterAdapterOMCIReceiveMessage(ctx, msg)
-		}
-	case ic.InterAdapterMessageType_TECH_PROFILE_DOWNLOAD_REQUEST:
-		{
-			return dh.processInterAdapterTechProfileDownloadReqMessage(ctx, msg)
-		}
-	case ic.InterAdapterMessageType_DELETE_GEM_PORT_REQUEST:
-		{
-			return dh.processInterAdapterDeleteGemPortReqMessage(ctx, msg)
-
-		}
-	case ic.InterAdapterMessageType_DELETE_TCONT_REQUEST:
-		{
-			return dh.processInterAdapterDeleteTcontReqMessage(ctx, msg)
-		}
-	default:
-		{
-			logger.Errorw(ctx, "inter-adapter-unhandled-type", log.Fields{
-				"msgType": msg.Header.Type, "device-id": dh.deviceID})
-			return fmt.Errorf("inter-adapter-unhandled-type: %d, %s", msg.Header.Type, dh.deviceID)
-		}
-	}
-}
-
 //FlowUpdateIncremental removes and/or adds the flow changes on a given device
 func (dh *deviceHandler) FlowUpdateIncremental(ctx context.Context,
-	apOfFlowChanges *openflow_13.FlowChanges,
-	apOfGroupChanges *openflow_13.FlowGroupChanges, apFlowMetaData *voltha.FlowMetadata) error {
+	apOfFlowChanges *of.FlowChanges,
+	apOfGroupChanges *of.FlowGroupChanges, apFlowMetaData *voltha.FlowMetadata) error {
 	logger.Debugw(ctx, "FlowUpdateIncremental started", log.Fields{"device-id": dh.deviceID, "metadata": apFlowMetaData})
 	var retError error = nil
 	//Remove flows (always remove flows first - remove old and add new with same cookie may be part of the same request)
@@ -806,8 +725,11 @@
 		} else {
 			logger.Debugw(ctx, "DeviceStateUpdate upon disable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 				"OperStatus": voltha.OperStatus_UNKNOWN, "device-id": dh.deviceID})
-			if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx),
-				dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN); err != nil {
+			if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+				DeviceId:   dh.deviceID,
+				ConnStatus: voltha.ConnectStatus_REACHABLE,
+				OperStatus: voltha.OperStatus_UNKNOWN,
+			}); err != nil {
 				//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 				logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 			}
@@ -914,10 +836,16 @@
 		techProfsFound = true // set to true if we found TP once for any UNI port
 		for tpID := range uniData.PersTpPathMap {
 			// Request the TpInstance again from the openolt adapter in case of reconcile
-			iaTechTpInst, err := dh.AdapterProxy.TechProfileInstanceRequest(ctx, uniData.PersTpPathMap[tpID],
-				dh.device.ParentPortNo, dh.device.ProxyAddress.OnuId, uint32(uniData.PersUniID),
-				dh.pOpenOnuAc.config.Topic, dh.ProxyAddressType,
-				dh.parentID, dh.ProxyAddressID)
+			iaTechTpInst, err := dh.getTechProfileInstanceFromParentAdapter(ctx,
+				dh.device.ProxyAddress.AdapterEndpoint,
+				&ic.TechProfileInstanceRequestMessage{
+					DeviceId:       dh.device.Id,
+					TpInstancePath: uniData.PersTpPathMap[tpID],
+					ParentDeviceId: dh.parentID,
+					ParentPonPort:  dh.device.ParentPortNo,
+					OnuId:          dh.device.ProxyAddress.OnuId,
+					UniId:          uint32(uniData.PersUniID),
+				})
 			if err != nil || iaTechTpInst == nil {
 				logger.Errorw(ctx, "error fetching tp instance",
 					log.Fields{"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.deviceID, "err": err})
@@ -926,7 +854,7 @@
 			}
 			var tpInst tech_profile.TechProfileInstance
 			switch techTpInst := iaTechTpInst.TechTpInstance.(type) {
-			case *ic.InterAdapterTechProfileDownloadMessage_TpInstance: // supports only GPON, XGPON, XGS-PON
+			case *ic.TechProfileDownloadMessage_TpInstance: // supports only GPON, XGPON, XGS-PON
 				tpInst = *techTpInst.TpInstance
 				logger.Debugw(ctx, "received-tp-instance-successfully-after-reconcile", log.Fields{
 					"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.deviceID})
@@ -1152,8 +1080,11 @@
 
 	logger.Debugw(ctx, "call DeviceStateUpdate upon reboot", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 		"OperStatus": voltha.OperStatus_DISCOVERED, "device-id": dh.deviceID})
-	if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.ConnectStatus_REACHABLE,
-		voltha.OperStatus_DISCOVERED); err != nil {
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: voltha.ConnectStatus_REACHABLE,
+		OperStatus: voltha.OperStatus_DISCOVERED,
+	}); err != nil {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 		logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 		return
@@ -1294,8 +1225,7 @@
 	dh.lockUpgradeFsm.RLock()
 	if dh.pOnuUpradeFsm != nil {
 		dh.lockUpgradeFsm.RUnlock()
-		onuVolthaDevice, getErr := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-			dh.deviceID, dh.deviceID)
+		onuVolthaDevice, getErr := dh.getDeviceFromCore(ctx, dh.deviceID)
 		if getErr != nil || onuVolthaDevice == nil {
 			logger.Errorw(ctx, "Failed to fetch Onu device for image activation", log.Fields{"device-id": dh.deviceID, "err": getErr})
 			return nil, fmt.Errorf("could not fetch device for device-id: %s", dh.deviceID)
@@ -1358,8 +1288,7 @@
 	dh.lockUpgradeFsm.RLock()
 	if dh.pOnuUpradeFsm != nil {
 		dh.lockUpgradeFsm.RUnlock()
-		onuVolthaDevice, getErr := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-			dh.deviceID, dh.deviceID)
+		onuVolthaDevice, getErr := dh.getDeviceFromCore(ctx, dh.deviceID)
 		if getErr != nil || onuVolthaDevice == nil {
 			logger.Errorw(ctx, "Failed to fetch Onu device for image commitment", log.Fields{"device-id": dh.deviceID, "err": getErr})
 			return nil, fmt.Errorf("could not fetch device for device-id: %s", dh.deviceID)
@@ -1513,7 +1442,9 @@
 
 	if !dh.isReconciling() {
 		logger.Infow(ctx, "DeviceUpdate", log.Fields{"deviceReason": dh.device.Reason, "device-id": dh.deviceID})
-		_ = dh.coreProxy.DeviceUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.device)
+		if err := dh.updateDeviceInCore(ctx, dh.device); err != nil {
+			logger.Errorw(ctx, "device-update-failed", log.Fields{"device-id": dh.device.Id, "error": err})
+		}
 		//TODO Need to Update Device Reason To CORE as part of device update userstory
 	} else {
 		logger.Debugw(ctx, "reconciling - don't notify core about DeviceUpdate",
@@ -1547,6 +1478,7 @@
 		}
 
 		pPonPort := &voltha.Port{
+			DeviceId:   dh.deviceID,
 			PortNo:     ponPortNo,
 			Label:      fmt.Sprintf("pon-%d", ponPortNo),
 			Type:       voltha.Port_PON_ONU,
@@ -1554,7 +1486,7 @@
 			Peers: []*voltha.Port_PeerPort{{DeviceId: dh.parentID, // Peer device  is OLT
 				PortNo: ponPortNo}}, // Peer port is parent's port number
 		}
-		if err = dh.coreProxy.PortCreated(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, pPonPort); err != nil {
+		if err = dh.createPortInCore(ctx, pPonPort); err != nil {
 			logger.Fatalf(ctx, "Device FSM: PortCreated-failed-%s", err)
 			e.Cancel(err)
 			return
@@ -1814,8 +1746,12 @@
 		}
 		logger.Debugw(ctx, "call DeviceStateUpdate upon create interface", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 			"OperStatus": voltha.OperStatus_ACTIVATING, "device-id": dh.deviceID})
-		if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
-			voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING); err != nil {
+
+		if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+			DeviceId:   dh.deviceID,
+			OperStatus: voltha.OperStatus_ACTIVATING,
+			ConnStatus: voltha.ConnectStatus_REACHABLE,
+		}); err != nil {
 			//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 			logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 		}
@@ -2031,8 +1967,11 @@
 		}
 		logger.Debugw(ctx, "call DeviceStateUpdate upon update interface", log.Fields{"ConnectStatus": voltha.ConnectStatus_UNREACHABLE,
 			"OperStatus": voltha.OperStatus_DISCOVERED, "device-id": dh.deviceID})
-		if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
-			voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_DISCOVERED); err != nil {
+		if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+			DeviceId:   dh.deviceID,
+			ConnStatus: voltha.ConnectStatus_UNREACHABLE,
+			OperStatus: voltha.OperStatus_DISCOVERED,
+		}); err != nil {
 			//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 			logger.Errorw(ctx, "error-updating-device-state unreachable-discovered",
 				log.Fields{"device-id": dh.deviceID, "error": err})
@@ -2208,8 +2147,11 @@
 		// in case of adapter restart connected to an ONU upgrade I would not rely on the image quality
 		// maybe some 'forced' commitment can be done in this situation from system management (or upgrade restarted)
 		dh.checkOnOnuImageCommit(ctx)
-		if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
-			voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE); err != nil {
+		if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+			DeviceId:   dh.deviceID,
+			ConnStatus: voltha.ConnectStatus_REACHABLE,
+			OperStatus: voltha.OperStatus_ACTIVE,
+		}); err != nil {
 			//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 			logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 		} else {
@@ -2292,8 +2234,12 @@
 func (dh *deviceHandler) processUniDisableStateDoneEvent(ctx context.Context, devEvent OnuDeviceEvent) {
 	logger.Debugw(ctx, "DeviceStateUpdate upon disable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 		"OperStatus": voltha.OperStatus_UNKNOWN, "device-id": dh.deviceID})
-	if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx),
-		dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN); err != nil {
+
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: voltha.ConnectStatus_REACHABLE,
+		OperStatus: voltha.OperStatus_UNKNOWN,
+	}); err != nil {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 		logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 	}
@@ -2322,8 +2268,11 @@
 func (dh *deviceHandler) processUniEnableStateDoneEvent(ctx context.Context, devEvent OnuDeviceEvent) {
 	logger.Debugw(ctx, "DeviceStateUpdate upon re-enable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 		"OperStatus": voltha.OperStatus_ACTIVE, "device-id": dh.deviceID})
-	if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.ConnectStatus_REACHABLE,
-		voltha.OperStatus_ACTIVE); err != nil {
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: voltha.ConnectStatus_REACHABLE,
+		OperStatus: voltha.OperStatus_ACTIVE,
+	}); err != nil {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 		logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 	}
@@ -2537,7 +2486,16 @@
 			uniPort.setOperState(vc.OperStatus_ACTIVE)
 			if !dh.isReconciling() {
 				//maybe also use getter functions on uniPort - perhaps later ...
-				go dh.coreProxy.PortStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.Port_ETHERNET_UNI, uniPort.portNo, uniPort.operState)
+				go func(port *onuUniPort) {
+					if err := dh.updatePortStateInCore(ctx, &ic.PortState{
+						DeviceId:   dh.deviceID,
+						PortType:   voltha.Port_ETHERNET_UNI,
+						PortNo:     port.portNo,
+						OperStatus: port.operState,
+					}); err != nil {
+						logger.Errorw(ctx, "port-state-update-failed", log.Fields{"error": err, "port-no": uniPort.portNo, "device-id": dh.deviceID})
+					}
+				}(uniPort)
 			} else {
 				logger.Debugw(ctx, "reconciling - don't notify core about PortStateUpdate", log.Fields{"device-id": dh.deviceID})
 			}
@@ -2557,7 +2515,16 @@
 			uniPort.setOperState(vc.OperStatus_UNKNOWN)
 			if !dh.isReconciling() {
 				//maybe also use getter functions on uniPort - perhaps later ...
-				go dh.coreProxy.PortStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.Port_ETHERNET_UNI, uniPort.portNo, uniPort.operState)
+				go func(port *onuUniPort) {
+					if err := dh.updatePortStateInCore(ctx, &ic.PortState{
+						DeviceId:   dh.deviceID,
+						PortType:   voltha.Port_ETHERNET_UNI,
+						PortNo:     port.portNo,
+						OperStatus: port.operState,
+					}); err != nil {
+						logger.Errorw(ctx, "port-state-update-failed", log.Fields{"error": err, "port-no": uniPort.portNo, "device-id": dh.deviceID})
+					}
+				}(uniPort)
 			} else {
 				logger.Debugw(ctx, "reconciling - don't notify core about PortStateUpdate", log.Fields{"device-id": dh.deviceID})
 			}
@@ -2573,7 +2540,7 @@
 	eventContext := make(map[string]string)
 	//Populating event context
 	//  assume giving ParentId in GetDevice twice really gives the ParentDevice (there is no GetParentDevice()...)
-	parentDevice, err := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx), dh.parentID, dh.parentID)
+	parentDevice, err := dh.getDeviceFromCore(ctx, dh.parentID)
 	if err != nil || parentDevice == nil {
 		logger.Errorw(ctx, "Failed to fetch parent device for OnuEvent",
 			log.Fields{"parentID": dh.parentID, "err": err})
@@ -2853,7 +2820,7 @@
 
 	return kvbackend
 }
-func (dh *deviceHandler) getFlowOfbFields(ctx context.Context, apFlowItem *ofp.OfpFlowStats, loMatchVlan *uint16,
+func (dh *deviceHandler) getFlowOfbFields(ctx context.Context, apFlowItem *of.OfpFlowStats, loMatchVlan *uint16,
 	loAddPcp *uint8, loIPProto *uint32) {
 
 	for _, field := range flow.GetOfbFields(apFlowItem) {
@@ -2930,7 +2897,7 @@
 	} //for all OfbFields
 }
 
-func (dh *deviceHandler) getFlowActions(ctx context.Context, apFlowItem *ofp.OfpFlowStats, loSetPcp *uint8, loSetVlan *uint16) {
+func (dh *deviceHandler) getFlowActions(ctx context.Context, apFlowItem *of.OfpFlowStats, loSetPcp *uint8, loSetVlan *uint16) {
 	for _, action := range flow.GetActions(apFlowItem) {
 		switch action.Type {
 		/* not used:
@@ -2976,7 +2943,7 @@
 }
 
 //addFlowItemToUniPort parses the actual flow item to add it to the UniPort
-func (dh *deviceHandler) addFlowItemToUniPort(ctx context.Context, apFlowItem *ofp.OfpFlowStats, apUniPort *onuUniPort,
+func (dh *deviceHandler) addFlowItemToUniPort(ctx context.Context, apFlowItem *of.OfpFlowStats, apUniPort *onuUniPort,
 	apFlowMetaData *voltha.FlowMetadata) error {
 	var loSetVlan uint16 = uint16(of.OfpVlanId_OFPVID_NONE)      //noValidEntry
 	var loMatchVlan uint16 = uint16(of.OfpVlanId_OFPVID_PRESENT) //reserved VLANID entry
@@ -3067,7 +3034,7 @@
 }
 
 //removeFlowItemFromUniPort parses the actual flow item to remove it from the UniPort
-func (dh *deviceHandler) removeFlowItemFromUniPort(ctx context.Context, apFlowItem *ofp.OfpFlowStats, apUniPort *onuUniPort) error {
+func (dh *deviceHandler) removeFlowItemFromUniPort(ctx context.Context, apFlowItem *of.OfpFlowStats, apUniPort *onuUniPort) error {
 	//optimization and assumption: the flow cookie uniquely identifies the flow and with that the internal rule
 	//hence only the cookie is used here to find the relevant flow and possibly remove the rule
 	//no extra check is done on the rule parameters
@@ -3300,7 +3267,10 @@
 	dh.setDeviceReason(deviceReason)
 	if notifyCore {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing at calling position
-		if err := dh.coreProxy.DeviceReasonUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, deviceReasonMap[deviceReason]); err != nil {
+		if err := dh.updateDeviceReasonInCore(ctx, &ic.DeviceReason{
+			DeviceId: dh.deviceID,
+			Reason:   deviceReasonMap[deviceReason],
+		}); err != nil {
 			logger.Errorf(ctx, "DeviceReasonUpdate error: %s",
 				log.Fields{"device-id": dh.deviceID, "error": err}, deviceReasonMap[deviceReason])
 			return err
@@ -3715,7 +3685,11 @@
 					}
 					logger.Debugw(ctx, "reconciling has been finished in time",
 						log.Fields{"device-id": dh.deviceID})
-					if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.deviceID, connectStatus, operState); err != nil {
+					if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+						DeviceId:   dh.deviceID,
+						ConnStatus: connectStatus,
+						OperStatus: operState,
+					}); err != nil {
 						logger.Errorw(ctx, "unable to update device state to core",
 							log.Fields{"device-id": dh.deviceID, "Err": err})
 					}
@@ -3830,8 +3804,130 @@
 	}
 
 	logger.Debugw(ctx, "Core DeviceStateUpdate", log.Fields{"connectStatus": connectStatus, "operState": voltha.OperStatus_RECONCILING_FAILED})
-	if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.deviceID, connectStatus, voltha.OperStatus_RECONCILING_FAILED); err != nil {
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: connectStatus,
+		OperStatus: voltha.OperStatus_RECONCILING_FAILED,
+	}); err != nil {
 		logger.Errorw(ctx, "unable to update device state to core",
 			log.Fields{"device-id": dh.deviceID, "Err": err})
 	}
 }
+
+/*
+Helper functions to communicate with Core
+*/
+
+func (dh *deviceHandler) getDeviceFromCore(ctx context.Context, deviceID string) (*voltha.Device, error) {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	logger.Debugw(subCtx, "get-device-from-core", log.Fields{"device-id": deviceID})
+	return cClient.GetDevice(subCtx, &vc.ID{Id: deviceID})
+}
+
+func (dh *deviceHandler) updateDeviceStateInCore(ctx context.Context, deviceStateFilter *ic.DeviceStateFilter) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DeviceStateUpdate(subCtx, deviceStateFilter)
+	logger.Debugw(subCtx, "device-updated-in-core", log.Fields{"device-state": deviceStateFilter, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updatePMConfigInCore(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DevicePMConfigUpdate(subCtx, pmConfigs)
+	logger.Debugw(subCtx, "pmconfig-updated-in-core", log.Fields{"pm-configs": pmConfigs, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updateDeviceInCore(ctx context.Context, device *voltha.Device) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DeviceUpdate(subCtx, device)
+	logger.Debugw(subCtx, "device-updated-in-core", log.Fields{"device-id": device.Id, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) createPortInCore(ctx context.Context, port *voltha.Port) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.PortCreated(subCtx, port)
+	logger.Debugw(subCtx, "port-created-in-core", log.Fields{"port": port, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updatePortStateInCore(ctx context.Context, portState *ic.PortState) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.PortStateUpdate(subCtx, portState)
+	logger.Debugw(subCtx, "port-state-updated-in-core", log.Fields{"port-state": portState, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updateDeviceReasonInCore(ctx context.Context, reason *ic.DeviceReason) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DeviceReasonUpdate(subCtx, reason)
+	logger.Debugw(subCtx, "device-reason-updated-in-core", log.Fields{"reason": reason, "error": err})
+	return err
+}
+
+/*
+Helper functions to communicate with parent adapter
+*/
+
+func (dh *deviceHandler) getTechProfileInstanceFromParentAdapter(ctx context.Context, parentEndpoint string,
+	request *ic.TechProfileInstanceRequestMessage) (*ic.TechProfileDownloadMessage, error) {
+	pgClient, err := dh.pOpenOnuAc.getParentAdapterServiceClient(parentEndpoint)
+	if err != nil || pgClient == nil {
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.MaxTimeoutInterAdapterComm)
+	defer cancel()
+	logger.Debugw(subCtx, "get-tech-profile-instance", log.Fields{"request": request, "parent-endpoint": parentEndpoint})
+	return pgClient.GetTechProfileInstance(subCtx, request)
+}
+
+func (dh *deviceHandler) sendOMCIRequest(ctx context.Context, parentEndpoint string, request *ic.OmciMessage) error {
+	pgClient, err := dh.pOpenOnuAc.getParentAdapterServiceClient(parentEndpoint)
+	if err != nil || pgClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.MaxTimeoutInterAdapterComm)
+	defer cancel()
+	logger.Debugw(subCtx, "send-omci-request", log.Fields{"request": request, "parent-endpoint": parentEndpoint})
+	_, err = pgClient.ProxyOmciRequest(subCtx, request)
+	if err != nil {
+		logger.Errorw(ctx, "omci-failure", log.Fields{"request": request, "error": err, "request-parent": request.ParentDeviceId, "request-child": request.ChildDeviceId, "request-proxy": request.ProxyAddress})
+	}
+	return err
+}
diff --git a/internal/pkg/onuadaptercore/file_download_manager.go b/internal/pkg/onuadaptercore/file_download_manager.go
index 4793e69..5c3226e 100644
--- a/internal/pkg/onuadaptercore/file_download_manager.go
+++ b/internal/pkg/onuadaptercore/file_download_manager.go
@@ -29,7 +29,7 @@
 	"sync"
 	"time"
 
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 const cDefaultLocalDir = "/tmp" //this is the default local dir to download to
diff --git a/internal/pkg/onuadaptercore/mib_download.go b/internal/pkg/onuadaptercore/mib_download.go
index be63ead..5d77931 100644
--- a/internal/pkg/onuadaptercore/mib_download.go
+++ b/internal/pkg/onuadaptercore/mib_download.go
@@ -26,10 +26,10 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"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/openflow_13"
-	//"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/openflow_13"
+	//"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 func (onuDeviceEntry *OnuDeviceEntry) enterDLStartingState(ctx context.Context, e *fsm.Event) {
diff --git a/internal/pkg/onuadaptercore/mib_sync.go b/internal/pkg/onuadaptercore/mib_sync.go
index 3716352..1660791 100644
--- a/internal/pkg/onuadaptercore/mib_sync.go
+++ b/internal/pkg/onuadaptercore/mib_sync.go
@@ -28,17 +28,12 @@
 
 	"github.com/looplab/fsm"
 
-	//"sync"
 	"time"
 
-	//"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"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/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 type sLastTxMeParameter struct {
diff --git a/internal/pkg/onuadaptercore/omci_ani_config.go b/internal/pkg/onuadaptercore/omci_ani_config.go
index f617f41..33e323d 100644
--- a/internal/pkg/onuadaptercore/omci_ani_config.go
+++ b/internal/pkg/onuadaptercore/omci_ani_config.go
@@ -30,10 +30,10 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"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/openflow_13"
-	//"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/openflow_13"
+	//"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/omci_cc.go b/internal/pkg/onuadaptercore/omci_cc.go
index 4006283..be930b8 100644
--- a/internal/pkg/onuadaptercore/omci_cc.go
+++ b/internal/pkg/onuadaptercore/omci_cc.go
@@ -33,14 +33,15 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
 
-	"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"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+
+	"github.com/opencord/voltha-protos/v5/go/common"
+	//"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"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/openflow_13"
+	//"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 // ### OMCI related definitions - retrieved from Python adapter code/trace ####
@@ -99,8 +100,7 @@
 	pOnuDeviceEntry    *OnuDeviceEntry
 	deviceID           string
 	pBaseDeviceHandler *deviceHandler
-	coreProxy          adapterif.CoreProxy
-	adapterProxy       adapterif.AdapterProxy
+	coreClient         *vgrpc.Client
 	supportExtMsg      bool
 	rxOmciFrameError   tOmciReceiveError
 
@@ -139,15 +139,14 @@
 //mib_db (as well as not inluded alarm_db not really used in this code? VERIFY!!)
 func newOmciCC(ctx context.Context, onuDeviceEntry *OnuDeviceEntry,
 	deviceID string, deviceHandler *deviceHandler,
-	coreProxy adapterif.CoreProxy, adapterProxy adapterif.AdapterProxy) *omciCC {
+	coreClient *vgrpc.Client) *omciCC {
 	logger.Debugw(ctx, "init-omciCC", log.Fields{"device-id": deviceID})
 	var omciCC omciCC
 	omciCC.enabled = false
 	omciCC.pOnuDeviceEntry = onuDeviceEntry
 	omciCC.deviceID = deviceID
 	omciCC.pBaseDeviceHandler = deviceHandler
-	omciCC.coreProxy = coreProxy
-	omciCC.adapterProxy = adapterProxy
+	omciCC.coreClient = coreClient
 	omciCC.supportExtMsg = false
 	omciCC.rxOmciFrameError = cOmciMessageReceiveNoError
 	omciCC.txFrames = 0
@@ -590,18 +589,18 @@
 				"TxOmciMessage": hex.EncodeToString(omciTxRequest.txFrame),
 				"device-id":     oo.deviceID,
 				"toDeviceType":  oo.pBaseDeviceHandler.ProxyAddressType,
-				"proxyDeviceID": oo.pBaseDeviceHandler.ProxyAddressID})
+				"proxyDeviceID": oo.pBaseDeviceHandler.ProxyAddressID,
+				"proxyAddress":  oo.pBaseDeviceHandler.device.ProxyAddress})
 		}
-		omciMsg := &ic.InterAdapterOmciMessage{
-			Message:       omciTxRequest.txFrame,
-			ProxyAddress:  oo.pBaseDeviceHandler.device.ProxyAddress,
-			ConnectStatus: common.ConnectStatus_REACHABLE, // If we are sending OMCI messages means we are connected, else we should not be here
+		omciMsg := &ic.OmciMessage{
+			ParentDeviceId: oo.pBaseDeviceHandler.ProxyAddressID,
+			ChildDeviceId:  oo.deviceID,
+			Message:        omciTxRequest.txFrame,
+			ProxyAddress:   oo.pBaseDeviceHandler.device.ProxyAddress,
+			ConnectStatus:  common.ConnectStatus_REACHABLE, // If we are sending OMCI messages means we are connected, else we should not be here
 		}
-		if sendErr := oo.adapterProxy.SendInterAdapterMessage(log.WithSpanFromContext(context.Background(), ctx), omciMsg,
-			ic.InterAdapterMessageType_OMCI_REQUEST,
-			//fromTopic,toType,toDevId, ProxyDevId
-			oo.pOnuDeviceEntry.baseDeviceHandler.pOpenOnuAc.config.Topic, oo.pBaseDeviceHandler.ProxyAddressType,
-			oo.deviceID, oo.pBaseDeviceHandler.ProxyAddressID, ""); sendErr != nil {
+		sendErr := oo.pBaseDeviceHandler.sendOMCIRequest(ctx, oo.pBaseDeviceHandler.device.ProxyAddress.AdapterEndpoint, omciMsg)
+		if sendErr != nil {
 			logger.Errorw(ctx, "send omci request error", log.Fields{"ChildId": oo.deviceID, "error": sendErr})
 			return sendErr
 		}
diff --git a/internal/pkg/onuadaptercore/omci_onu_upgrade.go b/internal/pkg/onuadaptercore/omci_onu_upgrade.go
index 1c86e97..8e8a82e 100644
--- a/internal/pkg/onuadaptercore/omci_onu_upgrade.go
+++ b/internal/pkg/onuadaptercore/omci_onu_upgrade.go
@@ -29,8 +29,8 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"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"
 )
 
 const cMaxUint32 = ^uint32(0)
diff --git a/internal/pkg/onuadaptercore/omci_self_test_handler.go b/internal/pkg/onuadaptercore/omci_self_test_handler.go
index ce82db8..13b6ec3 100644
--- a/internal/pkg/onuadaptercore/omci_self_test_handler.go
+++ b/internal/pkg/onuadaptercore/omci_self_test_handler.go
@@ -20,13 +20,14 @@
 import (
 	"context"
 	"fmt"
+	"sync"
+	"time"
+
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	"github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	"sync"
-	"time"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/omci_test_request.go b/internal/pkg/onuadaptercore/omci_test_request.go
index f0b76dc..235f21d 100644
--- a/internal/pkg/onuadaptercore/omci_test_request.go
+++ b/internal/pkg/onuadaptercore/omci_test_request.go
@@ -21,18 +21,11 @@
 	"context"
 	"fmt"
 
-	//"sync"
-	//"time"
-
 	gp "github.com/google/gopacket"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
 
-	//"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/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 //omciTestRequest structure holds the information for the OMCI test
diff --git a/internal/pkg/onuadaptercore/omci_vlan_config.go b/internal/pkg/onuadaptercore/omci_vlan_config.go
index 1751cdb..c828804 100644
--- a/internal/pkg/onuadaptercore/omci_vlan_config.go
+++ b/internal/pkg/onuadaptercore/omci_vlan_config.go
@@ -27,15 +27,15 @@
 	"sync"
 	"time"
 
-	meters "github.com/opencord/voltha-lib-go/v5/pkg/meters"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	meters "github.com/opencord/voltha-lib-go/v7/pkg/meters"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 
 	gp "github.com/google/gopacket"
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	of "github.com/opencord/voltha-protos/v4/go/openflow_13"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	of "github.com/opencord/voltha-protos/v5/go/openflow_13"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/onu_device_db.go b/internal/pkg/onuadaptercore/onu_device_db.go
index 83f419b..17f352b 100644
--- a/internal/pkg/onuadaptercore/onu_device_db.go
+++ b/internal/pkg/onuadaptercore/onu_device_db.go
@@ -25,7 +25,7 @@
 	"sync"
 
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 type meDbMap map[me.ClassID]map[uint16]me.AttributeValueMap
diff --git a/internal/pkg/onuadaptercore/onu_device_entry.go b/internal/pkg/onuadaptercore/onu_device_entry.go
index 91510c9..bfb1ab5 100644
--- a/internal/pkg/onuadaptercore/onu_device_entry.go
+++ b/internal/pkg/onuadaptercore/onu_device_entry.go
@@ -25,22 +25,14 @@
 	"sync"
 	"time"
 
+	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
 
-	//"sync"
-	//"time"
-
-	"github.com/looplab/fsm"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	"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/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/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 const (
@@ -269,8 +261,7 @@
 	deviceID                   string
 	baseDeviceHandler          *deviceHandler
 	pOpenOnuAc                 *OpenONUAC
-	coreProxy                  adapterif.CoreProxy
-	adapterProxy               adapterif.AdapterProxy
+	coreClient                 *vgrpc.Client
 	PDevOmciCC                 *omciCC
 	pOnuDB                     *onuDeviceDB
 	mibTemplateKVStore         *db.Backend
@@ -319,8 +310,7 @@
 	onuDeviceEntry.deviceID = dh.deviceID
 	onuDeviceEntry.baseDeviceHandler = dh
 	onuDeviceEntry.pOpenOnuAc = dh.pOpenOnuAc
-	onuDeviceEntry.coreProxy = dh.coreProxy
-	onuDeviceEntry.adapterProxy = dh.AdapterProxy
+	onuDeviceEntry.coreClient = dh.coreClient
 	onuDeviceEntry.devState = DeviceStatusInit
 	onuDeviceEntry.sOnuPersistentData.PersUniConfig = make([]uniPersConfig, 0)
 	onuDeviceEntry.sOnuPersistentData.PersTcontMap = make(map[uint16]uint16)
@@ -520,8 +510,7 @@
 func (oo *OnuDeviceEntry) start(ctx context.Context) error {
 	logger.Debugw(ctx, "OnuDeviceEntry-starting", log.Fields{"for device-id": oo.deviceID})
 	if oo.PDevOmciCC == nil {
-		oo.PDevOmciCC = newOmciCC(ctx, oo, oo.deviceID, oo.baseDeviceHandler,
-			oo.coreProxy, oo.adapterProxy)
+		oo.PDevOmciCC = newOmciCC(ctx, oo, oo.deviceID, oo.baseDeviceHandler, oo.coreClient)
 		if oo.PDevOmciCC == nil {
 			logger.Errorw(ctx, "Could not create devOmciCc - abort", log.Fields{"for device-id": oo.deviceID})
 			return fmt.Errorf("could not create devOmciCc %s", oo.deviceID)
diff --git a/internal/pkg/onuadaptercore/onu_image_status.go b/internal/pkg/onuadaptercore/onu_image_status.go
index f36a44e..d226134 100755
--- a/internal/pkg/onuadaptercore/onu_image_status.go
+++ b/internal/pkg/onuadaptercore/onu_image_status.go
@@ -26,8 +26,8 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"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"
 )
 
 //OnuImageStatus implements methods to get status info of onu images
diff --git a/internal/pkg/onuadaptercore/onu_metrics_manager.go b/internal/pkg/onuadaptercore/onu_metrics_manager.go
index c33fb99..c2a2ffb 100644
--- a/internal/pkg/onuadaptercore/onu_metrics_manager.go
+++ b/internal/pkg/onuadaptercore/onu_metrics_manager.go
@@ -28,11 +28,11 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"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-protos/v4/go/extension"
-	"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/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/onu_uni_port.go b/internal/pkg/onuadaptercore/onu_uni_port.go
index 9cfa7de..f4b255f 100644
--- a/internal/pkg/onuadaptercore/onu_uni_port.go
+++ b/internal/pkg/onuadaptercore/onu_uni_port.go
@@ -27,11 +27,11 @@
 	//"sync"
 	//"time"
 
-	//"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	vc "github.com/opencord/voltha-protos/v4/go/common"
-	of "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/kafka"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	vc "github.com/opencord/voltha-protos/v5/go/common"
+	of "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 type uniPortType uint8
@@ -114,6 +114,7 @@
 		"name": name, "hwAddr": ofHwAddr, "OperState": ofUniPortState})
 
 	pUniPort := &voltha.Port{
+		DeviceId:   apDeviceHandler.deviceID,
 		PortNo:     oo.portNo,
 		Label:      oo.name,
 		Type:       voltha.Port_ETHERNET_UNI,
@@ -136,8 +137,7 @@
 	retryCnt := 0
 	var err error
 	for retryCnt = 0; retryCnt < maxRetry; retryCnt++ {
-		if err = apDeviceHandler.coreProxy.PortCreated(log.WithSpanFromContext(context.TODO(), ctx),
-			apDeviceHandler.deviceID, pUniPort); err != nil {
+		if err = apDeviceHandler.createPortInCore(ctx, pUniPort); err != nil {
 			logger.Errorf(ctx, "Device FSM: PortCreated-failed-%s, retrying after a delay", err)
 			// retry after a sleep
 			time.Sleep(2 * time.Second)
diff --git a/internal/pkg/onuadaptercore/onu_uni_tp.go b/internal/pkg/onuadaptercore/onu_uni_tp.go
index 6e365ab..1e3f9af 100644
--- a/internal/pkg/onuadaptercore/onu_uni_tp.go
+++ b/internal/pkg/onuadaptercore/onu_uni_tp.go
@@ -24,9 +24,8 @@
 	"strings"
 	"sync"
 
-	"github.com/opencord/voltha-protos/v4/go/tech_profile"
-
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/tech_profile"
 )
 
 //definitions for TechProfileProcessing - copied from OltAdapter:openolt_flowmgr.go
diff --git a/internal/pkg/onuadaptercore/openonu.go b/internal/pkg/onuadaptercore/openonu.go
index cfa2661..eecd002 100644
--- a/internal/pkg/onuadaptercore/openonu.go
+++ b/internal/pkg/onuadaptercore/openonu.go
@@ -24,19 +24,20 @@
 	"sync"
 	"time"
 
-	conf "github.com/opencord/voltha-lib-go/v5/pkg/config"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
 
-	"github.com/golang/protobuf/ptypes"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	"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"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	oop "github.com/opencord/voltha-protos/v4/go/openolt"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	conf "github.com/opencord/voltha-lib-go/v7/pkg/config"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"google.golang.org/grpc"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"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/extension"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 
 	"github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
 )
@@ -46,10 +47,10 @@
 	deviceHandlers              map[string]*deviceHandler
 	deviceHandlersCreateChan    map[string]chan bool //channels for deviceHandler create events
 	lockDeviceHandlersMap       sync.RWMutex
-	coreProxy                   adapterif.CoreProxy
-	adapterProxy                adapterif.AdapterProxy
+	coreClient                  *vgrpc.Client
+	parentAdapterClients        map[string]*vgrpc.Client
+	lockParentAdapterClients    sync.RWMutex
 	eventProxy                  eventif.EventProxy
-	kafkaICProxy                kafka.InterContainerProxy
 	kvClient                    kvstore.Client
 	cm                          *conf.ConfigManager
 	config                      *config.AdapterFlags
@@ -74,23 +75,22 @@
 	omciTimeout                int // in seconds
 	alarmAuditInterval         time.Duration
 	dlToOnuTimeout4M           time.Duration
+	rpcTimeout                 time.Duration
 }
 
 //NewOpenONUAC returns a new instance of OpenONU_AC
-func NewOpenONUAC(ctx context.Context, kafkaICProxy kafka.InterContainerProxy,
-	coreProxy adapterif.CoreProxy, adapterProxy adapterif.AdapterProxy,
-	eventProxy eventif.EventProxy, kvClient kvstore.Client, cfg *config.AdapterFlags, cm *conf.ConfigManager) *OpenONUAC {
+func NewOpenONUAC(ctx context.Context, coreClient *vgrpc.Client, eventProxy eventif.EventProxy,
+	kvClient kvstore.Client, cfg *config.AdapterFlags, cm *conf.ConfigManager) *OpenONUAC {
 	var openOnuAc OpenONUAC
 	openOnuAc.exitChannel = make(chan int, 1)
 	openOnuAc.deviceHandlers = make(map[string]*deviceHandler)
 	openOnuAc.deviceHandlersCreateChan = make(map[string]chan bool)
+	openOnuAc.parentAdapterClients = make(map[string]*vgrpc.Client)
 	openOnuAc.lockDeviceHandlersMap = sync.RWMutex{}
-	openOnuAc.kafkaICProxy = kafkaICProxy
 	openOnuAc.config = cfg
 	openOnuAc.cm = cm
+	openOnuAc.coreClient = coreClient
 	openOnuAc.numOnus = cfg.OnuNumber
-	openOnuAc.coreProxy = coreProxy
-	openOnuAc.adapterProxy = adapterProxy
 	openOnuAc.eventProxy = eventProxy
 	openOnuAc.kvClient = kvClient
 	openOnuAc.KVStoreAddress = cfg.KVStoreAddress
@@ -110,6 +110,7 @@
 	openOnuAc.omciTimeout = int(cfg.OmciTimeout.Seconds())
 	openOnuAc.alarmAuditInterval = cfg.AlarmAuditInterval
 	openOnuAc.dlToOnuTimeout4M = cfg.DownloadToOnuTimeout4MB
+	openOnuAc.rpcTimeout = cfg.RPCTimeout
 
 	openOnuAc.pSupportedFsms = &OmciDeviceFsms{
 		"mib-synchronizer": {
@@ -202,200 +203,106 @@
 	return agent
 }
 
-func (oo *OpenONUAC) processInterAdapterONUIndReqMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	msgBody := msg.GetBody()
-	onuIndication := &oop.OnuIndication{}
-	if err := ptypes.UnmarshalAny(msgBody, onuIndication); err != nil {
-		logger.Warnw(ctx, "onu-ind-request-cannot-unmarshal-msg-body", log.Fields{"error": err})
-		return err
-	}
-	//ToDeviceId should address a DeviceHandler instance
-	targetDevice := msg.Header.ToDeviceId
-
-	onuOperstate := onuIndication.GetOperState()
-	waitForDhInstPresent := false
-	if onuOperstate == "up" {
-		//Race condition (relevant in BBSIM-environment only): Due to unsynchronized processing of olt-adapter and rw_core,
-		//ONU_IND_REQUEST msg by olt-adapter could arrive a little bit earlier than rw_core was able to announce the corresponding
-		//ONU by RPC of Adopt_device(). Therefore it could be necessary to wait with processing of ONU_IND_REQUEST until call of
-		//Adopt_device() arrived and DeviceHandler instance was created
-		waitForDhInstPresent = true
-	}
-	if handler := oo.getDeviceHandler(ctx, targetDevice, waitForDhInstPresent); handler != nil {
-		logger.Infow(ctx, "onu-ind-request", log.Fields{"device-id": targetDevice,
-			"OnuId":      onuIndication.GetOnuId(),
-			"AdminState": onuIndication.GetAdminState(), "OperState": onuOperstate,
-			"SNR": onuIndication.GetSerialNumber()})
-
-		if onuOperstate == "up" {
-			return handler.createInterface(ctx, onuIndication)
-		} else if (onuOperstate == "down") || (onuOperstate == "unreachable") {
-			return handler.updateInterface(ctx, onuIndication)
-		} else {
-			logger.Errorw(ctx, "unknown-onu-ind-request operState", log.Fields{"OnuId": onuIndication.GetOnuId()})
-			return fmt.Errorf("invalidOperState: %s, %s", onuOperstate, targetDevice)
-		}
-	}
-	logger.Warnw(ctx, "no handler found for received onu-ind-request", log.Fields{
-		"msgToDeviceId": targetDevice})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", targetDevice))
+// GetHealthStatus is used as a service readiness validation as a grpc connection
+func (oo *OpenONUAC) GetHealthStatus(ctx context.Context, empty *empty.Empty) (*voltha.HealthStatus, error) {
+	return &voltha.HealthStatus{State: voltha.HealthStatus_HEALTHY}, nil
 }
 
-// Adapter interface required methods ############## begin #########
-// #################################################################
-
-// for original content compare: (needs according deviceHandler methods)
-// /voltha-openolt-adapter/adaptercore/openolt.go
-
-// Adopt_device creates a new device handler if not present already and then adopts the device
-func (oo *OpenONUAC) Adopt_device(ctx context.Context, device *voltha.Device) error {
+// AdoptDevice creates a new device handler if not present already and then adopts the device
+func (oo *OpenONUAC) AdoptDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	if device == nil {
 		logger.Warn(ctx, "voltha-device-is-nil")
-		return errors.New("nil-device")
+		return nil, errors.New("nil-device")
 	}
 	logger.Infow(ctx, "adopt-device", log.Fields{"device-id": device.Id})
 	var handler *deviceHandler
 	if handler = oo.getDeviceHandler(ctx, device.Id, false); handler == nil {
-		handler := newDeviceHandler(ctx, oo.coreProxy, oo.adapterProxy, oo.eventProxy, device, oo)
+		handler := newDeviceHandler(ctx, oo.coreClient, oo.eventProxy, device, oo)
 		oo.addDeviceHandlerToMap(ctx, handler)
-		go handler.adoptOrReconcileDevice(ctx, device)
-		// Launch the creation of the device topic
-		// go oo.createDeviceTopic(device)
+
+		// Setup the grpc communication with the parent adapter
+		if err := oo.setupParentInterAdapterClient(ctx, device.ProxyAddress.AdapterEndpoint); err != nil {
+			// TODO: Cleanup on failure needed
+			return nil, err
+		}
+
+		go handler.adoptOrReconcileDevice(log.WithSpanFromContext(context.Background(), ctx), device)
 	}
-	return nil
+	return &empty.Empty{}, nil
 }
 
-//Get_ofp_device_info returns OFP information for the given device
-func (oo *OpenONUAC) Get_ofp_device_info(ctx context.Context, device *voltha.Device) (*ic.SwitchCapability, error) {
-	logger.Errorw(ctx, "device-handler-not-set", log.Fields{"device-id": device.Id})
-	return nil, fmt.Errorf("device-handler-not-set %s", device.Id)
-}
-
-//Get_ofp_port_info returns OFP port information for the given device
-//200630: method removed as per [VOL-3202]: OF port info is now to be delivered within UniPort create
-// cmp changes in onu_uni_port.go::CreateVolthaPort()
-
-//Process_inter_adapter_message sends messages to a target device (between adapters)
-func (oo *OpenONUAC) Process_inter_adapter_message(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	logger.Debugw(ctx, "Process_inter_adapter_message", log.Fields{"msgId": msg.Header.Id,
-		"msgProxyDeviceId": msg.Header.ProxyDeviceId, "msgToDeviceId": msg.Header.ToDeviceId, "Type": msg.Header.Type})
-
-	if msg.Header.Type == ic.InterAdapterMessageType_ONU_IND_REQUEST {
-		// we have to handle ONU_IND_REQUEST already here - see comments in processInterAdapterONUIndReqMessage()
-		return oo.processInterAdapterONUIndReqMessage(ctx, msg)
-	}
-	//ToDeviceId should address a DeviceHandler instance
-	targetDevice := msg.Header.ToDeviceId
-	if handler := oo.getDeviceHandler(ctx, targetDevice, false); handler != nil {
-		/* 200724: modification towards synchronous implementation - possible errors within processing shall be
-		 * 	 in the accordingly delayed response, some timing effect might result in Techprofile processing for multiple UNI's
-		 */
-		return handler.processInterAdapterMessage(ctx, msg)
-		/* so far the processing has been in background with according commented error treatment restrictions:
-		go handler.ProcessInterAdapterMessage(msg)
-		// error treatment might be more sophisticated
-		// by now let's just accept the message on 'communication layer'
-		// message content problems have to be evaluated then in the handler
-		//   and are by now not reported to the calling party (to force what reaction there?)
-		return nil
-		*/
-	}
-	logger.Warnw(ctx, "no handler found for received Inter-Proxy-message", log.Fields{
-		"msgToDeviceId": targetDevice})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", targetDevice))
-}
-
-//Process_tech_profile_instance_request not implemented
-func (oo *OpenONUAC) Process_tech_profile_instance_request(ctx context.Context, msg *ic.InterAdapterTechProfileInstanceRequestMessage) *ic.InterAdapterTechProfileDownloadMessage {
-	logger.Error(ctx, "unImplemented")
-	return nil
-}
-
-//Adapter_descriptor not implemented
-func (oo *OpenONUAC) Adapter_descriptor(ctx context.Context) error {
-	return errors.New("unImplemented")
-}
-
-//Device_types unimplemented
-func (oo *OpenONUAC) Device_types(ctx context.Context) (*voltha.DeviceTypes, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Health  returns unimplemented
-func (oo *OpenONUAC) Health(ctx context.Context) (*voltha.HealthStatus, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Reconcile_device is called once when the adapter needs to re-create device - usually on core restart
-func (oo *OpenONUAC) Reconcile_device(ctx context.Context, device *voltha.Device) error {
+//ReconcileDevice is called once when the adapter needs to re-create device - usually on core restart
+func (oo *OpenONUAC) ReconcileDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	if device == nil {
 		logger.Warn(ctx, "reconcile-device-voltha-device-is-nil")
-		return errors.New("nil-device")
+		return nil, errors.New("nil-device")
 	}
 	logger.Infow(ctx, "reconcile-device", log.Fields{"device-id": device.Id})
 	var handler *deviceHandler
 	if handler = oo.getDeviceHandler(ctx, device.Id, false); handler == nil {
-		handler := newDeviceHandler(ctx, oo.coreProxy, oo.adapterProxy, oo.eventProxy, device, oo)
+		handler := newDeviceHandler(ctx, oo.coreClient, oo.eventProxy, device, oo)
 		oo.addDeviceHandlerToMap(ctx, handler)
 		handler.device = device
-		if err := handler.coreProxy.DeviceStateUpdate(ctx, device.Id, device.ConnectStatus, voltha.OperStatus_RECONCILING); err != nil {
-			return fmt.Errorf("not able to update device state to reconciling. Err : %s", err.Error())
+		if err := handler.updateDeviceStateInCore(log.WithSpanFromContext(context.Background(), ctx), &ic.DeviceStateFilter{
+			DeviceId:   device.Id,
+			OperStatus: voltha.OperStatus_RECONCILING,
+			ConnStatus: device.ConnectStatus,
+		}); err != nil {
+			return nil, fmt.Errorf("not able to update device state to reconciling. Err : %s", err.Error())
 		}
-		handler.startReconciling(ctx, false)
-		go handler.adoptOrReconcileDevice(ctx, handler.device)
+		// Setup the grpc communication with the parent adapter
+		if err := oo.setupParentInterAdapterClient(ctx, device.ProxyAddress.AdapterEndpoint); err != nil {
+			// TODO: Cleanup on failure needed
+			return nil, err
+		}
+
+		handler.startReconciling(log.WithSpanFromContext(context.Background(), ctx), false)
+		go handler.adoptOrReconcileDevice(log.WithSpanFromContext(context.Background(), ctx), handler.device)
 		// reconcilement will be continued after onu-device entry is added
 	} else {
-		return fmt.Errorf(fmt.Sprintf("device-already-reconciled-or-active-%s", device.Id))
+		return nil, fmt.Errorf(fmt.Sprintf("device-already-reconciled-or-active-%s", device.Id))
 	}
-	return nil
+	return &empty.Empty{}, nil
 }
 
-//Abandon_device unimplemented
-func (oo *OpenONUAC) Abandon_device(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
-
-//Disable_device disables the given device
-func (oo *OpenONUAC) Disable_device(ctx context.Context, device *voltha.Device) error {
+//DisableDevice disables the given device
+func (oo *OpenONUAC) DisableDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	logger.Infow(ctx, "disable-device", log.Fields{"device-id": device.Id})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		go handler.disableDevice(ctx, device)
-		return nil
+		go handler.disableDevice(log.WithSpanFromContext(context.Background(), ctx), device)
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-disable", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
 }
 
-//Reenable_device enables the onu device after disable
-func (oo *OpenONUAC) Reenable_device(ctx context.Context, device *voltha.Device) error {
+//ReEnableDevice enables the onu device after disable
+func (oo *OpenONUAC) ReEnableDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	logger.Infow(ctx, "reenable-device", log.Fields{"device-id": device.Id})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		go handler.reEnableDevice(ctx, device)
-		return nil
+		go handler.reEnableDevice(log.WithSpanFromContext(context.Background(), ctx), device)
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-reenable", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
 }
 
-//Reboot_device reboots the given device
-func (oo *OpenONUAC) Reboot_device(ctx context.Context, device *voltha.Device) error {
+//RebootDevice reboots the given device
+func (oo *OpenONUAC) RebootDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	logger.Infow(ctx, "reboot-device", log.Fields{"device-id": device.Id})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		go handler.rebootDevice(ctx, true, device) //reboot request with device checking
-		return nil
+		go handler.rebootDevice(log.WithSpanFromContext(context.Background(), ctx), true, device) //reboot request with device checking
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-reboot", log.Fields{"device-id": device.Id})
-	return fmt.Errorf("handler-not-found-for-device: %s", device.Id)
+	return nil, fmt.Errorf("handler-not-found-for-device: %s", device.Id)
 }
 
-//Self_test_device unimplemented
-func (oo *OpenONUAC) Self_test_device(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
+// DeleteDevice deletes the given device
+func (oo *OpenONUAC) DeleteDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
+	nctx := log.WithSpanFromContext(context.Background(), ctx)
 
-// Delete_device deletes the given device
-func (oo *OpenONUAC) Delete_device(ctx context.Context, device *voltha.Device) error {
-	logger.Infow(ctx, "delete-device", log.Fields{"device-id": device.Id, "SerialNumber": device.SerialNumber})
+	logger.Infow(ctx, "delete-device", log.Fields{"device-id": device.Id, "SerialNumber": device.SerialNumber, "ctx": ctx, "nctx": nctx})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
 		var errorsList []error
 
@@ -433,162 +340,104 @@
 		oo.deleteDeviceHandlerToMap(handler)
 		if len(errorsList) > 0 {
 			logger.Errorw(ctx, "one-or-more-error-during-device-delete", log.Fields{"device-id": device.Id})
-			return fmt.Errorf("one-or-more-error-during-device-delete, errors:%v", errorsList)
+			return nil, fmt.Errorf("one-or-more-error-during-device-delete, errors:%v", errorsList)
 		}
-		return nil
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-deletion", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
 }
 
-//Get_device_details unimplemented
-func (oo *OpenONUAC) Get_device_details(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
+//UpdateFlowsIncrementally updates (add/remove) the flows on a given device
+func (oo *OpenONUAC) UpdateFlowsIncrementally(ctx context.Context, incrFlows *ic.IncrementalFlows) (*empty.Empty, error) {
+	logger.Infow(ctx, "update-flows-incrementally", log.Fields{"device-id": incrFlows.Device.Id})
 
-//Update_flows_bulk returns
-func (oo *OpenONUAC) Update_flows_bulk(ctx context.Context, device *voltha.Device, flows *voltha.Flows, groups *voltha.FlowGroups, flowMetadata *voltha.FlowMetadata) error {
-	return errors.New("unImplemented")
-}
-
-//Update_flows_incrementally updates (add/remove) the flows on a given device
-func (oo *OpenONUAC) Update_flows_incrementally(ctx context.Context, device *voltha.Device,
-	flows *openflow_13.FlowChanges, groups *openflow_13.FlowGroupChanges, flowMetadata *voltha.FlowMetadata) error {
-
-	logger.Infow(ctx, "update-flows-incrementally", log.Fields{"device-id": device.Id})
 	//flow config is relayed to handler even if the device might be in some 'inactive' state
 	// let the handler or related FSM's decide, what to do with the modified flow state info
 	// at least the flow-remove must be done in respect to internal data, while OMCI activity might not be needed here
 
 	// For now, there is no support for group changes (as in the actual Py-adapter code)
 	//   but processing is continued for flowUpdate possibly also set in the request
-	if groups.ToAdd != nil && groups.ToAdd.Items != nil {
-		logger.Warnw(ctx, "Update-flow-incr: group add not supported (ignored)", log.Fields{"device-id": device.Id})
+	if incrFlows.Groups.ToAdd != nil && incrFlows.Groups.ToAdd.Items != nil {
+		logger.Warnw(ctx, "Update-flow-incr: group add not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
 	}
-	if groups.ToRemove != nil && groups.ToRemove.Items != nil {
-		logger.Warnw(ctx, "Update-flow-incr: group remove not supported (ignored)", log.Fields{"device-id": device.Id})
+	if incrFlows.Groups.ToRemove != nil && incrFlows.Groups.ToRemove.Items != nil {
+		logger.Warnw(ctx, "Update-flow-incr: group remove not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
 	}
-	if groups.ToUpdate != nil && groups.ToUpdate.Items != nil {
-		logger.Warnw(ctx, "Update-flow-incr: group update not supported (ignored)", log.Fields{"device-id": device.Id})
+	if incrFlows.Groups.ToUpdate != nil && incrFlows.Groups.ToUpdate.Items != nil {
+		logger.Warnw(ctx, "Update-flow-incr: group update not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
 	}
 
-	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		err := handler.FlowUpdateIncremental(ctx, flows, groups, flowMetadata)
-		return err
+	if handler := oo.getDeviceHandler(ctx, incrFlows.Device.Id, false); handler != nil {
+		if err := handler.FlowUpdateIncremental(log.WithSpanFromContext(context.Background(), ctx), incrFlows.Flows, incrFlows.Groups, incrFlows.FlowMetadata); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
 	}
-	logger.Warnw(ctx, "no handler found for incremental flow update", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	logger.Warnw(ctx, "no handler found for incremental flow update", log.Fields{"device-id": incrFlows.Device.Id})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", incrFlows.Device.Id))
 }
 
-//Update_pm_config returns PmConfigs nil or error
-func (oo *OpenONUAC) Update_pm_config(ctx context.Context, device *voltha.Device, pmConfigs *voltha.PmConfigs) error {
-	logger.Infow(ctx, "update-pm-config", log.Fields{"device-id": device.Id})
-	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		return handler.updatePmConfig(ctx, pmConfigs)
+//UpdatePmConfig returns PmConfigs nil or error
+func (oo *OpenONUAC) UpdatePmConfig(ctx context.Context, configs *ic.PmConfigsInfo) (*empty.Empty, error) {
+	logger.Infow(ctx, "update-pm-config", log.Fields{"device-id": configs.DeviceId})
+	if handler := oo.getDeviceHandler(ctx, configs.DeviceId, false); handler != nil {
+		if err := handler.updatePmConfig(log.WithSpanFromContext(context.Background(), ctx), configs.PmConfigs); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
 	}
-	logger.Warnw(ctx, "no handler found for update-pm-config", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	logger.Warnw(ctx, "no handler found for update-pm-config", log.Fields{"device-id": configs.DeviceId})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", configs.DeviceId))
 }
 
-//Receive_packet_out sends packet out to the device
-func (oo *OpenONUAC) Receive_packet_out(ctx context.Context, deviceID string, egressPortNo int, packet *openflow_13.OfpPacketOut) error {
-	return errors.New("unImplemented")
-}
-
-//Suppress_event unimplemented
-func (oo *OpenONUAC) Suppress_event(ctx context.Context, filter *voltha.EventFilter) error {
-	return errors.New("unImplemented")
-}
-
-//Unsuppress_event  unimplemented
-func (oo *OpenONUAC) Unsuppress_event(ctx context.Context, filter *voltha.EventFilter) error {
-	return errors.New("unImplemented")
-}
-
-//Download_image requests downloading some image according to indications as given in request
+//DownloadImage requests downloading some image according to indications as given in request
 //The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Download_image(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	if request != nil && (*request).Name != "" {
-		if !oo.pDownloadManager.imageExists(ctx, request) {
-			logger.Debugw(ctx, "start image download", log.Fields{"image-description": request})
+func (oo *OpenONUAC) DownloadImage(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	ctx = log.WithSpanFromContext(context.Background(), ctx)
+	if imageInfo != nil && imageInfo.Image != nil && imageInfo.Image.Name != "" {
+		if !oo.pDownloadManager.imageExists(ctx, imageInfo.Image) {
+			logger.Debugw(ctx, "start image download", log.Fields{"image-description": imageInfo.Image})
 			// Download_image is not supposed to be blocking, anyway let's call the DownloadManager still synchronously to detect 'fast' problems
 			// the download itself is later done in background
-			err := oo.pDownloadManager.startDownload(ctx, request)
-			return request, err
+			if err := oo.pDownloadManager.startDownload(ctx, imageInfo.Image); err != nil {
+				return nil, err
+			}
+			return imageInfo.Image, nil
 		}
 		// image already exists
-		logger.Debugw(ctx, "image already downloaded", log.Fields{"image-description": request})
-		return request, nil
+		logger.Debugw(ctx, "image already downloaded", log.Fields{"image-description": imageInfo.Image})
+		return imageInfo.Image, nil
 	}
-	return request, errors.New("invalid image definition")
+
+	return nil, errors.New("invalid image definition")
 }
 
-//Get_image_download_status unimplemented
-//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Get_image_download_status(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Cancel_image_download unimplemented
-//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Cancel_image_download(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Activate_image_update requests downloading some Onu Software image to the ONU via OMCI
+//ActivateImageUpdate requests downloading some Onu Software image to the INU via OMCI
 //  according to indications as given in request and on success activate the image on the ONU
 //The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Activate_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	if request != nil && (*request).Name != "" {
-		if oo.pDownloadManager.imageLocallyDownloaded(ctx, request) {
-			if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
+func (oo *OpenONUAC) ActivateImageUpdate(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	if imageInfo != nil && imageInfo.Image != nil && imageInfo.Image.Name != "" {
+		if oo.pDownloadManager.imageLocallyDownloaded(ctx, imageInfo.Image) {
+			if handler := oo.getDeviceHandler(ctx, imageInfo.Device.Id, false); handler != nil {
 				logger.Debugw(ctx, "image download on omci requested", log.Fields{
-					"image-description": request, "device-id": device.Id})
-				err := handler.doOnuSwUpgrade(ctx, request, oo.pDownloadManager)
-				return request, err
+					"image-description": imageInfo.Image, "device-id": imageInfo.Device.Id})
+				if err := handler.doOnuSwUpgrade(ctx, imageInfo.Image, oo.pDownloadManager); err != nil {
+					return nil, err
+				}
+				return imageInfo.Image, nil
 			}
-			logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": device.Id})
-			return request, fmt.Errorf(fmt.Sprintf("handler-not-found - device-id: %s", device.Id))
+			logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": imageInfo.Device.Id})
+			return nil, fmt.Errorf(fmt.Sprintf("handler-not-found - device-id: %s", imageInfo.Device.Id))
 		}
-		logger.Debugw(ctx, "image not yet downloaded on activate request", log.Fields{"image-description": request})
-		return request, fmt.Errorf(fmt.Sprintf("image-not-yet-downloaded - device-id: %s", device.Id))
+		logger.Debugw(ctx, "image not yet downloaded on activate request", log.Fields{"image-description": imageInfo.Image})
+		return nil, fmt.Errorf(fmt.Sprintf("image-not-yet-downloaded - device-id: %s", imageInfo.Device.Id))
 	}
-	return request, errors.New("invalid image definition")
+	return nil, errors.New("invalid image definition")
 }
 
-//Revert_image_update unimplemented
-func (oo *OpenONUAC) Revert_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	return nil, errors.New("unImplemented")
-}
-
-// Enable_port to Enable PON/NNI interface - seems not to be used/required according to python code
-func (oo *OpenONUAC) Enable_port(ctx context.Context, deviceID string, port *voltha.Port) error {
-	return errors.New("unImplemented")
-}
-
-// Disable_port to Disable pon/nni interface  - seems not to be used/required according to python code
-func (oo *OpenONUAC) Disable_port(ctx context.Context, deviceID string, port *voltha.Port) error {
-	return errors.New("unImplemented")
-}
-
-//Child_device_lost - unimplemented
-//needed for if update >= 3.1.x
-func (oo *OpenONUAC) Child_device_lost(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
-
-// Start_omci_test unimplemented
-func (oo *OpenONUAC) Start_omci_test(ctx context.Context, device *voltha.Device, request *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
-	return nil, errors.New("unImplemented")
-}
-
-// Get_ext_value - unimplemented
-func (oo *OpenONUAC) Get_ext_value(ctx context.Context, deviceID string, device *voltha.Device, valueparam voltha.ValueType_Type) (*voltha.ReturnValues, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Single_get_value_request handles the core request to retrieve uni status
-func (oo *OpenONUAC) Single_get_value_request(ctx context.Context, request extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
+//GetSingleValue handles the core request to retrieve uni status
+func (oo *OpenONUAC) GetSingleValue(ctx context.Context, request *extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
 	logger.Infow(ctx, "Single_get_value_request", log.Fields{"request": request})
 
 	if handler := oo.getDeviceHandler(ctx, request.TargetId, false); handler != nil {
@@ -599,7 +448,7 @@
 			commChan := make(chan Message)
 			respChan := make(chan extension.SingleGetValueResponse)
 			// Initiate the self test request
-			if err := handler.pSelfTestHdlr.SelfTestRequestStart(ctx, request, commChan, respChan); err != nil {
+			if err := handler.pSelfTestHdlr.SelfTestRequestStart(ctx, *request, commChan, respChan); err != nil {
 				return &extension.SingleGetValueResponse{
 					Response: &extension.GetValueResponse{
 						Status:    extension.GetValueResponse_ERROR,
@@ -627,9 +476,9 @@
 //   The reason for that was never finally investigated.
 //   To be on the safe side argument names are left here always as defined in iAdapter.go .
 
-// Download_onu_image downloads (and optionally activates and commits) the indicated ONU image to the requested ONU(s)
+// DownloadOnuImage downloads (and optionally activates and commits) the indicated ONU image to the requested ONU(s)
 //   if the image is not yet present on the adapter it has to be automatically downloaded
-func (oo *OpenONUAC) Download_onu_image(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) DownloadOnuImage(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
 	if request != nil && len((*request).DeviceId) > 0 && (*request).Image.Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*request).Image.Version
@@ -645,8 +494,18 @@
 			loDeviceImageState.ImageState = &loImageState
 			loDeviceImageState.ImageState.Version = (*request).Image.Version
 
-			onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-				loDeviceID, loDeviceID)
+			handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+			if handler == nil {
+				//cannot start ONU download for requested device
+				logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": loDeviceID})
+				loDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_FAILED
+				loDeviceImageState.ImageState.Reason = voltha.ImageState_UNKNOWN_ERROR
+				loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+				loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, &loDeviceImageState)
+				continue
+			}
+
+			onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
 			if err != nil || onuVolthaDevice == nil {
 				logger.Warnw(ctx, "Failed to fetch Onu device for image download",
 					log.Fields{"device-id": loDeviceID, "err": err})
@@ -718,6 +577,17 @@
 					loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
 				}
 			}
+
+			// start the ONU download activity for each possible device
+			logger.Debugw(ctx, "image download on omci requested", log.Fields{
+				"image-id": imageIdentifier, "device-id": loDeviceID})
+			//onu upgrade handling called in background without immediate error evaluation here
+			//  as the processing can be done for multiple ONU's and an error on one ONU should not stop processing for others
+			//  state/progress/success of the request has to be verified using the Get_onu_image_status() API
+			go handler.onuSwUpgradeAfterDownload(ctx, request, oo.pFileManager, imageIdentifier)
+			loDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_STARTED
+			loDeviceImageState.ImageState.Reason = voltha.ImageState_NO_ERROR
+			loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
 			loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, &loDeviceImageState)
 		}
 		pImageResp := &loResponse
@@ -726,9 +596,9 @@
 	return nil, errors.New("invalid image download parameters")
 }
 
-// Get_onu_image_status delivers the adapter-related information about the download/activation/commitment
+// GetOnuImageStatus delivers the adapter-related information about the download/activation/commitment
 //   status for the requested image
-func (oo *OpenONUAC) Get_onu_image_status(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) GetOnuImageStatus(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -737,12 +607,20 @@
 		var vendorID string
 		for _, pCommonID := range (*in).DeviceId {
 			loDeviceID := (*pCommonID).Id
-			pDeviceImageState := &voltha.DeviceImageState{
-				DeviceId: loDeviceID,
+			pDeviceImageState := &voltha.DeviceImageState{DeviceId: loDeviceID}
+			handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+			if handler == nil {
+				//cannot get the handler
+				logger.Warnw(ctx, "no handler found for image status request ", log.Fields{"device-id": loDeviceID})
+				pDeviceImageState.DeviceId = loDeviceID
+				pDeviceImageState.ImageState.Version = (*in).Version
+				pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_FAILED
+				pDeviceImageState.ImageState.Reason = voltha.ImageState_UNKNOWN_ERROR
+				pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+				loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
+				continue
 			}
-			vendorIDSet = false
-			onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-				loDeviceID, loDeviceID)
+			onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
 			if err != nil || onuVolthaDevice == nil {
 				logger.Warnw(ctx, "Failed to fetch Onu device to get image status",
 					log.Fields{"device-id": loDeviceID, "err": err})
@@ -780,23 +658,10 @@
 					}
 					pDeviceImageState.ImageState = pImageState
 				} else {
-					// get the handler for the device
-					if handler := oo.getDeviceHandler(ctx, loDeviceID, false); handler != nil {
-						logger.Debugw(ctx, "image status request for", log.Fields{
-							"image-id": imageIdentifier, "device-id": loDeviceID})
-						//status request is called synchronously to collect the indications for all concerned devices
-						pDeviceImageState.ImageState = handler.requestOnuSwUpgradeState(ctx, imageIdentifier, (*in).Version)
-					} else {
-						//cannot get the handler
-						logger.Warnw(ctx, "no handler found for image status request ", log.Fields{"device-id": loDeviceID})
-						pImageState := &voltha.ImageState{
-							Version:       (*in).Version,
-							DownloadState: voltha.ImageState_DOWNLOAD_UNKNOWN, //no statement about last activity possible
-							Reason:        voltha.ImageState_UNKNOWN_ERROR,    //something like "DEVICE_NOT_EXISTS" would be better (proto def)
-							ImageState:    voltha.ImageState_IMAGE_UNKNOWN,
-						}
-						pDeviceImageState.ImageState = pImageState
-					}
+					logger.Debugw(ctx, "image status request for", log.Fields{
+						"image-id": imageIdentifier, "device-id": loDeviceID})
+					//status request is called synchronously to collect the indications for all concerned devices
+					pDeviceImageState.ImageState = handler.requestOnuSwUpgradeState(ctx, imageIdentifier, (*in).Version)
 				}
 			}
 			loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
@@ -807,8 +672,8 @@
 	return nil, errors.New("invalid image status request parameters")
 }
 
-// Abort_onu_image_upgrade stops the actual download/activation/commitment process (on next possibly step)
-func (oo *OpenONUAC) Abort_onu_image_upgrade(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+// AbortOnuImageUpgrade stops the actual download/activation/commitment process (on next possibly step)
+func (oo *OpenONUAC) AbortOnuImageUpgrade(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -816,16 +681,30 @@
 		var vendorID string
 		for _, pCommonID := range (*in).DeviceId {
 			loDeviceID := (*pCommonID).Id
-			onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-				loDeviceID, loDeviceID)
+			pDeviceImageState := &voltha.DeviceImageState{}
+			loImageState := voltha.ImageState{}
+			pDeviceImageState.ImageState = &loImageState
+
+			handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+			if handler == nil {
+				//cannot start ONU download for requested device
+				logger.Warnw(ctx, "no handler found for aborting upgrade ", log.Fields{"device-id": loDeviceID})
+				pDeviceImageState.DeviceId = loDeviceID
+				pDeviceImageState.ImageState.Version = (*in).Version
+				//nolint:misspell
+				pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_CANCELLED
+				//nolint:misspell
+				pDeviceImageState.ImageState.Reason = voltha.ImageState_CANCELLED_ON_REQUEST
+				pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+				loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
+				continue
+			}
+			onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
 			if err != nil || onuVolthaDevice == nil {
 				logger.Warnw(ctx, "Failed to fetch Onu device to abort its download",
 					log.Fields{"device-id": loDeviceID, "err": err})
 				continue //try the work with next deviceId
 			}
-			pDeviceImageState := &voltha.DeviceImageState{}
-			loImageState := voltha.ImageState{}
-			pDeviceImageState.ImageState = &loImageState
 
 			if firstDevice {
 				//start/verify download of the image to the adapter based on first found device only
@@ -844,20 +723,10 @@
 			}
 
 			// cancel the ONU upgrade activity for each possible device
-			if handler := oo.getDeviceHandler(ctx, loDeviceID, false); handler != nil {
-				logger.Debugw(ctx, "image upgrade abort requested", log.Fields{
-					"image-id": imageIdentifier, "device-id": loDeviceID})
-				//upgrade cancel is called synchronously to collect the imageResponse indications for all concerned devices
-				handler.cancelOnuSwUpgrade(ctx, imageIdentifier, (*in).Version, pDeviceImageState)
-			} else {
-				//cannot start aborting ONU download for requested device
-				logger.Warnw(ctx, "no handler found for aborting upgrade ", log.Fields{"device-id": loDeviceID})
-				pDeviceImageState.DeviceId = loDeviceID
-				pDeviceImageState.ImageState.Version = (*in).Version
-				pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_CANCELLED
-				pDeviceImageState.ImageState.Reason = voltha.ImageState_CANCELLED_ON_REQUEST //something better would be possible with protos modification
-				pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
-			}
+			logger.Debugw(ctx, "image upgrade abort requested", log.Fields{
+				"image-id": imageIdentifier, "device-id": loDeviceID})
+			//upgrade cancel is called synchronously to collect the imageResponse indications for all concerned devices
+			handler.cancelOnuSwUpgrade(ctx, imageIdentifier, (*in).Version, pDeviceImageState)
 			loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
 		}
 		if !firstDevice {
@@ -871,23 +740,23 @@
 	return nil, errors.New("invalid image upgrade abort parameters")
 }
 
-// Get_onu_images retrieves the ONU SW image status information via OMCI
-func (oo *OpenONUAC) Get_onu_images(ctx context.Context, deviceID string) (*voltha.OnuImages, error) {
-	logger.Infow(ctx, "Get_onu_images", log.Fields{"device-id": deviceID})
-	if handler := oo.getDeviceHandler(ctx, deviceID, false); handler != nil {
+// GetOnuImages retrieves the ONU SW image status information via OMCI
+func (oo *OpenONUAC) GetOnuImages(ctx context.Context, id *common.ID) (*voltha.OnuImages, error) {
+	logger.Infow(ctx, "Get_onu_images", log.Fields{"device-id": id.Id})
+	if handler := oo.getDeviceHandler(ctx, id.Id, false); handler != nil {
 		images, err := handler.getOnuImages(ctx)
 		if err == nil {
 			return images, nil
 		}
-		return nil, fmt.Errorf(fmt.Sprintf("%s-%s", err, deviceID))
+		return nil, fmt.Errorf(fmt.Sprintf("%s-%s", err, id.Id))
 	}
-	logger.Warnw(ctx, "no handler found for Get_onu_images", log.Fields{"device-id": deviceID})
-	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", deviceID))
+	logger.Warnw(ctx, "no handler found for Get_onu_images", log.Fields{"device-id": id.Id})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", id.Id))
 }
 
-// Activate_onu_image initiates the activation of the image for the requested ONU(s)
+// ActivateOnuImage initiates the activation of the image for the requested ONU(s)
 //  precondition: image downloaded and not yet activated or image refers to current inactive image
-func (oo *OpenONUAC) Activate_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) ActivateOnuImage(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -933,9 +802,9 @@
 	return nil, errors.New("invalid image activation parameters")
 }
 
-// Commit_onu_image enforces the commitment of the image for the requested ONU(s)
+// CommitOnuImage enforces the commitment of the image for the requested ONU(s)
 //  precondition: image activated and not yet committed
-func (oo *OpenONUAC) Commit_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) CommitOnuImage(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -983,3 +852,281 @@
 
 // Adapter interface required methods ################ end #########
 // #################################################################
+
+/*
+ *
+ * ONU inter adapter service
+ *
+ */
+
+// OnuIndication is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) OnuIndication(ctx context.Context, onuInd *ic.OnuIndicationMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "onu-indication", log.Fields{"onu-indication": onuInd})
+
+	if onuInd == nil || onuInd.OnuIndication == nil {
+		return nil, fmt.Errorf("invalid-onu-indication-%v", onuInd)
+	}
+
+	onuIndication := onuInd.OnuIndication
+	onuOperstate := onuIndication.GetOperState()
+	waitForDhInstPresent := false
+	if onuOperstate == "up" {
+		//Race condition (relevant in BBSIM-environment only): Due to unsynchronized processing of olt-adapter and rw_core,
+		//ONU_IND_REQUEST msg by olt-adapter could arrive a little bit earlier than rw_core was able to announce the corresponding
+		//ONU by RPC of Adopt_device(). Therefore it could be necessary to wait with processing of ONU_IND_REQUEST until call of
+		//Adopt_device() arrived and DeviceHandler instance was created
+		waitForDhInstPresent = true
+	}
+	if handler := oo.getDeviceHandler(ctx, onuInd.DeviceId, waitForDhInstPresent); handler != nil {
+		logger.Infow(ctx, "onu-ind-request", log.Fields{"device-id": onuInd.DeviceId,
+			"OnuId":      onuIndication.GetOnuId(),
+			"AdminState": onuIndication.GetAdminState(), "OperState": onuOperstate,
+			"SNR": onuIndication.GetSerialNumber()})
+
+		if onuOperstate == "up" {
+			if err := handler.createInterface(ctx, onuIndication); err != nil {
+				return nil, err
+			}
+			return &empty.Empty{}, nil
+		} else if (onuOperstate == "down") || (onuOperstate == "unreachable") {
+			return nil, handler.updateInterface(ctx, onuIndication)
+		} else {
+			logger.Errorw(ctx, "unknown-onu-ind-request operState", log.Fields{"OnuId": onuIndication.GetOnuId()})
+			return nil, fmt.Errorf("invalidOperState: %s, %s", onuOperstate, onuInd.DeviceId)
+		}
+	}
+	logger.Warnw(ctx, "no handler found for received onu-ind-request", log.Fields{
+		"msgToDeviceId": onuInd.DeviceId})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", onuInd.DeviceId))
+}
+
+// OmciIndication is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) OmciIndication(ctx context.Context, msg *ic.OmciMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "omci-response", log.Fields{"parent-device-id": msg.ParentDeviceId, "child-device-id": msg.ChildDeviceId})
+
+	if handler := oo.getDeviceHandler(ctx, msg.ChildDeviceId, false); handler != nil {
+		if err := handler.handleOMCIIndication(log.WithSpanFromContext(context.Background(), ctx), msg); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", msg.ChildDeviceId))
+}
+
+// DownloadTechProfile is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DownloadTechProfile(ctx context.Context, tProfile *ic.TechProfileDownloadMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "download-tech-profile", log.Fields{"uni-id": tProfile.UniId})
+
+	if handler := oo.getDeviceHandler(ctx, tProfile.DeviceId, false); handler != nil {
+		if err := handler.handleTechProfileDownloadRequest(log.WithSpanFromContext(context.Background(), ctx), tProfile); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", tProfile.DeviceId))
+}
+
+// DeleteGemPort is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DeleteGemPort(ctx context.Context, gPort *ic.DeleteGemPortMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "delete-gem-port", log.Fields{"device-id": gPort.DeviceId, "uni-id": gPort.UniId})
+
+	if handler := oo.getDeviceHandler(ctx, gPort.DeviceId, false); handler != nil {
+		if err := handler.handleDeleteGemPortRequest(log.WithSpanFromContext(context.Background(), ctx), gPort); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", gPort.DeviceId))
+}
+
+// DeleteTCont is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DeleteTCont(ctx context.Context, tConf *ic.DeleteTcontMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "delete-tcont", log.Fields{"tconf": tConf})
+
+	if handler := oo.getDeviceHandler(ctx, tConf.DeviceId, false); handler != nil {
+		if err := handler.handleDeleteTcontRequest(log.WithSpanFromContext(context.Background(), ctx), tConf); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", tConf.DeviceId))
+}
+
+/*
+ * Parent GRPC clients
+ */
+
+func (oo *OpenONUAC) setupParentInterAdapterClient(ctx context.Context, endpoint string) error {
+	logger.Infow(ctx, "setting-parent-adapter-connection", log.Fields{"parent-endpoint": endpoint})
+	oo.lockParentAdapterClients.Lock()
+	defer oo.lockParentAdapterClients.Unlock()
+	if _, ok := oo.parentAdapterClients[endpoint]; ok {
+		return nil
+	}
+
+	childClient, err := vgrpc.NewClient(endpoint,
+		oo.oltAdapterRestarted,
+		vgrpc.ActivityCheck(true))
+
+	if err != nil {
+		return err
+	}
+
+	oo.parentAdapterClients[endpoint] = childClient
+
+	go oo.parentAdapterClients[endpoint].Start(log.WithSpanFromContext(context.TODO(), ctx), setAndTestAdapterServiceHandler)
+
+	// Wait until we have a connection to the child adapter.
+	// Unlimited retries or until context expires
+	subCtx := log.WithSpanFromContext(context.TODO(), ctx)
+	backoff := vgrpc.NewBackoff(oo.config.MinBackoffRetryDelay, oo.config.MaxBackoffRetryDelay, 0)
+	for {
+		client, err := oo.parentAdapterClients[endpoint].GetOltInterAdapterServiceClient()
+		if err == nil && client != nil {
+			logger.Infow(subCtx, "connected-to-parent-adapter", log.Fields{"parent-endpoint": endpoint})
+			break
+		}
+		logger.Warnw(subCtx, "connection-to-parent-adapter-not-ready", log.Fields{"error": err, "parent-endpoint": endpoint})
+		// Backoff
+		if err = backoff.Backoff(subCtx); err != nil {
+			logger.Errorw(subCtx, "received-error-on-backoff", log.Fields{"error": err, "parent-endpoint": endpoint})
+			break
+		}
+	}
+	return nil
+}
+
+func (oo *OpenONUAC) getParentAdapterServiceClient(endpoint string) (adapter_services.OltInterAdapterServiceClient, error) {
+	// First check from cache
+	oo.lockParentAdapterClients.RLock()
+	if pgClient, ok := oo.parentAdapterClients[endpoint]; ok {
+		oo.lockParentAdapterClients.RUnlock()
+		return pgClient.GetOltInterAdapterServiceClient()
+	}
+	oo.lockParentAdapterClients.RUnlock()
+
+	// Set the parent connection - can occur on restarts
+	ctx, cancel := context.WithTimeout(context.Background(), oo.config.RPCTimeout)
+	err := oo.setupParentInterAdapterClient(ctx, endpoint)
+	cancel()
+	if err != nil {
+		return nil, err
+	}
+
+	// Get the parent client now
+	oo.lockParentAdapterClients.RLock()
+	defer oo.lockParentAdapterClients.RUnlock()
+	if pgClient, ok := oo.parentAdapterClients[endpoint]; ok {
+		return pgClient.GetOltInterAdapterServiceClient()
+	}
+
+	return nil, fmt.Errorf("no-client-for-endpoint-%s", endpoint)
+}
+
+// TODO:  Any action the adapter needs to do following an olt adapter restart?
+func (oo *OpenONUAC) oltAdapterRestarted(ctx context.Context, endPoint string) error {
+	logger.Errorw(ctx, "olt-adapter-restarted", log.Fields{"endpoint": endPoint})
+	return nil
+}
+
+// setAndTestAdapterServiceHandler is used to test whether the remote gRPC service is up
+func setAndTestAdapterServiceHandler(ctx context.Context, conn *grpc.ClientConn) interface{} {
+	svc := adapter_services.NewOltInterAdapterServiceClient(conn)
+	if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+		return nil
+	}
+	return svc
+}
+
+/*
+ *
+ * Unimplemented APIs
+ *
+ */
+
+//GetOfpDeviceInfo returns OFP information for the given device.  Method not implemented as per [VOL-3202].
+// OF port info is now to be delivered within UniPort create cmp changes in onu_uni_port.go::CreateVolthaPort()
+//
+func (oo *OpenONUAC) GetOfpDeviceInfo(ctx context.Context, device *voltha.Device) (*ic.SwitchCapability, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SimulateAlarm is unimplemented
+func (oo *OpenONUAC) SimulateAlarm(context.Context, *ic.SimulateAlarmMessage) (*common.OperationResp, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SetExtValue is unimplemented
+func (oo *OpenONUAC) SetExtValue(context.Context, *ic.SetExtValueMessage) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SetSingleValue is unimplemented
+func (oo *OpenONUAC) SetSingleValue(context.Context, *extension.SingleSetValueRequest) (*extension.SingleSetValueResponse, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//StartOmciTest not implemented
+func (oo *OpenONUAC) StartOmciTest(ctx context.Context, test *ic.OMCITest) (*voltha.TestResponse, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SuppressEvent unimplemented
+func (oo *OpenONUAC) SuppressEvent(ctx context.Context, filter *voltha.EventFilter) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//UnSuppressEvent  unimplemented
+func (oo *OpenONUAC) UnSuppressEvent(ctx context.Context, filter *voltha.EventFilter) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//GetImageDownloadStatus is unimplemented
+func (oo *OpenONUAC) GetImageDownloadStatus(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//CancelImageDownload is unimplemented
+func (oo *OpenONUAC) CancelImageDownload(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//RevertImageUpdate is unimplemented
+func (oo *OpenONUAC) RevertImageUpdate(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// UpdateFlowsBulk is unimplemented
+func (oo *OpenONUAC) UpdateFlowsBulk(ctx context.Context, flows *ic.BulkFlows) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SelfTestDevice unimplented
+func (oo *OpenONUAC) SelfTestDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SendPacketOut sends packet out to the device
+func (oo *OpenONUAC) SendPacketOut(ctx context.Context, packet *ic.PacketOut) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// EnablePort to Enable PON/NNI interface - seems not to be used/required according to python code
+func (oo *OpenONUAC) EnablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// DisablePort to Disable pon/nni interface  - seems not to be used/required according to python code
+func (oo *OpenONUAC) DisablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// GetExtValue - unimplemented
+func (oo *OpenONUAC) GetExtValue(ctx context.Context, extInfo *ic.GetExtValueMessage) (*voltha.ReturnValues, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// ChildDeviceLost - unimplemented
+func (oo *OpenONUAC) ChildDeviceLost(ctx context.Context, childDevice *voltha.Device) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
diff --git a/internal/pkg/onuadaptercore/openonuimpl.go b/internal/pkg/onuadaptercore/openonuimpl.go
index 8e6e187..efa4e96 100644
--- a/internal/pkg/onuadaptercore/openonuimpl.go
+++ b/internal/pkg/onuadaptercore/openonuimpl.go
@@ -20,7 +20,6 @@
 import (
 	"context"
 	"errors"
-	//"github.com/opencord/voltha-lib-go/v5/pkg/log"
 )
 
 /*
diff --git a/internal/pkg/onuadaptercore/uniportadmin.go b/internal/pkg/onuadaptercore/uniportadmin.go
index 0c22500..2ceea25 100644
--- a/internal/pkg/onuadaptercore/uniportadmin.go
+++ b/internal/pkg/onuadaptercore/uniportadmin.go
@@ -27,9 +27,7 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"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/openflow_13"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 //lockStateFsm defines the structure for the state machine to lock/unlock the ONU UNI ports via OMCI
diff --git a/internal/pkg/onuadaptercore/uniportstatus.go b/internal/pkg/onuadaptercore/uniportstatus.go
index fe9af4c..f7cfe4b 100644
--- a/internal/pkg/onuadaptercore/uniportstatus.go
+++ b/internal/pkg/onuadaptercore/uniportstatus.go
@@ -19,11 +19,12 @@
 
 import (
 	"context"
+	"time"
+
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	"time"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
 )
 
 const (