VOL-3503 Add a device operational state of RECONCILING

Change-Id: I55dad67a24acdfac0af9448e6f19ec9d35edc39e
diff --git a/rw_core/core/api/grpc_nbi_handler_test.go b/rw_core/core/api/grpc_nbi_handler_test.go
index 631a56c..8d9cb16 100755
--- a/rw_core/core/api/grpc_nbi_handler_test.go
+++ b/rw_core/core/api/grpc_nbi_handler_test.go
@@ -123,7 +123,7 @@
 	proxy := model.NewDBPath(backend)
 	nb.adapterMgr = adapter.NewAdapterManager(ctx, proxy, nb.coreInstanceID, nb.kClient)
 	eventProxy := events.NewEventProxy(events.MsgClient(nb.kEventClient), events.MsgTopic(kafka.Topic{Name: cfg.EventTopic}))
-	nb.deviceMgr, nb.logicalDeviceMgr = device.NewManagers(proxy, nb.adapterMgr, nb.kmp, endpointMgr, cfg.CoreTopic, nb.coreInstanceID, cfg.DefaultCoreTimeout, eventProxy, cfg.VolthaStackID)
+	nb.deviceMgr, nb.logicalDeviceMgr = device.NewManagers(proxy, nb.adapterMgr, nb.kmp, endpointMgr, cfg, nb.coreInstanceID, eventProxy)
 	nb.adapterMgr.Start(ctx)
 
 	if err := nb.kmp.Start(ctx); err != nil {
diff --git a/rw_core/core/core.go b/rw_core/core/core.go
index e066914..9d9f5da 100644
--- a/rw_core/core/core.go
+++ b/rw_core/core/core.go
@@ -167,7 +167,7 @@
 
 	// create the core of the system, the device managers
 	endpointMgr := kafka.NewEndpointManager(backend)
-	deviceMgr, logicalDeviceMgr := device.NewManagers(dbPath, adapterMgr, kmp, endpointMgr, cf.CoreTopic, id, cf.DefaultCoreTimeout, eventProxy, cf.VolthaStackID)
+	deviceMgr, logicalDeviceMgr := device.NewManagers(dbPath, adapterMgr, kmp, endpointMgr, cf, id, eventProxy)
 
 	// register kafka RPC handler
 	registerAdapterRequestHandlers(ctx, kmp, deviceMgr, adapterMgr, cf, "adapter-request-handler")
diff --git a/rw_core/core/device/agent.go b/rw_core/core/device/agent.go
index 003f77a..6e32bfe 100755
--- a/rw_core/core/device/agent.go
+++ b/rw_core/core/device/agent.go
@@ -25,9 +25,11 @@
 	"sync"
 	"time"
 
+	"github.com/cenkalti/backoff/v3"
 	"github.com/gogo/protobuf/proto"
 	"github.com/golang/protobuf/ptypes"
 	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-go/rw_core/config"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 
@@ -50,21 +52,24 @@
 
 // Agent represents device agent attributes
 type Agent struct {
-	deviceID       string
-	parentID       string
-	deviceType     string
-	isRootDevice   bool
-	adapterProxy   *remote.AdapterProxy
-	adapterMgr     *adapter.Manager
-	deviceMgr      *Manager
-	dbProxy        *model.Proxy
-	exitChannel    chan int
-	device         *voltha.Device
-	requestQueue   *coreutils.RequestQueue
-	defaultTimeout time.Duration
-	startOnce      sync.Once
-	stopOnce       sync.Once
-	stopped        bool
+	deviceID             string
+	parentID             string
+	deviceType           string
+	isRootDevice         bool
+	adapterProxy         *remote.AdapterProxy
+	adapterMgr           *adapter.Manager
+	deviceMgr            *Manager
+	dbProxy              *model.Proxy
+	exitChannel          chan int
+	device               *voltha.Device
+	requestQueue         *coreutils.RequestQueue
+	defaultTimeout       time.Duration
+	startOnce            sync.Once
+	stopOnce             sync.Once
+	stopped              bool
+	stopReconciling      chan int
+	stopReconcilingMutex sync.RWMutex
+	config               *config.RWCoreFlags
 
 	flowLoader           *flow.Loader
 	groupLoader          *group.Loader
@@ -92,6 +97,7 @@
 		defaultTimeout:       timeout,
 		device:               proto.Clone(device).(*voltha.Device),
 		requestQueue:         coreutils.NewRequestQueue(),
+		config:               deviceMgr.config,
 		flowLoader:           flow.NewLoader(dbPath.SubPath("flows").Proxy(deviceID)),
 		groupLoader:          group.NewLoader(dbPath.SubPath("groups").Proxy(deviceID)),
 		portLoader:           port.NewLoader(dbPath.SubPath("ports").Proxy(deviceID)),
@@ -411,14 +417,11 @@
 		desc = fmt.Sprintf("cannot-enable-an-already-enabled-device: %s", oldDevice.Id)
 		return status.Error(codes.FailedPrecondition, desc)
 	}
-	if agent.isDeletionInProgress() {
+	if !agent.proceedWithRequestNoLock() {
 		agent.requestQueue.RequestComplete()
 
-		operStatus.Code = common.OperationResp_OPERATION_IN_PROGRESS
-
-		desc = fmt.Sprintf("deviceId:%s, Device deletion is in progress.", agent.deviceID)
+		desc = fmt.Sprintf("deviceId:%s, Device deletion or reconciling is in progress.", agent.deviceID)
 		return status.Error(codes.FailedPrecondition, desc)
-
 	}
 	// First figure out which adapter will handle this device type.  We do it at this stage as allow devices to be
 	// pre-provisioned with the required adapter not registered.   At this stage, since we need to communicate
@@ -586,10 +589,13 @@
 		desc = fmt.Sprintf("deviceId:%s, invalid-admin-state:%s", agent.deviceID, cloned.AdminState)
 		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, invalid-admin-state:%s", agent.deviceID, cloned.AdminState)
 	}
-	if agent.isDeletionInProgress() {
+
+	if !agent.proceedWithRequestNoLock() {
 		agent.requestQueue.RequestComplete()
-		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device deletion is in progress.", agent.deviceID)
+		desc = fmt.Sprintf("deviceId:%s, Device deletion or reconciling is in progress.", agent.deviceID)
+		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device reconciling is in progress.", agent.deviceID)
 	}
+
 	// Update the Admin State and operational state before sending the request out
 	cloned.AdminState = voltha.AdminState_DISABLED
 	cloned.OperStatus = voltha.OperStatus_UNKNOWN
@@ -631,8 +637,9 @@
 	logger.Debugw(ctx, "reboot-device", log.Fields{"device-id": agent.deviceID})
 
 	device := agent.getDeviceReadOnlyWithoutLock()
-	if agent.isDeletionInProgress() {
-		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device deletion is in progress.", agent.deviceID)
+	if !agent.proceedWithRequestNoLock() {
+		desc = fmt.Sprintf("deviceId:%s, Device delection or reconciling is in progress.", agent.deviceID)
+		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device reconciling is in progress.", agent.deviceID)
 	}
 	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
 	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
@@ -671,6 +678,10 @@
 		agent.logDeviceUpdate(ctx, "deleteDeviceForce", nil, nil, operStatus, &desc)
 		return status.Error(codes.FailedPrecondition, desc)
 	}
+
+	//Send stop Reconcile if in progress
+	agent.stopReconcile()
+
 	device := agent.cloneDeviceWithoutLock()
 	if err := agent.updateDeviceWithTransientStateAndReleaseLock(ctx, device,
 		voltha.DeviceTransientState_FORCE_DELETING, previousDeviceTransientState); err != nil {
@@ -711,6 +722,13 @@
 		desc = err.Error()
 		return err
 	}
+
+	if agent.isReconcileInProgress() {
+		agent.requestQueue.RequestComplete()
+		desc = fmt.Sprintf("deviceId:%s, Device Reconciling is in progress", agent.deviceID)
+		return status.Error(codes.FailedPrecondition, desc)
+	}
+
 	// Get the device Transient state, return err if it is DELETING
 	previousDeviceTransientState := agent.getTransientState()
 
@@ -956,12 +974,14 @@
 	if prevDevice.OperStatus != device.OperStatus || prevDevice.ConnectStatus != device.ConnectStatus || prevDevice.AdminState != device.AdminState {
 		_ = agent.deviceMgr.Agent.SendDeviceStateChangeEvent(ctx, prevDevice.OperStatus, prevDevice.ConnectStatus, prevDevice.AdminState, device, time.Now().Unix())
 	}
+	deviceTransientState := agent.getTransientState()
+
 	// release lock before processing transition
 	agent.requestQueue.RequestComplete()
 	subCtx := coreutils.WithSpanAndRPCMetadataFromContext(ctx)
 
 	if err := agent.deviceMgr.stateTransitions.ProcessTransition(subCtx,
-		device, prevDevice, voltha.DeviceTransientState_NONE, voltha.DeviceTransientState_NONE); err != nil {
+		device, prevDevice, deviceTransientState, deviceTransientState); err != nil {
 		logger.Errorw(ctx, "failed-process-transition", log.Fields{"device-id": device.Id, "previous-admin-state": prevDevice.AdminState, "current-admin-state": device.AdminState})
 		// Sending RPC EVENT here
 		rpce := agent.deviceMgr.NewRPCEvent(ctx, agent.deviceID, err.Error(), nil)
@@ -1245,3 +1265,167 @@
 
 	return resp, nil
 }
+
+// The device lock MUST be held by the caller.
+func (agent *Agent) proceedWithRequestNoLock() bool {
+	return !agent.isDeletionInProgress() && !agent.isReconcileInProgress()
+}
+
+func (agent *Agent) stopReconcile() {
+	agent.stopReconcilingMutex.Lock()
+	if agent.stopReconciling != nil {
+		agent.stopReconciling <- 0
+	}
+	agent.stopReconcilingMutex.Unlock()
+}
+
+func (agent *Agent) ReconcileDevice(ctx context.Context, device *voltha.Device) {
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+
+	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		desc = err.Error()
+		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		return
+	}
+
+	if !agent.proceedWithRequestNoLock() {
+		agent.requestQueue.RequestComplete()
+		desc = fmt.Sprintf("Either device is in deletion or reconcile is already in progress for device : %s", device.Id)
+		logger.Errorf(ctx, desc)
+		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		return
+	}
+
+	//set transient state to RECONCILE IN PROGRESS
+	err := agent.updateTransientState(ctx, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	if err != nil {
+		agent.requestQueue.RequestComplete()
+		desc = fmt.Sprintf("Not able to set device transient state to Reconcile in progress."+
+			"Err: %s", err.Error())
+		logger.Errorf(ctx, desc)
+		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		return
+	}
+
+	logger.Debugw(ctx, "retrying-reconciling", log.Fields{"deviceID": device.Id})
+	reconcilingBackoff := backoff.NewExponentialBackOff()
+	reconcilingBackoff.InitialInterval = agent.config.BackoffRetryInitialInterval
+	reconcilingBackoff.MaxElapsedTime = agent.config.BackoffRetryMaxElapsedTime
+	reconcilingBackoff.MaxInterval = agent.config.BackoffRetryMaxInterval
+
+	//making here to keep lifecycle of this channel within the scope of retryReconcile
+	agent.stopReconcilingMutex.Lock()
+	agent.stopReconciling = make(chan int)
+	agent.stopReconcilingMutex.Unlock()
+
+Loop:
+	for {
+		// Use an exponential back off to prevent getting into a tight loop
+		duration := reconcilingBackoff.NextBackOff()
+		//This case should never occur in default case as max elapsed time for backoff is 0(by default) , so it will never return stop
+		if duration == backoff.Stop {
+			// If we reach a maximum then warn and reset the backoff
+			// timer and keep attempting.
+			logger.Warnw(ctx, "maximum-reconciling-backoff-reached--resetting-backoff-timer",
+				log.Fields{"max-reconciling-backoff": reconcilingBackoff.MaxElapsedTime,
+					"device-id": device.Id})
+			reconcilingBackoff.Reset()
+			duration = reconcilingBackoff.NextBackOff()
+		}
+
+		backoffTimer := time.NewTimer(duration)
+
+		// Send a reconcile request to the adapter.
+		ch, err := agent.adapterProxy.ReconcileDevice(ctx, agent.device)
+		//release lock before moving further
+		agent.requestQueue.RequestComplete()
+		if err != nil {
+			desc := fmt.Sprintf("Failed reconciling from adapter side. Err: %s", err.Error())
+			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+			<-backoffTimer.C
+			// backoffTimer expired continue
+			// Take lock back before retrying
+			if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+				desc = err.Error()
+				agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+				return
+			}
+			continue
+		}
+
+		// if return err retry if not then break loop and quit retrying reconcile
+		if err = agent.waitForReconcileResponse(backoffTimer, ch); err != nil {
+			desc = err.Error()
+			logger.Errorf(ctx, desc)
+			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		} else {
+			operStatus = &common.OperationResp{Code: common.OperationResp_OPERATION_IN_PROGRESS}
+			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+			break Loop
+		}
+
+		// Take lock back before retrying
+		if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+			desc = err.Error()
+			agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+			return
+		}
+	}
+}
+
+func (agent *Agent) waitForReconcileResponse(backoffTimer *time.Timer, ch chan *kafka.RpcResponse) error {
+	select {
+	// wait for response
+	case resp, ok := <-ch:
+		if !ok {
+			//channel-closed
+			return errors.New("channel on which reconcile response is awaited is closed")
+		} else if resp.Err != nil {
+			//error encountered
+			return errors.New("error encountered while retrying reconcile")
+		}
+
+		//In case of success quit retrying and wait for adapter to reset operation state of device
+		agent.stopReconcilingMutex.Lock()
+		agent.stopReconciling = nil
+		agent.stopReconcilingMutex.Unlock()
+		return nil
+
+	//if reconciling need to be stopped
+	case _, ok := <-agent.stopReconciling:
+		agent.stopReconcilingMutex.Lock()
+		agent.stopReconciling = nil
+		agent.stopReconcilingMutex.Unlock()
+		if !ok {
+			//channel-closed
+			return errors.New("channel used to notify to stop reconcile is closed")
+		}
+		return nil
+	//continue if timer expired
+	case <-backoffTimer.C:
+	}
+	return nil
+}
+
+func (agent *Agent) reconcilingCleanup(ctx context.Context) error {
+	var desc string
+	operStatus := &common.OperationResp{Code: common.OperationResp_OPERATION_FAILURE}
+	if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+		desc = err.Error()
+		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		return err
+	}
+	defer agent.requestQueue.RequestComplete()
+	err := agent.updateTransientState(ctx, voltha.DeviceTransientState_NONE)
+	if err != nil {
+		desc = fmt.Sprintf("Not able to clear device transient state from Reconcile in progress."+
+			"Err: %s", err.Error())
+		logger.Errorf(ctx, desc)
+		agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+		return err
+	}
+	operStatus = &common.OperationResp{Code: common.OperationResp_OPERATION_SUCCESS}
+	agent.logDeviceUpdate(ctx, "Reconciling", nil, nil, operStatus, &desc)
+	return nil
+}
diff --git a/rw_core/core/device/agent_image.go b/rw_core/core/device/agent_image.go
index 3781540..94a8706 100644
--- a/rw_core/core/device/agent_image.go
+++ b/rw_core/core/device/agent_image.go
@@ -38,9 +38,15 @@
 	logger.Debugw(ctx, "download-image", log.Fields{"device-id": agent.deviceID})
 
 	if agent.device.Root {
+		agent.requestQueue.RequestComplete()
 		return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, is an OLT. Image update "+
 			"not supported by VOLTHA. Use Device Manager or other means", agent.deviceID)
 	}
