[VOL-5464] - Reduce call flows in voltha during ONU reboot

Change-Id: I081dabb9d5fa5a5d177a4fee03879e207caa4dc8
Signed-off-by: Sridhar Ravindra <sridhar.ravindra@radisys.com>
diff --git a/internal/pkg/core/device_handler.go b/internal/pkg/core/device_handler.go
index fc5efef..a3cdb6c 100755
--- a/internal/pkg/core/device_handler.go
+++ b/internal/pkg/core/device_handler.go
@@ -181,7 +181,8 @@
 	UniVlanConfigFsmMap            map[uint8]*avcfg.UniVlanConfigFsm
 	pOnuUpradeFsm                  *swupg.OnuUpgradeFsm
 	chUniVlanConfigReconcilingDone chan uint16 //channel to indicate that VlanConfig reconciling for a specific UNI has been finished
-	chReconcilingFinished          chan bool   //channel to indicate that reconciling has been finished
+	chUniVlanConfigOnRebootDone    chan uint16
+	chReconcilingFinished          chan bool //channel to indicate that reconciling has been finished
 	pLastUpgradeImageState         *voltha.ImageState
 	upgradeFsmChan                 chan struct{}
 
@@ -625,6 +626,10 @@
 	logger.Debugw(ctx, "FlowUpdateIncremental started", log.Fields{"device-id": dh.DeviceID, "flow": apOfFlowChanges, "metadata": apFlowMetaData})
 	var errorsList []error
 	var retError error
+	if dh.GetPersRebootFlag(ctx) {
+		logger.Warnw(ctx, "FlowUpdateIncremental ignored as deivce is being configured post reboot", log.Fields{"device-id": dh.DeviceID})
+		return fmt.Errorf("errors-installing-one-or-more-flows-groups-reboot-in-progress")
+	}
 	//Remove flows (always remove flows first - remove old and add new with same cookie may be part of the same request)
 	if apOfFlowChanges.ToRemove != nil {
 		for _, flowItem := range apOfFlowChanges.ToRemove.Items {
@@ -1134,13 +1139,13 @@
 		//the CookieSlice can be passed 'by value' here, - which internally passes its reference
 		if _, exist := dh.UniVlanConfigFsmMap[loUniID]; exist {
 			if err := dh.UniVlanConfigFsmMap[loUniID].SetUniFlowParams(ctx, flowData.VlanRuleParams.TpID,
-				flowData.CookieSlice, uint16(flowData.VlanRuleParams.MatchVid), uint8(flowData.VlanRuleParams.MatchPcp), uint16(flowData.VlanRuleParams.SetVid), uint8(flowData.VlanRuleParams.SetPcp), flowData.VlanRuleParams.InnerCvlan, lastFlowToReconcile, flowData.Meter, nil); err != nil {
+				flowData.CookieSlice, uint16(flowData.VlanRuleParams.MatchVid), uint8(flowData.VlanRuleParams.MatchPcp), uint16(flowData.VlanRuleParams.SetVid), uint8(flowData.VlanRuleParams.SetPcp), flowData.VlanRuleParams.InnerCvlan, lastFlowToReconcile, false, flowData.Meter, nil); err != nil {
 				logger.Errorw(ctx, err.Error(), log.Fields{"device-id": dh.DeviceID})
 			}
 		} else {
 			if err := dh.createVlanFilterFsm(ctx, apUniPort, flowData.VlanRuleParams.TpID, flowData.CookieSlice,
 				uint16(flowData.VlanRuleParams.MatchVid), uint8(flowData.VlanRuleParams.MatchPcp), uint16(flowData.VlanRuleParams.SetVid),
-				uint8(flowData.VlanRuleParams.SetPcp), flowData.VlanRuleParams.InnerCvlan, cmn.OmciVlanFilterAddDone, lastFlowToReconcile, flowData.Meter, nil); err != nil {
+				uint8(flowData.VlanRuleParams.SetPcp), flowData.VlanRuleParams.InnerCvlan, cmn.OmciVlanFilterAddDone, lastFlowToReconcile, false, flowData.Meter, nil); err != nil {
 				logger.Errorw(ctx, err.Error(), log.Fields{"device-id": dh.DeviceID})
 			}
 		}
@@ -1234,6 +1239,303 @@
 	}
 }
 