+	if !agent.proceedWithRequestNoLock() {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device reconciling or deletion is in progress.",
+			agent.deviceID)
+	}
 
 	device := agent.cloneDeviceWithoutLock()
 	if device.ImageDownloads != nil {
@@ -99,6 +105,12 @@
 	}
 	logger.Debugw(ctx, "cancel-image-download", log.Fields{"device-id": agent.deviceID})
 
+	if !agent.proceedWithRequestNoLock() {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device reconciling or deletion is in progress.",
+			agent.deviceID)
+	}
+
 	// Update image download state
 	cloned := agent.cloneDeviceWithoutLock()
 	_, index, err := getImage(img, cloned)
@@ -137,6 +149,12 @@
 	}
 	logger.Debugw(ctx, "activate-image", log.Fields{"device-id": agent.deviceID})
 
+	if !agent.proceedWithRequestNoLock() {
+		agent.requestQueue.RequestComplete()
+		return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device reconciling or deletion is in progress.",
+			agent.deviceID)
+	}
+
 	// Update image download state
 	cloned := agent.cloneDeviceWithoutLock()
 	image, index, err := getImage(img, cloned)
@@ -250,6 +268,12 @@
 	}
 	logger.Debugw(ctx, "updating-image-download", log.Fields{"device-id": agent.deviceID, "img": img})
 
+	if !agent.proceedWithRequestNoLock() {
+		agent.requestQueue.RequestComplete()
+		return status.Errorf(codes.FailedPrecondition, "deviceId:%s, Device reconciling or deletion is in progress.",
+			agent.deviceID)
+	}
+
 	// Update the image as well as remove it if the download was cancelled
 	cloned := agent.cloneDeviceWithoutLock()
 	clonedImages := make([]*voltha.ImageDownload, len(cloned.ImageDownloads))
@@ -301,10 +325,14 @@
 	// original context has failed due to timeout , let's open a new one
 	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), agent.defaultTimeout)
 	subCtx = coreutils.WithRPCMetadataFromContext(subCtx, ctx)
+	defer cancel()
 
 	if err := agent.requestQueue.WaitForGreenLight(subCtx); err != nil {
 		logger.Errorw(subCtx, "can't obtain lock", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": err, "args": reqArgs})
-		cancel()
+		return
+	}
+	if !agent.proceedWithRequestNoLock() {
+		agent.requestQueue.RequestComplete()
 		return
 	}
 	if res, ok := response.(error); ok {
@@ -325,7 +353,6 @@
 
 		if imageFailed == nil {
 			logger.Errorw(subCtx, "can't find image", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
-			cancel()
 			return
 		}
 
@@ -341,10 +368,8 @@
 			logger.Errorw(subCtx, "failed-enable-device-after-image-failure",
 				log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": res, "args": reqArgs})
 		}
-		cancel()
 	} else {
 		logger.Errorw(subCtx, "rpc-failed-invalid-error", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
-		cancel()
 		return
 	}
 	// TODO: Post failure message onto kafka
@@ -356,6 +381,10 @@
 		logger.Errorw(ctx, "cannot-obtain-lock", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": err, "args": reqArgs})
 		return
 	}
+	if !agent.proceedWithRequestNoLock() {
+		agent.requestQueue.RequestComplete()
+		return
+	}
 	logger.Infow(ctx, "rpc-successful", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "response": response, "args": reqArgs})
 	cloned := agent.cloneDeviceWithoutLock()
 	//TODO base this on IMAGE ID when created
diff --git a/rw_core/core/device/agent_test.go b/rw_core/core/device/agent_test.go
index 501cc61..daf2508 100755
--- a/rw_core/core/device/agent_test.go
+++ b/rw_core/core/device/agent_test.go
@@ -143,7 +143,7 @@
 	proxy := model.NewDBPath(backend)
 	dat.adapterMgr = adapter.NewAdapterManager(ctx, proxy, dat.coreInstanceID, dat.kClient)
 	eventProxy := events.NewEventProxy(events.MsgClient(dat.kEventClient), events.MsgTopic(kafka.Topic{Name: cfg.EventTopic}))
-	dat.deviceMgr, dat.logicalDeviceMgr = NewManagers(proxy, dat.adapterMgr, dat.kmp, endpointMgr, cfg.CoreTopic, dat.coreInstanceID, cfg.DefaultCoreTimeout, eventProxy, cfg.VolthaStackID)
+	dat.deviceMgr, dat.logicalDeviceMgr = NewManagers(proxy, dat.adapterMgr, dat.kmp, endpointMgr, cfg, dat.coreInstanceID, eventProxy)
 	dat.adapterMgr.Start(context.Background())
 	if err = dat.kmp.Start(ctx); err != nil {
 		logger.Fatal(ctx, "Cannot start InterContainerProxy")
diff --git a/rw_core/core/device/agent_transient_state.go b/rw_core/core/device/agent_transient_state.go
index 776d60d..6b5b100 100644
--- a/rw_core/core/device/agent_transient_state.go
+++ b/rw_core/core/device/agent_transient_state.go
@@ -18,6 +18,7 @@
 
 import (
 	"context"
+	"github.com/opencord/voltha-protos/v4/go/common"
 	"github.com/opencord/voltha-protos/v4/go/voltha"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
@@ -30,7 +31,17 @@
 	return deviceTransientState
 }
 
+func (agent *Agent) matchTransientState(transientState voltha.DeviceTransientState_Types) bool {
+	transientStateHandle := agent.transientStateLoader.Lock()
+	defer transientStateHandle.UnLock()
+	return transientState == transientStateHandle.GetReadOnly()
+}
+
 func (agent *Agent) updateTransientState(ctx context.Context, transientState voltha.DeviceTransientState_Types) error {
+	// Already in same transientState
+	if transientState == agent.getTransientState() {
+		return nil
+	}
 	// Update device transient state
 	transientStateHandle := agent.transientStateLoader.Lock()
 	if err := transientStateHandle.Update(ctx, transientState); err != nil {
@@ -62,3 +73,9 @@
 	transientStateHandle.UnLock()
 	return nil
 }
+
+func (agent *Agent) isReconcileInProgress() bool {
+	device := agent.getDeviceReadOnlyWithoutLock()
+	return device.OperStatus == common.OperStatus_RECONCILING ||
+		agent.matchTransientState(voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+}
diff --git a/rw_core/core/device/logical_agent_test.go b/rw_core/core/device/logical_agent_test.go
index 818e6ac..94f336c 100644
--- a/rw_core/core/device/logical_agent_test.go
+++ b/rw_core/core/device/logical_agent_test.go
@@ -162,7 +162,7 @@
 	proxy := model.NewDBPath(backend)
 	adapterMgr := adapter.NewAdapterManager(ctx, proxy, lda.coreInstanceID, lda.kClient)
 	eventProxy := events.NewEventProxy(events.MsgClient(lda.kEventClient), events.MsgTopic(kafka.Topic{Name: cfg.EventTopic}))
-	lda.deviceMgr, lda.logicalDeviceMgr = NewManagers(proxy, adapterMgr, lda.kmp, endpointMgr, cfg.CoreTopic, lda.coreInstanceID, cfg.DefaultCoreTimeout, eventProxy, cfg.VolthaStackID)
+	lda.deviceMgr, lda.logicalDeviceMgr = NewManagers(proxy, adapterMgr, lda.kmp, endpointMgr, cfg, lda.coreInstanceID, eventProxy)
 	if err = lda.kmp.Start(ctx); err != nil {
 		logger.Fatal(ctx, "Cannot start InterContainerProxy")
 	}
diff --git a/rw_core/core/device/manager.go b/rw_core/core/device/manager.go
index dd64ab9..2c77186 100755
--- a/rw_core/core/device/manager.go
+++ b/rw_core/core/device/manager.go
@@ -19,6 +19,7 @@
 import (
 	"context"
 	"errors"
+	"github.com/opencord/voltha-go/rw_core/config"
 	"sync"
 	"time"
 
@@ -58,31 +59,33 @@
 	defaultTimeout          time.Duration
 	devicesLoadingLock      sync.RWMutex
 	deviceLoadingInProgress map[string][]chan int
+	config                  *config.RWCoreFlags
 }
 
 //NewManagers creates the Manager and the Logical Manager.
-func NewManagers(dbPath *model.Path, adapterMgr *adapter.Manager, kmp kafka.InterContainerProxy, endpointMgr kafka.EndpointManager, coreTopic, coreInstanceID string, defaultCoreTimeout time.Duration, eventProxy *events.EventProxy, stackID string) (*Manager, *LogicalManager) {
+func NewManagers(dbPath *model.Path, adapterMgr *adapter.Manager, kmp kafka.InterContainerProxy, endpointMgr kafka.EndpointManager, cf *config.RWCoreFlags, coreInstanceID string, eventProxy *events.EventProxy) (*Manager, *LogicalManager) {
 	deviceMgr := &Manager{
 		rootDevices:             make(map[string]bool),
 		kafkaICProxy:            kmp,
-		adapterProxy:            remote.NewAdapterProxy(kmp, coreTopic, endpointMgr),
+		adapterProxy:            remote.NewAdapterProxy(kmp, cf.CoreTopic, endpointMgr),
 		coreInstanceID:          coreInstanceID,
 		dbPath:                  dbPath,
 		dProxy:                  dbPath.Proxy("devices"),
 		adapterMgr:              adapterMgr,
-		defaultTimeout:          defaultCoreTimeout,
-		Agent:                   event.NewAgent(eventProxy, coreInstanceID, stackID),
+		defaultTimeout:          cf.DefaultCoreTimeout,
+		Agent:                   event.NewAgent(eventProxy, coreInstanceID, cf.VolthaStackID),
 		deviceLoadingInProgress: make(map[string][]chan int),
+		config:                  cf,
 	}
 	deviceMgr.stateTransitions = state.NewTransitionMap(deviceMgr)
 
 	logicalDeviceMgr := &LogicalManager{
-		Manager:                        event.NewManager(eventProxy, coreInstanceID, stackID),
+		Manager:                        event.NewManager(eventProxy, coreInstanceID, cf.VolthaStackID),
 		deviceMgr:                      deviceMgr,
 		kafkaICProxy:                   kmp,
 		dbPath:                         dbPath,
 		ldProxy:                        dbPath.Proxy("logical_devices"),
-		defaultTimeout:                 defaultCoreTimeout,
+		defaultTimeout:                 cf.DefaultCoreTimeout,
 		logicalDeviceLoadingInProgress: make(map[string][]chan int),
 	}
 	deviceMgr.logicalDeviceMgr = logicalDeviceMgr
@@ -651,17 +654,6 @@
 	return &empty.Empty{}, nil
 }
 
-// isOkToReconcile validates whether a device is in the correct status to be reconciled
-func (dMgr *Manager) isOkToReconcile(ctx context.Context, device *voltha.Device) bool {
-	if device == nil {
-		return false
-	}
-	if agent := dMgr.getDeviceAgent(ctx, device.Id); agent != nil {
-		return device.AdminState != voltha.AdminState_PREPROVISIONED && (!agent.isDeletionInProgress())
-	}
-	return false
-}
-
 // adapterRestarted is invoked whenever an adapter is restarted
 func (dMgr *Manager) adapterRestarted(ctx context.Context, adapter *voltha.Adapter) error {
 	logger.Debugw(ctx, "adapter-restarted", log.Fields{"adapter-id": adapter.Id, "vendor": adapter.Vendor,
@@ -673,7 +665,11 @@
 		return nil
 	}
 
-	responses := make([]utils.Response, 0)
+	if len(dMgr.rootDevices) == 0 {
+		logger.Debugw(ctx, "no-managed-device-to-reconcile", log.Fields{"adapter-id": adapter.Id})
+		return nil
+	}
+
 	for rootDeviceID := range dMgr.rootDevices {
 		dAgent := dMgr.getDeviceAgent(ctx, rootDeviceID)
 		if dAgent == nil {
@@ -691,9 +687,9 @@
 				continue
 			}
 			if isDeviceOwnedByService {
-				if dMgr.isOkToReconcile(ctx, rootDevice) {
+				if rootDevice.AdminState != voltha.AdminState_PREPROVISIONED {
 					logger.Debugw(ctx, "reconciling-root-device", log.Fields{"rootId": rootDevice.Id})
-					responses = append(responses, dMgr.sendReconcileDeviceRequest(ctx, rootDevice))
+					go dAgent.ReconcileDevice(ctx, rootDevice)
 				} else {
 					logger.Debugw(ctx, "not-reconciling-root-device", log.Fields{"rootId": rootDevice.Id, "state": rootDevice.AdminState})
 				}
@@ -708,9 +704,9 @@
 								logger.Warnw(ctx, "is-device-owned-by-service", log.Fields{"error": err, "child-device-id": childDevice.Id, "adapter-type": adapter.Type, "replica-number": adapter.CurrentReplica})
 							}
 							if isDeviceOwnedByService {
-								if dMgr.isOkToReconcile(ctx, childDevice) {
+								if childDevice.AdminState != voltha.AdminState_PREPROVISIONED {
 									logger.Debugw(ctx, "reconciling-child-device", log.Fields{"child-device-id": childDevice.Id})
-									responses = append(responses, dMgr.sendReconcileDeviceRequest(ctx, childDevice))
+									go dAgent.ReconcileDevice(ctx, childDevice)
 								} else {
 									logger.Debugw(ctx, "not-reconciling-child-device", log.Fields{"child-device-id": childDevice.Id, "state": childDevice.AdminState})
 								}
@@ -725,54 +721,25 @@
 			}
 		}
 	}
-	if len(responses) > 0 {
-		// Wait for completion
-		if res := utils.WaitForNilOrErrorResponses(dMgr.defaultTimeout, responses...); res != nil {
-			return status.Errorf(codes.Aborted, "errors-%s", res)
-		}
-	} else {
-		logger.Debugw(ctx, "no-managed-device-to-reconcile", log.Fields{"adapter-id": adapter.Id})
-	}
+	logger.Debugw(ctx, "Reconciling for device on adapter restart is initiated", log.Fields{"adapter-id": adapter.Id})
+
 	return nil
 }
 
-func (dMgr *Manager) sendReconcileDeviceRequest(ctx context.Context, device *voltha.Device) utils.Response {
-	// Send a reconcile request to the adapter. Since this Core may not be managing this device then there is no
-	// point of creating a device agent (if the device is not being managed by this Core) before sending the request
-	// to the adapter.   We will therefore bypass the adapter adapter and send the request directly to the adapter via
-	// the adapter proxy.
-	response := utils.NewResponse()
-	ch, err := dMgr.adapterProxy.ReconcileDevice(ctx, device)
-	if err != nil {
-		response.Error(err)
-	}
-	// Wait for adapter response in its own routine
-	go func() {
-		resp, ok := <-ch
-		if !ok {
-			response.Error(status.Errorf(codes.Aborted, "channel-closed-device: %s", device.Id))
-		} else if resp.Err != nil {
-			response.Error(resp.Err)
-		}
-		response.Done()
-	}()
-	return response
-}
-
 func (dMgr *Manager) ReconcileChildDevices(ctx context.Context, parentDeviceID string) error {
+	dAgent := dMgr.getDeviceAgent(ctx, parentDeviceID)
+	if dAgent == nil {
+		return status.Errorf(codes.NotFound, "error-unable to get agent from device")
+	}
 	if parentDevicePorts, err := dMgr.listDevicePorts(ctx, parentDeviceID); err == nil {
-		responses := make([]utils.Response, 0)
 		for _, port := range parentDevicePorts {
 			for _, peer := range port.Peers {
 				if childDevice, err := dMgr.getDeviceFromModel(ctx, peer.DeviceId); err == nil {
-					responses = append(responses, dMgr.sendReconcileDeviceRequest(ctx, childDevice))
+					go dAgent.ReconcileDevice(ctx, childDevice)
 				}
 			}
 		}
-		// Wait for completion
-		if res := utils.WaitForNilOrErrorResponses(dMgr.defaultTimeout, responses...); res != nil {
-			return status.Errorf(codes.Aborted, "errors-%s", res)
-		}
+		logger.Debugw(ctx, "Reconcile initiated for child devices", log.Fields{"parent-device-id": parentDeviceID})
 	}
 	return nil
 }
@@ -1961,3 +1928,17 @@
 		}
 	}
 }
+
+func (dMgr *Manager) ReconcilingCleanup(ctx context.Context, device *voltha.Device) error {
+	agent := dMgr.getDeviceAgent(ctx, device.Id)
+	if agent == nil {
+		logger.Errorf(ctx, "Not able to get device agent.")
+		return status.Errorf(codes.NotFound, "Not able to get device agent for device : %s", device.Id)
+	}
+	err := agent.reconcilingCleanup(ctx)
+	if err != nil {
+		logger.Errorf(ctx, err.Error())
+		return status.Errorf(codes.Internal, err.Error())
+	}
+	return nil
+}
diff --git a/rw_core/core/device/manager_test.go b/rw_core/core/device/manager_test.go
index 40407f9..cf93502 100644
--- a/rw_core/core/device/manager_test.go
+++ b/rw_core/core/device/manager_test.go
@@ -103,7 +103,7 @@
 	proxy := model.NewDBPath(backend)
 	dat.adapterMgr = adapter.NewAdapterManager(ctx, proxy, dat.coreInstanceID, dat.kClient)
 	eventProxy := events.NewEventProxy(events.MsgClient(dat.kEventClient), events.MsgTopic(kafka.Topic{Name: cfg.EventTopic}))
-	dat.deviceMgr, dat.logicalDeviceMgr = NewManagers(proxy, dat.adapterMgr, dat.kmp, endpointMgr, cfg.CoreTopic, dat.coreInstanceID, cfg.DefaultCoreTimeout, eventProxy, cfg.VolthaStackID)
+	dat.deviceMgr, dat.logicalDeviceMgr = NewManagers(proxy, dat.adapterMgr, dat.kmp, endpointMgr, cfg, dat.coreInstanceID, eventProxy)
 	dat.adapterMgr.Start(context.Background())
 	if err = dat.kmp.Start(ctx); err != nil {
 		logger.Fatal(ctx, "Cannot start InterContainerProxy")
diff --git a/rw_core/core/device/state/transitions.go b/rw_core/core/device/state/transitions.go
index f4b442c..c3bd372 100644
--- a/rw_core/core/device/state/transitions.go
+++ b/rw_core/core/device/state/transitions.go
@@ -100,6 +100,7 @@
 	ChildDeviceLost(ctx context.Context, curr *voltha.Device) error
 	DeleteAllLogicalPorts(ctx context.Context, curr *voltha.Device) error
 	DeleteAllDeviceFlows(ctx context.Context, curr *voltha.Device) error
+	ReconcilingCleanup(ctx context.Context, curr *voltha.Device) error
 }
 
 // NewTransitionMap creates transition map
@@ -276,6 +277,54 @@
 			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
 			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_NONE},
 			handlers:      []transitionHandler{dMgr.NotifyInvalidTransition}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
+	transitionMap.transitions = append(transitionMap.transitions,
+		transition{
+			deviceType:    any,
+			previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_RECONCILING, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			currentState:  deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN, Transient: voltha.DeviceTransientState_RECONCILE_IN_PROGRESS},
+			handlers:      []transitionHandler{dMgr.ReconcilingCleanup}})
 
 	return &transitionMap
 }
@@ -415,7 +464,7 @@
 		"prev-transient-state": prevDeviceTransientState,
 	})
 	handlers := tMap.getTransitionHandler(ctx, device, prevDevice, deviceTransientState, prevDeviceTransientState)