+func (dh *deviceHandler) DeviceFlowConfigOnReboot(ctx context.Context) {
+	logger.Debugw(ctx, "rebooting - trigger flow config", log.Fields{"device-id": dh.DeviceID})
+
+	defer dh.UpdateAndStoreRebootState(ctx, false)
+	pDevEntry := dh.GetOnuDeviceEntry(ctx, false)
+	if pDevEntry == nil {
+		logger.Errorw(ctx, "rebooting - no valid OnuDevice - aborting", log.Fields{"device-id": dh.DeviceID})
+		return
+	}
+
+	pDevEntry.MutexPersOnuConfig.RLock()
+	if len(pDevEntry.SOnuPersistentData.PersUniConfig) == 0 {
+		pDevEntry.MutexPersOnuConfig.RUnlock()
+		logger.Warn(ctx, "rebooting - no uni-configs have been stored - aborting",
+			log.Fields{"device-id": dh.DeviceID})
+		return
+	}
+	flowsFound := false
+	var uniVlanConfigEntries []uint8
+	var loWaitGroupWTO cmn.WaitGroupWithTimeOut
+
+	for _, uniData := range pDevEntry.SOnuPersistentData.PersUniConfig {
+		//TODO: check for uni-port specific reconcilement in case of multi-uni-port-per-onu-support
+		if len(uniData.PersFlowParams) == 0 {
+			logger.Debugw(ctx, "rebooting - no flows stored for uniID",
+				log.Fields{"uni-id": uniData.PersUniID, "device-id": dh.DeviceID})
+			continue
+		}
+		if !dh.anyTpPathExists(uniData.PersTpPathMap) {
+			logger.Warnw(ctx, "rebooting - but no TPs stored for uniID, abort",
+				log.Fields{"uni-id": uniData.PersUniID, "device-id": dh.DeviceID})
+			// It doesn't make sense to configure any flows if no TPs are available
+			continue
+		}
+		//release MutexPersOnuConfig before VlanConfig processing as otherwise the reception of
+		//  OMCI frames may get completely stuck due to lock request within IncrementMibDataSync() at OMCI
+		//  frame reception may also lock the complete OMCI reception processing based on mutexRxSchedMap
+		pDevEntry.MutexPersOnuConfig.RUnlock()
+
+		var uniPort *cmn.OnuUniPort
+		var exist bool
+		uniNo := platform.MkUniPortNum(ctx, dh.pOnuIndication.GetIntfId(), dh.pOnuIndication.GetOnuId(), uint32(uniData.PersUniID))
+		if uniPort, exist = dh.uniEntityMap[uniNo]; !exist {
+			logger.Errorw(ctx, "rebooting - OnuUniPort data not found  - terminate flow config",
+				log.Fields{"uniNo": uniNo, "device-id": dh.DeviceID})
+			return
+		}
+
+		dh.updateOnRebootFlowConfig(ctx, uniPort, uniData.PersFlowParams, uniVlanConfigEntries, &loWaitGroupWTO, &flowsFound)
+
+		logger.Debugw(ctx, "rebooting - flows processed", log.Fields{
+			"device-id": dh.DeviceID, "uni-id": uniData.PersUniID,
+			"NumUniFlows":       dh.UniVlanConfigFsmMap[uniData.PersUniID].NumUniFlows,
+			"ConfiguredUniFlow": dh.UniVlanConfigFsmMap[uniData.PersUniID].ConfiguredUniFlow})
+		// this can't be used as global finished reconciling flag because
+		// assumes is getting called before the state machines for the last flow is completed,
+		// while this is not guaranteed.
+		pDevEntry.MutexPersOnuConfig.RLock() //set protection again for loop test on SOnuPersistentData
+	} // for all UNI entries from SOnuPersistentData
+	pDevEntry.MutexPersOnuConfig.RUnlock()
+
+	if !flowsFound {
+		logger.Warn(ctx, "rebooting - no flows have been stored before device reboot - terminate flow config",
+			log.Fields{"device-id": dh.DeviceID})
+		return
+	}
+	logger.Debugw(ctx, "rebooting - waiting on ready indication of requested UNIs", log.Fields{
+		"device-id": dh.DeviceID, "expiry": dh.reconcileExpiryVlanConfig})
+	if executed := loWaitGroupWTO.WaitTimeout(dh.reconcileExpiryVlanConfig); executed {
+		logger.Debugw(ctx, "rebooting - flow config for all UNI's has been finished in time",
+			log.Fields{"device-id": dh.DeviceID})
+	} else {
+		logger.Errorw(ctx, "rebooting - timeout waiting for flow config for all UNI's to be finished!",
+			log.Fields{"device-id": dh.DeviceID})
+		return
+	}
+}
+
+func (dh *deviceHandler) UpdateAndStoreRebootState(ctx context.Context, flag bool) {
+	dh.UpdateRebootPersData(ctx, flag)
+	if err := dh.StorePersistentData(ctx); err != nil {
+		logger.Errorw(ctx, "rebooting - failed to store persistent data in kv store", log.Fields{"device-id": dh.DeviceID})
+	}
+}
+
+func (dh *deviceHandler) updateOnRebootFlowConfig(ctx context.Context, apUniPort *cmn.OnuUniPort,
+	aPersFlowParam []cmn.UniVlanFlowParams, aUniVlanConfigEntries []uint8,
+	apWaitGroup *cmn.WaitGroupWithTimeOut, apFlowsFound *bool) {
+	flowsProcessed := 0
+	lastFlowToConfigOnReboot := false
+	loUniID := apUniPort.UniID
+	for _, flowData := range aPersFlowParam {
+		if !(*apFlowsFound) {
+			*apFlowsFound = true
+			syncChannel := make(chan struct{})
+			// start go routine with select() on reconciling vlan config channel before
+			// starting vlan config reconciling process to prevent loss of any signal
+			// this routine just collects all the received 'flow-reconciled' signals - possibly from different UNI's
+			go dh.waitOnUniVlanConfigOnRebootReady(ctx, syncChannel, apWaitGroup)
+			//block until the wait routine is really blocked on channel input
+			//  in order to prevent to early ready signal from VlanConfig processing
+			<-syncChannel
+		}
+		if flowsProcessed == len(aPersFlowParam)-1 {
+			var uniAdded bool
+			lastFlowToConfigOnReboot = true
+			if aUniVlanConfigEntries, uniAdded = dh.appendIfMissing(aUniVlanConfigEntries, loUniID); uniAdded {
+				apWaitGroup.Add(1) //increment the waiting group
+			}
+		}
+		logger.Debugw(ctx, "rebooting - add flow with cookie slice", log.Fields{
+			"device-id": dh.DeviceID, "uni-id": loUniID,
+			"flowsProcessed": flowsProcessed, "cookies": flowData.CookieSlice})
+		dh.lockVlanConfig.Lock()
+		//the CookieSlice can be passed 'by value' here, - which internally passes its reference
+		if _, exist := dh.UniVlanConfigFsmMap[loUniID]; exist {
+			if err := dh.UniVlanConfigFsmMap[loUniID].SetUniFlowParams(ctx, flowData.VlanRuleParams.TpID,
+				flowData.CookieSlice, uint16(flowData.VlanRuleParams.MatchVid), uint8(flowData.VlanRuleParams.MatchPcp), uint16(flowData.VlanRuleParams.SetVid), uint8(flowData.VlanRuleParams.SetPcp), flowData.VlanRuleParams.InnerCvlan, false, lastFlowToConfigOnReboot, flowData.Meter, nil); err != nil {
+				logger.Errorw(ctx, err.Error(), log.Fields{"device-id": dh.DeviceID})
+			}
+		} else {
+			if err := dh.createVlanFilterFsm(ctx, apUniPort, flowData.VlanRuleParams.TpID, flowData.CookieSlice,
+				uint16(flowData.VlanRuleParams.MatchVid), uint8(flowData.VlanRuleParams.MatchPcp), uint16(flowData.VlanRuleParams.SetVid),
+				uint8(flowData.VlanRuleParams.SetPcp), flowData.VlanRuleParams.InnerCvlan, cmn.OmciVlanFilterAddDone, false, lastFlowToConfigOnReboot, flowData.Meter, nil); err != nil {
+				logger.Errorw(ctx, err.Error(), log.Fields{"device-id": dh.DeviceID})
+			}
+		}
+		dh.lockVlanConfig.Unlock()
+		flowsProcessed++
+	} //for all flows of this UNI
+}
+
+func (dh *deviceHandler) waitOnUniVlanConfigOnRebootReady(ctx context.Context, aSyncChannel chan<- struct{},
+	waitGroup *cmn.WaitGroupWithTimeOut) {
+	var rebootUniVlanConfigEntries []uint8
+	var appended bool
+	expiry := dh.GetReconcileExpiryVlanConfigAbort()
+	logger.Debugw(ctx, "start waiting on vlanConfig ready indications on reboot", log.Fields{
+		"device-id": dh.DeviceID, "expiry": expiry})
+	// indicate blocking on channel now to the caller
+	aSyncChannel <- struct{}{}
+	for {
+		select {
+		case <-dh.deviceDeleteCommChan:
+			// Cancel the context and return
+			logger.Warnw(ctx, "Device Deletion invoked , stop further processing ", log.Fields{"device-id": dh.DeviceID})
+			return
+
+		case uniIndication := <-dh.chUniVlanConfigOnRebootDone:
+			switch uniIndication {
+			// this should be a valid UNI vlan config done indication
+			default:
+				if uniIndication < platform.MaxUnisPerOnu {
+					logger.Info(ctx, "rebooting - configuring flows has been finished in time for this UNI",
+						log.Fields{"device-id": dh.DeviceID, "uni-id": uniIndication})
+					if rebootUniVlanConfigEntries, appended =
+						dh.appendIfMissing(rebootUniVlanConfigEntries, uint8(uniIndication)); appended {
+						waitGroup.Done()
+					}
+				} else {
+					logger.Errorw(ctx, "received unexpected UNI flowConfig done indication - is ignored", log.Fields{"device-id": dh.DeviceID, "uni-id": uniIndication})
+				}
+			} //switch uniIndication
+
+		case <-time.After(expiry):
+			logger.Errorw(ctx, "timeout waiting for configuring all UNI flows to be finished!",
+				log.Fields{"device-id": dh.DeviceID})
+			return
+		}
+	}
+}
+
+func (dh *deviceHandler) SendChUniVlanConfigFinishedOnReboot(value uint16) {
+	if dh != nil { //if the object still exists (might have been already deleted in background)
+		//use asynchronous channel sending to avoid stucking on non-waiting receiver
+		select {
+		case dh.chUniVlanConfigOnRebootDone <- value:
+		default:
+		}
+	}
+}
+
+func (dh *deviceHandler) CheckForDeviceTechProf(ctx context.Context) bool {
+	logger.Info(ctx, "Check for tech profile config", log.Fields{"device-id": dh.DeviceID})
+	techProfInstLoadFailed := false
+	continueWithFlowConfig := false
+	defer dh.UpdateAndStoreRebootState(ctx, continueWithFlowConfig)
+	// Stop any on going reconciling thread as the flow configuration post reboot will be performed here
+	dh.stopReconciling(ctx, true, cWaitReconcileFlowNoActivity)
+
+	pDevEntry := dh.GetOnuDeviceEntry(ctx, false)
+	if pDevEntry == nil {
+		logger.Errorw(ctx, "no valid OnuDevice - aborting", log.Fields{"device-id": dh.DeviceID})
+		return continueWithFlowConfig
+	}
+	dh.lockDevice.RLock()
+	if dh.pOnuTP == nil {
+		//should normally not happen ...
+		logger.Warnw(ctx, "onuTechProf instance not set up - ignoring request",
+			log.Fields{"device-id": dh.DeviceID})
+		dh.lockDevice.RUnlock()
+		return continueWithFlowConfig
+	}
+
+	dh.pOnuTP.LockTpProcMutex()
+	defer dh.pOnuTP.UnlockTpProcMutex()
+	defer dh.lockDevice.RUnlock()
+
+	pDevEntry.MutexPersOnuConfig.RLock()
+	persMutexLock := true
+	if len(pDevEntry.SOnuPersistentData.PersUniConfig) == 0 {
+		pDevEntry.MutexPersOnuConfig.RUnlock()
+		logger.Info(ctx, "no uni-configs have been stored - aborting",
+			log.Fields{"device-id": dh.DeviceID})
+		return continueWithFlowConfig
+	}
+
+outerLoop:
+	for _, uniData := range pDevEntry.SOnuPersistentData.PersUniConfig {
+		uniID := uniData.PersUniID
+
+		if !dh.anyTpPathExists(uniData.PersTpPathMap) {
+			logger.Debugw(ctx, "no TPs stored for uniID",
+				log.Fields{"uni-id": uniID, "device-id": dh.DeviceID})
+			continue
+		}
+		//release MutexPersOnuConfig before TechProfile (ANIConfig) processing as otherwise the reception of
+		//  OMCI frames may get completely stuck due to lock request within IncrementMibDataSync() at OMCI
+		//  frame reception may also lock the complete OMCI reception processing based on mutexRxSchedMap
+		pDevEntry.MutexPersOnuConfig.RUnlock()
+		persMutexLock = false
+		for tpID, tpPath := range uniData.PersTpPathMap {
+			if tpPath != "" {
+				logger.Infow(ctx, "Starting retrieval for TechProfileInstance", log.Fields{
+					"uniID": uniID, "tpID": tpID, "tpPath": tpPath, "device-id": dh.DeviceID,
+				})
+				// Attempt the initial call before entering the retry loop
+				iaTechTpInst, err := dh.GetTechProfileInstanceFromParentAdapter(ctx, uniID, tpPath)
+				if err != nil {
+					logger.Warnw(ctx, "Starting retrieval for TechProfileInstance", log.Fields{
+						"uniID": uniID, "tpID": tpID, "tpPath": tpPath, "device-id": dh.DeviceID,
+					})
+					techProfInstLoadFailed = true
+					break outerLoop
+				}
+				if iaTechTpInst != nil {
+					var tpInst tech_profile.TechProfileInstance
+					switch techTpInst := iaTechTpInst.TechTpInstance.(type) {
+					case *ia.TechProfileDownloadMessage_TpInstance: // supports only GPON, XGPON, XGS-PON
+						tpInst = *techTpInst.TpInstance
+						logger.Debugw(ctx, "received-tp-instance-successfully-after-reboot", log.Fields{
+							"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.DeviceID})
+					default: // do not support epon or other tech
+						logger.Errorw(ctx, "unsupported-tech-profile", log.Fields{
+							"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.DeviceID})
+						techProfInstLoadFailed = true
+						break outerLoop
+					}
+
+					continueWithFlowConfig = true
+					deadline := time.Now().Add(dh.pOpenOnuAc.maxTimeoutInterAdapterComm) //allowed run time to finish before execution
+					dctx, cancel := context.WithDeadline(ctx, deadline)
+
+					dh.pOnuTP.ResetTpProcessingErrorIndication(uniData.PersUniID, tpID)
+					var wg sync.WaitGroup
+					wg.Add(1) // for the 1 go routine to finish
+					go dh.pOnuTP.ConfigureUniTp(log.WithSpanFromContext(dctx, ctx), uniData.PersUniID, uniData.PersTpPathMap[tpID], tpInst, &wg)
+					// Wait for either completion or cancellation
+					dh.waitForCompletion(ctx, cancel, &wg, "TechProfDwldDuringReboot")
+					if tpErr := dh.pOnuTP.GetTpProcessingErrorIndication(uniID, tpID); tpErr != nil {
+						logger.Errorw(ctx, "error-processing-tp", log.Fields{"device-id": dh.DeviceID, "err": tpErr, "tp-path": uniData.PersTpPathMap[tpID]})
+						techProfInstLoadFailed = true
+						continueWithFlowConfig = false
+						break outerLoop
+					}
+				} else {
+					logger.Errorw(ctx, "Tp instance is not valid", log.Fields{"tp-id": tpID, "tpPath": tpPath, "device-id": dh.DeviceID, "err": err})
+					techProfInstLoadFailed = true
+					break outerLoop
+				}
+			} else {
+				logger.Errorw(ctx, "Tp instance is nil", log.Fields{"tp-id": tpID, "tpPath": tpPath,
+					"uni-id": uniID, "device-id": dh.DeviceID})
+				techProfInstLoadFailed = true
+				break outerLoop
+			}
+		}
+		pDevEntry.MutexPersOnuConfig.RLock() //set protection again for loop test on SOnuPersistentData
+		persMutexLock = true
+	}
+	if persMutexLock {
+		pDevEntry.MutexPersOnuConfig.RUnlock()
+	}
+	go dh.deviceRebootStateUpdate(ctx, techProfInstLoadFailed)
+	return continueWithFlowConfig
+}
+
 func (dh *deviceHandler) deleteDevicePersistencyData(ctx context.Context) error {
 	logger.Debugw(ctx, "delete device persistency data", log.Fields{"device-id": dh.DeviceID})
 
@@ -1972,10 +2274,6 @@
 	}
 
 	if !dh.IsReconciling() {
-		if err := dh.StorePersistentData(ctx); err != nil {
-			logger.Warnw(ctx, "store persistent data error - continue as there will be additional write attempts",
-				log.Fields{"device-id": dh.DeviceID, "err": err})
-		}
 		logger.Debugw(ctx, "call DeviceStateUpdate upon create interface", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 			"OperStatus": voltha.OperStatus_ACTIVATING, "device-id": dh.DeviceID})
 
@@ -1986,6 +2284,19 @@
 		}); 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 fmt.Errorf("no valid OnuDevice: %s", dh.DeviceID)
+		}
+		// On onu reboot, we have to perform mib-reset and persist the reboot state for reconciling scenario
+		if dh.GetDeviceTechProfOnReboot() {
+			pDevEntry.MutexPersOnuConfig.Lock()
+			pDevEntry.SOnuPersistentData.PersMibLastDbSync = 0
+			pDevEntry.SOnuPersistentData.PersRebootInProgress = true
+			pDevEntry.MutexPersOnuConfig.Unlock()
+		}
+		// Moving the previous call to write to KV store here, to store the ONU pers data after the update.
+		if err := dh.StorePersistentData(ctx); err != nil {
+			logger.Warnw(ctx, "store persistent data error - continue as there will be additional write attempts",
+				log.Fields{"device-id": dh.DeviceID, "err": err})
 		}
 	} else {
 		logger.Info(ctx, "reconciling - don't notify core about DeviceStateUpdate to ACTIVATING",
@@ -2128,7 +2439,14 @@
 	 *   as further OltAdapter processing may rely on the deviceReason event 'MibUploadDone' as a result of the FSM processing
 	 *   otherwise some processing synchronization would be required - cmp. e.g TechProfile processing
 	 */
+
 	//call MibUploadFSM - transition up to state UlStInSync
+	// Breaking this part of code due to sca complexity
+	err := dh.CheckAndStartMibUploadFsm(ctx, pDevEntry)
+	return err
+}
+
+func (dh *deviceHandler) CheckAndStartMibUploadFsm(ctx context.Context, pDevEntry *mib.OnuDeviceEntry) error {
 	pMibUlFsm := pDevEntry.PMibUploadFsm.PFsm
 	if pMibUlFsm != nil {
 		if pMibUlFsm.Is(mib.UlStDisabled) {
@@ -2179,7 +2497,9 @@
 			return err
 		}
 		//all stored persistent data are not valid anymore (loosing knowledge about the connected ONU)
-		_ = dh.deleteDevicePersistencyData(ctx) //ignore possible errors here and continue, hope is that data is synchronized with new ONU-Up
+		if !dh.GetDeviceTechProfOnReboot() {
+			_ = dh.deleteDevicePersistencyData(ctx) //ignore possible errors here and continue, hope is that data is synchronized with new ONU-Up
+		}
 
 		//deviceEntry stop without omciCC reset here, regarding the OMCI_CC still valid for this ONU
 		//stop the device entry to allow for all system event transfers again
@@ -2514,6 +2834,12 @@
 	dh.EnableUniPortStateUpdate(ctx) //cmp python yield self.enable_ports()
 
 	if !dh.IsReconciling() {
+		// Check if TPs are available post device reboot. If TPs are available start processing them and configure flows
+		if dh.GetDeviceTechProfOnReboot() {
+			if dh.CheckForDeviceTechProf(ctx) {
+				go dh.DeviceFlowConfigOnReboot(ctx)
+			}
+		}
 		logger.Infow(ctx, "UniUnlockStateDone event: Sending OnuUp event", log.Fields{"device-id": dh.DeviceID})
 		raisedTs := time.Now().Unix()
 		go dh.sendOnuOperStateEvent(ctx, voltha.OperStatus_ACTIVE, dh.DeviceID, raisedTs) //cmp python onu_active_event
@@ -2810,6 +3136,7 @@
 	dh.isFlowMonitoringRoutineActive = make([]bool, uniCnt)
 	//chUniVlanConfigReconcilingDone needs to have the capacity of all UniPorts as flow reconcile may run parallel for all of them
 	dh.chUniVlanConfigReconcilingDone = make(chan uint16, uniCnt)
+	dh.chUniVlanConfigOnRebootDone = make(chan uint16, uniCnt)
 	for i := 0; i < int(uniCnt); i++ {
 		dh.flowCbChan[i] = make(chan FlowCb, dh.pOpenOnuAc.config.MaxConcurrentFlowsPerUni)
 		dh.stopFlowMonitoringRoutine[i] = make(chan bool, 1)
@@ -3395,7 +3722,7 @@
 		//  in order to allow for according flow removal lockVlanConfig may only be used with RLock here
 		// Also the error is returned to caller via response channel
 		_ = dh.UniVlanConfigFsmMap[apUniPort.UniID].SetUniFlowParams(ctx, loTpID, loCookieSlice,
-			loMatchVlan, loMatchPcp, loSetVlan, loSetPcp, loInnerCvlan, false, meter, respChan)
+			loMatchVlan, loMatchPcp, loSetVlan, loSetPcp, loInnerCvlan, false, false, meter, respChan)
 		dh.lockVlanConfig.RUnlock()
 		dh.lockVlanAdd.Unlock() //re-admit new Add-flow-processing
 		return
@@ -3403,7 +3730,7 @@
 	dh.lockVlanConfig.RUnlock()
 	dh.lockVlanConfig.Lock() //createVlanFilterFsm should always be a non-blocking operation and requires r+w lock
 	err := dh.createVlanFilterFsm(ctx, apUniPort, loTpID, loCookieSlice,
-		loMatchVlan, loMatchPcp, loSetVlan, loSetPcp, loInnerCvlan, cmn.OmciVlanFilterAddDone, false, meter, respChan)
+		loMatchVlan, loMatchPcp, loSetVlan, loSetPcp, loInnerCvlan, cmn.OmciVlanFilterAddDone, false, false, meter, respChan)
 	dh.lockVlanConfig.Unlock()
 	dh.lockVlanAdd.Unlock() //re-admit new Add-flow-processing
 	if err != nil {
@@ -3468,7 +3795,7 @@
 // if this function is called from possibly concurrent processes it must be mutex-protected from the caller!
 // precondition: dh.lockVlanConfig is locked by the caller!
 func (dh *deviceHandler) createVlanFilterFsm(ctx context.Context, apUniPort *cmn.OnuUniPort, aTpID uint8, aCookieSlice []uint64,
-	aMatchVlan uint16, aMatchPcp uint8, aSetVlan uint16, aSetPcp uint8, innerCvlan uint16, aDevEvent cmn.OnuDeviceEvent, lastFlowToReconcile bool, aMeter *of.OfpMeterConfig, respChan *chan error) error {
+	aMatchVlan uint16, aMatchPcp uint8, aSetVlan uint16, aSetPcp uint8, innerCvlan uint16, aDevEvent cmn.OnuDeviceEvent, lastFlowToReconcile bool, lastFlowToConfOnReboot bool, aMeter *of.OfpMeterConfig, respChan *chan error) error {
 	chVlanFilterFsm := make(chan cmn.Message, 2)
 
 	pDevEntry := dh.GetOnuDeviceEntry(ctx, true)
@@ -3484,7 +3811,7 @@
 
 	pVlanFilterFsm := avcfg.NewUniVlanConfigFsm(ctx, dh, pDevEntry, pDevEntry.PDevOmciCC, apUniPort, dh.pOnuTP,
 		pDevEntry.GetOnuDB(), aTpID, aDevEvent, "UniVlanConfigFsm", chVlanFilterFsm,
-		dh.pOpenOnuAc.AcceptIncrementalEvto, aCookieSlice, aMatchVlan, aMatchPcp, aSetVlan, aSetPcp, innerCvlan, lastFlowToReconcile, aMeter, respChan)
+		dh.pOpenOnuAc.AcceptIncrementalEvto, aCookieSlice, aMatchVlan, aMatchPcp, aSetVlan, aSetPcp, innerCvlan, lastFlowToReconcile, lastFlowToConfOnReboot, aMeter, respChan)
 	if pVlanFilterFsm != nil {
 		//dh.lockVlanConfig is locked (by caller) throughout the state transition to 'starting'
 		// to prevent unintended (ignored) events to be sent there (from parallel processing)
@@ -3718,6 +4045,30 @@
 	return dh.startWritingOnuDataToKvStore(ctx, pDevEntry)
 }
 
+func (dh *deviceHandler) UpdateRebootPersData(ctx context.Context, flag bool) {
+	pDevEntry := dh.GetOnuDeviceEntry(ctx, false)
+	if pDevEntry != nil {
+		pDevEntry.MutexPersOnuConfig.Lock()
+		pDevEntry.SOnuPersistentData.PersRebootInProgress = flag
+		pDevEntry.MutexPersOnuConfig.Unlock()
+	} else {
+		logger.Errorw(ctx, "No valid OnuDevice", log.Fields{"device-id": dh.DeviceID})
+	}
+}
+
+func (dh *deviceHandler) GetPersRebootFlag(ctx context.Context) bool {
+	rebootFlag := false
+	pDevEntry := dh.GetOnuDeviceEntry(ctx, false)
+	if pDevEntry != nil {
+		pDevEntry.MutexPersOnuConfig.RLock()
+		rebootFlag = pDevEntry.SOnuPersistentData.PersRebootInProgress
+		pDevEntry.MutexPersOnuConfig.RUnlock()
+	} else {
+		logger.Errorw(ctx, "No valid OnuDevice", log.Fields{"device-id": dh.DeviceID})
+	}
+	return rebootFlag
+}
+
 // getUniPortMEEntityID takes uniPortNo as the input and returns the Entity ID corresponding to this UNI-G ME Instance
 // nolint: unused
 func (dh *deviceHandler) getUniPortMEEntityID(uniPortNo uint32) (uint16, error) {
@@ -4359,6 +4710,29 @@
 	}
 }
 
+func (dh *deviceHandler) deviceRebootStateUpdate(ctx context.Context, techProfInstLoadFailed bool) {
+	if techProfInstLoadFailed {
+		if err := dh.ReasonUpdate(ctx, cmn.DrTechProfileConfigDownloadFailed, true); err != nil {
+			logger.Errorw(ctx, "unable to update device reason to core",
+				log.Fields{"device-id": dh.DeviceID, "Err": err})
+		}
+		context := make(map[string]string)
+		context["device-id"] = dh.DeviceID
+		context["onu-serial-number"] = dh.device.SerialNumber
+		context["parent-id"] = dh.parentID
+
+		// Send event on flow configuration failure so that corrective action can be triggered from NB
+		deviceEvent := &voltha.DeviceEvent{
+			ResourceId:      dh.DeviceID,
+			DeviceEventName: cmn.OnuFlowConfigFailed,
+			Description:     cmn.OnuFlowConfigFailedDesc,
+			Context:         context,
+		}
+		logger.Debugw(ctx, "send device event", log.Fields{"deviceEvent": deviceEvent, "device-id": dh.DeviceID})
+		_ = dh.EventProxy.SendDeviceEvent(ctx, deviceEvent, voltha.EventCategory_EQUIPMENT, voltha.EventSubCategory_ONU, time.Now().Unix())
+	}
+}
+
 /*
 Helper functions to communicate with Core
 */
@@ -4745,6 +5119,10 @@
 	return dh.pOpenOnuAc.skipOnuConfig
 }
 
+func (dh *deviceHandler) GetDeviceTechProfOnReboot() bool {
+	return dh.pOpenOnuAc.CheckDeviceTechProfOnReboot
+}
+
 // InitPmConfigs - TODO: add comment
 func (dh *deviceHandler) InitPmConfigs() {
 	dh.pmConfigs = &voltha.PmConfigs{}