-	if handlers == nil {
+	if len(handlers) == 0 {
 		logger.Debugw(ctx, "no-op-transition", log.Fields{"deviceId": device.Id})
 		return nil
 	}
diff --git a/rw_core/core/device/state/transitions_test.go b/rw_core/core/device/state/transitions_test.go
index cc1aa6d..ee4986b 100644
--- a/rw_core/core/device/state/transitions_test.go
+++ b/rw_core/core/device/state/transitions_test.go
@@ -58,9 +58,9 @@
 	assert.True(t, reflect.ValueOf(tdm.NotifyInvalidTransition).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 }
 
-func assertNoOpTransition(t *testing.T, device, prevDevice *voltha.Device) {
-	handlers := transitionMap.getTransitionHandler(context.Background(), device, prevDevice, voltha.DeviceTransientState_NONE,
-		voltha.DeviceTransientState_NONE)
+func assertNoOpTransition(t *testing.T, device, prevDevice *voltha.Device, transientState voltha.DeviceTransientState_Types) {
+	handlers := transitionMap.getTransitionHandler(context.Background(), device, prevDevice, transientState,
+		transientState)
 	assert.Equal(t, 0, len(handlers))
 }
 
@@ -236,6 +236,41 @@
 	assert.Equal(t, 1, len(handlers))
 	assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
 
+	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assert.Equal(t, 1, len(handlers))
+	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
+	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assert.Equal(t, 1, len(handlers))
+	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assert.Equal(t, 1, len(handlers))
+	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+	previousDevice = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
+	device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assert.Equal(t, 1, len(handlers))
+	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
+	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+	handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS,
+		voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+	assert.Equal(t, 1, len(handlers))
+	assert.True(t, reflect.ValueOf(tdm.ReconcilingCleanup).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
 	var deleteDeviceTest = struct {
 		previousDevices        []*voltha.Device
 		devices                []*voltha.Device
@@ -327,35 +362,71 @@
 
 	previousDevice := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
 	device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
 	device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
 
 	previousDevice = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
 	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
-	assertNoOpTransition(t, device, previousDevice)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_NONE)
+
+	previousDevice = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
+	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+	device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+	device = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
+	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+	device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+	device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+	device = getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
+
+	previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
+	device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_RECONCILING)
+	assertNoOpTransition(t, device, previousDevice, voltha.DeviceTransientState_RECONCILE_IN_PROGRESS)
 }
 
 func TestMatch(t *testing.T) {