First Commit of Voltha-Go-Controller from Radisys

Change-Id: I8e2e908e7ab09a4fe3d86849da18b6d69dcf4ab0
diff --git a/internal/pkg/application/application.go b/internal/pkg/application/application.go
new file mode 100644
index 0000000..8b1e763
--- /dev/null
+++ b/internal/pkg/application/application.go
@@ -0,0 +1,2057 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"context"
+	"encoding/hex"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"net"
+	"strconv"
+	"strings"
+	"sync"
+	"time"
+
+	"github.com/google/gopacket"
+	"github.com/google/gopacket/layers"
+
+	"voltha-go-controller/internal/pkg/controller"
+	cntlr "voltha-go-controller/internal/pkg/controller"
+	"voltha-go-controller/database"
+	"voltha-go-controller/internal/pkg/intf"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/tasks"
+	"voltha-go-controller/internal/pkg/util"
+	errorCodes "voltha-go-controller/internal/pkg/errorcodes"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+var ctx = context.TODO()
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
+
+const (
+	// TODO - Need to identify a right place for this
+
+	// PriorityNone constant.
+	PriorityNone uint8 = 8
+	// AnyVlan constant.
+	AnyVlan uint16 = 0xFFFF
+)
+
+// List of Mac Learning Type
+const (
+        MacLearningNone MacLearningType = iota
+        Learn
+        ReLearn
+)
+
+// MacLearningType represents Mac Learning Type
+type MacLearningType int
+
+var (
+	tickCount         uint16
+	vgcRebooted       bool
+	isUpgradeComplete bool
+)
+
+var db database.DBIntf
+
+// PacketHandlers : packet handler for different protocols
+var PacketHandlers map[string]CallBack
+
+// CallBack : registered call back function for different protocol packets
+type CallBack func(device string, port string, pkt gopacket.Packet)
+
+const (
+	// ARP packet
+	ARP string = "ARP"
+	// DHCPv4 packet
+	DHCPv4 string = "DHCPv4"
+	// DHCPv6 packet
+	DHCPv6 string = "DHCPv6"
+	// IGMP packet
+	IGMP string = "IGMP"
+	// PPPOE packet
+	PPPOE string = "PPPOE"
+	// US packet side
+	US string = "US"
+	// DS packet side
+	DS string = "DS"
+	// NNI port name
+	NNI string = "nni"
+)
+
+// RegisterPacketHandler : API to register callback function for every protocol
+func RegisterPacketHandler(protocol string, callback CallBack) {
+	if PacketHandlers == nil {
+		PacketHandlers = make(map[string]CallBack)
+	}
+	PacketHandlers[protocol] = callback
+}
+
+// ---------------------------------------------------------------------
+// VOLT Ports
+// ---------------------------------------------------------------------
+// VOLT Ports are ports associated with VOLT devices. Each port is classified into
+// Access/NNI. Each port is identified by Name (Identity known to the NB) and
+// Id (Identity used on the SB). Both identities are presented when a port is
+// discovered in the SB.
+
+// VoltPortType type for Port Type
+type VoltPortType uint8
+
+const (
+	// VoltPortTypeAccess constant.
+	VoltPortTypeAccess VoltPortType = 0
+	// VoltPortTypeNni constant.
+	VoltPortTypeNni VoltPortType = 1
+)
+
+// PortState type for Port State.
+type PortState uint8
+
+const (
+	// PortStateDown constant.
+	PortStateDown PortState = 0
+	// PortStateUp constant.
+	PortStateUp PortState = 1
+)
+
+// VoltPort structure that is used to store the ports. The name is the
+// the main identity used by the application. The SB and NB both present name
+// as the identity. The SB is abstracted by VPAgent and the VPAgent transacts
+// using name as identity
+type VoltPort struct {
+	ID                       uint32
+	Name                     string
+	Device                   string
+	PonPort                  uint32
+	Type                     VoltPortType
+	State                    PortState
+	ActiveChannels           uint32
+	ChannelPerSubAlarmRaised bool
+}
+
+// NewVoltPort : Constructor for the port.
+func NewVoltPort(device string, name string, id uint32) *VoltPort {
+	var vp VoltPort
+	vp.Device = device
+	vp.Name = name
+	vp.ID = id
+	if util.IsNniPort(id) {
+		vp.Type = VoltPortTypeNni
+	} else {
+		vp.PonPort = GetPonPortIDFromUNIPort(id)
+	}
+	vp.State = PortStateDown
+	vp.ChannelPerSubAlarmRaised = false
+	return &vp
+}
+
+// SetPortID : The ID is used when constructing flows as the flows require ID.
+func (vp *VoltPort) SetPortID(id uint32) {
+	vp.ID = id
+	if util.IsNniPort(id) {
+		vp.Type = VoltPortTypeNni
+	}
+}
+
+// ---------------------------------------------------------------------
+// VOLT Device
+// ---------------------------------------------------------------------
+//
+// VoltDevice is an OLT which contains ports of type access and NNI. Each OLT
+// can only have one NNI port in the current release. The NNI port always uses
+// identity 65536 and all the access ports use identities less than 65535. The
+// identification of NNI is done by comparing the port identity with 65535
+
+// VoltDevice fields :
+// Name:         This is the name presented by the device/VOLTHA. This doesn't
+//               have any relation to the physical device
+// SerialNum:    This is the serial number of the device and can be used to
+//               correlate the devices
+// NniPort:      The identity of the NNI port
+// Ports:        List of all ports added to the device
+type VoltDevice struct {
+	Name                         string
+	SerialNum                    string
+	State                        controller.DeviceState
+	SouthBoundID                 string
+	NniPort                      string
+	Ports                        sync.Map
+	VlanPortStatus               sync.Map
+	VpvsBySvlan                  *util.ConcurrentMap // map[svlan]map[vnet_port]*VoltPortVnet
+	IgmpDsFlowAppliedForMvlan    map[uint16]bool
+	ConfiguredVlanForDeviceFlows *util.ConcurrentMap //map[string]map[string]bool
+	icmpv6GroupAdded             bool
+	ActiveChannelsPerPon         sync.Map            // [PonPortID]*PonPortCfg
+	ActiveChannelCountLock       sync.Mutex          // This lock is used to update ActiveIGMPChannels
+	PonPortList                  sync.Map            // [PonPortID]map[string]string
+	FlowAddEventMap              *util.ConcurrentMap //map[string]*FlowEvent
+	FlowDelEventMap              *util.ConcurrentMap //map[string]*FlowEvent
+	MigratingServices            *util.ConcurrentMap //<vnetID,<RequestID, MigrateServicesRequest>>
+	GlobalDhcpFlowAdded          bool
+}
+
+// NewVoltDevice : Constructor for the device
+func NewVoltDevice(name string, slno, southBoundID string) *VoltDevice {
+	var d VoltDevice
+	d.Name = name
+	d.SouthBoundID = southBoundID
+	d.State = controller.DeviceStateDOWN
+	d.NniPort = ""
+	d.SouthBoundID = southBoundID
+	d.SerialNum = slno
+	d.icmpv6GroupAdded = false
+	d.IgmpDsFlowAppliedForMvlan = make(map[uint16]bool)
+	d.ConfiguredVlanForDeviceFlows = util.NewConcurrentMap()
+	d.MigratingServices = util.NewConcurrentMap()
+	d.VpvsBySvlan = util.NewConcurrentMap()
+	d.FlowAddEventMap = util.NewConcurrentMap()
+	d.FlowDelEventMap = util.NewConcurrentMap()
+	d.GlobalDhcpFlowAdded = false
+	return &d
+}
+
+//GetAssociatedVpvsForDevice - return the associated VPVs for given device & svlan
+func (va *VoltApplication) GetAssociatedVpvsForDevice(device string, svlan of.VlanType) *util.ConcurrentMap {
+	if d := va.GetDevice(device); d != nil {
+		return d.GetAssociatedVpvs(svlan)
+	}
+	return nil
+}
+
+//AssociateVpvsToDevice - updates the associated VPVs for given device & svlan
+func (va *VoltApplication) AssociateVpvsToDevice(device string, vpv *VoltPortVnet) {
+	if d := va.GetDevice(device); d != nil {
+
+		vpvMap := d.GetAssociatedVpvs(vpv.SVlan)
+		vpvMap.Set(vpv, true)
+		d.VpvsBySvlan.Set(vpv.SVlan, vpvMap)
+		logger.Infow(ctx, "VPVMap: SET", log.Fields{"Map": vpvMap.Length()})
+		return
+	}
+	logger.Errorw(ctx, "Set VPVMap failed: Device Not Found", log.Fields{"Svlan": vpv.SVlan, "Device": device})
+}
+
+//DisassociateVpvsFromDevice - disassociated VPVs from given device & svlan
+func (va *VoltApplication) DisassociateVpvsFromDevice(device string, vpv *VoltPortVnet) {
+	if d := va.GetDevice(device); d != nil {
+		vpvMap := d.GetAssociatedVpvs(vpv.SVlan)
+		vpvMap.Remove(vpv)
+		d.VpvsBySvlan.Set(vpv.SVlan, vpvMap)
+		logger.Infow(ctx, "VPVMap: Remove", log.Fields{"Map": vpvMap.Length()})
+		return
+	}
+	logger.Errorw(ctx, "Remove VPVMap failed: Device Not Found", log.Fields{"Svlan": vpv.SVlan, "Device": device})
+}
+
+//GetAssociatedVpvs - returns the associated VPVs for the given Svlan
+func (d *VoltDevice) GetAssociatedVpvs(svlan of.VlanType) *util.ConcurrentMap {
+
+	var vpvMap *util.ConcurrentMap
+	var mapIntf interface{}
+	var ok bool
+
+	if mapIntf, ok = d.VpvsBySvlan.Get(svlan); ok {
+		vpvMap = mapIntf.(*util.ConcurrentMap)
+	} else {
+		vpvMap = util.NewConcurrentMap()
+	}
+	logger.Infow(ctx, "VPVMap: GET", log.Fields{"Map": vpvMap.Length()})
+	return vpvMap
+}
+
+// AddPort add port to the device.
+func (d *VoltDevice) AddPort(port string, id uint32) *VoltPort {
+	addPonPortFromUniPort := func(vPort *VoltPort) {
+		if vPort.Type == VoltPortTypeAccess {
+			ponPortID := GetPonPortIDFromUNIPort(vPort.ID)
+
+			if ponPortUniList, ok := d.PonPortList.Load(ponPortID); !ok {
+				uniList := make(map[string]uint32)
+				uniList[port] = vPort.ID
+				d.PonPortList.Store(ponPortID, uniList)
+			} else {
+				ponPortUniList.(map[string]uint32)[port] = vPort.ID
+				d.PonPortList.Store(ponPortID, ponPortUniList)
+			}
+		}
+	}
+	va := GetApplication()
+	if pIntf, ok := d.Ports.Load(port); ok {
+		voltPort := pIntf.(*VoltPort)
+		addPonPortFromUniPort(voltPort)
+		va.AggActiveChannelsCountPerSub(d.Name, port, voltPort)
+		d.Ports.Store(port, voltPort)
+		return voltPort
+	}
+	p := NewVoltPort(d.Name, port, id)
+	va.AggActiveChannelsCountPerSub(d.Name, port, p)
+	d.Ports.Store(port, p)
+	if util.IsNniPort(id) {
+		d.NniPort = port
+	}
+	addPonPortFromUniPort(p)
+	return p
+}
+
+// GetPort to get port information from the device.
+func (d *VoltDevice) GetPort(port string) *VoltPort {
+	if pIntf, ok := d.Ports.Load(port); ok {
+		return pIntf.(*VoltPort)
+	}
+	return nil
+}
+
+// DelPort to delete port from the device
+func (d *VoltDevice) DelPort(port string) {
+	if _, ok := d.Ports.Load(port); ok {
+		d.Ports.Delete(port)
+	} else {
+		logger.Warnw(ctx, "Port doesn't exist", log.Fields{"Device": d.Name, "Port": port})
+	}
+}
+
+// pushFlowsForUnis to send port-up-indication for uni ports.
+func (d *VoltDevice) pushFlowsForUnis() {
+
+	logger.Info(ctx, "NNI Discovered, Sending Port UP Ind for UNIs")
+	d.Ports.Range(func(key, value interface{}) bool {
+		port := key.(string)
+		vp := value.(*VoltPort)
+
+		logger.Infow(ctx, "NNI Discovered. Sending Port UP Ind for UNI", log.Fields{"Port" : port})
+		//Ignore if UNI port is not UP
+		if vp.State != PortStateUp {
+			return true
+		}
+
+		//Obtain all VPVs associated with the port
+		vnets, ok := GetApplication().VnetsByPort.Load(port)
+		if !ok {
+			return true
+		}
+
+		for _, vpv := range vnets.([]*VoltPortVnet) {
+			vpv.VpvLock.Lock()
+			vpv.PortUpInd(d, port)
+			vpv.VpvLock.Unlock()
+
+		}
+		return true
+	})
+}
+
+// ----------------------------------------------------------
+// VOLT Application - hosts all other objects
+// ----------------------------------------------------------
+//
+// The VOLT application is a singleton implementation where
+// there is just one instance in the system and is the gateway
+// to all other components within the controller
+// The declaration of the singleton object
+var vapplication *VoltApplication
+
+// VoltApplication fields :
+// ServiceByName - Stores the services by the name as key
+//                 A record of NB configuration.
+// VnetsByPort   - Stores the VNETs by the ports configured
+//                 from NB. A record of NB configuration.
+// VnetsByTag    - Stores the VNETs by the VLANS configured
+//                 from NB. A record of NB configuration.
+// VnetsByName   - Stores the VNETs by the name configured
+//                 from NB. A record of NB configuration.
+// DevicesDisc   - Stores the devices discovered from SB.
+//                 Should be updated only by events from SB
+// PortsDisc     - Stores the ports discovered from SB.
+//                 Should be updated only by events from SB
+type VoltApplication struct {
+	ServiceByName       sync.Map // [serName]*VoltService
+	VnetsByPort         sync.Map // [portName][]*VoltPortVnet
+	VnetsByTag          sync.Map // [svlan-cvlan-uvlan]*VoltVnet
+	VnetsByName         sync.Map // [vnetName]*VoltVnet
+	VnetsBySvlan        *util.ConcurrentMap
+	DevicesDisc         sync.Map
+	PortsDisc           sync.Map
+	IgmpGroups          sync.Map // [grpKey]*IgmpGroup
+	IgmpGroupIds        []*IgmpGroup
+	MvlanProfilesByTag  sync.Map
+	MvlanProfilesByName sync.Map
+	Icmpv6Receivers     sync.Map
+	MeterMgr
+	IgmpTasks           tasks.Tasks
+	IndicationsTasks    tasks.Tasks
+	MulticastAlarmTasks tasks.Tasks
+	portLock            sync.Mutex
+	DataMigrationInfo   DataMigration
+	DeviceCounters      sync.Map //[logicalDeviceId]*DeviceCounters
+	ServiceCounters     sync.Map //[serviceName]*ServiceCounters
+	NbDevice            sync.Map // [OLTSouthBoundID]*NbDevice
+	IgmpKPIsTasks       tasks.Tasks
+	pppoeTasks          tasks.Tasks
+	IgmpProfilesByName  sync.Map
+	OltIgmpInfoBySerial sync.Map
+	McastConfigMap      sync.Map //[OltSerialNo_MvlanProfileID]*McastConfig
+	// MacAddress-Port MAP to avoid swap of mac accross ports.
+	macPortLock sync.RWMutex
+	macPortMap  map[string]string
+
+	IgmpPendingPool map[string]map[*IgmpGroup]bool //[grpkey, map[groupObj]bool]  //mvlan_grpName/IP
+	PendingPoolLock sync.RWMutex
+
+	VnetsToDelete             map[string]bool
+	ServicesToDelete          map[string]bool
+	VoltPortVnetsToDelete     map[*VoltPortVnet]bool
+	PortAlarmProfileCache     map[string]map[string]int // [portAlarmID][ThresholdLevelString]ThresholdLevel
+	vendorID                  string
+}
+
+// PonPortCfg contains NB port config and activeIGMPChannels count
+type PonPortCfg struct {
+	PortID             uint32
+	MaxActiveChannels  uint32
+	ActiveIGMPChannels uint32
+	EnableMulticastKPI bool
+	PortAlarmProfileID string
+}
+
+// NbDevice OLT Device info
+type NbDevice struct {
+	SouthBoundID string
+	PonPorts     sync.Map // [PortID]*PonPortCfg
+}
+
+// RestoreNbDeviceFromDb restores the NB Device in case of VGC pod restart.
+func (va *VoltApplication) RestoreNbDeviceFromDb(deviceID string) *NbDevice {
+
+	nbDevice := NewNbDevice()
+	nbDevice.SouthBoundID = deviceID
+
+	nbPorts, _ := db.GetAllNbPorts(deviceID)
+
+	for key, p := range nbPorts {
+		b, ok := p.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var port PonPortCfg
+		err := json.Unmarshal(b, &port)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of PonPortCfg failed")
+			continue
+		}
+		logger.Debugw(ctx, "Port recovered", log.Fields{"port": port})
+		ponPortID, _ := strconv.Atoi(key)
+		nbDevice.PonPorts.Store(uint32(ponPortID), &port)
+	}
+	va.NbDevice.Store(deviceID, nbDevice)
+	return nbDevice
+}
+
+// NewNbDevice Constructor for NbDevice
+func NewNbDevice() *NbDevice {
+	var nbDevice NbDevice
+	return &nbDevice
+}
+
+// WriteToDb writes nb device port config to kv store
+func (nbd *NbDevice) WriteToDb(portID uint32, ponPort *PonPortCfg) {
+	b, err := json.Marshal(ponPort)
+	if err != nil {
+		logger.Errorw(ctx, "PonPortConfig-marshal-failed", log.Fields{"err": err})
+		return
+	}
+	db.PutNbDevicePort(nbd.SouthBoundID, portID, string(b))
+}
+
+// AddPortToNbDevice Adds pon port to NB Device and DB
+func (nbd *NbDevice) AddPortToNbDevice(portID, allowedChannels uint32,
+	enableMulticastKPI bool, portAlarmProfileID string) *PonPortCfg {
+
+	ponPort := &PonPortCfg{
+		PortID:             portID,
+		MaxActiveChannels:  allowedChannels,
+		EnableMulticastKPI: enableMulticastKPI,
+		PortAlarmProfileID: portAlarmProfileID,
+	}
+	nbd.PonPorts.Store(portID, ponPort)
+	nbd.WriteToDb(portID, ponPort)
+	return ponPort
+}
+
+// UpdatePortToNbDevice Adds pon port to NB Device and DB
+func (nbd *NbDevice) UpdatePortToNbDevice(portID, allowedChannels uint32, enableMulticastKPI bool, portAlarmProfileID string) *PonPortCfg {
+
+	p, exists := nbd.PonPorts.Load(portID)
+	if !exists {
+		logger.Errorw(ctx, "PON port not exists in nb-device", log.Fields{"portID": portID})
+		return nil
+	}
+	port := p.(*PonPortCfg)
+	if allowedChannels != 0 {
+		port.MaxActiveChannels = allowedChannels
+		port.EnableMulticastKPI = enableMulticastKPI
+		port.PortAlarmProfileID = portAlarmProfileID
+	}
+
+	nbd.PonPorts.Store(portID, port)
+	nbd.WriteToDb(portID, port)
+	return port
+}
+
+// DeletePortFromNbDevice Deletes pon port from NB Device and DB
+func (nbd *NbDevice) DeletePortFromNbDevice(portID uint32) {
+
+	if _, ok := nbd.PonPorts.Load(portID); ok {
+		nbd.PonPorts.Delete(portID)
+	}
+	db.DelNbDevicePort(nbd.SouthBoundID, portID)
+}
+
+// GetApplication : Interface to access the singleton object
+func GetApplication() *VoltApplication {
+	if vapplication == nil {
+		vapplication = newVoltApplication()
+	}
+	return vapplication
+}
+
+// newVoltApplication : Constructor for the singleton object. Hence this is not
+// an exported function
+func newVoltApplication() *VoltApplication {
+	var va VoltApplication
+	va.IgmpTasks.Initialize(context.TODO())
+	va.MulticastAlarmTasks.Initialize(context.TODO())
+	va.IgmpKPIsTasks.Initialize(context.TODO())
+	va.pppoeTasks.Initialize(context.TODO())
+	va.storeIgmpProfileMap(DefaultIgmpProfID, newDefaultIgmpProfile())
+	va.MeterMgr.Init()
+	va.AddIgmpGroups(5000)
+	va.macPortMap = make(map[string]string)
+	va.IgmpPendingPool = make(map[string]map[*IgmpGroup]bool)
+	va.VnetsBySvlan = util.NewConcurrentMap()
+	va.VnetsToDelete = make(map[string]bool)
+	va.ServicesToDelete = make(map[string]bool)
+	va.VoltPortVnetsToDelete = make(map[*VoltPortVnet]bool)
+	go va.Start(TimerCfg{tick: 100 * time.Millisecond}, tickTimer)
+	go va.Start(TimerCfg{tick: time.Duration(GroupExpiryTime) * time.Minute}, pendingPoolTimer)
+	InitEventFuncMapper()
+	db = database.GetDatabase()
+	return &va
+}
+
+//GetFlowEventRegister - returs the register based on flow mod type
+func (d *VoltDevice) GetFlowEventRegister(flowModType of.Command) (*util.ConcurrentMap, error) {
+
+	switch flowModType {
+	case of.CommandDel:
+		return d.FlowDelEventMap, nil
+	case of.CommandAdd:
+		return d.FlowAddEventMap, nil
+	default:
+		logger.Error(ctx, "Unknown Flow Mod received")
+	}
+	return util.NewConcurrentMap(), errors.New("Unknown Flow Mod")
+}
+
+// RegisterFlowAddEvent to register a flow event.
+func (d *VoltDevice) RegisterFlowAddEvent(cookie string, event *FlowEvent) {
+	logger.Debugw(ctx, "Registered Flow Add Event", log.Fields{"Cookie": cookie, "Event": event})
+	d.FlowAddEventMap.MapLock.Lock()
+	defer d.FlowAddEventMap.MapLock.Unlock()
+	d.FlowAddEventMap.Set(cookie, event)
+}
+
+// RegisterFlowDelEvent to register a flow event.
+func (d *VoltDevice) RegisterFlowDelEvent(cookie string, event *FlowEvent) {
+	logger.Debugw(ctx, "Registered Flow Del Event", log.Fields{"Cookie": cookie, "Event": event})
+	d.FlowDelEventMap.MapLock.Lock()
+	defer d.FlowDelEventMap.MapLock.Unlock()
+	d.FlowDelEventMap.Set(cookie, event)
+}
+
+// UnRegisterFlowEvent to unregister a flow event.
+func (d *VoltDevice) UnRegisterFlowEvent(cookie string, flowModType of.Command) {
+	logger.Debugw(ctx, "UnRegistered Flow Add Event", log.Fields{"Cookie": cookie, "Type": flowModType})
+	flowEventMap, err := d.GetFlowEventRegister(flowModType)
+	if err != nil {
+		logger.Debugw(ctx, "Flow event map does not exists", log.Fields{"flowMod": flowModType, "Error": err})
+		return
+	}
+	flowEventMap.MapLock.Lock()
+	defer flowEventMap.MapLock.Unlock()
+	flowEventMap.Remove(cookie)
+}
+
+// AddIgmpGroups to add Igmp groups.
+func (va *VoltApplication) AddIgmpGroups(numOfGroups uint32) {
+	//TODO: Temp change to resolve group id issue in pOLT
+	//for i := 1; uint32(i) <= numOfGroups; i++ {
+	for i := 2; uint32(i) <= (numOfGroups + 1); i++ {
+		ig := IgmpGroup{}
+		ig.GroupID = uint32(i)
+		va.IgmpGroupIds = append(va.IgmpGroupIds, &ig)
+	}
+}
+
+// GetAvailIgmpGroupID to get id of available igmp group.
+func (va *VoltApplication) GetAvailIgmpGroupID() *IgmpGroup {
+	var ig *IgmpGroup
+	if len(va.IgmpGroupIds) > 0 {
+		ig, va.IgmpGroupIds = va.IgmpGroupIds[0], va.IgmpGroupIds[1:]
+		return ig
+	}
+	return nil
+}
+
+// GetIgmpGroupID to get id of igmp group.
+func (va *VoltApplication) GetIgmpGroupID(gid uint32) (*IgmpGroup, error) {
+	for id, ig := range va.IgmpGroupIds {
+		if ig.GroupID == gid {
+			va.IgmpGroupIds = append(va.IgmpGroupIds[0:id], va.IgmpGroupIds[id+1:]...)
+			return ig, nil
+		}
+	}
+	return nil, errors.New("Group Id Missing")
+}
+
+// PutIgmpGroupID to add id of igmp group.
+func (va *VoltApplication) PutIgmpGroupID(ig *IgmpGroup) {
+	va.IgmpGroupIds = append([]*IgmpGroup{ig}, va.IgmpGroupIds[0:]...)
+}
+
+//RestoreUpgradeStatus - gets upgrade/migration status from DB and updates local flags
+func (va *VoltApplication) RestoreUpgradeStatus() {
+	Migrate := new(DataMigration)
+	if err := GetMigrationInfo(Migrate); err == nil {
+		if Migrate.Status == MigrationInProgress {
+			isUpgradeComplete = false
+			return
+		}
+	}
+	isUpgradeComplete = true
+
+	logger.Infow(ctx, "Upgrade Status Restored", log.Fields{"Upgrade Completed": isUpgradeComplete})
+}
+
+// ReadAllFromDb : If we are restarted, learn from the database the current execution
+// stage
+func (va *VoltApplication) ReadAllFromDb() {
+	logger.Info(ctx, "Reading the meters from DB")
+	va.RestoreMetersFromDb()
+	logger.Info(ctx, "Reading the VNETs from DB")
+	va.RestoreVnetsFromDb()
+	logger.Info(ctx, "Reading the VPVs from DB")
+	va.RestoreVpvsFromDb()
+	logger.Info(ctx, "Reading the Services from DB")
+	va.RestoreSvcsFromDb()
+	logger.Info(ctx, "Reading the MVLANs from DB")
+	va.RestoreMvlansFromDb()
+	logger.Info(ctx, "Reading the IGMP profiles from DB")
+	va.RestoreIGMPProfilesFromDb()
+	logger.Info(ctx, "Reading the Mcast configs from DB")
+	va.RestoreMcastConfigsFromDb()
+	logger.Info(ctx, "Reading the IGMP groups for DB")
+	va.RestoreIgmpGroupsFromDb()
+	logger.Info(ctx, "Reading Upgrade status from DB")
+	va.RestoreUpgradeStatus()
+	logger.Info(ctx, "Reconciled from DB")
+}
+
+// InitStaticConfig to initialise static config.
+func (va *VoltApplication) InitStaticConfig() {
+	va.InitIgmpSrcMac()
+}
+
+// SetVendorID to set vendor id
+func (va *VoltApplication) SetVendorID(vendorID string) {
+	va.vendorID = vendorID
+}
+
+// GetVendorID to get vendor id
+func (va *VoltApplication) GetVendorID() string {
+	return va.vendorID
+}
+
+// SetRebootFlag to set reboot flag
+func (va *VoltApplication) SetRebootFlag(flag bool) {
+	vgcRebooted = flag
+}
+
+// GetUpgradeFlag to get reboot status
+func (va *VoltApplication) GetUpgradeFlag() bool {
+	return isUpgradeComplete
+}
+
+// SetUpgradeFlag to set reboot status
+func (va *VoltApplication) SetUpgradeFlag(flag bool) {
+	isUpgradeComplete = flag
+}
+
+// ------------------------------------------------------------
+// Device related functions
+
+// AddDevice : Add a device and typically the device stores the NNI port on the device
+// The NNI port is used when the packets are emitted towards the network.
+// The outport is selected as the NNI port of the device. Today, we support
+// a single NNI port per OLT. This is true whether the network uses any
+// protection mechanism (LAG, ERPS, etc.). The aggregate of the such protection
+// is represented by a single NNI port
+func (va *VoltApplication) AddDevice(device string, slno, southBoundID string) {
+	logger.Warnw(ctx, "Received Device Ind: Add", log.Fields{"Device": device, "SrNo": slno})
+	if _, ok := va.DevicesDisc.Load(device); ok {
+		logger.Warnw(ctx, "Device Exists", log.Fields{"Device": device})
+	}
+	d := NewVoltDevice(device, slno, southBoundID)
+
+	addPort := func(key, value interface{}) bool {
+		portID := key.(uint32)
+		port := value.(*PonPortCfg)
+		va.AggActiveChannelsCountForPonPort(device, portID, port)
+		d.ActiveChannelsPerPon.Store(portID, port)
+		return true
+	}
+	if nbDevice, exists := va.NbDevice.Load(southBoundID); exists {
+		// Pon Ports added before OLT activate.
+		nbDevice.(*NbDevice).PonPorts.Range(addPort)
+	} else {
+		// Check if NbPort exists in DB. VGC restart case.
+		nbd := va.RestoreNbDeviceFromDb(southBoundID)
+		nbd.PonPorts.Range(addPort)
+	}
+	va.DevicesDisc.Store(device, d)
+}
+
+// GetDevice to get a device.
+func (va *VoltApplication) GetDevice(device string) *VoltDevice {
+	if d, ok := va.DevicesDisc.Load(device); ok {
+		return d.(*VoltDevice)
+	}
+	return nil
+}
+
+// DelDevice to delete a device.
+func (va *VoltApplication) DelDevice(device string) {
+	logger.Warnw(ctx, "Received Device Ind: Delete", log.Fields{"Device": device})
+	if vdIntf, ok := va.DevicesDisc.Load(device); ok {
+		vd := vdIntf.(*VoltDevice)
+		va.DevicesDisc.Delete(device)
+		_ = db.DelAllRoutesForDevice(device)
+		va.HandleFlowClearFlag(device, vd.SerialNum, vd.SouthBoundID)
+		_ = db.DelAllGroup(device)
+		_ = db.DelAllMeter(device)
+		_ = db.DelAllPorts(device)
+		logger.Debugw(ctx, "Device deleted", log.Fields{"Device": device})
+	} else {
+		logger.Warnw(ctx, "Device Doesn't Exist", log.Fields{"Device": device})
+	}
+}
+
+// GetDeviceBySerialNo to get a device by serial number.
+// TODO - Transform this into a MAP instead
+func (va *VoltApplication) GetDeviceBySerialNo(slno string) *VoltDevice {
+	var device *VoltDevice
+	getserial := func(key interface{}, value interface{}) bool {
+		device = value.(*VoltDevice)
+		return device.SerialNum != slno
+	}
+	va.DevicesDisc.Range(getserial)
+	return device
+}
+
+// PortAddInd : This is a PORT add indication coming from the VPAgent, which is essentially
+// a request coming from VOLTHA. The device and identity of the port is provided
+// in this request. Add them to the application for further use
+func (va *VoltApplication) PortAddInd(device string, id uint32, portName string) {
+	logger.Infow(ctx, "Received Port Ind: Add", log.Fields{"Device": device, "Port": portName})
+	va.portLock.Lock()
+	if d := va.GetDevice(device); d != nil {
+		p := d.AddPort(portName, id)
+		va.PortsDisc.Store(portName, p)
+		va.portLock.Unlock()
+		nni, _ := va.GetNniPort(device)
+		if nni == portName {
+			d.pushFlowsForUnis()
+		}
+	} else {
+		va.portLock.Unlock()
+		logger.Warnw(ctx, "Device Not Found - Dropping Port Ind: Add", log.Fields{"Device": device, "Port": portName})
+	}
+}
+
+// PortDelInd : Only the NNI ports are recorded in the device for now. When port delete
+// arrives, only the NNI ports need adjustments.
+func (va *VoltApplication) PortDelInd(device string, port string) {
+	logger.Infow(ctx, "Received Port Ind: Delete", log.Fields{"Device": device, "Port": port})
+	if d := va.GetDevice(device); d != nil {
+		p := d.GetPort(port)
+		if p != nil && p.State == PortStateUp {
+			logger.Infow(ctx, "Port state is UP. Trigerring Port Down Ind before deleting", log.Fields{"Port": p})
+			va.PortDownInd(device, port)
+		}
+		va.portLock.Lock()
+		defer va.portLock.Unlock()
+		d.DelPort(port)
+		if _, ok := va.PortsDisc.Load(port); ok {
+			va.PortsDisc.Delete(port)
+		}
+	} else {
+		logger.Warnw(ctx, "Device Not Found - Dropping Port Ind: Delete", log.Fields{"Device": device, "Port": port})
+	}
+}
+
+//PortUpdateInd Updates port Id incase of ONU movement
+func (va *VoltApplication) PortUpdateInd(device string, portName string, id uint32) {
+	logger.Infow(ctx, "Received Port Ind: Update", log.Fields{"Device": device, "Port": portName})
+	va.portLock.Lock()
+	defer va.portLock.Unlock()
+	if d := va.GetDevice(device); d != nil {
+		vp := d.GetPort(portName)
+		vp.ID = id
+	} else {
+		logger.Warnw(ctx, "Device Not Found", log.Fields{"Device": device, "Port": portName})
+	}
+}
+
+// AddNbPonPort Add pon port to nbDevice
+func (va *VoltApplication) AddNbPonPort(oltSbID string, portID, maxAllowedChannels uint32,
+	enableMulticastKPI bool, portAlarmProfileID string) error {
+
+	var nbd *NbDevice
+	nbDevice, ok := va.NbDevice.Load(oltSbID)
+
+	if !ok {
+		nbd = NewNbDevice()
+		nbd.SouthBoundID = oltSbID
+	} else {
+		nbd = nbDevice.(*NbDevice)
+	}
+	port := nbd.AddPortToNbDevice(portID, maxAllowedChannels, enableMulticastKPI, portAlarmProfileID)
+
+	// Add this port to voltDevice
+	addPort := func(key, value interface{}) bool {
+		voltDevice := value.(*VoltDevice)
+		if oltSbID == voltDevice.SouthBoundID {
+			if _, exists := voltDevice.ActiveChannelsPerPon.Load(portID); !exists {
+				voltDevice.ActiveChannelsPerPon.Store(portID, port)
+			}
+			return false
+		}
+		return true
+	}
+	va.DevicesDisc.Range(addPort)
+	va.NbDevice.Store(oltSbID, nbd)
+
+	return nil
+}
+
+// UpdateNbPonPort update pon port to nbDevice
+func (va *VoltApplication) UpdateNbPonPort(oltSbID string, portID, maxAllowedChannels uint32, enableMulticastKPI bool, portAlarmProfileID string) error {
+
+	var nbd *NbDevice
+	nbDevice, ok := va.NbDevice.Load(oltSbID)
+
+	if !ok {
+		logger.Errorw(ctx, "Device-doesn't-exists", log.Fields{"deviceID": oltSbID})
+		return fmt.Errorf("Device-doesn't-exists-%v", oltSbID)
+	}
+	nbd = nbDevice.(*NbDevice)
+
+	port := nbd.UpdatePortToNbDevice(portID, maxAllowedChannels, enableMulticastKPI, portAlarmProfileID)
+	if port == nil {
+		return fmt.Errorf("Port-doesn't-exists-%v", portID)
+	}
+	va.NbDevice.Store(oltSbID, nbd)
+
+	// Add this port to voltDevice
+	updPort := func(key, value interface{}) bool {
+		voltDevice := value.(*VoltDevice)
+		if oltSbID == voltDevice.SouthBoundID {
+			voltDevice.ActiveChannelCountLock.Lock()
+			if p, exists := voltDevice.ActiveChannelsPerPon.Load(portID); exists {
+				oldPort := p.(*PonPortCfg)
+				if port.MaxActiveChannels != 0 {
+					oldPort.MaxActiveChannels = port.MaxActiveChannels
+					oldPort.EnableMulticastKPI = port.EnableMulticastKPI
+					voltDevice.ActiveChannelsPerPon.Store(portID, oldPort)
+				}
+			}
+			voltDevice.ActiveChannelCountLock.Unlock()
+			return false
+		}
+		return true
+	}
+	va.DevicesDisc.Range(updPort)
+
+	return nil
+}
+
+// DeleteNbPonPort Delete pon port to nbDevice
+func (va *VoltApplication) DeleteNbPonPort(oltSbID string, portID uint32) error {
+	nbDevice, ok := va.NbDevice.Load(oltSbID)
+	if ok {
+		nbDevice.(*NbDevice).DeletePortFromNbDevice(portID)
+		va.NbDevice.Store(oltSbID, nbDevice.(*NbDevice))
+	} else {
+		logger.Warnw(ctx, "Delete pon received for unknown device", log.Fields{"oltSbID": oltSbID})
+		return nil
+	}
+	// Delete this port from voltDevice
+	delPort := func(key, value interface{}) bool {
+		voltDevice := value.(*VoltDevice)
+		if oltSbID == voltDevice.SouthBoundID {
+			if _, exists := voltDevice.ActiveChannelsPerPon.Load(portID); exists {
+				voltDevice.ActiveChannelsPerPon.Delete(portID)
+			}
+			return false
+		}
+		return true
+	}
+	va.DevicesDisc.Range(delPort)
+	return nil
+}
+
+// GetNniPort : Get the NNI port for a device. Called from different other applications
+// as a port to match or destination for a packet out. The VOLT application
+// is written with the assumption that there is a single NNI port. The OLT
+// device is responsible for translating the combination of VLAN and the
+// NNI port ID to identify possibly a single physical port or a logical
+// port which is a result of protection methods applied.
+func (va *VoltApplication) GetNniPort(device string) (string, error) {
+	va.portLock.Lock()
+	defer va.portLock.Unlock()
+	d, ok := va.DevicesDisc.Load(device)
+	if !ok {
+		return "", errors.New("Device Doesn't Exist")
+	}
+	return d.(*VoltDevice).NniPort, nil
+}
+
+// NniDownInd process for Nni down indication.
+func (va *VoltApplication) NniDownInd(deviceID string, devSrNo string) {
+
+	logger.Debugw(ctx, "NNI Down Ind", log.Fields{"device": devSrNo})
+
+	handleIgmpDsFlows := func(key interface{}, value interface{}) bool {
+		mvProfile := value.(*MvlanProfile)
+		mvProfile.removeIgmpMcastFlows(devSrNo)
+		return true
+	}
+	va.MvlanProfilesByName.Range(handleIgmpDsFlows)
+
+	//Clear Static Group
+	va.ReceiverDownInd(deviceID, StaticPort)
+}
+
+// DeviceUpInd changes device state to up.
+func (va *VoltApplication) DeviceUpInd(device string) {
+	logger.Warnw(ctx, "Received Device Ind: UP", log.Fields{"Device": device})
+	if d := va.GetDevice(device); d != nil {
+		d.State = controller.DeviceStateUP
+	} else {
+		logger.Errorw(ctx, "Ignoring Device indication: UP. Device Missing", log.Fields{"Device": device})
+	}
+}
+
+// DeviceDownInd changes device state to down.
+func (va *VoltApplication) DeviceDownInd(device string) {
+	logger.Warnw(ctx, "Received Device Ind: DOWN", log.Fields{"Device": device})
+	if d := va.GetDevice(device); d != nil {
+		d.State = controller.DeviceStateDOWN
+	} else {
+		logger.Errorw(ctx, "Ignoring Device indication: DOWN. Device Missing", log.Fields{"Device": device})
+	}
+}
+
+// DeviceRebootInd process for handling flow clear flag for device reboot
+func (va *VoltApplication) DeviceRebootInd(device string, serialNum string, southBoundID string) {
+	logger.Warnw(ctx, "Received Device Ind: Reboot", log.Fields{"Device": device, "SerialNumber": serialNum})
+
+	if d := va.GetDevice(device); d != nil {
+		if d.State == controller.DeviceStateREBOOTED {
+			logger.Warnw(ctx, "Ignoring Device Ind: Reboot, Device already in Reboot state", log.Fields{"Device": device, "SerialNumber": serialNum, "State": d.State})
+			return
+		}
+		d.State = controller.DeviceStateREBOOTED
+	}
+	va.HandleFlowClearFlag(device, serialNum, southBoundID)
+
+}
+
+// DeviceDisableInd handles device deactivation process
+func (va *VoltApplication) DeviceDisableInd(device string) {
+	logger.Warnw(ctx, "Received Device Ind: Disable", log.Fields{"Device": device})
+
+	d := va.GetDevice(device)
+	if d == nil {
+		logger.Errorw(ctx, "Ignoring Device indication: DISABLED. Device Missing", log.Fields{"Device": device})
+		return
+	}
+
+	d.State = controller.DeviceStateDISABLED
+	va.HandleFlowClearFlag(device, d.SerialNum, d.SouthBoundID)
+}
+
+// ProcessIgmpDSFlowForMvlan for processing Igmp DS flow for device
+func (va *VoltApplication) ProcessIgmpDSFlowForMvlan(d *VoltDevice, mvp *MvlanProfile, addFlow bool) {
+
+	logger.Debugw(ctx, "Process IGMP DS Flows for MVlan", log.Fields{"device": d.Name, "Mvlan": mvp.Mvlan, "addFlow": addFlow})
+	portState := false
+	p := d.GetPort(d.NniPort)
+	if p != nil && p.State == PortStateUp {
+		portState = true
+	}
+
+	if addFlow {
+		if portState {
+			mvp.pushIgmpMcastFlows(d.SerialNum)
+		}
+	} else {
+		mvp.removeIgmpMcastFlows(d.SerialNum)
+	}
+}
+
+// ProcessIgmpDSFlowForDevice for processing Igmp DS flow for device
+func (va *VoltApplication) ProcessIgmpDSFlowForDevice(d *VoltDevice, addFlow bool) {
+	logger.Debugw(ctx, "Process IGMP DS Flows for device", log.Fields{"device": d.Name, "addFlow": addFlow})
+
+	handleIgmpDsFlows := func(key interface{}, value interface{}) bool {
+		mvProfile := value.(*MvlanProfile)
+		va.ProcessIgmpDSFlowForMvlan(d, mvProfile, addFlow)
+		return true
+	}
+	va.MvlanProfilesByName.Range(handleIgmpDsFlows)
+}
+
+// GetDeviceFromPort : This is suitable only for access ports as their naming convention
+// makes them unique across all the OLTs. This must be called with
+// port name that is an access port. Currently called from VNETs, attached
+// only to access ports, and the services which are also attached only
+// to access ports
+func (va *VoltApplication) GetDeviceFromPort(port string) (*VoltDevice, error) {
+	va.portLock.Lock()
+	defer va.portLock.Unlock()
+	var err error
+	err = nil
+	p, ok := va.PortsDisc.Load(port)
+	if !ok {
+		return nil, errorCodes.ErrPortNotFound
+	}
+	d := va.GetDevice(p.(*VoltPort).Device)
+	if d == nil {
+		err = errorCodes.ErrDeviceNotFound
+	}
+	return d, err
+}
+
+// GetPortID : This too applies only to access ports. The ports can be indexed
+// purely by their names without the device forming part of the key
+func (va *VoltApplication) GetPortID(port string) (uint32, error) {
+	va.portLock.Lock()
+	defer va.portLock.Unlock()
+	p, ok := va.PortsDisc.Load(port)
+	if !ok {
+		return 0, errorCodes.ErrPortNotFound
+	}
+	return p.(*VoltPort).ID, nil
+}
+
+// GetPortName : This too applies only to access ports. The ports can be indexed
+// purely by their names without the device forming part of the key
+func (va *VoltApplication) GetPortName(port uint32) (string, error) {
+	va.portLock.Lock()
+	defer va.portLock.Unlock()
+	var portName string
+	va.PortsDisc.Range(func(key interface{}, value interface{}) bool {
+		portInfo := value.(*VoltPort)
+		if portInfo.ID == port {
+			portName = portInfo.Name
+			return false
+		}
+		return true
+	})
+	return portName, nil
+}
+
+// GetPonFromUniPort to get Pon info from UniPort
+func (va *VoltApplication) GetPonFromUniPort(port string) (string, error) {
+	uniPortID, err := va.GetPortID(port)
+	if err == nil {
+		ponPortID := (uniPortID & 0x0FF00000) >> 20 //pon(8) + onu(8) + uni(12)
+		return strconv.FormatUint(uint64(ponPortID), 10), nil
+	}
+	return "", err
+}
+
+// GetPortState : This too applies only to access ports. The ports can be indexed
+// purely by their names without the device forming part of the key
+func (va *VoltApplication) GetPortState(port string) (PortState, error) {
+	va.portLock.Lock()
+	defer va.portLock.Unlock()
+	p, ok := va.PortsDisc.Load(port)
+	if !ok {
+		return 0, errors.New("Port not configured")
+	}
+	return p.(*VoltPort).State, nil
+}
+
+// GetIcmpv6Receivers to get Icmp v6 receivers
+func (va *VoltApplication) GetIcmpv6Receivers(device string) []uint32 {
+	var receiverList []uint32
+	receivers, _ := va.Icmpv6Receivers.Load(device)
+	if receivers != nil {
+		receiverList = receivers.([]uint32)
+	}
+	return receiverList
+}
+
+// AddIcmpv6Receivers to add Icmp v6 receivers
+func (va *VoltApplication) AddIcmpv6Receivers(device string, portID uint32) []uint32 {
+	var receiverList []uint32
+	receivers, _ := va.Icmpv6Receivers.Load(device)
+	if receivers != nil {
+		receiverList = receivers.([]uint32)
+	}
+	receiverList = append(receiverList, portID)
+	va.Icmpv6Receivers.Store(device, receiverList)
+	logger.Debugw(ctx, "Receivers after addition", log.Fields{"Receivers": receiverList})
+	return receiverList
+}
+
+// DelIcmpv6Receivers to delete Icmp v6 receievers
+func (va *VoltApplication) DelIcmpv6Receivers(device string, portID uint32) []uint32 {
+	var receiverList []uint32
+	receivers, _ := va.Icmpv6Receivers.Load(device)
+	if receivers != nil {
+		receiverList = receivers.([]uint32)
+	}
+	for i, port := range receiverList {
+		if port == portID {
+			receiverList = append(receiverList[0:i], receiverList[i+1:]...)
+			va.Icmpv6Receivers.Store(device, receiverList)
+			break
+		}
+	}
+	logger.Debugw(ctx, "Receivers After deletion", log.Fields{"Receivers": receiverList})
+	return receiverList
+}
+
+// ProcessDevFlowForDevice - Process DS ICMPv6 & ARP flow for provided device and vnet profile
+// device - Device Obj
+// vnet - vnet profile name
+// enabled - vlan enabled/disabled - based on the status, the flow shall be added/removed
+func (va *VoltApplication) ProcessDevFlowForDevice(device *VoltDevice, vnet *VoltVnet, enabled bool) {
+	_, applied := device.ConfiguredVlanForDeviceFlows.Get(VnetKey(vnet.SVlan, vnet.CVlan, 0))
+	if enabled {
+		va.PushDevFlowForVlan(vnet)
+	} else if !enabled && applied {
+		//va.DeleteDevFlowForVlan(vnet)
+		va.DeleteDevFlowForVlanFromDevice(vnet, device.SerialNum)
+	}
+}
+
+//NniVlanIndToIgmp - Trigger receiver up indication to all ports with igmp enabled
+//and has the provided mvlan
+func (va *VoltApplication) NniVlanIndToIgmp(device *VoltDevice, mvp *MvlanProfile) {
+
+	logger.Infow(ctx, "Sending Igmp Receiver UP indication for all Services", log.Fields{"Vlan": mvp.Mvlan})
+
+	//Trigger nni indication for receiver only for first time
+	if device.IgmpDsFlowAppliedForMvlan[uint16(mvp.Mvlan)] {
+		return
+	}
+	device.Ports.Range(func(key, value interface{}) bool {
+		port := key.(string)
+
+		if state, _ := va.GetPortState(port); state == PortStateUp {
+			vpvs, _ := va.VnetsByPort.Load(port)
+			if vpvs == nil {
+				return true
+			}
+			for _, vpv := range vpvs.([]*VoltPortVnet) {
+				//Send indication only for subscribers with the received mvlan profile
+				if vpv.IgmpEnabled && vpv.MvlanProfileName == mvp.Name {
+					vpv.services.Range(ReceiverUpInd)
+				}
+			}
+		}
+		return true
+	})
+}
+
+// PortUpInd :
+// -----------------------------------------------------------------------
+// Port status change handling
+// ----------------------------------------------------------------------
+// Port UP indication is passed to all services associated with the port
+// so that the services can configure flows applicable when the port goes
+// up from down state
+func (va *VoltApplication) PortUpInd(device string, port string) {
+	d := va.GetDevice(device)
+
+	if d == nil {
+		logger.Warnw(ctx, "Device Not Found - Dropping Port Ind: UP", log.Fields{"Device": device, "Port": port})
+		return
+	}
+
+	//Fixme: If Port Update Comes in large numbers, this will result in slow update per device
+	va.portLock.Lock()
+	// Do not defer the port mutex unlock here
+	// Some of the following func calls needs the port lock, so defering the lock here
+	// may lead to dead-lock
+	p := d.GetPort(port)
+
+	if p == nil {
+		logger.Infow(ctx, "Ignoring Port Ind: UP, Port doesnt exist", log.Fields{"Device": device, "PortName": port, "PortId": p})
+		va.portLock.Unlock()
+		return
+	}
+	p.State = PortStateUp
+	va.portLock.Unlock()
+
+	logger.Infow(ctx, "Received SouthBound Port Ind: UP", log.Fields{"Device": device, "PortName": port, "PortId": p.ID})
+	if p.Type == VoltPortTypeNni {
+
+		logger.Warnw(ctx, "Received NNI Port Ind: UP", log.Fields{"Device": device, "PortName": port, "PortId": p.ID})
+		//va.PushDevFlowForDevice(d)
+		//Build Igmp TrapFlowRule
+		//va.ProcessIgmpDSFlowForDevice(d, true)
+	}
+	vpvs, ok := va.VnetsByPort.Load(port)
+	if !ok || nil == vpvs || len(vpvs.([]*VoltPortVnet)) == 0 {
+		logger.Infow(ctx, "No VNETs on port", log.Fields{"Device": device, "Port": port})
+		//msgbus.ProcessPortInd(msgbus.PortUp, d.SerialNum, p.Name, false, getServiceList(port))
+		return
+	}
+
+	//If NNI port is not UP, do not push Flows
+	if d.NniPort == "" {
+		logger.Warnw(ctx, "NNI port not UP. Not sending Port UP Ind for VPVs", log.Fields{"NNI": d.NniPort})
+		return
+	}
+
+	vpvList := vpvs.([]*VoltPortVnet)
+	if vpvList[0].PonPort != 0xFF && vpvList[0].PonPort != p.PonPort {
+		logger.Errorw(ctx, "UNI port discovered on wrong PON Port. Dropping Port Indication", log.Fields{"Device": device, "Port": port, "DetectedPon": p.PonPort, "ExpectedPon": vpvList[0].PonPort})
+
+		//Remove the flow (if any) which are already installed - Valid for PON switching when VGC pod is DOWN
+		for _, vpv := range vpvs.([]*VoltPortVnet) {
+			vpv.VpvLock.Lock()
+			logger.Warnw(ctx, "Removing existing VPVs/Services flows for for Subscriber: UNI Detected on wrong PON", log.Fields{"Port": vpv.Port, "Vnet": vpv.VnetName})
+			vpv.PortDownInd(device, port)
+			if vpv.IgmpEnabled {
+				va.ReceiverDownInd(device, port)
+			}
+			vpv.VpvLock.Unlock()
+		}
+		return
+	}
+
+/*
+	if p.Type != VoltPortTypeNni {
+		// Process port up indication
+		indTask := cntlr.NewAddPortInd(p.Name, msgbus.PortUp, d.SerialNum, true, getServiceList(port))
+		cntlr.GetController().PostIndication(device, indTask)
+	}
+*/
+
+	for _, vpv := range vpvs.([]*VoltPortVnet) {
+		vpv.VpvLock.Lock()
+
+		//Do not trigger indication for the vpv which is already removed from vpv list as
+		// part of service delete (during the lock wait duration)
+		// In that case, the services associated wil be zero
+		if vpv.servicesCount.Load() != 0 {
+			vpv.PortUpInd(d, port)
+		}
+		vpv.VpvLock.Unlock()
+	}
+	// At the end of processing inform the other entities that
+	// are interested in the events
+}
+
+/*
+func getServiceList(port string) map[string]bool {
+	serviceList := make(map[string]bool)
+
+	getServiceNames := func(key interface{}, value interface{}) bool {
+		serviceList[key.(string)] = value.(*VoltService).DsHSIAFlowsApplied
+		return true
+	}
+
+	if vpvs, _ := GetApplication().VnetsByPort.Load(port); vpvs != nil {
+		vpvList := vpvs.([]*VoltPortVnet)
+		for _, vpv := range vpvList {
+			vpv.services.Range(getServiceNames)
+		}
+	}
+	return serviceList
+
+}*/
+
+//ReceiverUpInd - Send receiver up indication for service with Igmp enabled
+func ReceiverUpInd(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	var vlan of.VlanType
+
+	if !svc.IPAssigned() {
+		logger.Infow(ctx, "IP Not assigned, skipping general query", log.Fields{"Service": svc})
+		return false
+	}
+
+	//Send port up indication to igmp only for service with igmp enabled
+	if svc.IgmpEnabled {
+		if svc.VlanControl == ONUCVlan || svc.VlanControl == ONUCVlanOLTSVlan {
+			vlan = svc.CVlan
+		} else {
+			vlan = svc.UniVlan
+		}
+		if device, _ := GetApplication().GetDeviceFromPort(svc.Port); device != nil {
+			GetApplication().ReceiverUpInd(device.Name, svc.Port, svc.MvlanProfileName, vlan, svc.Pbits)
+		}
+		return false
+	}
+	return true
+}
+
+// PortDownInd : Port down indication is passed on to the services so that the services
+// can make changes at this transition.
+func (va *VoltApplication) PortDownInd(device string, port string) {
+	logger.Infow(ctx, "Received SouthBound Port Ind: DOWN", log.Fields{"Device": device, "Port": port})
+	d := va.GetDevice(device)
+
+	if d == nil {
+		logger.Warnw(ctx, "Device Not Found - Dropping Port Ind: DOWN", log.Fields{"Device": device, "Port": port})
+		return
+	}
+	//Fixme: If Port Update Comes in large numbers, this will result in slow update per device
+	va.portLock.Lock()
+	// Do not defer the port mutex unlock here
+	// Some of the following func calls needs the port lock, so defering the lock here
+	// may lead to dead-lock
+	p := d.GetPort(port)
+	if p == nil {
+		logger.Infow(ctx, "Ignoring Port Ind: Down, Port doesnt exist", log.Fields{"Device": device, "PortName": port, "PortId": p})
+		va.portLock.Unlock()
+		return
+	}
+	p.State = PortStateDown
+	va.portLock.Unlock()
+
+	if d.State == controller.DeviceStateREBOOTED {
+		logger.Infow(ctx, "Ignoring Port Ind: Down, Device has been Rebooted", log.Fields{"Device": device, "PortName": port, "PortId": p})
+		return
+	}
+
+	if p.Type == VoltPortTypeNni {
+		logger.Warnw(ctx, "Received NNI Port Ind: DOWN", log.Fields{"Device": device, "Port": port})
+		va.DeleteDevFlowForDevice(d)
+		va.NniDownInd(device, d.SerialNum)
+		va.RemovePendingGroups(device, true)
+	}
+	vpvs, ok := va.VnetsByPort.Load(port)
+	if !ok || nil == vpvs || len(vpvs.([]*VoltPortVnet)) == 0 {
+		logger.Infow(ctx, "No VNETs on port", log.Fields{"Device": device, "Port": port})
+		//msgbus.ProcessPortInd(msgbus.PortDown, d.SerialNum, p.Name, false, getServiceList(port))
+		return
+	}
+/*
+	if p.Type != VoltPortTypeNni {
+		// Process port down indication
+		indTask := cntlr.NewAddPortInd(p.Name, msgbus.PortDown, d.SerialNum, true, getServiceList(port))
+		cntlr.GetController().PostIndication(device, indTask)
+	}
+*/
+	for _, vpv := range vpvs.([]*VoltPortVnet) {
+		vpv.VpvLock.Lock()
+		vpv.PortDownInd(device, port)
+		if vpv.IgmpEnabled {
+			va.ReceiverDownInd(device, port)
+		}
+		vpv.VpvLock.Unlock()
+	}
+}
+
+// PacketInInd :
+// -----------------------------------------------------------------------
+// PacketIn Processing
+// Packet In Indication processing. It arrives with the identities of
+// the device and port on which the packet is received. At first, the
+// packet is decoded and the right processor is called. Currently, we
+// plan to support only DHCP and IGMP. In future, we can add more
+// capabilities as needed
+func (va *VoltApplication) PacketInInd(device string, port string, pkt []byte) {
+	// Decode the incoming packet
+	packetSide := US
+	if strings.Contains(port, NNI) {
+		packetSide = DS
+	}
+
+	logger.Debugw(ctx, "Received a Packet-In Indication", log.Fields{"Device": device, "Port": port})
+
+	gopkt := gopacket.NewPacket(pkt, layers.LayerTypeEthernet, gopacket.Default)
+
+	var dot1qFound = false
+	for _, l := range gopkt.Layers() {
+		if l.LayerType() == layers.LayerTypeDot1Q {
+			dot1qFound = true
+			break
+		}
+	}
+
+	if !dot1qFound {
+		logger.Debugw(ctx, "Ignoring Received Packet-In Indication without Dot1Q Header",
+			log.Fields{"Device": device, "Port": port})
+		return
+	}
+
+	logger.Debugw(ctx, "Received Southbound Packet In", log.Fields{"Pkt": hex.EncodeToString(gopkt.Data())})
+
+	// Classify the packet into packet types that we support
+	// The supported types are DHCP and IGMP. The DHCP packet is
+	// identified by matching the L4 protocol to UDP. The IGMP packet
+	// is identified by matching L3 protocol to IGMP
+	arpl := gopkt.Layer(layers.LayerTypeARP)
+	if arpl != nil {
+		if callBack, ok := PacketHandlers[ARP]; ok {
+			callBack(device, port, gopkt)
+		} else {
+			logger.Debugw(ctx, "ARP handler is not registered, dropping the packet", log.Fields{"Pkt": hex.EncodeToString(gopkt.Data())})
+		}
+		return
+	}
+	ipv4l := gopkt.Layer(layers.LayerTypeIPv4)
+	if ipv4l != nil {
+		ip := ipv4l.(*layers.IPv4)
+
+		if ip.Protocol == layers.IPProtocolUDP {
+			logger.Debugw(ctx, "Received Southbound UDP ipv4 packet in", log.Fields{"StreamSide": packetSide})
+			dhcpl := gopkt.Layer(layers.LayerTypeDHCPv4)
+			if dhcpl != nil {
+				if callBack, ok := PacketHandlers[DHCPv4]; ok {
+					callBack(device, port, gopkt)
+				} else {
+					logger.Debugw(ctx, "DHCPv4 handler is not registered, dropping the packet", log.Fields{"Pkt": hex.EncodeToString(gopkt.Data())})
+				}
+			}
+		} else if ip.Protocol == layers.IPProtocolIGMP {
+			logger.Debugw(ctx, "Received Southbound IGMP packet in", log.Fields{"StreamSide": packetSide})
+			if callBack, ok := PacketHandlers[IGMP]; ok {
+				callBack(device, port, gopkt)
+			} else {
+				logger.Debugw(ctx, "IGMP handler is not registered, dropping the packet", log.Fields{"Pkt": hex.EncodeToString(gopkt.Data())})
+			}
+		}
+		return
+	}
+	ipv6l := gopkt.Layer(layers.LayerTypeIPv6)
+	if ipv6l != nil {
+		ip := ipv6l.(*layers.IPv6)
+		if ip.NextHeader == layers.IPProtocolUDP {
+			logger.Debug(ctx, "Received Southbound UDP ipv6 packet in")
+			dhcpl := gopkt.Layer(layers.LayerTypeDHCPv6)
+			if dhcpl != nil {
+				if callBack, ok := PacketHandlers[DHCPv6]; ok {
+					callBack(device, port, gopkt)
+				} else {
+					logger.Debugw(ctx, "DHCPv6 handler is not registered, dropping the packet", log.Fields{"Pkt": hex.EncodeToString(gopkt.Data())})
+				}
+			}
+		}
+		return
+	}
+
+	pppoel := gopkt.Layer(layers.LayerTypePPPoE)
+	if pppoel != nil {
+		logger.Debugw(ctx, "Received Southbound PPPoE packet in", log.Fields{"StreamSide": packetSide})
+		if callBack, ok := PacketHandlers[PPPOE]; ok {
+			callBack(device, port, gopkt)
+		} else {
+			logger.Debugw(ctx, "PPPoE handler is not registered, dropping the packet", log.Fields{"Pkt": hex.EncodeToString(gopkt.Data())})
+		}
+	}
+}
+
+// GetVlans : This utility gets the VLANs from the packet. The VLANs are
+// used to identify the right service that must process the incoming
+// packet
+func GetVlans(pkt gopacket.Packet) []of.VlanType {
+	var vlans []of.VlanType
+	for _, l := range pkt.Layers() {
+		if l.LayerType() == layers.LayerTypeDot1Q {
+			q, ok := l.(*layers.Dot1Q)
+			if ok {
+				vlans = append(vlans, of.VlanType(q.VLANIdentifier))
+			}
+		}
+	}
+	return vlans
+}
+
+// GetPriority to get priority
+func GetPriority(pkt gopacket.Packet) uint8 {
+	for _, l := range pkt.Layers() {
+		if l.LayerType() == layers.LayerTypeDot1Q {
+			q, ok := l.(*layers.Dot1Q)
+			if ok {
+				return q.Priority
+			}
+		}
+	}
+	return PriorityNone
+}
+
+// HandleFlowClearFlag to handle flow clear flag during reboot
+func (va *VoltApplication) HandleFlowClearFlag(deviceID string, serialNum, southBoundID string) {
+	logger.Warnw(ctx, "Clear All flags for Device", log.Fields{"Device": deviceID, "SerialNum": serialNum, "SBID": southBoundID})
+	dev, ok := va.DevicesDisc.Load(deviceID)
+	if ok && dev != nil {
+		logger.Infow(ctx, "Clear Flags for device", log.Fields{"voltDevice": dev.(*VoltDevice).Name})
+		dev.(*VoltDevice).icmpv6GroupAdded = false
+		logger.Infow(ctx, "Clearing DS Icmpv6 Map",
+			log.Fields{"voltDevice": dev.(*VoltDevice).Name})
+		dev.(*VoltDevice).ConfiguredVlanForDeviceFlows = util.NewConcurrentMap()
+		logger.Infow(ctx, "Clearing DS IGMP Map",
+			log.Fields{"voltDevice": dev.(*VoltDevice).Name})
+		for k := range dev.(*VoltDevice).IgmpDsFlowAppliedForMvlan {
+			delete(dev.(*VoltDevice).IgmpDsFlowAppliedForMvlan, k)
+		}
+		//Delete group 1 - ICMPv6/ARP group
+		if err := ProcessIcmpv6McGroup(deviceID, true); err != nil {
+			logger.Errorw(ctx, "ProcessIcmpv6McGroup failed", log.Fields{"Device": deviceID, "Error": err})
+		}
+	} else {
+		logger.Warnw(ctx, "VoltDevice not found for device ", log.Fields{"deviceID": deviceID})
+	}
+
+	getVpvs := func(key interface{}, value interface{}) bool {
+		vpvs := value.([]*VoltPortVnet)
+		for _, vpv := range vpvs {
+			if vpv.Device == deviceID {
+				logger.Infow(ctx, "Clear Flags for vpv",
+					log.Fields{"device": vpv.Device, "port": vpv.Port,
+						"svlan": vpv.SVlan, "cvlan": vpv.CVlan, "univlan": vpv.UniVlan})
+				vpv.ClearAllServiceFlags()
+				vpv.ClearAllVpvFlags()
+
+				if vpv.IgmpEnabled {
+					va.ReceiverDownInd(vpv.Device, vpv.Port)
+					//Also clear service igmp stats
+					vpv.ClearServiceCounters()
+				}
+			}
+		}
+		return true
+	}
+	va.VnetsByPort.Range(getVpvs)
+
+	//Clear Static Group
+	va.ReceiverDownInd(deviceID, StaticPort)
+
+	logger.Warnw(ctx, "All flags cleared for device", log.Fields{"Device": deviceID})
+
+	//Reset pending group pool
+	va.RemovePendingGroups(deviceID, true)
+
+	//Process all Migrate Service Request - force udpate all profiles since resources are already cleaned up
+	if dev != nil {
+		triggerForceUpdate := func(key, value interface{}) bool {
+			msrList := value.(*util.ConcurrentMap)
+			forceUpdateServices := func(key, value interface{}) bool {
+				msr := value.(*MigrateServicesRequest)
+				forceUpdateAllServices(msr)
+				return true
+			}
+			msrList.Range(forceUpdateServices)
+			return true
+		}
+		dev.(*VoltDevice).MigratingServices.Range(triggerForceUpdate)
+	} else {
+		va.FetchAndProcessAllMigrateServicesReq(deviceID, forceUpdateAllServices)
+	}
+}
+
+//GetPonPortIDFromUNIPort to get pon port id from uni port
+func GetPonPortIDFromUNIPort(uniPortID uint32) uint32 {
+	ponPortID := (uniPortID & 0x0FF00000) >> 20
+	return ponPortID
+}
+
+//ProcessFlowModResultIndication - Processes Flow mod operation indications from controller
+func (va *VoltApplication) ProcessFlowModResultIndication(flowStatus intf.FlowStatus) {
+
+	d := va.GetDevice(flowStatus.Device)
+	if d == nil {
+		logger.Errorw(ctx, "Dropping Flow Mod Indication. Device not found", log.Fields{"Cookie": flowStatus.Cookie, "Device": flowStatus.Device})
+		return
+	}
+
+	cookieExists := ExecuteFlowEvent(d, flowStatus.Cookie, flowStatus)
+
+	if flowStatus.Flow != nil {
+		flowAdd := (flowStatus.FlowModType == of.CommandAdd)
+		if !cookieExists && !isFlowStatusSuccess(flowStatus.Status, flowAdd) {
+			pushFlowFailureNotif(flowStatus)
+		}
+	}
+}
+
+func pushFlowFailureNotif(flowStatus intf.FlowStatus) {
+	subFlow := flowStatus.Flow
+	cookie := subFlow.Cookie
+	uniPort := cookie >> 16 & 0xFFFFFFFF
+	logger.Errorw(ctx, "Flow Failure Notification", log.Fields{"uniPort": uniPort, "Cookie": cookie})
+/*
+	device := flowStatus.Device
+	priority := subFlow.Priority
+	isIgmp := false
+	var devSerialNum string
+	var service *VoltService
+
+	if subFlow.Match.L4Protocol == of.IPProtocolIgmp {
+		isIgmp = true
+	} else if priority != of.HsiaFlowPriority {
+		logger.Info(ctx, "Not HSIA flow, ignoring the failure notification")
+		return
+	}
+
+	cookie := subFlow.Cookie
+	pbit := subFlow.Pbits
+	uniPort := cookie >> 16 & 0xFFFFFFFF
+	portName, _ := GetApplication().GetPortName(uint32(uniPort))
+	portState := msgbus.PortDown
+	logger.Errorw(ctx, "Construct Flow Failure Notification", log.Fields{"uniPort": uniPort, "Cookie": cookie, "Pbit": pbit, "isIgmp": isIgmp})
+
+	if isIgmp {
+		cvlan := subFlow.TableMetadata & 0xFFFF
+		service = GetApplication().GetMatchingMcastService(portName, device, of.VlanType(cvlan))
+	} else {
+		service = GetApplication().GetServiceNameFromCookie(cookie, portName, uint8(pbit), device, subFlow.TableMetadata)
+	}
+	var trigger infra.Reason
+	if nil != service {
+		logger.Errorw(ctx, "Sending Flow Failure Notification", log.Fields{"uniPort": uniPort, "Cookie": cookie, "Pbit": pbit, "Service": service.Name, "ErrorCode": flowStatus.Status})
+		if vd := GetApplication().GetDevice(device); vd != nil {
+			devSerialNum = vd.SerialNum
+			if portSt, _ := GetApplication().GetPortState(service.Port); portSt == PortStateUp {
+				portState = msgbus.PortUp
+			}
+			trigger = service.getSrvDeactTrigger(vd, portState)
+		}
+		msgbus.PostAccessConfigInd(msgbus.Failed, devSerialNum, msgbus.HSIA, service.Name, int(flowStatus.Status), subFlow.ErrorReason, trigger, portState)
+	}
+*/
+}
+
+//UpdateMvlanProfilesForDevice to update mvlan profile for device
+func (va *VoltApplication) UpdateMvlanProfilesForDevice(device string) {
+
+	checkAndAddMvlanUpdateTask := func(key, value interface{}) bool {
+		mvp := value.(*MvlanProfile)
+		if mvp.IsUpdateInProgressForDevice(device) {
+			mvp.UpdateProfile(device)
+		}
+		return true
+	}
+	va.MvlanProfilesByName.Range(checkAndAddMvlanUpdateTask)
+}
+
+// TaskInfo structure that is used to store the task Info.
+type TaskInfo struct {
+	ID        string
+	Name      string
+	Timestamp string
+}
+
+// GetTaskList to get task list information.
+func (va *VoltApplication) GetTaskList(device string) map[int]*TaskInfo {
+	taskList := cntlr.GetController().GetTaskList(device)
+	taskMap := make(map[int]*TaskInfo)
+	for i, task := range taskList {
+		taskID := strconv.Itoa(int(task.TaskID()))
+		name := task.Name()
+		timestamp := task.Timestamp()
+		taskInfo := &TaskInfo{ID: taskID, Name: name, Timestamp: timestamp}
+		taskMap[i] = taskInfo
+	}
+	return taskMap
+}
+
+// UpdateDeviceSerialNumberList to update the device serial number list after device serial number is updated for vnet and mvlan
+func (va *VoltApplication) UpdateDeviceSerialNumberList(oldOltSlNo string, newOltSlNo string) {
+
+	voltDevice := va.GetDeviceBySerialNo(oldOltSlNo)
+
+	if voltDevice != nil {
+		// Device is present with old serial number ID
+		logger.Errorw(ctx, "OLT Migration cannot be completed as there are dangling devices", log.Fields{"Serial Number": oldOltSlNo})
+
+	} else {
+		logger.Infow(ctx, "No device present with old serial number", log.Fields{"Serial Number": oldOltSlNo})
+
+		// Add Serial Number to Blocked Devices List.
+		cntlr.GetController().AddBlockedDevices(oldOltSlNo)
+		cntlr.GetController().AddBlockedDevices(newOltSlNo)
+
+		updateSlNoForVnet := func(key, value interface{}) bool {
+			vnet := value.(*VoltVnet)
+			for i, deviceSlNo := range vnet.VnetConfig.DevicesList {
+				if deviceSlNo == oldOltSlNo {
+					vnet.VnetConfig.DevicesList[i] = newOltSlNo
+					logger.Infow(ctx, "device serial number updated for vnet profile", log.Fields{"Updated Serial Number": deviceSlNo, "Previous Serial Number": oldOltSlNo})
+					break
+				}
+			}
+			return true
+		}
+
+		updateSlNoforMvlan := func(key interface{}, value interface{}) bool {
+			mvProfile := value.(*MvlanProfile)
+			for deviceSlNo := range mvProfile.DevicesList {
+				if deviceSlNo == oldOltSlNo {
+					mvProfile.DevicesList[newOltSlNo] = mvProfile.DevicesList[oldOltSlNo]
+					delete(mvProfile.DevicesList, oldOltSlNo)
+					logger.Infow(ctx, "device serial number updated for mvlan profile", log.Fields{"Updated Serial Number": deviceSlNo, "Previous Serial Number": oldOltSlNo})
+					break
+				}
+			}
+			return true
+		}
+
+		va.VnetsByName.Range(updateSlNoForVnet)
+		va.MvlanProfilesByName.Range(updateSlNoforMvlan)
+
+		// Clear the serial number from Blocked Devices List
+		cntlr.GetController().DelBlockedDevices(oldOltSlNo)
+		cntlr.GetController().DelBlockedDevices(newOltSlNo)
+
+	}
+}
+
+// GetVpvsForDsPkt to get vpv for downstream packets
+func (va *VoltApplication) GetVpvsForDsPkt(cvlan of.VlanType, svlan of.VlanType, clientMAC net.HardwareAddr,
+	pbit uint8) ([]*VoltPortVnet, error) {
+
+	var matchVPVs []*VoltPortVnet
+	findVpv := func(key, value interface{}) bool {
+		vpvs := value.([]*VoltPortVnet)
+		for _, vpv := range vpvs {
+			if vpv.isVlanMatching(cvlan, svlan) && vpv.MatchesPriority(pbit) != nil {
+				var subMac net.HardwareAddr
+				if NonZeroMacAddress(vpv.MacAddr) {
+					subMac = vpv.MacAddr
+				} else if vpv.LearntMacAddr != nil && NonZeroMacAddress(vpv.LearntMacAddr) {
+					subMac = vpv.LearntMacAddr
+				} else {
+					matchVPVs = append(matchVPVs, vpv)
+					continue
+				}
+				if util.MacAddrsMatch(subMac, clientMAC) {
+					matchVPVs = append([]*VoltPortVnet{}, vpv)
+					logger.Infow(ctx, "Matching VPV found", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "MAC": clientMAC})
+					return false
+				}
+			}
+		}
+		return true
+	}
+	va.VnetsByPort.Range(findVpv)
+
+	if len(matchVPVs) != 1 {
+		logger.Infow(ctx, "No matching VPV found or multiple vpvs found", log.Fields{"Match VPVs": matchVPVs, "MAC": clientMAC})
+		return nil, errors.New("No matching VPV found or multiple vpvs found")
+	}
+	return matchVPVs, nil
+}
+
+// GetMacInPortMap to get PORT value  based on MAC key
+func (va *VoltApplication) GetMacInPortMap(macAddr net.HardwareAddr) string {
+	if NonZeroMacAddress(macAddr) {
+		va.macPortLock.Lock()
+		defer va.macPortLock.Unlock()
+		if port, ok := va.macPortMap[macAddr.String()]; ok {
+			logger.Debugw(ctx, "found-entry-macportmap", log.Fields{"MacAddr": macAddr.String(), "Port": port})
+			return port
+		}
+	}
+	logger.Infow(ctx, "entry-not-found-macportmap", log.Fields{"MacAddr": macAddr.String()})
+	return ""
+}
+
+// UpdateMacInPortMap to update MAC PORT (key value) information in MacPortMap
+func (va *VoltApplication) UpdateMacInPortMap(macAddr net.HardwareAddr, port string) {
+	if NonZeroMacAddress(macAddr) {
+		va.macPortLock.Lock()
+		va.macPortMap[macAddr.String()] = port
+		va.macPortLock.Unlock()
+		logger.Debugw(ctx, "updated-macportmap", log.Fields{"MacAddr": macAddr.String(), "Port": port})
+	}
+}
+
+// DeleteMacInPortMap to remove MAC key from MacPortMap
+func (va *VoltApplication) DeleteMacInPortMap(macAddr net.HardwareAddr) {
+	if NonZeroMacAddress(macAddr) {
+		port := va.GetMacInPortMap(macAddr)
+		va.macPortLock.Lock()
+		delete(va.macPortMap, macAddr.String())
+		va.macPortLock.Unlock()
+		logger.Debugw(ctx, "deleted-from-macportmap", log.Fields{"MacAddr": macAddr.String(), "Port": port})
+	}
+}
+
+//AddGroupToPendingPool - adds the IgmpGroup with active group table entry to global pending pool
+func (va *VoltApplication) AddGroupToPendingPool(ig *IgmpGroup) {
+	var grpMap map[*IgmpGroup]bool
+	var ok bool
+
+	va.PendingPoolLock.Lock()
+	defer va.PendingPoolLock.Unlock()
+
+	logger.Infow(ctx, "Adding IgmpGroup to Global Pending Pool", log.Fields{"GroupID": ig.GroupID, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr, "PendingDevices": len(ig.Devices)})
+	// Do Not Reset any current profile info since group table entry tied to mvlan profile
+	// The PonVlan is part of set field in group installed
+	// Hence, Group created is always tied to the same mvlan profile until deleted
+
+	for device := range ig.Devices {
+		key := getPendingPoolKey(ig.Mvlan, device)
+
+		if grpMap, ok = va.IgmpPendingPool[key]; !ok {
+			grpMap = make(map[*IgmpGroup]bool)
+		}
+		grpMap[ig] = true
+
+		//Add grpObj reference to all associated devices
+		va.IgmpPendingPool[key] = grpMap
+	}
+}
+
+//RemoveGroupFromPendingPool - removes the group from global pending group pool
+func (va *VoltApplication) RemoveGroupFromPendingPool(device string, ig *IgmpGroup) bool {
+	GetApplication().PendingPoolLock.Lock()
+	defer GetApplication().PendingPoolLock.Unlock()
+
+	logger.Infow(ctx, "Removing IgmpGroup from Global Pending Pool", log.Fields{"Device": device, "GroupID": ig.GroupID, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr, "PendingDevices": len(ig.Devices)})
+
+	key := getPendingPoolKey(ig.Mvlan, device)
+	if _, ok := va.IgmpPendingPool[key]; ok {
+		delete(va.IgmpPendingPool[key], ig)
+		return true
+	}
+	return false
+}
+
+//RemoveGroupsFromPendingPool - removes the group from global pending group pool
+func (va *VoltApplication) RemoveGroupsFromPendingPool(device string, mvlan of.VlanType) {
+	GetApplication().PendingPoolLock.Lock()
+	defer GetApplication().PendingPoolLock.Unlock()
+
+	logger.Infow(ctx, "Removing IgmpGroups from Global Pending Pool for given Deivce & Mvlan", log.Fields{"Device": device, "Mvlan": mvlan.String()})
+
+	key := getPendingPoolKey(mvlan, device)
+	va.RemoveGroupListFromPendingPool(key)
+}
+
+//RemoveGroupListFromPendingPool - removes the groups for provided key
+// 1. Deletes the group from device
+// 2. Delete the IgmpGroup obj and release the group ID to pool
+// Note: Make sure to obtain PendingPoolLock lock before calling this func
+func (va *VoltApplication) RemoveGroupListFromPendingPool(key string) {
+	if grpMap, ok := va.IgmpPendingPool[key]; ok {
+		delete(va.IgmpPendingPool, key)
+		for ig := range grpMap {
+			for device := range ig.Devices {
+				ig.DeleteIgmpGroupDevice(device)
+			}
+		}
+	}
+}
+
+//RemoveGroupDevicesFromPendingPool - removes the group from global pending group pool
+func (va *VoltApplication) RemoveGroupDevicesFromPendingPool(ig *IgmpGroup) {
+
+	logger.Infow(ctx, "Removing IgmpGroup for all devices from Global Pending Pool", log.Fields{"GroupID": ig.GroupID, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr, "PendingDevices": len(ig.Devices)})
+	for device := range ig.PendingGroupForDevice {
+		va.RemoveGroupFromPendingPool(device, ig)
+	}
+}
+
+//GetGroupFromPendingPool - Returns IgmpGroup obj from global pending pool
+func (va *VoltApplication) GetGroupFromPendingPool(mvlan of.VlanType, device string) *IgmpGroup {
+
+	var ig *IgmpGroup
+
+	va.PendingPoolLock.Lock()
+	defer va.PendingPoolLock.Unlock()
+
+	key := getPendingPoolKey(mvlan, device)
+	logger.Infow(ctx, "Getting IgmpGroup from Global Pending Pool", log.Fields{"Device": device, "Mvlan": mvlan.String(), "Key": key})
+
+	//Gets all IgmpGrp Obj for the device
+	grpMap, ok := va.IgmpPendingPool[key]
+	if !ok || len(grpMap) == 0 {
+		logger.Infow(ctx, "Matching IgmpGroup not found in Global Pending Pool", log.Fields{"Device": device, "Mvlan": mvlan.String()})
+		return nil
+	}
+
+	//Gets a random obj from available grps
+	for ig = range grpMap {
+
+		//Remove grp obj reference from all devices associated in pending pool
+		for dev := range ig.Devices {
+			key := getPendingPoolKey(mvlan, dev)
+			delete(va.IgmpPendingPool[key], ig)
+		}
+
+		//Safety check to avoid re-allocating group already in use
+		if ig.NumDevicesActive() == 0 {
+			return ig
+		}
+
+		//Iteration will continue only if IG is not allocated
+	}
+	return nil
+}
+
+//RemovePendingGroups - removes all pending groups for provided reference from global pending pool
+// reference - mvlan/device ID
+// isRefDevice - true  - Device as reference
+//               false - Mvlan as reference
+func (va *VoltApplication) RemovePendingGroups(reference string, isRefDevice bool) {
+	va.PendingPoolLock.Lock()
+	defer va.PendingPoolLock.Unlock()
+
+	logger.Infow(ctx, "Removing IgmpGroups from Global Pending Pool", log.Fields{"Reference": reference, "isRefDevice": isRefDevice})
+
+	//Pending Pool key: "<mvlan>_<DeviceID>""
+	paramPosition := 0
+	if isRefDevice {
+		paramPosition = 1
+	}
+
+	// 1.Remove the Entry from pending pool
+	// 2.Deletes the group from device
+	// 3.Delete the IgmpGroup obj and release the group ID to pool
+	for key := range va.IgmpPendingPool {
+		keyParams := strings.Split(key, "_")
+		if keyParams[paramPosition] == reference {
+			va.RemoveGroupListFromPendingPool(key)
+		}
+	}
+}
+
+func getPendingPoolKey(mvlan of.VlanType, device string) string {
+	return mvlan.String() + "_" + device
+}
+
+func (va *VoltApplication) removeExpiredGroups() {
+	logger.Debug(ctx, "Check for expired Igmp Groups")
+	removeExpiredGroups := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+		ig.removeExpiredGroupFromDevice()
+		return true
+	}
+	va.IgmpGroups.Range(removeExpiredGroups)
+}
+
+//TriggerPendingProfileDeleteReq - trigger pending profile delete request
+func (va *VoltApplication) TriggerPendingProfileDeleteReq(device string) {
+	va.TriggerPendingServiceDeleteReq(device)
+	va.TriggerPendingVpvDeleteReq(device)
+	va.TriggerPendingVnetDeleteReq(device)
+	logger.Warnw(ctx, "All Pending Profile Delete triggered for device", log.Fields{"Device": device})
+}
+
+//TriggerPendingServiceDeleteReq - trigger pending service delete request
+func (va *VoltApplication) TriggerPendingServiceDeleteReq(device string) {
+
+	logger.Warnw(ctx, "Pending Services to be deleted", log.Fields{"Count": len(va.ServicesToDelete)})
+	for serviceName := range va.ServicesToDelete {
+		logger.Debugw(ctx, "Trigger Service Delete", log.Fields{"Service": serviceName})
+		if vs := va.GetService(serviceName); vs != nil {
+			if vs.Device == device {
+				logger.Warnw(ctx, "Triggering Pending Service delete", log.Fields{"Service": vs.Name})
+				vs.DelHsiaFlows()
+				if vs.ForceDelete {
+					vs.DelFromDb()
+					/*
+					portState := msgbus.PortDown
+					if d, err := va.GetDeviceFromPort(vs.Port); d != nil {
+
+						if portSt, _ := GetApplication().GetPortState(vs.Port); portSt == PortStateUp {
+							portState = msgbus.PortUp
+						}
+						indTask := cntlr.NewAddServiceIndTask(vs.Name, d.SerialNum, msgbus.DelHSIA, msgbus.Success, "", portState, infra.DelHSIAFromNB)
+						cntlr.GetController().PostIndication(d.Name, indTask)
+					} else {
+						// Port Not found can occur during ONU movement. However, port delete had already handled flow deletion,
+						// hence indication can be sent immediately
+						var devSrNo string
+						logger.Errorw(ctx, "Device/Port not found. Send indication directly", log.Fields{"serviceName": vs.Name, "error": err})
+						if vd := va.GetDevice(vs.Device); vd != nil {
+							devSrNo = vd.SerialNum
+						}
+						msgbus.PostAccessConfigInd(msgbus.Success, devSrNo, msgbus.DelHSIA, vs.Name, 0, "", infra.DelHSIAFromNB, portState)
+					}*/
+				}
+			}
+		} else {
+			logger.Errorw(ctx, "Pending Service Not found", log.Fields{"Service": serviceName})
+		}
+	}
+}
+
+//TriggerPendingVpvDeleteReq - trigger pending VPV delete request
+func (va *VoltApplication) TriggerPendingVpvDeleteReq(device string) {
+
+	logger.Warnw(ctx, "Pending VPVs to be deleted", log.Fields{"Count": len(va.VoltPortVnetsToDelete)})
+	for vpv := range va.VoltPortVnetsToDelete {
+		if vpv.Device == device {
+			logger.Warnw(ctx, "Triggering Pending VPv flow delete", log.Fields{"Port": vpv.Port, "Device": vpv.Device, "Vnet": vpv.VnetName})
+			va.DelVnetFromPort(vpv.Port, vpv)
+		}
+	}
+}
+
+//TriggerPendingVnetDeleteReq - trigger pending vnet delete request
+func (va *VoltApplication) TriggerPendingVnetDeleteReq(device string) {
+
+	logger.Warnw(ctx, "Pending Vnets to be deleted", log.Fields{"Count": len(va.VnetsToDelete)})
+	for vnetName := range va.VnetsToDelete {
+		if vnetIntf, _ := va.VnetsByName.Load(vnetName); vnetIntf != nil {
+			vnet := vnetIntf.(*VoltVnet)
+			logger.Warnw(ctx, "Triggering Pending Vnet flows delete", log.Fields{"Vnet": vnet.Name})
+			if d := va.GetDeviceBySerialNo(vnet.PendingDeviceToDelete); d != nil && d.SerialNum == vnet.PendingDeviceToDelete {
+				va.DeleteDevFlowForVlanFromDevice(vnet, vnet.PendingDeviceToDelete)
+				va.deleteVnetConfig(vnet)
+			} else {
+				logger.Warn(ctx, "Vnet Delete Failed : Device Not Found", log.Fields{"Vnet": vnet.Name, "Device": vnet.PendingDeviceToDelete})
+			}
+		}
+	}
+}
diff --git a/internal/pkg/application/dhcprelay.go b/internal/pkg/application/dhcprelay.go
new file mode 100644
index 0000000..06aa3db
--- /dev/null
+++ b/internal/pkg/application/dhcprelay.go
@@ -0,0 +1,1343 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"encoding/hex"
+	"errors"
+	"net"
+	"sync"
+
+	"github.com/google/gopacket"
+	"github.com/google/gopacket/layers"
+
+	cntlr "voltha-go-controller/internal/pkg/controller"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/util"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// DhcpRelayState type
+type DhcpRelayState uint8
+
+const (
+	// DhcpRelayStateNone constant
+	DhcpRelayStateNone DhcpRelayState = iota
+	// DhcpRelayStateDiscover constant
+	DhcpRelayStateDiscover
+	// DhcpRelayStateOffer constant
+	DhcpRelayStateOffer
+	// DhcpRelayStateRequest constant
+	DhcpRelayStateRequest
+	// DhcpRelayStateAck constant
+	DhcpRelayStateAck
+	// DhcpRelayStateNAK constant
+	DhcpRelayStateNAK
+	// DhcpRelayStateRelease constant
+	DhcpRelayStateRelease
+)
+
+// RemoteIDType represents data type for various RemoteID types
+type RemoteIDType string
+
+// List of RemoteID types supported
+const (
+        MACAddress      RemoteIDType = "MAC_ADDRESS"
+        CustomRemotedID RemoteIDType = "Custom"
+)
+
+// MaxLenDhcpv6DUID constant
+const MaxLenDhcpv6DUID = 130 // 2: DUID-Type, 128: MaxLen of DUID value
+
+// opt82 constant
+const opt82 = 82
+
+// Dhcpv6RelayState type
+type Dhcpv6RelayState uint8
+
+const (
+	// Dhcpv6RelayStateNone constant
+	Dhcpv6RelayStateNone Dhcpv6RelayState = iota
+	// Dhcpv6RelayStateSolicit constant
+	Dhcpv6RelayStateSolicit
+	// Dhcpv6RelayStateReply constant
+	Dhcpv6RelayStateReply
+	// Dhcpv6RelayStateRelease constant
+	Dhcpv6RelayStateRelease
+)
+
+var (
+	// ErrSessionDoNotExist error type
+	ErrSessionDoNotExist = errors.New("Session Doesn't Exist")
+)
+
+// IDhcpRelaySession to get dhcp session field value
+type IDhcpRelaySession interface {
+	GetCircuitID() []byte
+	GetRemoteID() []byte
+	GetNniVlans() (uint16, uint16)
+	GetDhcpState() DhcpRelayState
+	GetDhcpv6State() Dhcpv6RelayState
+	SetDhcpState(DhcpRelayState)
+	SetDhcpv6State(Dhcpv6RelayState)
+	SetMacAddr(net.HardwareAddr)
+	DhcpResultInd(*layers.DHCPv4)
+	Dhcpv6ResultInd(ipv6Addr net.IP, leaseTime uint32)
+}
+
+// DhcpRelayVnet : The DHCP relay sessions are stored in a map to be retrieved from when
+// a response is received from the network. The map uses the VLANs and the
+// the MAC address as key to finding the service
+// DHCP Relay Virtual Network hosts a set of DHCP relay sessions that belong
+// to the network. It supports two VLANs as its identify. If a single VLAN or
+// no VLAN is to be used, those two should be passed as 4096 (VlanNone)
+type DhcpRelayVnet struct {
+	OuterVlan   uint16
+	InnerVlan   uint16
+	sessions    map[[6]byte]IDhcpRelaySession
+	sessionsv6  map[[MaxLenDhcpv6DUID]byte]IDhcpRelaySession
+	sessionLock sync.RWMutex
+}
+
+// DhcpNetworks hosts different DHCP networks that in turn hold the DHCP
+// sessions
+type DhcpNetworks struct {
+	Networks map[uint32]*DhcpRelayVnet
+}
+
+func init() {
+	RegisterPacketHandler(DHCPv4, ProcessUDP4Packet)
+	RegisterPacketHandler(DHCPv6, ProcessUDP6Packet)
+}
+
+// NewDhcpRelayVnet is constructor for a DHCP Relay Virtual network
+func NewDhcpRelayVnet(outerVlan uint16, innerVlan uint16) *DhcpRelayVnet {
+	var drv DhcpRelayVnet
+
+	drv.OuterVlan = outerVlan
+	drv.InnerVlan = innerVlan
+	drv.sessions = make(map[[6]byte]IDhcpRelaySession)
+	drv.sessionsv6 = make(map[[MaxLenDhcpv6DUID]byte]IDhcpRelaySession)
+	return &drv
+}
+
+// GetDhcpVnet to add dhcp vnet
+func (dn *DhcpNetworks) GetDhcpVnet(outerVlan uint16, innerVlan uint16) *DhcpRelayVnet {
+	comboVlan := uint32(outerVlan)<<16 + uint32(innerVlan)
+	drv, ok := dn.Networks[comboVlan]
+	if ok {
+		return drv
+	}
+	return nil
+}
+
+// AddDhcpVnet to add dhcp vnet
+func (dn *DhcpNetworks) AddDhcpVnet(outerVlan uint16, innerVlan uint16) *DhcpRelayVnet {
+	comboVlan := uint32(outerVlan)<<16 + uint32(innerVlan)
+	if drv, ok := dn.Networks[comboVlan]; ok {
+		return drv
+	}
+	drv := NewDhcpRelayVnet(outerVlan, innerVlan)
+	dn.Networks[comboVlan] = drv
+	return drv
+}
+
+// NewDhcpNetworks to get new dhcp network
+func NewDhcpNetworks() *DhcpNetworks {
+	var dn DhcpNetworks
+	dn.Networks = make(map[uint32]*DhcpRelayVnet)
+	return &dn
+}
+
+// AddDhcpSession to add dhcp session
+func (dn *DhcpNetworks) AddDhcpSession(pkt gopacket.Packet, session IDhcpRelaySession) error {
+	var key [6]byte
+	ethl := pkt.Layer(layers.LayerTypeEthernet)
+	eth, _ := ethl.(*layers.Ethernet)
+	addr := eth.SrcMAC
+	if len(addr) != 6 {
+		logger.Errorw(ctx, "Invalid MAC address", log.Fields{"Addr": addr})
+		return errors.New("Invalid MAC address")
+	}
+	copy(key[:], addr[0:6])
+
+	drv := dn.AddDhcpVnet(session.GetNniVlans())
+
+	drv.sessionLock.Lock()
+	drv.sessions[key] = session
+	drv.sessionLock.Unlock()
+	return nil
+}
+
+// DelDhcpSession to delete dhcp session
+func (dn *DhcpNetworks) DelDhcpSession(pkt gopacket.Packet, session IDhcpRelaySession) {
+	var key [6]byte
+	ethl := pkt.Layer(layers.LayerTypeEthernet)
+	eth, _ := ethl.(*layers.Ethernet)
+	addr := eth.SrcMAC
+	if len(addr) != 6 {
+		logger.Errorw(ctx, "Invalid MAC address", log.Fields{"Addr": addr})
+		return
+	}
+	copy(key[:], addr[0:6])
+	drv := dn.AddDhcpVnet(session.GetNniVlans())
+	drv.sessionLock.Lock()
+	delete(drv.sessions, key)
+	drv.sessionLock.Unlock()
+}
+
+// delDhcpSessions to delete dhcp sessions
+func delDhcpSessions(addr net.HardwareAddr, outervlan of.VlanType, innervlan of.VlanType, sessionKey [MaxLenDhcpv6DUID]byte) {
+	var key [6]byte
+	if addr == nil || !NonZeroMacAddress(addr) {
+		logger.Warnw(ctx, "Invalid MAC address", log.Fields{"Addr": addr})
+		return
+	}
+	copy(key[:], addr[0:6])
+	drv := dhcpNws.AddDhcpVnet(uint16(outervlan), uint16(innervlan))
+	drv.sessionLock.Lock()
+	delete(drv.sessions, key)
+	delete(drv.sessionsv6, sessionKey)
+	drv.sessionLock.Unlock()
+	logger.Infow(ctx, "DHCP Sessions deleted", log.Fields{"MAC": addr})
+}
+
+// AddDhcp6Session to add dhcpv6 session
+func (dn *DhcpNetworks) AddDhcp6Session(key [MaxLenDhcpv6DUID]byte, session IDhcpRelaySession) error {
+	outerVlan, innerVlan := session.GetNniVlans()
+	logger.Infow(ctx, "Adding Session", log.Fields{"outerVlan": outerVlan, "innerVlan": innerVlan, "Addr": key})
+	drv := dn.AddDhcpVnet(outerVlan, innerVlan)
+	drv.sessionLock.Lock()
+	drv.sessionsv6[key] = session
+	drv.sessionLock.Unlock()
+	return nil
+}
+
+// DelDhcp6Session to delete dhcpv6 session
+func (dn *DhcpNetworks) DelDhcp6Session(key [MaxLenDhcpv6DUID]byte, session IDhcpRelaySession) {
+	outerVlan, innerVlan := session.GetNniVlans()
+	logger.Infow(ctx, "Get Session", log.Fields{"OuterVLAN": outerVlan, "InnerVLAN": innerVlan, "Addr": key})
+	drv := dn.GetDhcpVnet(outerVlan, innerVlan)
+	drv.sessionLock.Lock()
+	delete(drv.sessionsv6, key)
+	drv.sessionLock.Unlock()
+}
+
+// GetDhcpSession to get dhcp session info
+func (dn *DhcpNetworks) GetDhcpSession(outerVlan uint16, innerVlan uint16, addr net.HardwareAddr) (IDhcpRelaySession, error) {
+	var key [6]byte
+	if len(addr) != 6 {
+		logger.Errorw(ctx, "Invalid MAC address", log.Fields{"Addr": addr})
+		return nil, errors.New("Invalid MAC address")
+	}
+	copy(key[:], addr[0:6])
+	drv := dn.AddDhcpVnet(outerVlan, innerVlan)
+	drv.sessionLock.RLock()
+	defer drv.sessionLock.RUnlock()
+	if session, ok := drv.sessions[key]; ok {
+		return session, nil
+	}
+	return nil, ErrSessionDoNotExist
+}
+
+// GetDhcp6Session to get Dhcp6Session
+func (dn *DhcpNetworks) GetDhcp6Session(outerVlan uint16, innerVlan uint16, key [MaxLenDhcpv6DUID]byte) (IDhcpRelaySession, error) {
+	logger.Infow(ctx, "Locating Session", log.Fields{"OuterVlan": outerVlan, "InnerVlan": innerVlan, "key": key})
+
+	drv := dn.AddDhcpVnet(outerVlan, innerVlan)
+	drv.sessionLock.RLock()
+	defer drv.sessionLock.RUnlock()
+	if session, ok := drv.sessionsv6[key]; ok {
+		return session, nil
+	}
+	return nil, ErrSessionDoNotExist
+}
+
+// GetVlansFromPacket to get vlans from the packet
+func GetVlansFromPacket(pkt gopacket.Packet) (innerVlan of.VlanType, outerVlan of.VlanType) {
+
+	vlans := GetVlans(pkt)
+	if len(vlans) == 1 {
+		outerVlan = vlans[0]
+		innerVlan = of.VlanNone
+	} else if len(vlans) == 0 {
+		innerVlan = of.VlanNone
+		outerVlan = of.VlanNone
+	} else {
+		innerVlan = vlans[1]
+		outerVlan = vlans[0]
+	}
+	return
+}
+
+// GetVnetForV4Nni to get vnet for v4 Nni
+func GetVnetForV4Nni(dhcp *layers.DHCPv4, cvlan of.VlanType, svlan of.VlanType, pbit uint8) ([]*VoltPortVnet, error) {
+	var err error
+	var session IDhcpRelaySession
+	var vpvList []*VoltPortVnet
+	logger.Infow(ctx, "Mac Obtained MAC: ", log.Fields{"Addr": dhcp.ClientHWAddr})
+	session, err = dhcpNws.GetDhcpSession(uint16(svlan), uint16(cvlan), dhcp.ClientHWAddr)
+
+	if session != nil {
+		vpv, ok := session.(*VoltPortVnet)
+		logger.Infow(ctx, "Session Exist: VPV found", log.Fields{"VPV": vpv})
+		if ok {
+			vpvList = append(vpvList, vpv)
+			return vpvList, nil
+		}
+	}
+
+	if err == ErrSessionDoNotExist {
+		//No DHCP Session found, find matching VPV to send the packet out
+		logger.Info(ctx, "Session Doesnt Exist: Finding matching VPV")
+		return GetApplication().GetVpvsForDsPkt(cvlan, svlan, dhcp.ClientHWAddr, pbit)
+	}
+	return nil, errors.New("The session retrieved of wrong type")
+}
+
+// GetVnetForV6Nni to get vnet for v6 Nni
+func GetVnetForV6Nni(dhcp *layers.DHCPv6, cvlan of.VlanType, svlan of.VlanType,
+	pbit uint8, clientMAC net.HardwareAddr) ([]*VoltPortVnet, net.HardwareAddr, error) {
+	var err error
+	var session IDhcpRelaySession
+	var vpvList []*VoltPortVnet
+
+	var sessionKey [MaxLenDhcpv6DUID]byte
+
+	clientDuid, decodedDuid := getDhcpv6ClientDUID(dhcp)
+	if clientDuid == nil || decodedDuid == nil {
+		copy(sessionKey[:], clientMAC)
+	} else {
+		copy(sessionKey[:], clientDuid[0:])
+		if decodedDuid.Type == layers.DHCPv6DUIDTypeLLT || decodedDuid.Type == layers.DHCPv6DUIDTypeLL {
+			clientMAC = decodedDuid.LinkLayerAddress
+		}
+	}
+	session, err = dhcpNws.GetDhcp6Session(uint16(svlan), uint16(cvlan), sessionKey)
+	if session != nil {
+		vpv, ok := session.(*VoltPortVnet)
+		logger.Infow(ctx, "Session Exist: VPV found", log.Fields{"VPV": vpv})
+		if ok {
+			vpvList = append(vpvList, vpv)
+			return vpvList, clientMAC, nil
+		}
+	}
+
+	if err == ErrSessionDoNotExist {
+		//No DHCP Session found, find matching VPV to send the packet out
+		logger.Info(ctx, "Session Doesnt Exist: Finding matching VPV")
+		vpvList, err := GetApplication().GetVpvsForDsPkt(cvlan, svlan, clientMAC, pbit)
+		return vpvList, clientMAC, err
+	}
+	return nil, clientMAC, errors.New("The session retrieved of wrong type")
+}
+
+/*
+// getDhcpv4ClientMacAddr to get mac address for dhcpv4 client
+func getDhcpv4ClientMacAddr(pkt gopacket.Packet) net.HardwareAddr {
+	dhcp := pkt.Layer(layers.LayerTypeDHCPv4).(*layers.DHCPv4)
+	logger.Infow(ctx, "Mac Obtained v4: ", log.Fields{"Addr": dhcp.ClientHWAddr})
+	return dhcp.ClientHWAddr
+}
+
+// getDhcpv6ClientMacAddr to get mac address for dhcpv6 client
+func getDhcpv6ClientMacAddr(dhcpv6 *layers.DHCPv6) net.HardwareAddr {
+	var cID layers.DHCPv6Option
+	for _, option := range dhcpv6.Options {
+		if option.Code == layers.DHCPv6OptClientID {
+			cID = option
+		}
+	}
+	duid := &layers.DHCPv6DUID{}
+
+	//If cID is not found, DecodeFromBytes() returns error on empty cID
+	if err := duid.DecodeFromBytes(cID.Data); err == nil {
+		logger.Infow(ctx, "Mac Obtained v6: ", log.Fields{"Addr": duid.LinkLayerAddress, "Option": cID.String()})
+		return duid.LinkLayerAddress
+	}
+	return nil
+}*/
+
+// getDhcpv6ClientDUID to get Dhcpv6 client DUID
+func getDhcpv6ClientDUID(dhcpv6 *layers.DHCPv6) ([]byte, *layers.DHCPv6DUID) {
+
+	for _, option := range dhcpv6.Options {
+		logger.Debugw(ctx, "DHCPv6 Options", log.Fields{"option": option.Code})
+		if option.Code == layers.DHCPv6OptClientID {
+			duid := &layers.DHCPv6DUID{}
+			err := duid.DecodeFromBytes(option.Data)
+			if err == nil {
+				logger.Infow(ctx, "ClientIdentifier", log.Fields{"DUID": duid, "Option": option.String()})
+				duidLen := len(option.Data)
+				if duidLen > 130 {
+					duidLen = 130
+				}
+				return option.Data[0:duidLen], duid
+			}
+			logger.Errorw(ctx, "Client DUID decode failed", log.Fields{"error": err})
+			break
+		}
+	}
+	logger.Error(ctx, "Client DUID is not present in the packet")
+	return nil, nil
+}
+
+// AddDhcpv4Option82 : DHCPv4 packet operations
+// Addition of DHCP Option 82 which codes circuit-id and remote-id
+// into the packet. This happens as the request is relayed to the
+// DHCP servers on the NNI
+func AddDhcpv4Option82(svc *VoltService, rID []byte, dhcpv4 *layers.DHCPv4) {
+	//NOTE : both cID and rID should not be empty if this function is called
+	cID := svc.GetCircuitID()
+	var data []byte
+	if len(cID) != 0 {
+		data = append(data, 0x01)
+		data = append(data, byte(len(cID)))
+		data = append(data, cID...)
+	}
+	if len(rID) != 0 {
+		data = append(data, 0x02)
+		data = append(data, byte(len(rID)))
+		data = append(data, rID...)
+	}
+
+	if svc.isDataRateAttrPresent() {
+		minDrUs := util.Uint32ToByte(svc.MinDataRateUs)
+		data = append(data, TYPEMINDATAUS)
+		data = append(data, byte(len(minDrUs)))
+		data = append(data, minDrUs...)
+
+		minDrDs := util.Uint32ToByte(svc.MinDataRateDs)
+		data = append(data, TYPEMINDATADS)
+		data = append(data, byte(len(minDrDs)))
+		data = append(data, minDrDs...)
+
+		maxDrUs := util.Uint32ToByte(svc.MaxDataRateUs)
+		data = append(data, TYPEMAXDATAUS)
+		data = append(data, byte(len(maxDrUs)))
+		data = append(data, maxDrUs...)
+
+		maxDrDs := util.Uint32ToByte(svc.MaxDataRateDs)
+		data = append(data, TYPEMAXDATADS)
+		data = append(data, byte(len(maxDrDs)))
+		data = append(data, maxDrDs...)
+	}
+
+	option := layers.NewDHCPOption(82, data)
+	dhcpv4.Options = append(dhcpv4.Options, option)
+}
+
+// DelOption82 : Deletion of option 82 from the packet received on the NNI interface.
+// Once the packet is received, the option 82 is stripped off and the
+// packet is forwarded towards access
+func DelOption82(dhcpv4 *layers.DHCPv4) {
+	for index, option := range dhcpv4.Options {
+		if option.Type == opt82 {
+			dhcpv4.Options = append(dhcpv4.Options[0:index], dhcpv4.Options[index+1:]...)
+			return
+		}
+	}
+}
+
+// DhcpMsgType returns the DHCP message type from the packet
+func DhcpMsgType(dhcp *layers.DHCPv4) layers.DHCPMsgType {
+	for _, option := range dhcp.Options {
+		if option.Type == layers.DHCPOptMessageType {
+			return layers.DHCPMsgType(option.Data[0])
+		}
+	}
+	return layers.DHCPMsgTypeUnspecified
+}
+
+// GetIpv4Addr returns the IP address in the DHCP reply
+func GetIpv4Addr(dhcp *layers.DHCPv4) (net.IP, int64) {
+	var leaseTime uint32
+	for _, opt := range dhcp.Options {
+		if opt.Type == layers.DHCPOptLeaseTime {
+			leaseTime = GetIPv4LeaseTime(opt)
+		}
+	}
+	return dhcp.YourClientIP, int64(leaseTime)
+}
+
+//GetIPv4LeaseTime get ip lease time
+func GetIPv4LeaseTime(opt layers.DHCPOption) uint32 {
+	return uint32(opt.Data[0])<<24 | uint32(opt.Data[1])<<16 | uint32(opt.Data[2])<<8 | uint32(opt.Data[3])
+}
+
+// GetIpv6Addr returns the IPv6 address in the DHCPv6 reply
+func GetIpv6Addr(dhcp6 *layers.DHCPv6) (net.IP, uint32) {
+	var ipv6Addr net.IP
+	var leaseTime uint32
+
+	//Check for IANA allocation, if not present, then look for IAPD allocation
+	if dhcp6.MsgType == layers.DHCPv6MsgTypeReply {
+		ipv6Addr, leaseTime = GetIANAAddress(dhcp6)
+		if ipv6Addr == nil {
+			ipv6Addr, leaseTime = GetIAPDAddress(dhcp6)
+		}
+	}
+	return ipv6Addr, leaseTime
+}
+
+// GetIANAAddress returns the IPv6 address in the DHCPv6 reply
+func GetIANAAddress(dhcp6 *layers.DHCPv6) (net.IP, uint32) {
+	var ipv6Addr net.IP
+	var leaseTime uint32
+	if dhcp6.MsgType == layers.DHCPv6MsgTypeReply {
+		for _, o := range dhcp6.Options {
+			if o.Code == layers.DHCPv6OptIANA {
+
+				iana := &layers.DHCPv6IANA{}
+				err := iana.DecodeFromBytes(o.Data)
+				if err == nil {
+					ipv6Addr = iana.IA.IPv6Addr
+					leaseTime = iana.IA.ValidLifeTime
+					logger.Debugw(ctx, "IPv6 Allocated", log.Fields{"IANA IPv6": ipv6Addr})
+					return ipv6Addr, leaseTime
+				}
+				logger.Warn(ctx, "Decode of IANA Failed", log.Fields{"Reason": err.Error()})
+				break
+			}
+		}
+	}
+	return nil, 0
+}
+
+// GetIAPDAddress returns the IPv6 address in the DHCPv6 reply
+func GetIAPDAddress(dhcp6 *layers.DHCPv6) (net.IP, uint32) {
+	var ipv6Addr net.IP
+	var leaseTime uint32
+	if dhcp6.MsgType == layers.DHCPv6MsgTypeReply {
+		for _, o := range dhcp6.Options {
+			if o.Code == layers.DHCPv6OptIAPD {
+
+				iapd := &layers.DHCPv6IAPD{}
+				if err := iapd.DecodeFromBytes(o.Data); err == nil {
+					ipv6Addr = iapd.PD.Prefix
+					leaseTime = iapd.PD.ValidLifeTime
+					logger.Debugw(ctx, "IPv6 Allocated", log.Fields{"IAPD IPv6": ipv6Addr})
+					break
+				} else {
+					logger.Warn(ctx, "Decode of IAPD Failed", log.Fields{"Reason": err.Error()})
+					break
+				}
+			}
+		}
+	}
+	return ipv6Addr, leaseTime
+}
+
+// ProcessDsDhcpv4Packet : DHCPv4 packet processor functions
+// This function processes DS DHCP packet received on the NNI port.
+// The services are attached to the access ports. Thus, the DHCP
+// session is derived from the list of DHCP sessions stored in the
+// common map. The key for retrieval includes the VLAN tags in the
+// the packet and the MAC address of the client.
+func (va *VoltApplication) ProcessDsDhcpv4Packet(device string, port string, pkt gopacket.Packet) {
+
+	// Retrieve the layers to build the outgoing packet. It is not
+	// possible to add/remove layers to the existing packet and thus
+	// the lyayers are extracted to build the outgoing packet
+	eth := pkt.Layer(layers.LayerTypeEthernet).(*layers.Ethernet)
+	ip := pkt.Layer(layers.LayerTypeIPv4).(*layers.IPv4)
+	udp := pkt.Layer(layers.LayerTypeUDP).(*layers.UDP)
+	dhcp4 := pkt.Layer(layers.LayerTypeDHCPv4).(*layers.DHCPv4)
+	msgType := DhcpMsgType(dhcp4)
+
+	// Need to locate the service from the packet alone as the services
+	// are not attached to NNI port. The service is stored on DHCP relay
+	// application
+	logger.Infow(ctx, "Processing Southbound DS DHCPv4 packet", log.Fields{"Port": port, "Type": msgType})
+
+	// Retrieve the priority and drop eligible flags from the
+	// packet received
+	var priority uint8
+	var dsPbit uint8
+	var dropEligible bool
+	dot1ql := pkt.Layer(layers.LayerTypeDot1Q)
+	if dot1ql != nil {
+		dot1q := dot1ql.(*layers.Dot1Q)
+		priority = dot1q.Priority
+		dropEligible = dot1q.DropEligible
+	}
+
+	pktInnerlan, pktOuterlan := GetVlansFromPacket(pkt)
+	vpvList, _ := GetVnetForV4Nni(dhcp4, pktInnerlan, pktOuterlan, priority)
+	if len(vpvList) == 0 {
+		logger.Warn(ctx, "VNET couldn't be found for NNI")
+		return
+	}
+
+	// The DHCP option 82, if it exists is removed from the packet
+	DelOption82(dhcp4)
+	ipAddr, leaseTime := GetIpv4Addr(dhcp4)
+
+	for _, vpv := range vpvList {
+		dsPbit = vpv.GetRemarkedPriority(priority)
+		// Raise DHCP ACK/NCK indication
+		if vpv.DhcpRelay {
+			// Inform dhcp response information to dhcp server handler
+			dhcpResponseReceived(uint16(vpv.CVlan), uint16(vpv.SVlan))
+			// Process the Ack/Nack to track to state of the IP layer of the connection
+			if msgType == layers.DHCPMsgTypeAck || msgType == layers.DHCPMsgTypeNak {
+				// Install DS HSIA flows after DHCP ACK.
+				if msgType == layers.DHCPMsgTypeAck {
+					// Voltha will push US and DS HSIA flow on receivng the DS HSIA
+					// flow installation request, VGC to update US HSIA flow with leanrt MAC.
+					// separate go rotuine is spawned to avoid drop of ACK packet
+					// as HSIA flows will be deleted if new MAC is learnt.
+					go vpv.SetMacAddr(dhcp4.ClientHWAddr)
+				}
+				vpv.DhcpResultInd(dhcp4)
+
+			}
+			raiseDHCPv4Indication(msgType, vpv, dhcp4.ClientHWAddr, ipAddr, dsPbit, device, leaseTime)
+		}
+
+		// Create the outgoing bufer and set the checksum in the packet
+		buff := gopacket.NewSerializeBuffer()
+		if err := udp.SetNetworkLayerForChecksum(ip); err != nil {
+			logger.Error(ctx, "Error in setting checksum")
+			return
+		}
+		opts := gopacket.SerializeOptions{
+			FixLengths:       true,
+			ComputeChecksums: true,
+		}
+
+		cTagType := layers.EthernetTypeIPv4
+		eth.EthernetType = layers.EthernetTypeDot1Q
+
+		var pktLayers []gopacket.SerializableLayer
+		pktLayers = append(pktLayers, eth)
+
+		var qVlans []of.VlanType
+		var qVlanLayers []gopacket.SerializableLayer
+
+		if vpv.AllowTransparent {
+			vlanThreshold := 2
+			// In case of ONU_CVLAN or OLT_SVLAN, the DS pkts have single configured vlan
+			// In case of ONU_CVLAN_OLT_SVLAN or OLT_CVLAN_OLT_SVLAN, the DS pkts have 2 configured vlan
+			// Based on that, the no. of vlans should be ignored to get only transparent vlans
+			if vpv.VlanControl == ONUCVlan || vpv.VlanControl == OLTSVlan || vpv.VlanControl == None {
+				vlanThreshold = 1
+			}
+			nxtLayer := layers.EthernetTypeDot1Q
+			if vlans := GetVlans(pkt); len(vlans) > vlanThreshold {
+				qVlans = vlans[vlanThreshold:]
+				cTagType = layers.EthernetTypeDot1Q
+			}
+			for i, qVlan := range qVlans {
+				vlan := uint16(qVlan)
+				if i == (len(qVlans) - 1) {
+					nxtLayer = layers.EthernetTypeIPv4
+				}
+				qdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: vlan, DropEligible: dropEligible, Type: nxtLayer}
+				qVlanLayers = append(qVlanLayers, qdot1q)
+			}
+		}
+		switch vpv.VlanControl {
+		case ONUCVlanOLTSVlan:
+			cdot1q := &layers.Dot1Q{Priority: dsPbit, VLANIdentifier: uint16(vpv.CVlan), DropEligible: dropEligible, Type: cTagType}
+			pktLayers = append(pktLayers, cdot1q)
+		case ONUCVlan,
+			None:
+			sdot1q := &layers.Dot1Q{Priority: dsPbit, VLANIdentifier: uint16(vpv.SVlan), DropEligible: dropEligible, Type: cTagType}
+			pktLayers = append(pktLayers, sdot1q)
+		case OLTCVlanOLTSVlan,
+			OLTSVlan:
+			udot1q := &layers.Dot1Q{Priority: dsPbit, VLANIdentifier: uint16(vpv.UniVlan), DropEligible: dropEligible, Type: cTagType}
+			pktLayers = append(pktLayers, udot1q)
+		default:
+			logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+		}
+
+		pktLayers = append(pktLayers, qVlanLayers...)
+		pktLayers = append(pktLayers, ip)
+		pktLayers = append(pktLayers, udp)
+		pktLayers = append(pktLayers, dhcp4)
+		logger.Debugw(ctx, "Layers Count", log.Fields{"Count": len(pktLayers)})
+		if err := gopacket.SerializeMultiLayers(buff, opts, pktLayers); err != nil {
+			logger.Errorw(ctx, "Packet Serialization Failed", log.Fields{"Reason": err.Error()})
+			return
+		}
+
+		if err := cntlr.GetController().PacketOutReq(device, vpv.Port, port, buff.Bytes(), false); err != nil {
+			logger.Errorw(ctx, "PacketOutReq Failed",  log.Fields{"Error" : err})
+		}
+	}
+}
+
+// raiseDHCPv4Indication process DHCPv4 packet and raise indication
+func raiseDHCPv4Indication(msgType layers.DHCPMsgType, vpv *VoltPortVnet, smac net.HardwareAddr,
+	ip net.IP, pktPbit uint8, device string, leaseTime int64) {
+
+	logger.Debugw(ctx, "Processing Dhcpv4 packet", log.Fields{"ethsrcMac": smac.String(),
+		"MacLearningInVPV": vpv.MacLearning, "MacConfigured": vpv.MacAddr, "dhcpType": msgType,
+		"vlanPriority": pktPbit, "VPVLearntMac": vpv.LearntMacAddr})
+
+	matchServiceAndRaiseInd := func(key, value interface{}) bool {
+		// walk through all svcs under vpv and match pbit with packet.
+		svc := value.(*VoltService)
+
+		if svc.IsPbitExist(of.PbitType(pktPbit)) {
+			logger.Debugw(ctx, "Matching Pbit found in service config", log.Fields{"ServiceName": svc.Name, "Pbit": pktPbit})
+			return false
+		}
+		return true
+	}
+
+	switch msgType {
+	case layers.DHCPMsgTypeDiscover, layers.DHCPMsgTypeRequest:
+		if msgType == layers.DHCPMsgTypeDiscover {
+			vpv.SetDhcpState(DhcpRelayStateDiscover)
+		} else if msgType == layers.DHCPMsgTypeRequest {
+			vpv.SetDhcpState(DhcpRelayStateRequest)
+		}
+	// Reset learnt mac address in case of DHCPv4 release
+	case layers.DHCPMsgTypeRelease:
+		vpv.LearntMacAddr, _ = net.ParseMAC("00:00:00:00:00:00")
+		vpv.services.Range(matchServiceAndRaiseInd)
+		vpv.SetDhcpState(DhcpRelayStateRelease)
+
+	case layers.DHCPMsgTypeAck, layers.DHCPMsgTypeNak:
+		vpv.services.Range(matchServiceAndRaiseInd)
+		if msgType == layers.DHCPMsgTypeAck {
+			vpv.SetDhcpState(DhcpRelayStateAck)
+		} else if msgType == layers.DHCPMsgTypeNak {
+			vpv.SetDhcpState(DhcpRelayStateNAK)
+		}
+	case layers.DHCPMsgTypeOffer:
+		vpv.SetDhcpState(DhcpRelayStateOffer)
+	}
+}
+
+// raiseDHCPv6Indication process DHCPv6 packet and raise indication
+func raiseDHCPv6Indication(msgType layers.DHCPv6MsgType, vpv *VoltPortVnet,
+	smac net.HardwareAddr, ip net.IP, pktPbit uint8, device string, leaseTime uint32) {
+
+	logger.Debugw(ctx, "Processing DHCPv6 packet", log.Fields{"dhcpType": msgType,
+		"vlanPriority": pktPbit, "dhcpClientMac": smac.String(),
+		"MacLearningInVPV": vpv.MacLearning, "MacConfigured": vpv.MacAddr,
+		"VPVLearntMac": vpv.LearntMacAddr})
+
+	matchServiceAndRaiseInd := func(key, value interface{}) bool {
+		svc := value.(*VoltService)
+		if svc.IsPbitExist(of.PbitType(pktPbit)) {
+			logger.Debugw(ctx, "Matching Pbit found in service config", log.Fields{"ServiceName": svc.Name, "Pbit": pktPbit})
+			return false
+		}
+		return true
+	}
+
+	switch msgType {
+	case layers.DHCPv6MsgTypeSolicit:
+		vpv.SetDhcpv6State(Dhcpv6RelayStateSolicit)
+	// Reset learnt mac address in case of DHCPv6 release
+	case layers.DHCPv6MsgTypeRelease:
+		vpv.LearntMacAddr, _ = net.ParseMAC("00:00:00:00:00:00")
+		vpv.services.Range(matchServiceAndRaiseInd)
+		vpv.SetDhcpv6State(Dhcpv6RelayStateRelease)
+
+	case layers.DHCPv6MsgTypeReply:
+		vpv.services.Range(matchServiceAndRaiseInd)
+		vpv.SetDhcpv6State(Dhcpv6RelayStateReply)
+	}
+}
+
+// ProcessUsDhcpv4Packet : The US DHCPv4 packet is identified the DHCP OP in the packet. A request is considered upstream
+// and the service associated with the packet is located by the port and VLANs in the packet
+func (va *VoltApplication) ProcessUsDhcpv4Packet(device string, port string, pkt gopacket.Packet) {
+	// We received the packet on an access port and the service for the packet can be
+	// gotten from the port and the packet
+	vpv, svc := va.GetVnetFromPkt(device, port, pkt)
+	if vpv == nil {
+		logger.Warn(ctx, "VNET couldn't be found from packet")
+		return
+	}
+
+	outport, _ := va.GetNniPort(device)
+	if outport == "" || outport == "0" {
+		logger.Errorw(ctx, "NNI Port not found for device. Dropping Packet", log.Fields{"NNI": outport})
+		return
+	}
+
+	// Extract the layers in the packet to prepare the outgoing packet
+	// We use the layers to build the outgoing packet from scratch as
+	// the packet received can't be modified to add/remove layers
+	eth := pkt.Layer(layers.LayerTypeEthernet).(*layers.Ethernet)
+	ip := pkt.Layer(layers.LayerTypeIPv4).(*layers.IPv4)
+	udp := pkt.Layer(layers.LayerTypeUDP).(*layers.UDP)
+	dhcp4 := pkt.Layer(layers.LayerTypeDHCPv4).(*layers.DHCPv4)
+	msgType := DhcpMsgType(dhcp4)
+	logger.Infow(ctx, "Processing Southbound US DHCPv4 packet", log.Fields{"Device": device, "Port": port, "Type": msgType})
+
+	// Learn the 8021P values from the packet received
+	var priority uint8
+	var dropEligible bool
+	dot1ql := pkt.Layer(layers.LayerTypeDot1Q)
+	if dot1ql != nil {
+		dot1q := dot1ql.(*layers.Dot1Q)
+		priority = dot1q.Priority
+		dropEligible = dot1q.DropEligible
+	}
+	// If this is the first message in the DHCP sequence, the service
+	// is added to the DHCP relay application. The reply packets locate
+	// the associated service/session from the relay application.
+	if msgType == layers.DHCPMsgTypeDiscover || msgType == layers.DHCPMsgTypeRequest {
+		if err := dhcpNws.AddDhcpSession(pkt, vpv); err != nil {
+			logger.Errorw(ctx, "Adding dhcp session failed", log.Fields{"Error": err})
+		}
+	}
+
+	// Raise mac-learnt(DHCP Discover) indication when mac learning is enabled and learnt mac
+	// is not same as received mac address. If mac learning disabled, we have mac address in the
+	// service configuration. Hence mac learnt indication is not raised
+	// Reset learnt mac address in case of DHCP release and raise the indication
+	if vpv.DhcpRelay {
+		// If this is the first message in the DHCP sequence, the service
+		// is added to the DHCP relay application. The reply packets locate
+		// the associated service/session from the relay application.
+		// DS HSIA flows will be added after DHCP ACK .
+		if msgType == layers.DHCPMsgTypeDiscover || msgType == layers.DHCPMsgTypeRequest {
+			if !util.MacAddrsMatch(vpv.MacAddr, dhcp4.ClientHWAddr) {
+				// MAC is different and relearning is disabled.
+				if NonZeroMacAddress(vpv.MacAddr) && vpv.MacLearning == Learn {
+					// update learnt mac for debug purpose
+					vpv.LearntMacAddr = dhcp4.ClientHWAddr
+					vpv.WriteToDb()
+					logger.Warnw(ctx, "Dropping the packet Mac relearn is disabled",
+						log.Fields{"vpv.MacAddr": vpv.MacAddr, "LearntMac": dhcp4.ClientHWAddr})
+					return
+				}
+				expectedPort := va.GetMacInPortMap(dhcp4.ClientHWAddr)
+				if expectedPort != "" && expectedPort != vpv.Port {
+					logger.Errorw(ctx, "mac-learnt-from-different-port-ignoring-dhcp-message", log.Fields{"MsgType": msgType, "ExpectedPort": expectedPort, "ReceivedPort": vpv.Port, "LearntMacAdrr": vpv.MacAddr, "NewMacAdrr": dhcp4.ClientHWAddr.String()})
+					return
+				}
+			}
+		}
+		raiseDHCPv4Indication(msgType, vpv, dhcp4.ClientHWAddr, vpv.Ipv4Addr, priority, device, 0)
+
+		// Check IsOption82Disabled flag in configuration. if true(disabled), do not add option82 into dhcpv4 header.
+		// Remote id can be custom or mac address.
+		// If remote id is custom, then add service will carry the remote id
+		// If remote id is mac address, and if mac is configured, then add service will carry the remote id
+		// If remote id is mac address, in mac learning case, then mac has to be taken from dhcp packet
+		if !svc.IsOption82Disabled {
+			var remoteID []byte
+			if svc.RemoteIDType == string(MACAddress) {
+				remoteID = []byte((dhcp4.ClientHWAddr).String())
+			} else if svc.RemoteID != nil {
+				remoteID = svc.RemoteID
+			}
+			AddDhcpv4Option82(svc, remoteID, dhcp4)
+		}
+	}
+
+	buff := gopacket.NewSerializeBuffer()
+	if err := udp.SetNetworkLayerForChecksum(ip); err != nil {
+		logger.Error(ctx, "Error in setting checksum")
+		return
+	}
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+
+	cTagType := layers.EthernetTypeIPv4
+	outerVlan, innerVlan := vpv.GetNniVlans()
+	logger.Debugw(ctx, "Vnet Vlans", log.Fields{"Svlan": outerVlan, "Cvlan": innerVlan})
+	eth.EthernetType = vpv.SVlanTpid
+
+	var pktLayers []gopacket.SerializableLayer
+	pktLayers = append(pktLayers, eth)
+
+	var qVlans []of.VlanType
+	var qVlanLayers []gopacket.SerializableLayer
+
+	if vpv.AllowTransparent {
+		nxtLayer := layers.EthernetTypeDot1Q
+		if vlans := GetVlans(pkt); len(vlans) > 1 {
+			qVlans = vlans[1:]
+			logger.Debugw(ctx, "Q Vlans", log.Fields{"Vlan List": qVlans})
+			cTagType = layers.EthernetTypeDot1Q
+		}
+		for i, qVlan := range qVlans {
+			vlan := uint16(qVlan)
+			if i == (len(qVlans) - 1) {
+				nxtLayer = layers.EthernetTypeIPv4
+			}
+			qdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: vlan, DropEligible: dropEligible, Type: nxtLayer}
+			qVlanLayers = append(qVlanLayers, qdot1q)
+		}
+	}
+	switch vpv.VlanControl {
+	case ONUCVlanOLTSVlan,
+		OLTCVlanOLTSVlan:
+		sdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: outerVlan, DropEligible: dropEligible, Type: layers.EthernetTypeDot1Q}
+		pktLayers = append(pktLayers, sdot1q)
+		cdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: innerVlan, DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, cdot1q)
+	case ONUCVlan,
+		OLTSVlan,
+		None:
+		cdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: outerVlan, DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, cdot1q)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+	}
+
+	pktLayers = append(pktLayers, qVlanLayers...)
+	pktLayers = append(pktLayers, ip)
+	pktLayers = append(pktLayers, udp)
+	pktLayers = append(pktLayers, dhcp4)
+	logger.Debugw(ctx, "Layers Count", log.Fields{"Count": len(pktLayers)})
+	if err := gopacket.SerializeMultiLayers(buff, opts, pktLayers); err != nil {
+		return
+	}
+
+	// Now the packet constructed is output towards the switch to be emitted on
+	// the NNI port
+	if err := cntlr.GetController().PacketOutReq(device, outport, port, buff.Bytes(), false); err != nil {
+		logger.Errorw(ctx, "PacketOutReq Failed",  log.Fields{"Error" : err})
+	}
+	if vpv.DhcpRelay {
+		// Inform dhcp request information to dhcp server handler
+		dhcpRequestReceived(uint16(vpv.CVlan), uint16(vpv.SVlan), eth.SrcMAC.String())
+	}
+}
+
+// ProcessUDP4Packet : CallBack function registered with application to handle DHCP packetIn
+func ProcessUDP4Packet(device string, port string, pkt gopacket.Packet) {
+	GetApplication().ProcessUDP4Packet(device, port, pkt)
+}
+
+// ProcessUDP4Packet : The packet is a UDP packet and currently only DHCP relay application is supported
+// We determine the packet direction and process it based on the direction
+func (va *VoltApplication) ProcessUDP4Packet(device string, port string, pkt gopacket.Packet) {
+	// Currently DHCP is the only application supported by the application
+	// We check for DHCP before proceeding futher. In future, this could be
+	// based on registration and the callbacks
+	dhcpl := pkt.Layer(layers.LayerTypeDHCPv4)
+	if dhcpl == nil {
+		return
+	}
+	//logger.Debugw(ctx, "Received Packet In", log.Fields{"Pkt": hex.EncodeToString(pkt.Data())})
+	dhcp4 := pkt.Layer(layers.LayerTypeDHCPv4).(*layers.DHCPv4)
+	if dhcp4.Operation == layers.DHCPOpRequest {
+		// This is treated as an upstream packet in the VOLT application
+		// as VOLT serves access subscribers who use DHCP to acquire IP
+		// address and these packets go upstream to the network
+		va.ProcessUsDhcpv4Packet(device, port, pkt)
+	} else {
+		// This is a downstream packet
+		va.ProcessDsDhcpv4Packet(device, port, pkt)
+	}
+
+}
+
+// ProcessUDP6Packet : CallBack function registered with application to handle DHCPv6 packetIn
+func ProcessUDP6Packet(device string, port string, pkt gopacket.Packet) {
+	GetApplication().ProcessUDP6Packet(device, port, pkt)
+}
+
+// ProcessUDP6Packet : As a LDRA node, we expect to see only RelayReply from the DHCP server and we always
+// pack the received request and send it to the server as a RelayForward message
+// We expect to see Solicit, Request in the most normal cases. Before the lease expires
+// we should also see Renew. However, we should always pack the US message by adding
+// additional option that identifies to the server that the DHCP packet is forwarded
+// by an LDRA node.
+func (va *VoltApplication) ProcessUDP6Packet(device string, port string, pkt gopacket.Packet) []byte {
+	dhcpl := pkt.Layer(layers.LayerTypeDHCPv6)
+	if dhcpl == nil {
+		return nil
+	}
+	logger.Infow(ctx, "Processing DHCPv6 packet", log.Fields{"Port": port})
+	dhcpv6 := dhcpl.(*layers.DHCPv6)
+	switch dhcpv6.MsgType {
+	case layers.DHCPv6MsgTypeSolicit, layers.DHCPv6MsgTypeRequest, layers.DHCPv6MsgTypeRenew,
+		layers.DHCPv6MsgTypeRelease, layers.DHCPv6MsgTypeRebind, layers.DHCPv6MsgTypeInformationRequest,
+		layers.DHCPv6MsgTypeDecline:
+		va.ProcessUsDhcpv6Packet(device, port, pkt)
+	case layers.DHCPv6MsgTypeAdvertise, layers.DHCPv6MsgTypeConfirm, layers.DHCPv6MsgTypeReconfigure:
+		logger.Warnw(ctx, "SouthBound DHCPv6 DS Messages Expected For a Relay Agent", log.Fields{"Type": dhcpv6.MsgType})
+	case layers.DHCPv6MsgTypeRelayForward:
+		logger.Warn(ctx, "As the first DHCPv6 Relay Agent, Unexpected Relay Forward")
+	case layers.DHCPv6MsgTypeRelayReply:
+		// We received a response from the server
+		va.ProcessDsDhcpv6Packet(device, port, pkt)
+	}
+	return nil
+}
+
+// GetRelayReplyBytes to get relay reply bytes
+func GetRelayReplyBytes(dhcp6 *layers.DHCPv6) []byte {
+	for _, o := range dhcp6.Options {
+		logger.Infow(ctx, "Received Option", log.Fields{"Code": o.Code})
+		if o.Code == layers.DHCPv6OptRelayMessage {
+			return o.Data
+		}
+	}
+	return nil
+}
+
+// BuildRelayFwd to build forward relay
+func BuildRelayFwd(paddr net.IP, intfID []byte, remoteID []byte, payload []byte, isOption82Disabled bool, dhcpRelay bool) *layers.DHCPv6 {
+	dhcp6 := &layers.DHCPv6{MsgType: layers.DHCPv6MsgTypeRelayForward, LinkAddr: net.ParseIP("::"), PeerAddr: []byte(paddr)}
+	dhcp6.Options = append(dhcp6.Options, layers.NewDHCPv6Option(layers.DHCPv6OptRelayMessage, payload))
+	// Check IsOption82Disabled flag in configuration. if true(disabled), do not add remoteID and circuitID into dhcpv6 header.
+	if dhcpRelay {
+		if !isOption82Disabled {
+			remote := &layers.DHCPv6RemoteId{RemoteId: remoteID}
+			if len(remoteID) != 0 {
+				dhcp6.Options = append(dhcp6.Options, layers.NewDHCPv6Option(layers.DHCPv6OptRemoteID, remote.Encode()))
+			}
+			if len(intfID) != 0 {
+				intf := &layers.DHCPv6IntfId{Data: intfID}
+				dhcp6.Options = append(dhcp6.Options, layers.NewDHCPv6Option(layers.DHCPv6OptInterfaceID, intf.Encode()))
+			}
+		}
+	}
+	return dhcp6
+}
+
+// ProcessUsDhcpv6Packet to rpocess upstream DHCPv6 packet
+func (va *VoltApplication) ProcessUsDhcpv6Packet(device string, port string, pkt gopacket.Packet) {
+	// We received the packet on an access port and the service for the packet can be
+	// gotten from the port and the packet
+	logger.Infow(ctx, "Processing Southbound US DHCPv6 packet", log.Fields{"Port": port})
+	logger.Debugw(ctx, "Packet IN", log.Fields{"Pkt": hex.EncodeToString(pkt.Data())})
+	vpv, svc := va.GetVnetFromPkt(device, port, pkt)
+	if vpv == nil {
+		logger.Warn(ctx, "VNET couldn't be found from packet")
+		return
+	}
+
+	outport, _ := va.GetNniPort(device)
+	if outport == "" || outport == "0" {
+		logger.Errorw(ctx, "NNI Port not found for device. Dropping Packet", log.Fields{"NNI": outport})
+		return
+	}
+
+	// Extract the layers in the packet to prepare the outgoing packet
+	// We use the layers to build the outgoing packet from scratch as
+	// the packet received can't be modified to add/remove layers
+	eth := pkt.Layer(layers.LayerTypeEthernet).(*layers.Ethernet)
+	ip := pkt.Layer(layers.LayerTypeIPv6).(*layers.IPv6)
+	udp := pkt.Layer(layers.LayerTypeUDP).(*layers.UDP)
+	idhcp6 := pkt.Layer(layers.LayerTypeDHCPv6).(*layers.DHCPv6)
+
+	// Remote id can be custom or mac address.
+	// If remote id is custom, then add service will carry the remote id
+	// If remote id is mac address, and if mac is configured, then add service will carry the remote id
+	// If remote id is mac address, in mac learning case, then mac has to be taken from dhcp packet
+	var remoteID []byte
+	if svc.RemoteIDType == string(MACAddress) {
+		remoteID = []byte((eth.SrcMAC).String())
+	} else if svc.RemoteID != nil {
+		remoteID = svc.RemoteID
+	}
+	dhcp6 := BuildRelayFwd(ip.SrcIP, svc.GetCircuitID(), remoteID, udp.Payload, svc.IsOption82Disabled, vpv.DhcpRelay)
+
+	var sourceMac = eth.SrcMAC
+	var sessionKey [MaxLenDhcpv6DUID]byte
+
+	clientDuid, decodedDuid := getDhcpv6ClientDUID(idhcp6)
+	if clientDuid == nil || decodedDuid == nil {
+		copy(sessionKey[:], eth.SrcMAC)
+	} else {
+		copy(sessionKey[:], clientDuid[0:])
+		if decodedDuid.Type == layers.DHCPv6DUIDTypeLLT || decodedDuid.Type == layers.DHCPv6DUIDTypeLL {
+			sourceMac = decodedDuid.LinkLayerAddress
+		}
+	}
+	// Learn the 8021P values from the packet received
+	var priority uint8
+	var dropEligible bool
+	dot1ql := pkt.Layer(layers.LayerTypeDot1Q)
+	if dot1ql != nil {
+		dot1q := dot1ql.(*layers.Dot1Q)
+		priority = dot1q.Priority
+		dropEligible = dot1q.DropEligible
+	}
+	if idhcp6.MsgType == layers.DHCPv6MsgTypeSolicit {
+		if err := dhcpNws.AddDhcp6Session(sessionKey, vpv); err != nil {
+			logger.Errorw(ctx, "Adding dhcpv6 session failed", log.Fields{"Error": err})
+		}
+		vpv.DHCPv6DUID = sessionKey
+	}
+
+	// Raise mac-learnt(DHCPv6MsgTypeSolicit) indication when mac learning is enabled and learnt mac
+	// is not same as received mac address. If mac learning disabled, we have mac address in the
+	// service configuration. Hence mac learnt indication is not raised
+	if vpv.DhcpRelay {
+		if idhcp6.MsgType == layers.DHCPv6MsgTypeSolicit {
+			if !util.MacAddrsMatch(vpv.MacAddr, sourceMac) {
+				// MAC is different and relearning is disabled.
+				if NonZeroMacAddress(vpv.MacAddr) && vpv.MacLearning == Learn {
+					// update learnt mac for debug purpose
+					vpv.LearntMacAddr = sourceMac
+					vpv.WriteToDb()
+					logger.Warnw(ctx, "Dropping the packet Mac relearn is disabled",
+						log.Fields{"vpv.MacAddr": vpv.MacAddr, "LearntMac": sourceMac})
+					return
+				}
+				expectedPort := va.GetMacInPortMap(sourceMac)
+				if expectedPort != "" && expectedPort != vpv.Port {
+					logger.Errorw(ctx, "mac-learnt-from-different-port-ignoring-dhcp-message", log.Fields{"MsgType": idhcp6.MsgType, "ExpectedPort": expectedPort, "ReceivedPort": vpv.Port, "LearntMacAdrr": vpv.MacAddr, "NewMacAdrr": sourceMac.String()})
+					return
+				}
+			}
+		}
+		raiseDHCPv6Indication(idhcp6.MsgType, vpv, sourceMac, vpv.Ipv6Addr, priority, device, 0)
+	}
+
+	// Create the buffer and the encode options for the outgoing packet
+	buff := gopacket.NewSerializeBuffer()
+	if err := udp.SetNetworkLayerForChecksum(ip); err != nil {
+		logger.Error(ctx, "Error in setting checksum")
+		return
+	}
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+
+	cTagType := layers.EthernetTypeIPv6
+	outerVlan, innerVlan := vpv.GetNniVlans()
+	eth.EthernetType = vpv.SVlanTpid
+
+	var pktLayers []gopacket.SerializableLayer
+	pktLayers = append(pktLayers, eth)
+
+	var qVlans []of.VlanType
+	var qVlanLayers []gopacket.SerializableLayer
+
+	if vpv.AllowTransparent {
+		nxtLayer := layers.EthernetTypeDot1Q
+		if vlans := GetVlans(pkt); len(vlans) > 1 {
+			qVlans = vlans[1:]
+			cTagType = layers.EthernetTypeDot1Q
+		}
+		for i, qVlan := range qVlans {
+			vlan := uint16(qVlan)
+			if i == (len(qVlans) - 1) {
+				nxtLayer = layers.EthernetTypeIPv6
+			}
+			qdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: vlan, DropEligible: dropEligible, Type: nxtLayer}
+			qVlanLayers = append(qVlanLayers, qdot1q)
+		}
+
+	}
+	switch vpv.VlanControl {
+	case ONUCVlanOLTSVlan,
+		OLTCVlanOLTSVlan:
+		sdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: outerVlan, DropEligible: dropEligible, Type: layers.EthernetTypeDot1Q}
+		pktLayers = append(pktLayers, sdot1q)
+		cdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: innerVlan, DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, cdot1q)
+	case ONUCVlan,
+		OLTSVlan,
+		None:
+		cdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: outerVlan, DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, cdot1q)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+	}
+
+	pktLayers = append(pktLayers, qVlanLayers...)
+	pktLayers = append(pktLayers, ip)
+	pktLayers = append(pktLayers, udp)
+	pktLayers = append(pktLayers, dhcp6)
+	logger.Debugw(ctx, "Layers Count", log.Fields{"Count": len(pktLayers)})
+	if err := gopacket.SerializeMultiLayers(buff, opts, pktLayers); err != nil {
+		return
+	}
+	// Now the packet constructed is output towards the switch to be emitted on
+	// the NNI port
+	if err := cntlr.GetController().PacketOutReq(device, outport, port, buff.Bytes(), false); err != nil {
+		logger.Errorw(ctx, "PacketOutReq Failed",  log.Fields{"Error" : err})
+	}
+	if vpv.DhcpRelay {
+		// Inform dhcp request information to dhcp server handler
+		dhcpRequestReceived(uint16(vpv.CVlan), uint16(vpv.SVlan), eth.SrcMAC.String())
+	}
+}
+
+// GetDhcpv6 to get dhcpv6 info
+func GetDhcpv6(payload []byte) (*layers.DHCPv6, error) {
+	pkt := gopacket.NewPacket(payload, layers.LayerTypeDHCPv6, gopacket.Default)
+	if dl := pkt.Layer(layers.LayerTypeDHCPv6); dl != nil {
+		if dhcp6, ok := dl.(*layers.DHCPv6); ok {
+			return dhcp6, nil
+		}
+	}
+	return nil, errors.New("Failed to decode DHCPv6")
+}
+
+// ProcessDsDhcpv6Packet to process downstream dhcpv6 packet
+func (va *VoltApplication) ProcessDsDhcpv6Packet(device string, port string, pkt gopacket.Packet) {
+	logger.Infow(ctx, "Processing Southbound DS DHCPv6 packet", log.Fields{"Port": port})
+	logger.Debugw(ctx, "Packet IN", log.Fields{"Pkt": hex.EncodeToString(pkt.Data())})
+
+	// Retrieve the layers to build the outgoing packet. It is not
+	// possible to add/remove layers to the existing packet and thus
+	// the lyayers are extracted to build the outgoing packet
+	// The DHCP layer is handled differently. The Relay-Reply option
+	// of DHCP is extracted and is made the UDP payload.
+	eth := pkt.Layer(layers.LayerTypeEthernet).(*layers.Ethernet)
+	ip := pkt.Layer(layers.LayerTypeIPv6).(*layers.IPv6)
+	udp := pkt.Layer(layers.LayerTypeUDP).(*layers.UDP)
+	idhcp6 := pkt.Layer(layers.LayerTypeDHCPv6).(*layers.DHCPv6)
+	//var dhcp6 *layers.DHCPv6
+	var payload []byte
+	if payload = GetRelayReplyBytes(idhcp6); payload == nil {
+		logger.Warn(ctx, "Didn't Receive RelayMessage IE")
+		return
+	}
+
+	dhcp6, err := GetDhcpv6(payload)
+	if err != nil {
+		logger.Warnw(ctx, "DHCPv6 Decode Failed", log.Fields{"Reason": err.Error()})
+		return
+	}
+
+	// Learn the 8021P values from the packet received
+	var priority uint8
+	var dsPbit uint8
+	var dropEligible bool
+	dot1ql := pkt.Layer(layers.LayerTypeDot1Q)
+	if dot1ql != nil {
+		dot1q := dot1ql.(*layers.Dot1Q)
+		priority = dot1q.Priority
+		dropEligible = dot1q.DropEligible
+	}
+
+	pktInnerlan, pktOuterlan := GetVlansFromPacket(pkt)
+	vpvList, clientMac, err := GetVnetForV6Nni(dhcp6, pktInnerlan, pktOuterlan, priority, eth.DstMAC)
+	if len(vpvList) == 0 {
+		logger.Warnw(ctx, "VNET couldn't be found for NNI", log.Fields{"Reason": err})
+		return
+	}
+
+	ipv6Addr, leaseTime := GetIpv6Addr(dhcp6)
+
+	for _, vpv := range vpvList {
+
+		dsPbit = vpv.GetRemarkedPriority(priority)
+		// Raise DHCPv6 Reply indication
+		if vpv.DhcpRelay {
+			// Inform dhcp response information to dhcp server handler
+			dhcpResponseReceived(uint16(vpv.CVlan), uint16(vpv.SVlan))
+
+			if dhcp6.MsgType == layers.DHCPv6MsgTypeReply && ipv6Addr != nil {
+				// separate go rotuine is spawned to avoid drop of ACK packet
+				// as HSIA flows will be deleted if new MAC is learnt.
+				if len(vpvList) == 1 {
+					go vpv.SetMacAddr(clientMac)
+				}
+				vpv.Dhcpv6ResultInd(ipv6Addr, leaseTime)
+			}
+			raiseDHCPv6Indication(dhcp6.MsgType, vpv, clientMac, ipv6Addr, dsPbit, device, leaseTime)
+		}
+
+		//Replace dst Port value to 546
+		udp.DstPort = 546
+		logger.Infow(ctx, "Packet Out UDP Port..", log.Fields{"UDP": udp, "Port": udp.DstPort})
+
+		// Create the buffer and the encode options for the outgoing packet
+		buff := gopacket.NewSerializeBuffer()
+		if err := udp.SetNetworkLayerForChecksum(ip); err != nil {
+			logger.Error(ctx, "Error in setting checksum")
+			return
+		}
+		opts := gopacket.SerializeOptions{
+			FixLengths:       true,
+			ComputeChecksums: true,
+		}
+
+		cTagType := layers.EthernetTypeIPv6
+		eth.EthernetType = layers.EthernetTypeDot1Q
+
+		var pktLayers []gopacket.SerializableLayer
+		pktLayers = append(pktLayers, eth)
+
+		var qVlans []of.VlanType
+		var qVlanLayers []gopacket.SerializableLayer
+
+		if vpv.AllowTransparent {
+			vlanThreshold := 2
+			// In case of ONU_CVLAN or OLT_SVLAN, the DS pkts have single configured vlan
+			// In case of ONU_CVLAN_OLT_SVLAN or OLT_CVLAN_OLT_SVLAN, the DS pkts have 2 configured vlan
+			// Based on that, the no. of vlans should be ignored to get only transparent vlans
+			if vpv.VlanControl == ONUCVlan || vpv.VlanControl == OLTSVlan || vpv.VlanControl == None {
+				vlanThreshold = 1
+			}
+			nxtLayer := layers.EthernetTypeDot1Q
+			if vlans := GetVlans(pkt); len(vlans) > vlanThreshold {
+				qVlans = vlans[vlanThreshold:]
+				cTagType = layers.EthernetTypeDot1Q
+			}
+			for i, qVlan := range qVlans {
+				vlan := uint16(qVlan)
+				if i == (len(qVlans) - 1) {
+					nxtLayer = layers.EthernetTypeIPv6
+				}
+				qdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: vlan, DropEligible: dropEligible, Type: nxtLayer}
+				qVlanLayers = append(qVlanLayers, qdot1q)
+			}
+
+		}
+		switch vpv.VlanControl {
+		case ONUCVlanOLTSVlan:
+			cdot1q := &layers.Dot1Q{Priority: dsPbit, VLANIdentifier: uint16(vpv.CVlan), DropEligible: dropEligible, Type: cTagType}
+			pktLayers = append(pktLayers, cdot1q)
+		case ONUCVlan,
+			None:
+			sdot1q := &layers.Dot1Q{Priority: dsPbit, VLANIdentifier: uint16(vpv.SVlan), DropEligible: dropEligible, Type: cTagType}
+			pktLayers = append(pktLayers, sdot1q)
+		case OLTCVlanOLTSVlan,
+			OLTSVlan:
+			udot1q := &layers.Dot1Q{Priority: dsPbit, VLANIdentifier: uint16(vpv.UniVlan), DropEligible: dropEligible, Type: cTagType}
+			pktLayers = append(pktLayers, udot1q)
+		default:
+			logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+		}
+
+		pktLayers = append(pktLayers, qVlanLayers...)
+		pktLayers = append(pktLayers, ip)
+		pktLayers = append(pktLayers, udp)
+		pktLayers = append(pktLayers, dhcp6)
+		logger.Debugw(ctx, "Layers Count", log.Fields{"Count": len(pktLayers)})
+		if err := gopacket.SerializeMultiLayers(buff, opts, pktLayers); err != nil {
+			logger.Errorw(ctx, "Packet Serialization Failed", log.Fields{"Reason": err.Error()})
+			return
+		}
+
+		if err := cntlr.GetController().PacketOutReq(device, vpv.Port, port, buff.Bytes(), false); err != nil {
+			logger.Errorw(ctx, "PacketOutReq Failed", log.Fields{"Reason": err.Error()})
+		}
+	}
+}
+
+// The DHCP relay application is maintained within the structures below
+var dhcpNws *DhcpNetworks
+
+func init() {
+	dhcpNws = NewDhcpNetworks()
+}
diff --git a/internal/pkg/application/dhcpserverhandler.go b/internal/pkg/application/dhcpserverhandler.go
new file mode 100644
index 0000000..c208910
--- /dev/null
+++ b/internal/pkg/application/dhcpserverhandler.go
@@ -0,0 +1,248 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"sync"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	dhcpTimeout uint8 = 60
+)
+
+// done channel required to gracefully stop dhcp server handler thread
+var done = make(chan bool)
+
+// dhcpServerInfo map having dhcp network as key and dhcp request response transaction as value
+var dhcpServerInfo map[dhcpServerTag]dhcpTransactionInfo
+
+// alarmsRaised is struct having array of dhcp network for which dhcp unreachable alarm raised
+var alarmsRaised alarmsRaisedInfo
+
+// mux is mutex variable used for lock unlock
+var mux sync.Mutex
+
+// StartDhcpServerHandler starts go routine periodically(every second) to verify DHCP server reachability.
+func StartDhcpServerHandler() {
+	// Intialize global dhcp map and ticker as one second
+	dhcpServerInfo = make(map[dhcpServerTag]dhcpTransactionInfo)
+	ticker := time.NewTicker(1 * time.Second)
+
+	// go routine runs checkDhcpTimeout every second and exit if done value is set.
+	go func() {
+		for {
+			select {
+			case <-done:
+				ticker.Stop()
+				return
+			case <-ticker.C:
+				mux.Lock()
+				checkDhcpTimeout()
+				mux.Unlock()
+
+			}
+		}
+	}()
+}
+
+// checkDhcpTimeout method called every second to verify dhcp timeout for each DHCP network
+func checkDhcpTimeout() {
+	// logger.Debugw(ctx, "[dhcptimeout] DHCP MAP Info", log.Fields{"Map": dhcpServerInfo})
+	for dsTag, dtInfo := range dhcpServerInfo {
+		dtInfo.decrementTimer()
+		if dtInfo.getTimer() == 0 {
+			logger.Debugw(ctx, "[dhcptimeout]Timer Expired", log.Fields{"ctag": dsTag.cTag, "stag": dsTag.sTag})
+			if dtInfo.getReceivedResponseCount() == 0 && !alarmsRaised.isexist(dsTag) {
+				alarmsRaised.add(dsTag)
+				logger.Infow(ctx, "Alarms Raised", log.Fields{"ctag": dsTag.cTag, "stag": dsTag.sTag})
+			}
+
+			// Reset helps in
+			// case 1: when 2 requests, 1 response received within timeout interval.
+			// case 2: 1 request and no response even after timeout. (Unreachable alarm raised)
+			// In both cases, reset method provides additional timeout to receive response before deleting
+			dtInfo.resetRequestResponseCount(dhcpTimeout)
+
+			// Delete dhcp entry in map and continue to process next entry if pending request set to 0
+			if dtInfo.getPendingRequestCount() == 0 {
+				delete(dhcpServerInfo, dsTag)
+				logger.Debugw(ctx, "[dhcptimeout]DhcpServerTag info removed", log.Fields{"ctag": dsTag.cTag, "stag": dsTag.sTag})
+				// logger.Debugw(ctx, "[dhcptimeout] DHCP MAP Info", log.Fields{"Map": dhcpServerInfo})
+				continue
+			}
+		}
+		// Update decremented timer value and continue loop
+		dhcpServerInfo[dsTag] = dtInfo
+	}
+}
+
+// dhcpRequestReceived called for every DHCP request received from client.
+func dhcpRequestReceived(cTag, sTag uint16, smac string) {
+	var dtInfo dhcpTransactionInfo
+	var valueExist bool
+	dsTag := newDhcpServerTag(cTag, sTag)
+
+	mux.Lock()
+	logger.Debugw(ctx, "dhcpRequestReceived", log.Fields{"ctag": cTag, "stag": sTag, "smac": smac})
+	if dtInfo, valueExist = dhcpServerInfo[dsTag]; !valueExist {
+		dtInfo = newDhcpTransactionInfo(dhcpTimeout, smac)
+		dtInfo.incrementPendingRequestCount()
+	}
+
+	// Source mac received in dhcp request is not same as dtInfo mac then
+	// Its new subscriber request, hence increment pending request count.
+	// If multiple dhcp request received with same mac are ignored.
+	if dtInfo.smac != smac {
+		dtInfo.incrementPendingRequestCount()
+	}
+
+	dhcpServerInfo[dsTag] = dtInfo
+	mux.Unlock()
+}
+
+// dhcpResponseReceived called for every DHCP response received from dhcp server.
+func dhcpResponseReceived(cTag, sTag uint16) {
+	var dtInfo dhcpTransactionInfo
+	var valueExist bool
+	dsTag := newDhcpServerTag(cTag, sTag)
+
+	mux.Lock()
+	logger.Debugw(ctx, "dhcpResponseReceived", log.Fields{"ctag": cTag, "stag": sTag})
+	if dtInfo, valueExist = dhcpServerInfo[dsTag]; !valueExist {
+		logger.Warnw(ctx, "Ignore unknown response", log.Fields{"DhcpResp": dsTag})
+		mux.Unlock()
+		return
+	}
+
+	// If already unreachable alarm raised, clear and remove from array
+	if alarmsRaised.isexist(dsTag) {
+		alarmsRaised.remove(dsTag)
+		logger.Infow(ctx, "Alarm Cleared", log.Fields{"ctag": dsTag.cTag, "stag": dsTag.sTag})
+	}
+
+	// Increments received count and decrement pending count
+	dtInfo.responseReceived()
+	logger.Debugw(ctx, "Updated dtInfo", log.Fields{"pendingReq": dtInfo.pendingRequestCount, "receivedReq": dtInfo.receivedResponseCount})
+
+	if dtInfo.getPendingRequestCount() == 0 {
+		delete(dhcpServerInfo, dsTag)
+	} else {
+		dhcpServerInfo[dsTag] = dtInfo
+	}
+	mux.Unlock()
+}
+
+// StopDhcpServerHandler stops dhcp server handler go routine
+func StopDhcpServerHandler() {
+	done <- true
+}
+
+// dhcpServerTag contains unique dhcp network information
+type dhcpServerTag struct {
+	sTag uint16
+	cTag uint16
+}
+
+func newDhcpServerTag(cTag, sTag uint16) dhcpServerTag {
+	var d dhcpServerTag
+	d.sTag = sTag
+	d.cTag = cTag
+	return d
+}
+
+// dhcpTransactionInfo contains DHCP request response transaction information.
+type dhcpTransactionInfo struct {
+	timer                 uint8
+	pendingRequestCount   uint32
+	receivedResponseCount uint32
+	previousRequestCount  uint32
+	smac                  string
+}
+
+func newDhcpTransactionInfo(timer uint8, smac string) dhcpTransactionInfo {
+	var dt dhcpTransactionInfo
+	dt.timer = timer
+	dt.smac = smac
+	return dt
+}
+
+func (dt *dhcpTransactionInfo) getTimer() uint8 {
+	return dt.timer
+}
+
+func (dt *dhcpTransactionInfo) decrementTimer() uint8 {
+	dt.timer--
+	return dt.timer
+}
+
+func (dt *dhcpTransactionInfo) getPendingRequestCount() uint32 {
+	return dt.pendingRequestCount
+}
+
+func (dt *dhcpTransactionInfo) incrementPendingRequestCount() {
+	dt.pendingRequestCount++
+}
+
+func (dt *dhcpTransactionInfo) getReceivedResponseCount() uint32 {
+	return dt.receivedResponseCount
+}
+
+func (dt *dhcpTransactionInfo) responseReceived() {
+	dt.receivedResponseCount++
+	dt.pendingRequestCount--
+}
+
+func (dt *dhcpTransactionInfo) resetRequestResponseCount(timer uint8) {
+	if dt.pendingRequestCount >= dt.previousRequestCount {
+		dt.pendingRequestCount = dt.pendingRequestCount - dt.previousRequestCount
+	}
+	dt.previousRequestCount = dt.pendingRequestCount
+	dt.receivedResponseCount = 0
+	dt.timer = timer
+}
+
+// alarmsRaisedInfo contains the all networks alarm raised information
+type alarmsRaisedInfo struct {
+	arrayInfo []dhcpServerTag
+}
+
+// add an entry into alarm raised array
+func (a *alarmsRaisedInfo) add(val dhcpServerTag) {
+	a.arrayInfo = append(a.arrayInfo, val)
+}
+
+// isexist check if entry exist in alarm raised array
+func (a *alarmsRaisedInfo) isexist(val dhcpServerTag) bool {
+	for _, srvTag := range a.arrayInfo {
+		if srvTag == val {
+			return true
+		}
+	}
+	return false
+}
+
+// remove deletes given entry from alarm raised array
+func (a *alarmsRaisedInfo) remove(val dhcpServerTag) {
+	for ind := range a.arrayInfo {
+		if a.arrayInfo[ind] == val {
+			a.arrayInfo = append(a.arrayInfo[:ind], a.arrayInfo[ind+1:]...)
+			break
+		}
+	}
+}
diff --git a/internal/pkg/application/flowevent.go b/internal/pkg/application/flowevent.go
new file mode 100644
index 0000000..4fe1a0a
--- /dev/null
+++ b/internal/pkg/application/flowevent.go
@@ -0,0 +1,192 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	infraerrorcode "voltha-go-controller/internal/pkg/errorcodes/service"
+
+	"voltha-go-controller/internal/pkg/intf"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+//Generic Framework to enabling all flow based event trigger and handling.
+//The eventMapper can be updated for dynamic func caller for future events
+
+//FlowEventType - Type of event enumeration
+type FlowEventType string
+
+//FlowEventHandler - Func prototype for flow event handling funcs
+type FlowEventHandler func(*FlowEvent, intf.FlowStatus)
+
+var eventMapper map[FlowEventType]FlowEventHandler
+
+const (
+	//EventTypeUsIgmpFlowAdded - Event type for IGMP US flow add
+	EventTypeUsIgmpFlowAdded FlowEventType = "USIgmpFlowAdded"
+	//EventTypeServiceFlowAdded - Event type for Service flow add
+	EventTypeServiceFlowAdded FlowEventType = "ServiceFlowAdded"
+	//EventTypeControlFlowAdded - Event type for Control flow add
+	EventTypeControlFlowAdded FlowEventType = "ControlFlowAdded"
+
+	//EventTypeDeviceFlowRemoved - Event type for Device flow del
+	EventTypeDeviceFlowRemoved FlowEventType = "DeviceFlowRemoved"
+	//EventTypeMcastFlowRemoved - Event type for Mcast flow del
+	EventTypeMcastFlowRemoved FlowEventType = "McastFlowRemoved"
+
+	//EventTypeServiceFlowRemoved - Event type for Service flow del
+	EventTypeServiceFlowRemoved FlowEventType = "ServiceFlowRemoved"
+	//EventTypeControlFlowRemoved - Event type for Control flow del
+	EventTypeControlFlowRemoved FlowEventType = "ControlFlowRemoved"
+)
+
+//FlowEvent - Event info for Flow event processing
+type FlowEvent struct {
+	eType     FlowEventType
+	device    string
+	cookie    string
+	eventData interface{}
+}
+
+//InitEventFuncMapper - Initialization of flow event mapper
+func InitEventFuncMapper() {
+	eventMapper = map[FlowEventType]FlowEventHandler{
+		EventTypeUsIgmpFlowAdded:    ProcessUsIgmpFlowAddEvent,
+		EventTypeControlFlowAdded:   ProcessControlFlowAddEvent,
+		EventTypeServiceFlowAdded:   ProcessServiceFlowAddEvent,
+		EventTypeControlFlowRemoved: ProcessControlFlowDelEvent,
+		EventTypeServiceFlowRemoved: ProcessServiceFlowDelEvent,
+		EventTypeDeviceFlowRemoved:  ProcessDeviceFlowDelEvent,
+		EventTypeMcastFlowRemoved:   ProcessMcastFlowDelEvent,
+	}
+}
+
+//ExecuteFlowEvent - Process flow based event triggers
+func ExecuteFlowEvent(vd *VoltDevice, cookie string, flowStatus intf.FlowStatus) bool {
+	var event interface{}
+
+	flowEventMap, err := vd.GetFlowEventRegister(flowStatus.FlowModType)
+	if err != nil {
+		logger.Debugw(ctx, "Flow event map does not exists", log.Fields{"flowMod": flowStatus.FlowModType, "Error": err})
+		return false
+	}
+	flowEventMap.MapLock.Lock()
+
+	if event, _ = flowEventMap.Get(cookie); event == nil {
+		logger.Debugw(ctx, "Event already processed or event not registered for the cookie", log.Fields{"Cookie": cookie})
+		flowEventMap.MapLock.Unlock()
+		return false
+	}
+	flowEventMap.Remove(cookie)
+	flowEventMap.MapLock.Unlock()
+	flowEvent := event.(*FlowEvent)
+	eventMapper[flowEvent.eType](flowEvent, flowStatus)
+	return true
+}
+
+//ProcessUsIgmpFlowAddEvent - Process Us Igmp Flow event trigger
+func ProcessUsIgmpFlowAddEvent(event *FlowEvent, flowStatus intf.FlowStatus) {
+
+	logger.Infow(ctx, "Processing Post Flow Add Event for US Igmp", log.Fields{"Cookie": event.cookie, "event": event})
+	vpv := event.eventData.(*VoltPortVnet)
+	if isFlowStatusSuccess(flowStatus.Status, true) {
+		vpv.services.Range(ReceiverUpInd)
+	} else {
+		vpv.IgmpFlowInstallFailure(event.cookie, flowStatus.Status, flowStatus.Reason)
+	}
+}
+
+//ProcessServiceFlowAddEvent - Process Service Flow event trigger
+func ProcessServiceFlowAddEvent(event *FlowEvent, flowStatus intf.FlowStatus) {
+
+	logger.Infow(ctx, "Processing Post Flow Add Event for Service", log.Fields{"Cookie": event.cookie, "event": event})
+	vs := event.eventData.(*VoltService)
+	if isFlowStatusSuccess(flowStatus.Status, true) {
+		vs.FlowInstallSuccess(event.cookie, flowStatus.AdditionalData)
+	} else {
+		vs.FlowInstallFailure(event.cookie, flowStatus.Status, flowStatus.Reason)
+	}
+}
+
+//ProcessControlFlowAddEvent - Process Control Flow event trigger
+func ProcessControlFlowAddEvent(event *FlowEvent, flowStatus intf.FlowStatus) {
+
+	logger.Infow(ctx, "Processing Post Flow Add Event for VPV", log.Fields{"Cookie": event.cookie, "event": event})
+	vpv := event.eventData.(*VoltPortVnet)
+	if !isFlowStatusSuccess(flowStatus.Status, true) {
+		vpv.FlowInstallFailure(event.cookie, flowStatus.Status, flowStatus.Reason)
+	}
+}
+
+//ProcessServiceFlowDelEvent - Process Service Flow event trigger
+func ProcessServiceFlowDelEvent(event *FlowEvent, flowStatus intf.FlowStatus) {
+
+	logger.Infow(ctx, "Processing Post Flow Remove Event for Service", log.Fields{"Cookie": event.cookie, "event": event})
+	vs := event.eventData.(*VoltService)
+	if isFlowStatusSuccess(flowStatus.Status, false) {
+		vs.FlowRemoveSuccess(event.cookie)
+	} else {
+		vs.FlowRemoveFailure(event.cookie, flowStatus.Status, flowStatus.Reason)
+	}
+}
+
+//ProcessControlFlowDelEvent - Process Control Flow event trigger
+func ProcessControlFlowDelEvent(event *FlowEvent, flowStatus intf.FlowStatus) {
+
+	logger.Infow(ctx, "Processing Post Flow Remove Event for VPV", log.Fields{"Cookie": event.cookie, "event": event})
+	vpv := event.eventData.(*VoltPortVnet)
+	if isFlowStatusSuccess(flowStatus.Status, false) {
+		vpv.FlowRemoveSuccess(event.cookie, event.device)
+	} else {
+		vpv.FlowRemoveFailure(event.cookie, event.device, flowStatus.Status, flowStatus.Reason)
+	}
+}
+
+//ProcessMcastFlowDelEvent - Process Control Flow event trigger
+func ProcessMcastFlowDelEvent(event *FlowEvent, flowStatus intf.FlowStatus) {
+
+	logger.Infow(ctx, "Processing Post Flow Remove Event for Mcast/Igmp", log.Fields{"Cookie": event.cookie, "event": event})
+	mvp := event.eventData.(*MvlanProfile)
+	if isFlowStatusSuccess(flowStatus.Status, false) {
+		mvp.FlowRemoveSuccess(event.cookie, event.device)
+	} else {
+		mvp.FlowRemoveFailure(event.cookie, event.device, flowStatus.Status, flowStatus.Reason)
+	}
+}
+
+//ProcessDeviceFlowDelEvent - Process Control Flow event trigger
+func ProcessDeviceFlowDelEvent(event *FlowEvent, flowStatus intf.FlowStatus) {
+
+	logger.Infow(ctx, "Processing Post Flow Remove Event for VNET", log.Fields{"Cookie": event.cookie, "event": event})
+	vnet := event.eventData.(*VoltVnet)
+	if isFlowStatusSuccess(flowStatus.Status, false) {
+		vnet.FlowRemoveSuccess(event.cookie, event.device)
+	} else {
+		vnet.FlowRemoveFailure(event.cookie, event.device, flowStatus.Status, flowStatus.Reason)
+	}
+}
+
+//TODO: Update the func or flowStatus struct once all flow status are based on NB error code
+func isFlowStatusSuccess(status uint32, flowAdd bool) bool {
+	result := false
+	errorCode := infraerrorcode.ErrorCode(status)
+
+	if errorCode == infraerrorcode.ErrOk {
+		result = true
+	} else if !flowAdd && errorCode == infraerrorcode.ErrNotExists {
+		result = true
+	}
+	return result
+}
diff --git a/internal/pkg/application/igmp.go b/internal/pkg/application/igmp.go
new file mode 100644
index 0000000..fa28c8e
--- /dev/null
+++ b/internal/pkg/application/igmp.go
@@ -0,0 +1,5275 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package application
+
+import (
+	"encoding/json"
+	"errors"
+	"net"
+	"reflect"
+	"voltha-go-controller/internal/pkg/types"
+	"strconv"
+	"strings"
+	"sync"
+	"time"
+
+	"github.com/google/gopacket"
+	"github.com/google/gopacket/layers"
+
+	cntlr "voltha-go-controller/internal/pkg/controller"
+	"voltha-go-controller/database"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/util"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	// IgmpVersion0 constant (Default init value)
+	IgmpVersion0 uint8 = 0
+	// IgmpVersion1 constant
+	IgmpVersion1 uint8 = 1
+	// IgmpVersion2 constant
+	IgmpVersion2 uint8 = 2
+	// IgmpVersion3 constant
+	IgmpVersion3 uint8 = 3
+	// MinKeepAliveInterval constant
+	MinKeepAliveInterval uint32 = 10
+	// MaxDiffKAIntervalResp constant
+	MaxDiffKAIntervalResp uint32 = 5
+	// StaticGroup constant
+	StaticGroup string = "static"
+	// DynamicGroup constant
+	DynamicGroup string = "dynamic"
+	// StaticPort constant
+	StaticPort string = "static_port"
+	// DefaultIgmpProfID constant
+	DefaultIgmpProfID = ""
+	//GroupExpiryTime - group expiry time in minutes
+	GroupExpiryTime uint32 = 15
+)
+
+const (
+	// JoinUnsuccessful constant
+	JoinUnsuccessful string = "JOIN-UNSUCCESSFUL"
+	// JoinUnsuccessfulExceededIGMPChanel constant
+	JoinUnsuccessfulExceededIGMPChanel string = "Exceeded subscriber or PON port IGMP channels threshold"
+	// JoinUnsuccessfulAddFlowGroupFailed constant
+	JoinUnsuccessfulAddFlowGroupFailed string = "Failed to add flow or group for a channel"
+	// JoinUnsuccessfulGroupNotConfigured constant
+	JoinUnsuccessfulGroupNotConfigured string = "Join received from a subscriber on non-configured group"
+	// JoinUnsuccessfulVlanDisabled constant
+	JoinUnsuccessfulVlanDisabled string = "Vlan is disabled"
+	// JoinUnsuccessfulDescription constant
+	JoinUnsuccessfulDescription string = "igmp join unsuccessful"
+	// QueryExpired constant
+	QueryExpired string = "QUERY-EXPIRED"
+	// QueryExpiredGroupSpecific constant
+	QueryExpiredGroupSpecific string = "Group specific multicast query expired"
+	// QueryExpiredDescription constant
+	QueryExpiredDescription string = "igmp query expired"
+)
+
+// IgmpProfile structure
+type IgmpProfile struct {
+	ProfileID          string
+	UnsolicitedTimeOut uint32 //In seconds
+	MaxResp            uint32
+	KeepAliveInterval  uint32
+	KeepAliveCount     uint32
+	LastQueryInterval  uint32
+	LastQueryCount     uint32
+	FastLeave          bool
+	PeriodicQuery      bool
+	IgmpCos            uint8
+	WithRAUpLink       bool
+	WithRADownLink     bool
+	IgmpVerToServer    string
+	IgmpSourceIP       net.IP
+	Version            string
+}
+
+// McastConfig structure
+type McastConfig struct {
+	OltSerialNum   string
+	MvlanProfileID string
+	IgmpProfileID  string
+	IgmpProxyIP    net.IP
+	OperState      OperInProgress
+	Version        string
+	// This map will help in updating the igds whenever there is a igmp profile id update
+	IgmpGroupDevices sync.Map `json:"-"` // Key is group id
+}
+
+var (
+	// NullIPAddr is null ip address var
+	NullIPAddr = net.ParseIP("0.0.0.0")
+	// igmpSrcMac for the proxy
+	igmpSrcMac string
+)
+
+func init() {
+	RegisterPacketHandler(IGMP, ProcessIgmpPacket)
+}
+
+// ProcessIgmpPacket : CallBack function registered with application to handle IGMP packetIn
+func ProcessIgmpPacket(device string, port string, pkt gopacket.Packet) {
+	GetApplication().IgmpPacketInd(device, port, pkt)
+}
+
+func ipv4ToUint(ip net.IP) uint32 {
+	result := uint32(0)
+	addr := ip.To4()
+	if addr == nil {
+		logger.Warnw(ctx, "Invalid Group Addr", log.Fields{"IP": ip})
+		return 0
+	}
+	result = result + uint32(addr[0])<<24
+	result = result + uint32(addr[1])<<16
+	result = result + uint32(addr[2])<<8
+	result = result + uint32(addr[3])
+	return result
+}
+
+func getPodMacAddr() (string, error) {
+	ifas, err := net.Interfaces()
+	if err != nil {
+		return "", err
+	}
+	var ipv4Addr net.IP
+	for _, ifa := range ifas {
+		addrs, err := ifa.Addrs()
+		if err != nil {
+			return "", err
+		}
+		for _, addr := range addrs {
+			if ipv4Addr = addr.(*net.IPNet).IP.To4(); ipv4Addr != nil {
+				if ipv4Addr.IsGlobalUnicast() {
+					logger.Infow(ctx, "Igmp Static config", log.Fields{"MacAddr": ifa.HardwareAddr.String(), "ipAddr": ipv4Addr})
+					return ifa.HardwareAddr.String(), nil
+				}
+			}
+		}
+
+	}
+	return "", errors.New("MAC Address not found,Setting default")
+}
+
+// IgmpUsEthLayer : Layers defined for upstream communication
+// Ethernet layer for upstream communication
+func IgmpUsEthLayer(mcip net.IP) *layers.Ethernet {
+	eth := &layers.Ethernet{}
+	// TODO: Set the source MAC properly and remove hardcoding
+	eth.SrcMAC, _ = net.ParseMAC(igmpSrcMac)
+	eth.DstMAC, _ = net.ParseMAC("01:00:5e:00:00:00")
+	eth.DstMAC[3] = mcip[1] & 0x7f
+	eth.DstMAC[4] = mcip[2]
+	eth.DstMAC[5] = mcip[3]
+	eth.EthernetType = layers.EthernetTypeDot1Q
+	return eth
+}
+
+// IgmpUsDot1qLayer set US VLAN layer
+func IgmpUsDot1qLayer(vlan of.VlanType, priority uint8) *layers.Dot1Q {
+	dot1q := &layers.Dot1Q{}
+	dot1q.Priority = priority
+	dot1q.DropEligible = false
+	dot1q.VLANIdentifier = uint16(vlan)
+	dot1q.Type = layers.EthernetTypeIPv4
+	return dot1q
+}
+
+// Igmpv2UsIpv4Layer : Set the IP layer for IGMPv2
+// TODO - Identify correct way of obtaining source IP
+// This should be the configured IGMP proxy address which should be per OLT
+// We should probably be able to have a single function for both
+// upstream and downstream
+func Igmpv2UsIpv4Layer(src net.IP, mcip net.IP) *layers.IPv4 {
+	ip := &layers.IPv4{}
+	ip.Version = 4
+	ip.Protocol = layers.IPProtocolIGMP
+	ip.TTL = 1
+	ip.SrcIP = src
+	ip.DstIP = mcip
+	return ip
+}
+
+// Igmpv3UsIpv4Layer : Set the IP layer for IGMPv3
+// TODO - Identify correct way of obtaining source IP
+// This should be the configured IGMP proxy address which should be per OLT
+// We should probably be able to have a single function for both
+// upstream and downstream
+func Igmpv3UsIpv4Layer(src net.IP) *layers.IPv4 {
+	ip := &layers.IPv4{}
+	ip.Version = 4
+	ip.Protocol = layers.IPProtocolIGMP
+	ip.TTL = 1
+	ip.SrcIP = src
+	ip.DstIP = net.ParseIP("224.0.0.22")
+	return ip
+}
+
+// IgmpDsEthLayer : Layers defined for downstream communication
+// Ethernet layer for downstream communication
+func IgmpDsEthLayer(mcip net.IP) *layers.Ethernet {
+	eth := &layers.Ethernet{}
+	// TODO: Set the source and dest MAC properly and remove hardcoding
+	eth.SrcMAC, _ = net.ParseMAC(igmpSrcMac)
+	eth.DstMAC, _ = net.ParseMAC("01:00:5e:00:00:00")
+	eth.DstMAC[3] = mcip[1] & 0x7f
+	eth.DstMAC[4] = mcip[2]
+	eth.DstMAC[5] = mcip[3]
+	eth.EthernetType = layers.EthernetTypeDot1Q
+	return eth
+}
+
+// IgmpDsDot1qLayer set the DS VLAN layer
+func IgmpDsDot1qLayer(vlan of.VlanType, priority uint8) *layers.Dot1Q {
+	dot1q := &layers.Dot1Q{}
+	dot1q.Priority = priority
+	dot1q.DropEligible = false
+	dot1q.VLANIdentifier = uint16(vlan)
+	dot1q.Type = layers.EthernetTypeIPv4
+	return dot1q
+}
+
+// IgmpDsIpv4Layer set the IP layer
+func IgmpDsIpv4Layer(src net.IP, mcip net.IP) *layers.IPv4 {
+	ip := &layers.IPv4{}
+	ip.Version = 4
+	ip.Protocol = layers.IPProtocolIGMP
+	ip.TTL = 1
+	ip.SrcIP = src
+	if mcip.Equal(net.ParseIP("0.0.0.0")) {
+		mcip = net.ParseIP("224.0.0.1")
+	}
+	ip.DstIP = mcip
+	return ip
+}
+
+// IgmpQueryv2Layer : IGMP Query Layer
+func IgmpQueryv2Layer(mcip net.IP, resptime time.Duration) *layers.IGMPv1or2 {
+	igmp := &layers.IGMPv1or2{}
+	igmp.Type = layers.IGMPMembershipQuery
+	igmp.GroupAddress = mcip
+	igmp.MaxResponseTime = resptime
+	return igmp
+}
+
+// IgmpQueryv3Layer : IGMP v3 Query Layer
+func IgmpQueryv3Layer(mcip net.IP, resptime time.Duration) *layers.IGMP {
+	igmp := &layers.IGMP{}
+	igmp.Type = layers.IGMPMembershipQuery
+	igmp.GroupAddress = mcip
+	igmp.MaxResponseTime = resptime
+	return igmp
+}
+
+// IgmpReportv2Layer : IGMP Layer
+func IgmpReportv2Layer(mcip net.IP) *layers.IGMPv1or2 {
+	igmp := &layers.IGMPv1or2{}
+	igmp.Type = layers.IGMPMembershipReportV2
+	igmp.GroupAddress = mcip
+	return igmp
+}
+
+// IgmpLeavev2Layer : IGMP Leave Layer
+func IgmpLeavev2Layer(mcip net.IP) *layers.IGMPv1or2 {
+	igmp := &layers.IGMPv1or2{}
+	igmp.Type = layers.IGMPLeaveGroup
+	igmp.GroupAddress = mcip
+	return igmp
+}
+
+// IgmpReportv3Layer : IGMP v3 Report Layer
+func IgmpReportv3Layer(mcip net.IP, incl bool, srclist []net.IP) *layers.IGMP {
+	// IGMP base
+	igmp := &layers.IGMP{}
+	igmp.Type = layers.IGMPMembershipReportV3
+	igmp.NumberOfGroupRecords = 1
+
+	// IGMP Group
+	group := layers.IGMPv3GroupRecord{}
+	if incl {
+		group.Type = layers.IGMPIsIn
+	} else {
+		group.Type = layers.IGMPIsEx
+	}
+	group.MulticastAddress = mcip
+	group.NumberOfSources = uint16(len(srclist))
+	group.SourceAddresses = srclist
+	igmp.GroupRecords = append(igmp.GroupRecords, group)
+
+	return igmp
+}
+
+// Igmpv2QueryPacket : IGMP Query in Downstream
+func Igmpv2QueryPacket(mcip net.IP, vlan of.VlanType, selfip net.IP, pbit uint8, maxResp uint32) ([]byte, error) {
+	// Construct the layers that form the packet
+	eth := IgmpDsEthLayer(mcip)
+	dot1q := IgmpDsDot1qLayer(vlan, pbit)
+	ip := IgmpDsIpv4Layer(selfip, mcip)
+	igmp := IgmpQueryv2Layer(mcip, time.Duration(maxResp)*time.Second)
+
+	// Now prepare the buffer into which the layers are to be serialized
+	buff := gopacket.NewSerializeBuffer()
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+	if err := gopacket.SerializeLayers(buff, opts, eth, dot1q, ip, igmp); err != nil {
+		logger.Error(ctx, "Error in serializing layers")
+		return nil, err
+	}
+	return buff.Bytes(), nil
+}
+
+// Igmpv3QueryPacket : IGMPv3 Query in Downstream
+func Igmpv3QueryPacket(mcip net.IP, vlan of.VlanType, selfip net.IP, pbit uint8, maxResp uint32) ([]byte, error) {
+	// Construct the layers that form the packet
+	eth := IgmpDsEthLayer(mcip)
+	dot1q := IgmpDsDot1qLayer(vlan, pbit)
+	ip := IgmpDsIpv4Layer(selfip, mcip)
+	igmp := IgmpQueryv3Layer(mcip, time.Duration(maxResp)*time.Second)
+
+	// Now prepare the buffer into which the layers are to be serialized
+	buff := gopacket.NewSerializeBuffer()
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+	if err := gopacket.SerializeLayers(buff, opts, eth, dot1q, ip, igmp); err != nil {
+		logger.Error(ctx, "Error in serializing layers")
+		return nil, err
+	}
+	return buff.Bytes(), nil
+}
+
+// IgmpReportv2Packet : Packet - IGMP v2 report in upstream
+func IgmpReportv2Packet(mcip net.IP, vlan of.VlanType, priority uint8, selfip net.IP) ([]byte, error) {
+	// Construct the layers that form the packet
+	eth := IgmpUsEthLayer(mcip)
+	dot1q := IgmpUsDot1qLayer(vlan, priority)
+	ip := Igmpv2UsIpv4Layer(selfip, mcip)
+	igmp := IgmpReportv2Layer(mcip)
+
+	// Now prepare the buffer into which the layers are to be serialized
+	buff := gopacket.NewSerializeBuffer()
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+	if err := gopacket.SerializeLayers(buff, opts, eth, dot1q, ip, igmp); err != nil {
+		logger.Error(ctx, "Error in serializing layers")
+		return nil, err
+	}
+	return buff.Bytes(), nil
+}
+
+// Igmpv3ReportPacket : Packet - IGMP v3 report in upstream
+func Igmpv3ReportPacket(mcip net.IP, vlan of.VlanType, priority uint8, selfip net.IP, incl bool, srclist []net.IP) ([]byte, error) {
+	// Construct the layers that form the packet
+	eth := IgmpUsEthLayer(net.ParseIP("224.0.0.22").To4())
+	dot1q := IgmpUsDot1qLayer(vlan, priority)
+	ip := Igmpv3UsIpv4Layer(selfip)
+	igmp := IgmpReportv3Layer(mcip, incl, srclist)
+
+	// Now prepare the buffer into which the layers are to be serialized
+	buff := gopacket.NewSerializeBuffer()
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+	if err := gopacket.SerializeLayers(buff, opts, eth, dot1q, ip, igmp); err != nil {
+		logger.Error(ctx, "Error in serializing layers")
+		return nil, err
+	}
+	return buff.Bytes(), nil
+}
+
+// IgmpLeavePacket : Packet- IGMP Leave in upstream
+func IgmpLeavePacket(mcip net.IP, vlan of.VlanType, priority uint8, selfip net.IP) ([]byte, error) {
+	// Construct the layers that form the packet
+	eth := IgmpUsEthLayer(mcip)
+	dot1q := IgmpUsDot1qLayer(vlan, priority)
+	ip := Igmpv2UsIpv4Layer(selfip, mcip)
+	igmp := IgmpLeavev2Layer(mcip)
+
+	// Now prepare the buffer into which the layers are to be serialized
+	buff := gopacket.NewSerializeBuffer()
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+	if err := gopacket.SerializeLayers(buff, opts, eth, dot1q, ip, igmp); err != nil {
+		logger.Error(ctx, "Error in serializing layers")
+		return nil, err
+	}
+	return buff.Bytes(), nil
+}
+
+// getVersion to get igmp version type
+func getVersion(ver string) uint8 {
+	if ver == "2" || ver == "v2" {
+		return IgmpVersion2
+	}
+	return IgmpVersion3
+}
+
+// IsIPPresent is Utility to check if an IP address is in a list
+func IsIPPresent(i net.IP, ips []net.IP) bool {
+	for _, ip := range ips {
+		if i.Equal(ip) {
+			return true
+		}
+	}
+	return false
+}
+
+// IgmpGroupPort : IGMP port implements a port which is associated with an IGMP
+// version and the list of sources it implements for a given IGMP
+// channel. We may improve this to have all IGMP channels so that
+// we can implement per subscriber IGMP channel registration limits
+// As a rule a single port cannot have both include and exclude
+// lists. If we receive a include list we should purge the other
+// list which is TODO
+type IgmpGroupPort struct {
+	Port              string
+	CVlan             uint16
+	Pbit              uint8
+	Version           uint8
+	Exclude           bool
+	ExcludeList       []net.IP
+	IncludeList       []net.IP
+	QueryTimeoutCount uint32
+	PonPortID         uint32
+}
+
+// NewIgmpGroupPort is constructor for a port
+func NewIgmpGroupPort(port string, cvlan uint16, pbit uint8, version uint8, incl bool, ponPortID uint32) *IgmpGroupPort {
+	var igp IgmpGroupPort
+	igp.Port = port
+	igp.CVlan = cvlan
+	igp.Pbit = pbit
+	igp.Version = version
+	igp.Exclude = !incl
+	igp.QueryTimeoutCount = 0
+	igp.PonPortID = ponPortID
+	return &igp
+}
+
+// InclSourceIsIn checks if a source is in include list
+func (igp *IgmpGroupPort) InclSourceIsIn(src net.IP) bool {
+	return IsIPPresent(src, igp.IncludeList)
+}
+
+// ExclSourceIsIn checks if a source is in exclude list
+func (igp *IgmpGroupPort) ExclSourceIsIn(src net.IP) bool {
+	return IsIPPresent(src, igp.ExcludeList)
+}
+
+// AddInclSource adds a source is in include list
+func (igp *IgmpGroupPort) AddInclSource(src net.IP) {
+	logger.Debugw(ctx, "Adding Include Source", log.Fields{"Port": igp.Port, "Src": src})
+	igp.IncludeList = append(igp.IncludeList, src)
+}
+
+// AddExclSource adds a source is in exclude list
+func (igp *IgmpGroupPort) AddExclSource(src net.IP) {
+	logger.Debugw(ctx, "Adding Exclude Source", log.Fields{"Port": igp.Port, "Src": src})
+	igp.ExcludeList = append(igp.ExcludeList, src)
+}
+
+// DelInclSource deletes a source is in include list
+func (igp *IgmpGroupPort) DelInclSource(src net.IP) {
+	logger.Debugw(ctx, "Deleting Include Source", log.Fields{"Port": igp.Port, "Src": src})
+	for i, addr := range igp.IncludeList {
+		if addr.Equal(src) {
+			igp.IncludeList = append(igp.IncludeList[:i], igp.IncludeList[i+1:]...)
+			return
+		}
+	}
+}
+
+// DelExclSource deletes a source is in exclude list
+func (igp *IgmpGroupPort) DelExclSource(src net.IP) {
+	logger.Debugw(ctx, "Deleting Exclude Source", log.Fields{"Port": igp.Port, "Src": src})
+	for i, addr := range igp.ExcludeList {
+		if addr.Equal(src) {
+			igp.ExcludeList = append(igp.ExcludeList[:i], igp.ExcludeList[i+1:]...)
+			return
+		}
+	}
+}
+
+// WriteToDb is utility to write IGMP Group Port Info to database
+func (igp *IgmpGroupPort) WriteToDb(mvlan of.VlanType, gip net.IP, device string) error {
+	b, err := json.Marshal(igp)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutIgmpRcvr(mvlan, gip, device, igp.Port, string(b)); err1 != nil {
+		return err1
+	}
+	return nil
+}
+
+// NewIgmpGroupPortFromBytes create the IGMP group port from a byte slice
+func NewIgmpGroupPortFromBytes(b []byte) (*IgmpGroupPort, error) {
+	var igp IgmpGroupPort
+	if err := json.Unmarshal(b, &igp); err != nil {
+		logger.Warnw(ctx, "Decode of port failed", log.Fields{"str": string(b)})
+		return nil, err
+	}
+	return &igp, nil
+}
+
+// IgmpGroupChannel structure
+type IgmpGroupChannel struct {
+	Device       string
+	GroupID      uint32
+	GroupName    string
+	GroupAddr    net.IP
+	Mvlan        of.VlanType
+	Exclude      int
+	ExcludeList  []net.IP
+	IncludeList  []net.IP
+	Version      uint8
+	ServVersion  *uint8                    `json:"-"`
+	CurReceivers map[string]*IgmpGroupPort `json:"-"`
+	NewReceivers map[string]*IgmpGroupPort `json:"-"`
+	proxyCfg     **IgmpProfile
+	IgmpProxyIP  **net.IP                  `json:"-"`
+}
+
+// NewIgmpGroupChannel is constructor for a channel. The default IGMP version is set to 3
+// as the protocol defines the way to manage backward compatibility
+// The implementation handles simultaneous presense of lower versioned
+// receivers
+func NewIgmpGroupChannel(igd *IgmpGroupDevice, groupAddr net.IP, version uint8) *IgmpGroupChannel {
+	var igc IgmpGroupChannel
+	igc.Device = igd.Device
+	igc.GroupID = igd.GroupID
+	igc.GroupName = igd.GroupName
+	igc.GroupAddr = groupAddr
+	igc.Mvlan = igd.Mvlan
+	igc.Version = version
+	igc.CurReceivers = make(map[string]*IgmpGroupPort)
+	igc.NewReceivers = make(map[string]*IgmpGroupPort)
+	igc.proxyCfg = &igd.proxyCfg
+	igc.IgmpProxyIP = &igd.IgmpProxyIP
+	igc.ServVersion = igd.ServVersion
+	return &igc
+}
+
+// NewIgmpGroupChannelFromBytes create the IGMP group channel from a byte slice
+func NewIgmpGroupChannelFromBytes(b []byte) (*IgmpGroupChannel, error) {
+	var igc IgmpGroupChannel
+	if err := json.Unmarshal(b, &igc); err != nil {
+		return nil, err
+	}
+	igc.CurReceivers = make(map[string]*IgmpGroupPort)
+	igc.NewReceivers = make(map[string]*IgmpGroupPort)
+	return &igc, nil
+}
+
+// RestorePorts to restore ports
+func (igc *IgmpGroupChannel) RestorePorts() {
+
+	igc.migrateIgmpPorts()
+	ports, _ := db.GetIgmpRcvrs(igc.Mvlan, igc.GroupAddr, igc.Device)
+	for _, port := range ports {
+		b, ok := port.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		if igp, err := NewIgmpGroupPortFromBytes(b); err == nil {
+			igc.NewReceivers[igp.Port] = igp
+			logger.Infow(ctx, "Group Port Restored", log.Fields{"IGP": igp})
+		} else {
+			logger.Warn(ctx, "Failed to decode port from DB")
+		}
+	}
+	if err := igc.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group channel Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+	}
+}
+
+// WriteToDb is utility to write IGMPGroupChannel Info to database
+func (igc *IgmpGroupChannel) WriteToDb() error {
+	b, err := json.Marshal(igc)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutIgmpChannel(igc.Mvlan, igc.GroupName, igc.Device, igc.GroupAddr, string(b)); err1 != nil {
+		return err1
+	}
+	logger.Info(ctx, "IGC Updated")
+	return nil
+}
+
+// UniPortList : UNI Port list per channle has stores the UNI port list for this
+// channel.
+type UniPortList struct {
+	UNIList *util.ConcurrentMap // [UNIPort] UNIPort
+}
+
+// NewUniPortsList is Constructor for UniPortList structure
+func NewUniPortsList() *UniPortList {
+	var uniPortsList UniPortList
+
+	uniPortsList.UNIList = util.NewConcurrentMap()
+	return &uniPortsList
+}
+
+// GetUniPortCount returns the number of UNI ports subscribed to
+// current channel.
+func (uniPortsList *UniPortList) GetUniPortCount() uint64 {
+	return uniPortsList.UNIList.Length()
+}
+
+// PonPortChannels : PON port channel map keeps the active channel list and its
+// count for this group.
+type PonPortChannels struct {
+	ChannelList *util.ConcurrentMap // [channelIP]*UniPortList
+}
+
+// NewPonPortChannels is constructor for PonPortChannel.
+func NewPonPortChannels() *PonPortChannels {
+	var ponPortChannel PonPortChannels
+
+	ponPortChannel.ChannelList = util.NewConcurrentMap()
+	return &ponPortChannel
+}
+
+// GetActiveChannelCount returns the number of active channel count
+// for this pon port in the current group.
+func (ponPortChannels *PonPortChannels) GetActiveChannelCount() uint32 {
+	return uint32(ponPortChannels.ChannelList.Length())
+}
+
+// AddChannelToMap Adds new channel to the pon port map
+func (ponPortChannels *PonPortChannels) AddChannelToMap(uniPort, channel string) bool {
+
+	isNewChannel := bool(false)
+	uniList, ok := ponPortChannels.ChannelList.Get(channel)
+	if !ok {
+		// Channel doesn't exists. Adding new channel.
+		uniList = NewUniPortsList()
+		isNewChannel = true
+	}
+	uniList.(*UniPortList).UNIList.Set(uniPort, uniPort)
+	ponPortChannels.ChannelList.Set(channel, uniList)
+	return isNewChannel
+}
+
+// RemoveChannelFromMap Removed channel from the pon port map
+func (ponPortChannels *PonPortChannels) RemoveChannelFromMap(uniPort, channel string) bool {
+
+	isDeleted := bool(false)
+	uniList, ok := ponPortChannels.ChannelList.Get(channel)
+	if ok {
+		uniList.(*UniPortList).UNIList.Remove(uniPort)
+		if uniList.(*UniPortList).UNIList.Length() == 0 {
+			// Last port from the channel is removed.
+			// Removing channel from PON port map.
+			ponPortChannels.ChannelList.Remove(channel)
+			isDeleted = true
+		} else {
+			ponPortChannels.ChannelList.Set(channel, uniList)
+		}
+	} else {
+		logger.Warnw(ctx, "Channel doesn't exists in the active channels list", log.Fields{"Channel": channel})
+		return isDeleted
+	}
+	return isDeleted
+}
+
+// IgmpGroupDevice : IGMP Group Device manages the IGMP group for all listerns on
+// a single OLT. It aggregates reports received on a single group
+// and performs the count. It is responsible for sending upstream
+// report when the first listener joins and is responsible for
+// sending responses to upstream queries
+type IgmpGroupDevice struct {
+	Device            string
+	SerialNo          string
+	GroupID           uint32
+	GroupName         string
+	GroupAddr         net.IP
+	RecvVersion       uint8
+	ServVersion       *uint8
+	RecvVersionExpiry time.Time
+	ServVersionExpiry time.Time
+	Mvlan             of.VlanType
+	PonVlan           of.VlanType
+	IsPonVlanPresent  bool
+	GroupInstalled    bool
+	GroupChannels     sync.Map            `json:"-"` // [ipAddr]*IgmpGroupChannel
+	PortChannelMap    sync.Map            `json:"-"` // [portName][]net.IP
+	PonPortChannelMap *util.ConcurrentMap `json:"-"` // [ponPortId]*PonPortChannels
+	proxyCfg          *IgmpProfile                   // IgmpSrcIp from IgmpProfile is not used, it is kept for backward compatibility
+	IgmpProxyIP       *net.IP             `json:"-"`
+	NextQueryTime     time.Time
+	QueryExpiryTime   time.Time
+}
+
+// NewIgmpGroupDevice is constructor for a device. The default IGMP version is set to 3
+// as the protocol defines the way to manage backward compatibility
+// The implementation handles simultaneous presense of lower versioned
+// receivers
+func NewIgmpGroupDevice(name string, ig *IgmpGroup, id uint32, version uint8) *IgmpGroupDevice {
+	var igd IgmpGroupDevice
+	igd.Device = name
+	igd.GroupID = id
+	igd.GroupName = ig.GroupName
+	igd.GroupAddr = ig.GroupAddr
+	igd.Mvlan = ig.Mvlan
+	igd.PonVlan = ig.PonVlan
+	igd.IsPonVlanPresent = ig.IsPonVlanPresent
+	igd.GroupInstalled = false
+	igd.RecvVersion = version
+	igd.RecvVersionExpiry = time.Now()
+	igd.ServVersionExpiry = time.Now()
+	igd.PonPortChannelMap = util.NewConcurrentMap()
+
+	va := GetApplication()
+	if vd := va.GetDevice(igd.Device); vd != nil {
+		igd.SerialNo = vd.SerialNum
+	} else {
+		logger.Errorw(ctx, "Volt Device not found.  log.Fields", log.Fields{"igd.Device": igd.Device})
+		return nil
+	}
+	mvp := GetApplication().GetMvlanProfileByTag(igd.Mvlan)
+	igd.ServVersion = mvp.IgmpServVersion[igd.SerialNo]
+
+	var mcastCfg *McastConfig
+	igd.proxyCfg, igd.IgmpProxyIP, mcastCfg = getIgmpProxyCfgAndIP(ig.Mvlan, igd.SerialNo)
+
+	// mvlan profile id + olt serial number---igmp group id
+	//igmpgroup id
+	igd.NextQueryTime = time.Now().Add(time.Duration(igd.proxyCfg.KeepAliveInterval) * time.Second)
+	igd.QueryExpiryTime = time.Now().Add(time.Duration(igd.proxyCfg.KeepAliveInterval) * time.Second)
+
+	if mcastCfg != nil {
+		mcastCfg.IgmpGroupDevices.Store(id, &igd)
+		logger.Debugw(ctx, "Igd added to mcast config", log.Fields{"mvlan": mcastCfg.MvlanProfileID, "groupId": id})
+	}
+	return &igd
+}
+
+// IgmpGroupDeviceReInit is re-initializer for a device. The default IGMP version is set to 3
+// as the protocol defines the way to manage backward compatibility
+func (igd *IgmpGroupDevice) IgmpGroupDeviceReInit(ig *IgmpGroup) {
+
+	logger.Infow(ctx, "Reinitialize Igmp Group Device", log.Fields{"Device": igd.Device, "GroupID": ig.GroupID, "OldName": igd.GroupName, "Name": ig.GroupName, "OldAddr": igd.GroupAddr.String(), "GroupAddr": ig.GroupAddr.String()})
+
+	if (igd.GroupName != ig.GroupName) || !igd.GroupAddr.Equal(ig.GroupAddr) {
+		_ = db.DelIgmpDevice(igd.Mvlan, igd.GroupName, igd.GroupAddr, igd.Device)
+		igd.GroupName = ig.GroupName
+		igd.GroupAddr = ig.GroupAddr
+	}
+	igd.RecvVersionExpiry = time.Now()
+	igd.ServVersionExpiry = time.Now()
+	igd.PonPortChannelMap = util.NewConcurrentMap()
+
+	var mcastCfg *McastConfig
+	igd.proxyCfg, igd.IgmpProxyIP, mcastCfg = getIgmpProxyCfgAndIP(ig.Mvlan, igd.SerialNo)
+
+	igd.NextQueryTime = time.Now().Add(time.Duration(igd.proxyCfg.KeepAliveInterval) * time.Second)
+	igd.QueryExpiryTime = time.Now().Add(time.Duration(igd.proxyCfg.KeepAliveInterval) * time.Second)
+
+	if mcastCfg != nil {
+		mcastCfg.IgmpGroupDevices.Store(ig.GroupID, igd)
+		logger.Debugw(ctx, "Igd added to mcast config", log.Fields{"mvlan": mcastCfg.MvlanProfileID, "groupId": ig.GroupID})
+	}
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device, "GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+}
+
+func getIgmpProxyCfgAndIP(mvlan of.VlanType, serialNo string) (*IgmpProfile, *net.IP, *McastConfig) {
+	va := GetApplication()
+	mVLANProfileID := va.GetMvlanProfileByTag(mvlan).Name
+	var mcastCfg *McastConfig
+	if mcastCfg = va.GetMcastConfig(serialNo, mVLANProfileID); mcastCfg == nil || (mcastCfg != nil && mcastCfg.IgmpProfileID == "") {
+		logger.Debugw(ctx, "Default IGMP config to be used", log.Fields{"mVLANProfileID": mVLANProfileID, "OltSerialNo": serialNo})
+		igmpProf := va.getIgmpProfileMap(DefaultIgmpProfID)
+		return igmpProf, &igmpProf.IgmpSourceIP, mcastCfg
+	}
+	return va.getIgmpProfileMap(mcastCfg.IgmpProfileID), &mcastCfg.IgmpProxyIP, mcastCfg
+}
+
+// updateGroupName to update the group name
+func (igd *IgmpGroupDevice) updateGroupName(newGroupName string) {
+
+	oldName := igd.GroupName
+	igd.GroupName = newGroupName
+	updateGroupName := func(key, value interface{}) bool {
+		igc := value.(*IgmpGroupChannel)
+		igc.GroupName = newGroupName
+		if err := igc.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Igmp group channel Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+		}
+		_ = db.DelIgmpChannel(igc.Mvlan, oldName, igc.Device, igc.GroupAddr)
+		return true
+	}
+	igd.GroupChannels.Range(updateGroupName)
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device, "GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+	_ = db.DelIgmpDevice(igd.Mvlan, oldName, igd.GroupAddr, igd.Device)
+}
+
+// NewIgmpGroupDeviceFromBytes is to create the IGMP group port from a byte slice
+func NewIgmpGroupDeviceFromBytes(b []byte) (*IgmpGroupDevice, error) {
+	var igd IgmpGroupDevice
+	if err := json.Unmarshal(b, &igd); err != nil {
+		return nil, err
+	}
+	return &igd, nil
+}
+
+// GetKey to get group name as key
+func (igd *IgmpGroupDevice) GetKey() string {
+
+	if !net.ParseIP("0.0.0.0").Equal(igd.GroupAddr) {
+		return igd.GroupName + "_" + igd.GroupAddr.String()
+	}
+	return igd.GroupName
+
+}
+
+// RestoreChannel to restore channel
+func (igd *IgmpGroupDevice) RestoreChannel(igmpGroupChannel []byte) {
+
+	if igc, err := NewIgmpGroupChannelFromBytes(igmpGroupChannel); err == nil {
+		igc.ServVersion = igd.ServVersion
+		igc.IgmpProxyIP = &igd.IgmpProxyIP
+		igc.proxyCfg = &igd.proxyCfg
+		igd.GroupChannels.Store(igc.GroupAddr.String(), igc)
+		igc.RestorePorts()
+
+		for port, igp := range igc.NewReceivers {
+			ipsList := []net.IP{}
+			ipsIntf, _ := igd.PortChannelMap.Load(port)
+			if ipsIntf != nil {
+				ipsList = ipsIntf.([]net.IP)
+			}
+
+			ipsList = append(ipsList, igc.GroupAddr)
+			igd.PortChannelMap.Store(port, ipsList)
+			logger.Infow(ctx, "Group Channels Restored", log.Fields{"IGC": igc})
+			igd.AddChannelToChannelsPerPon(port, igc.GroupAddr, igp.PonPortID)
+		}
+	} else {
+		logger.Warnw(ctx, "Failed to decode port from DB", log.Fields{"err": err})
+	}
+	logger.Info(ctx, "Group Device & Channels Restored")
+	igd.PortChannelMap.Range(printPortChannel)
+	igd.GroupChannels.Range(printChannel)
+
+}
+
+// RestoreChannels to restore channels
+func (igd *IgmpGroupDevice) RestoreChannels() {
+
+	igd.migrateIgmpChannels()
+	channels, _ := db.GetIgmpChannels(igd.Mvlan, igd.GroupName, igd.Device)
+	for _, channel := range channels {
+
+		b, ok := channel.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		igd.RestoreChannel(b)
+	}
+
+}
+
+// printChannel to print channel info
+func printChannel(key interface{}, value interface{}) bool {
+	logger.Infow(ctx, "ChannelMap", log.Fields{"Channel": key.(string), "Igc": value.(*IgmpGroupChannel)})
+	return true
+}
+
+// printPortChannel to print port channel
+func printPortChannel(key interface{}, value interface{}) bool {
+	logger.Infow(ctx, "PortChannelMap", log.Fields{"Port": key.(string), "List": value.([]net.IP)})
+	return true
+}
+
+// WriteToDb is utility to write IGMP Group Device Info to the database
+func (igd *IgmpGroupDevice) WriteToDb() error {
+	b, err := json.Marshal(igd)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutIgmpDevice(igd.Mvlan, igd.GroupName, igd.GroupAddr, igd.Device, string(b)); err1 != nil {
+		return err1
+	}
+	logger.Info(ctx, "IGD Updated")
+	return nil
+}
+
+// Tick processes timing tick used to run timers within the device
+func (igd *IgmpGroupDevice) Tick() uint8 {
+	/* Not using RecvVersionExpiry as it is not used anywhere
+	if time.Now().After(igd.RecvVersionExpiry) {
+		igd.RecvVersion = IgmpVersion3
+		return true
+	}
+	*/
+	return 0
+}
+
+// GetSubscriberCountForChannelAndPonPort Gets the active subscriber count
+// for the given channel for one particular PON port
+func (igd *IgmpGroupDevice) GetSubscriberCountForChannelAndPonPort(ponPortID uint32, channelIP net.IP) uint64 {
+	if portMapIntf, ok := igd.PonPortChannelMap.Get(ponPortID); ok {
+		portChannelMap := portMapIntf.(*PonPortChannels)
+
+		if channel, present := portChannelMap.ChannelList.Get(channelIP.String()); present {
+			return channel.(*UniPortList).UNIList.Length()
+		}
+	} else {
+		logger.Warnw(ctx, "PON port not found in PortChannelMap", log.Fields{"PON": ponPortID, "channel": channelIP})
+	}
+	return 0
+}
+
+// AddChannelToChannelsPerPon Adds the new channel into the per Pon channel list
+func (igd *IgmpGroupDevice) AddChannelToChannelsPerPon(uniPort string, channelIP net.IP, ponPortID uint32) bool {
+	logger.Debugw(ctx, "Adding channel to ActiveChannelsPerPon list", log.Fields{"PonPort": ponPortID, "channelIP": channelIP})
+
+	isNewChannel := bool(false)
+	isNewReceiver := false
+	if port, ok := igd.PonPortChannelMap.Get(ponPortID); !ok {
+		// PON port not exists in igd. adding it.
+		isNewReceiver = true
+		ponPortChannels := NewPonPortChannels()
+		isNewChannel = ponPortChannels.AddChannelToMap(uniPort, channelIP.String())
+		igd.PonPortChannelMap.Set(ponPortID, ponPortChannels)
+	} else {
+		// PON port exists in igd. Appending the channel list
+		// in the PON port.
+		isNewChannel = port.(*PonPortChannels).AddChannelToMap(uniPort, channelIP.String())
+		igd.PonPortChannelMap.Set(ponPortID, port)
+		count := port.(*PonPortChannels).GetActiveChannelCount()
+
+		logger.Debugw(ctx, "activeChannelCount", log.Fields{"count": count})
+	}
+	GetApplication().UpdateActiveChannelCountForPonPort(igd.Device, uniPort, ponPortID, true, isNewChannel, igd)
+	return isNewReceiver
+}
+
+// RemoveChannelFromChannelsPerPon removes the channel from the per pon channel list.
+func (igd *IgmpGroupDevice) RemoveChannelFromChannelsPerPon(uniPort string, channelIP net.IP, ponPortID uint32) bool {
+	logger.Debugw(ctx, "Removing channel from ActiveChannelsPerPon list", log.Fields{"PonPort": ponPortID, "channelIP": channelIP})
+	var deleted bool
+	ponRemoved := false
+
+	if port, ok := igd.PonPortChannelMap.Get(ponPortID); ok {
+		channelPortMap := port.(*PonPortChannels)
+		deleted = channelPortMap.RemoveChannelFromMap(uniPort, channelIP.String())
+		if deleted && channelPortMap.ChannelList.Length() == 0 {
+			igd.PonPortChannelMap.Remove(ponPortID)
+			ponRemoved = true
+		}
+		GetApplication().UpdateActiveChannelCountForPonPort(igd.Device, uniPort, ponPortID, false, deleted, igd)
+	} else {
+		logger.Warnw(ctx, "PON port doesn't exists in the igd", log.Fields{"PonPortID": ponPortID})
+	}
+	return ponRemoved
+}
+
+// InclSourceIsIn checks if a source is in include list
+func (igc *IgmpGroupChannel) InclSourceIsIn(src net.IP) bool {
+	return IsIPPresent(src, igc.IncludeList)
+}
+
+// ExclSourceIsIn checks if a source is in exclude list
+func (igc *IgmpGroupChannel) ExclSourceIsIn(src net.IP) bool {
+	return IsIPPresent(src, igc.ExcludeList)
+}
+
+// AddInclSource adds a source is in include list
+func (igc *IgmpGroupChannel) AddInclSource(src net.IP) {
+	logger.Debugw(ctx, "Adding Include Source for Channel", log.Fields{"Channel": igc.GroupAddr.String(), "Device": igc.Device, "Src": src})
+	igc.IncludeList = append(igc.IncludeList, src)
+}
+
+// AddExclSource adds a source is in exclude list
+func (igc *IgmpGroupChannel) AddExclSource(src net.IP) {
+	logger.Debugw(ctx, "Adding Exclude Source for Channel", log.Fields{"Channel": igc.GroupAddr.String(), "Device": igc.Device, "Src": src})
+	igc.ExcludeList = append(igc.ExcludeList, src)
+}
+
+// UpdateExclSource update excl source list for the given channel
+func (igc *IgmpGroupChannel) UpdateExclSource(srcList []net.IP) bool {
+
+	logger.Debugw(ctx, "Updating Exclude Source for Channel", log.Fields{"Channel": igc.GroupAddr.String(), "Device": igc.Device, "Current List": igc.ExcludeList, "Incoming List": srcList})
+	if !igc.IsExclListChanged(srcList) {
+		return false
+	}
+
+	if igc.NumReceivers() == 1 {
+		igc.ExcludeList = srcList
+	} else {
+		igc.ExcludeList = igc.computeExclList(srcList)
+	}
+
+	logger.Debugw(ctx, "Updated Exclude Source for Channel", log.Fields{"Channel": igc.GroupAddr.String(), "Device": igc.Device, "Updated Excl List": igc.ExcludeList})
+	return true
+}
+
+// computeExclList computes intersection of pervious & current src list
+func (igc *IgmpGroupChannel) computeExclList(srcList []net.IP) []net.IP {
+
+	updatedSrcList := []net.IP{}
+	for _, src := range srcList {
+		for _, excl := range igc.ExcludeList {
+			if src.Equal(excl) {
+				updatedSrcList = append(updatedSrcList, src)
+			}
+		}
+	}
+	return updatedSrcList
+}
+
+// IsExclListChanged checks if excl list has been updated
+func (igc *IgmpGroupChannel) IsExclListChanged(srcList []net.IP) bool {
+
+	srcPresent := false
+	if len(igc.ExcludeList) != len(srcList) {
+		return true
+	}
+
+	for _, src := range srcList {
+		for _, excl := range igc.ExcludeList {
+			srcPresent = false
+			if src.Equal(excl) {
+				srcPresent = true
+				break
+			}
+		}
+		if !srcPresent {
+			return true
+		}
+	}
+	return false
+}
+
+// DelInclSource deletes a source is in include list
+func (igc *IgmpGroupChannel) DelInclSource(src net.IP) {
+	mvp := GetApplication().GetMvlanProfileByTag(igc.Mvlan)
+	/* If the SSM proxy is configured, then we can del the src ip from igc as whatever is in proxy that is final list */
+	if _, ok := mvp.Proxy[igc.GroupName]; !ok {
+		logger.Debugw(ctx, "Deleting Include Source for Channel", log.Fields{"Channel": igc.GroupAddr.String(), "Device": igc.Device, "Src": src})
+		for _, igp := range igc.CurReceivers {
+			if igp.InclSourceIsIn(src) {
+				logger.Infow(ctx, "Skipping deletion: Source Present for another Receiver", log.Fields{"Receiver": igp.Port})
+				return
+			}
+		}
+		for _, igp := range igc.NewReceivers {
+			if igp.InclSourceIsIn(src) {
+				logger.Infow(ctx, "Skipping deletion: Source Present for another Receiver", log.Fields{"Receiver": igp.Port})
+				return
+			}
+		}
+	} else {
+		logger.Debug(ctx, "Proxy configured, not Deleting Include Source for Channel")
+	}
+	for i, addr := range igc.IncludeList {
+		if addr.Equal(src) {
+			igc.IncludeList = append(igc.IncludeList[:i], igc.IncludeList[i+1:]...)
+			return
+		}
+	}
+}
+
+// DelExclSource deletes a source is in exclude list
+func (igc *IgmpGroupChannel) DelExclSource(src net.IP) {
+	logger.Debugw(ctx, "Deleting Exclude Source for Channel", log.Fields{"Channel": igc.GroupAddr.String(), "Device": igc.Device, "Src": src})
+
+	for _, igp := range igc.CurReceivers {
+		if igp.ExclSourceIsIn(src) {
+			logger.Infow(ctx, "Skipping deletion: Source Present for another Receiver", log.Fields{"Receiver": igp.Port})
+			return
+		}
+	}
+	for _, igp := range igc.NewReceivers {
+		if igp.ExclSourceIsIn(src) {
+			logger.Infow(ctx, "Skipping deletion: Source Present for another Receiver", log.Fields{"Receiver": igp.Port})
+			return
+		}
+	}
+	for i, addr := range igc.ExcludeList {
+		if addr.Equal(src) {
+			igc.ExcludeList = append(igc.ExcludeList[:i], igc.ExcludeList[i+1:]...)
+			return
+		}
+	}
+}
+
+// ProcessSources process the received list of either included sources or the excluded sources
+// The return value indicate sif the group is modified and needs to be informed
+// to the upstream multicast servers
+func (igc *IgmpGroupChannel) ProcessSources(port string, ip []net.IP, incl bool) (bool, bool) {
+	groupChanged := false
+	groupExclUpdated := false
+	receiverSrcListEmpty := false
+	// If the version type is 2, there isn't anything to process here
+	if igc.Version == IgmpVersion2 && *igc.ServVersion == IgmpVersion2 {
+		return false, false
+	}
+
+	igp := igc.GetReceiver(port)
+	if igp == nil {
+		logger.Warnw(ctx, "Receiver not found", log.Fields{"Port": port})
+		return false, false
+	}
+	mvp := GetApplication().GetMvlanProfileByTag(igc.Mvlan)
+	if incl {
+		for _, src := range ip {
+
+			if igp.ExclSourceIsIn(src) {
+				igp.DelExclSource(src)
+				if igc.ExclSourceIsIn(src) {
+					igc.DelExclSource(src)
+					groupChanged = true
+				}
+			}
+
+			// If the source is not in the list of include sources for the port
+			// add it. If so, check also if it is in list of include sources
+			// at the device level.
+			if !igp.InclSourceIsIn(src) {
+				igp.AddInclSource(src)
+				if !igc.InclSourceIsIn(src) {
+					igc.AddInclSource(src)
+					groupChanged = true
+				}
+			}
+		}
+		/* If any of the existing ip in the source list is removed we need to remove from the list in igp and igc */
+		if _, ok := mvp.Proxy[igc.GroupName]; ok {
+			/* If we get leave message from any subscriber, we do not have to delete the entries in the src list
+			   Only if ther is any modification in the src list by proxy config update only then we need to update */
+			if len(ip) != 0 && len(ip) != len(igc.IncludeList) {
+				for i := len(igc.IncludeList) - 1; i >= 0; i-- {
+					src := igc.IncludeList[i]
+					if !IsIPPresent(src, ip) {
+						igp.DelInclSource(src)
+						igc.DelInclSource(src)
+						groupChanged = true
+					}
+				}
+			}
+		}
+	} else {
+		for _, src := range ip {
+
+			if igp.InclSourceIsIn(src) {
+				igp.DelInclSource(src)
+				if igc.InclSourceIsIn(src) {
+					igc.DelInclSource(src)
+					groupChanged = true
+				}
+				if len(igp.IncludeList) == 0 {
+					receiverSrcListEmpty = true
+				}
+			}
+
+			// If the source is not in the list of exclude sources for the port
+			// add it. If so, check also if it is in list of include sources
+			// at the device level.
+			if !igp.ExclSourceIsIn(src) {
+				igp.AddExclSource(src)
+				/* If there is any update in the src list of proxy we need to update the igc */
+				if _, ok := mvp.Proxy[igc.GroupName]; ok {
+					if !igc.ExclSourceIsIn(src) {
+						igc.AddExclSource(src)
+						groupChanged = true
+					}
+				}
+			}
+		}
+		/* If any of the existing ip in the source list is removed we need to remove from the list in igp and igc */
+		if _, ok := mvp.Proxy[igc.GroupName]; ok {
+			if len(ip) != len(igc.ExcludeList) {
+				for i := len(igc.ExcludeList) - 1; i >= 0; i-- {
+					src := igc.ExcludeList[i]
+					if !IsIPPresent(src, ip) {
+						igp.DelExclSource(src)
+						igc.DelExclSource(src)
+						groupChanged = true
+					}
+				}
+			}
+		}
+		groupExclUpdated = igc.UpdateExclSource(ip)
+	}
+	if err := igp.WriteToDb(igc.Mvlan, igc.GroupAddr, igc.Device); err != nil {
+		logger.Errorw(ctx, "Igmp group port Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+	}
+	return (groupChanged || groupExclUpdated), receiverSrcListEmpty
+}
+
+// GetReceiver to get receiver info
+func (igc *IgmpGroupChannel) GetReceiver(port string) *IgmpGroupPort {
+	igp := igc.NewReceivers[port]
+	if igp == nil {
+		igp = igc.CurReceivers[port]
+	}
+	return igp
+}
+
+// AddReceiver add the receiver to the device and perform other actions such as adding the group
+// to the physical device, add members, add flows to point the MC packets to the
+// group. Also, send a IGMP report upstream if there is a change in the group
+func (igd *IgmpGroupDevice) AddReceiver(port string, groupAddr net.IP,
+	group *layers.IGMPv3GroupRecord, version uint8, cvlan uint16, pbit uint8, ponPortID uint32) {
+
+	var igc *IgmpGroupChannel
+	logger.Debugw(ctx, "Processing receiver for device", log.Fields{"Channel": groupAddr, "Port": port, "Device": igd.Device})
+
+	igcIntf, ok := igd.GroupChannels.Load(groupAddr.String())
+	if !ok {
+		igc = NewIgmpGroupChannel(igd, groupAddr, version)
+		igd.GroupChannels.Store(groupAddr.String(), igc)
+	} else {
+		igc = igcIntf.(*IgmpGroupChannel)
+	}
+
+	if !igd.GroupInstalled {
+		igd.AddNewReceiver(port, groupAddr, group, cvlan, pbit, ponPortID)
+		return
+	}
+
+	isNewReceiver := igc.AddReceiver(port, group, cvlan, pbit)
+	if isNewReceiver {
+		ipsList := []net.IP{}
+		ipsIntf, _ := igd.PortChannelMap.Load(port)
+		if ipsIntf != nil {
+			ipsList = ipsIntf.([]net.IP)
+		}
+		ipsList = append(ipsList, groupAddr)
+		igd.PortChannelMap.Store(port, ipsList)
+		logger.Debugw(ctx, "Port Channel Updated", log.Fields{"Port": port, "AddedChannelList": ipsList, "Addr": groupAddr})
+
+		isNewPonReceiver := igd.AddChannelToChannelsPerPon(port, groupAddr, ponPortID)
+		//Modify group only if this is the first time the port is subscribing for the group
+		if isNewPonReceiver {
+			igd.ModMcGroup()
+		}
+	}
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device, "GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+}
+
+// AddNewReceiver to add new receiver
+func (igd *IgmpGroupDevice) AddNewReceiver(port string, groupAddr net.IP, group *layers.IGMPv3GroupRecord, cvlan uint16, pbit uint8, ponPortID uint32) {
+
+	logger.Debugw(ctx, "Adding New Device Receiver", log.Fields{"Channel": groupAddr, "Port": port, "Device": igd.Device})
+	igcIntf, _ := igd.GroupChannels.Load(groupAddr.String())
+	if igcIntf == nil {
+		logger.Warnw(ctx, "No Group Channel present for given channel", log.Fields{"Channel": groupAddr, "Port": port, "Device": igd.Device})
+		return
+	}
+
+	igc := igcIntf.(*IgmpGroupChannel)
+	ipsList := []net.IP{}
+	ipsIntf, _ := igd.PortChannelMap.Load(port)
+	if ipsIntf != nil {
+		ipsList = ipsIntf.([]net.IP)
+	}
+	ipsList = append(ipsList, groupAddr)
+	igd.PortChannelMap.Store(port, ipsList)
+	igd.AddChannelToChannelsPerPon(port, groupAddr, ponPortID)
+	logger.Debugw(ctx, "Port Channel Updated", log.Fields{"Port": port, "NewChannelList": ipsList, "Addr": groupAddr})
+
+	igd.AddMcGroup()
+	igc.AddReceiver(port, group, cvlan, pbit)
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device, "GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+}
+
+// AddReceiver add the receiver to the device and perform other actions such as adding the group
+// to the physical device, add members, add flows to point the MC packets to the
+// group. Also, send a IGMP report upstream if there is a change in the group
+func (igc *IgmpGroupChannel) AddReceiver(port string, group *layers.IGMPv3GroupRecord, cvlan uint16, pbit uint8) bool {
+
+	var igp *IgmpGroupPort
+	var groupModified = false
+	var isNewReceiver = false
+
+	var ip []net.IP
+	incl := false
+	mvp := GetApplication().GetMvlanProfileByTag(igc.Mvlan)
+	if _, ok := mvp.Proxy[igc.GroupName]; ok {
+		if mvp.Proxy[igc.GroupName].Mode == common.Include {
+			incl = true
+		}
+		ip = mvp.Proxy[igc.GroupName].SourceList
+	} else if group != nil {
+		incl = isIncl(group.Type)
+		ip = group.SourceAddresses
+	}
+	logger.Debugw(ctx, "Attempting to add receiver", log.Fields{"Version": igc.Version, "Port": port, "Incl": incl, "srcIp": ip})
+
+	//logger.Infow(ctx, "Receivers", log.Fields{"New": igc.NewReceivers, "Current": igc.CurReceivers})
+	logger.Debugw(ctx, "Receiver Group", log.Fields{"Igd GId": igc.GroupID})
+	logger.Debugw(ctx, "Receiver Channel", log.Fields{"Igd addr": igc.GroupAddr})
+	logger.Debugw(ctx, "Receiver Mvlan", log.Fields{"Igd mvlan": igc.Mvlan})
+	logger.Debugw(ctx, "Receiver Sources", log.Fields{"Igd addr": ip})
+
+	ponPortID := GetApplication().GetPonPortID(igc.Device, port)
+
+	// Process the IGMP receiver. If it is already in, we should only process the changes
+	// to source list.
+	var newRcvExists bool
+	igp, newRcvExists = igc.NewReceivers[port]
+	if !newRcvExists {
+		// Add the receiver to the list of receivers and make the necessary group modification
+		// if this is the first time the receiver is added
+		var curRcvExists bool
+		if igp, curRcvExists = igc.CurReceivers[port]; curRcvExists {
+			logger.Debugw(ctx, "Existing IGMP receiver", log.Fields{"Group": igc.GroupAddr.String(), "Port": port})
+			delete(igc.CurReceivers, port)
+			igp.QueryTimeoutCount = 0
+			igc.NewReceivers[port] = igp
+		} else {
+			// New receiver who wasn't part of earlier list
+			// Need to send out IGMP group modification for this port
+			igp = NewIgmpGroupPort(port, cvlan, pbit, igc.Version, incl, uint32(ponPortID))
+			igc.NewReceivers[port] = igp
+			isNewReceiver = true
+			logger.Debugw(ctx, "New IGMP receiver", log.Fields{"Group": igc.GroupAddr.String(), "Port": port})
+			if len(igc.NewReceivers) == 1 && len(igc.CurReceivers) == 0 {
+				groupModified = true
+				igc.AddMcFlow()
+				logger.Debugw(ctx, "Added New Flow", log.Fields{"Group": igc.GroupAddr.String(), "Port": port})
+			}
+			if !incl {
+				igc.Exclude++
+			}
+		}
+	}
+
+	// Process the include/exclude list which may end up modifying the group
+	if change, _ := igc.ProcessSources(port, ip, incl); change {
+		groupModified = true
+	}
+	igc.ProcessMode(port, incl)
+
+	// If the group is modified as this is the first receiver or due to include/exclude list modification
+	// send a report to the upstream multicast servers
+	if groupModified {
+		logger.Debug(ctx, "Group Modified and IGMP report sent to the upstream server")
+		igc.SendReport(false)
+	} else if newRcvExists {
+		return false
+	}
+
+	logger.Debugw(ctx, "Channel Receiver Added", log.Fields{"Group Channel": igc.GroupAddr, "Group Port": igp})
+
+	if err := igc.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group channel Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+	}
+	if err := igp.WriteToDb(igc.Mvlan, igc.GroupAddr, igc.Device); err != nil {
+		logger.Errorw(ctx, "Igmp group port Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+	}
+	return isNewReceiver
+}
+
+// DelReceiver is called when Query expiry happened for a receiver. This removes the receiver from the
+// the group
+func (igc *IgmpGroupChannel) DelReceiver(port string, incl bool, srcList []net.IP) bool {
+	// The receiver may exist either in NewReceiver list or
+	// the CurReceivers list. Find and remove it from either
+	// of the lists.
+	logger.Debugw(ctx, "Deleting Receiver from Channel", log.Fields{"Port": port, "SrcList": srcList, "Incl": incl})
+	logger.Debugw(ctx, "New Receivers", log.Fields{"New": igc.NewReceivers})
+	logger.Debugw(ctx, "Current Receivers", log.Fields{"Current": igc.CurReceivers})
+
+	receiversUpdated := false
+	groupModified, receiverSrcListEmpty := igc.ProcessSources(port, srcList, incl)
+
+	if len(srcList) == 0 || len(igc.IncludeList) == 0 || receiverSrcListEmpty {
+		if igp, ok := igc.NewReceivers[port]; ok {
+			logger.Debug(ctx, "Deleting from NewReceivers")
+			delete(igc.NewReceivers, port)
+			receiversUpdated = true
+			if igp.Exclude {
+				igc.Exclude--
+			}
+		} else {
+			if igp, ok1 := igc.CurReceivers[port]; ok1 {
+				logger.Debug(ctx, "Deleting from CurReceivers")
+				delete(igc.CurReceivers, port)
+				receiversUpdated = true
+				if igp.Exclude {
+					igc.Exclude--
+				}
+			} else {
+				logger.Debug(ctx, "Receiver doesnot exist. Dropping Igmp leave")
+				return false
+			}
+		}
+		_ = db.DelIgmpRcvr(igc.Mvlan, igc.GroupAddr, igc.Device, port)
+	}
+
+	if igc.NumReceivers() == 0 {
+		igc.DelMcFlow()
+		mvp := GetApplication().GetMvlanProfileByTag(igc.Mvlan)
+		/* If proxy is configured and NumReceivers is 0, then we can reset the igc src list so that we send leave */
+		if _, ok := mvp.Proxy[igc.GroupName]; ok {
+			igc.IncludeList = []net.IP{}
+		}
+		igc.SendLeaveToServer()
+		logger.Debugw(ctx, "Deleted the receiver Flow", log.Fields{"Num Receivers": igc.NumReceivers()})
+		return true
+	}
+	if groupModified {
+		igc.SendReport(false)
+		logger.Infow(ctx, "Updated SourceList for Channel", log.Fields{"Current": igc.CurReceivers, "New": igc.NewReceivers})
+	}
+	if err := igc.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group channel Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+	}
+	logger.Infow(ctx, "Updated Receiver info for Channel", log.Fields{"Current": igc.CurReceivers, "New": igc.NewReceivers})
+
+	return receiversUpdated
+}
+
+// NumReceivers to get number of receivers
+func (igd *IgmpGroupDevice) NumReceivers() int {
+	var numReceivers int
+	len := func(key interface{}, value interface{}) bool {
+		numReceivers++
+		return true
+	}
+	igd.PortChannelMap.Range(len)
+	return numReceivers
+}
+
+// DelReceiver is called when Query expiry happened for a receiver. This removes the receiver from the
+// the group
+func (igd *IgmpGroupDevice) DelReceiver(groupAddr net.IP, port string, group *layers.IGMPv3GroupRecord, ponPortID uint32) {
+
+	logger.Debugw(ctx, "Deleting Receiver for Device", log.Fields{"port": port, "GroupIP": groupAddr.String()})
+	var igc *IgmpGroupChannel
+	var igcIntf interface{}
+	var ok bool
+	var srcList []net.IP
+	incl := false
+	mvp := GetApplication().GetMvlanProfileByTag(igd.Mvlan)
+
+	if _, ok := mvp.Proxy[igd.GroupName]; ok {
+		incl = true
+	} else if group != nil {
+		srcList = group.SourceAddresses
+		incl = isIncl(group.Type)
+	}
+
+	if igcIntf, ok = igd.GroupChannels.Load(groupAddr.String()); !ok {
+		logger.Warnw(ctx, "Igmp Channel for group IP doesnt exist", log.Fields{"GroupAddr": groupAddr.String()})
+		return
+	}
+	igc = igcIntf.(*IgmpGroupChannel)
+	if ok := igc.DelReceiver(port, incl, srcList); !ok {
+		return
+	}
+
+	if igc.NumReceivers() == 0 {
+		igd.DelIgmpGroupChannel(igc)
+	}
+	igd.DelPortFromChannel(port, groupAddr)
+	isGroupModified := igd.RemoveChannelFromChannelsPerPon(port, groupAddr, ponPortID)
+
+	//Remove port from receiver if port has no subscription to any of the group channels
+	if isGroupModified {
+		igd.ModMcGroup()
+	}
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device, "GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+}
+
+// DelChannelReceiver is called when Query expiry happened for a receiver. This removes the receiver from the
+// the group
+func (igd *IgmpGroupDevice) DelChannelReceiver(groupAddr net.IP) map[string]*IgmpGroupPort {
+
+	portsRemoved := make(map[string]*IgmpGroupPort)
+	groupModified := false
+	// ifEmpty := true
+	igcIntf, _ := igd.GroupChannels.Load(groupAddr.String())
+
+	if igcIntf == nil {
+		return portsRemoved
+	}
+	igc := igcIntf.(*IgmpGroupChannel)
+
+	for port, igp := range igc.NewReceivers {
+		_ = db.DelIgmpRcvr(igc.Mvlan, igc.GroupAddr, igc.Device, port) //TODO: Y not here
+		igd.DelPortFromChannel(port, igc.GroupAddr)
+		ponPortID := GetApplication().GetPonPortID(igd.Device, port)
+		groupModified = igd.RemoveChannelFromChannelsPerPon(port, igc.GroupAddr, ponPortID)
+		delete(igc.NewReceivers, port)
+		portsRemoved[port] = igp
+	}
+	for port, igp := range igc.CurReceivers {
+		_ = db.DelIgmpRcvr(igc.Mvlan, igc.GroupAddr, igc.Device, port)
+		igd.DelPortFromChannel(port, igc.GroupAddr)
+		ponPortID := GetApplication().GetPonPortID(igd.Device, port)
+		groupModified = igd.RemoveChannelFromChannelsPerPon(port, igc.GroupAddr, ponPortID)
+		delete(igc.CurReceivers, port)
+		portsRemoved[port] = igp
+	}
+
+	igc.DelMcFlow()
+	igd.DelIgmpGroupChannel(igc)
+	igc.Exclude = 0
+	igc.SendLeaveToServer()
+
+	if groupModified {
+		igd.ModMcGroup()
+	}
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device, "GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+	logger.Debugw(ctx, "Deleted the receiver Flow", log.Fields{"Num Receivers": igc.NumReceivers()})
+	return portsRemoved
+}
+
+// DelIgmpGroupChannel to delete igmp group channel
+func (igd *IgmpGroupDevice) DelIgmpGroupChannel(igc *IgmpGroupChannel) {
+
+	if igc.NumReceivers() != 0 {
+		igc.DelAllReceivers()
+	}
+	_ = db.DelIgmpChannel(igc.Mvlan, igc.GroupName, igc.Device, igc.GroupAddr)
+	igd.GroupChannels.Delete(igc.GroupAddr.String())
+	logger.Infow(ctx, "Deleted the Channel from Device", log.Fields{"Channel": igc.GroupAddr.String()})
+	isLenZero := true
+	checkIfEmpty := func(key interface{}, value interface{}) bool {
+		isLenZero = false
+		return false
+	}
+	igd.GroupChannels.Range(checkIfEmpty)
+	if isLenZero {
+		logger.Infow(ctx, "No more active channels. Deleting MC Group", log.Fields{"Device": igd.Device, "Group": igd.GroupName})
+		igd.DelMcGroup(false)
+	}
+}
+
+// func (igd *IgmpGroupDevice) DelIgmpChannel(igc *IgmpGroupChannel) {
+// 	db.DelIgmpChannel(igc.GroupName, igc.Device, igc.GroupAddr)
+// 	delete(igd.GroupChannels, igc.GroupAddr.String())
+// 	logger.Debugw(ctx, "Deleted the Channel", log.Fields{"Num Receivers": igc.NumReceivers()})
+// }
+
+// DelPortFromChannel to delete port from channel
+func (igd *IgmpGroupDevice) DelPortFromChannel(port string, groupAddr net.IP) bool {
+	ipsList := []net.IP{}
+	ipsListIntf, _ := igd.PortChannelMap.Load(port)
+	if ipsListIntf != nil {
+		ipsList = ipsListIntf.([]net.IP)
+	}
+	for i, addr := range ipsList {
+		if addr.Equal(groupAddr) {
+			ipsList = append(ipsList[:i], ipsList[i+1:]...)
+			//Remove port from receiver if port has no subscription to any of the group channels
+			if len(ipsList) == 0 {
+				igd.PortChannelMap.Delete(port)
+			} else {
+				//Update the map with modified ips list
+				igd.PortChannelMap.Store(port, ipsList)
+			}
+			logger.Debugw(ctx, "Port Channel Updated", log.Fields{"Port": port, "DelChannelList": ipsList, "Addr": groupAddr.String()})
+			return true
+		}
+	}
+	return false
+}
+
+// DelIgmpGroup deletes all devices for the provided igmp group
+func (ig *IgmpGroup) DelIgmpGroup() {
+	logger.Infow(ctx, "Deleting All Device for Group", log.Fields{"Group": ig.GroupName})
+	for _, igd := range ig.Devices {
+		ig.DelIgmpGroupDevice(igd)
+	}
+	GetApplication().DelIgmpGroup(ig)
+}
+
+// DelAllChannels deletes all receiver for the provided igmp device
+func (igd *IgmpGroupDevice) DelAllChannels() {
+	logger.Infow(ctx, "Deleting All Channel for Device", log.Fields{"Device": igd.Device, "Group": igd.GroupName})
+	delGroupChannels := func(key interface{}, value interface{}) bool {
+		igc := value.(*IgmpGroupChannel)
+		igd.DelIgmpGroupChannel(igc)
+		return true
+	}
+	igd.GroupChannels.Range(delGroupChannels)
+}
+
+// DelAllReceivers deletes all receiver for the provided igmp device
+func (igc *IgmpGroupChannel) DelAllReceivers() {
+	logger.Infow(ctx, "Deleting All Receiver for Channel", log.Fields{"Device": igc.Device, "Channel": igc.GroupAddr.String()})
+	_ = db.DelAllIgmpRcvr(igc.Mvlan, igc.GroupAddr, igc.Device)
+	igc.Exclude = 0
+	igc.DelMcFlow()
+	igc.SendLeaveToServer()
+	logger.Infow(ctx, "MC Flow deleted and Leave sent", log.Fields{"Channel": igc.GroupAddr.String(), "Device": igc.Device})
+}
+
+// ProcessQuery process query received from the upstream IGMP server
+func (igd *IgmpGroupDevice) ProcessQuery(groupAddr net.IP, ver uint8) {
+	logger.Debugw(ctx, "Received Query From Server", log.Fields{"Version": ver})
+	if ver != *igd.ServVersion {
+		igd.ServVersionExpiry = time.Now().Add(time.Duration(2*igd.proxyCfg.KeepAliveInterval) * time.Second)
+		*igd.ServVersion = ver
+		mvp := GetApplication().GetMvlanProfileByTag(igd.Mvlan)
+		if err := mvp.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Mvlan profile write to DB failed", log.Fields{"ProfileName": mvp.Name})
+		}
+	}
+	if igc, ok := igd.GroupChannels.Load(groupAddr.String()); ok {
+		igc.(*IgmpGroupChannel).SendReport(true)
+		return
+	}
+	logger.Infow(ctx, "No Members for Channel. Dropping Igmp Query", log.Fields{"Group": igd.GroupName, "Channel": groupAddr.String()})
+}
+
+// Igmpv2ReportPacket build an IGMPv2 Report for the upstream servers
+func (igc *IgmpGroupChannel) Igmpv2ReportPacket() ([]byte, error) {
+	logger.Debugw(ctx, "Buidling IGMP version 2 Report", log.Fields{"Device": igc.Device})
+	return IgmpReportv2Packet(igc.GroupAddr, igc.Mvlan, (*igc.proxyCfg).IgmpCos, **igc.IgmpProxyIP)
+}
+
+// Igmpv3ReportPacket build an IGMPv3 Report for the upstream servers
+func (igc *IgmpGroupChannel) Igmpv3ReportPacket() ([]byte, error) {
+	logger.Debugw(ctx, "Buidling IGMP version 3 Report", log.Fields{"Device": igc.Device, "Exclude": igc.Exclude})
+	if igc.Exclude > 0 {
+		return Igmpv3ReportPacket(igc.GroupAddr, igc.Mvlan, (*igc.proxyCfg).IgmpCos, **igc.IgmpProxyIP, false, igc.ExcludeList)
+	}
+	return Igmpv3ReportPacket(igc.GroupAddr, igc.Mvlan, (*igc.proxyCfg).IgmpCos, **igc.IgmpProxyIP, true, igc.IncludeList)
+}
+
+// SendReport send a consolidated report to the server
+func (igc *IgmpGroupChannel) SendReport(isQuery bool) {
+	var report []byte
+	var err error
+	logger.Debugw(ctx, "Checking Version", log.Fields{"IGC Version": igc.Version, "Proxy Version": (*igc.proxyCfg).IgmpVerToServer,
+		"Result": (getVersion((*igc.proxyCfg).IgmpVerToServer) == IgmpVersion2)})
+
+	/**
+	                               +------------------------------------------------------------------------+
+	                               |         IGMP version(towards BNG) Configured at VGC                    |
+	                               +-------------------------------+----------------------------------------+
+	                               |                  v2           |                 v3                     |
+	+===================+==========+===============================+========================================+
+	| Received From RG  | V2 Join  | Process and Send as V2 to BNG | Process, Convert to v3 and Send to BNG |
+	|                   |          |                               | Process, Send as v2, if the BNG is v2  |
+	+===================+----------+-------------------------------+----------------------------------------+
+	                    | V3 Join  | Process and Send as V2 to BNG | Process, Send v3 to BNG                |
+	                    |          |                               | Process, Convert, Send as v2, if the   |
+	                    |          |                               | BNG is v2                              |
+	+===================+==========+===============================+========================================+
+	| Received From BNG | V2 Query | V2 response to BNG            | V2 response to BNG                     |
+	+===================+----------+-------------------------------+----------------------------------------+
+	                    | V3 Query | Discard                       | V3 response to BNG                     |
+	                    +==========+===============================+========================================+
+	*/
+	// igc.Version: 	igmp version received from RG.
+	// igc.ServVersion: igmp version received from BNG or IgmpVerToServer present in proxy igmp conf.
+
+	if isQuery && *igc.ServVersion == IgmpVersion3 && getVersion((*igc.proxyCfg).IgmpVerToServer) == IgmpVersion2 {
+		// This is the last scenario where we must discard the query processing.
+		logger.Debug(ctx, "Dropping query packet since the server verion is v3 but igmp proxy version is v2")
+		return
+	}
+
+	if *igc.ServVersion == IgmpVersion2 || getVersion((*igc.proxyCfg).IgmpVerToServer) == IgmpVersion2 {
+		report, err = igc.Igmpv2ReportPacket()
+	} else {
+		report, err = igc.Igmpv3ReportPacket()
+	}
+	if err != nil {
+		logger.Warnw(ctx, "Error Preparing Report", log.Fields{"Device": igc.Device, "Ver": igc.Version, "Reason": err.Error()})
+		return
+	}
+	nni, err := GetApplication().GetNniPort(igc.Device)
+	if err == nil {
+		_ = cntlr.GetController().PacketOutReq(igc.Device, nni, nni, report, false)
+	} else {
+		logger.Warnw(ctx, "Didn't find NNI port", log.Fields{"Device": igc.Device})
+	}
+}
+
+// AddMcFlow adds flow to the device when the first receiver joins
+func (igc *IgmpGroupChannel) AddMcFlow() {
+	flow, err := igc.BuildMcFlow()
+	if err != nil {
+		logger.Warnw(ctx, "MC Flow Build Failed", log.Fields{"Reason": err.Error()})
+		return
+	}
+	port, _ := GetApplication().GetNniPort(igc.Device)
+	_ = cntlr.GetController().AddFlows(port, igc.Device, flow)
+}
+
+// DelMcFlow deletes flow from the device when the last receiver leaves
+func (igc *IgmpGroupChannel) DelMcFlow() {
+	flow, err := igc.BuildMcFlow()
+	if err != nil {
+		logger.Warnw(ctx, "MC Flow Build Failed", log.Fields{"Reason": err.Error()})
+		return
+	}
+	flow.ForceAction = true
+	device := GetApplication().GetDevice(igc.Device)
+
+	if mvpIntf, _ := GetApplication().MvlanProfilesByTag.Load(igc.Mvlan); mvpIntf != nil {
+		mvp := mvpIntf.(*MvlanProfile)
+		err := mvp.DelFlows(device, flow)
+		if err != nil {
+			logger.Warnw(ctx, "Delering IGMP Flow for device failed ", log.Fields{"Device": device, "err": err})
+		}
+	}
+}
+
+// BuildMcFlow builds the flow using which it is added/deleted
+func (igc *IgmpGroupChannel) BuildMcFlow() (*of.VoltFlow, error) {
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	//va := GetApplication()
+	logger.Infow(ctx, "Building Mcast flow", log.Fields{"Mcast Group": igc.GroupAddr.String(), "Mvlan": igc.Mvlan.String()})
+	uintGroupAddr := ipv4ToUint(igc.GroupAddr)
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetMatchVlan(igc.Mvlan)
+	subFlow.SetIpv4Match()
+	subFlow.SetMatchDstIpv4(igc.GroupAddr)
+	mvp := GetApplication().GetMvlanProfileByTag(igc.Mvlan)
+	//nni, err := va.GetNniPort(igc.Device)
+	//if err != nil {
+	//	return nil, err
+	//}
+	//inport, err := va.GetPortID(nni)
+	//if err != nil {
+	//	return nil, err
+	//}
+	//subFlow.SetInPort(inport)
+	subFlow.SetOutGroup(igc.GroupID)
+	cookiePort := uintGroupAddr
+	subFlow.Cookie = uint64(cookiePort)<<32 | uint64(igc.Mvlan)
+	subFlow.Priority = of.McFlowPriority
+	metadata := uint64(mvp.PonVlan)
+	subFlow.SetTableMetadata(metadata)
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built Mcast flow", log.Fields{"cookie": subFlow.Cookie, "subflow": subFlow})
+	return flow, nil
+}
+
+//DelFlows - Triggers flow deletion after registering for flow indication event
+func (mvp *MvlanProfile) DelFlows(device *VoltDevice, flow *of.VoltFlow) error {
+	mvp.mvpFlowLock.Lock()
+	defer mvp.mvpFlowLock.Unlock()
+
+	var flowMap map[string]bool
+	var ok bool
+
+	for cookie := range flow.SubFlows {
+		cookie := strconv.FormatUint(cookie, 10)
+		fe := &FlowEvent{
+			eType:     EventTypeMcastFlowRemoved,
+			device:    device.Name,
+			cookie:    cookie,
+			eventData: mvp,
+		}
+		device.RegisterFlowDelEvent(cookie, fe)
+
+		if flowMap, ok = mvp.PendingDeleteFlow[device.Name]; !ok {
+			flowMap = make(map[string]bool)
+		}
+		flowMap[cookie] = true
+		mvp.PendingDeleteFlow[device.Name] = flowMap
+	}
+	if err := mvp.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Mvlan profile write to DB failed", log.Fields{"ProfileName": mvp.Name})
+	}
+	return cntlr.GetController().DelFlows(device.NniPort, device.Name, flow)
+}
+
+//FlowRemoveSuccess - Process flow success indication
+func (mvp *MvlanProfile) FlowRemoveSuccess(cookie string, device string) {
+	mvp.mvpFlowLock.Lock()
+	defer mvp.mvpFlowLock.Unlock()
+
+	logger.Infow(ctx, "Mvlan Flow Remove Success Notification", log.Fields{"MvlanProfile": mvp.Name, "Cookie": cookie, "Device": device})
+
+	if _, ok := mvp.PendingDeleteFlow[device]; ok {
+		delete(mvp.PendingDeleteFlow[device], cookie)
+	}
+
+	if err := mvp.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Mvlan profile write to DB failed", log.Fields{"ProfileName": mvp.Name})
+	}
+}
+
+//FlowRemoveFailure - Process flow failure indication
+func (mvp *MvlanProfile) FlowRemoveFailure(cookie string, device string, errorCode uint32, errReason string) {
+
+	mvp.mvpFlowLock.Lock()
+	defer mvp.mvpFlowLock.Unlock()
+
+	if flowMap, ok := mvp.PendingDeleteFlow[device]; ok {
+		if _, ok := flowMap[cookie]; ok {
+			logger.Errorw(ctx, "Mvlan Flow Remove Failure Notification", log.Fields{"MvlanProfile": mvp.Name, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason, "Device": device})
+			return
+		}
+	}
+	logger.Errorw(ctx, "Mvlan Flow Del Failure Notification for Unknown cookie", log.Fields{"MvlanProfile": mvp.Name, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason})
+
+}
+
+// AddMcGroup add the new group on the device when a receiver joins the group
+func (igd *IgmpGroupDevice) AddMcGroup() {
+	if !igd.GroupInstalled {
+		group := &of.Group{}
+		group.Command = of.GroupCommandAdd
+		group.GroupID = igd.GroupID
+		group.Device = igd.Device
+		group.SetVlan = igd.PonVlan
+		group.IsPonVlanPresent = igd.IsPonVlanPresent
+
+		addbuckets := func(key interface{}, value interface{}) bool {
+			port := key.(string)
+			var portID uint32
+			if d := GetApplication().GetDevice(group.Device); d != nil {
+				GetApplication().portLock.Lock()
+				p := d.GetPort(port)
+				GetApplication().portLock.Unlock()
+				portID = p.ID
+			}
+			//ponPortID := key.(uint32)
+			if portID != 0xFF {
+				group.Buckets = append(group.Buckets, portID)
+			}
+			return true
+		}
+		igd.PortChannelMap.Range(addbuckets)
+
+		port, _ := GetApplication().GetNniPort(igd.Device)
+		_ = cntlr.GetController().GroupUpdate(port, igd.Device, group)
+		igd.GroupInstalled = true
+	}
+}
+
+// ModMcGroup updates the group on the device when either a receiver leaves
+// or joins the group
+func (igd *IgmpGroupDevice) ModMcGroup() {
+	if igd.GroupInstalled {
+		group := &of.Group{}
+		group.Command = of.GroupCommandMod
+		group.GroupID = igd.GroupID
+		group.Device = igd.Device
+		group.SetVlan = igd.PonVlan
+		group.IsPonVlanPresent = igd.IsPonVlanPresent
+
+		addbuckets := func(key interface{}, value interface{}) bool {
+                       port := key.(string)
+                       var portID uint32
+                       if d := GetApplication().GetDevice(group.Device); d != nil {
+                               GetApplication().portLock.Lock()
+                               p := d.GetPort(port)
+                               GetApplication().portLock.Unlock()
+                               portID = p.ID
+                       }
+                       //ponPortID := key.(uint32)
+                       if portID != 0xFF {
+                               group.Buckets = append(group.Buckets, portID)
+			}
+			return true
+		}
+		igd.PortChannelMap.Range(addbuckets)
+
+		port, _ := GetApplication().GetNniPort(igd.Device)
+		_ = cntlr.GetController().GroupUpdate(port, igd.Device, group)
+	} else {
+		logger.Warnw(ctx, "Update Group Failed. Group not yet created", log.Fields{"Igd": igd.Device})
+	}
+}
+
+// DelMcGroup : The group is deleted when the last receiver leaves the group
+func (igd *IgmpGroupDevice) DelMcGroup(forceDelete bool) {
+
+	logger.Infow(ctx, "Delete Mc Group Request", log.Fields{"Device": igd.Device, "GroupID": igd.GroupID, "ForceFlag": forceDelete, "GroupInstalled": igd.GroupInstalled})
+	/*
+	if !forceDelete && !checkIfForceGroupRemove(igd.Device) {
+		if success := AddToPendingPool(igd.Device, igd.getKey()); success {
+			return
+		}
+	}*/
+	if igd.GroupInstalled {
+		logger.Debugw(ctx, "Deleting Group", log.Fields{"Device": igd.Device, "Id": igd.GroupID})
+		group := &of.Group{}
+		group.Command = of.GroupCommandDel
+		group.GroupID = igd.GroupID
+		group.Device = igd.Device
+		group.ForceAction = true
+
+		port, _ := GetApplication().GetNniPort(igd.Device)
+		_ = cntlr.GetController().GroupUpdate(port, igd.Device, group)
+		igd.GroupInstalled = false
+	}
+}
+
+//AddToPendingPool - adds Igmp Device obj to pending pool
+func AddToPendingPool(device string, groupKey string) bool {
+
+	logger.Infow(ctx, "Add Device to IgmpGroup Pending Pool", log.Fields{"Device": device, "GroupKey": groupKey})
+	if grp, ok := GetApplication().IgmpGroups.Load(groupKey); ok {
+		ig := grp.(*IgmpGroup)
+		ig.PendingPoolLock.Lock()
+		logger.Infow(ctx, "Adding Device to IgmpGroup Pending Pool", log.Fields{"Device": device, "GroupID": ig.GroupID, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr.String()})
+		ig.PendingGroupForDevice[device] = time.Now().Add(time.Duration(GroupExpiryTime) * time.Minute)
+		ig.PendingPoolLock.Unlock()
+		if err := ig.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+		}
+		return true
+	}
+	return false
+}
+
+/*
+func checkIfForceGroupRemove(device string) bool {
+	if d := GetApplication().GetDevice(device); d != nil {
+		if d.State == cntlr.DeviceStateREBOOTED || d.State == cntlr.DeviceStateDOWN {
+			return true
+		}
+	}
+	return false
+}*/
+
+// IgmpLeaveToServer sends IGMP leave to server. Called when the last receiver leaves the group
+func (igc *IgmpGroupChannel) IgmpLeaveToServer() {
+	if leave, err := IgmpLeavePacket(igc.GroupAddr, igc.Mvlan, (*igc.proxyCfg).IgmpCos, **igc.IgmpProxyIP); err == nil {
+		nni, err1 := GetApplication().GetNniPort(igc.Device)
+		if err1 == nil {
+			_ = cntlr.GetController().PacketOutReq(igc.Device, nni, nni, leave, false)
+		}
+	}
+}
+
+// SendLeaveToServer delete the group when the last receiver leaves the group
+func (igc *IgmpGroupChannel) SendLeaveToServer() {
+	/**
+	                               +-------------------------------------------------------------------------+
+	                               |         IGMP version(towards BNG) Configured at VGC                     |
+	                               +-------------------------------+-----------------------------------------+
+	                               |                  v2           |                 v3                      |
+	+===================+==========+===============================+=========================================+
+	| Received From RG  | V2 Leave | Process and Send as V2 to BNG | Process, Convert to V3 and Send to BNG/ |
+	|                   |          |                               | Process, Send as V2, if the BNG is V2   |
+	+===================+----------+-------------------------------+-----------------------------------------+
+	                    | V3 Leave | Process and Send as V2 to BNG | Process, Send V3 to BNG                 |
+	                    |          |                               | Process, Convert, Send as V2, if the    |
+	                    |          |                               | BNG is v2                               |
+	                    +==========+===============================+=========================================+
+	*/
+	// igc.Version: 	igmp version received from RG.
+	// igc.ServVersion: igmp version received from BNG or IgmpVerToServer present in proxy igmp conf.
+
+	logger.Debugw(ctx, "Sending IGMP leave upstream", log.Fields{"Device": igc.Device})
+	if *igc.ServVersion == IgmpVersion2 || getVersion((*igc.proxyCfg).IgmpVerToServer) == IgmpVersion2 {
+		igc.IgmpLeaveToServer()
+	} else {
+		igc.SendReport(false)
+	}
+}
+
+// QueryExpiry processes query expiry. Upon expiry, take stock of the situation
+// add either retain/release the group based on number of receivers left
+func (igd *IgmpGroupDevice) QueryExpiry() {
+	logger.Debugw(ctx, "Query Expiry", log.Fields{"Device": igd.Device})
+
+
+	// Delete the IGMP flow added for this port if port state is down or query count exceeded
+	handleQueryExp := func(key interface{}, value interface{}) bool {
+		igc := value.(*IgmpGroupChannel)
+		for portKey, port := range igc.CurReceivers {
+
+			if portKey == StaticPort {
+				continue
+			}
+
+			logger.Warnw(ctx, "Expired Receiver Port", log.Fields{"PortKey": portKey, "IGP": port, "GroupAddr": igc.GroupAddr,
+				"Count": port.QueryTimeoutCount})
+			state, err := cntlr.GetController().GetPortState(igc.Device, portKey)
+			logger.Debugw(ctx, "Expired Member Port State", log.Fields{"state": state})
+			ponPortID := GetApplication().GetPonPortID(igd.Device, portKey)
+			if err == nil && state == cntlr.PortStateDown {
+				igd.DelReceiver(igc.GroupAddr, portKey, nil, ponPortID)
+			}
+
+			port.QueryTimeoutCount++
+			logger.Debugw(ctx, "Expired Port TimeoutCount", log.Fields{"count": port.QueryTimeoutCount})
+			if port.QueryTimeoutCount >= (*igc.proxyCfg).KeepAliveCount {
+				logger.Errorw(ctx, "Expiry Timeout count exceeded. Trigger delete receiver", log.Fields{"PortKey": portKey,
+					"GroupAddr": igc.GroupAddr, "Count": port.QueryTimeoutCount})
+				igd.DelReceiver(igc.GroupAddr, portKey, nil, ponPortID)
+				SendQueryExpiredEventGroupSpecific(portKey, igd, igc)
+			} else {
+				_ = port.WriteToDb(igc.Mvlan, igc.GroupAddr, igc.Device)
+			}
+		}
+		return true
+	}
+	igd.GroupChannels.Range(handleQueryExp)
+}
+
+// SendQueryExpiredEventGroupSpecific to send group specific query expired event.
+func SendQueryExpiredEventGroupSpecific(portKey string, igd *IgmpGroupDevice, igc *IgmpGroupChannel) {
+
+	logger.Info(ctx, "Processing-SendQueryExpiredEventGroupSpecific-Event")
+	va := GetApplication()
+	mvpName := va.GetMvlanProfileByTag(igd.Mvlan).Name
+
+	sendEvent := func(key interface{}, value interface{}) bool {
+		if value.(*VoltService).IgmpEnabled && value.(*VoltService).MvlanProfileName == mvpName {
+			logger.Debugw(ctx, "sending-query-expired-group-specific-event", log.Fields{"EventType": QueryExpiredGroupSpecific, "ServiceName": value.(*VoltService).Name})
+		}
+		return false
+	}
+
+	// Fetching service name to send with query expired event.
+	vpvs, _ := va.VnetsByPort.Load(portKey)
+	if vpvs == nil {
+		logger.Errorw(ctx, "volt-port-vnet-is-nil", log.Fields{"vpvs": vpvs})
+		return
+	}
+
+	for _, vpv := range vpvs.([]*VoltPortVnet) {
+		vpv.services.Range(sendEvent)
+	}
+}
+
+// GetMcastServiceForSubAlarm to get mcast service name for subscriber alarm.
+func GetMcastServiceForSubAlarm(uniPort *VoltPort, mvp *MvlanProfile) string {
+
+	var serviceName string
+	mvpName := mvp.Name
+
+	va := GetApplication()
+
+	sendAlm := func(key interface{}, value interface{}) bool {
+		if value.(*VoltService).IgmpEnabled && value.(*VoltService).MvlanProfileName == mvpName {
+			serviceName = value.(*VoltService).Name
+		}
+		return true
+	}
+
+	// Fetching service name to send with active channels exceeded per subscriber alarm.
+	vpvs, _ := va.VnetsByPort.Load(uniPort.Name)
+	if vpvs == nil {
+		logger.Errorw(ctx, "volt-port-vnet-is-nil", log.Fields{"vpvs": vpvs})
+		return serviceName
+	}
+
+	for _, vpv := range vpvs.([]*VoltPortVnet) {
+		vpv.services.Range(sendAlm)
+	}
+
+	return serviceName
+
+}
+
+// NumReceivers returns total number of receivers left on the group
+func (igc *IgmpGroupChannel) NumReceivers() uint32 {
+	return uint32(len(igc.CurReceivers) + len(igc.NewReceivers))
+}
+
+// SendQuery sends query to the receivers for counting purpose
+func (igc *IgmpGroupChannel) SendQuery() {
+	//var b []byte
+	//var err error
+	for portKey, port := range igc.NewReceivers {
+		igc.CurReceivers[portKey] = port
+	}
+
+	igc.NewReceivers = make(map[string]*IgmpGroupPort)
+
+	logger.Debugw(ctx, "Sending Query to receivers", log.Fields{"Receivers": igc.CurReceivers})
+	for port, groupPort := range igc.CurReceivers {
+		if port == StaticPort {
+			continue
+		}
+		if queryPkt, err := igc.buildQuery(igc.GroupAddr, of.VlanType(groupPort.CVlan), groupPort.Pbit); err == nil {
+			_ = cntlr.GetController().PacketOutReq(igc.Device, port, port, queryPkt, false)
+			logger.Debugw(ctx, "Query Sent", log.Fields{"Device": igc.Device, "Port": port, "Packet": queryPkt})
+		} else {
+			logger.Warnw(ctx, "Query Creation Failed", log.Fields{"Reason": err.Error()})
+		}
+	}
+
+}
+
+// buildQuery to build query packet
+func (igc *IgmpGroupChannel) buildQuery(groupAddr net.IP, cVlan of.VlanType, pbit uint8) ([]byte, error) {
+	if igc.Version == IgmpVersion2 {
+		return Igmpv2QueryPacket(igc.GroupAddr, cVlan, **igc.IgmpProxyIP, pbit, (*igc.proxyCfg).MaxResp)
+	}
+	return Igmpv3QueryPacket(igc.GroupAddr, cVlan, **igc.IgmpProxyIP, pbit, (*igc.proxyCfg).MaxResp)
+}
+
+// IgmpGroup implements a single MCIP that may have multiple receivers
+// connected via multiple devices (OLTs). The IGMP group is stored on the
+// VOLT application.
+type IgmpGroup struct {
+	GroupID               uint32
+	Mvlan                 of.VlanType
+	PonVlan               of.VlanType
+	GroupName             string
+	GroupAddr             net.IP
+	Devices               map[string]*IgmpGroupDevice `json:"-"`
+	PendingGroupForDevice map[string]time.Time        //map [deviceId, timestamp]  (ExpiryTime  = leave time + 15mins)
+	Version               string
+	IsPonVlanPresent      bool
+	IsChannelBasedGroup   bool
+	PendingPoolLock       sync.RWMutex
+	IsGroupStatic         bool
+	IgmpGroupLock         sync.RWMutex
+}
+
+// NewIgmpGroup is constructor for an IGMP group
+func NewIgmpGroup(name string, vlan of.VlanType) *IgmpGroup {
+	ig := IgmpGroup{}
+	ig.GroupName = name
+	ig.Mvlan = vlan
+	ig.Devices = make(map[string]*IgmpGroupDevice)
+	ig.PendingGroupForDevice = make(map[string]time.Time)
+	return &ig
+}
+
+// IgmpGroupInit to initialize igmp group members
+func (ig *IgmpGroup) IgmpGroupInit(name string, gip net.IP, mvp *MvlanProfile) {
+	ig.GroupName = name
+	ig.Mvlan = mvp.Mvlan
+	ig.PonVlan = mvp.PonVlan
+	ig.IsPonVlanPresent = mvp.IsPonVlanPresent
+	ig.Devices = make(map[string]*IgmpGroupDevice)
+	ig.PendingGroupForDevice = make(map[string]time.Time)
+	ig.IsChannelBasedGroup = mvp.IsChannelBasedGroup
+	ig.IsGroupStatic = mvp.Groups[name].IsStatic
+	if ig.IsChannelBasedGroup {
+		ig.GroupAddr = gip
+	} else {
+		ig.GroupAddr = net.ParseIP("0.0.0.0")
+	}
+}
+
+// IgmpGroupReInit to re-initialize igmp group members
+func (ig *IgmpGroup) IgmpGroupReInit(name string, gip net.IP) {
+
+	logger.Infow(ctx, "Reinitialize Igmp Group", log.Fields{"GroupID": ig.GroupID, "OldName": ig.GroupName, "Name": name, "OldAddr": ig.GroupAddr.String(), "GroupAddr": gip.String()})
+
+	ig.GroupName = name
+	if ig.IsChannelBasedGroup {
+		ig.GroupAddr = gip
+	} else {
+		ig.GroupAddr = net.ParseIP("0.0.0.0")
+	}
+
+	for _, igd := range ig.Devices {
+		igd.IgmpGroupDeviceReInit(ig)
+	}
+}
+
+// IsStaticGroup to check if group is static
+func (mvp *MvlanProfile) IsStaticGroup(groupName string) bool {
+	return mvp.Groups[groupName].IsStatic
+}
+
+// updateGroupName to update group name
+func (ig *IgmpGroup) updateGroupName(newGroupName string) {
+	if !ig.IsChannelBasedGroup {
+		logger.Errorw(ctx, "Group name update not supported for GroupChannel based group", log.Fields{"Ig": ig})
+		return
+	}
+	oldKey := ig.getKey()
+	ig.GroupName = newGroupName
+	for _, igd := range ig.Devices {
+		igd.updateGroupName(newGroupName)
+	}
+	if err := ig.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+	}
+	if !ig.IsChannelBasedGroup {
+		_ = db.DelIgmpGroup(oldKey)
+	}
+}
+
+//HandleGroupMigration - handles migration of group members between static & dynamic
+func (ig *IgmpGroup) HandleGroupMigration(deviceID string, groupAddr net.IP) {
+
+	var group *layers.IGMPv3GroupRecord
+	app := GetApplication()
+	if deviceID == "" {
+		logger.Infow(ctx, "Handle Group Migration Request for all devices", log.Fields{"DeviceID": deviceID, "GroupAddr": groupAddr, "IG": ig.GroupName, "Mvlan": ig.Mvlan})
+		for device := range ig.Devices {
+			ig.HandleGroupMigration(device, groupAddr)
+		}
+	} else {
+		logger.Infow(ctx, "Handle Group Migration Request", log.Fields{"DeviceID": deviceID, "GroupAddr": groupAddr, "IG": ig.GroupName})
+		var newIg *IgmpGroup
+		receivers := ig.DelIgmpChannel(deviceID, groupAddr)
+		if ig.NumDevicesActive() == 0 {
+			app.DelIgmpGroup(ig)
+		}
+		if newIg = app.GetIgmpGroup(ig.Mvlan, groupAddr); newIg == nil {
+			logger.Infow(ctx, "IG Group doesn't exist, creating new group", log.Fields{"DeviceID": deviceID, "GroupAddr": groupAddr, "IG": ig.GroupName, "Mvlan": ig.Mvlan})
+			if newIg = app.AddIgmpGroup(app.GetMvlanProfileByTag(ig.Mvlan).Name, groupAddr, deviceID); newIg == nil {
+				logger.Errorw(ctx, "Group Creation failed during group migration", log.Fields{"DeviceID": deviceID, "GroupAddr": groupAddr})
+				return
+			}
+		}
+		mvp := app.GetMvlanProfileByTag(ig.Mvlan)
+		isStaticGroup := mvp.IsStaticGroup(ig.GroupName)
+		logger.Infow(ctx, "Existing receivers for old group", log.Fields{"Receivers": receivers})
+		newIg.IgmpGroupLock.Lock()
+		for port, igp := range receivers {
+			if !isStaticGroup && port == StaticPort {
+				continue
+			}
+			group = nil
+			var reqType layers.IGMPv3GroupRecordType
+			srcAddresses := []net.IP{}
+			if igp.Version == IgmpVersion3 {
+				if igp.Exclude {
+					srcAddresses = append(srcAddresses, igp.ExcludeList...)
+					reqType = layers.IGMPIsEx
+				} else {
+					srcAddresses = append(srcAddresses, igp.IncludeList...)
+					reqType = layers.IGMPIsIn
+				}
+				group = &layers.IGMPv3GroupRecord{
+					SourceAddresses: srcAddresses,
+					Type:            reqType,
+				}
+			}
+			logger.Infow(ctx, "Adding receiver to new group", log.Fields{"DeviceID": deviceID, "GroupAddr": groupAddr, "newIg": newIg.GroupName, "IGP": igp})
+			ponPort := GetApplication().GetPonPortID(deviceID, port)
+			newIg.AddReceiver(deviceID, port, groupAddr, group, igp.Version, igp.CVlan, igp.Pbit, ponPort)
+		}
+		newIg.IgmpGroupLock.Unlock()
+	}
+}
+
+// AddIgmpGroupDevice add a device to the group which happens when the first receiver of the device
+// is added to the IGMP group.
+func (ig *IgmpGroup) AddIgmpGroupDevice(device string, id uint32, version uint8) *IgmpGroupDevice {
+	logger.Infow(ctx, "Adding Device to IGMP group", log.Fields{"Device": device, "GroupName": ig.GroupName})
+	igd := NewIgmpGroupDevice(device, ig, id, version)
+	ig.Devices[device] = igd
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device, "GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+	return igd
+}
+
+// DelIgmpGroupDevice delete the device from the group which happens when we receive a leave or when
+// there is not response for IGMP query from the receiver
+func (ig *IgmpGroup) DelIgmpGroupDevice(igd *IgmpGroupDevice) {
+	logger.Infow(ctx, "Deleting Device from IGMP group", log.Fields{"Device": igd.Device, "Name": ig.GroupName})
+	va := GetApplication()
+	countersToBeUpdated := false
+	if igd.NumReceivers() != 0 {
+		countersToBeUpdated = true
+	}
+	igd.DelAllChannels()
+
+	//Clear all internal maps so that the groups can be reused
+	igd.PortChannelMap.Range(func(key, value interface{}) bool {
+
+		//Update the counters only if not already updated
+		//(i.e) 1. In case of channel remove during Mvlan Update
+		if countersToBeUpdated {
+			port := key.(string)
+			channelList := value.([]net.IP)
+			ponPortID := va.GetPonPortID(igd.Device, port)
+
+			for _, channel := range channelList {
+				igd.RemoveChannelFromChannelsPerPon(port, channel, ponPortID)
+			}
+		}
+
+		igd.PortChannelMap.Delete(key)
+		return true
+	})
+	igd.PonPortChannelMap = util.NewConcurrentMap()
+
+	if mcastCfg := va.GetMcastConfig(igd.SerialNo, va.GetMvlanProfileByTag(igd.Mvlan).Name); mcastCfg != nil {
+		mcastCfg.IgmpGroupDevices.Delete(igd.GroupID)
+		logger.Debugw(ctx, "Igd deleted from mcast config", log.Fields{"mvlan": mcastCfg.MvlanProfileID, "groupId": igd.GroupID})
+	}
+	if !igd.GroupInstalled {
+		_ = db.DelIgmpDevice(igd.Mvlan, ig.GroupName, ig.GroupAddr, igd.Device)
+		delete(ig.Devices, igd.Device)
+	}
+}
+
+// AddReceiver delete the device from the group which happens when we receive a leave or when
+// there is not response for IGMP query from the receiver
+func (ig *IgmpGroup) AddReceiver(device string, port string, groupIP net.IP,
+	group *layers.IGMPv3GroupRecord, ver uint8, cvlan uint16, pbit uint8, ponPort uint32) {
+
+	logger.Debugw(ctx, "Adding Receiver", log.Fields{"Port": port})
+	if igd, ok := ig.getIgmpGroupDevice(device); !ok {
+		igd = ig.AddIgmpGroupDevice(device, ig.GroupID, ver)
+		igd.AddReceiver(port, groupIP, group, ver, cvlan, pbit, ponPort)
+	} else {
+		logger.Infow(ctx, "IGMP Group Receiver", log.Fields{"IGD": igd.Device})
+		igd.AddReceiver(port, groupIP, group, ver, cvlan, pbit, ponPort)
+	}
+}
+
+func (ig *IgmpGroup) getIgmpGroupDevice(device string) (*IgmpGroupDevice, bool) {
+	ig.PendingPoolLock.Lock()
+	defer ig.PendingPoolLock.Unlock()
+
+	if _, ok := ig.PendingGroupForDevice[device]; ok {
+		logger.Infow(ctx, "Removing the IgmpGroupDevice from pending pool", log.Fields{"GroupID": ig.GroupID, "Device": device, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr.String()})
+		delete(ig.PendingGroupForDevice, device)
+		if err := ig.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+		}
+	}
+	igd, ok := ig.Devices[device]
+	return igd, ok
+}
+
+// DelReceiveronDownInd deletes a receiver which is the combination of device (OLT)
+// and port on Port Down event
+func (ig *IgmpGroup) DelReceiveronDownInd(device string, port string, ponPortID uint32) {
+	logger.Debugw(ctx, "Deleting Receiver for Group", log.Fields{"Device": device, "port": port})
+
+	mvp := GetApplication().GetMvlanProfileByTag(ig.Mvlan)
+	mvp.mvpLock.RLock()
+	defer mvp.mvpLock.RUnlock()
+	igd, ok := ig.Devices[device]
+	if !ok {
+		logger.Infow(ctx, "IGMP Group device was not found for ", log.Fields{"Device": device})
+		return
+	}
+	ipsList := []net.IP{}
+	ipsListIntf, ok := igd.PortChannelMap.Load(port)
+	if ok {
+		ipsList = append(ipsList, ipsListIntf.([]net.IP)...)
+	}
+	logger.Infow(ctx, "Port Channel List", log.Fields{"Port": port, "IPsList": ipsList})
+	igd.PortChannelMap.Range(printPortChannel)
+
+
+	for _, groupAddr := range ipsList {
+		logger.Debugw(ctx, "Port Channels", log.Fields{"Port": port, "IPsList": ipsList, "GroupAddr": groupAddr, "Len": len(ipsList)})
+		igd.DelReceiver(groupAddr, port, nil, ponPortID)
+	}
+
+	if igd.NumReceivers() == 0 {
+		ig.DelIgmpGroupDevice(igd)
+	}
+}
+
+// DelReceiver deletes a receiver which is the combination of device (OLT)
+// and port
+func (ig *IgmpGroup) DelReceiver(device string, port string, groupAddr net.IP, group *layers.IGMPv3GroupRecord, ponPortID uint32) {
+	logger.Debugw(ctx, "Deleting Receiver for Group", log.Fields{"Device": device, "port": port, "GroupIP": groupAddr.String()})
+	if igd, ok := ig.Devices[device]; ok {
+		//igd.DelReceiverForGroupAddr(groupAddr, port)
+		igd.DelReceiver(groupAddr, port, group, ponPortID)
+		if igd.NumReceivers() == 0 {
+			ig.DelIgmpGroupDevice(igd)
+		}
+	}
+}
+
+// GetAllIgmpChannelForDevice - Returns all channels with active members associated to the Igmp Group for the given device
+func (ig *IgmpGroup) GetAllIgmpChannelForDevice(deviceID string) map[string]string {
+
+	if deviceID == "" {
+		return ig.GetAllIgmpChannel()
+	}
+
+	allChannels := make(map[string]string)
+	igd := ig.Devices[deviceID]
+	getAllChannels := func(key interface{}, value interface{}) bool {
+		channels := key.(string)
+		allChannels[channels] = channels //same value as only key is required
+		return true
+	}
+	igd.GroupChannels.Range(getAllChannels)
+
+	return allChannels
+}
+
+// GetAllIgmpChannel - Returns all channels with active members associated to the Igmp Group
+func (ig *IgmpGroup) GetAllIgmpChannel() map[string]string {
+	allChannels := make(map[string]string)
+	for _, igd := range ig.Devices {
+		getAllChannels := func(key interface{}, value interface{}) bool {
+			channels := key.(string)
+			allChannels[channels] = channels
+			return true
+		}
+		igd.GroupChannels.Range(getAllChannels)
+	}
+	return allChannels
+}
+
+// DelIgmpChannel deletes all receivers for the provided igmp group channel for the given device
+func (ig *IgmpGroup) DelIgmpChannel(deviceID string, groupAddr net.IP) map[string]*IgmpGroupPort {
+	logger.Infow(ctx, "Deleting Channel from devices", log.Fields{"Device": deviceID, "Group": ig.GroupName, "Channel": groupAddr.String()})
+	if deviceID == "" {
+		for device := range ig.Devices {
+			ig.DelIgmpChannel(device, groupAddr)
+		}
+		return nil
+	}
+	igd := ig.Devices[deviceID]
+	receivers := igd.DelChannelReceiver(groupAddr)
+	if igd.NumReceivers() == 0 {
+		ig.DelIgmpGroupDevice(igd)
+	}
+	return receivers
+}
+
+// IsNewReceiver checks if the received port is new receiver or existing one.
+// Returns true if new receiver.
+func (ig *IgmpGroup) IsNewReceiver(device, uniPortID string, groupAddr net.IP) bool {
+	if ig == nil {
+		// IGMP group does not exists. So considering it as new receiver.
+		return true
+	}
+	logger.Debugw(ctx, "IGMP Group", log.Fields{"channel": groupAddr, "groupName": ig.GroupName}) // TODO: Remove me
+	igd, exists := ig.Devices[device]
+	if !exists || !igd.GroupInstalled {
+		// IGMP group not exists OR Group is not created in the device.
+		// So this is a new receiver.
+		logger.Debugw(ctx, "igd not exists or group is not created in device", log.Fields{"exists": exists}) // TODO: Remove me
+		return true
+	}
+	if igc, ok := igd.GroupChannels.Load(groupAddr.String()); ok {
+		logger.Debugw(ctx, "IGMP Channel receivers", log.Fields{"igc-receivers": igc.(*IgmpGroupChannel).CurReceivers}) // TODO: Remove me
+		_, rcvrExistCur := igc.(*IgmpGroupChannel).CurReceivers[uniPortID]
+		_, rcvrExistNew := igc.(*IgmpGroupChannel).NewReceivers[uniPortID]
+		if rcvrExistCur || rcvrExistNew {
+			// Existing receiver
+			return false
+		}
+	}
+	return true
+}
+
+// Tick for Addition of groups to an MVLAN profile
+func (ig *IgmpGroup) Tick() {
+	now := time.Now()
+	for _, igd := range ig.Devices {
+		var igdChangeCnt uint8
+
+		if _, ok := GetApplication().DevicesDisc.Load(igd.Device); !ok {
+			logger.Info(ctx, "Skipping Query and Expiry check since Device is unavailable")
+			continue
+		}
+		if now.After(igd.NextQueryTime) {
+			// Set the next query time and the query expiry time to
+			// KeepAliveInterval and MaxResp seconds after current time
+			igd.NextQueryTime = now.Add(time.Duration(igd.proxyCfg.KeepAliveInterval) * time.Second)
+			igd.QueryExpiryTime = now.Add(time.Duration(igd.proxyCfg.MaxResp) * time.Second)
+			logger.Debugw(ctx, "Query Start", log.Fields{"NextQuery": igd.NextQueryTime, "Expiry": igd.QueryExpiryTime})
+			igdChangeCnt++
+			logger.Debugw(ctx, "Sending Query to device", log.Fields{"Device": igd.Device})
+			sendQueryForAllChannels := func(key interface{}, value interface{}) bool {
+				igc := value.(*IgmpGroupChannel)
+				//TODO - Do generic query to avoid multiple msgs
+				igc.SendQuery()
+				return true
+			}
+			igd.GroupChannels.Range(sendQueryForAllChannels)
+		}
+		if now.After(igd.QueryExpiryTime) {
+			igd.QueryExpiry()
+			// This will keep it quiet till the next query time and then
+			// it will be reset to a value after the query initiation time
+			igd.QueryExpiryTime = igd.NextQueryTime
+			logger.Debugw(ctx, "Expiry", log.Fields{"NextQuery": igd.NextQueryTime, "Expiry": igd.QueryExpiryTime})
+			igdChangeCnt++
+			if igd.NumReceivers() == 0 {
+				ig.DelIgmpGroupDevice(igd)
+				continue
+			}
+		}
+
+		igdChangeCnt += igd.Tick()
+
+		if igdChangeCnt > 0 {
+			if err := igd.WriteToDb(); err != nil {
+				logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device,
+							"GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+			}
+		}
+	}
+}
+
+// QueryExpiry processes expiry of query sent to the receivers. Up on
+// expiry, process the consolidated response for each of the devices participating
+// in the MC stream. When a device has no receivers, the device is deleted
+// from the group.
+func (ig *IgmpGroup) QueryExpiry() {
+	for _, igd := range ig.Devices {
+		if _, ok := GetApplication().DevicesDisc.Load(igd.Device); ok {
+			igd.QueryExpiry()
+			if igd.NumReceivers() == 0 {
+				ig.DelIgmpGroupDevice(igd)
+			}
+
+		} else {
+			logger.Info(ctx, "Skipping Expiry since Device is unavailable")
+		}
+	}
+}
+
+// Hash : The IGMP group hash is used to distribute the processing of timers so that
+// the processing is spread across doesn't spike at one instant. This also
+// ensures that there is sufficient responsiveness to other requests happening
+// simultaneously.
+func (ig *IgmpGroup) Hash() uint16 {
+	mvp := GetApplication().GetMvlanProfileByTag(ig.Mvlan)
+
+	if mvp == nil {
+		return 0
+	}
+
+	mvp.mvpLock.RLock()
+	defer mvp.mvpLock.RUnlock()
+	group := mvp.Groups[ig.GroupName]
+
+	//Case where mvlan update in-progress
+	if group == nil || len(group.McIPs) == 0 {
+		return 0
+	}
+	groupIP := group.McIPs[0]
+	return uint16(groupIP[2])<<8 + uint16(groupIP[3])
+}
+
+// NumDevicesAll returns the number of devices (OLT) active on the IGMP group. When
+// the last device leaves the IGMP group is removed. If this is not done,
+// the number of IGMP groups only keep increasing and can impact CPU when
+// the system runs for a very long duration
+func (ig *IgmpGroup) NumDevicesAll() int {
+	return len(ig.Devices)
+}
+
+// NumDevicesActive returns the number of devices (OLT) active on the IGMP group. When
+// the last device leaves the IGMP group is removed. If this is not done,
+// the number of IGMP groups only keep increasing and can impact CPU when
+// the system runs for a very long duration
+func (ig *IgmpGroup) NumDevicesActive() int {
+	count := 0
+	for _, igd := range ig.Devices {
+		if igd.NumReceivers() == 0 && igd.GroupInstalled {
+			continue
+		}
+		count++
+	}
+	return count
+}
+
+// NumReceivers to return receiver list
+func (ig *IgmpGroup) NumReceivers() map[string]int {
+	receiverList := make(map[string]int)
+	for device, igd := range ig.Devices {
+		receiverList[device] = igd.NumReceivers()
+	}
+	return receiverList
+}
+
+// RestoreDevices : IGMP group write to DB
+func (ig *IgmpGroup) RestoreDevices() {
+
+	ig.migrateIgmpDevices()
+	devices, _ := db.GetIgmpDevices(ig.Mvlan, ig.GroupName, ig.GroupAddr)
+	for _, device := range devices {
+		b, ok := device.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		if igd, err := NewIgmpGroupDeviceFromBytes(b); err == nil {
+			igd.PonPortChannelMap = util.NewConcurrentMap()
+			// Update the proxy config pointers.
+			var mcastCfg *McastConfig
+			igd.proxyCfg, igd.IgmpProxyIP, mcastCfg = getIgmpProxyCfgAndIP(ig.Mvlan, igd.SerialNo)
+			if mcastCfg != nil {
+				mcastCfg.IgmpGroupDevices.Store(igd.GroupID, igd)
+				logger.Debugw(ctx, "Igd added to mcast config", log.Fields{"mvlan": mcastCfg.MvlanProfileID, "groupId": igd.GroupID})
+			}
+
+			mvp := GetApplication().GetMvlanProfileByTag(igd.Mvlan)
+			igd.ServVersion = mvp.IgmpServVersion[igd.SerialNo]
+
+			// During vgc upgrade from old version, igd.NextQueryTime and igd.QueryExpiryTime will not be present in db.
+			// hence they are initialized with current time offset.
+			emptyTime := time.Time{}
+			if emptyTime == igd.NextQueryTime {
+				logger.Debugw(ctx, "VGC igd upgrade", log.Fields{"igd grp name": igd.GroupName})
+				igd.NextQueryTime = time.Now().Add(time.Duration(igd.proxyCfg.KeepAliveInterval) * time.Second)
+				igd.QueryExpiryTime = time.Now().Add(time.Duration(igd.proxyCfg.KeepAliveInterval) * time.Second)
+				if err := igd.WriteToDb(); err != nil {
+					logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device,
+								"GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+				}
+			}
+
+			ig.Devices[igd.Device] = igd
+			if ig.IsChannelBasedGroup {
+				channel, _ := db.GetIgmpChannel(igd.Mvlan, igd.GroupName, igd.Device, igd.GroupAddr)
+				igd.RestoreChannel([]byte(channel))
+			} else {
+				igd.RestoreChannels()
+			}
+			igd.PortChannelMap.Range(printPortChannel)
+			logger.Infow(ctx, "Group Device Restored", log.Fields{"IGD": igd})
+		} else {
+			logger.Warnw(ctx, "Unable to decode device from database", log.Fields{"str": string(b)})
+		}
+	}
+}
+
+// getKey to return group key
+func (ig *IgmpGroup) getKey() string {
+	profile, ok := GetApplication().MvlanProfilesByTag.Load(ig.Mvlan)
+	if ok {
+		mvp := profile.(*MvlanProfile)
+		return mvp.generateGroupKey(ig.GroupName, ig.GroupAddr.String())
+	}
+	return ""
+}
+
+/*
+// getKey to return group key
+func (igd *IgmpGroupDevice) getKey() string {
+	profile, ok := GetApplication().MvlanProfilesByTag.Load(igd.Mvlan)
+	if ok {
+		mvp := profile.(*MvlanProfile)
+		return mvp.generateGroupKey(igd.GroupName, igd.GroupAddr.String())
+	}
+	return ""
+}*/
+
+// generateGroupKey to generate group key
+func (mvp *MvlanProfile) generateGroupKey(name string, ipAddr string) string {
+	if mvp.IsChannelBasedGroup {
+		return mvp.Mvlan.String() + "_" + ipAddr
+	}
+	return mvp.Mvlan.String() + "_" + name
+}
+
+// WriteToDb is utility to write Igmp Group Info to database
+func (ig *IgmpGroup) WriteToDb() error {
+	ig.Version = database.PresentVersionMap[database.IgmpGroupPath]
+	b, err := json.Marshal(ig)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutIgmpGroup(ig.getKey(), string(b)); err1 != nil {
+		return err1
+	}
+	return nil
+}
+
+// RestoreIgmpGroupsFromDb to restore igmp groups from database
+func (va *VoltApplication) RestoreIgmpGroupsFromDb() {
+
+	groups, _ := db.GetIgmpGroups()
+	for _, group := range groups {
+		b, ok := group.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var ig IgmpGroup
+		err := json.Unmarshal(b, &ig)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of IGMP Group failed")
+			continue
+		}
+		ig.Devices = make(map[string]*IgmpGroupDevice)
+
+		//For Upgrade Case
+		if len(ig.PendingGroupForDevice) == 0 {
+			ig.PendingGroupForDevice = make(map[string]time.Time)
+		}
+		logger.Infow(ctx, "Restoring Groups", log.Fields{"igGroupID": ig.GroupID, "igGroupName": ig.GroupName, "igMvlan": ig.Mvlan})
+		grpKey := ig.getKey()
+		va.IgmpGroups.Store(grpKey, &ig)
+		// Just delete and lose the IGMP group with the same group Id
+		if _, err := va.GetIgmpGroupID(ig.GroupID); err != nil {
+			logger.Warnw(ctx, "GetIgmpGroupID Failed", log.Fields{"igGroupID": ig.GroupID, "Error": err})
+		}
+		ig.RestoreDevices()
+
+		if ig.NumDevicesActive() == 0 {
+			va.AddGroupToPendingPool(&ig)
+		}
+		logger.Infow(ctx, "Restored Groups", log.Fields{"igGroupID": ig.GroupID, "igGroupName": ig.GroupName, "igMvlan": ig.Mvlan})
+	}
+}
+
+// AddIgmpGroup : When the first IGMP packet is received, the MVLAN profile is identified
+// for the IGMP group and grp obj is obtained from the available pending pool of groups.
+// If not, new group obj will be created based on available group IDs
+func (va *VoltApplication) AddIgmpGroup(mvpName string, gip net.IP, device string) *IgmpGroup {
+
+	var ig *IgmpGroup
+	if mvp, grpName := va.GetMvlanProfileForMcIP(mvpName, gip); mvp != nil {
+		if ig = va.GetGroupFromPendingPool(mvp.Mvlan, device); ig != nil {
+			logger.Infow(ctx, "Igmp Group obtained from global pending pool", log.Fields{"MvlanProfile": mvpName, "GroupID": ig.GroupID, "Device": device, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr.String()})
+			oldKey := mvp.generateGroupKey(ig.GroupName, ig.GroupAddr.String())
+			ig.IgmpGroupReInit(grpName, gip)
+			ig.IsGroupStatic = mvp.Groups[grpName].IsStatic
+			ig.UpdateIgmpGroup(oldKey, ig.getKey())
+		} else {
+			logger.Infow(ctx, "No Igmp Group available in global pending pool. Creating new Igmp Group", log.Fields{"MvlanProfile": mvpName, "Device": device, "GroupAddr": gip.String()})
+			if ig = va.GetAvailIgmpGroupID(); ig == nil {
+				logger.Error(ctx, "Igmp Group Creation Failed: Group Id Unavailable")
+				return nil
+			}
+			ig.IgmpGroupInit(grpName, gip, mvp)
+			grpKey := ig.getKey()
+			va.IgmpGroups.Store(grpKey, ig)
+		}
+		if err := ig.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+		}
+		return ig
+	}
+	logger.Error(ctx, "GetMvlan Pro failed", log.Fields{"Group": gip})
+	return nil
+}
+
+// GetIgmpGroup helps search for the IGMP group from the list of
+// active IGMP groups. For now, the assumption is that a group
+// cannot belong to more than on MVLAN. If we change that definition,
+// we have to take a relook at this implementation. The key will include
+// both MVLAN and the group IP.
+func (va *VoltApplication) GetIgmpGroup(mvlan of.VlanType, gip net.IP) *IgmpGroup {
+
+	profile, _ := va.MvlanProfilesByTag.Load(mvlan)
+	if profile == nil {
+		logger.Errorw(ctx, "Mvlan Profile not found for incoming packet. Dropping Request", log.Fields{"Mvlan": mvlan, "GroupAddr": gip.String()})
+		return nil
+	}
+	mvp := profile.(*MvlanProfile)
+	_, gName := va.GetMvlanProfileForMcIP(mvp.Name, gip)
+	grpKey := mvp.generateGroupKey(gName, gip.String())
+	logger.Debugw(ctx, "Get IGMP Group", log.Fields{"Group": grpKey})
+	igIntf, ok := va.IgmpGroups.Load(grpKey)
+	if ok {
+		logger.Debugw(ctx, "Get IGMP Group Success", log.Fields{"Group": grpKey})
+		ig := igIntf.(*IgmpGroup)
+
+		//Case: Group was part of pending and Join came with same channel or different channel from same group
+		// (from same or different device)
+		// In that case, the same group will be allocated since the group is still part of va.IgmpGroups
+		// So, the groups needs to be removed from global pending pool
+		va.RemoveGroupDevicesFromPendingPool(ig)
+		return ig
+	}
+	return nil
+}
+
+// GetStaticGroupName to get static igmp group
+func (mvp *MvlanProfile) GetStaticGroupName(gip net.IP) string {
+	for _, mvg := range mvp.Groups {
+		if mvg.IsStatic {
+			if doesIPMatch(gip, mvg.McIPs) {
+				return mvg.Name
+			}
+		}
+	}
+	return ""
+}
+
+// GetStaticIgmpGroup to get static igmp group
+func (mvp *MvlanProfile) GetStaticIgmpGroup(gip net.IP) *IgmpGroup {
+
+	staticGroupName := mvp.GetStaticGroupName(gip)
+	grpKey := mvp.generateGroupKey(staticGroupName, gip.String())
+	logger.Debugw(ctx, "Get Static IGMP Group", log.Fields{"Group": grpKey})
+	ig, ok := GetApplication().IgmpGroups.Load(grpKey)
+	if ok {
+		logger.Debugw(ctx, "Get Static IGMP Group Success", log.Fields{"Group": grpKey})
+		return ig.(*IgmpGroup)
+	}
+	return nil
+}
+
+// UpdateIgmpGroup : When the pending group is allocated to new
+func (ig *IgmpGroup) UpdateIgmpGroup(oldKey, newKey string) {
+
+	//If the group is allocated to same McastGroup, no need to update the
+	//IgmpGroups map
+	if oldKey == newKey {
+		return
+	}
+	logger.Infow(ctx, "Updating Igmp Group with new MVP Group Info", log.Fields{"OldKey": oldKey, "NewKey": newKey, "GroupID": ig.GroupID})
+
+	GetApplication().IgmpGroups.Delete(oldKey)
+	_ = db.DelIgmpGroup(oldKey)
+
+	GetApplication().IgmpGroups.Store(newKey, ig)
+	if err := ig.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+	}
+}
+
+// DelIgmpGroup : When the last subscriber leaves the IGMP group across all the devices
+// the IGMP group is removed.
+func (va *VoltApplication) DelIgmpGroup(ig *IgmpGroup) {
+
+	profile, found := GetApplication().MvlanProfilesByTag.Load(ig.Mvlan)
+	if found {
+		mvp := profile.(*MvlanProfile)
+
+		grpKey := mvp.generateGroupKey(ig.GroupName, ig.GroupAddr.String())
+
+		if igIntf, ok := va.IgmpGroups.Load(grpKey); ok {
+			ig := igIntf.(*IgmpGroup)
+			ig.IgmpGroupLock.Lock()
+			if ig.NumDevicesAll() == 0 {
+				logger.Debugw(ctx, "Deleting IGMP Group", log.Fields{"Group": grpKey})
+				va.PutIgmpGroupID(ig)
+				va.IgmpGroups.Delete(grpKey)
+				_ = db.DelIgmpGroup(grpKey)
+			} else {
+				logger.Infow(ctx, "Skipping IgmpGroup Device. Pending Igmp Group Devices present", log.Fields{"GroupID": ig.GroupID, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr.String(), "PendingDevices": len(ig.Devices)})
+				va.AddGroupToPendingPool(ig)
+				if err := ig.WriteToDb(); err != nil {
+					logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+				}
+			}
+			ig.IgmpGroupLock.Unlock()
+		}
+
+	}
+}
+
+// GetPonPortID Gets the PON port ID from uniPortID
+func (va *VoltApplication) GetPonPortID(device, uniPortID string) uint32 {
+
+	isNNI := strings.Contains(uniPortID, "nni")
+	if isNNI || uniPortID == StaticPort {
+		logger.Debugw(ctx, "Cannot get pon port from UNI port", log.Fields{"port": uniPortID})
+		return 0xFF
+	}
+	dIntf, ok := va.DevicesDisc.Load(device)
+	if !ok {
+		return 0xFF
+	}
+	d := dIntf.(*VoltDevice)
+
+	uniPort := d.GetPort(uniPortID)
+	if uniPort == nil {
+		return 0xFF
+	}
+	return GetPonPortIDFromUNIPort(uniPort.ID)
+}
+
+// AggActiveChannelsCountPerSub aggregates the active channel count for given uni port.
+// It will iterate over all the groups and store the sum of active channels in VoltPort.
+func (va *VoltApplication) AggActiveChannelsCountPerSub(device, uniPort string, port *VoltPort) {
+	var activeChannelCount uint32
+
+	collectActiveChannelCount := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+		igd := ig.Devices[device]
+		if igd == nil {
+			return true
+		}
+		if portChannels, ok := igd.PortChannelMap.Load(uniPort); ok {
+			channelList := portChannels.([]net.IP)
+			activeChannelCount += uint32(len(channelList))
+		}
+		return true
+	}
+	va.IgmpGroups.Range(collectActiveChannelCount)
+
+	logger.Debugw(ctx, "AggrActiveChannelCount for Subscriber",
+		log.Fields{"UniPortID": uniPort, "count": activeChannelCount})
+
+	port.ActiveChannels = activeChannelCount
+}
+
+// AggActiveChannelsCountForPonPort Aggregates the active channel count for given pon port.
+// It will iterate over all the groups and store the sum of active channels in VoltDevice.
+func (va *VoltApplication) AggActiveChannelsCountForPonPort(device string, ponPortID uint32, port *PonPortCfg) {
+
+	var activeChannelCount uint32
+
+	collectActiveChannelCount := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+		igd := ig.Devices[device]
+		if igd == nil {
+			return true
+		}
+		if ponPortChannels, ok := igd.PonPortChannelMap.Get(ponPortID); ok {
+			activeChannelCount += ponPortChannels.(*PonPortChannels).GetActiveChannelCount()
+		}
+		return true
+	}
+	va.IgmpGroups.Range(collectActiveChannelCount)
+
+	logger.Debugw(ctx, "AggrActiveChannelCount for Pon Port",
+		log.Fields{"PonPortID": ponPortID, "count": activeChannelCount})
+
+	port.ActiveIGMPChannels = activeChannelCount
+}
+
+// UpdateActiveChannelCountForPonPort increments the global counter for active
+// channel count per pon port.
+func (va *VoltApplication) UpdateActiveChannelCountForPonPort(device, uniPortID string, ponPortID uint32, isAdd, isChannel bool, igd *IgmpGroupDevice) {
+	incrDecr := func(value uint32) uint32 {
+		if isAdd {
+			return value + 1
+		}
+		return value - 1
+	}
+	if d, exists := va.DevicesDisc.Load(device); exists {
+		voltDevice := d.(*VoltDevice)
+
+		if isChannel {
+			voltDevice.ActiveChannelCountLock.Lock()
+			// If New channel is added/deleted, then only update the ActiveChannelsPerPon
+			if value, ok := voltDevice.ActiveChannelsPerPon.Load(ponPortID); ok {
+				port := value.(*PonPortCfg)
+				port.ActiveIGMPChannels = incrDecr(port.ActiveIGMPChannels)
+				voltDevice.ActiveChannelsPerPon.Store(ponPortID, port)
+				logger.Debugw(ctx, "+++ActiveChannelsPerPon", log.Fields{"count": port.ActiveIGMPChannels}) // TODO: remove me
+			}
+			voltDevice.ActiveChannelCountLock.Unlock()
+		}
+		if uPort, ok := voltDevice.Ports.Load(uniPortID); ok {
+			uniPort := uPort.(*VoltPort)
+			uniPort.ActiveChannels = incrDecr(uniPort.ActiveChannels)
+			voltDevice.Ports.Store(uniPortID, uniPort)
+			logger.Debugw(ctx, "+++ActiveChannelsPerSub", log.Fields{"count": uniPort.ActiveChannels}) // TODO: remove me
+		}
+	}
+}
+
+// IsMaxChannelsCountExceeded checks if the PON port active channel
+// capacity and subscriber level channel capacity is reached to max allowed
+// channel per pon threshold. If Exceeds, return true else return false.
+func (va *VoltApplication) IsMaxChannelsCountExceeded(device, uniPortID string,
+	ponPortID uint32, ig *IgmpGroup, channelIP net.IP, mvp *MvlanProfile) bool {
+
+	// New receiver check is required to identify the IgmpReportMsg received
+	// in response to the IGMP Query sent from VGC.
+	if newReceiver := ig.IsNewReceiver(device, uniPortID, channelIP); !newReceiver {
+		logger.Debugw(ctx, "Not a new receiver. It is a response to IGMP Query",
+			log.Fields{"port": uniPortID, "channel": channelIP})
+		return false
+	}
+
+	if vDev, exists := va.DevicesDisc.Load(device); exists {
+		voltDevice := vDev.(*VoltDevice)
+
+		// Checking subscriber active channel count with maxChannelsAllowedPerSub
+		if uniPort, present := voltDevice.Ports.Load(uniPortID); present {
+			if uniPort.(*VoltPort).ActiveChannels >= mvp.MaxActiveChannels {
+				logger.Errorw(ctx, "Max allowed channels per subscriber is exceeded",
+					log.Fields{"activeCount": uniPort.(*VoltPort).ActiveChannels, "channel": channelIP, "UNI": uniPort.(*VoltPort).Name})
+				if !(uniPort.(*VoltPort).ChannelPerSubAlarmRaised) {
+					serviceName := GetMcastServiceForSubAlarm(uniPort.(*VoltPort), mvp)
+					logger.Debugw(ctx, "Raising-SendActiveChannelPerSubscriberAlarm-Initiated", log.Fields{"ActiveChannels": uniPort.(*VoltPort).ActiveChannels, "ServiceName": serviceName})
+					uniPort.(*VoltPort).ChannelPerSubAlarmRaised = true
+				}
+				return true
+			}
+		} else {
+			logger.Errorw(ctx, "UNI port not found in VoltDevice", log.Fields{"uniPortID": uniPortID})
+		}
+		if value, ok := voltDevice.ActiveChannelsPerPon.Load(ponPortID); ok {
+			ponPort := value.(*PonPortCfg)
+
+			logger.Debugw(ctx, "----Active channels count for PON port",
+				log.Fields{"PonPortID": ponPortID, "activeChannels": ponPort.ActiveIGMPChannels,
+					"maxAllowedChannelsPerPon": ponPort.MaxActiveChannels})
+
+			if ponPort.ActiveIGMPChannels < ponPort.MaxActiveChannels {
+				// PON port active channel capacity is not yet reached to max allowed channels per pon.
+				// So allowing to add receiver.
+				return false
+			} else if ponPort.ActiveIGMPChannels >= ponPort.MaxActiveChannels && ig != nil {
+				// PON port active channel capacity is reached to max allowed channels per pon.
+				// Check if same channel is already configured on that PON port.
+				// If that channel is present, then allow AddReceiver else it will be rejected.
+				igd, isPresent := ig.Devices[device]
+				if isPresent {
+					if channelListForPonPort, _ := igd.PonPortChannelMap.Get(ponPortID); channelListForPonPort != nil {
+						if _, isExists := channelListForPonPort.(*PonPortChannels).ChannelList.Get(channelIP.String()); isExists {
+							return false
+						}
+					}
+				}
+			}
+			logger.Errorw(ctx, "Active channels count for PON port exceeded",
+				log.Fields{"PonPortID": ponPortID, "activeChannels": ponPort.ActiveIGMPChannels, "channel": channelIP, "UNI": uniPortID})
+		} else {
+			logger.Warnw(ctx, "PON port level active channel count does not exists",
+				log.Fields{"ponPortID": ponPortID})
+			return false
+		}
+	}
+	logger.Warnw(ctx, "Max allowed channels per pon threshold is reached", log.Fields{"PonPortID": ponPortID})
+	return true
+}
+
+// ProcessIgmpv2Pkt : This is IGMPv2 packet.
+func (va *VoltApplication) ProcessIgmpv2Pkt(device string, port string, pkt gopacket.Packet) {
+	// First get the layers of interest
+	dot1Q := pkt.Layer(layers.LayerTypeDot1Q).(*layers.Dot1Q)
+	pktVlan := of.VlanType(dot1Q.VLANIdentifier)
+	igmpv2 := pkt.Layer(layers.LayerTypeIGMP).(*layers.IGMPv1or2)
+
+	ponPortID := va.GetPonPortID(device, port)
+
+	var vpv *VoltPortVnet
+
+	logger.Debugw(ctx, "Received IGMPv2 Type", log.Fields{"Type": igmpv2.Type})
+
+	if igmpv2.Type == layers.IGMPMembershipReportV2 || igmpv2.Type == layers.IGMPMembershipReportV1 {
+
+		logger.Infow(ctx, "IGMP Join received: v2", log.Fields{"Addr": igmpv2.GroupAddress, "Port": port})
+
+		// This is a report coming from the PON. We must be able to first find the
+		// subscriber from the VLAN tag and port and verify if the IGMP proxy is
+		// enabled for the subscriber
+		vpv, _ = va.GetVnetFromPkt(device, port, pkt)
+
+		if vpv == nil {
+			logger.Errorw(ctx, "Couldn't find VNET associated with port", log.Fields{"Port": port})
+			return
+		} else if !vpv.IgmpEnabled {
+			logger.Errorw(ctx, "IGMP is not activated on the port", log.Fields{"Port": port})
+			return
+		}
+
+		mvp := va.GetMvlanProfileByName(vpv.MvlanProfileName)
+		if mvp == nil {
+			logger.Errorw(ctx, "Igmp Packet Received for Subscriber with Missing Mvlan Profile",
+				log.Fields{"Receiver": vpv.Port, "MvlanProfile": vpv.MvlanProfileName})
+			return
+		}
+		mvlan := mvp.Mvlan
+
+		mvp.mvpLock.RLock()
+		defer mvp.mvpLock.RUnlock()
+		// The subscriber is validated and now process the IGMP report
+		ig := va.GetIgmpGroup(mvlan, igmpv2.GroupAddress)
+
+		if yes := va.IsMaxChannelsCountExceeded(device, port, ponPortID, ig, igmpv2.GroupAddress, mvp); yes {
+			logger.Warnw(ctx, "Dropping IGMP Join v2: Active channel threshold exceeded",
+				log.Fields{"PonPortID": ponPortID, "Addr": igmpv2.GroupAddress, "MvlanProfile": vpv.MvlanProfileName})
+			return
+		}
+		if ig != nil {
+			logger.Infow(ctx, "IGMP Group", log.Fields{"Group": ig.GroupID, "devices": ig.Devices})
+			// If the IGMP group is already created. just add the receiver
+			ig.IgmpGroupLock.Lock()
+			// Check for port state to avoid race condition where PortDown event
+			// acquired lock before packet processing
+			vd := GetApplication().GetDevice(device)
+			vp := vd.GetPort(port)
+			if vp == nil || vp.State != PortStateUp {
+				logger.Warnw(ctx, "Join received from a Port that is DOWN or not present",
+						log.Fields{"Port": port})
+				ig.IgmpGroupLock.Unlock()
+				return
+			}
+			ig.AddReceiver(device, port, igmpv2.GroupAddress, nil, IgmpVersion2, dot1Q.VLANIdentifier, dot1Q.Priority, ponPortID)
+			ig.IgmpGroupLock.Unlock()
+		} else {
+			// Create the IGMP group and then add the receiver to the group
+			if ig := va.AddIgmpGroup(vpv.MvlanProfileName, igmpv2.GroupAddress, device); ig != nil {
+				logger.Infow(ctx, "New IGMP Group", log.Fields{"Group": ig.GroupID, "devices": ig.Devices})
+				ig.IgmpGroupLock.Lock()
+				// Check for port state to avoid race condition where PortDown event
+				// acquired lock before packet processing
+				vd := GetApplication().GetDevice(device)
+				vp := vd.GetPort(port)
+				if vp == nil || vp.State != PortStateUp {
+					logger.Warnw(ctx, "Join received from a Port that is DOWN or not present",
+							log.Fields{"Port": port})
+					ig.IgmpGroupLock.Unlock()
+					return
+				}
+				ig.AddReceiver(device, port, igmpv2.GroupAddress, nil, IgmpVersion2, dot1Q.VLANIdentifier, dot1Q.Priority, ponPortID)
+				ig.IgmpGroupLock.Unlock()
+			} else {
+				logger.Errorw(ctx, "IGMP Group Creation Failed", log.Fields{"Addr": igmpv2.GroupAddress})
+				return
+			}
+		}
+	} else if igmpv2.Type == layers.IGMPLeaveGroup {
+		// This is a IGMP leave coming from one of the receivers. We essentially remove the
+		// the receiver.
+		logger.Infow(ctx, "IGMP Leave received: v2", log.Fields{"Addr": igmpv2.GroupAddress, "Port": port})
+
+		vpv, _ = va.GetVnetFromPkt(device, port, pkt)
+		if vpv == nil {
+			logger.Errorw(ctx, "Couldn't find VNET associated with port", log.Fields{"Port": port})
+			return
+		} else if !vpv.IgmpEnabled {
+			logger.Errorw(ctx, "IGMP is not activated on the port", log.Fields{"Port": port})
+			return
+		}
+
+		mvp := va.GetMvlanProfileByName(vpv.MvlanProfileName)
+		mvp.mvpLock.RLock()
+		defer mvp.mvpLock.RUnlock()
+		mvlan := mvp.Mvlan
+		// The subscriber is validated and now process the IGMP report
+		if ig := va.GetIgmpGroup(mvlan, igmpv2.GroupAddress); ig != nil {
+			ig.IgmpGroupLock.Lock()
+			// Delete the receiver once the IgmpGroup is identified
+			ig.DelReceiver(device, port, igmpv2.GroupAddress, nil, ponPortID)
+			ig.IgmpGroupLock.Unlock()
+			if ig.NumDevicesActive() == 0 {
+				va.DelIgmpGroup(ig)
+			}
+		}
+	} else {
+		// This must be a query on the NNI port. However, we dont make that assumption.
+		// Need to look for the IGMP group based on the VLAN in the packet as
+		// the MVLAN
+
+		//Check if mvlan profile exist for the incoming pkt vlan
+		profile, _ := va.MvlanProfilesByTag.Load(pktVlan)
+		if profile == nil {
+			logger.Errorw(ctx, "Mvlan Profile not found for incoming packet. Dropping Request", log.Fields{"Mvlan": pktVlan})
+			return
+		}
+		mvp := profile.(*MvlanProfile)
+		mvp.mvpLock.RLock()
+		defer mvp.mvpLock.RUnlock()
+
+		if net.ParseIP("0.0.0.0").Equal(igmpv2.GroupAddress) {
+			va.processIgmpQueries(device, pktVlan, IgmpVersion2)
+		} else {
+			if ig := va.GetIgmpGroup(pktVlan, igmpv2.GroupAddress); ig != nil {
+				ig.IgmpGroupLock.Lock()
+				igd, ok := ig.Devices[device]
+				if ok {
+					igd.ProcessQuery(igmpv2.GroupAddress, IgmpVersion2)
+				} else {
+					logger.Warnw(ctx, "IGMP Device not found", log.Fields{"Device": device, "Group": igmpv2.GroupAddress})
+				}
+				ig.IgmpGroupLock.Unlock()
+			}
+		}
+	}
+}
+
+// ProcessIgmpv3Pkt : Process IGMPv3 packet
+func (va *VoltApplication) ProcessIgmpv3Pkt(device string, port string, pkt gopacket.Packet) {
+	// First get the layers of interest
+	dot1QLayer := pkt.Layer(layers.LayerTypeDot1Q)
+
+	if dot1QLayer == nil {
+		logger.Error(ctx, "Igmp Packet Received without Vlan - Dropping pkt")
+		return
+	}
+	dot1Q := dot1QLayer.(*layers.Dot1Q)
+	pktVlan := of.VlanType(dot1Q.VLANIdentifier)
+	igmpv3 := pkt.Layer(layers.LayerTypeIGMP).(*layers.IGMP)
+
+	ponPortID := va.GetPonPortID(device, port)
+
+	var vpv *VoltPortVnet
+	logger.Debugw(ctx, "Received IGMPv3 Type", log.Fields{"Type": igmpv3.Type})
+
+	if igmpv3.Type == layers.IGMPMembershipReportV3 {
+		// This is a report coming from the PON. We must be able to first find the
+		// subscriber from the VLAN tag and port and verify if the IGMP proxy is
+		// enabled for the subscriber
+		vpv, _ = va.GetVnetFromPkt(device, port, pkt)
+		if vpv == nil {
+			logger.Errorw(ctx, "Couldn't find VNET associated with port", log.Fields{"Port": port})
+			return
+		} else if !vpv.IgmpEnabled {
+			logger.Errorw(ctx, "IGMP is not activated on the port", log.Fields{"Port": port})
+			return
+		}
+		mvp := va.GetMvlanProfileByName(vpv.MvlanProfileName)
+		if mvp == nil {
+			logger.Errorw(ctx, "Igmp Packet received for Subscriber with Missing Mvlan Profile",
+				log.Fields{"Receiver": vpv.Port, "MvlanProfile": vpv.MvlanProfileName})
+			return
+		}
+		mvp.mvpLock.RLock()
+		defer mvp.mvpLock.RUnlock()
+		mvlan := mvp.Mvlan
+
+		for _, group := range igmpv3.GroupRecords {
+
+			isJoin := isIgmpJoin(group.Type, group.SourceAddresses)
+			// The subscriber is validated and now process the IGMP report
+			ig := va.GetIgmpGroup(mvlan, group.MulticastAddress)
+			if isJoin {
+				if yes := va.IsMaxChannelsCountExceeded(device, port, ponPortID, ig, group.MulticastAddress, mvp); yes {
+					logger.Warnw(ctx, "Dropping IGMP Join v3: Active channel threshold exceeded",
+						log.Fields{"PonPortID": ponPortID, "Addr": group.MulticastAddress, "MvlanProfile": vpv.MvlanProfileName})
+
+					return
+				}
+				if ig != nil {
+					// If the IGMP group is already created. just add the receiver
+					logger.Infow(ctx, "IGMP Join received for existing group", log.Fields{"Addr": group.MulticastAddress, "Port": port})
+					ig.IgmpGroupLock.Lock()
+					// Check for port state to avoid race condition where PortDown event
+					// acquired lock before packet processing
+					vd := GetApplication().GetDevice(device)
+					vp := vd.GetPort(port)
+					if vp == nil || vp.State != PortStateUp {
+						logger.Warnw(ctx, "Join received from a Port that is DOWN or not present",
+								log.Fields{"Port": port})
+						ig.IgmpGroupLock.Unlock()
+						return
+					}
+					ig.AddReceiver(device, port, group.MulticastAddress, &group, IgmpVersion3,
+						dot1Q.VLANIdentifier, dot1Q.Priority, ponPortID)
+					ig.IgmpGroupLock.Unlock()
+				} else {
+					// Create the IGMP group and then add the receiver to the group
+					logger.Infow(ctx, "IGMP Join received for new group", log.Fields{"Addr": group.MulticastAddress, "Port": port})
+					if ig := va.AddIgmpGroup(vpv.MvlanProfileName, group.MulticastAddress, device); ig != nil {
+						ig.IgmpGroupLock.Lock()
+						// Check for port state to avoid race condition where PortDown event
+						// acquired lock before packet processing
+						vd := GetApplication().GetDevice(device)
+						vp := vd.GetPort(port)
+						if vp == nil || vp.State != PortStateUp {
+							logger.Warnw(ctx, "Join received from a Port that is DOWN or not present",
+									log.Fields{"Port": port})
+							ig.IgmpGroupLock.Unlock()
+							return
+						}
+						ig.AddReceiver(device, port, group.MulticastAddress, &group, IgmpVersion3,
+							dot1Q.VLANIdentifier, dot1Q.Priority, ponPortID)
+						ig.IgmpGroupLock.Unlock()
+					} else {
+						logger.Warnw(ctx, "IGMP Group Creation Failed", log.Fields{"Addr": group.MulticastAddress})
+					}
+				}
+			} else if ig != nil {
+				logger.Infow(ctx, "IGMP Leave received for existing group", log.Fields{"Addr": group.MulticastAddress, "Port": port})
+				ig.IgmpGroupLock.Lock()
+				ig.DelReceiver(device, port, group.MulticastAddress, &group, ponPortID)
+				ig.IgmpGroupLock.Unlock()
+				if ig.NumDevicesActive() == 0 {
+					va.DelIgmpGroup(ig)
+				}
+			} else {
+				logger.Warnw(ctx, "IGMP Leave received for unknown group", log.Fields{"Addr": group.MulticastAddress})
+			}
+		}
+	} else {
+		// This must be a query on the NNI port. However, we dont make that assumption.
+		// Need to look for the IGMP group based on the VLAN in the packet as
+		// the MVLAN
+
+		//Check if mvlan profile exist for the incoming pkt vlan
+		profile, _ := va.MvlanProfilesByTag.Load(pktVlan)
+		if profile == nil {
+			logger.Errorw(ctx, "Mvlan Profile not found for incoming packet. Dropping Request", log.Fields{"Mvlan": pktVlan})
+			return
+		}
+		mvp := profile.(*MvlanProfile)
+		mvp.mvpLock.RLock()
+		defer mvp.mvpLock.RUnlock()
+
+		if net.ParseIP("0.0.0.0").Equal(igmpv3.GroupAddress) {
+			va.processIgmpQueries(device, pktVlan, IgmpVersion3)
+		} else {
+			if ig := va.GetIgmpGroup(pktVlan, igmpv3.GroupAddress); ig != nil {
+				ig.IgmpGroupLock.Lock()
+				igd, ok := ig.Devices[device]
+				if ok {
+					igd.ProcessQuery(igmpv3.GroupAddress, IgmpVersion3)
+				} else {
+					logger.Warnw(ctx, "IGMP Device not found", log.Fields{"Device": device, "Group": igmpv3.GroupAddress})
+				}
+				ig.IgmpGroupLock.Unlock()
+			}
+		}
+	}
+}
+
+// processIgmpQueries to process the igmp queries
+func (va *VoltApplication) processIgmpQueries(device string, pktVlan of.VlanType, version uint8) {
+	// This is a generic query and respond with all the groups channels in currently being viewed.
+	processquery := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+		ig.IgmpGroupLock.Lock()
+		if ig.Mvlan != pktVlan {
+			ig.IgmpGroupLock.Unlock()
+			return true
+		}
+		igd, ok := ig.Devices[device]
+		if !ok {
+			logger.Warnw(ctx, "IGMP Device not found", log.Fields{"Device": device})
+			ig.IgmpGroupLock.Unlock()
+			return true
+		}
+		processQueryForEachChannel := func(key interface{}, value interface{}) bool {
+			groupAddr := key.(string)
+			igd.ProcessQuery(net.ParseIP(groupAddr), version)
+			return true
+		}
+		igd.GroupChannels.Range(processQueryForEachChannel)
+		ig.IgmpGroupLock.Unlock()
+		return true
+	}
+	va.IgmpGroups.Range(processquery)
+}
+
+// isIgmpJoin to check if it is igmp join
+func isIgmpJoin(recordType layers.IGMPv3GroupRecordType, sourceAddr []net.IP) bool {
+	var join = false
+
+	if (layers.IGMPToEx == recordType) || (layers.IGMPIsEx == recordType) {
+		join = true
+	} else if layers.IGMPBlock == recordType {
+		if len(sourceAddr) == 0 {
+			join = true
+		}
+	} else if (layers.IGMPToIn == recordType) || (layers.IGMPIsIn == recordType) || (layers.IGMPAllow == recordType) {
+		if len(sourceAddr) != 0 {
+			join = true
+		}
+	}
+	return join
+}
+
+func isIncl(recordType layers.IGMPv3GroupRecordType) bool {
+
+	if (layers.IGMPToIn == recordType) || (layers.IGMPIsIn == recordType) || (layers.IGMPAllow == recordType) {
+		return true
+	}
+	return false
+}
+
+// IgmpProcessPkt to process the IGMP packet received. The packet received brings along with it
+// the port on which the packet is received and the device the port is in.
+func (va *VoltApplication) IgmpProcessPkt(device string, port string, pkt gopacket.Packet) {
+	igmpl := pkt.Layer(layers.LayerTypeIGMP)
+	if igmpl == nil {
+		logger.Error(ctx, "Invalid IGMP packet arrived as IGMP packet")
+		return
+	}
+	if igmp, ok := igmpl.(*layers.IGMPv1or2); ok {
+		// This is an IGMPv2 packet.
+		logger.Debugw(ctx, "IGMPv2 Packet Received", log.Fields{"IPAddr": igmp.GroupAddress})
+		va.ProcessIgmpv2Pkt(device, port, pkt)
+		return
+	}
+	if igmpv3, ok := igmpl.(*layers.IGMP); ok {
+		logger.Debugw(ctx, "IGMPv3 Packet Received", log.Fields{"NumOfGroups": igmpv3.NumberOfGroupRecords})
+		va.ProcessIgmpv3Pkt(device, port, pkt)
+	}
+}
+
+// IgmpPacketInd for igmp packet indication
+func (va *VoltApplication) IgmpPacketInd(device string, port string, pkt gopacket.Packet) {
+	pt := NewIgmpPacketTask(device, port, pkt)
+	va.IgmpTasks.AddTask(pt)
+}
+
+// ------------------------------------------------------------
+// MVLAN related implemnetation
+//
+// Each MVLAN is configured with groups of multicast IPs. The idea of
+// groups is to be able to group some multicast channels into an individual
+// PON group and have a unique multicast GEM port for that set. However, in
+// the current implementation, the concept of grouping is not fully utilized.
+
+// MvlanGroup structure
+// A set of MC IPs form a group
+
+// MCGroupProxy identifies source specific multicast(SSM) config.
+type MCGroupProxy struct {
+	// Mode represents source list include/exclude
+	Mode common.MulticastSrcListMode
+	// SourceList represents list of multicast server IP addresses.
+	SourceList []net.IP
+}
+
+// MvlanGroup identifies MC group info
+type MvlanGroup struct {
+	Name     string
+	Wildcard bool
+	McIPs    []string
+	IsStatic bool
+}
+
+// OperInProgress type
+type OperInProgress uint8
+
+const (
+	// UpdateInProgress constant
+	UpdateInProgress OperInProgress = 2
+	// NoOp constant
+	NoOp OperInProgress = 1
+	// Nil constant
+	Nil OperInProgress = 0
+)
+
+// MvlanProfile : A set of groups of MC IPs for a MVLAN profile. It is assumed that
+// the MVLAN IP is not repeated within multiples groups and across
+// MVLAN profiles. The first match is used up on search to lcoate the
+// MVLAN profile for an MC IP
+type MvlanProfile struct {
+	Name                string
+	Mvlan               of.VlanType
+	PonVlan             of.VlanType
+	Groups              map[string]*MvlanGroup
+	Proxy               map[string]*MCGroupProxy
+	Version             string
+	IsPonVlanPresent    bool
+	IsChannelBasedGroup bool
+	DevicesList         map[string]OperInProgress //device serial number //here
+	oldGroups           map[string]*MvlanGroup
+	oldProxy            map[string]*MCGroupProxy
+	MaxActiveChannels   uint32
+	PendingDeleteFlow   map[string]map[string]bool
+	DeleteInProgress    bool
+	IgmpServVersion     map[string]*uint8
+	mvpLock             sync.RWMutex
+	mvpFlowLock         sync.RWMutex
+}
+
+// NewMvlanProfile is constructor for MVLAN profile.
+func NewMvlanProfile(name string, mvlan of.VlanType, ponVlan of.VlanType, isChannelBasedGroup bool, OLTSerialNums []string, actChannelPerPon uint32) *MvlanProfile {
+	var mvp MvlanProfile
+	mvp.Name = name
+	mvp.Mvlan = mvlan
+	mvp.PonVlan = ponVlan
+	mvp.mvpLock = sync.RWMutex{}
+	mvp.Groups = make(map[string]*MvlanGroup)
+	mvp.Proxy = make(map[string]*MCGroupProxy)
+	mvp.DevicesList = make(map[string]OperInProgress)
+	mvp.PendingDeleteFlow = make(map[string]map[string]bool)
+	mvp.IsChannelBasedGroup = isChannelBasedGroup
+	mvp.MaxActiveChannels = actChannelPerPon
+	mvp.DeleteInProgress = false
+	mvp.IgmpServVersion = make(map[string]*uint8)
+
+	if (ponVlan != of.VlanNone) && (ponVlan != 0) {
+		mvp.IsPonVlanPresent = true
+	}
+	return &mvp
+}
+
+// AddMvlanProxy for addition of groups to an MVLAN profile
+func (mvp *MvlanProfile) AddMvlanProxy(name string, proxyInfo common.MulticastGroupProxy) {
+	proxy := &MCGroupProxy{}
+	proxy.Mode = proxyInfo.Mode
+	proxy.SourceList = util.GetExpIPList(proxyInfo.SourceList)
+
+	if _, ok := mvp.Proxy[name]; !ok {
+		logger.Debugw(ctx, "Added MVLAN Proxy", log.Fields{"Name": name, "Proxy": proxy})
+	} else {
+		logger.Debugw(ctx, "Updated MVLAN Proxy", log.Fields{"Name": name, "Proxy": proxy})
+	}
+	if proxyInfo.IsStatic == common.IsStaticYes {
+		mvp.Groups[name].IsStatic = true
+	}
+	mvp.Proxy[name] = proxy
+}
+
+// AddMvlanGroup for addition of groups to an MVLAN profile
+func (mvp *MvlanProfile) AddMvlanGroup(name string, ips []string) {
+	mvg := &MvlanGroup{}
+	mvg.Name = name
+	mvg.Wildcard = len(ips) == 0
+	mvg.McIPs = ips
+	mvg.IsStatic = false
+	if _, ok := mvp.Groups[name]; !ok {
+		logger.Debugw(ctx, "Added MVLAN Group", log.Fields{"VLAN": mvp.Mvlan, "Name": name, "mvg": mvg, "IPs": mvg.McIPs})
+	} else {
+		logger.Debugw(ctx, "Updated MVLAN Group", log.Fields{"VLAN": mvp.Mvlan, "Name": name})
+	}
+	mvp.Groups[name] = mvg
+}
+
+// GetUsMatchVlan provides mvlan for US Match parameter
+func (mvp *MvlanProfile) GetUsMatchVlan() of.VlanType {
+	if mvp.IsPonVlanPresent {
+		return mvp.PonVlan
+	}
+	return mvp.Mvlan
+}
+
+// WriteToDb is utility to write Mvlan Profile Info to database
+func (mvp *MvlanProfile) WriteToDb() error {
+
+	if mvp.DeleteInProgress {
+		logger.Warnw(ctx, "Skipping Redis Update for MvlanProfile, MvlanProfile delete in progress", log.Fields{"Mvlan": mvp.Mvlan})
+		return nil
+	}
+
+	mvp.Version = database.PresentVersionMap[database.MvlanPath]
+	b, err := json.Marshal(mvp)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutMvlan(uint16(mvp.Mvlan), string(b)); err1 != nil {
+		return err1
+	}
+	return nil
+}
+
+//isChannelStatic - Returns true if the given channel is part of static group in the Mvlan Profile
+func (mvp *MvlanProfile) isChannelStatic(channel net.IP) bool {
+	for _, mvg := range mvp.Groups {
+		if mvg.IsStatic {
+			if isChannelStatic := doesIPMatch(channel, mvg.McIPs); isChannelStatic {
+				return true
+			}
+		}
+	}
+	return false
+}
+
+//containsStaticChannels - Returns if any static channels is part of the Mvlan Profile
+func (mvp *MvlanProfile) containsStaticChannels() bool {
+	for _, mvg := range mvp.Groups {
+		if mvg.IsStatic && len(mvg.McIPs) != 0 {
+			return true
+		}
+	}
+	return false
+}
+
+//getAllStaticChannels - Returns all static channels in the Mvlan Profile
+func (mvp *MvlanProfile) getAllStaticChannels() ([]net.IP, bool) {
+	channelList := []net.IP{}
+	containsStatic := false
+	for _, mvg := range mvp.Groups {
+		if mvg.IsStatic {
+			staticChannels, _ := mvg.getAllChannels()
+			channelList = append(channelList, staticChannels...)
+		}
+	}
+	if len(channelList) > 0 {
+		containsStatic = true
+	}
+	return channelList, containsStatic
+}
+
+//getAllOldGroupStaticChannels - Returns all static channels in the Mvlan Profile
+func (mvp *MvlanProfile) getAllOldGroupStaticChannels() ([]net.IP, bool) {
+	channelList := []net.IP{}
+	containsStatic := false
+	for _, mvg := range mvp.oldGroups {
+		if mvg.IsStatic {
+			staticChannels, _ := mvg.getAllChannels()
+			channelList = append(channelList, staticChannels...)
+		}
+	}
+	if len(channelList) > 0 {
+		containsStatic = true
+	}
+	return channelList, containsStatic
+}
+
+//getAllChannels - Returns all channels in the Mvlan Profile
+func (mvg *MvlanGroup) getAllChannels() ([]net.IP, bool) {
+	channelList := []net.IP{}
+
+	if mvg == nil || len(mvg.McIPs) == 0 {
+		return []net.IP{}, false
+	}
+
+	grpChannelOrRange := mvg.McIPs
+	for _, channelOrRange := range grpChannelOrRange {
+		if strings.Contains(channelOrRange, "-") {
+			var splits = strings.Split(channelOrRange, "-")
+			ipStart := util.IP2LongConv(net.ParseIP(splits[0]))
+			ipEnd := util.IP2LongConv(net.ParseIP(splits[1]))
+
+			for i := ipStart; i <= ipEnd; i++ {
+				channelList = append(channelList, util.Long2ipConv(i))
+			}
+		} else {
+			channelList = append(channelList, net.ParseIP(channelOrRange))
+		}
+	}
+	return channelList, true
+}
+
+//SetUpdateStatus - Sets profile update status for devices
+func (mvp *MvlanProfile) SetUpdateStatus(serialNum string, status OperInProgress) {
+	if serialNum != "" {
+		mvp.DevicesList[serialNum] = status
+		return
+	}
+
+	for srNo := range mvp.DevicesList {
+		mvp.DevicesList[srNo] = status
+	}
+}
+
+//isUpdateInProgress - checking is update is in progress for the mvlan profile
+func (mvp *MvlanProfile) isUpdateInProgress() bool {
+
+	for srNo := range mvp.DevicesList {
+		if mvp.DevicesList[srNo] == UpdateInProgress {
+			return true
+		}
+	}
+	return false
+}
+
+//IsUpdateInProgressForDevice - Checks is Mvlan Profile update is is progress for the given device
+func (mvp *MvlanProfile) IsUpdateInProgressForDevice(device string) bool {
+	if vd := GetApplication().GetDevice(device); vd != nil {
+		if mvp.DevicesList[vd.SerialNum] == UpdateInProgress {
+			return true
+		}
+	}
+	return false
+}
+
+// DelFromDb to delere mvlan from database
+func (mvp *MvlanProfile) DelFromDb() {
+	_ = db.DelMvlan(uint16(mvp.Mvlan))
+}
+
+// storeMvlansMap to store mvlan map
+func (va *VoltApplication) storeMvlansMap(mvlan of.VlanType, name string, mvp *MvlanProfile) {
+	va.MvlanProfilesByTag.Store(mvlan, mvp)
+	va.MvlanProfilesByName.Store(name, mvp)
+}
+
+// deleteMvlansMap to delete mvlan map
+func (va *VoltApplication) deleteMvlansMap(mvlan of.VlanType, name string) {
+	va.MvlanProfilesByTag.Delete(mvlan)
+	va.MvlanProfilesByName.Delete(name)
+}
+
+// RestoreMvlansFromDb to read from the DB and restore all the MVLANs
+func (va *VoltApplication) RestoreMvlansFromDb() {
+	mvlans, _ := db.GetMvlans()
+	for _, mvlan := range mvlans {
+		b, ok := mvlan.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var mvp MvlanProfile
+		err := json.Unmarshal(b, &mvp)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of MVLAN failed")
+			continue
+		}
+		va.storeMvlansMap(mvp.Mvlan, mvp.Name, &mvp)
+
+		for srNo := range mvp.DevicesList {
+			if mvp.IgmpServVersion[srNo] == nil {
+				servVersion := IgmpVersion0
+				mvp.IgmpServVersion[srNo] = &servVersion
+			}
+		}
+		logger.Infow(ctx, "Restored Mvlan Profile", log.Fields{"MVPName": mvp.Name})
+	}
+}
+
+// GetMvlanProfileByTag fetches MVLAN profile based on the MC VLAN
+func (va *VoltApplication) GetMvlanProfileByTag(vlan of.VlanType) *MvlanProfile {
+	if mvp, ok := va.MvlanProfilesByTag.Load(vlan); ok {
+		return mvp.(*MvlanProfile)
+	}
+	return nil
+}
+
+// GetMvlanProfileByName fetches MVLAN profile based on the profile name.
+func (va *VoltApplication) GetMvlanProfileByName(name string) *MvlanProfile {
+	if mvp, ok := va.MvlanProfilesByName.Load(name); ok {
+		return mvp.(*MvlanProfile)
+	}
+	return nil
+}
+
+//UpdateMvlanProfile - only channel groups be updated
+func (va *VoltApplication) UpdateMvlanProfile(name string, vlan of.VlanType, groups map[string][]string, activeChannelCount int, proxy map[string]common.MulticastGroupProxy) error {
+
+	mvpIntf, ok := va.MvlanProfilesByName.Load(name)
+	if !ok {
+		logger.Error(ctx, "Update Mvlan Failed: Profile does not exist")
+		return errors.New("MVLAN profile not found")
+	}
+	mvp := mvpIntf.(*MvlanProfile)
+	// check if groups are same then just update the OLTSerial numbers, push the config on new serial numbers
+
+	existingGroup := mvp.Groups
+	existingProxy := mvp.Proxy
+	mvp.Groups = make(map[string]*MvlanGroup)
+	mvp.Proxy = make(map[string]*MCGroupProxy)
+
+	/* Need to protect groups and proxy write lock */
+	mvp.mvpLock.Lock()
+	for grpName, grpIPList := range groups {
+		mvp.AddMvlanGroup(grpName, grpIPList)
+	}
+	for grpName, proxyInfo := range proxy {
+		mvp.AddMvlanProxy(grpName, proxyInfo)
+	}
+	if _, ok := mvp.Groups[common.StaticGroup]; ok {
+		if _, yes := mvp.Proxy[common.StaticGroup]; !yes {
+			mvp.Groups[common.StaticGroup].IsStatic = true
+		}
+	}
+	prevMaxActiveChannels := mvp.MaxActiveChannels
+	if reflect.DeepEqual(mvp.Groups, existingGroup) && reflect.DeepEqual(mvp.Proxy, existingProxy) {
+		logger.Info(ctx, "No change in groups config")
+		if uint32(activeChannelCount) != mvp.MaxActiveChannels {
+			mvp.MaxActiveChannels = uint32(activeChannelCount)
+			if err := mvp.WriteToDb(); err != nil {
+				logger.Errorw(ctx, "Mvlan profile Write to DB failed", log.Fields{"ProfileName": mvp.Name})
+			}
+			if prevMaxActiveChannels != mvp.MaxActiveChannels {
+				mvp.UpdateActiveChannelSubscriberAlarm()
+			}
+		}
+		mvp.mvpLock.Unlock()
+		return nil
+	}
+	mvp.mvpLock.Unlock()
+	mvp.MaxActiveChannels = uint32(activeChannelCount)
+
+	// Status is maintained so that in the event of any crash or reboot during update,
+	// the recovery is possible once the pod is UP again
+	mvp.SetUpdateStatus("", UpdateInProgress)
+	mvp.oldGroups = existingGroup
+	mvp.oldProxy = existingProxy
+	va.storeMvlansMap(vlan, name, mvp)
+	if err := mvp.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Mvlan profile Write to DB failed", log.Fields{"ProfileName": mvp.Name})
+	}
+	if prevMaxActiveChannels != mvp.MaxActiveChannels {
+		mvp.UpdateActiveChannelSubscriberAlarm()
+	}
+
+	// The update task is added as part of Igm p task list, so that any parallel igmp pkt processing is avoided
+	// Until, the update operation is completed, the igmp pkt processing will be enqueued
+	updateTask := NewUpdateMvlanTask(mvp, "")
+	va.IgmpTasks.AddTask(updateTask)
+	return nil
+}
+
+// isDeviceInList to check if device is the list
+func isDeviceInList(serialNum string, OLTSerialNums []string) bool {
+	for _, oltSerialNum := range OLTSerialNums {
+		if serialNum == oltSerialNum {
+			return true
+		}
+	}
+	return false
+}
+
+// McastConfigKey creates the key using the olt serial number and mvlan profile id
+func McastConfigKey(oltSerialNum string, mvlanProfID string) string {
+	return oltSerialNum + "_" + mvlanProfID
+}
+
+// GetMcastConfig to get McastConfig Information by OLT and Mvlan Profile ID
+func (va *VoltApplication) GetMcastConfig(oltSerialNum string, mvlanProfID string) *McastConfig {
+	if mc, ok := va.McastConfigMap.Load(McastConfigKey(oltSerialNum, mvlanProfID)); ok {
+		return mc.(*McastConfig)
+	}
+	return nil
+}
+
+func (va *VoltApplication) storeMcastConfig(oltSerialNum string, mvlanProfID string, mcastConfig *McastConfig) {
+	va.McastConfigMap.Store(McastConfigKey(oltSerialNum, mvlanProfID), mcastConfig)
+}
+
+func (va *VoltApplication) deleteMcastConfig(oltSerialNum string, mvlanProfID string) {
+	va.McastConfigMap.Delete(McastConfigKey(oltSerialNum, mvlanProfID))
+}
+
+// AddMcastConfig for addition of a MVLAN profile
+func (va *VoltApplication) AddMcastConfig(MvlanProfileID string, IgmpProfileID string, IgmpProxyIP string, OltSerialNum string) error {
+	var mcastCfg *McastConfig
+
+	mcastCfg = va.GetMcastConfig(OltSerialNum, MvlanProfileID)
+	if mcastCfg == nil {
+		mcastCfg = &McastConfig{}
+	} else {
+		logger.Debugw(ctx, "Mcast Config already exists", log.Fields{"OltSerialNum": mcastCfg.OltSerialNum,
+			"MVLAN Profile ID": mcastCfg.MvlanProfileID})
+	}
+
+	// Update all igds available
+	mvpIntf, ok := va.MvlanProfilesByName.Load(MvlanProfileID)
+	if !ok {
+		return errors.New("MVLAN profile not found during add mcast config")
+	}
+	mvlan := mvpIntf.(*MvlanProfile).Mvlan
+
+	mcastCfg.OltSerialNum = OltSerialNum
+	mcastCfg.MvlanProfileID = MvlanProfileID
+	mcastCfg.IgmpProfileID = IgmpProfileID
+	mcastCfg.IgmpProxyIP = net.ParseIP(IgmpProxyIP)
+
+	proxyCfg := va.getIgmpProfileMap(IgmpProfileID)
+
+	iterIgmpGroups := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+		if ig.Mvlan != mvlan {
+			return true
+		}
+
+		for _, igd := range ig.Devices {
+			if igd.SerialNo != OltSerialNum {
+				continue
+			}
+			igd.proxyCfg = proxyCfg
+			if IgmpProfileID == "" {
+				igd.IgmpProxyIP = &igd.proxyCfg.IgmpSourceIP
+			} else {
+				igd.IgmpProxyIP = &mcastCfg.IgmpProxyIP
+			}
+			mcastCfg.IgmpGroupDevices.Store(igd.GroupID, igd)
+			logger.Debugw(ctx, "Igd updated with proxyCfg and proxyIP", log.Fields{"name": igd.GroupName,
+				"IgmpProfileID": IgmpProfileID, "ProxyIP": mcastCfg.IgmpProxyIP})
+		}
+		return true
+	}
+	va.IgmpGroups.Range(iterIgmpGroups)
+
+	va.storeMcastConfig(OltSerialNum, MvlanProfileID, mcastCfg)
+	if err := mcastCfg.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "McastConfig Write to DB failed", log.Fields{"OltSerialNum": mcastCfg.OltSerialNum, "MvlanProfileID": mcastCfg.MvlanProfileID})
+	}
+	va.addOltToMvlan(MvlanProfileID, OltSerialNum)
+
+	return nil
+}
+
+func (va *VoltApplication) addOltToMvlan(MvlanProfileID string, OltSerialNum string) {
+	var mvp *MvlanProfile
+	if mvpIntf, ok := va.MvlanProfilesByName.Load(MvlanProfileID); ok {
+		servVersion := IgmpVersion0
+		mvp = mvpIntf.(*MvlanProfile)
+		mvp.DevicesList[OltSerialNum] = NoOp
+		mvp.IgmpServVersion[OltSerialNum] = &servVersion
+		if err := mvp.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Mvlan profile Write to DB failed", log.Fields{"ProfileName": mvp.Name})
+		}
+		mvp.pushIgmpMcastFlows(OltSerialNum)
+	}
+}
+
+func (va *VoltApplication) delOltFromMvlan(MvlanProfileID string, OltSerialNum string) {
+	var mvp *MvlanProfile
+	if mvpIntf, ok := va.MvlanProfilesByName.Load(MvlanProfileID); ok {
+		mvp = mvpIntf.(*MvlanProfile)
+		//Delete from mvp list
+		mvp.removeIgmpMcastFlows(OltSerialNum)
+		delete(mvp.DevicesList, OltSerialNum)
+		if err := mvp.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Mvlan profile Write to DB failed", log.Fields{"ProfileName": mvp.Name})
+		}
+	}
+}
+
+// DelMcastConfig for addition of a MVLAN profile
+func (va *VoltApplication) DelMcastConfig(MvlanProfileID string, IgmpProfileID string, IgmpProxyIP string, OltSerialNum string) {
+
+	va.delOltFromMvlan(MvlanProfileID, OltSerialNum)
+	va.deleteMcastConfig(OltSerialNum, MvlanProfileID)
+	_ = db.DelMcastConfig(McastConfigKey(OltSerialNum, MvlanProfileID))
+	if d := va.GetDeviceBySerialNo(OltSerialNum); d != nil {
+		if mvp := va.GetMvlanProfileByName(MvlanProfileID); mvp != nil {
+			va.RemoveGroupsFromPendingPool(d.Name, mvp.Mvlan)
+		}
+	}
+}
+
+// DelAllMcastConfig for deletion of all mcast config
+func (va *VoltApplication) DelAllMcastConfig(OltSerialNum string) error {
+
+	deleteIndividualMcastConfig := func(key interface{}, value interface{}) bool {
+		mcastCfg := value.(*McastConfig)
+		if mcastCfg.OltSerialNum == OltSerialNum {
+			va.DelMcastConfig(mcastCfg.MvlanProfileID, mcastCfg.IgmpProfileID, mcastCfg.IgmpProxyIP.String(), mcastCfg.OltSerialNum)
+		}
+		return true
+	}
+	va.McastConfigMap.Range(deleteIndividualMcastConfig)
+	return nil
+}
+
+// UpdateMcastConfig for addition of a MVLAN profile
+func (va *VoltApplication) UpdateMcastConfig(MvlanProfileID string, IgmpProfileID string, IgmpProxyIP string, OltSerialNum string) error {
+
+	mcastCfg := va.GetMcastConfig(OltSerialNum, MvlanProfileID)
+	if mcastCfg == nil {
+		logger.Warnw(ctx, "Mcast Config not found. Unable to update", log.Fields{"Mvlan Profile ID": MvlanProfileID, "OltSerialNum": OltSerialNum})
+		return nil
+	}
+
+	oldProfID := mcastCfg.IgmpProfileID
+	mcastCfg.IgmpProfileID = IgmpProfileID
+	mcastCfg.IgmpProxyIP = net.ParseIP(IgmpProxyIP)
+
+	va.storeMcastConfig(OltSerialNum, MvlanProfileID, mcastCfg)
+
+	// Update all igds
+	if oldProfID != mcastCfg.IgmpProfileID {
+		updateIgdProxyCfg := func(key interface{}, value interface{}) bool {
+			igd := value.(*IgmpGroupDevice)
+			igd.proxyCfg = va.getIgmpProfileMap(mcastCfg.IgmpProfileID)
+			if IgmpProfileID == "" {
+				igd.IgmpProxyIP = &igd.proxyCfg.IgmpSourceIP
+			} else {
+				igd.IgmpProxyIP = &mcastCfg.IgmpProxyIP
+			}
+			return true
+		}
+		mcastCfg.IgmpGroupDevices.Range(updateIgdProxyCfg)
+	}
+
+	if err := mcastCfg.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "McastConfig Write to DB failed", log.Fields{"OltSerialNum": mcastCfg.OltSerialNum, "MvlanProfileID": mcastCfg.MvlanProfileID})
+	}
+
+	return nil
+}
+
+// WriteToDb is utility to write Mcast config Info to database
+func (mc *McastConfig) WriteToDb() error {
+	mc.Version = database.PresentVersionMap[database.McastConfigPath]
+	b, err := json.Marshal(mc)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutMcastConfig(McastConfigKey(mc.OltSerialNum, mc.MvlanProfileID), string(b)); err1 != nil {
+		return err1
+	}
+	return nil
+}
+
+// RestoreMcastConfigsFromDb to read from the DB and restore Mcast configs
+func (va *VoltApplication) RestoreMcastConfigsFromDb() {
+	mcastConfigs, _ := db.GetMcastConfigs()
+	for hash, mcastConfig := range mcastConfigs {
+		b, ok := mcastConfig.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var mc McastConfig
+		err := json.Unmarshal(b, &mc)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of Mcast config failed")
+			continue
+		}
+		va.storeMcastConfig(mc.OltSerialNum, mc.MvlanProfileID, &mc)
+		logger.Infow(ctx, "Restored Mcast config", log.Fields{"OltSerialNum": mc.OltSerialNum, "MvlanProfileID": mc.MvlanProfileID, "hash": hash})
+	}
+}
+
+// AddMvlanProfile for addition of a MVLAN profile
+func (va *VoltApplication) AddMvlanProfile(name string, mvlan of.VlanType, ponVlan of.VlanType,
+	groups map[string][]string, isChannelBasedGroup bool, OLTSerialNum []string, activeChannelsPerPon int, proxy map[string]common.MulticastGroupProxy) error {
+	var mvp *MvlanProfile
+
+	if mvp = va.GetMvlanProfileByTag(mvlan); mvp != nil {
+		logger.Errorw(ctx, "Duplicate MVLAN ID configured", log.Fields{"mvlan": mvlan})
+		return errors.New("MVLAN profile with same VLANID exists")
+	}
+	if mvpIntf, ok := va.MvlanProfilesByName.Load(name); ok {
+		mvp = mvpIntf.(*MvlanProfile)
+		for _, serialNum := range OLTSerialNum {
+			if mvp.DevicesList[serialNum] != Nil {
+				//This is backup restore scenario, just update the profile
+				logger.Info(ctx, "Add Mvlan : Profile Name already exists, update-the-profile")
+				return va.UpdateMvlanProfile(name, mvlan, groups, activeChannelsPerPon, proxy)
+			}
+		}
+	}
+
+	if mvp == nil {
+		mvp = NewMvlanProfile(name, mvlan, ponVlan, isChannelBasedGroup, OLTSerialNum, uint32(activeChannelsPerPon))
+	}
+
+	va.storeMvlansMap(mvlan, name, mvp)
+
+	/* Need to protect groups and proxy write lock */
+	mvp.mvpLock.Lock()
+	for grpName, grpInfo := range groups {
+		mvp.AddMvlanGroup(grpName, grpInfo)
+	}
+	for grpName, proxyInfo := range proxy {
+		mvp.AddMvlanProxy(grpName, proxyInfo)
+	}
+	if _, ok := mvp.Groups[common.StaticGroup]; ok {
+		if _, yes := mvp.Proxy[common.StaticGroup]; !yes {
+			mvp.Groups[common.StaticGroup].IsStatic = true
+		}
+	}
+
+	logger.Debugw(ctx, "Added MVLAN Profile", log.Fields{"MVLAN": mvp.Mvlan, "PonVlan": mvp.PonVlan, "Name": mvp.Name, "Grp IPs": mvp.Groups, "IsPonVlanPresent": mvp.IsPonVlanPresent})
+	mvp.mvpLock.Unlock()
+
+	if err := mvp.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Mvlan profile Write to DB failed", log.Fields{"ProfileName": mvp.Name})
+	}
+
+	return nil
+}
+
+//pushIgmpMcastFlows - Adds all IGMP related flows (generic DS flow & static group flows)
+func (mvp *MvlanProfile) pushIgmpMcastFlows(OLTSerialNum string) {
+
+	mvp.mvpLock.RLock()
+	defer mvp.mvpLock.RUnlock()
+
+	if mvp.DevicesList[OLTSerialNum] == Nil {
+		logger.Infow(ctx, "Mvlan Profile not configure for device", log.Fields{"Device": OLTSerialNum, "Mvlan": mvp.Mvlan})
+		return
+	}
+
+	d := GetApplication().GetDeviceBySerialNo(OLTSerialNum)
+	if d == nil {
+		logger.Warnw(ctx, "Skipping Igmp & Mcast Flow processing: Device Not Found", log.Fields{"Device_SrNo": OLTSerialNum, "Mvlan": mvp.Mvlan})
+		return
+	}
+
+	p := d.GetPort(d.NniPort)
+
+	if p != nil && p.State == PortStateUp {
+		logger.Infow(ctx, "NNI Port Status is: UP & Vlan Enabled", log.Fields{"Device": d, "port": p})
+
+		//Push Igmp DS Control Flows
+		err := mvp.ApplyIgmpDSFlowForMvp(d.Name)
+		if err != nil {
+			logger.Errorw(ctx, "DS IGMP Flow Add Failed for device",
+				log.Fields{"Reason": err.Error(), "device": d.Name})
+		}
+
+		//Trigger Join for static channels
+		if channelList, containsStatic := mvp.getAllStaticChannels(); containsStatic {
+			mvp.ProcessStaticGroup(d.Name, channelList, true)
+		} else {
+			logger.Infow(ctx, "No Static Channels Present", log.Fields{"mvp": mvp.Name, "Mvlan": mvp.Mvlan})
+		}
+	}
+}
+
+/*
+//pushIgmpMcastFlowsToAllOlt - Adds all IGMP related flows (generic DS flow & static group flows) to all OLTs
+func (mvp *MvlanProfile) pushIgmpMcastFlowsToAllOlt() {
+
+	//for all devices apply igmp DS trap flow rules
+	pushIgmpFlows := func(key interface{}, value interface{}) bool {
+		d := value.(*VoltDevice)
+		p := d.GetPort(d.NniPort)
+			if p != nil && p.State == PortStateUp {
+				logger.Infow(ctx, "NNI Port Status is: UP & Vlan Enabled", log.Fields{"Device": d, "port": p})
+
+				//Push Igmp DS Control Flows
+				err := mvp.ApplyIgmpDSFlowForMvp(d.Name)
+				if err != nil {
+					logger.Errorw(ctx, "DS IGMP Flow Add Failed for device",
+						log.Fields{"Reason": err.Error(), "device": d.Name})
+				}
+
+				//Trigger Join for static channels
+				if channelList, containsStatic := mvp.getAllStaticChannels(); containsStatic {
+					mvp.ProcessStaticGroup(d.Name, channelList, true)
+				} else {
+					logger.Infow(ctx, "No Static Channels Present", log.Fields{"mvp": mvp.Name, "Mvlan": mvp.Mvlan})
+				}
+			}
+		return true
+	}
+	mvp.mvpLock.RLock()
+	defer mvp.mvpLock.RUnlock()
+	GetApplication().DevicesDisc.Range(pushIgmpFlows)
+}
+
+//removeIgmpFlows - Removes all IGMP related flows (generic DS flow)
+func (mvp *MvlanProfile) removeIgmpFlows(oltSerialNum string) {
+
+	if d := GetApplication().GetDeviceBySerialNo(oltSerialNum); d != nil {
+		p := d.GetPort(d.NniPort)
+		if p != nil {
+			logger.Infow(ctx, "NNI Port Status is: UP", log.Fields{"Device": d, "port": p})
+			err := mvp.RemoveIgmpDSFlowForMvp(d.Name)
+			if err != nil {
+				logger.Errorw(ctx, "DS IGMP Flow Del Failed", log.Fields{"Reason": err.Error(), "device": d.Name})
+			}
+		}
+	}
+}*/
+
+//removeIgmpMcastFlows - Removes all IGMP related flows (generic DS flow & static group flows)
+func (mvp *MvlanProfile) removeIgmpMcastFlows(oltSerialNum string) {
+
+	mvp.mvpLock.RLock()
+	defer mvp.mvpLock.RUnlock()
+
+	if d := GetApplication().GetDeviceBySerialNo(oltSerialNum); d != nil {
+		p := d.GetPort(d.NniPort)
+		if p != nil {
+			logger.Infow(ctx, "NNI Port Status is: UP", log.Fields{"Device": d, "port": p})
+
+			// ***Do not change the order***
+			// When Vlan is disabled, the process end is determined by the DS Igmp flag in device
+
+			//Trigger Leave for static channels
+			if channelList, containsStatic := mvp.getAllStaticChannels(); containsStatic {
+				mvp.ProcessStaticGroup(d.Name, channelList, false)
+			} else {
+				logger.Infow(ctx, "No Static Channels Present", log.Fields{"mvp": mvp.Name, "Mvlan": mvp.Mvlan})
+			}
+
+			//Remove all dynamic members for the Mvlan Profile
+			GetApplication().IgmpGroups.Range(func(key, value interface{}) bool {
+				ig := value.(*IgmpGroup)
+				if ig.Mvlan == mvp.Mvlan {
+					igd := ig.Devices[d.Name]
+					ig.DelIgmpGroupDevice(igd)
+					if ig.NumDevicesActive() == 0 {
+						GetApplication().DelIgmpGroup(ig)
+					}
+				}
+				return true
+			})
+
+			//Remove DS Igmp trap flow
+			err := mvp.RemoveIgmpDSFlowForMvp(d.Name)
+			if err != nil {
+				logger.Errorw(ctx, "DS IGMP Flow Del Failed", log.Fields{"Reason": err.Error(), "device": d.Name})
+			}
+		}
+	}
+}
+
+// ApplyIgmpDSFlowForMvp to apply Igmp DS flow for mvlan.
+func (mvp *MvlanProfile) ApplyIgmpDSFlowForMvp(device string) error {
+	va := GetApplication()
+	dIntf, ok := va.DevicesDisc.Load(device)
+	if !ok {
+		return errors.New("Device Doesn't Exist")
+	}
+	d := dIntf.(*VoltDevice)
+	mvlan := mvp.Mvlan
+
+	flowAlreadyApplied, ok := d.IgmpDsFlowAppliedForMvlan[uint16(mvlan)]
+	if !ok || !flowAlreadyApplied {
+		flows, err := mvp.BuildIgmpDSFlows(device)
+		if err == nil {
+			err = cntlr.GetController().AddFlows(d.NniPort, device, flows)
+			if err != nil {
+				logger.Warnw(ctx, "Configuring IGMP Flow for device failed ", log.Fields{"Device": device, "err": err})
+				return err
+			}
+			d.IgmpDsFlowAppliedForMvlan[uint16(mvlan)] = true
+			logger.Infow(ctx, "Updating voltDevice that IGMP DS flow as \"added\" for ",
+				log.Fields{"device": d.SerialNum, "mvlan": mvlan})
+		} else {
+			logger.Errorw(ctx, "DS IGMP Flow Add Failed", log.Fields{"Reason": err.Error(), "Mvlan": mvlan})
+		}
+	}
+
+	return nil
+}
+
+// RemoveIgmpDSFlowForMvp to remove Igmp DS flow for mvlan.
+func (mvp *MvlanProfile) RemoveIgmpDSFlowForMvp(device string) error {
+
+	va := GetApplication()
+	mvlan := mvp.Mvlan
+
+	dIntf, ok := va.DevicesDisc.Load(device)
+	if !ok {
+		return errors.New("Device Doesn't Exist")
+	}
+	d := dIntf.(*VoltDevice)
+	/* No need of strict check during DS IGMP deletion
+	flowAlreadyApplied, ok := d.IgmpDsFlowAppliedForMvlan[uint16(mvlan)]
+	if ok && flowAlreadyApplied
+	*/
+	flows, err := mvp.BuildIgmpDSFlows(device)
+	if err == nil {
+		flows.ForceAction = true
+
+		err = mvp.DelFlows(d, flows)
+		if err != nil {
+			logger.Warnw(ctx, "De-Configuring IGMP Flow for device failed ", log.Fields{"Device": device, "err": err})
+			return err
+		}
+		d.IgmpDsFlowAppliedForMvlan[uint16(mvlan)] = false
+		logger.Infow(ctx, "Updating voltDevice that IGMP DS flow as \"removed\" for ",
+			log.Fields{"device": d.SerialNum, "mvlan": mvlan})
+	} else {
+		logger.Errorw(ctx, "DS IGMP Flow Del Failed", log.Fields{"Reason": err.Error()})
+	}
+
+	return nil
+}
+
+// BuildIgmpDSFlows to build Igmp DS flows for NNI port
+func (mvp *MvlanProfile) BuildIgmpDSFlows(device string) (*of.VoltFlow, error) {
+	dIntf, ok := GetApplication().DevicesDisc.Load(device)
+	if !ok {
+		return nil, errors.New("Device Doesn't Exist")
+	}
+	d := dIntf.(*VoltDevice)
+
+	logger.Infow(ctx, "Building DS IGMP Flow for NNI port", log.Fields{"vs": d.NniPort, "Mvlan": mvp.Mvlan})
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+	subFlow.SetMatchVlan(mvp.Mvlan)
+
+	nniPort, err := GetApplication().GetNniPort(device)
+	if err != nil {
+		return nil, err
+	}
+	nniPortID, err1 := GetApplication().GetPortID(nniPort)
+	if err1 != nil {
+		return nil, errors.New("Unknown NNI outport")
+	}
+	subFlow.SetInPort(nniPortID)
+	subFlow.SetIgmpMatch()
+	subFlow.SetReportToController()
+	subFlow.Cookie = uint64(nniPortID)<<32 | uint64(mvp.Mvlan)
+	subFlow.Priority = of.IgmpFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built DS IGMP flow", log.Fields{"cookie": subFlow.Cookie, "subflow": subFlow})
+	return flow, nil
+}
+
+//updateStaticGroups - Generates static joins & leaves for newly added and removed static channels respectively
+func (mvp *MvlanProfile) updateStaticGroups(deviceID string, added []net.IP, removed []net.IP) {
+
+	//Update static group configs for all associated devices
+	updateGroups := func(key interface{}, value interface{}) bool {
+		d := value.(*VoltDevice)
+
+		if mvp.DevicesList[d.SerialNum] == Nil {
+			logger.Infow(ctx, "Mvlan Profile not configure for device", log.Fields{"Device": d, "Profile Device List": mvp.DevicesList})
+			return true
+		}
+		//TODO if mvp.IsChannelBasedGroup {
+		mvp.ProcessStaticGroup(d.Name, added, true)
+		mvp.ProcessStaticGroup(d.Name, removed, false)
+		//}
+		return true
+	}
+
+	if deviceID != "" {
+		vd := GetApplication().GetDevice(deviceID)
+		updateGroups(deviceID, vd)
+	} else {
+		GetApplication().DevicesDisc.Range(updateGroups)
+	}
+}
+
+//updateDynamicGroups - Generates joins with updated sources for existing channels
+func (mvp *MvlanProfile) updateDynamicGroups(deviceID string, added []net.IP, removed []net.IP) {
+
+	//mvlan := mvp.Mvlan
+	va := GetApplication()
+
+	updateGroups := func(key interface{}, value interface{}) bool {
+		d := value.(*VoltDevice)
+
+		if mvp.DevicesList[d.SerialNum] == Nil {
+			logger.Infow(ctx, "Mvlan Profile not configure for device", log.Fields{"Device": d, "Profile Device List": mvp.DevicesList})
+			return true
+		}
+		for _, groupAddr := range added {
+
+			_, gName := va.GetMvlanProfileForMcIP(mvp.Name, groupAddr)
+			grpKey := mvp.generateGroupKey(gName, groupAddr.String())
+			logger.Debugw(ctx, "IGMP Group", log.Fields{"Group": grpKey, "groupAddr": groupAddr})
+			if igIntf, ok := va.IgmpGroups.Load(grpKey); ok {
+				ig := igIntf.(*IgmpGroup)
+				if igd, ok := ig.getIgmpGroupDevice(d.Name); ok {
+					if igcIntf, ok := igd.GroupChannels.Load(groupAddr.String()); ok {
+						igc := igcIntf.(*IgmpGroupChannel)
+						incl := false
+						var ip []net.IP
+						var groupModified = false
+						if _, ok := mvp.Proxy[igc.GroupName]; ok {
+							if mvp.Proxy[igc.GroupName].Mode == common.Include {
+								incl = true
+							}
+							ip = mvp.Proxy[igc.GroupName].SourceList
+						}
+						for port, igp := range igc.NewReceivers {
+							// Process the include/exclude list which may end up modifying the group
+							if change, _ := igc.ProcessSources(port, ip, incl); change {
+								groupModified = true
+							}
+							igc.ProcessMode(port, incl)
+
+							if err := igp.WriteToDb(igc.Mvlan, igc.GroupAddr, igc.Device); err != nil {
+								logger.Errorw(ctx, "Igmp group port Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+							}
+						}
+						// If the group is modified as this is the first receiver or due to include/exclude list modification
+						// send a report to the upstream multicast servers
+						if groupModified {
+							logger.Debug(ctx, "Group Modified and IGMP report sent to the upstream server")
+							igc.SendReport(false)
+						}
+						if err := igc.WriteToDb(); err != nil {
+							logger.Errorw(ctx, "Igmp group channel Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+						}
+					}
+				}
+			}
+		}
+
+		return true
+	}
+
+	if deviceID != "" {
+		vd := GetApplication().GetDevice(deviceID)
+		updateGroups(deviceID, vd)
+	} else {
+		GetApplication().DevicesDisc.Range(updateGroups)
+	}
+}
+
+//GroupsUpdated - Handles removing of Igmp Groups, flows & group table entries for
+//channels removed as part of update
+func (mvp *MvlanProfile) GroupsUpdated(deviceID string) {
+
+	deleteChannelIfRemoved := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+
+		if ig.Mvlan != mvp.Mvlan {
+			return true
+		}
+		grpName := ig.GroupName
+		logger.Infow(ctx, "###Update Cycle", log.Fields{"IG": ig.GroupName, "Addr": ig.GroupAddr})
+		//Check if group exists and remove the entire group object otherwise
+		if currentChannels := mvp.Groups[grpName]; currentChannels != nil {
+
+			if mvp.IsChannelBasedGroup {
+				channelPresent := doesIPMatch(ig.GroupAddr, currentChannels.McIPs)
+				if channelPresent || mvp.isChannelStatic(ig.GroupAddr) {
+					return true
+				}
+			} else {
+				allExistingChannels := ig.GetAllIgmpChannelForDevice(deviceID)
+				for channel := range allExistingChannels {
+					channelIP := net.ParseIP(channel)
+					channelPresent := mvp.IsChannelPresent(channelIP, currentChannels.McIPs, mvp.IsStaticGroup(ig.GroupName))
+					if channelPresent {
+						staticChannel := mvp.isChannelStatic(channelIP)
+						logger.Infow(ctx, "###Channel Comparision", log.Fields{"staticChannel": staticChannel, "Group": mvp.IsStaticGroup(ig.GroupName), "Channel": channel})
+						// Logic:
+						// If channel is Static & existing Group is also static - No migration required
+						// If channel is not Static & existing Group is also not static - No migration required
+
+						// If channel is Static and existing Group is not static - Migrate (from dynamic to static)
+						//    (Channel already part of dynamic, added to static)
+
+						// If channel is not Static but existing Group is static - Migrate (from static to dynamic)
+						//    (Channel removed from satic but part of dynamic)
+						if (staticChannel != mvp.IsStaticGroup(ig.GroupName)) || (ig.IsGroupStatic != mvp.IsStaticGroup(ig.GroupName)) { // Equivalent of XOR
+							ig.HandleGroupMigration(deviceID, channelIP)
+						} else {
+							if (ig.IsGroupStatic) && mvp.IsStaticGroup(ig.GroupName) {
+								if ig.GroupName != mvp.GetStaticGroupName(channelIP) {
+									ig.HandleGroupMigration(deviceID, channelIP)
+								}
+							}
+							continue
+						}
+					} else {
+						logger.Debugw(ctx, "Channel Removed", log.Fields{"Channel": channel, "Group": grpName})
+						ig.DelIgmpChannel(deviceID, net.ParseIP(channel))
+						if ig.NumDevicesActive() == 0 {
+							GetApplication().DelIgmpGroup(ig)
+						}
+					}
+				}
+				ig.IsGroupStatic = mvp.IsStaticGroup(ig.GroupName)
+				if err := ig.WriteToDb(); err != nil {
+					logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+				}
+				return true
+			}
+		}
+		logger.Debugw(ctx, "Group Removed", log.Fields{"Channel": ig.GroupAddr, "Group": grpName, "ChannelBasedGroup": ig.IsChannelBasedGroup})
+		ig.DelIgmpGroup()
+		logger.Debugw(ctx, "Removed Igmp Group", log.Fields{"Channel": ig.GroupAddr, "Group": grpName})
+		return true
+	}
+	GetApplication().IgmpGroups.Range(deleteChannelIfRemoved)
+}
+
+// IsChannelPresent to check if channel is present
+func (mvp *MvlanProfile) IsChannelPresent(channelIP net.IP, groupChannelList []string, IsStaticGroup bool) bool {
+	// Only in case of static group, migration need to be supported.
+	// Dynamic to dynamic group migration not supported currently
+	if doesIPMatch(channelIP, groupChannelList) || mvp.isChannelStatic(channelIP) {
+		return true
+	} else if IsStaticGroup {
+		return (mvp.GetMvlanGroup(channelIP) != "")
+	}
+
+	return false
+}
+
+// GetMvlanProfileForMcIP - Get an MVLAN profile for a given MC IP. This is used when an
+// IGMP report is received from the PON port. The MVLAN profile
+// located is used to idnetify the MC VLAN used in upstream for
+// join/leave
+func (va *VoltApplication) GetMvlanProfileForMcIP(profileName string, ip net.IP) (*MvlanProfile, string) {
+	if mvpIntf, ok := va.MvlanProfilesByName.Load(profileName); ok {
+		mvp := mvpIntf.(*MvlanProfile)
+		if grpName := mvp.GetMvlanGroup(ip); grpName != "" {
+			return mvp, grpName
+		}
+	} else {
+		logger.Warnw(ctx, "Mvlan Profile not found for given profile name", log.Fields{"Profile": profileName})
+	}
+	return nil, ""
+}
+
+// GetMvlanGroup to get mvlan group
+func (mvp *MvlanProfile) GetMvlanGroup(ip net.IP) string {
+	//Check for Static Group First
+	if mvp.containsStaticChannels() {
+		grpName := mvp.GetStaticGroupName(ip)
+		if grpName != "" {
+			return grpName
+		}
+	}
+
+	for _, mvg := range mvp.Groups {
+		if mvg.Wildcard {
+			return mvg.Name
+		}
+		if doesIPMatch(ip, mvg.McIPs) {
+			return mvg.Name
+		}
+	}
+	return ""
+}
+
+// IgmpTick for igmp tick info
+func (va *VoltApplication) IgmpTick() {
+	tickCount++
+	if (tickCount % 1000) == 0 {
+		logger.Debugw(ctx, "Time @ Tick", log.Fields{"Tick": tickCount, "Time": time.Now()})
+	}
+	igmptick := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+		if ig.NumDevicesActive() != 0 {
+			if tickCount%10 == ig.Hash()%10 {
+				ig.IgmpGroupLock.Lock()
+				ig.Tick()
+				ig.IgmpGroupLock.Unlock()
+				if ig.NumDevicesActive() == 0 {
+					va.DelIgmpGroup(ig)
+				}
+			}
+		}
+		return true
+	}
+	va.IgmpGroups.Range(igmptick)
+}
+
+// Tick to add Tick Task
+func (va *VoltApplication) Tick() {
+	tt := NewTickTask()
+	va.IgmpTasks.AddTask(tt)
+	// va.IgmpTick()
+}
+
+//AddIgmpProfile for addition of IGMP Profile
+func (va *VoltApplication) AddIgmpProfile(igmpProfileConfig *common.IGMPConfig) error {
+	var igmpProfile *IgmpProfile
+
+	if igmpProfileConfig.ProfileID == DefaultIgmpProfID {
+		logger.Info(ctx, "Updating default IGMP profile")
+		return va.UpdateIgmpProfile(igmpProfileConfig)
+	}
+
+	igmpProfile = va.checkIgmpProfileMap(igmpProfileConfig.ProfileID)
+	if igmpProfile == nil {
+		igmpProfile = newIgmpProfile(igmpProfileConfig)
+	} else {
+		logger.Errorw(ctx, "IGMP profile already exists", log.Fields{"IgmpProfile": igmpProfileConfig.ProfileID})
+		return errors.New("IGMP Profile already exists")
+	}
+
+	va.storeIgmpProfileMap(igmpProfileConfig.ProfileID, igmpProfile)
+
+	if err := igmpProfile.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp profile Write to DB failed", log.Fields{"profileID": igmpProfile.ProfileID})
+	}
+
+	return nil
+}
+
+func newIgmpProfile(igmpProfileConfig *common.IGMPConfig) *IgmpProfile {
+	var igmpProfile IgmpProfile
+	igmpProfile.ProfileID = igmpProfileConfig.ProfileID
+	igmpProfile.UnsolicitedTimeOut = uint32(igmpProfileConfig.UnsolicitedTimeOut)
+	igmpProfile.MaxResp = uint32(igmpProfileConfig.MaxResp)
+
+	keepAliveInterval := uint32(igmpProfileConfig.KeepAliveInterval)
+
+	//KeepAliveInterval should have a min of 10 seconds
+	if keepAliveInterval < MinKeepAliveInterval {
+		keepAliveInterval = MinKeepAliveInterval
+		logger.Infow(ctx, "Auto adjust keepAliveInterval - Value < 10", log.Fields{"Received": igmpProfileConfig.KeepAliveInterval, "Configured": keepAliveInterval})
+	}
+	igmpProfile.KeepAliveInterval = keepAliveInterval
+
+	igmpProfile.KeepAliveCount = uint32(igmpProfileConfig.KeepAliveCount)
+	igmpProfile.LastQueryInterval = uint32(igmpProfileConfig.LastQueryInterval)
+	igmpProfile.LastQueryCount = uint32(igmpProfileConfig.LastQueryCount)
+	igmpProfile.FastLeave = *igmpProfileConfig.FastLeave
+	igmpProfile.PeriodicQuery = *igmpProfileConfig.PeriodicQuery
+	igmpProfile.IgmpCos = uint8(igmpProfileConfig.IgmpCos)
+	igmpProfile.WithRAUpLink = *igmpProfileConfig.WithRAUpLink
+	igmpProfile.WithRADownLink = *igmpProfileConfig.WithRADownLink
+
+	if igmpProfileConfig.IgmpVerToServer == "2" || igmpProfileConfig.IgmpVerToServer == "v2" {
+		igmpProfile.IgmpVerToServer = "2"
+	} else {
+		igmpProfile.IgmpVerToServer = "3"
+	}
+	igmpProfile.IgmpSourceIP = net.ParseIP(igmpProfileConfig.IgmpSourceIP)
+
+	return &igmpProfile
+}
+
+// checkIgmpProfileMap to get Igmp Profile. If not found return nil
+func (va *VoltApplication) checkIgmpProfileMap(name string) *IgmpProfile {
+	if igmpProfileIntf, ok := va.IgmpProfilesByName.Load(name); ok {
+		return igmpProfileIntf.(*IgmpProfile)
+	}
+	return nil
+}
+
+// newDefaultIgmpProfile Igmp profiles with default values
+func newDefaultIgmpProfile() *IgmpProfile {
+	return &IgmpProfile{
+		ProfileID:          DefaultIgmpProfID,
+		UnsolicitedTimeOut: 60,
+		MaxResp:            10, // seconds
+		KeepAliveInterval:  60, // seconds
+		KeepAliveCount:     3,  // TODO - May not be needed
+		LastQueryInterval:  0,  // TODO - May not be needed
+		LastQueryCount:     0,  // TODO - May not be needed
+		FastLeave:          true,
+		PeriodicQuery:      false, // TODO - May not be needed
+		IgmpCos:            7,     //p-bit value included in the IGMP packet
+		WithRAUpLink:       false, // TODO - May not be needed
+		WithRADownLink:     false, // TODO - May not be needed
+		IgmpVerToServer:    "3",
+		IgmpSourceIP:       net.ParseIP("172.27.0.1"), // This will be replaced by configuration
+	}
+}
+
+func (va *VoltApplication) resetIgmpProfileToDefault() {
+	igmpProf := va.getIgmpProfileMap(DefaultIgmpProfID)
+	defIgmpProf := newDefaultIgmpProfile()
+
+	igmpProf.UnsolicitedTimeOut = defIgmpProf.UnsolicitedTimeOut
+	igmpProf.MaxResp = defIgmpProf.MaxResp
+	igmpProf.KeepAliveInterval = defIgmpProf.KeepAliveInterval
+	igmpProf.KeepAliveCount = defIgmpProf.KeepAliveCount
+	igmpProf.LastQueryInterval = defIgmpProf.LastQueryInterval
+	igmpProf.LastQueryCount = defIgmpProf.LastQueryCount
+	igmpProf.FastLeave = defIgmpProf.FastLeave
+	igmpProf.PeriodicQuery = defIgmpProf.PeriodicQuery
+	igmpProf.IgmpCos = defIgmpProf.IgmpCos
+	igmpProf.WithRAUpLink = defIgmpProf.WithRAUpLink
+	igmpProf.WithRADownLink = defIgmpProf.WithRADownLink
+	igmpProf.IgmpVerToServer = defIgmpProf.IgmpVerToServer
+	igmpProf.IgmpSourceIP = defIgmpProf.IgmpSourceIP
+
+	if err := igmpProf.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp profile Write to DB failed", log.Fields{"profileID": igmpProf.ProfileID})
+	}
+}
+
+// getIgmpProfileMap to get Igmp Profile. If not found return default IGMP config
+func (va *VoltApplication) getIgmpProfileMap(name string) *IgmpProfile {
+	if igmpProfileIntf, ok := va.IgmpProfilesByName.Load(name); ok {
+		return igmpProfileIntf.(*IgmpProfile)
+	}
+
+	// There will be always a default igmp profile.
+	defaultIgmpProfileIntf, _ := va.IgmpProfilesByName.Load(DefaultIgmpProfID)
+	return defaultIgmpProfileIntf.(*IgmpProfile)
+}
+
+// storeIgmpProfileMap to store Igmp Profile
+func (va *VoltApplication) storeIgmpProfileMap(name string, igmpProfile *IgmpProfile) {
+	va.IgmpProfilesByName.Store(name, igmpProfile)
+}
+
+// deleteIgmpProfileMap to delete Igmp Profile
+func (va *VoltApplication) deleteIgmpProfileMap(name string) {
+	va.IgmpProfilesByName.Delete(name)
+}
+
+// WriteToDb is utility to write Igmp Config Info to database
+func (igmpProfile *IgmpProfile) WriteToDb() error {
+	igmpProfile.Version = database.PresentVersionMap[database.IgmpProfPath]
+	b, err := json.Marshal(igmpProfile)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutIgmpProfile(igmpProfile.ProfileID, string(b)); err1 != nil {
+		return err1
+	}
+	return nil
+}
+
+//DelIgmpProfile for addition of IGMP Profile
+func (va *VoltApplication) DelIgmpProfile(igmpProfileConfig *common.IGMPConfig) error {
+	// Deletion of default igmp profile is blocked from submgr. Keeping additional check for safety.
+	if igmpProfileConfig.ProfileID == DefaultIgmpProfID {
+		logger.Info(ctx, "Resetting default IGMP profile")
+		va.resetIgmpProfileToDefault()
+		return nil
+	}
+	igmpProfile := va.checkIgmpProfileMap(igmpProfileConfig.ProfileID)
+	if igmpProfile == nil {
+		logger.Warnw(ctx, "Igmp Profile not found. Unable to delete", log.Fields{"Profile ID": igmpProfileConfig.ProfileID})
+		return nil
+	}
+
+	va.deleteIgmpProfileMap(igmpProfileConfig.ProfileID)
+
+	_ = db.DelIgmpProfile(igmpProfileConfig.ProfileID)
+
+	return nil
+}
+
+//UpdateIgmpProfile for addition of IGMP Profile
+func (va *VoltApplication) UpdateIgmpProfile(igmpProfileConfig *common.IGMPConfig) error {
+	igmpProfile := va.checkIgmpProfileMap(igmpProfileConfig.ProfileID)
+	if igmpProfile == nil {
+		logger.Errorw(ctx, "Igmp Profile not found. Unable to update", log.Fields{"Profile ID": igmpProfileConfig.ProfileID})
+		return errors.New("IGMP Profile not found")
+	}
+
+	igmpProfile.ProfileID = igmpProfileConfig.ProfileID
+	igmpProfile.UnsolicitedTimeOut = uint32(igmpProfileConfig.UnsolicitedTimeOut)
+	igmpProfile.MaxResp = uint32(igmpProfileConfig.MaxResp)
+
+	keepAliveInterval := uint32(igmpProfileConfig.KeepAliveInterval)
+
+	//KeepAliveInterval should have a min of 10 seconds
+	if keepAliveInterval < MinKeepAliveInterval {
+		keepAliveInterval = MinKeepAliveInterval
+		logger.Infow(ctx, "Auto adjust keepAliveInterval - Value < 10", log.Fields{"Received": igmpProfileConfig.KeepAliveInterval, "Configured": keepAliveInterval})
+	}
+	igmpProfile.KeepAliveInterval = keepAliveInterval
+
+	igmpProfile.KeepAliveCount = uint32(igmpProfileConfig.KeepAliveCount)
+	igmpProfile.LastQueryInterval = uint32(igmpProfileConfig.LastQueryInterval)
+	igmpProfile.LastQueryCount = uint32(igmpProfileConfig.LastQueryCount)
+	igmpProfile.FastLeave = *igmpProfileConfig.FastLeave
+	igmpProfile.PeriodicQuery = *igmpProfileConfig.PeriodicQuery
+	igmpProfile.IgmpCos = uint8(igmpProfileConfig.IgmpCos)
+	igmpProfile.WithRAUpLink = *igmpProfileConfig.WithRAUpLink
+	igmpProfile.WithRADownLink = *igmpProfileConfig.WithRADownLink
+
+	if igmpProfileConfig.IgmpVerToServer == "2" || igmpProfileConfig.IgmpVerToServer == "v2" {
+		igmpProfile.IgmpVerToServer = "2"
+	} else {
+		igmpProfile.IgmpVerToServer = "3"
+	}
+
+	if igmpProfileConfig.IgmpSourceIP != "" {
+		igmpProfile.IgmpSourceIP = net.ParseIP(igmpProfileConfig.IgmpSourceIP)
+	}
+
+	if err := igmpProfile.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp profile Write to DB failed", log.Fields{"profileID": igmpProfile.ProfileID})
+	}
+
+	return nil
+}
+
+// RestoreIGMPProfilesFromDb to read from the DB and restore IGMP Profiles
+func (va *VoltApplication) RestoreIGMPProfilesFromDb() {
+	// Loading IGMP profiles
+	igmpProfiles, _ := db.GetIgmpProfiles()
+	for _, igmpProfile := range igmpProfiles {
+		b, ok := igmpProfile.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var igmpProf IgmpProfile
+		err := json.Unmarshal(b, &igmpProf)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of IGMP Profile failed")
+			continue
+		}
+		va.storeIgmpProfileMap(igmpProf.ProfileID, &igmpProf)
+		logger.Infow(ctx, "Restored Igmp Profile", log.Fields{"Conf": igmpProf})
+	}
+}
+
+// InitIgmpSrcMac for initialization of igmp source mac
+func (va *VoltApplication) InitIgmpSrcMac() {
+	srcMac, err := getPodMacAddr()
+	if err != nil {
+		igmpSrcMac = "00:11:11:11:11:11"
+		return
+	}
+	igmpSrcMac = srcMac
+}
+
+// removeIPFromList to remove ip from the list
+func removeIPFromList(s []net.IP, value net.IP) []net.IP {
+	i := 0
+	for i = 0; i < len(s); i++ {
+		if s[i].Equal(value) {
+			break
+		}
+	}
+	if i != len(s) {
+		//It means value is found in the slice
+		return append(s[0:i], s[i+1:]...)
+	}
+	return s
+}
+
+// DelMvlanProfile for deletion of a MVLAN group
+func (va *VoltApplication) DelMvlanProfile(name string) error {
+	if mvpIntf, ok := va.MvlanProfilesByName.Load(name); ok {
+		mvp := mvpIntf.(*MvlanProfile)
+
+		if len(mvp.DevicesList) == 0 {
+			mvp.DeleteInProgress = true
+			mvp.DelFromDb()
+			va.deleteMvlansMap(mvp.Mvlan, name)
+			logger.Debugw(ctx, "Deleted MVLAN Profile", log.Fields{"Name": mvp.Name})
+		} else {
+			logger.Errorw(ctx, "Unable to delete Mvlan Profile as there is still an OLT attached to it", log.Fields{"Name": mvp.Name,
+				"Device List": mvp.DevicesList})
+			return errors.New("MVLAN attached to devices")
+		}
+
+		return nil
+	}
+	logger.Errorw(ctx, "MVLAN Profile not found", log.Fields{"MvlanProfile Name": name})
+	return nil
+}
+
+// ReceiverUpInd for receiver up indication
+func (va *VoltApplication) ReceiverUpInd(device string, port string, mvpName string, vlan of.VlanType, pbits []of.PbitType) {
+	logger.Infow(ctx, "Receiver Indication: UP", log.Fields{"device": device, "port": port, "MVP": mvpName, "vlan": vlan, "pbits": pbits})
+	if mvpIntf, ok := va.MvlanProfilesByName.Load(mvpName); ok {
+		mvp := mvpIntf.(*MvlanProfile)
+		if devIntf, ok := va.DevicesDisc.Load(device); ok {
+			dev := devIntf.(*VoltDevice)
+			proxyCfg, proxyIP, _ := getIgmpProxyCfgAndIP(mvp.Mvlan, dev.SerialNum)
+			for _, pbit := range pbits {
+				sendGeneralQuery(device, port, vlan, uint8(pbit), proxyCfg, proxyIP)
+			}
+		} else {
+			logger.Warnw(ctx, "Device not found for given port", log.Fields{"device": device, "port": port})
+		}
+	} else {
+		logger.Warnw(ctx, "Mvlan Profile not found for given profileName", log.Fields{"MVP": mvpName, "vlan": vlan})
+	}
+}
+
+// sendGeneralQuery to send general query
+func sendGeneralQuery(device string, port string, cVlan of.VlanType, pbit uint8, proxyCfg *IgmpProfile, proxyIP *net.IP) {
+
+	if queryPkt, err := Igmpv2QueryPacket(NullIPAddr, cVlan, *proxyIP, pbit, proxyCfg.MaxResp); err == nil {
+		if err := cntlr.GetController().PacketOutReq(device, port, port, queryPkt, false); err != nil {
+			logger.Warnw(ctx, "General Igmpv2 Query Failed to send", log.Fields{"Device": device, "Port": port, "Packet": queryPkt, "Pbit": pbit})
+		} else {
+			logger.Debugw(ctx, "General Igmpv2 Query Sent", log.Fields{"Device": device, "Port": port, "Packet": queryPkt, "Pbit": pbit})
+		}
+	}
+	if getVersion(proxyCfg.IgmpVerToServer) == IgmpVersion3 {
+		if queryPkt, err := Igmpv3QueryPacket(NullIPAddr, cVlan, *proxyIP, pbit, proxyCfg.MaxResp); err == nil {
+			if err := cntlr.GetController().PacketOutReq(device, port, port, queryPkt, false); err != nil {
+				logger.Warnw(ctx, "General Igmpv3 Query Failed to send", log.Fields{"Device": device, "Port": port, "Packet": queryPkt, "Pbit": pbit})
+			} else {
+				logger.Debugw(ctx, "General Igmpv3 Query Sent", log.Fields{"Device": device, "Port": port, "Packet": queryPkt, "Pbit": pbit})
+			}
+		}
+	}
+}
+
+// ReceiverDownInd to send receiver down indication
+func (va *VoltApplication) ReceiverDownInd(device string, port string) {
+	logger.Infow(ctx, " Receiver Indication: DOWN", log.Fields{"device": device, "port": port})
+
+	ponPortID := va.GetPonPortID(device, port)
+
+	del := func(key interface{}, value interface{}) bool {
+		ig := value.(*IgmpGroup)
+		ig.IgmpGroupLock.Lock()
+		ig.DelReceiveronDownInd(device, port, ponPortID)
+		ig.IgmpGroupLock.Unlock()
+		if ig.NumDevicesActive() == 0 {
+			va.DelIgmpGroup(ig)
+		}
+		return true
+	}
+	va.IgmpGroups.Range(del)
+}
+
+// doesIPMatch to check if ip match with any ip from the list
+func doesIPMatch(ip net.IP, ipsOrRange []string) bool {
+	for _, ipOrRange := range ipsOrRange {
+		if strings.Contains(ipOrRange, "-") {
+			var splits = strings.Split(ipOrRange, "-")
+			ipStart := util.IP2LongConv(net.ParseIP(splits[0]))
+			ipEnd := util.IP2LongConv(net.ParseIP(splits[1]))
+			if ipEnd < ipStart {
+				return false
+			}
+			ipInt := util.IP2LongConv(ip)
+			if ipInt >= ipStart && ipInt <= ipEnd {
+				return true
+			}
+		} else if ip.Equal(net.ParseIP(ipOrRange)) {
+			return true
+		}
+	}
+	return false
+}
+
+// ProcessStaticGroup - Process Static Join/Leave Req for static channels
+func (mvp *MvlanProfile) ProcessStaticGroup(device string, groupAddresses []net.IP, isJoin bool) {
+
+	logger.Debugw(ctx, "Received Static Group Request", log.Fields{"Device": device, "Join": isJoin, "Group Address List": groupAddresses})
+
+	mvlan := mvp.Mvlan
+	va := GetApplication()
+
+	//TODO - Handle bulk add of groupAddr
+	for _, groupAddr := range groupAddresses {
+
+		ig := mvp.GetStaticIgmpGroup(groupAddr)
+		if isJoin {
+			vd := va.GetDevice(device)
+			igmpProf, _, _ := getIgmpProxyCfgAndIP(mvlan, vd.SerialNum)
+			ver := igmpProf.IgmpVerToServer
+
+			if ig == nil {
+				// First time group Creation: Create the IGMP group and then add the receiver to the group
+				logger.Infow(ctx, "Static IGMP Add received for new group", log.Fields{"Addr": groupAddr, "Port": StaticPort})
+				if ig := GetApplication().AddIgmpGroup(mvp.Name, groupAddr, device); ig != nil {
+					ig.IgmpGroupLock.Lock()
+					ig.AddReceiver(device, StaticPort, groupAddr, nil, getVersion(ver),
+						0, 0, 0xFF)
+					ig.IgmpGroupLock.Unlock()
+				} else {
+					logger.Warnw(ctx, "Static IGMP Group Creation Failed", log.Fields{"Addr": groupAddr})
+				}
+			} else {
+				//Converting existing dynamic group to static group
+				if !mvp.IsStaticGroup(ig.GroupName) {
+					ig.updateGroupName(ig.GroupName)
+				}
+				// Update case: If the IGMP group is already created. just add the receiver
+				logger.Infow(ctx, "Static IGMP Add received for existing group", log.Fields{"Addr": groupAddr, "Port": StaticPort})
+				ig.IgmpGroupLock.Lock()
+				ig.AddReceiver(device, StaticPort, groupAddr, nil, getVersion(ver),
+					0, 0, 0xFF)
+				ig.IgmpGroupLock.Unlock()
+			}
+		} else if ig != nil {
+			logger.Infow(ctx, "Static IGMP Del received for existing group", log.Fields{"Addr": groupAddr, "Port": StaticPort})
+
+			if ig.IsChannelBasedGroup {
+				grpName := mvp.GetMvlanGroup(ig.GroupAddr)
+				if grpName != "" {
+					ig.IgmpGroupLock.Lock()
+					ig.DelReceiver(device, StaticPort, groupAddr, nil, 0xFF)
+					ig.IgmpGroupLock.Unlock()
+					ig.updateGroupName(grpName)
+				} else {
+					ig.DelIgmpGroup()
+				}
+			} else {
+				ig.IgmpGroupLock.Lock()
+				ig.DelReceiver(device, StaticPort, groupAddr, nil, 0xFF)
+				ig.IgmpGroupLock.Unlock()
+			}
+			if ig.NumDevicesActive() == 0 {
+				GetApplication().DelIgmpGroup(ig)
+			}
+		} else {
+			logger.Warnw(ctx, "Static IGMP Del received for unknown group", log.Fields{"Addr": groupAddr})
+		}
+	}
+}
+
+//getStaticChannelDiff - return the static channel newly added and removed from existing static group
+func (mvp *MvlanProfile) getStaticChannelDiff() (newlyAdded []net.IP, removed []net.IP, common []net.IP) {
+
+	var commonChannels []net.IP
+	newChannelList, _ := mvp.getAllStaticChannels()
+	existingChannelList, _ := mvp.getAllOldGroupStaticChannels()
+	if len(existingChannelList) == 0 {
+		return newChannelList, []net.IP{}, []net.IP{}
+	}
+	for _, newChannel := range append([]net.IP{}, newChannelList...) {
+		for _, existChannel := range append([]net.IP{}, existingChannelList...) {
+
+			//Remove common channels between existing and new list
+			// The remaining in the below slices give the results
+			// Remaining in newChannelList: Newly added
+			// Remaining in existingChannelList: Removed channels
+			if existChannel.Equal(newChannel) {
+				existingChannelList = removeIPFromList(existingChannelList, existChannel)
+				newChannelList = removeIPFromList(newChannelList, newChannel)
+				commonChannels = append(commonChannels, newChannel)
+				logger.Infow(ctx, "#############Channel: "+existChannel.String()+" New: "+newChannel.String(), log.Fields{"Added": newChannelList, "Removed": existingChannelList})
+				break
+			}
+		}
+	}
+	return newChannelList, existingChannelList, commonChannels
+}
+
+//getGroupChannelDiff - return the channel newly added and removed from existing group
+func (mvp *MvlanProfile) getGroupChannelDiff(newGroup *MvlanGroup, oldGroup *MvlanGroup) (newlyAdded []net.IP, removed []net.IP, common []net.IP) {
+
+	var commonChannels []net.IP
+	newChannelList, _ := newGroup.getAllChannels()
+	existingChannelList, _ := oldGroup.getAllChannels()
+	if len(existingChannelList) == 0 {
+		return newChannelList, []net.IP{}, []net.IP{}
+	}
+	for _, newChannel := range append([]net.IP{}, newChannelList...) {
+		for _, existChannel := range append([]net.IP{}, existingChannelList...) {
+
+			//Remove common channels between existing and new list
+			// The remaining in the below slices give the results
+			// Remaining in newChannelList: Newly added
+			// Remaining in existingChannelList: Removed channels
+			if existChannel.Equal(newChannel) {
+				existingChannelList = removeIPFromList(existingChannelList, existChannel)
+				newChannelList = removeIPFromList(newChannelList, newChannel)
+				commonChannels = append(commonChannels, newChannel)
+				logger.Infow(ctx, "#############Channel: "+existChannel.String()+" New: "+newChannel.String(), log.Fields{"Added": newChannelList, "Removed": existingChannelList})
+				break
+			}
+		}
+	}
+	return newChannelList, existingChannelList, commonChannels
+}
+
+// UpdateProfile - Updates the group & member info w.r.t the mvlan profile for the given device
+func (mvp *MvlanProfile) UpdateProfile(deviceID string) {
+	logger.Infow(ctx, "Update Mvlan Profile task triggered", log.Fields{"Mvlan": mvp.Mvlan})
+	var removedStaticChannels []net.IP
+	addedStaticChannels := []net.IP{}
+	/* Taking mvpLock to protect the mvp groups and proxy */
+	mvp.mvpLock.RLock()
+	defer mvp.mvpLock.RUnlock()
+
+	serialNo := ""
+	if deviceID != "" {
+		if vd := GetApplication().GetDevice(deviceID); vd != nil {
+			serialNo = vd.SerialNum
+			if mvp.DevicesList[serialNo] != UpdateInProgress {
+				logger.Warnw(ctx, "Exiting Update Task since device not present in MvlanProfile", log.Fields{"Device": deviceID, "SerialNum": vd.SerialNum, "MvlanProfile": mvp})
+				return
+			}
+		} else {
+			logger.Errorw(ctx, "Volt Device not found. Stopping Update Mvlan Profile processing for device", log.Fields{"SerialNo": deviceID, "MvlanProfile": mvp})
+			return
+		}
+	}
+
+	//Update the groups based on static channels added & removed
+	if mvp.containsStaticChannels() {
+		addedStaticChannels, removedStaticChannels, _ = mvp.getStaticChannelDiff()
+		logger.Debugw(ctx, "Update Task - Static Group Changes", log.Fields{"Added": addedStaticChannels, "Removed": removedStaticChannels})
+
+		if len(addedStaticChannels) > 0 || len(removedStaticChannels) > 0 {
+			mvp.updateStaticGroups(deviceID, []net.IP{}, removedStaticChannels)
+		}
+	}
+	mvp.GroupsUpdated(deviceID)
+	if len(addedStaticChannels) > 0 {
+		mvp.updateStaticGroups(deviceID, addedStaticChannels, []net.IP{})
+	}
+
+	/* Need to handle if SSM params are modified for groups */
+	for key := range mvp.Groups {
+		_, _, commonChannels := mvp.getGroupChannelDiff(mvp.Groups[key], mvp.oldGroups[key])
+		if mvp.checkStaticGrpSSMProxyDiff(mvp.oldProxy[key], mvp.Proxy[key]) {
+			if mvp.Groups[key].IsStatic {
+				/* Static group proxy modified, need to trigger membership report with new mode/src-list for existing channels */
+				mvp.updateStaticGroups(deviceID, commonChannels, []net.IP{})
+			} else {
+				/* Dynamic group proxy modified, need to trigger membership report with new mode/src-list for existing channels */
+				mvp.updateDynamicGroups(deviceID, commonChannels, []net.IP{})
+			}
+		}
+	}
+
+	mvp.SetUpdateStatus(serialNo, NoOp)
+
+	if deviceID == "" || !mvp.isUpdateInProgress() {
+		mvp.oldGroups = nil
+	}
+	if err := mvp.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Mvlan profile write to DB failed", log.Fields{"ProfileName": mvp.Name})
+	}
+	logger.Debugw(ctx, "Updated MVLAN Profile", log.Fields{"VLAN": mvp.Mvlan, "Name": mvp.Name, "Grp IPs": mvp.Groups})
+}
+
+//checkStaticGrpSSMProxyDiff- return true if the proxy of oldGroup is modified in newGroup
+func (mvp *MvlanProfile) checkStaticGrpSSMProxyDiff(oldProxy *MCGroupProxy, newProxy *MCGroupProxy) bool {
+
+	if oldProxy == nil && newProxy == nil {
+		return false
+	}
+	if (oldProxy == nil && newProxy != nil) ||
+		(oldProxy != nil && newProxy == nil) {
+		return true
+	}
+
+	if oldProxy.Mode != newProxy.Mode {
+		return true
+	}
+
+	oldSrcLst := oldProxy.SourceList
+	newSrcLst := newProxy.SourceList
+	oLen := len(oldSrcLst)
+	nLen := len(newSrcLst)
+	if oLen != nLen {
+		return true
+	}
+
+	visited := make([]bool, nLen)
+
+	/* check if any new IPs added in the src list, return true if present */
+	for i := 0; i < nLen; i++ {
+		found := false
+		element := newSrcLst[i]
+		for j := 0; j < oLen; j++ {
+			if visited[j] {
+				continue
+			}
+			if element.Equal(oldSrcLst[j]) {
+				visited[j] = true
+				found = true
+				break
+			}
+		}
+		if !found {
+			return true
+		}
+	}
+
+	visited = make([]bool, nLen)
+	/* check if any IPs removed from existing  src list, return true if removed */
+	for i := 0; i < oLen; i++ {
+		found := false
+		element := oldSrcLst[i]
+		for j := 0; j < nLen; j++ {
+			if visited[j] {
+				continue
+			}
+			if element.Equal(newSrcLst[j]) {
+				visited[j] = true
+				found = true
+				break
+			}
+		}
+		if !found {
+			return true
+		}
+	}
+	return false
+}
+
+// ProcessMode process the received mode and updated the igp
+func (igc *IgmpGroupChannel) ProcessMode(port string, incl bool) {
+	/* Update the mode in igp if the mode has changed */
+	igp := igc.GetReceiver(port)
+	if igp.Exclude && incl {
+		igp.Exclude = !incl
+		if igc.Exclude > 0 {
+			igc.Exclude--
+		}
+	} else if !incl && !igp.Exclude {
+		igp.Exclude = !incl
+		igc.Exclude++
+	}
+}
+
+func (ig *IgmpGroup) removeExpiredGroupFromDevice() {
+	ig.PendingPoolLock.Lock()
+	defer ig.PendingPoolLock.Unlock()
+
+	for device, timer := range ig.PendingGroupForDevice {
+
+		// To ensure no race-condition between the expiry time and the new Join,
+		// ensure the group exists in pending pool before deletion
+		groupExistsInPendingPool := true
+
+		if !time.Now().After(timer) {
+			continue
+		}
+
+		// Check if the IgmpGroup obj has no active member across any device
+		// If Yes, then this group is part of global pending pool (IgmpPendingPool), hence if expired,
+		// Remove only the IgmpGroup obj referenced to this device from global pool also.
+		if ig.NumDevicesActive() == 0 {
+			groupExistsInPendingPool = GetApplication().RemoveGroupFromPendingPool(device, ig)
+		}
+
+		// Remove the group entry from device and remove the IgmpDev Obj
+		// from IgmpGrp Pending pool
+		if groupExistsInPendingPool {
+			ig.DeleteIgmpGroupDevice(device)
+		}
+	}
+}
+
+//DeleteIgmpGroupDevice - removes the IgmpGroupDevice obj from IgmpGroup and database
+func (ig *IgmpGroup) DeleteIgmpGroupDevice(device string) {
+
+	logger.Infow(ctx, "Deleting IgmpGroupDevice from IG Pending Pool", log.Fields{"Device": device, "GroupID": ig.GroupID, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr.String(), "PendingDevices": len(ig.Devices)})
+
+	igd := ig.Devices[device]
+	igd.DelMcGroup(true)
+	delete(ig.Devices, device)
+	delete(ig.PendingGroupForDevice, device)
+	_ = db.DelIgmpDevice(igd.Mvlan, igd.GroupName, igd.GroupAddr, igd.Device)
+
+	//If the group is not associated to any other device, then the entire Igmp Group obj itself can be removed
+	if ig.NumDevicesAll() == 0 {
+		logger.Infow(ctx, "Deleting IgmpGroup as all pending groups has expired", log.Fields{"Device": device, "GroupID": ig.GroupID, "GroupName": ig.GroupName, "GroupAddr": ig.GroupAddr.String(), "PendingDevices": len(ig.Devices)})
+		GetApplication().DelIgmpGroup(ig)
+		return
+	}
+	if err := ig.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+	}
+}
+
+//UpdateActiveChannelSubscriberAlarm - Updates the Active Channel Subscriber Alarm
+func (mvp *MvlanProfile) UpdateActiveChannelSubscriberAlarm() {
+	va := GetApplication()
+	logger.Debugw(ctx, "Update of Active Channel Subscriber Alarm", log.Fields{"Mvlan": mvp.Mvlan})
+	for srNo := range mvp.DevicesList {
+		d := va.GetDeviceBySerialNo(srNo)
+		if d == nil {
+			logger.Warnw(ctx, "Device info not found", log.Fields{"Device_SrNo": srNo, "Mvlan": mvp.Mvlan})
+			return
+		}
+		d.Ports.Range(func(key, value interface{}) bool {
+			//port := key.(string)
+			vp := value.(*VoltPort)
+			if vp.Type != VoltPortTypeAccess {
+				return true
+			}
+			if mvp.MaxActiveChannels > vp.ActiveChannels && vp.ChannelPerSubAlarmRaised {
+				serviceName := GetMcastServiceForSubAlarm(vp, mvp)
+				logger.Debugw(ctx, "Clearing-SendActiveChannelPerSubscriberAlarm-due-to-update", log.Fields{"ActiveChannels": vp.ActiveChannels, "ServiceName": serviceName})
+				vp.ChannelPerSubAlarmRaised = false
+			} else if mvp.MaxActiveChannels < vp.ActiveChannels && !vp.ChannelPerSubAlarmRaised {
+				/* When the max active channel count is reduced via update, we raise an alarm.
+				   But the previous excess channels still exist until a leave or expiry */
+				serviceName := GetMcastServiceForSubAlarm(vp, mvp)
+				logger.Debugw(ctx, "Raising-SendActiveChannelPerSubscriberAlarm-due-to-update", log.Fields{"ActiveChannels": vp.ActiveChannels, "ServiceName": serviceName})
+				vp.ChannelPerSubAlarmRaised = true
+			}
+			return true
+		})
+	}
+}
+
+//TriggerAssociatedFlowDelete - Re-trigger delete for pending delete flows
+func (mvp *MvlanProfile) TriggerAssociatedFlowDelete(device string) bool {
+	mvp.mvpFlowLock.Lock()
+
+	cookieList := []uint64{}
+	flowMap := mvp.PendingDeleteFlow[device]
+
+	for cookie := range flowMap {
+		cookieList = append(cookieList, convertToUInt64(cookie))
+	}
+	mvp.mvpFlowLock.Unlock()
+
+	if len(cookieList) == 0 {
+		return false
+	}
+
+	for _, cookie := range cookieList {
+		if vd := GetApplication().GetDevice(device); vd != nil {
+			flow := &of.VoltFlow{}
+			flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+			subFlow := of.NewVoltSubFlow()
+			subFlow.Cookie = cookie
+			flow.SubFlows[cookie] = subFlow
+			logger.Infow(ctx, "Retriggering Vnet Delete Flow", log.Fields{"Device": device, "Mvlan": mvp.Mvlan.String(), "Cookie": cookie})
+			err := mvp.DelFlows(vd, flow)
+			if err != nil {
+				logger.Warnw(ctx, "De-Configuring IGMP Flow for device failed ", log.Fields{"Device": device, "err": err})
+			}
+		}
+	}
+	return true
+}
diff --git a/internal/pkg/application/igmptasks.go b/internal/pkg/application/igmptasks.go
new file mode 100644
index 0000000..af42562
--- /dev/null
+++ b/internal/pkg/application/igmptasks.go
@@ -0,0 +1,175 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"context"
+	"time"
+
+	"github.com/google/gopacket"
+)
+
+// ------------------------------------------------------------------
+// ********** Tasks
+//
+// IGMP related tasks which essentially process packets and the ticks
+// This is to serailize access to data structures and this also limits
+// the amount of CPU consumed. We can bring more capacity by running
+// more groups in parallel as we need to add parallelism
+
+// -----------------------------------------------------------------
+// ** Timer Task **
+//
+// Timer processing - Tick is a poke that the IGMP processing receives
+// from the timer thread. The entire IGMP processing receives a single
+// tick.
+
+// TickTask structure
+type TickTask struct {
+	ctx    context.Context
+	taskID uint8
+	ts     string
+}
+
+// NewTickTask is constructor for TickTask
+func NewTickTask() *TickTask {
+	return &TickTask{}
+}
+
+// Name to return the name of the task
+func (tt *TickTask) Name() string {
+	return "Process Tick"
+}
+
+// TaskID to return the task id
+func (tt *TickTask) TaskID() uint8 {
+	return tt.taskID
+}
+
+// Timestamp to return the timestamp of task
+func (tt *TickTask) Timestamp() string {
+	return tt.ts
+}
+
+// Stop to stop the task
+func (tt *TickTask) Stop() {
+}
+
+// Start to start the task
+func (tt *TickTask) Start(ctx context.Context, taskID uint8) error {
+	tt.taskID = taskID
+	tt.ctx = ctx
+	GetApplication().IgmpTick()
+	return nil
+}
+
+// ---------------------------------------------------------------
+// ** Packet processing Task **
+//
+//
+
+// IgmpPacketTask structure
+type IgmpPacketTask struct {
+	ctx    context.Context
+	taskID uint8
+	Device string
+	Port   string
+	Pkt    gopacket.Packet
+	ts     string
+}
+
+// NewIgmpPacketTask is the constructor for IgmpPacketTask
+func NewIgmpPacketTask(device string, port string, pkt gopacket.Packet) *IgmpPacketTask {
+	var pt IgmpPacketTask
+	pt.Device = device
+	pt.Port = port
+	pt.Pkt = pkt
+	pt.ts = (time.Now()).Format(time.RFC3339Nano)
+	return &pt
+}
+
+// Name to return name of the task
+func (pt *IgmpPacketTask) Name() string {
+	return "Igmp Packet Task"
+}
+
+// TaskID to return the task id
+func (pt *IgmpPacketTask) TaskID() uint8 {
+	return pt.taskID
+}
+
+// Timestamp to return the timestamp for the task
+func (pt *IgmpPacketTask) Timestamp() string {
+	return pt.ts
+}
+
+// Stop to stop the task
+func (pt *IgmpPacketTask) Stop() {
+}
+
+// Start to start the task
+func (pt *IgmpPacketTask) Start(ctx context.Context, taskID uint8) error {
+	pt.taskID = taskID
+	pt.ctx = ctx
+	GetApplication().IgmpProcessPkt(pt.Device, pt.Port, pt.Pkt)
+	return nil
+}
+
+// UpdateMvlanTask structure
+type UpdateMvlanTask struct {
+	ctx      context.Context
+	taskID   uint8
+	DeviceID string
+	mvp      *MvlanProfile
+	ts       string
+}
+
+// NewUpdateMvlanTask is the constructor for UpdateMvlanTask
+func NewUpdateMvlanTask(mvp *MvlanProfile, deviceID string) *UpdateMvlanTask {
+	var mt UpdateMvlanTask
+	mt.mvp = mvp
+	mt.DeviceID = deviceID
+	mt.ts = (time.Now()).Format(time.RFC3339Nano)
+	return &mt
+}
+
+// Name to retun the name of the task
+func (mt *UpdateMvlanTask) Name() string {
+	return "Update Mvlan Task"
+}
+
+// TaskID to return the task id of the task
+func (mt *UpdateMvlanTask) TaskID() uint8 {
+	return mt.taskID
+}
+
+// Timestamp to return the timestamp of the task
+func (mt *UpdateMvlanTask) Timestamp() string {
+	return mt.ts
+}
+
+// Stop to stop the task
+func (mt *UpdateMvlanTask) Stop() {
+}
+
+// Start to start the task
+func (mt *UpdateMvlanTask) Start(ctx context.Context, taskID uint8) error {
+	mt.taskID = taskID
+	mt.ctx = ctx
+	mvp := mt.mvp
+	mvp.UpdateProfile(mt.DeviceID)
+	return nil
+}
diff --git a/internal/pkg/application/major_upgrade.go b/internal/pkg/application/major_upgrade.go
new file mode 100644
index 0000000..13c3762
--- /dev/null
+++ b/internal/pkg/application/major_upgrade.go
@@ -0,0 +1,618 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"context"
+	"encoding/json"
+	"errors"
+	"voltha-go-controller/internal/pkg/types"
+	"sync"
+
+	"github.com/google/gopacket/layers"
+
+	"voltha-go-controller/database"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	//MigrationComplete Represents the Migration Complete
+	MigrationComplete = "Completed"
+	//MigrationInProgress Represents the Migration Inprogress
+	MigrationInProgress = "InProgress"
+	//MigrationFailed  Represents the Migration Failed
+	MigrationFailed = "Failed"
+	// StatusNone for no operations
+	StatusNone = "NONE"
+	//ModuleToBeDeleted - module where old version is deleted
+	ModuleToBeDeleted = "ModuleToBeDeleted"
+)
+
+//DataMigration represents the Verison and Status info for Major Version Upgrade.
+type DataMigration struct {
+	Version   string
+	Status    string
+	ModuleVer map[string]string // eg. "service": "v1"
+}
+
+type paramsMigrationFunc func([]byte) string
+
+//map to store conversion functions
+var migrationMap = map[string]paramsMigrationFunc{
+	database.ServicePath:        MigrateServices,
+	database.DevicePath:         MigrateDevices,
+	database.DevicePortPath:     MigrateDevicePorts,
+	database.DeviceFlowPath:     MigrateDeviceFlows,
+	database.DeviceGroupPath:    MigrateDeviceGroups,
+	database.DeviceMeterPath:    MigrateDeviceMeters,
+	database.VnetPath:           MigrateVnets,
+	database.VpvPath:            MigrateVpvs,
+	database.MvlanPath:          MigrateMvlans,
+	database.MeterPath:          MigrateMeters,
+	database.IgmpConfPath:       MigrateIgmpConfs,
+	database.IgmpGroupPath:      MigrateIgmpGroups,
+	database.IgmpDevicePath:     MigrateIgmpDevices,
+	database.IgmpChannelPath:    MigrateIgmpChannels,
+	database.IgmpPortPath:       MigrateIgmpPorts,
+	database.IgmpProfPath:       MigrateIgmpProfs,
+	database.McastConfigPath:    MigrateMcastConfs,
+	database.LogLevelPath:       MigrateLogLevels,
+	database.HealthPath:         MigrateHealth,
+	database.PonCounterPath:     MigratePonCounters,
+	database.ChannelCounterPath: MigrateChannelCounters,
+	database.ServiceCounterPath: MigrateServiceCounters,
+	database.NbDevicePath:       MigrateNbDevices,
+	database.DeviceFlowHashPath: MigrateDeviceFlowHash,
+}
+
+// WriteToDb write a meter profile to DB
+func (md *DataMigration) WriteToDb() error {
+	b, err := json.Marshal(md)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutMigrationInfo(string(b)); err1 != nil {
+		return err1
+	}
+	return nil
+}
+
+// DelFromDb delete a meter profile from DB
+func (md *DataMigration) DelFromDb() {
+	if err := db.DelMigrationInfo(); err != nil {
+		logger.Warnw(ctx, "DelMigrationInfo Failed", log.Fields{"Error": err})
+	}
+}
+
+// GetMigrationInfo to get data migration info
+func GetMigrationInfo(dmInfo *DataMigration) error {
+	var migrationInfo string
+	var err error
+	if db == nil {
+		db = database.GetDatabase()
+	}
+	if migrationInfo, err = db.GetMigrationInfo(); err != nil {
+		return err
+	}
+	err = json.Unmarshal([]byte(migrationInfo), &dmInfo)
+	if err != nil {
+		logger.Warn(ctx, "Unmarshal of migrationinfo failed")
+		return err
+	}
+	return nil
+}
+
+// CheckIfMigrationRequired Checks if Migration is Completed
+// Only Data Migration and Reboot would be handled in the Below function
+// When Roll back happens just Delete of DB keys has to happen
+// which will be done once delete key request is received from MSM
+func CheckIfMigrationRequired(ctx context.Context) bool {
+	Migrate := new(DataMigration)
+	var NoDataInDB bool
+	err := GetMigrationInfo(Migrate)
+	logger.Debug(ctx, "Migration data", log.Fields{"DataMigration": Migrate})
+	// No DB entry represents N verison Bring Up for the First time
+	if err != nil {
+		NoDataInDB = true
+		logger.Error(ctx, "Failed to read the Migration Data from DB ")
+	}
+	// Covers N verison bringup and Reboot Senarios
+	if NoDataInDB {
+		logger.Info(ctx, "Data Migration Not Required")
+		Migrate.Version = database.PresentVersion
+		Migrate.Status = MigrationComplete
+		Migrate.ModuleVer = database.PresentVersionMap
+		if err := Migrate.WriteToDb(); err != nil {
+			logger.Error(ctx, "DB Write failed for Migration Path", log.Fields{"error": err})
+		}
+		//MigrateProbestatus has to be Updated to Complete when No Migration is Required
+		logger.Debug(ctx, "Migration Probe Status", log.Fields{"Migration Probe": Migrate.Status})
+		//probe.UpdateDBMigrationStatus(ctx, true)
+		return false
+		// Migration required when vgc moves to Higher Versions
+	} else if Migrate.ModuleVer == nil {
+		// This case will hit when DataMigration is present with old schema
+		// and DataMigration schema has changed.
+		// In this case compare previous and current version configured in the models.
+		for key, currVer := range database.PresentVersionMap {
+			if currVer > database.PreviousVersionMap[key] {
+				logger.Infow(ctx, "DB Migration needed for", log.Fields{"comp": key})
+				return true
+			}
+		}
+	} else {
+		var isVersionChanged bool
+		// Compare the current version with previous version present in DB.
+		// This case will also hit in case of POD restart.
+		for key, currVer := range database.PresentVersionMap {
+			if dbVer := Migrate.ModuleVer[key]; dbVer != "" {
+				if currVer > dbVer {
+					logger.Infow(ctx, "DB Migration needed for", log.Fields{"comp": key})
+					isVersionChanged = true
+				}
+			}
+		}
+		database.DBVersionMap = Migrate.ModuleVer // Store DB data
+
+		if isVersionChanged {
+			return true
+		}
+	}
+
+	// In case Service Reboots/Rolls Back then Probe Success to MSM
+	logger.Debug(ctx, "Migration Probe Status", log.Fields{"Migration Probe": Migrate.Status})
+	//probe.UpdateDBMigrationStatus(ctx, true)
+	return false
+}
+
+// InitiateDataMigration Migrates the DB data
+// depending on the bool value returned by CheckIfMigrationDone
+func InitiateDataMigration(ctx context.Context) {
+	var err error
+	Migrate := new(DataMigration)
+	var migrationWG sync.WaitGroup
+
+	//Keeping it outside to avoid race condition where the
+	// wait check is reached before the go toutine for data migraiton is triggered
+	migrationWG.Add(1)
+
+	go func() {
+		logger.Debug(ctx, "Started Go Routine for data migration")
+		err = MigrateDBData()
+		if err != nil {
+			logger.Error(ctx, "Failed to Migrate the Data", log.Fields{"error": err})
+			Migrate.Status = MigrationFailed
+			if err := Migrate.WriteToDb(); err != nil {
+				logger.Error(ctx, "DB Write failed to Migration Path", log.Fields{"error": err})
+			}
+		}
+		logger.Debug(ctx, "Completed Go Routine for data migration")
+		migrationWG.Done()
+
+		Migrate.Version = database.PresentVersion
+		Migrate.Status = MigrationInProgress
+		Migrate.ModuleVer = database.PresentVersionMap
+		if err = Migrate.WriteToDb(); err != nil {
+			logger.Error(ctx, "DB Write failed for Migration Path", log.Fields{"error": err})
+			return
+		}
+	}()
+	// Failure Senario can be Exceptions, incase of panic Update the status as failed
+	defer func() {
+		if err := recover(); err != nil {
+			logger.Error(ctx, "Migration failure due to Exception happend", log.Fields{"reason": err})
+			Migrate.Status = MigrationFailed
+			if err := Migrate.WriteToDb(); err != nil {
+				logger.Error(ctx, "DB Write failed for Migration Path", log.Fields{"error": err})
+			}
+			//probe.UpdateDBMigrationStatus(ctx, false)
+			return
+		}
+	}()
+	// Wait for all the Db data  migration to complete
+	migrationWG.Wait()
+	//probe.UpdateDBMigrationStatus(ctx, true)
+	Migrate.Status = MigrationComplete
+	if err := Migrate.WriteToDb(); err != nil {
+		logger.Error(ctx, "DB Write failed for Migration Path", log.Fields{"error": err})
+	}
+	logger.Info(ctx, "Migration completed successfully", log.Fields{"Status": Migrate.Status})
+}
+
+// MigrateDBData to migrate database data
+func MigrateDBData() error {
+
+	var err error
+	for module, currentVersion := range database.PresentVersionMap {
+		if currentVersion == database.DBVersionMap[module] {
+			logger.Infow(ctx, "No Data Migration required for module", log.Fields{"Table": module, "Version": currentVersion})
+			continue
+		}
+
+		if _, ok := migrationMap[module]; ok {
+			switch module {
+			case database.DeviceFlowPath,
+				database.DevicePortPath,
+				database.DeviceMeterPath,
+				database.DeviceGroupPath,
+				database.DeviceFlowHashPath:
+				err = FetchAndMigrateDeviceDBData(module)
+			default:
+				err = FetchAndMigrateDBData(module)
+			}
+		} else {
+			logger.Infow(ctx, "No Data Migration handling found for module", log.Fields{"Table": module, "Version": currentVersion})
+		}
+
+		if err != nil {
+			logger.Errorw(ctx, "Error in data migration", log.Fields{"Module": module})
+			return err
+		}
+	}
+	return nil
+}
+
+//FetchAndMigrateDeviceDBData fetchs the data from database and migrte the same to latest versions and store ot back ot database
+func FetchAndMigrateDeviceDBData(module string) error {
+	logger.Error(ctx, "Data Migration not implemented for Device DB Data")
+	return nil
+}
+
+//FetchAndMigrateDBData fetchs the data from database and migrte the same to latest versions and store ot back ot database
+func FetchAndMigrateDBData(module string) error {
+
+	previousPath := database.GetModuleKeypath(module, database.PreviousVersionMap[module])
+	dbPathKeysValueMap, err := db.List(previousPath)
+	if err != nil {
+		logger.Error(ctx, "failed to Fetch the Keys from Redis", log.Fields{"error": err})
+		//No return required, Data might not be present in DB
+		return nil
+	}
+	if len(dbPathKeysValueMap) == 0 {
+		logger.Debug(ctx, "No data present in DB for the path", log.Fields{"dbPath": module})
+		return nil
+	}
+
+	// Fetch each Path from previous version and store to present version after data migration changes
+	for hash, value := range dbPathKeysValueMap {
+		logger.Debug(ctx, "DB path", log.Fields{"hash": hash})
+		//convert the value to a specific type based on the dbPath
+		b, ok := value.Value.([]byte)
+		if !ok {
+			logger.Error(ctx, "The value type is not []byte")
+			return errors.New("Error-in-migration")
+		}
+
+		presentParams := migrationMap[module](b)
+		logger.Infow(ctx, "Migrated data", log.Fields{"presentParams": presentParams})
+		if "" == presentParams {
+			logger.Error(ctx, "Error in migrating data\n")
+			return errors.New("Error-in-migration")
+		} else if ModuleToBeDeleted == presentParams {
+			return nil
+		}
+		presentPath := database.GetKeyPath(module) + hash
+		logger.Infow(ctx, "Before writing to DB", log.Fields{"presentParams": presentParams})
+		if err := db.Put(presentPath, presentParams); err != nil {
+			logger.Error(ctx, "Update Params failed", log.Fields{"key": presentPath, "presentparams": presentParams})
+			return err
+		}
+	}
+	return nil
+}
+
+//MigrateServices modifyies the old data as per current version requirement and updates the database
+func MigrateServices(data []byte) string {
+	var vs VoltService
+	var updatedData, updatedData1 []byte
+	var vsmap map[string]interface{}
+	var err1 error
+
+	err := json.Unmarshal(data, &vsmap)
+	if err != nil {
+		logger.Warn(ctx, "Unmarshal of VPV failed", log.Fields{"error": err})
+		return ""
+	}
+	// changes to handle change in data type of MacLearning parameter
+	if updatedData1, err1 = json.Marshal(&vsmap); err1 != nil {
+		logger.Warnw(ctx, "Marshal of Service failed", log.Fields{"Error": err1.Error()})
+		return ""
+	}
+
+	if err2 := json.Unmarshal(updatedData1, &vs); err != nil {
+		logger.Warnw(ctx, "Unmarshal-failed", log.Fields{"err": err2})
+		return ""
+	}
+
+	if vsmap["MacLearning"] == true {
+		vs.MacLearning = Learn
+
+	}
+
+	//Migration
+	vs.PendingFlows = make(map[string]bool)
+	vs.AssociatedFlows = make(map[string]bool)
+	vs.DeleteInProgress = false
+	vs.PonPort = 0xFF
+	if updatedData, err = json.Marshal(vs); err != nil {
+		logger.Warnw(ctx, "Marshal of Service failed", log.Fields{"Error": err.Error()})
+		return ""
+	}
+	logger.Infow(ctx, "Service Migrated", log.Fields{"Service": vs.Name, "PresentVersion": database.PresentVersionMap[database.ServicePath]})
+	return string(updatedData)
+}
+
+//MigrateDevices modifyies the old data as per current version requirement and updates the database
+func MigrateDevices(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Devices")
+	return ""
+}
+
+//MigrateDevicePorts modifyies the old data as per current version requirement and updates the database
+func MigrateDevicePorts(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Ports")
+	return ""
+}
+
+//MigrateDeviceFlows modifyies the old data as per current version requirement and updates the database
+func MigrateDeviceFlows(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Flows")
+	return ""
+}
+
+//MigrateDeviceGroups modifyies the old data as per current version requirement and updates the database
+func MigrateDeviceGroups(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Groups")
+	return ""
+}
+
+//MigrateDeviceMeters modifyies the old data as per current version requirement and updates the database
+func MigrateDeviceMeters(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Meters")
+	return ""
+}
+
+//MigrateDeviceFlowHash modifyies the old data as per current version requirement and updates the database
+func MigrateDeviceFlowHash(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for FlowHash")
+	return ""
+}
+
+//MigrateVnets modifyies the old data as per current version requirement and updates the database
+func MigrateVnets(data []byte) string {
+
+	var vnet VoltVnet
+	var updatedData []byte
+
+	err := json.Unmarshal(data, &vnet)
+	if err != nil {
+		logger.Warn(ctx, "Unmarshal of VNET failed", log.Fields{"error": err})
+		return ""
+	}
+
+	if vnet.SVlanTpid == 0 {
+		vnet.SVlanTpid = layers.EthernetTypeDot1Q
+	}
+	// MacLeanring parameter was not stored in vnets in 2.7 release.
+	if vnet.DhcpRelay || vnet.ArpLearning {
+		vnet.MacLearning = Learn
+	} else if !vnet.DhcpRelay && !vnet.ArpLearning {
+		vnet.MacLearning = MacLearningNone
+	}
+	vnet.PendingDeleteFlow = make(map[string]map[string]bool)
+	vnet.DeleteInProgress = false
+	if updatedData, err = json.Marshal(vnet); err != nil {
+		logger.Warnw(ctx, "Marshal of Vnet failed", log.Fields{"Error": err.Error()})
+		return ""
+	}
+	logger.Infow(ctx, "Vnet Migrated", log.Fields{"Vnet Name": vnet.Name, "PresentVersion": database.PresentVersionMap[database.VnetPath]})
+	return string(updatedData)
+}
+
+//MigrateVpvs modifyies the old data as per current version requirement and updates the database
+func MigrateVpvs(data []byte) string {
+	var vpv VoltPortVnet
+	var updatedData, updatedData1 []byte
+	var vpvmap map[string]interface{}
+	var err1 error
+	var usFlowsApplied, dsFlowsApplied bool
+
+	err := json.Unmarshal(data, &vpvmap)
+	if err != nil {
+		logger.Warn(ctx, "Unmarshal of VPV failed", log.Fields{"error": err})
+		return ""
+	}
+	// changes to handle change in data type of MacLearning parameter
+	if updatedData1, err1 = json.Marshal(&vpvmap); err1 != nil {
+		logger.Warnw(ctx, "Marshal of Service failed", log.Fields{"Error": err1.Error()})
+		return ""
+	}
+
+	if err2 := json.Unmarshal(updatedData1, &vpv); err != nil {
+		logger.Warnw(ctx, "Unmarshal-failed", log.Fields{"err": err2})
+
+	}
+
+	if vpvmap["MacLearning"] == true {
+		vpv.MacLearning = Learn
+
+	}
+	if vpvmap["UsFlowsApplied"] == true {
+		usFlowsApplied = true
+	}
+
+	if vpvmap["DsFlowsApplied"] == true {
+		dsFlowsApplied = true
+	}
+
+	if usFlowsApplied && dsFlowsApplied {
+		vpv.FlowsApplied = true
+	}
+	//Migration
+	if vpv.SVlanTpid == 0 {
+		vpv.SVlanTpid = layers.EthernetTypeDot1Q
+	}
+	vpv.VnetName = VnetKey(vpv.SVlan, vpv.CVlan, vpv.UniVlan)
+	vpv.PendingDeleteFlow = make(map[string]bool)
+	vpv.PonPort = 0xFF
+
+	if updatedData, err = json.Marshal(vpv); err != nil {
+		logger.Warnw(ctx, "Marshal of VPV failed", log.Fields{"Error": err.Error()})
+		return ""
+	}
+	logger.Infow(ctx, "VPV Migrated", log.Fields{"Device": vpv.Device, "port": vpv.Port, "SVlan": vpv.SVlan,
+			"CVlan": vpv.CVlan, "UniVlan": vpv.UniVlan, "PresentVersion": database.PresentVersionMap[database.VpvPath]})
+	return string(updatedData)
+}
+
+//MigrateMvlans modifyies the old data as per current version requirement and updates the database
+func MigrateMvlans(data []byte) string {
+	var mvp MvlanProfile
+	var updatedData []byte
+
+	err := json.Unmarshal(data, &mvp)
+	if err != nil {
+		logger.Warn(ctx, "Unmarshal of VPV failed")
+		return ""
+	}
+	// Mvlan Migration
+	mvp.IgmpServVersion = make(map[string]*uint8)
+	for srNo := range mvp.DevicesList {
+		var servVersion uint8
+		mvp.IgmpServVersion[srNo] = &servVersion
+	}
+
+	if updatedData, err = json.Marshal(mvp); err != nil {
+		logger.Warnw(ctx, "Marshal of Mvlan Profile failed", log.Fields{"Error": err.Error()})
+		return ""
+	}
+	logger.Infow(ctx, "Mvlan Profile Migrated", log.Fields{"MvlanProfileName": mvp.Name, "PresentVersion": database.PresentVersionMap[database.MvlanPath]})
+	return string(updatedData)
+}
+
+//MigrateMeters modifyies the old data as per current version requirement and updates the database
+func MigrateMeters(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Meters")
+	return ""
+}
+
+//MigrateIgmpConfs modifyies the old data as per current version requirement and updates the database
+func MigrateIgmpConfs(data []byte) string {
+	var igmpProfile IgmpProfile
+
+	err := json.Unmarshal(data, &igmpProfile)
+	if err != nil {
+		logger.Warn(ctx, "Unmarshal of IGMP failed")
+		return ""
+	}
+	if err := igmpProfile.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp profile Write to DB failed", log.Fields{"profileID": igmpProfile.ProfileID})
+	}
+
+	logger.Infow(ctx, "Igmp Conf Migrated", log.Fields{"Profile": igmpProfile, "PresentVersion": database.PresentVersionMap[database.VpvPath]})
+	return ModuleToBeDeleted
+}
+
+//MigrateIgmpGroups modifyies the old data as per current version requirement and updates the database
+func MigrateIgmpGroups(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for IGMP Groups")
+	return ""
+}
+
+//MigrateIgmpDevices modifyies the old data as per current version requirement and updates the database
+func MigrateIgmpDevices(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for IGMP Device")
+	return ""
+}
+
+//MigrateIgmpChannels modifyies the old data as per current version requirement and updates the database
+func MigrateIgmpChannels(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for IGMP Channels")
+	return ""
+}
+
+//MigrateIgmpPorts modifyies the old data as per current version requirement and updates the database
+func MigrateIgmpPorts(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for IGMP Ports")
+	return ""
+}
+
+//MigrateIgmpProfs modifyies the old data as per current version requirement and updates the database
+func MigrateIgmpProfs(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for IGMP Profs")
+	return ""
+}
+
+//MigrateMcastConfs modifyies the old data as per current version requirement and updates the database
+func MigrateMcastConfs(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Mcast Confs")
+	return ""
+}
+
+//MigrateLogLevels modifyies the old data as per current version requirement and updates the database
+func MigrateLogLevels(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Log Levels")
+	return ""
+}
+
+//MigrateHealth modifyies the old data as per current version requirement and updates the database
+func MigrateHealth(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Health")
+	return ""
+}
+
+//MigratePonCounters modifyies the old data as per current version requirement and updates the database
+func MigratePonCounters(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Pon Counters")
+	return ""
+}
+
+//MigrateChannelCounters modifyies the old data as per current version requirement and updates the database
+func MigrateChannelCounters(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Channel Counters")
+	return ""
+}
+
+//MigrateServiceCounters modifyies the old data as per current version requirement and updates the database
+func MigrateServiceCounters(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for Service Counters")
+	return ""
+}
+
+//MigrateNbDevices modifyies the old data as per current version requirement and updates the database
+func MigrateNbDevices(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for NB Devices")
+	return ""
+}
+
+//MigrateFlowHash modifyies the old data as per current version requirement and updates the database
+func MigrateFlowHash(data []byte) string {
+	logger.Error(ctx, "Data Migration not implemented for FLow Hash")
+	return ""
+}
+
+//DeleteDbPathKeys Deleted the paths from DB
+func DeleteDbPathKeys(keyPath string) error {
+	logger.Debug(ctx, "Deleting paths for version", log.Fields{"Path": keyPath})
+
+	// Delete all the keys
+	err := db.DeleteAll(keyPath)
+	if err != nil && err.Error() != common.ErrEntryNotFound.Error() {
+		logger.Error(ctx, "Delete Key failed", log.Fields{"error": err})
+		return err
+	}
+	return nil
+}
diff --git a/internal/pkg/application/meters.go b/internal/pkg/application/meters.go
new file mode 100644
index 0000000..4b5fd71
--- /dev/null
+++ b/internal/pkg/application/meters.go
@@ -0,0 +1,331 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"encoding/json"
+	"errors"
+	"sync"
+
+	cntlr "voltha-go-controller/internal/pkg/controller"
+	"voltha-go-controller/database"
+	"voltha-go-controller/internal/pkg/of"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// VoltShaperConfig is shaper profile configuration structure
+type VoltShaperConfig struct {
+	Name      string
+	BurstSize uint32
+}
+
+// VoltBwConfig is bandwidth profile configuration structure
+type VoltBwConfig struct {
+	Name string
+	Fir  uint32
+	Air  uint32
+	Eir  uint32
+}
+
+// VoltBandwidthProf is bandwidth profile stored at VGC
+type VoltBandwidthProf struct {
+	VoltBwConfig
+}
+
+// VoltShaperProf is shaper profile stored at VGC
+type VoltShaperProf struct {
+	VoltShaperConfig
+}
+
+// VoltMeterProf is meter profile stored at VGC
+type VoltMeterProf struct {
+	VoltMeter
+}
+
+// MeterMgr structure
+type MeterMgr struct {
+	Meters      sync.Map
+	MetersByID  sync.Map
+	LastMeterID uint32
+}
+
+// Init to initialize MeterMgr
+func (m *MeterMgr) Init() {
+	m.LastMeterID = 0
+}
+
+// VoltMeter : A VOLT meter is a combination of BW and shaper profiles
+// The ID is generated by the VOLT application
+type VoltMeter struct {
+	Name               string
+	ID                 uint32
+	Fir                uint32
+	Air                uint32
+	Eir                uint32
+	BurstSize          uint32
+	AssociatedServices uint32
+	Version            string
+	Cir                uint32
+	Cbs                uint32
+	Pir                uint32
+	Pbs                uint32
+	Gir                uint32
+	Ebs                uint32
+}
+
+// WriteToDb to write a meter profile to DB
+func (vm *VoltMeter) WriteToDb() error {
+	vm.Version = database.PresentVersionMap[database.MeterPath]
+	b, err := json.Marshal(vm)
+	if err != nil {
+		return err
+	}
+	if err1 := db.PutMeter(vm.Name, string(b)); err1 != nil {
+		return err1
+	}
+	return nil
+}
+
+// DelFromDb to delete a meter profile from DB
+func (vm *VoltMeter) DelFromDb() {
+	_ = db.DelMeter(vm.Name)
+}
+
+// GetMeterByName to get meter by name
+func (m *MeterMgr) GetMeterByName(name string) (*VoltMeter, bool) {
+	meter, ok := m.Meters.Load(name)
+	logger.Infow(ctx, "Meter Obtained Name", log.Fields{"Meter": meter})
+	if ok {
+		return meter.(*VoltMeter), ok
+	}
+	return nil, ok
+}
+
+// GetMeterByID to get meter by ID
+func (m *MeterMgr) GetMeterByID(id uint32) (*VoltMeter, bool) {
+	meter, ok := m.MetersByID.Load(id)
+	logger.Infow(ctx, "Meter Obtained ID", log.Fields{"Meter": meter})
+	if ok {
+		return meter.(*VoltMeter), ok
+	}
+	return nil, ok
+}
+
+// AddMeter to add meter
+func (m *MeterMgr) AddMeter(meter *VoltMeter) {
+	m.Meters.Store(meter.Name, meter)
+	m.MetersByID.Store(meter.ID, meter)
+	logger.Infow(ctx, "Meter Added/Updated", log.Fields{"Meter": meter, "Name": meter.Name, "Id": meter.ID})
+}
+
+// DelMeter to delete meter
+func (m *MeterMgr) DelMeter(meter *VoltMeter) {
+	m.Meters.Delete(meter.Name)
+	m.MetersByID.Delete(meter.ID)
+	logger.Infow(ctx, "Meter Deleted", log.Fields{"Meter": meter, "Name": meter.Name, "Id": meter.ID})
+}
+
+// AddToDevice to add meter to the device
+func (vm *VoltMeter) AddToDevice(port string, device string, aggVM *VoltMeter) {
+	logger.Debugw(ctx, "Adding Meter To Device", log.Fields{"Id": vm.ID, "Device": device, "Port": port})
+	meter := of.NewMeter(vm.ID)
+	// meter.AddBand(vm.Air, vm.BurstSize)
+	// meter.AddBand(vm.Eir, vm.BurstSize)
+	// if aggVM != nil {
+	// 	meter.AddBand(aggVM.Air, aggVM.BurstSize)
+	// 	meter.AddBand(aggVM.Eir, aggVM.BurstSize)
+	// }
+
+	//Community VGC Impl
+
+	//Set Cir
+	if vm.Cir != 0 {
+		meter.AddBand(vm.Cir, vm.Cbs)
+	}
+
+	//Set Air to 0 if both air & gir are set
+	if vm.Air != 0 && vm.Gir != 0 {
+		vm.Air = 0
+	}
+
+	//Set Pir & Pbs
+	var pir uint32
+	var pbs uint32
+	if vm.Pir != 0 {
+		pir = vm.Pir
+	} else {
+		pir = vm.Eir + vm.Cir + vm.Gir + vm.Air
+	}
+
+	if vm.Pbs != 0 {
+		pbs = vm.Pbs
+	} else {
+		pbs = vm.Ebs + vm.Cbs
+	}
+	meter.AddBand(pir, pbs)
+
+	//Set Gir
+	if vm.Gir != 0 {
+		meter.AddBand(vm.Gir, 0)
+	}
+
+	logger.Infow(ctx, "Meter Config", log.Fields{"Cir": vm.Cir, "Air": vm.Air, "Pir": vm.Pir, "Gir": vm.Gir, "Eir": vm.Eir})
+	logger.Infow(ctx, "Meter Burst Config", log.Fields{"Cbs": vm.Cbs, "Pbs": vm.Pbs})
+	logger.Infow(ctx, "Meter Burst Oper", log.Fields{"Pir": pir, "Pbs": pbs})
+	//Set Air
+	// Air is used in place of Gir only if Gir is
+	// not present and Air is not 0
+	if vm.Air != 0 {
+		meter.AddBand(vm.Air, 0)
+	}
+
+	logger.Debug(ctx, "Total Bands are", log.Fields{"meter": *meter})
+	if err := cntlr.GetController().ModMeter(port, device, of.MeterCommandAdd, meter); err != nil {
+		logger.Warnw(ctx, "Add meter to device Failed", log.Fields{"Id": vm.ID, "meter": *meter, "Error": err})
+	}
+}
+
+// AddMeterToDevice to add meter to the device
+func (m *MeterMgr) AddMeterToDevice(port string, device string, meterID uint32, aggMeterID uint32) {
+	var aggVM *VoltMeter
+	vm, err := m.GetMeterByProfID(meterID)
+	if err == nil {
+		if 0 != aggMeterID { //Assuming valid meter id will never be 0
+			if aggVM, err = m.GetMeterByProfID(aggMeterID); err != nil {
+				logger.Warnw(ctx, "Aggregated Meter not found", log.Fields{"Id": aggMeterID})
+			}
+		}
+		vm.AddToDevice(port, device, aggVM)
+	} else {
+		logger.Warnw(ctx, "Meter not found", log.Fields{"Id": meterID})
+	}
+}
+
+// RestoreMetersFromDb to read from the DB and restore all the services
+func (m *MeterMgr) RestoreMetersFromDb() {
+	// VNETS must be learnt first
+	logger.Infow(ctx, "LastMeterID on restart", log.Fields{"LastMeterID": m.LastMeterID})
+	ms, _ := db.GetMeters()
+	for _, mt := range ms {
+		b, ok := mt.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var meter VoltMeter
+		err := json.Unmarshal(b, &meter)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of meter profile failed")
+			continue
+		}
+		logger.Infow(ctx, "Retrieved Meter", log.Fields{"Meter": meter.Name})
+		m.AddMeter(&meter)
+		if meter.ID > m.LastMeterID {
+			m.LastMeterID = meter.ID
+		}
+	}
+	logger.Infow(ctx, "LastMeterID on reading DB", log.Fields{"LastMeterID": m.LastMeterID})
+}
+
+// AddMeterProf to add the meter profile name as key
+func (va *VoltApplication) AddMeterProf(cfg VoltMeter) {
+
+	mm := &va.MeterMgr
+	if _, ok := mm.GetMeterByName(cfg.Name); ok {
+		logger.Warnw(ctx, "Meter profile exists", log.Fields{"Name": cfg.Name})
+		return
+	}
+
+	mm.LastMeterID++
+	//FIX-ME: Hardcoded the meter-id temp till meter delete is introduced
+	//Restriction: Only one meter profile should be used across all services
+	//	id := uint32(1) //mm.LastMeterId
+	id := mm.LastMeterID
+	cfg.ID = id
+	mm.AddMeter(&cfg)
+	if err := cfg.WriteToDb(); err != nil {
+		logger.Warnw(ctx, "MeterProf Write to DB Failed", log.Fields{"MeterConfig": cfg, "Error": err})
+	}
+}
+
+// UpdateMeterProf to update the meter profile
+func (va *VoltApplication) UpdateMeterProf(cfg VoltMeter) {
+	mm := &va.MeterMgr
+	if _, ok := mm.GetMeterByName(cfg.Name); !ok {
+		logger.Warnw(ctx, "Meter profile does not exist", log.Fields{"Name": cfg.Name})
+		return
+	}
+	mm.AddMeter(&cfg)
+	if err := cfg.WriteToDb(); err != nil {
+		logger.Warnw(ctx, "MeterProf Write to DB Failed", log.Fields{"MeterConfig": cfg, "Error": err})
+	}
+}
+
+// GetMeterByProfID to get a meter based on the identities of bandwidth profile and shaper
+// profile names.
+func (m *MeterMgr) GetMeterByProfID(id uint32) (*VoltMeter, error) {
+	if mtr, ok := m.GetMeterByID(id); ok {
+		return mtr, nil
+	}
+	return nil, errors.New("Meter Missing")
+}
+
+// GetMeter to get a meter based on the identities of bandwidth profile and shaper
+// profile names.
+func (m *MeterMgr) GetMeter(meterID string) (*VoltMeter, error) {
+	if mt, ok := m.GetMeterByName(meterID); ok {
+		return mt, nil
+	}
+	return nil, errors.New("Meter Missing")
+}
+
+// DeleteFromDevice to delete meter from the device
+func (vm *VoltMeter) DeleteFromDevice(port string, device string) {
+
+	meter := of.NewMeter(vm.ID)
+
+	logger.Debugw(ctx, "Delete meter from device", log.Fields{"Id": vm.ID, "meter": *meter})
+	if err := cntlr.GetController().ModMeter(port, device, of.MeterCommandDel, meter); err != nil {
+		logger.Warnw(ctx, "Delete meter from device Failed", log.Fields{"Id": vm.ID, "meter": *meter, "Error": err})
+	}
+}
+
+// DelMeterProf to delete meter profile
+func (va *VoltApplication) DelMeterProf(name string) error {
+	mm := &va.MeterMgr
+	if _, ok := mm.GetMeterByName(name); !ok {
+		logger.Warnw(ctx, "Meter profile does not exist", log.Fields{"Name": name})
+		return errors.New("Meter profile doesn't exist")
+	}
+	cfg, _ := mm.GetMeterByName(name)
+	if cfg.AssociatedServices != 0 {
+		logger.Warnw(ctx, "Mismatch in submgr and vgc oeter profile service reference",
+			log.Fields{"MeterProfile": name, "serviceCount": cfg.AssociatedServices})
+		return errors.New("Service reference is not 0")
+	}
+	//TODO : delete from all devices
+	delmeterFromDevice := func(key interface{}, value interface{}) bool {
+		device := key.(string)
+		port, _ := GetApplication().GetNniPort(device)
+		cfg.DeleteFromDevice(port, device)
+		return true
+	}
+	va.DevicesDisc.Range(delmeterFromDevice)
+	cfg.DelFromDb()
+	//Delete meter from device will be invoked by caller separately
+	mm.DelMeter(cfg)
+	return nil
+}
diff --git a/internal/pkg/application/minor_upgrade.go b/internal/pkg/application/minor_upgrade.go
new file mode 100644
index 0000000..8953470
--- /dev/null
+++ b/internal/pkg/application/minor_upgrade.go
@@ -0,0 +1,247 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"errors"
+	"net"
+	"voltha-go-controller/internal/pkg/types"
+
+	"strings"
+
+	"github.com/google/gopacket/layers"
+	"voltha-go-controller/database"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+type paramsUpdationFunc func(hash string, value interface{}) error
+
+//map to store conversion functions
+var updationMap = map[string]paramsUpdationFunc{
+	database.VnetPath:       updateVnets,
+	database.VpvPath:        updateVpvs,
+	database.ServicePath:    updateServices,
+	database.MvlanPath:      updateMvlans,
+	database.IgmpGroupPath:  updateIgmpGroups,
+	database.IgmpDevicePath: updateIgmpDevices,
+	database.IgmpProfPath:   updateIgmpProfiles,
+}
+
+// UpdateDbData to update database data
+func UpdateDbData(dbPath, hash string, value interface{}) error {
+	if migrationFunc, ok := updationMap[dbPath]; ok {
+		err := migrationFunc(hash, value)
+		if err != nil {
+			logger.Error(ctx, "Error in migrating data\n")
+			return errors.New("Error-in-migration")
+		}
+	}
+	return nil
+}
+
+//This function modifyies the old data as per current version requirement and also
+//returns the new path on which the modified data has to be written
+func updateServices(hash string, value interface{}) error {
+	param := value.(*VoltService)
+	param.VnetID = VnetKey(param.SVlan, param.CVlan, param.UniVlan)
+	return nil
+}
+
+//This function modifyies the old data as per current version requirement and also
+//returns the new path on which the modified data has to be written
+func updateVnets(hash string, value interface{}) error {
+	param := value.(*VoltVnet)
+	newKey := VnetKey(param.SVlan, param.CVlan, param.UniVlan)
+	if newKey != hash {
+		//Delete the older key
+		_ = db.DelVnet(hash)
+	} else {
+		//Update SVlan Tag Protocol id param with default valud if not present
+		if param.SVlanTpid == 0 {
+			param.SVlanTpid = layers.EthernetTypeDot1Q
+		}
+	}
+	param.Name = newKey
+	if param.DevicesList == nil || len(param.DevicesList) == 0 {
+		param.DevicesList = append(param.DevicesList, "") //Empty OLT serial number as of now since submgr won't have proper serial num
+	}
+	return nil
+}
+
+//This function modifyies the old data as per current version requirement and also
+//returns the new path on which the modified data has to be written
+func updateVpvs(hash string, value interface{}) error {
+
+	//var param VoltPortVnet
+	param := value.(*VoltPortVnet)
+
+	//Update SVlan Tag Protocol id param with default valud if not present
+	if param.SVlanTpid == 0 {
+		param.SVlanTpid = layers.EthernetTypeDot1Q
+	}
+
+	if strings.Count(hash, "-") > 1 {
+		logger.Info(ctx, "Already upgraded")
+		return nil
+	}
+
+	//Add the vpv under new path
+	param.WriteToDb()
+	//delete the older path
+	fullPath := database.BasePath + database.VpvPath + hash
+	if err := db.Del(fullPath); err != nil {
+		logger.Errorw(ctx, "Vpv Delete from DB failed", log.Fields{"Error": err, "key": fullPath})
+	}
+	return nil
+}
+
+func updateMvlans(hash string, value interface{}) error {
+	param := value.(*MvlanProfile)
+	if param.DevicesList == nil || len(param.DevicesList) == 0 {
+		param.DevicesList = make(map[string]OperInProgress) //Empty OLT serial number as of now since submgr won't have proper serial num
+		if err := param.WriteToDb(); err != nil {
+			logger.Errorw(ctx, "Mvlan profile write to DB failed", log.Fields{"ProfileName": param.Name})
+		}
+
+	}
+	if _, ok := param.Groups[common.StaticGroup]; ok {
+		param.Groups[common.StaticGroup].IsStatic = true
+	}
+	return nil
+}
+
+//This function modifyies the old Igmp Group data as per current version requirement and also
+//returns the new path on which the modified data has to be written
+func updateIgmpGroups(hash string, value interface{}) error {
+
+	ig := value.(*IgmpGroup)
+	logger.Infow(ctx, "Group Data Migration", log.Fields{"ig": ig, "GroupAddr": ig.GroupAddr, "hash": hash})
+	if ig.GroupAddr == nil {
+		ig.GroupAddr = net.ParseIP("0.0.0.0")
+	}
+	if err := ig.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group Write to DB failed", log.Fields{"groupName": ig.GroupName})
+	}
+
+	return nil
+}
+
+//This function modifyies the old Igmp  Device data as per current version requirement and also
+//returns the new path on which the modified data has to be written
+func updateIgmpDevices(hash string, value interface{}) error {
+	igd := value.(*IgmpGroupDevice)
+	logger.Infow(ctx, "Group Device Migration", log.Fields{"igd": igd, "GroupAddr": igd.GroupAddr, "hash": hash})
+	if igd.GroupAddr == nil {
+		igd.GroupAddr = net.ParseIP("0.0.0.0")
+	}
+	if err := igd.WriteToDb(); err != nil {
+		logger.Errorw(ctx, "Igmp group device Write to DB failed", log.Fields{"Device": igd.Device,
+					"GroupName": igd.GroupName, "GroupAddr": igd.GroupAddr.String()})
+	}
+
+	return nil
+}
+
+//This function modifyies the old Igmp  Profile data as per current version requirement and also
+//returns the new path on which the modified data has to be written
+func updateIgmpProfiles(hash string, value interface{}) error {
+	igmpProfile := value.(*IgmpProfile)
+	logger.Infow(ctx, "IGMP Profile Migration", log.Fields{"igmpProfile": igmpProfile, "hash": hash})
+	return nil
+}
+
+func (ig *IgmpGroup) migrateIgmpDevices() {
+
+	devices, _ := db.GetPrevIgmpDevices(ig.Mvlan, ig.GroupName)
+	logger.Infow(ctx, "Migratable Devices", log.Fields{"Devices": devices})
+	for _, device := range devices {
+		b, ok := device.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		if igd, err := NewIgmpGroupDeviceFromBytes(b); err == nil {
+			key := database.BasePath + database.IgmpDevicePath + igd.Mvlan.String() + "/" + igd.GroupName + "/" + igd.Device
+			logger.Infow(ctx, "Deleting old entry", log.Fields{"Path": key, "igd": igd})
+			if err := db.Del(key); err != nil {
+				logger.Errorw(ctx, "Igmp Group Delete from DB failed", log.Fields{"Error": err, "key": key})
+			}
+			if err := UpdateDbData(database.IgmpDevicePath, key, igd); err != nil {
+				logger.Warnw(ctx, "Group Device Migration failed", log.Fields{"IGD": igd, "Error": err})
+			} else {
+				logger.Infow(ctx, "Group Device Migrated", log.Fields{"IGD": igd})
+			}
+		} else {
+			logger.Warnw(ctx, "Unable to decode device from database", log.Fields{"str": string(b)})
+		}
+	}
+}
+
+func (igd *IgmpGroupDevice) migrateIgmpChannels() {
+
+	channels, _ := db.GetPrevIgmpChannels(igd.GroupName, igd.Device)
+	logger.Infow(ctx, "Migratable Channels", log.Fields{"Channels": channels})
+	for _, channel := range channels {
+
+		b, ok := channel.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		if igc, err := NewIgmpGroupChannelFromBytes(b); err == nil {
+			key := database.BasePath + database.IgmpChannelPath + igc.GroupName + "/" + igc.Device + "/" + igc.GroupAddr.String()
+			logger.Infow(ctx, "Deleting old entry", log.Fields{"Path": key, "igc": igc})
+			if err := db.Del(key); err != nil {
+				logger.Errorw(ctx, "Igmp Group Delete from DB failed", log.Fields{"Error": err, "key": key})
+			}
+			if err := igc.WriteToDb(); err != nil {
+				logger.Errorw(ctx, "Igmp group channel Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+			}
+
+			logger.Infow(ctx, "Group Channel Migrated", log.Fields{"IGD": igc})
+		} else {
+			logger.Warnw(ctx, "Unable to decode channel from database", log.Fields{"str": string(b)})
+		}
+	}
+}
+
+func (igc *IgmpGroupChannel) migrateIgmpPorts() {
+
+	ports, _ := db.GetPrevIgmpRcvrs(igc.GroupAddr, igc.Device)
+	logger.Infow(ctx, "Migratable Ports", log.Fields{"Ports": ports})
+	for _, port := range ports {
+
+		b, ok := port.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		if igp, err := NewIgmpGroupPortFromBytes(b); err == nil {
+			key := database.BasePath + database.IgmpPortPath + igc.GroupAddr.String() + "/" + igc.Device + "/" + igp.Port
+			logger.Infow(ctx, "Deleting old entry", log.Fields{"Key": key, "Igp": igp})
+			if err := db.Del(key); err != nil {
+				logger.Errorw(ctx, "Igmp Group port Delete from DB failed", log.Fields{"Error": err, "key": key})
+			}
+			if err := igp.WriteToDb(igc.Mvlan, igc.GroupAddr, igc.Device); err != nil {
+				logger.Errorw(ctx, "Igmp group port Write to DB failed", log.Fields{"mvlan": igc.Mvlan, "GroupAddr": igc.GroupAddr})
+			}
+
+			logger.Infow(ctx, "Group Port Migrated", log.Fields{"IGD": igp})
+		} else {
+			logger.Warnw(ctx, "Unable to decode port from database", log.Fields{"str": string(b)})
+		}
+	}
+}
diff --git a/internal/pkg/application/pppoeia.go b/internal/pkg/application/pppoeia.go
new file mode 100644
index 0000000..8b5a214
--- /dev/null
+++ b/internal/pkg/application/pppoeia.go
@@ -0,0 +1,618 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package application
+
+import (
+	"context"
+	"errors"
+	"net"
+	"time"
+
+	"github.com/google/gopacket"
+	"github.com/google/gopacket/layers"
+
+	cntlr "voltha-go-controller/internal/pkg/controller"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/util"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// PppoeIaState type
+type PppoeIaState uint8
+
+const (
+	// PppoeIaStateNone constant
+	PppoeIaStateNone PppoeIaState = iota
+	// PppoeIaStatePADI constant
+	PppoeIaStatePADI
+	// PppoeIaStatePADO constant
+	PppoeIaStatePADO
+	// PppoeIaStatePADR constant
+	PppoeIaStatePADR
+	// PppoeIaStatePADS constant
+	PppoeIaStatePADS
+	// PppoeIaStatePADT constant
+	PppoeIaStatePADT
+)
+
+const (
+	// PPPoEVendorID constant
+	PPPoEVendorID uint32 = 0x0DE9
+	// TYPECIRCUITID constant
+	TYPECIRCUITID byte = 0x01
+	// TYPEREMOTEID constant
+	TYPEREMOTEID byte = 0x02
+	// TYPEMINDATAUS constant
+	TYPEMINDATAUS byte = 0x83
+	// TYPEMINDATADS constant
+	TYPEMINDATADS byte = 0x84
+	// TYPEMAXDATAUS constant
+	TYPEMAXDATAUS byte = 0x87
+	// TYPEMAXDATADS constant
+	TYPEMAXDATADS byte = 0x88
+)
+
+var (
+	// DSLATTRVendorID is PPPoEVendorID in byte format
+	DSLATTRVendorID = util.Uint32ToByte(PPPoEVendorID)
+)
+
+// IPppoeIaSession interface
+type IPppoeIaSession interface {
+	GetCircuitID() []byte
+	GetRemoteID() []byte
+	GetNniVlans() (uint16, uint16)
+	GetPppoeIaState() PppoeIaState
+	SetPppoeIaState(PppoeIaState)
+	SetMacAddr(net.HardwareAddr)
+}
+
+// PppoeIaRelayVnet : The PppoeIa relay sessions are stored in a map to be retrieved from when
+// a response is received from the network. The map uses the VLANs and the
+// the MAC address as key to finding the service
+// PppoeIa Relay Virtual Network hosts a set of PppoeIa relay sessions that belong
+// to the network. It supports two VLANs as its identify. If a single VLAN or
+// no VLAN is to be used, those two should be passed as 4096 (VlanNone)
+type PppoeIaRelayVnet struct {
+	OuterVlan uint16
+	InnerVlan uint16
+	sessions  *util.ConcurrentMap //map[[6]byte]IPppoeIaSession
+}
+
+// PppoeIaNetworks : PppoeIa Networks hosts different PppoeIa networks that in turn hold the PppoeIa
+// sessions
+type PppoeIaNetworks struct {
+	Networks *util.ConcurrentMap //map[uint32]*PppoeIaRelayVnet
+}
+
+// NewPppoeIaRelayVnet is constructor for a PppoeIa Relay Virtual network
+func NewPppoeIaRelayVnet(outerVlan uint16, innerVlan uint16) *PppoeIaRelayVnet {
+	var drv PppoeIaRelayVnet
+
+	drv.OuterVlan = outerVlan
+	drv.InnerVlan = innerVlan
+	drv.sessions = util.NewConcurrentMap() //make(map[[6]byte]IPppoeIaSession)
+	return &drv
+}
+
+// AddPppoeIaRelayVnet add pppoeia relay vnet
+func (dn *PppoeIaNetworks) AddPppoeIaRelayVnet(outerVlan uint16, innerVlan uint16) *PppoeIaRelayVnet {
+	comboVlan := uint32(outerVlan)<<16 + uint32(innerVlan)
+	if drv, ok := dn.Networks.Get(comboVlan); ok {
+		return drv.(*PppoeIaRelayVnet)
+	}
+	drv := NewPppoeIaRelayVnet(outerVlan, innerVlan)
+	dn.Networks.Set(comboVlan, drv)
+	return drv
+}
+
+// NewPppoeIaNetworks is constructor for PppoeIa network
+func NewPppoeIaNetworks() *PppoeIaNetworks {
+	var dn PppoeIaNetworks
+	dn.Networks = util.NewConcurrentMap() //make(map[uint32]*PppoeIaRelayVnet)
+	return &dn
+}
+
+// AddPppoeIaSession to add pppoeia session
+func (dn *PppoeIaNetworks) AddPppoeIaSession(pkt gopacket.Packet, session IPppoeIaSession) {
+	var key [6]byte
+	ethl := pkt.Layer(layers.LayerTypeEthernet)
+	eth, _ := ethl.(*layers.Ethernet)
+	addr := eth.SrcMAC
+	copy(key[:], addr[0:6])
+	drv := dn.AddPppoeIaRelayVnet(session.GetNniVlans())
+	drv.sessions.Set(key, session)
+}
+
+// DelPppoeIaSession to delete pppoeia session
+func (dn *PppoeIaNetworks) DelPppoeIaSession(pkt gopacket.Packet, session IPppoeIaSession) {
+	var key [6]byte
+	ethl := pkt.Layer(layers.LayerTypeEthernet)
+	eth, _ := ethl.(*layers.Ethernet)
+	addr := eth.SrcMAC
+	if len(addr) != 6 {
+		logger.Errorw(ctx, "Invalid MAC address", log.Fields{"Addr": addr})
+		return
+	}
+	copy(key[:], addr[0:6])
+	drv := dn.AddPppoeIaRelayVnet(session.GetNniVlans())
+	drv.sessions.Remove(key)
+}
+
+// delPppoeIaSessions to delete pppoeia sessions
+func delPppoeIaSessions(addr net.HardwareAddr, outervlan of.VlanType, innervlan of.VlanType) {
+
+	var key [6]byte
+	if addr == nil || !NonZeroMacAddress(addr) {
+		logger.Warnw(ctx, "Invalid MAC address", log.Fields{"Addr": addr})
+		return
+	}
+	copy(key[:], addr[0:6])
+	drv := pppoeIaNws.AddPppoeIaRelayVnet(uint16(outervlan), uint16(innervlan))
+	drv.sessions.Remove(key)
+	logger.Infow(ctx, "PppoeIa Sessions deleted", log.Fields{"MAC": addr})
+}
+
+// GetPppoeIaSession to get pppoeia sessions
+func (dn *PppoeIaNetworks) GetPppoeIaSession(outerVlan uint16, innerVlan uint16, addr net.HardwareAddr) (IPppoeIaSession, error) {
+	var key [6]byte
+	if len(addr) != 6 {
+		logger.Errorw(ctx, "Invalid MAC address", log.Fields{"Addr": addr})
+		return nil, errors.New("Invalid MAC address")
+	}
+	copy(key[:], addr[0:6])
+	drv := dn.AddPppoeIaRelayVnet(outerVlan, innerVlan)
+	logger.Infow(ctx, "Key for PPPoE session", log.Fields{"Key": key})
+	if session, ok := drv.sessions.Get(key); ok {
+		return session.(IPppoeIaSession), nil
+	}
+	return nil, ErrSessionDoNotExist
+}
+
+// GetVnetForNni to get vnet for nni port
+func GetVnetForNni(addr net.HardwareAddr, cvlan of.VlanType, svlan of.VlanType, pbit uint8) (*VoltPortVnet, error) {
+
+	var err error
+	var session IPppoeIaSession
+	logger.Infow(ctx, "Mac Obtained MAC: ", log.Fields{"Addr": addr})
+	if session, err = pppoeIaNws.GetPppoeIaSession(uint16(svlan), uint16(cvlan), addr); err != nil {
+		logger.Errorw(ctx, "PPPoE Session retrieval failed", log.Fields{"Error": err})
+		if err == ErrSessionDoNotExist {
+			logger.Info(ctx, "Finding matching VPV from packet")
+			vpvs, err1 := GetApplication().GetVpvsForDsPkt(cvlan, svlan, addr, pbit)
+			if len(vpvs) == 1 {
+				return vpvs[0], nil
+			}
+			return nil, err1
+		}
+		return nil, err
+	}
+
+	if session != nil {
+		vpv, ok := session.(*VoltPortVnet)
+
+		if ok {
+			logger.Infow(ctx, "Session Exist: VPV found", log.Fields{"VPV": vpv})
+			return vpv, nil
+		}
+	}
+	logger.Error(ctx, "PPPoE Session retrieved of wrong type")
+	return nil, errors.New("The session retrieved of wrong type")
+}
+
+// AddIaOption : Addition of PppoeIa Option 82 which codes circuit-id and remote-id
+// into the packet. This happens as the request is relayed to the
+// PppoeIa servers on the NNI
+func AddIaOption(svc *VoltService, pppoe *layers.PPPoE) {
+
+	//NOTE : both cID and rID should not be empty if this function is called
+	var data []byte
+	cID := svc.GetCircuitID()
+	rID := svc.RemoteID
+
+	if len(cID) != 0 || len(rID) != 0 || svc.isDataRateAttrPresent() {
+		data = append(data, DSLATTRVendorID...)
+	}
+
+	logger.Debugw(ctx, "Vendor Info", log.Fields{"Data": data})
+
+	if len(cID) != 0 {
+		data = append(data, TYPECIRCUITID)
+		data = append(data, byte(len(cID)))
+		data = append(data, cID...)
+	}
+	if len(rID) != 0 {
+		data = append(data, TYPEREMOTEID)
+		data = append(data, byte(len(rID)))
+		data = append(data, rID...)
+	}
+
+	if svc.isDataRateAttrPresent() {
+		minDrUs := util.Uint32ToByte(svc.MinDataRateUs)
+		data = append(data, TYPEMINDATAUS)
+		data = append(data, byte(len(minDrUs)))
+		data = append(data, minDrUs...)
+
+		minDrDs := util.Uint32ToByte(svc.MinDataRateDs)
+		data = append(data, TYPEMINDATADS)
+		data = append(data, byte(len(minDrDs)))
+		data = append(data, minDrDs...)
+
+		maxDrUs := util.Uint32ToByte(svc.MaxDataRateUs)
+		data = append(data, TYPEMAXDATAUS)
+		data = append(data, byte(len(maxDrUs)))
+		data = append(data, maxDrUs...)
+
+		maxDrDs := util.Uint32ToByte(svc.MaxDataRateDs)
+		data = append(data, TYPEMAXDATADS)
+		data = append(data, byte(len(maxDrDs)))
+		data = append(data, maxDrDs...)
+	}
+	option := layers.NewPPPoEOption(layers.PPPoEOptVendorSpecific, data)
+	pppoe.Options = append(pppoe.Options, option)
+}
+
+// DelIaOption for deletion of IA option from the packet received on the NNI interface.
+func DelIaOption(pppoe *layers.PPPoE) {
+	for index, option := range pppoe.Options {
+		if option.Type == layers.PPPoEOptVendorSpecific {
+			pppoe.Options = append(pppoe.Options[0:index], pppoe.Options[index+1:]...)
+			return
+		}
+	}
+}
+
+// ProcessDsPppoeIaPacket : This function processes DS PppoeIa packet received on the NNI port.
+// The services are attached to the access ports. Thus, the PppoeIa
+// session is derived from the list of PppoeIa sessions stored in the
+// common map. The key for retrieval includes the VLAN tags in the
+// the packet and the MAC address of the client.
+func (va *VoltApplication) ProcessDsPppoeIaPacket(device string, port string, pkt gopacket.Packet) {
+
+	// Retrieve the layers to build the outgoing packet. It is not
+	// possible to add/remove layers to the existing packet and thus
+	// the lyayers are extracted to build the outgoing packet
+	eth := pkt.Layer(layers.LayerTypeEthernet).(*layers.Ethernet)
+	pppoe := pkt.Layer(layers.LayerTypePPPoE).(*layers.PPPoE)
+
+	logger.Infow(ctx, "Processing Southbound DS PppoeIa packet", log.Fields{"Port": port, "Type": pppoe.Code})
+
+	// Retrieve the priority and drop eligible flags from the
+	// packet received
+	var priority uint8
+	var dropEligible bool
+	dot1ql := pkt.Layer(layers.LayerTypeDot1Q)
+	if dot1ql != nil {
+		dot1q := dot1ql.(*layers.Dot1Q)
+		priority = dot1q.Priority
+		dropEligible = dot1q.DropEligible
+	}
+
+	pktInnerlan, pktOuterlan := GetVlansFromPacket(pkt)
+	vpv, err := GetVnetForNni(eth.DstMAC, pktInnerlan, pktOuterlan, priority)
+	if err != nil {
+		logger.Errorw(ctx, "VNET couldn't be found for NNI", log.Fields{"Error": err})
+		return
+	}
+
+	// Do not modify pppoe header if vnet's mac_learning type is not PPPoE-IA.
+	if vpv.PppoeIa {
+		// Delete the IA option that may be included in the response
+		DelIaOption(pppoe)
+		if pppoe.Code == layers.PPPoECodePADO {
+			vpv.SetPppoeIaState(PppoeIaStatePADO)
+		} else if pppoe.Code == layers.PPPoECodePADS {
+			vpv.SetPppoeIaState(PppoeIaStatePADS)
+		} else if pppoe.Code == layers.PPPoECodePADT {
+			vpv.SetPppoeIaState(PppoeIaStatePADT)
+		}
+		vpv.WriteToDb()
+	}
+	// Create the outgoing bufer and set the checksum in the packet
+	buff := gopacket.NewSerializeBuffer()
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+
+	cTagType := layers.EthernetTypePPPoEDiscovery
+	eth.EthernetType = layers.EthernetTypeDot1Q
+	priority = vpv.GetRemarkedPriority(priority)
+
+	var pktLayers []gopacket.SerializableLayer
+	pktLayers = append(pktLayers, eth)
+
+	var qVlans []of.VlanType
+	var qVlanLayers []gopacket.SerializableLayer
+
+	if vpv.AllowTransparent {
+		vlanThreshold := 2
+		// In case of ONU_CVLAN or OLT_SVLAN, the DS pkts have single configured vlan
+		// In case of ONU_CVLAN_OLT_SVLAN or OLT_CVLAN_OLT_SVLAN, the DS pkts have 2 configured vlan
+		// Based on that, the no. of vlans should be ignored to get only transparent vlans
+		if vpv.VlanControl == ONUCVlan || vpv.VlanControl == OLTSVlan || vpv.VlanControl == None {
+			vlanThreshold = 1
+		}
+		nxtLayer := layers.EthernetTypeDot1Q
+		if vlans := GetVlans(pkt); len(vlans) > vlanThreshold {
+			qVlans = vlans[vlanThreshold:]
+			cTagType = layers.EthernetTypeDot1Q
+		}
+		for i, qVlan := range qVlans {
+			vlan := uint16(qVlan)
+			if i == (len(qVlans) - 1) {
+				nxtLayer = layers.EthernetTypePPPoEDiscovery
+			}
+			qdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: vlan, DropEligible: dropEligible, Type: nxtLayer}
+			qVlanLayers = append(qVlanLayers, qdot1q)
+		}
+	}
+
+	switch vpv.VlanControl {
+	case ONUCVlanOLTSVlan:
+		cdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: uint16(vpv.CVlan), DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, cdot1q)
+	case ONUCVlan,
+		None:
+		sdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: uint16(vpv.SVlan), DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, sdot1q)
+	case OLTCVlanOLTSVlan,
+		OLTSVlan:
+		udot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: uint16(vpv.UniVlan), DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, udot1q)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+		return
+	}
+
+	pktLayers = append(pktLayers, qVlanLayers...)
+	pktLayers = append(pktLayers, pppoe)
+
+	logger.Debugw(ctx, "Layers Count", log.Fields{"Count": len(pktLayers)})
+	if err := gopacket.SerializeMultiLayers(buff, opts, pktLayers); err != nil {
+		logger.Errorw(ctx, "Packet Serialization Failed", log.Fields{"Reason": err.Error()})
+		return
+	}
+
+	if err := cntlr.GetController().PacketOutReq(device, vpv.Port, port, buff.Bytes(), false); err != nil {
+		logger.Warnw(ctx, "PacketOutReq Failed", log.Fields{"Device": device, "Error": err})
+	}
+}
+
+// ProcessUsPppoeIaPacket : The US PppoeIa packet is identified the PppoeIa OP in the packet. A request is considered upstream
+// and the service associated with the packet is located by the port and VLANs in the packet
+func (va *VoltApplication) ProcessUsPppoeIaPacket(device string, port string, pkt gopacket.Packet) {
+	// We received the packet on an access port and the service for the packet can be
+	// gotten from the port and the packet
+	vpv, svc := va.GetVnetFromPkt(device, port, pkt)
+	if vpv == nil {
+		logger.Errorw(ctx, "VNET couldn't be found from packet", log.Fields{"Device": device, "Port": port})
+		return
+	}
+
+	outport, _ := va.GetNniPort(device)
+	if outport == "" || outport == "0" {
+		logger.Errorw(ctx, "NNI Port not found for device. Dropping Packet", log.Fields{"NNI": outport})
+		return
+	}
+
+	//Add PPPoE session for reference so that the DS pkts can be processed and re-directed
+	pppoeIaNws.AddPppoeIaSession(pkt, vpv)
+
+	// Extract the layers in the packet to prepare the outgoing packet
+	// We use the layers to build the outgoing packet from scratch as
+	// the packet received can't be modified to add/remove layers
+	eth := pkt.Layer(layers.LayerTypeEthernet).(*layers.Ethernet)
+	pppoe := pkt.Layer(layers.LayerTypePPPoE).(*layers.PPPoE)
+	msgType := pppoe.Code
+	logger.Infow(ctx, "Processing Southbound US PppoeIa packet", log.Fields{"Device": device, "Port": port, "Type": pppoe.Code})
+
+	AddIaOption(svc, pppoe)
+
+	// Learn the 8021P values from the packet received
+	var priority uint8
+	dropEligible := false
+	dot1ql := pkt.Layer(layers.LayerTypeDot1Q)
+	if dot1ql != nil {
+		dot1q := dot1ql.(*layers.Dot1Q)
+		priority = dot1q.Priority
+		dropEligible = dot1q.DropEligible
+	}
+
+	if vpv.PppoeIa {
+		//Maintain the session MAC as learnt MAC, since MAC is required for deletion of PPPoE session
+		if msgType == layers.PPPoECodePADI || msgType == layers.PPPoECodePADR {
+			if !util.MacAddrsMatch(vpv.MacAddr, eth.SrcMAC) {
+				expectedPort := va.GetMacInPortMap(eth.SrcMAC)
+				if expectedPort != "" && expectedPort != vpv.Port {
+					logger.Errorw(ctx, "mac-learnt-from-different-port-ignoring-pppoe-message",
+						log.Fields{"MsgType": msgType, "ExpectedPort": expectedPort, "ReceivedPort": vpv.Port, "LearntMacAdrr": vpv.MacAddr, "NewMacAdrr": eth.SrcMAC.String()})
+					return
+				}
+			}
+			vpv.SetMacAddr(eth.SrcMAC)
+		}
+
+		if pppoe.Code == layers.PPPoECodePADI {
+			vpv.SetPppoeIaState(PppoeIaStatePADI)
+		} else if pppoe.Code == layers.PPPoECodePADR {
+			vpv.SetPppoeIaState(PppoeIaStatePADR)
+		}
+		vpv.WriteToDb()
+	}
+
+	buff := gopacket.NewSerializeBuffer()
+	opts := gopacket.SerializeOptions{
+		FixLengths:       true,
+		ComputeChecksums: true,
+	}
+
+	cTagType := layers.EthernetTypePPPoEDiscovery
+	outerVlan, innerVlan := vpv.GetNniVlans()
+	logger.Debugw(ctx, "Vnet Vlans", log.Fields{"Svlan": outerVlan, "Cvlan": innerVlan})
+	eth.EthernetType = vpv.SVlanTpid
+
+	var pktLayers []gopacket.SerializableLayer
+	pktLayers = append(pktLayers, eth)
+
+	var qVlans []of.VlanType
+	var qVlanLayers []gopacket.SerializableLayer
+
+	if vpv.AllowTransparent {
+		nxtLayer := layers.EthernetTypeDot1Q
+		if vlans := GetVlans(pkt); len(vlans) > 1 {
+			qVlans = vlans[1:]
+			logger.Debugw(ctx, "Q Vlans", log.Fields{"Vlan List": qVlans})
+			cTagType = layers.EthernetTypeDot1Q
+		}
+		for i, qVlan := range qVlans {
+			vlan := uint16(qVlan)
+			if i == (len(qVlans) - 1) {
+				nxtLayer = layers.EthernetTypePPPoEDiscovery
+			}
+			qdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: vlan, DropEligible: dropEligible, Type: nxtLayer}
+			qVlanLayers = append(qVlanLayers, qdot1q)
+		}
+	}
+
+	switch vpv.VlanControl {
+	case ONUCVlanOLTSVlan,
+		OLTCVlanOLTSVlan:
+		sdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: outerVlan, DropEligible: dropEligible, Type: layers.EthernetTypeDot1Q}
+		pktLayers = append(pktLayers, sdot1q)
+		cdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: innerVlan, DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, cdot1q)
+	case ONUCVlan,
+		OLTSVlan,
+		None:
+		cdot1q := &layers.Dot1Q{Priority: priority, VLANIdentifier: outerVlan, DropEligible: dropEligible, Type: cTagType}
+		pktLayers = append(pktLayers, cdot1q)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+		return
+	}
+
+	pktLayers = append(pktLayers, qVlanLayers...)
+	pktLayers = append(pktLayers, pppoe)
+	logger.Debugw(ctx, "Layers Count", log.Fields{"Count": len(pktLayers)})
+	if err := gopacket.SerializeMultiLayers(buff, opts, pktLayers); err != nil {
+		return
+	}
+
+	// Now the packet constructed is output towards the switch to be emitted on
+	// the NNI port
+	if err := cntlr.GetController().PacketOutReq(device, outport, port, buff.Bytes(), false); err != nil {
+		logger.Warnw(ctx, "PacketOutReq Failed", log.Fields{"Device": device, "Error": err})
+	}
+
+}
+
+// ProcessPPPoEIaPacket to process Pppoeia packet
+func (va *VoltApplication) ProcessPPPoEIaPacket(device string, port string, pkt gopacket.Packet) {
+	// Make some error checks before proceeding
+	pppoel := pkt.Layer(layers.LayerTypePPPoE)
+	if pppoel == nil {
+		return
+	}
+	_, ok := pppoel.(*layers.PPPoE)
+	if !ok {
+		return
+	}
+
+	// Let us assess the direction of the packet. We can do so by the port
+	// which is more reliable or do by the PPPoE code which is less reliable
+	isUs := true
+	if nni, _ := GetApplication().GetNniPort(device); nni == port {
+		isUs = false
+	}
+
+	// This is a valid PPPoE packet and can be processed
+	if isUs {
+		// This is treated as an upstream packet in the VOLT application
+		// as VOLT serves access subscribers who use DHCP to acquire IP
+		// address and these packets go upstream to the network
+		va.ProcessUsPppoeIaPacket(device, port, pkt)
+	} else {
+		// This is a downstream packet
+		va.ProcessDsPppoeIaPacket(device, port, pkt)
+	}
+}
+
+// ProcessPPPoEPacket to process Pppoe packet
+func (va *VoltApplication) ProcessPPPoEPacket(device string, port string, pkt gopacket.Packet) {
+	dpt := NewPppoeIaPacketTask(pkt, device, port)
+	va.pppoeTasks.AddTask(dpt)
+}
+
+// pppoeIaNws : The DHCP relay application is maintained within the structures below
+var pppoeIaNws *PppoeIaNetworks
+
+func init() {
+	pppoeIaNws = NewPppoeIaNetworks()
+	RegisterPacketHandler(PPPOE, ProcessPPPoEPacket)
+}
+
+// ProcessPPPoEPacket : CallBack function registered with application to handle PPPoE packetIn
+func ProcessPPPoEPacket(device string, port string, pkt gopacket.Packet) {
+	GetApplication().ProcessPPPoEPacket(device, port, pkt)
+}
+
+// PppoeIaPacketTask : Task to add or delete flows of a service
+type PppoeIaPacketTask struct {
+	taskID    uint8
+	ctx       context.Context
+	pkt       gopacket.Packet
+	device    string
+	port      string
+	timestamp string
+}
+
+// NewPppoeIaPacketTask constructor for PppoeIaPacketTask
+func NewPppoeIaPacketTask(pkt gopacket.Packet, dev string, port string) *PppoeIaPacketTask {
+	var dpt PppoeIaPacketTask
+	dpt.pkt = pkt
+	dpt.device = dev
+	dpt.port = port
+	dpt.timestamp = (time.Now()).Format(time.RFC3339Nano)
+	return &dpt
+}
+
+// Name to return name for PppoeIaPacketTask
+func (dpt *PppoeIaPacketTask) Name() string {
+	return "DHCP Packet Task"
+}
+
+// TaskID to return task id for PppoeIaPacketTask
+func (dpt *PppoeIaPacketTask) TaskID() uint8 {
+	return dpt.taskID
+}
+
+// Timestamp to return timestamp for PppoeIaPacketTask
+func (dpt *PppoeIaPacketTask) Timestamp() string {
+	return dpt.timestamp
+}
+
+// Stop to stop the PppoeIaPacketTask
+func (dpt *PppoeIaPacketTask) Stop() {
+}
+
+// Start to start PppoeIaPacketTask
+func (dpt *PppoeIaPacketTask) Start(ctx context.Context, taskID uint8) error {
+	dpt.taskID = taskID
+	dpt.ctx = ctx
+	GetApplication().ProcessPPPoEIaPacket(dpt.device, dpt.port, dpt.pkt)
+	return nil
+}
diff --git a/internal/pkg/application/service.go b/internal/pkg/application/service.go
new file mode 100644
index 0000000..e90b948
--- /dev/null
+++ b/internal/pkg/application/service.go
@@ -0,0 +1,1978 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package application
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"net"
+	"reflect"
+	infraerrorCodes "voltha-go-controller/internal/pkg/errorcodes"
+	"sort"
+	"strconv"
+	"strings"
+	"sync"
+
+	"github.com/google/gopacket/layers"
+
+	"voltha-go-controller/internal/pkg/controller"
+	cntlr "voltha-go-controller/internal/pkg/controller"
+	"voltha-go-controller/database"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/util"
+	errorCodes "voltha-go-controller/internal/pkg/errorcodes"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	// DSLAttrEnabled constant
+	DSLAttrEnabled string = "ENABLED"
+)
+
+// VoltServiceCfg structure
+// Name -	Uniquely identifies a service across the entire application
+// UniVlan -	The VLAN of the packets entering the UNI of ONU
+// CVlan -	The VLAN to transalate to/from on the PON link
+// SVlan -	The outer VLAN to be used on the NNI of OLT.
+//       -	In general, 4096 is used as NO VLAN for all the above
+// SVlanTpid - SVlan Tag Protocl Identifier
+// Pbits -      Each bit of uint8 represents one p-bit. MSB is pbit 7
+// DhcpRelay -	Whether it is turned on/off
+// CircuitId -	The circuit id to be used with DHCP relay. Unused otherwise
+// RemoveId - 	Same as above
+// Port -	The access port for the service. Each service has a single access
+//		port. The converse is not always true
+// MacLearning - If MAC learning is turned on, the MAC address learned from the
+//		the service activation is used in programming flows
+// MacAddress -	The MAC hardware address learnt on the UNI interface
+// MacAddresses - Not yet implemented. To be used to learn more MAC addresses
+type VoltServiceCfg struct {
+	Name                       string
+	UniVlan                    of.VlanType
+	CVlan                      of.VlanType
+	SVlan                      of.VlanType
+	SVlanTpid                  layers.EthernetType
+	MacAddr                    net.HardwareAddr
+	Pbits                      []of.PbitType
+	DsRemarkPbitsMap           map[int]int // Ex: Remark case {0:0,1:0} and No-remark case {1:1}
+	TechProfileID              uint16
+	CircuitID                  string
+	RemoteID                   []byte
+	Port                       string
+	PonPort                    uint32
+	MacLearning                MacLearningType
+	IsOption82Disabled         bool
+	IgmpEnabled                bool
+	McastService               bool
+	ONTEtherTypeClassification int
+	VlanControl                VlanControl
+	UsMeterProfile             string
+	DsMeterProfile             string
+	AggDsMeterProfile          string
+	VnetID                     string
+	MvlanProfileName           string
+	RemoteIDType               string
+	SchedID                    int
+	AllowTransparent           bool
+	EnableMulticastKPI         bool
+	DataRateAttr               string
+	MinDataRateUs              uint32
+	MinDataRateDs              uint32
+	MaxDataRateUs              uint32
+	MaxDataRateDs              uint32
+
+	Trigger ServiceTrigger
+}
+
+// VoltServiceOper structure
+type VoltServiceOper struct {
+	//MacLearning  bool
+	//MacAddr      net.HardwareAddr
+	Device   string
+	Ipv4Addr net.IP
+	Ipv6Addr net.IP
+
+	UsMeterID    uint32
+	DsMeterID    uint32
+	AggDsMeterID uint32
+
+	//Multiservice-Fix
+	UsHSIAFlowsApplied bool
+	DsHSIAFlowsApplied bool
+	UsDhcpFlowsApplied bool
+	DsDhcpFlowsApplied bool
+	IgmpFlowsApplied   bool
+	Icmpv6FlowsApplied bool
+
+	ServiceLock      sync.RWMutex `json:"-"`
+	PendingFlows     map[string]bool
+	AssociatedFlows  map[string]bool
+	DeleteInProgress bool
+	ForceDelete      bool
+	BwAvailInfo      string
+
+	UpdateInProgress bool
+	Metadata         interface{}
+}
+
+// VoltService structure
+type VoltService struct {
+	VoltServiceCfg
+	VoltServiceOper
+	Version string
+}
+
+//ServiceTrigger - Service activation trigger
+type ServiceTrigger int
+
+const (
+	//NBActivate - Service added due to NB Action
+	NBActivate ServiceTrigger = 0
+	//ServiceVlanUpdate - Service added due to Svlan Update
+	ServiceVlanUpdate ServiceTrigger = 1
+)
+
+// AppMutexes structure
+type AppMutexes struct {
+	ServiceDataMutex sync.Mutex `json:"-"`
+	VnetMutex        sync.Mutex `json:"-"`
+}
+
+//MigrateServiceMetadata - migrate services request metadata
+type MigrateServiceMetadata struct {
+	NewVnetID string
+	RequestID string
+}
+
+// AppMutex variable
+var AppMutex AppMutexes
+
+// NewVoltService for constructor for volt service
+func NewVoltService(cfg *VoltServiceCfg) *VoltService {
+	var vs VoltService
+	vs.VoltServiceCfg = *cfg
+	vs.UsHSIAFlowsApplied = false
+	vs.DsHSIAFlowsApplied = false
+	vs.DeleteInProgress = false
+	//vs.MacAddr, _ = net.ParseMAC("00:00:00:00:00:00")
+
+	vs.MacAddr = cfg.MacAddr
+	vs.Ipv4Addr = net.ParseIP("0.0.0.0")
+	vs.Ipv6Addr = net.ParseIP("::")
+	vs.PendingFlows = make(map[string]bool)
+	vs.AssociatedFlows = make(map[string]bool)
+	return &vs
+}
+
+// WriteToDb commit a service to the DB if service delete is not in-progress
+func (vs *VoltService) WriteToDb() {
+
+	vs.ServiceLock.RLock()
+	defer vs.ServiceLock.RUnlock()
+
+	if vs.DeleteInProgress {
+		logger.Warnw(ctx, "Skipping Redis Update for Service, Service delete in progress", log.Fields{"Service": vs.Name})
+		return
+	}
+	vs.ForceWriteToDb()
+}
+
+//ForceWriteToDb force commit a service to the DB
+func (vs *VoltService) ForceWriteToDb() {
+	b, err := json.Marshal(vs)
+
+	if err != nil {
+		logger.Errorw(ctx, "Json Marshal Failed for Service", log.Fields{"Service": vs.Name})
+		return
+	}
+	if err1 := db.PutService(vs.Name, string(b)); err1 != nil {
+		logger.Errorw(ctx, "DB write oper failed for Service", log.Fields{"Service": vs.Name})
+	}
+}
+
+// isDataRateAttrPresent to check if data attribute is present
+func (vs *VoltService) isDataRateAttrPresent() bool {
+	return vs.DataRateAttr == DSLAttrEnabled
+}
+
+// DelFromDb delete a service from DB
+func (vs *VoltService) DelFromDb() {
+	logger.Debugw(ctx, "Deleting Service from DB", log.Fields{"Name": vs.Name})
+	//TODO - Need to understand and delete the second call
+	//Calling twice has worked though don't know why
+	_ = db.DelService(vs.Name)
+	_ = db.DelService(vs.Name)
+}
+
+// MatchesVlans find the service that matches the VLANs. In this case it is
+// purely based on CVLAN. The CVLAN can sufficiently be used to
+// match a service
+func (vs *VoltService) MatchesVlans(vlans []of.VlanType) bool {
+	if len(vlans) != 1 {
+		return false
+	}
+
+	if vlans[0] == vs.CVlan {
+		return true
+	}
+	return false
+}
+
+// MatchesPbits allows matching a service to a pbit. This is used
+// to search for a service matching the pbits, typically to identify
+// attributes for other flows such as DHCP, IGMP, etc.
+func (vs *VoltService) MatchesPbits(pbits []of.PbitType) bool {
+	for _, pbit := range pbits {
+		for _, pb := range vs.Pbits {
+			if pb == pbit {
+				return true
+			}
+		}
+	}
+	return false
+}
+
+// IsPbitExist allows matching a service to a pbit. This is used
+// to search for a service matching the pbit
+func (vs *VoltService) IsPbitExist(pbit of.PbitType) bool {
+	for _, pb := range vs.Pbits {
+		if pb == pbit {
+			return true
+		}
+	}
+	return false
+}
+
+// AddHsiaFlows - Adds US & DS HSIA Flows for the service
+func (vs *VoltService) AddHsiaFlows() {
+	if err := vs.AddUsHsiaFlows(); err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vs.triggerServiceFailureInd(statusCode, statusMessage)
+	}
+	if err := vs.AddDsHsiaFlows(); err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vs.triggerServiceFailureInd(statusCode, statusMessage)
+	}
+}
+
+//DelHsiaFlows - Deletes US & DS HSIA Flows for the service
+func (vs *VoltService) DelHsiaFlows() {
+	if err := vs.DelUsHsiaFlows(); err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vs.triggerServiceFailureInd(statusCode, statusMessage)
+	}
+
+	if err := vs.DelDsHsiaFlows(); err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vs.triggerServiceFailureInd(statusCode, statusMessage)
+	}
+}
+
+// AddUsHsiaFlows - Add US HSIA Flows for the service
+func (vs *VoltService) AddUsHsiaFlows() error {
+
+	if vs.DeleteInProgress || vs.UpdateInProgress {
+		logger.Errorw(ctx, "Ignoring US HSIA Flow Push, Service deleteion In-Progress", log.Fields{"Device": vs.Device, "Service": vs.Name})
+		return nil
+	}
+
+	va := GetApplication()
+	logger.Infow(ctx, "Configuring US HSIA Service Flows", log.Fields{"ServiceName": vs.Name})
+	if !vs.UsHSIAFlowsApplied || vgcRebooted {
+		device, err := va.GetDeviceFromPort(vs.Port)
+		if err != nil {
+			logger.Errorw(ctx, "Error Getting Device", log.Fields{"Reason": err.Error()})
+			return errorCodes.ErrDeviceNotFound
+		} else if device.State != controller.DeviceStateUP {
+			logger.Warnw(ctx, "Device state Down. Ignoring US HSIA Flow Push", log.Fields{"Service": vs.Name, "Port": vs.Port})
+			return nil
+		}
+
+		vs.Device = device.Name
+		va.AddMeterToDevice(vs.Port, device.Name, vs.UsMeterID, 0)
+		va.AddMeterToDevice(vs.Port, device.Name, vs.DsMeterID, vs.AggDsMeterID)
+
+		logger.Infow(ctx, "Adding HSIA flows", log.Fields{"Name": vs.Name})
+		pBits := vs.Pbits
+
+		//If no pbits configured for service, hence add PbitNone for flows
+		if len(vs.Pbits) == 0 {
+			pBits = append(pBits, PbitMatchNone)
+		}
+		for _, pbits := range pBits {
+			usflows, err := vs.BuildUsHsiaFlows(pbits)
+			if err != nil {
+				logger.Errorw(ctx, "Error Building HSIA US flows", log.Fields{"Reason": err.Error()})
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vs.triggerServiceFailureInd(statusCode, statusMessage)
+				continue
+			}
+			usflows.MigrateCookie = vgcRebooted
+			if err := vs.AddFlows(device, usflows); err != nil {
+				logger.Errorw(ctx, "Error adding HSIA US flows", log.Fields{"Reason": err.Error()})
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vs.triggerServiceFailureInd(statusCode, statusMessage)
+			}
+		}
+		vs.UsHSIAFlowsApplied = true
+		logger.Infow(ctx, "Pushed US HSIA Service Flows", log.Fields{"ServiceName": vs.Name})
+	}
+	vs.WriteToDb()
+	return nil
+}
+
+// AddDsHsiaFlows - Add DS HSIA Flows for the service
+func (vs *VoltService) AddDsHsiaFlows() error {
+	if vs.DeleteInProgress {
+		logger.Errorw(ctx, "Ignoring DS HSIA Flow Push, Service deleteion In-Progress", log.Fields{"Device": vs.Device, "Service": vs.Name})
+		return nil
+	}
+
+	va := GetApplication()
+	logger.Infow(ctx, "Configuring DS HSIA Service Flows", log.Fields{"ServiceName": vs.Name})
+	if !vs.DsHSIAFlowsApplied || vgcRebooted {
+		device, err := va.GetDeviceFromPort(vs.Port)
+		if err != nil {
+			logger.Errorw(ctx, "Error Getting Device", log.Fields{"Reason": err.Error()})
+			return errorCodes.ErrDeviceNotFound
+		} else if device.State != controller.DeviceStateUP {
+			logger.Warnw(ctx, "Device state Down. Ignoring DS HSIA Flow Push", log.Fields{"Service": vs.Name, "Port": vs.Port})
+			return nil
+		}
+
+		va.AddMeterToDevice(vs.Port, device.Name, vs.DsMeterID, vs.AggDsMeterID)
+		logger.Infow(ctx, "Adding HSIA flows", log.Fields{"Name": vs.Name})
+
+		//If no pbits configured for service, hence add PbitNone for flows
+		if len(vs.DsRemarkPbitsMap) == 0 {
+			dsflows, err := vs.BuildDsHsiaFlows(of.PbitType(of.PbitMatchNone))
+			if err != nil {
+				logger.Errorw(ctx, "Error Building HSIA DS flows", log.Fields{"Reason": err.Error()})
+				return err
+			}
+			dsflows.MigrateCookie = vgcRebooted
+			if err = vs.AddFlows(device, dsflows); err != nil {
+				logger.Errorw(ctx, "Failed to add HSIA DS flows", log.Fields{"Reason": err})
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vs.triggerServiceFailureInd(statusCode, statusMessage)
+			}
+		} else {
+			// if all 8 p-bits are to be remarked to one-pbit, configure all-to-one remarking flow
+			if _, ok := vs.DsRemarkPbitsMap[int(of.PbitMatchAll)]; ok {
+				dsflows, err := vs.BuildDsHsiaFlows(of.PbitType(of.PbitMatchAll))
+				if err != nil {
+					logger.Errorw(ctx, "Error Building HSIA DS flows", log.Fields{"Reason": err.Error()})
+					return err
+				}
+				logger.Debug(ctx, "Add-one-match-all-pbit-flow")
+				dsflows.MigrateCookie = vgcRebooted
+				if err := vs.AddFlows(device, dsflows); err != nil {
+					logger.Errorw(ctx, "Failed to add HSIA DS flows", log.Fields{"Reason": err})
+					statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+					vs.triggerServiceFailureInd(statusCode, statusMessage)
+				}
+			} else {
+				for matchPbit := range vs.DsRemarkPbitsMap {
+					dsflows, err := vs.BuildDsHsiaFlows(of.PbitType(matchPbit))
+					if err != nil {
+						logger.Errorw(ctx, "Error Building HSIA DS flows", log.Fields{"Reason": err.Error()})
+						statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+						vs.triggerServiceFailureInd(statusCode, statusMessage)
+						continue
+					}
+					dsflows.MigrateCookie = vgcRebooted
+					if err := vs.AddFlows(device, dsflows); err != nil {
+						logger.Errorw(ctx, "Failed to Add HSIA DS flows", log.Fields{"Reason": err})
+						statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+						vs.triggerServiceFailureInd(statusCode, statusMessage)
+					}
+				}
+			}
+		}
+		vs.DsHSIAFlowsApplied = true
+		logger.Infow(ctx, "Pushed DS HSIA Service Flows", log.Fields{"ServiceName": vs.Name})
+	}
+	vs.WriteToDb()
+	return nil
+}
+
+// DelUsHsiaFlows - Deletes US HSIA Flows for the service
+func (vs *VoltService) DelUsHsiaFlows() error {
+
+	logger.Infow(ctx, "Removing US HSIA Services", log.Fields{"Services": vs.Name})
+	if vs.UsHSIAFlowsApplied || vgcRebooted {
+		device, err := GetApplication().GetDeviceFromPort(vs.Port)
+		if err != nil {
+			logger.Errorw(ctx, "Error Getting Device", log.Fields{"Reason": err.Error()})
+			return errorCodes.ErrDeviceNotFound
+		}
+
+		logger.Infow(ctx, "Removing HSIA flows", log.Fields{"Name": vs.Name})
+		pBits := vs.Pbits
+
+		//If no pbits configured for service, hence add PbitNone for flows
+		if len(vs.Pbits) == 0 {
+			pBits = append(pBits, PbitMatchNone)
+		}
+		for _, pbits := range pBits {
+			usflows, err := vs.BuildUsHsiaFlows(pbits)
+			if err != nil {
+				logger.Errorw(ctx, "Error Building HSIA US flows", log.Fields{"Reason": err.Error()})
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vs.triggerServiceFailureInd(statusCode, statusMessage)
+				continue
+			}
+			usflows.MigrateCookie = vgcRebooted
+			if err = vs.DelFlows(device, usflows); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vs.triggerServiceFailureInd(statusCode, statusMessage)
+			}
+		}
+		vs.UsHSIAFlowsApplied = false
+	}
+	vs.WriteToDb()
+	return nil
+}
+
+// DelDsHsiaFlows - Deletes DS HSIA Flows for the service
+func (vs *VoltService) DelDsHsiaFlows() error {
+
+	logger.Infow(ctx, "Removing DS HSIA Services", log.Fields{"Services": vs.Name})
+	if vs.DsHSIAFlowsApplied || vgcRebooted {
+		device, err := GetApplication().GetDeviceFromPort(vs.Port)
+		if err != nil {
+			logger.Errorw(ctx, "Error Getting Device", log.Fields{"Reason": err.Error()})
+			return errorCodes.ErrDeviceNotFound
+		}
+
+		logger.Infow(ctx, "Removing HSIA flows", log.Fields{"Name": vs.Name})
+		var matchPbit int
+		//If no pbits configured for service, hence add PbitNone for flows
+		if len(vs.DsRemarkPbitsMap) == 0 {
+			dsflows, err := vs.BuildDsHsiaFlows(of.PbitType(PbitMatchNone))
+			if err != nil {
+				logger.Errorw(ctx, "Error Building HSIA DS flows", log.Fields{"Reason": err.Error()})
+				return err
+			}
+			dsflows.MigrateCookie = vgcRebooted
+			if err = vs.DelFlows(device, dsflows); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vs.triggerServiceFailureInd(statusCode, statusMessage)
+			}
+		} else if _, ok := vs.DsRemarkPbitsMap[int(PbitMatchAll)]; ok {
+			dsflows, err := vs.BuildDsHsiaFlows(of.PbitType(int(PbitMatchAll)))
+			if err != nil {
+				logger.Errorw(ctx, "Error Building HSIA DS flows", log.Fields{"Reason": err.Error()})
+				return err
+			}
+			dsflows.MigrateCookie = vgcRebooted
+			if err = vs.DelFlows(device, dsflows); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vs.triggerServiceFailureInd(statusCode, statusMessage)
+			}
+		} else {
+			for matchPbit = range vs.DsRemarkPbitsMap {
+				dsflows, err := vs.BuildDsHsiaFlows(of.PbitType(matchPbit))
+				if err != nil {
+					logger.Errorw(ctx, "Error Building HSIA DS flows", log.Fields{"Reason": err.Error()})
+					statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+					vs.triggerServiceFailureInd(statusCode, statusMessage)
+					continue
+				}
+				dsflows.MigrateCookie = vgcRebooted
+				if err = vs.DelFlows(device, dsflows); err != nil {
+					statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+					vs.triggerServiceFailureInd(statusCode, statusMessage)
+				}
+			}
+		}
+		vs.DsHSIAFlowsApplied = false
+	}
+	logger.Infow(ctx, "Deleted HSIA DS flows from DB successfuly", log.Fields{"ServiceName": vs.Name})
+	// Post HSIA configuration success indication on message bus
+	vs.WriteToDb()
+	return nil
+}
+
+// BuildDsHsiaFlows build the DS HSIA flows
+// Called for add/delete HSIA flows
+func (vs *VoltService) BuildDsHsiaFlows(pbits of.PbitType) (*of.VoltFlow, error) {
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+
+	// Get the out and in ports for the flows
+	device, err := GetApplication().GetDeviceFromPort(vs.Port)
+	if err != nil {
+		return nil, errorCodes.ErrDeviceNotFound
+	}
+	inport, _ := GetApplication().GetPortID(device.NniPort)
+	outport, _ := GetApplication().GetPortID(vs.Port)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = outport
+	flow.PortName = vs.Port
+	allowTransparent := 0
+	if vs.AllowTransparent {
+		allowTransparent = 1
+	}
+
+	// initialized actnPbit to 0 for cookie genration backward compatibility.
+	var actnPbit of.PbitType
+	remarkPbit, remarkExists := vs.DsRemarkPbitsMap[int(pbits)]
+
+	generateDSCookie := func(vlan of.VlanType, valToShift uint64) uint64 {
+		//| 12-bit cvlan/UniVlan | 4 bits action pbit | <32-bits uniport>| 16-bits HSIA mask OR flow mask OR pbit |
+		cookie := uint64(vlan)<<52 + uint64(actnPbit)<<48 + uint64(outport)<<16 | of.HsiaFlowMask
+		cookie = cookie | of.DsFlowMask
+		cookie = cookie + (valToShift << 4) + uint64(pbits)
+		return cookie
+	}
+
+	l2ProtoValue, err := GetMetadataForL2Protocol(vs.SVlanTpid)
+	if err != nil {
+		logger.Errorw(ctx, "DS HSIA flow push failed: Invalid SvlanTpid", log.Fields{"SvlanTpid": vs.SVlanTpid, "Service": vs.Name})
+		return nil, err
+	}
+
+	// Add Table-0 flow that deals with the outer VLAN in pOLT
+	{
+		subflow1 := of.NewVoltSubFlow()
+		subflow1.SetTableID(0)
+		subflow1.SetGoToTable(1)
+		subflow1.SetInPort(inport)
+
+		if pbits != PbitMatchNone {
+			subflow1.SetMatchPbit(pbits)
+		}
+
+		if remarkExists && (of.PbitType(remarkPbit) != pbits) {
+			subflow1.SetPcp(of.PbitType(remarkPbit))
+			// match & action pbits are different, set remark-pbit action
+			actnPbit = of.PbitType(remarkPbit)
+			// mask remark p-bit to 4bits
+			actnPbit = actnPbit & 0x0F
+		}
+
+		if err := vs.setDSMatchActionVlanT0(subflow1); err != nil {
+			return nil, err
+		}
+		logger.Info(ctx, "HSIA DS flows MAC Learning & MAC", log.Fields{"ML": vs.MacLearning, "Mac": vs.MacAddr})
+		if NonZeroMacAddress(vs.MacAddr) {
+			subflow1.SetMatchDstMac(vs.MacAddr)
+		}
+		subflow1.Priority = of.HsiaFlowPriority
+		subflow1.SetMeterID(vs.DsMeterID)
+
+		/* WriteMetaData 8 Byte(uint64) usage:
+		| Byte8    | Byte7    | Byte6 | Byte5  | Byte4  | Byte3   | Byte2  | Byte1 |
+		| reserved | reserved | TpID  | TpID   | uinID  | uniID   | uniID  | uniID | */
+		metadata := uint64(vs.CVlan)<<48 + uint64(vs.TechProfileID)<<32 + uint64(outport)
+		subflow1.SetWriteMetadata(metadata)
+
+		/* TableMetaData 8 Byte(uint64) Voltha usage:  (Considering MSB bit as 63rd bit and LSB bit as 0th bit)
+		|               Byte8                   | Byte7    | Byte6    |      Byte5       |  Byte4    | Byte3    | Byte2    | Byte1   |
+		| 0000     |    00     |    0     |  0  | 00000000 | 00000000 |  0000   0000     |  00000000 | 00000000 | 00000000 | 00000000|
+		| reserved | svlanTpID |  Buff us |  AT | schedID  | schedID  | onteth  vlanCtrl |  unitag   | unitag   | ctag     | ctag    |  */
+
+		//TODO-COMM:
+		/* TableMetaData 8 Byte(uint64) Community usage:  (Considering MSB bit as 63rd bit and LSB bit as 0th bit)
+		|               Byte8                   | Byte7    | Byte6    |      Byte5       |  Byte4    | Byte3    | Byte2    | Byte1   |
+		| 0000     |    00     |    0     |  0  | 00000000 | 00000000 |  0000   0000     |  00000000 | 00000000 | 00000000 | 00000000|
+		| reserved | svlanTpID |  Buff us |  AT | schedID  | schedID  | onteth  vlanCtrl |   ctag    |  ctag    |  ctag    | ctag    |  */
+
+		metadata = uint64(l2ProtoValue)<<58 | uint64(allowTransparent)<<56 | uint64(vs.SchedID)<<40 | uint64(vs.ONTEtherTypeClassification)<<36 | uint64(vs.VlanControl)<<32 | uint64(vs.CVlan)
+		subflow1.SetTableMetadata(metadata)
+		// TODO - We are using cookie as key and must come up with better cookie
+		// allocation algorithm
+		/**
+		 * Cokies may clash when -
+		 * on same uni-port we have two sub-service
+		 * 1. U=10, C=100, S=310, p-bit=4 - VLAN_Control = OLT_CVLAN_OLT_SVLAN
+		 * 2. U=10, C=10,  S=320, p-bit=4 - VLAN_control = ONU_CVLAN_ONU_SVLAN
+		 * However, this p-bit re-use will not be allowed by sub-mgr.
+		 */
+		if vs.VlanControl == OLTCVlanOLTSVlan {
+			/**
+			 * The new cookie generation is only for OLT_CVLAN_OLT_SVLAN case (TEF residential case) within a UNI.
+			 * After vgc upgrade, if we have to deactivate an already existing TEF residential service, then we have to
+			 * use old cookie.
+			 */
+			subflow1.Cookie = generateDSCookie(vs.UniVlan, 0)
+			if vgcRebooted {
+				subflow1.OldCookie = generateDSCookie(vs.CVlan, 0)
+			}
+		} else {
+			// In case of Olt_Svlan , CVLAN=UNIVLAN so cookie can be constructed with CVLAN as well
+			subflow1.Cookie = generateDSCookie(vs.CVlan, 0)
+		}
+
+		flow.SubFlows[subflow1.Cookie] = subflow1
+		logger.Infow(ctx, "Building downstream HSIA flow for T0", log.Fields{"cookie": subflow1.Cookie,
+			"subflow": subflow1})
+	}
+
+	//Add Table-1 flow that deals with inner VLAN at the ONU
+	{
+		subflow2 := of.NewVoltSubFlow()
+		subflow2.SetTableID(1)
+		subflow2.SetInPort(inport)
+		if NonZeroMacAddress(vs.MacAddr) {
+			subflow2.SetMatchDstMac(vs.MacAddr)
+		}
+
+		if err := vs.setDSMatchActionVlanT1(subflow2); err != nil {
+			return nil, err
+		}
+		if pbits != PbitMatchNone {
+			subflow2.SetMatchPbit(pbits)
+		}
+
+		if remarkExists && (of.PbitType(remarkPbit) != pbits) {
+			subflow2.SetPcp(of.PbitType(remarkPbit))
+		}
+
+		subflow2.SetOutPort(outport)
+		subflow2.SetMeterID(vs.DsMeterID)
+
+		// refer Table-0 flow generation for byte information
+		metadata := uint64(vs.CVlan)<<48 + uint64(vs.TechProfileID)<<32 + uint64(outport)
+		subflow2.SetWriteMetadata(metadata)
+
+		// Table-1 and inport is NNI: It is a DS flow for ONU, add uniport in metadata to make it unique
+		if util.IsNniPort(inport) {
+			metadata = uint64(outport)
+		} else {
+			// refer Table-0 flow generation for byte information
+			metadata = uint64(l2ProtoValue)<<58 | uint64(allowTransparent)<<56 | uint64(vs.SchedID)<<40 | uint64(vs.ONTEtherTypeClassification)<<36 | uint64(vs.VlanControl)<<32 | uint64(vs.CVlan)
+		}
+		subflow2.SetTableMetadata(metadata)
+		// Setting of Cookie - TODO - Improve the allocation algorithm
+		if vs.VlanControl == OLTCVlanOLTSVlan {
+			/**
+			 * The new cookie generation is only for OLT_CVLAN_OLT_SVLAN case (TEF residential case) within a UNI.
+			 * After vgc upgrade, if we have to deactivate an already existing TEF residential service, then we have to
+			 * use old cookie.
+			 */
+			subflow2.Cookie = generateDSCookie(vs.UniVlan, 1)
+			if vgcRebooted {
+				subflow2.OldCookie = generateDSCookie(vs.CVlan, 1)
+			}
+		} else {
+			// In case of Olt_Svlan , CVLAN=UNIVLAN so cookie can be constructed with CVLAN as well
+			subflow2.Cookie = generateDSCookie(vs.CVlan, 1)
+		}
+
+		subflow2.Priority = of.HsiaFlowPriority
+		flow.SubFlows[subflow2.Cookie] = subflow2
+		logger.Infow(ctx, "Building downstream HSIA flow for T1", log.Fields{"cookie": subflow2.Cookie,
+			"subflow": subflow2})
+	}
+
+	return flow, nil
+}
+
+// BuildUsHsiaFlows build the US HSIA flows
+// Called for add/delete HSIA flows
+func (vs *VoltService) BuildUsHsiaFlows(pbits of.PbitType) (*of.VoltFlow, error) {
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+
+	// Get the out and in ports for the flows
+	device, err := GetApplication().GetDeviceFromPort(vs.Port)
+	if err != nil {
+		return nil, errorCodes.ErrDeviceNotFound
+	}
+	outport, _ := GetApplication().GetPortID(device.NniPort)
+	inport, _ := GetApplication().GetPortID(vs.Port)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = inport
+	flow.PortName = vs.Port
+	allowTransparent := 0
+	reqBwInfo := 0
+	if vs.AllowTransparent {
+		allowTransparent = 1
+	}
+	if vs.BwAvailInfo == "" {
+		reqBwInfo = 1
+	}
+
+	// Add Table-0 flow that deals with the inner VLAN in ONU
+	{
+		subflow1 := of.NewVoltSubFlow()
+		subflow1.SetTableID(0)
+		subflow1.SetGoToTable(1)
+		subflow1.SetInPort(inport)
+
+		if pbits != PbitMatchNone {
+			subflow1.SetMatchPbit(pbits)
+		}
+		if err := vs.setUSMatchActionVlanT0(subflow1); err != nil {
+			return nil, err
+		}
+		subflow1.SetMeterID(vs.UsMeterID)
+
+		/* WriteMetaData 8 Byte(uint64) usage:
+		| Byte8    | Byte7    | Byte6 | Byte5  | Byte4  | Byte3   | Byte2  | Byte1 |
+		| reserved | reserved | TpID  | TpID   | uinID  | uniID   | uniID  | uniID | */
+		metadata := uint64(vs.CVlan)<<48 + uint64(vs.TechProfileID)<<32 + uint64(outport)
+		subflow1.SetWriteMetadata(metadata)
+
+		/* TableMetaData 8 Byte(uint64) usage: (Considering MSB bit as 63rd bit and LSB bit as 0th bit)
+		|                         Byte8                     |  Byte7    |  Byte6    |      Byte5       |  Byte4    | Byte3    | Byte2    | Byte1   |
+		| 000      |    0      |    00     |    0     |  0  |  00000000 |  00000000 |  0000   0000     |  00000000 | 00000000 | 00000000 | 00000000|
+		| reserved | reqBwInfo | svlanTpID |  Buff us |  AT |  schedID  |  schedID  | onteth  vlanCtrl |  unitag   | unitag   | ctag     | ctag    | */
+		metadata = uint64(reqBwInfo)<<60 | uint64(allowTransparent)<<56 | uint64(vs.SchedID)<<40 | uint64(vs.ONTEtherTypeClassification)<<36 | uint64(vs.VlanControl)<<32 | uint64(vs.CVlan)
+
+		// // In case of MAC Learning enabled voltha will buffer the US flow installation.
+		// if NonZeroMacAddress(vs.MacAddr) {
+		// 	subflow1.SetMatchSrcMac(vs.MacAddr)
+		// } else if vs.MacLearning != MacLearning {
+		// 	metadata |= 1 << 57
+		// 	logger.Infow(ctx, "Buffer us flow at adapter", log.Fields{"metadata": metadata})
+		// }
+		subflow1.SetTableMetadata(metadata)
+		if vs.VlanControl == OLTCVlanOLTSVlan {
+			/**
+			 * The new cookie generation is only for OLT_CVLAN_OLT_SVLAN case (TEF residential case) within a UNI.
+			 * After vgc upgrade, if we have to deactivate an already existing TEF residential service, then we have to
+			 * use old cookie.
+			 */
+			subflow1.Cookie = vs.generateUSCookie(vs.UniVlan, 0, inport, pbits)
+			if vgcRebooted {
+				subflow1.OldCookie = vs.generateUSCookie(vs.CVlan, 0, inport, pbits)
+			}
+		} else {
+			// In case of Olt_Svlan , CVLAN=UNIVLAN so cookie can be constructed with CVLAN as well
+			subflow1.Cookie = vs.generateUSCookie(vs.CVlan, 0, inport, pbits)
+		}
+		subflow1.Priority = of.HsiaFlowPriority
+		flow.SubFlows[subflow1.Cookie] = subflow1
+		logger.Infow(ctx, "Building upstream HSIA flow for T0", log.Fields{"cookie": subflow1.Cookie, "subflow": subflow1})
+	}
+
+	//Add Table-1 flow that deals with the outer vlan in pOLT
+	{
+		subflow2 := of.NewVoltSubFlow()
+		subflow2.SetTableID(1)
+		subflow2.SetInPort(inport)
+
+		if pbits != PbitMatchNone {
+			subflow2.SetMatchPbit(pbits)
+		}
+
+		if err := vs.setUSMatchActionVlanT1(subflow2); err != nil {
+			return nil, err
+		}
+		subflow2.SetInPort(inport)
+		subflow2.SetOutPort(outport)
+		subflow2.SetMeterID(vs.UsMeterID)
+
+		// refer Table-0 flow generation for byte information
+		metadata := uint64(vs.CVlan)<<48 + uint64(vs.TechProfileID)<<32 + uint64(outport)
+		subflow2.SetWriteMetadata(metadata)
+
+		// refer Table-0 flow generation for byte information
+		metadata = uint64(reqBwInfo)<<60 | uint64(allowTransparent)<<56 | uint64(vs.SchedID)<<40 | uint64(vs.ONTEtherTypeClassification)<<36 | uint64(vs.VlanControl)<<32 | uint64(vs.CVlan)
+		// // In case of MAC Learning enabled voltha will buffer the US flow installation.
+		// if NonZeroMacAddress(vs.MacAddr) {
+		// 	subflow2.SetMatchSrcMac(vs.MacAddr)
+		// } else if vs.MacLearning != MacLearningNone {
+		// 	metadata |= 1 << 57
+		// 	logger.Infow(ctx, "Buffer us flow at adapter", log.Fields{"metadata": metadata})
+		// }
+		subflow2.SetTableMetadata(metadata)
+		if vs.VlanControl == OLTCVlanOLTSVlan {
+			/**
+			 * The new cookie generation is only for OLT_CVLAN_OLT_SVLAN case (TEF residential case) within a UNI.
+			 * After vgc upgrade, if we have to deactivate an already existing TEF residential service, then we have to
+			 * use old cookie.
+			 */
+			subflow2.Cookie = vs.generateUSCookie(vs.UniVlan, 1, inport, pbits)
+			if vgcRebooted {
+				subflow2.OldCookie = vs.generateUSCookie(vs.CVlan, 1, inport, pbits)
+			}
+		} else {
+			// In case of Olt_Svlan , CVLAN=UNIVLAN so cookie can be constructed with CVLAN as well
+			subflow2.Cookie = vs.generateUSCookie(vs.CVlan, 1, inport, pbits)
+		}
+		subflow2.Priority = of.HsiaFlowPriority
+
+		flow.SubFlows[subflow2.Cookie] = subflow2
+		logger.Infow(ctx, "Building upstream HSIA flow for T1", log.Fields{"cookie": subflow2.Cookie, "subflow": subflow2})
+	}
+
+	return flow, nil
+}
+
+func (vs *VoltService) generateUSCookie(vlan of.VlanType, valToShift uint64, inport uint32, pbits of.PbitType) uint64 {
+	//| 12-bit cvlan/UniVlan | 4 bits empty | <32-bits uniport>| 16-bits HSIA mask OR flow mask OR pbit |
+	cookie := uint64(vlan)<<52 + uint64(inport)<<16 | of.HsiaFlowMask
+	cookie = cookie | of.UsFlowMask
+	cookie = cookie + (valToShift << 4) + uint64(pbits)
+	return cookie
+}
+
+// setUSMatchActionVlanT1 - Sets the Match & Action w.r.t Vlans for US Table-1
+// based on different Vlan Controls
+func (vs *VoltService) setUSMatchActionVlanT1(flow *of.VoltSubFlow) error {
+	switch vs.VlanControl {
+	case None:
+		flow.SetMatchVlan(vs.SVlan)
+	case ONUCVlanOLTSVlan:
+		flow.SetMatchVlan(vs.CVlan)
+		flow.SetPushVlan(vs.SVlan, vs.SVlanTpid)
+	case OLTCVlanOLTSVlan:
+		flow.SetMatchVlan(vs.UniVlan)
+		flow.SetSetVlan(vs.CVlan)
+		flow.SetPushVlan(vs.SVlan, vs.SVlanTpid)
+	case ONUCVlan:
+		flow.SetMatchVlan(vs.SVlan)
+	case OLTSVlan:
+		if vs.UniVlan != of.VlanAny && vs.UniVlan != of.VlanNone {
+			flow.SetMatchVlan(vs.UniVlan)
+			flow.SetSetVlan(vs.SVlan)
+		} else if vs.UniVlan != of.VlanNone {
+			flow.SetMatchVlan(vs.UniVlan)
+			flow.SetPushVlan(vs.SVlan, layers.EthernetTypeDot1Q)
+		} else {
+			flow.SetPushVlan(vs.SVlan, layers.EthernetTypeDot1Q)
+		}
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vs.VlanControl})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+	return nil
+}
+
+// setDSMatchActionVlanT0 - Sets the Match & Action w.r.t Vlans for DS Table-0
+// based on different Vlan Controls
+func (vs *VoltService) setDSMatchActionVlanT0(flow *of.VoltSubFlow) error {
+	switch vs.VlanControl {
+	case None:
+		flow.SetMatchVlan(vs.SVlan)
+	case ONUCVlanOLTSVlan:
+		flow.SetMatchVlan(vs.SVlan)
+		flow.SetPopVlan()
+	case OLTCVlanOLTSVlan:
+		flow.SetMatchVlan(vs.SVlan)
+		flow.SetPopVlan()
+		flow.SetSetVlan(vs.UniVlan)
+	case ONUCVlan:
+		flow.SetMatchVlan(vs.SVlan)
+	case OLTSVlan:
+		flow.SetMatchVlan(vs.SVlan)
+		if vs.UniVlan != of.VlanNone && vs.UniVlan != of.VlanAny {
+			flow.SetSetVlan(vs.UniVlan)
+		} else {
+			flow.SetPopVlan()
+		}
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vs.VlanControl})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+	return nil
+}
+
+// setUSMatchActionVlanT0 - Sets the Match & Action w.r.t Vlans for US Table-0
+// based on different Vlan Controls
+func (vs *VoltService) setUSMatchActionVlanT0(flow *of.VoltSubFlow) error {
+	switch vs.VlanControl {
+	case None:
+		flow.SetMatchVlan(vs.SVlan)
+	case ONUCVlanOLTSVlan:
+		if vs.UniVlan != of.VlanNone {
+			flow.SetMatchVlan(vs.UniVlan)
+			flow.SetSetVlan(vs.CVlan)
+		} else {
+			flow.SetPushVlan(vs.CVlan, layers.EthernetTypeDot1Q)
+		}
+	case OLTCVlanOLTSVlan:
+		flow.SetMatchVlan(vs.UniVlan)
+	case ONUCVlan:
+		if vs.UniVlan != of.VlanNone {
+			flow.SetMatchVlan(vs.UniVlan)
+			flow.SetSetVlan(vs.SVlan)
+		} else {
+			flow.SetPushVlan(vs.SVlan, layers.EthernetTypeDot1Q)
+		}
+	case OLTSVlan:
+		flow.SetMatchVlan(vs.UniVlan)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vs.VlanControl})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+	return nil
+}
+
+// setDSMatchActionVlanT1 - Sets the Match & Action w.r.t Vlans for DS Table-1
+// based on different Vlan Controls
+func (vs *VoltService) setDSMatchActionVlanT1(flow *of.VoltSubFlow) error {
+	switch vs.VlanControl {
+	case None:
+		flow.SetMatchVlan(vs.SVlan)
+	case ONUCVlanOLTSVlan:
+		flow.SetMatchVlan(vs.CVlan)
+		if vs.UniVlan != of.VlanNone {
+			flow.SetSetVlan(vs.UniVlan)
+		} else {
+			flow.SetPopVlan()
+		}
+	case OLTCVlanOLTSVlan:
+		flow.SetMatchVlan(vs.UniVlan)
+	case ONUCVlan:
+		flow.SetMatchVlan(vs.SVlan)
+		if vs.UniVlan != of.VlanNone {
+			flow.SetSetVlan(vs.UniVlan)
+		} else {
+			flow.SetPopVlan()
+		}
+	case OLTSVlan:
+		flow.SetMatchVlan(vs.UniVlan)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vs.VlanControl})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+	return nil
+}
+
+// SvcUpInd for service up indication
+func (vs *VoltService) SvcUpInd() {
+	vs.AddHsiaFlows()
+}
+
+// SvcDownInd for service down indication
+func (vs *VoltService) SvcDownInd() {
+	vs.DelHsiaFlows()
+}
+
+// SetIpv4Addr to set ipv4 address
+func (vs *VoltService) SetIpv4Addr(addr net.IP) {
+	vs.Ipv4Addr = addr
+}
+
+// SetIpv6Addr to set ipv6 address
+func (vs *VoltService) SetIpv6Addr(addr net.IP) {
+	vs.Ipv6Addr = addr
+}
+
+// SetMacAddr to set mac address
+func (vs *VoltService) SetMacAddr(addr net.HardwareAddr) {
+	vs.MacAddr = addr
+}
+
+// ----------------------------------------------
+// VOLT Application - Related to services
+// ---------------------------------------------
+// ---------------------------------------------------------------
+// Service CRUD functions. These are exposed to the overall binary
+// to be invoked from the point where the CRUD operations are received
+// from the external entities
+
+// AddService :  A service in the context of VOLT is a subscriber or service of a
+// subscriber which is uniquely identified by a combination of MAC
+// address, VLAN tags, 802.1p bits. However, in the context of the
+// current implementation, a service is an entity that is identified by a
+// unique L2 (MAC address + VLANs) or unique L3 (VLANs + IP address)
+// FUNC: Add Service
+func (va *VoltApplication) AddService(cfg VoltServiceCfg, oper *VoltServiceOper) error {
+	var mmUs, mmDs *VoltMeter
+	var err error
+
+	//Take the  Device lock only in case of NB add request.
+	// Allow internal adds since internal add happen only under
+	// 1. Restore Service from DB
+	// 2. Service Migration
+	if oper == nil {
+		if svc := va.GetService(cfg.Name); svc != nil {
+			logger.Warnw(ctx, "Service Already Exists. Ignoring Add Service Request", log.Fields{"Name": cfg.Name})
+			return errors.New("Service Already Exists")
+		}
+	}
+
+	logger.Infow(ctx, "Service to be configured", log.Fields{"Cfg": cfg})
+	// Service doesn't exist. So create it and add to the port
+	vs := NewVoltService(&cfg)
+	if oper != nil {
+		vs.UsHSIAFlowsApplied = oper.UsHSIAFlowsApplied
+		vs.DsHSIAFlowsApplied = oper.DsHSIAFlowsApplied
+		vs.Ipv4Addr = oper.Ipv4Addr
+		vs.Ipv6Addr = oper.Ipv6Addr
+		vs.MacLearning = cfg.MacLearning
+		vs.PendingFlows = oper.PendingFlows
+		vs.AssociatedFlows = oper.AssociatedFlows
+		vs.DeleteInProgress = oper.DeleteInProgress
+		vs.BwAvailInfo = oper.BwAvailInfo
+		vs.Device = oper.Device
+	} else {
+
+		//Sorting Pbit from highest
+		sort.Slice(vs.Pbits, func(i, j int) bool {
+			return vs.Pbits[i] > vs.Pbits[j]
+		})
+		logger.Infow(ctx, "Sorted Pbits", log.Fields{"Pbits": vs.Pbits})
+	}
+	logger.Infow(ctx, "VolthService...", log.Fields{"vs": vs.Name})
+
+	// The bandwidth and shaper profile combined into meter
+	if mmDs, err = va.GetMeter(cfg.DsMeterProfile); err == nil {
+		vs.DsMeterID = mmDs.ID
+	} else {
+		return errors.New("DownStream meter profile not found")
+	}
+
+	// The aggregated downstream meter profile
+	// if mmAg, err = va.GetMeter(cfg.AggDsMeterProfile); err == nil {
+	// 	vs.AggDsMeterID = mmAg.ID
+	// } else {
+	// 	return errors.New("Aggregated meter profile not found")
+	// }
+
+	// if cfg.AggDsMeterProfile == cfg.UsMeterProfile {
+	// 	vs.UsMeterID = mmAg.ID
+	// } else {
+	// The bandwidth and shaper profile combined into meter
+	if mmUs, err = va.GetMeter(cfg.UsMeterProfile); err == nil {
+		vs.UsMeterID = mmUs.ID
+	} else {
+		return errors.New("Upstream meter profile not found")
+	}
+	//}
+
+	AppMutex.ServiceDataMutex.Lock()
+	defer AppMutex.ServiceDataMutex.Unlock()
+
+	// Add the service to the VNET
+	vnet := va.GetVnet(cfg.SVlan, cfg.CVlan, cfg.UniVlan)
+	if vnet != nil {
+		if vpv := va.GetVnetByPort(vs.Port, cfg.SVlan, cfg.CVlan, cfg.UniVlan); vpv != nil {
+			vpv.VpvLock.Lock()
+			vpv.AddSvc(vs)
+			vpv.VpvLock.Unlock()
+		} else {
+			va.AddVnetToPort(vs.Port, vnet, vs)
+		}
+	} else {
+		logger.Error(ctx, "VNET-does-not-exist-for-service", log.Fields{"ServiceName": cfg.Name})
+		return errors.New("VNET doesn't exist")
+	}
+
+	vs.Version = database.PresentVersionMap[database.ServicePath]
+	// Add the service to the volt application
+	va.ServiceByName.Store(vs.Name, vs)
+	vs.WriteToDb()
+
+	if nil == oper {
+
+		if !vs.UsHSIAFlowsApplied {
+			vs.triggerServiceInProgressInd()
+		}
+
+		//Update meter profiles service count if service is being added from northbound
+		mmDs.AssociatedServices++
+		va.UpdateMeterProf(*mmDs)
+		if mmUs != nil {
+			mmUs.AssociatedServices++
+			va.UpdateMeterProf(*mmUs)
+		}
+		//mmAg.AssociatedServices++
+		//va.UpdateMeterProf(*mmAg)
+		logger.Debug(ctx, "northbound-service-add-sucessful", log.Fields{"ServiceName": vs.Name})
+	}
+
+	logger.Warnw(ctx, "Added Service to DB", log.Fields{"Name": vs.Name, "Port": (vs.Port), "ML": vs.MacLearning})
+	return nil
+}
+
+//DelServiceWithPrefix - Deletes service with the provided prefix.
+// Added for DT/TT usecase with sadis replica interface
+func (va *VoltApplication) DelServiceWithPrefix(prefix string) {
+	va.ServiceByName.Range(func(key, value interface{}) bool {
+		srvName := key.(string)
+		vs := value.(*VoltService)
+		if strings.Contains(srvName, prefix) {
+			va.DelService(srvName, true, nil, false)
+
+			vnetName := strconv.FormatUint(uint64(vs.SVlan), 10) + "-"
+			vnetName = vnetName + strconv.FormatUint(uint64(vs.CVlan), 10) + "-"
+			vnetName = vnetName + strconv.FormatUint(uint64(vs.UniVlan), 10)
+
+			if err := va.DelVnet(vnetName, ""); err != nil {
+				logger.Warnw(ctx, "Delete Vnet Failed", log.Fields{"Name": vnetName, "Error": err})
+			}
+		}
+		return true
+	})
+}
+
+// DelService delete a service form the application
+func (va *VoltApplication) DelService(name string, forceDelete bool, newSvc *VoltServiceCfg, serviceMigration bool) {
+
+	AppMutex.ServiceDataMutex.Lock()
+	defer AppMutex.ServiceDataMutex.Unlock()
+
+	logger.Warnw(ctx, "Delete Service Request", log.Fields{"Service": name, "ForceDelete": forceDelete, "serviceMigration": serviceMigration})
+	var noFlowsPresent bool
+
+	vsIntf, ok := va.ServiceByName.Load(name)
+	if !ok {
+		logger.Warnw(ctx, "Service doesn't exist", log.Fields{"ServiceName": name})
+		return
+	}
+	vs := vsIntf.(*VoltService)
+	vpv := va.GetVnetByPort(vs.Port, vs.SVlan, vs.CVlan, vs.UniVlan)
+	if vpv == nil {
+		logger.Errorw(ctx, "Vpv Not found for Service", log.Fields{"vs": vs.Name})
+		return
+	}
+
+	//Set this to avoid race-condition during flow result processing
+	vs.DeleteInProgress = true
+	vs.ForceDelete = forceDelete
+	vs.ForceWriteToDb()
+
+	if len(vs.AssociatedFlows) == 0 {
+		noFlowsPresent = true
+	}
+	vpv.VpvLock.Lock()
+	defer vpv.VpvLock.Unlock()
+
+	vs.DelHsiaFlows()
+
+	if vpv.IgmpEnabled {
+		va.ReceiverDownInd(vpv.Device, vpv.Port)
+	}
+	logger.Infow(ctx, "Delete Service from VPV", log.Fields{"VPV_Port": vpv.Port, "VPV_SVlan": vpv.SVlan, "VPV_CVlan": vpv.CVlan, "VPV_UniVlan": vpv.UniVlan, "ServiceName": name})
+	vpv.DelService(vs)
+	if vpv.servicesCount.Load() == 0 {
+		va.DelVnetFromPort(vs.Port, vpv)
+	}
+
+	// Delete the service immediately in case of Force Delete
+	// This will be enabled when profile reconciliation happens after restore
+	// of backedup data
+	if vs.ForceDelete {
+		vs.DelFromDb()
+		GetApplication().ServiceByName.Delete(vs.Name)
+		logger.Warnw(ctx, "Deleted service from DB/Cache successfully", log.Fields{"serviceName": vs.Name})
+	}
+
+	meterProfiles := make(map[*VoltMeter]bool)
+
+	if nil != newSvc {
+		logger.Infow(ctx, "Old Service meter profiles", log.Fields{"AGG": vs.AggDsMeterProfile, "DS": vs.DsMeterProfile, "US": vs.UsMeterProfile})
+		logger.Infow(ctx, "New Service meter profiles", log.Fields{"AGG": newSvc.AggDsMeterProfile, "DS": newSvc.DsMeterProfile, "US": newSvc.UsMeterProfile})
+	}
+	skipMeterDeletion := false
+	if aggMeter, ok := va.MeterMgr.GetMeterByID(vs.AggDsMeterID); ok {
+		if nil != newSvc && aggMeter.Name == newSvc.AggDsMeterProfile {
+			skipMeterDeletion = true
+		}
+
+		meterProfiles[aggMeter] = skipMeterDeletion
+		skipMeterDeletion = false
+	}
+	if dsMeter, ok := va.MeterMgr.GetMeterByID(vs.DsMeterID); ok {
+		if nil != newSvc && dsMeter.Name == newSvc.DsMeterProfile {
+			skipMeterDeletion = true
+		}
+		meterProfiles[dsMeter] = skipMeterDeletion
+		skipMeterDeletion = false
+	}
+	if vs.AggDsMeterID != vs.UsMeterID {
+		if usMeter, ok := va.MeterMgr.GetMeterByID(vs.UsMeterID); ok {
+			if nil != newSvc && usMeter.Name == newSvc.UsMeterProfile {
+				skipMeterDeletion = true
+			}
+			meterProfiles[usMeter] = skipMeterDeletion
+		}
+	}
+
+	for meter, skipMeterDeletion := range meterProfiles {
+		if nil == meter {
+			logger.Debug(ctx, "Null meter found, continuing")
+			continue
+		}
+		if meter.AssociatedServices > 0 {
+			meter.AssociatedServices--
+			if meter.AssociatedServices == 0 && !skipMeterDeletion {
+				logger.Info(ctx, "Meter should be deleted now\n", log.Fields{"MeterID": meter})
+				va.UpdateMeterProf(*meter)
+			}
+		}
+	}
+
+	if noFlowsPresent || vs.ForceDelete {
+		vs.CheckAndDeleteService()
+	}
+
+	//Delete the per service counter too
+	va.ServiceCounters.Delete(name)
+	if vs.IgmpEnabled && vs.EnableMulticastKPI {
+		_ = db.DelAllServiceChannelCounter(name)
+	}
+}
+
+//AddFlows - Adds the flow to the service
+// Triggers flow addition after registering for flow indication event
+func (vs *VoltService) AddFlows(device *VoltDevice, flow *of.VoltFlow) error {
+
+	// Using locks instead of concurrent map for PendingFlows to avoid
+	// race condition during flow response indication processing
+	vs.ServiceLock.Lock()
+	defer vs.ServiceLock.Unlock()
+
+	for cookie := range flow.SubFlows {
+		cookie := strconv.FormatUint(cookie, 10)
+		fe := &FlowEvent{
+			eType:     EventTypeServiceFlowAdded,
+			device:    device.Name,
+			cookie:    cookie,
+			eventData: vs,
+		}
+		device.RegisterFlowAddEvent(cookie, fe)
+		vs.PendingFlows[cookie] = true
+	}
+	return cntlr.GetController().AddFlows(vs.Port, device.Name, flow)
+}
+
+//FlowInstallSuccess - Called when corresponding service flow installation is success
+// If no more pending flows, HSIA indication wil be triggered
+func (vs *VoltService) FlowInstallSuccess(cookie string, bwAvailInfo of.BwAvailDetails) {
+	if vs.DeleteInProgress {
+		logger.Warnw(ctx, "Skipping Flow Add Success Notification. Service deletion in-progress", log.Fields{"Cookie": cookie, "Service": vs.Name})
+		return
+	}
+	vs.ServiceLock.Lock()
+
+	if _, ok := vs.PendingFlows[cookie]; !ok {
+		logger.Errorw(ctx, "Flow Add Success for unknown Cookie", log.Fields{"Service": vs.Name, "Cookie": cookie})
+		vs.ServiceLock.Unlock()
+		return
+	}
+
+	delete(vs.PendingFlows, cookie)
+	vs.AssociatedFlows[cookie] = true
+	vs.ServiceLock.Unlock()
+	var prevBwAvail, presentBwAvail string
+	if bwAvailInfo.PrevBw != "" && bwAvailInfo.PresentBw != "" {
+		prevBwAvail = bwAvailInfo.PrevBw
+		presentBwAvail = bwAvailInfo.PresentBw
+		vs.BwAvailInfo = prevBwAvail + "," + presentBwAvail
+		logger.Debug(ctx, "Bandwidth-value-formed", log.Fields{"BwAvailInfo": vs.BwAvailInfo})
+	}
+	vs.WriteToDb()
+
+	if len(vs.PendingFlows) == 0 && vs.DsHSIAFlowsApplied {
+
+		device, err := GetApplication().GetDeviceFromPort(vs.Port)
+		if err != nil {
+			logger.Errorw(ctx, "Error Getting Device. Dropping HSIA Success indication to NB", log.Fields{"Reason": err.Error(), "Service": vs.Name, "Port": vs.Port})
+			return
+		} else if device.State != controller.DeviceStateUP {
+			logger.Warnw(ctx, "Device state Down. Dropping HSIA Success indication to NB", log.Fields{"Service": vs.Name, "Port": vs.Port})
+			return
+		}
+
+		if vs.Trigger == ServiceVlanUpdate {
+			vs.Trigger = NBActivate
+			defer vs.WriteToDb()
+		}
+		logger.Infow(ctx, "All Flows installed for Service", log.Fields{"Service": vs.Name})
+		return
+	}
+	logger.Infow(ctx, "Processed Service Flow Add Success Indication", log.Fields{"Cookie": cookie, "Service": vs.Name, "DsFlowsApplied": vs.DsHSIAFlowsApplied})
+}
+
+//FlowInstallFailure - Called when corresponding service flow installation is failed
+// Trigger service failure indication to NB
+func (vs *VoltService) FlowInstallFailure(cookie string, errorCode uint32, errReason string) {
+	vs.ServiceLock.RLock()
+
+	if _, ok := vs.PendingFlows[cookie]; !ok {
+		logger.Errorw(ctx, "Flow Add Failure for unknown Cookie", log.Fields{"Service": vs.Name, "Cookie": cookie})
+		vs.ServiceLock.RUnlock()
+		return
+	}
+	vs.ServiceLock.RUnlock()
+	logger.Errorw(ctx, "HSIA Flow Add Failure Notification", log.Fields{"uniPort": vs.Port, "Cookie": cookie, "Service": vs.Name, "ErrorCode": errorCode, "ErrorReason": errReason})
+	vs.triggerServiceFailureInd(errorCode, errReason)
+}
+
+//DelFlows - Deletes the flow from the service
+// Triggers flow deletion after registering for flow indication event
+func (vs *VoltService) DelFlows(device *VoltDevice, flow *of.VoltFlow) error {
+
+	if !vs.ForceDelete {
+		// Using locks instead of concurrent map for AssociatedFlows to avoid
+		// race condition during flow response indication processing
+		vs.ServiceLock.Lock()
+		defer vs.ServiceLock.Unlock()
+
+		for cookie := range flow.SubFlows {
+			cookie := strconv.FormatUint(cookie, 10)
+			fe := &FlowEvent{
+				eType:     EventTypeServiceFlowRemoved,
+				cookie:    cookie,
+				eventData: vs,
+			}
+			device.RegisterFlowDelEvent(cookie, fe)
+		}
+	}
+	return cntlr.GetController().DelFlows(vs.Port, device.Name, flow)
+}
+
+//CheckAndDeleteService - remove service from DB is there are no pending flows to be removed
+func (vs *VoltService) CheckAndDeleteService() {
+	if vs.DeleteInProgress && len(vs.AssociatedFlows) == 0 && !vs.DsHSIAFlowsApplied {
+		vs.DelFromDb()
+		GetApplication().ServiceByName.Delete(vs.Name)
+		logger.Warnw(ctx, "Deleted service from DB/Cache successfully", log.Fields{"serviceName": vs.Name})
+	}
+}
+
+//FlowRemoveSuccess - Called when corresponding service flow removal is success
+// If no more associated flows, DelHSIA indication wil be triggered
+func (vs *VoltService) FlowRemoveSuccess(cookie string) {
+
+	// if vs.DeleteInProgress {
+	// 	logger.Warnw(ctx, "Skipping Flow Remove Success Notification. Service deletion in-progress", log.Fields{"Cookie": cookie, "Service": vs.Name})
+	// 	return
+	// }
+	vs.ServiceLock.Lock()
+	logger.Infow(ctx, "Processing Service Flow Remove Success Indication", log.Fields{"Cookie": cookie, "Service": vs.Name, "Associated Flows": vs.AssociatedFlows, "DsFlowsApplied": vs.DsHSIAFlowsApplied})
+
+	if _, ok := vs.AssociatedFlows[cookie]; ok {
+		delete(vs.AssociatedFlows, cookie)
+	} else if _, ok := vs.PendingFlows[cookie]; ok {
+		logger.Errorw(ctx, "Service Flow Remove: Cookie Present in Pending Flow list. No Action", log.Fields{"Service": vs.Name, "Cookie": cookie, "AssociatedFlows": vs.AssociatedFlows, "PendingFlows": vs.PendingFlows})
+	} else {
+		logger.Errorw(ctx, "Service Flow Remove Success for unknown Cookie", log.Fields{"Service": vs.Name, "Cookie": cookie, "AssociatedFlows": vs.AssociatedFlows, "PendingFlows": vs.PendingFlows})
+	}
+
+	vs.ServiceLock.Unlock()
+
+	vs.WriteToDb()
+
+	if len(vs.AssociatedFlows) == 0 && !vs.DsHSIAFlowsApplied {
+
+		device := GetApplication().GetDevice(vs.Device)
+		if device == nil {
+			logger.Errorw(ctx, "Error Getting Device. Dropping DEL_HSIA Success indication to NB", log.Fields{"Service": vs.Name, "Port": vs.Port})
+			return
+		} else if device.State != controller.DeviceStateUP {
+			logger.Warnw(ctx, "Device state Down. Dropping DEL_HSIA Success indication to NB", log.Fields{"Service": vs.Name, "Port": vs.Port})
+			return
+		}
+
+		if vs.UpdateInProgress {
+			vs.updateVnetProfile(vs.Device)
+			//Not sending DEL_HSIA Indication since it wil be generated internally by SubMgr
+			return
+		}
+		logger.Infow(ctx, "All Flows removed for Service. Triggering Service De-activation Success indication to NB", log.Fields{"Service": vs.Name, "DeleteFlag": vs.DeleteInProgress})
+		vs.CheckAndDeleteService()
+
+		return
+	}
+	logger.Infow(ctx, "Processed Service Flow Remove Success Indication", log.Fields{"Cookie": cookie, "Service": vs.Name, "Associated Flows": vs.AssociatedFlows, "DsFlowsApplied": vs.DsHSIAFlowsApplied})
+}
+
+//FlowRemoveFailure - Called when corresponding service flow installation is failed
+// Trigger service failure indication to NB
+func (vs *VoltService) FlowRemoveFailure(cookie string, errorCode uint32, errReason string) {
+	vs.ServiceLock.RLock()
+
+	if _, ok := vs.AssociatedFlows[cookie]; !ok {
+		logger.Errorw(ctx, "Flow Failure for unknown Cookie", log.Fields{"Service": vs.Name, "Cookie": cookie})
+		vs.ServiceLock.RUnlock()
+		return
+	}
+	if vs.DeleteInProgress {
+		delete(vs.AssociatedFlows, cookie)
+	}
+	vs.ServiceLock.RUnlock()
+	logger.Errorw(ctx, "Service Flow Remove Failure Notification", log.Fields{"uniPort": vs.Port, "Cookie": cookie, "Service": vs.Name, "ErrorCode": errorCode, "ErrorReason": errReason})
+
+	vs.triggerServiceFailureInd(errorCode, errReason)
+	vs.CheckAndDeleteService()
+}
+
+func (vs *VoltService) triggerServiceFailureInd(errorCode uint32, errReason string) {
+	device, err := GetApplication().GetDeviceFromPort(vs.Port)
+	if err != nil {
+		logger.Errorw(ctx, "Error Getting Device. Dropping DEL_HSIA Failure indication to NB", log.Fields{"Reason": err.Error(), "Service": vs.Name, "Port": vs.Port})
+		return
+	} else if device.State != controller.DeviceStateUP {
+		logger.Warnw(ctx, "Device state Down. Dropping DEL_HSIA Failure indication to NB", log.Fields{"Service": vs.Name, "Port": vs.Port})
+		return
+	}
+}
+
+// RestoreSvcsFromDb read from the DB and restore all the services
+func (va *VoltApplication) RestoreSvcsFromDb() {
+	// VNETS must be learnt first
+	vss, _ := db.GetServices()
+	for _, vs := range vss {
+		b, ok := vs.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var vvs VoltService
+		err := json.Unmarshal(b, &vvs)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of VNET failed")
+			continue
+		}
+		logger.Debugw(ctx, "Retrieved Service", log.Fields{"Service": vvs.VoltServiceCfg})
+		if err := va.AddService(vvs.VoltServiceCfg, &vvs.VoltServiceOper); err != nil {
+			logger.Warnw(ctx, "Add New Service Failed", log.Fields{"Service": vvs.Name, "Error": err})
+		}
+
+		if vvs.VoltServiceOper.DeleteInProgress {
+			va.ServicesToDelete[vvs.VoltServiceCfg.Name] = true
+			logger.Warnw(ctx, "Service (restored) to be deleted", log.Fields{"Service": vvs.Name})
+		}
+	}
+}
+
+// GetService to get service
+func (va *VoltApplication) GetService(name string) *VoltService {
+	if vs, ok := va.ServiceByName.Load(name); ok {
+		return vs.(*VoltService)
+	}
+	return nil
+}
+
+// GetCircuitID to get circuit id
+func (vs *VoltService) GetCircuitID() []byte {
+	return []byte(vs.CircuitID)
+}
+
+// GetRemoteID to get remote id
+func (vs *VoltService) GetRemoteID() []byte {
+	return []byte(vs.RemoteID)
+}
+
+// IPAssigned to check if ip is assigned
+func (vs *VoltService) IPAssigned() bool {
+	if vs.Ipv4Addr != nil && !vs.Ipv4Addr.Equal(net.ParseIP("0.0.0.0")) {
+		return true
+	} else if vs.Ipv6Addr != nil && !vs.Ipv6Addr.Equal(net.ParseIP("0:0:0:0:0:0:0:0")) {
+		return true
+	}
+	return false
+}
+
+// GetServiceNameFromCookie to get service name from cookie
+func (va *VoltApplication) GetServiceNameFromCookie(cookie uint64, portName string, pbit uint8, device string, tableMetadata uint64) *VoltService {
+
+	var vlan uint64
+	vlanControl := (tableMetadata >> 32) & 0xF
+
+	if vlanControl == uint64(OLTCVlanOLTSVlan) {
+		// Fetching UniVlan for vlanControl OLTCVLANOLTSVLAN
+		vlan = (tableMetadata >> 16) & 0xFFFF
+	} else {
+		//Fetching CVlan for other vlanControl
+		vlan = cookie >> 52
+	}
+	logger.Infow(ctx, "Configured Params", log.Fields{"VlanControl": vlanControl, "vlan": vlan})
+	var vlans []of.VlanType
+	vlans = append(vlans, of.VlanType(vlan))
+	service := GetApplication().GetServiceFromCvlan(device, portName, vlans, uint8(pbit))
+	if nil != service {
+		logger.Info(ctx, "Service Found for", log.Fields{"serviceName": service.Name, "portName": portName, "ctag": vlan})
+	} else {
+		logger.Errorw(ctx, "No Service for", log.Fields{"portName": portName, "ctag": vlan, "Pbit": pbit, "device": device, "VlanControl": vlanControl})
+	}
+	return service
+}
+
+//MigrateServicesReqStatus - update vnet request status
+type MigrateServicesReqStatus string
+
+const (
+	//MigrateSrvsReqInit constant
+	MigrateSrvsReqInit MigrateServicesReqStatus = "Init"
+	//MigrateSrvsReqDeactTriggered constant
+	MigrateSrvsReqDeactTriggered MigrateServicesReqStatus = "Profiles Deactivated"
+	//MigrateSrvsReqCompleted constant
+	MigrateSrvsReqCompleted MigrateServicesReqStatus = "Update Complete"
+)
+
+//MigrateServicesRequest - update vnet request params
+type MigrateServicesRequest struct {
+	ID                  string
+	OldVnetID           string
+	NewVnetID           string
+	ServicesList        map[string]bool
+	DeviceID            string
+	Status              MigrateServicesReqStatus
+	MigrateServicesLock sync.RWMutex
+}
+
+func newMigrateServicesRequest(id string, oldVnetID string, newVnetID string, serviceMap map[string]bool, deviceID string) *MigrateServicesRequest {
+
+	var msr MigrateServicesRequest
+	msr.OldVnetID = oldVnetID
+	msr.NewVnetID = newVnetID
+	msr.ID = id
+	msr.ServicesList = serviceMap
+	msr.DeviceID = deviceID
+	msr.Status = MigrateSrvsReqInit
+	return &msr
+}
+
+//GetMsrKey - generates migrate service request key
+func (msr *MigrateServicesRequest) GetMsrKey() string {
+	return msr.OldVnetID + "-" + msr.ID
+}
+
+// //isRequestComplete - return if all request has been processed and completed
+// // RequestProcessed indicates that all the profile de-activation has been triggered
+// // And the associated profiles indicates the profiles awaiting results
+// func (msr *MigrateServicesRequest) isRequestComplete() bool {
+// 	//return edr.RequestProcessed && (len(edr.AssociatedProfiles) == 0)
+// 	return (len(edr.AssociatedProfiles) == 0)
+// }
+
+//WriteToDB - writes the udpate vnet request details ot DB
+func (msr *MigrateServicesRequest) WriteToDB() {
+	logger.Debugw(ctx, "Adding Migrate Service Request to DB", log.Fields{"OldVnet": msr.OldVnetID, "NewVnet": msr.NewVnetID, "Device": msr.DeviceID, "RequestID": msr.ID, "ServiceCount": len(msr.ServicesList)})
+	if b, err := json.Marshal(msr); err == nil {
+		if err = db.PutMigrateServicesReq(msr.DeviceID, msr.GetMsrKey(), string(b)); err != nil {
+			logger.Warnw(ctx, "PutMigrateServicesReq Failed", log.Fields{"OldVnet": msr.OldVnetID, "NewVnet": msr.NewVnetID,
+									"Device": msr.DeviceID, "Error": err})
+		}
+	}
+}
+
+//MigrateServices - updated vnet profile for services
+func (va *VoltApplication) MigrateServices(serialNum string, reqID string, oldVnetID, newVnetID string, serviceList []string) error {
+
+	logger.Warnw(ctx, "Migrate Serviec Request Received", log.Fields{"SerialNum": serialNum, "RequestID": reqID, "OldVnet": oldVnetID, "NewVnet": newVnetID, "ServiceList": serviceList})
+	if _, ok := va.VnetsByName.Load(oldVnetID); !ok {
+		return errors.New("Old Vnet Id not found")
+	}
+	if _, ok := va.VnetsByName.Load(newVnetID); !ok {
+		return errors.New("New Vnet Id not found")
+	}
+
+	d := va.GetDeviceBySerialNo(serialNum)
+	if d == nil {
+		logger.Errorw(ctx, "Error Getting Device", log.Fields{"SerialNum": serialNum})
+		return errorCodes.ErrDeviceNotFound
+	}
+
+	serviceMap := make(map[string]bool)
+
+	for _, service := range serviceList {
+		serviceMap[service] = false
+	}
+	msr := newMigrateServicesRequest(reqID, oldVnetID, newVnetID, serviceMap, d.Name)
+	msr.WriteToDB()
+
+	d.AddMigratingServices(msr)
+	go msr.ProcessMigrateServicesProfRequest()
+	return nil
+}
+
+//ProcessMigrateServicesProfRequest - collects all associated profiles
+func (msr *MigrateServicesRequest) ProcessMigrateServicesProfRequest() {
+	va := GetApplication()
+	for srv, processed := range msr.ServicesList {
+
+		//Indicates new service is already created and only deletion of old one is pending
+		if processed {
+			va.DelService(srv, true, nil, true)
+			msr.serviceMigrated(srv)
+			continue
+		}
+
+		logger.Infow(ctx, "Migrate Service Triggering", log.Fields{"Service": srv})
+		if vsIntf, ok := va.ServiceByName.Load(srv); ok {
+			vs := vsIntf.(*VoltService)
+			vpv := va.GetVnetByPort(vs.Port, vs.SVlan, vs.CVlan, vs.UniVlan)
+			if vpv == nil {
+				logger.Errorw(ctx, "Vpv Not found for Service", log.Fields{"vs": vs.Name, "port": vs.Port, "Vnet": vs.VnetID})
+				continue
+			}
+			logger.Infow(ctx, "Migrating Service", log.Fields{"Service": vs.Name, "UsFlowApplied": vs.UsHSIAFlowsApplied})
+			vpv.Blocked = true
+
+			// setDeactTrigger := func(key, value interface{}) bool {
+			// 	vs := value.(*VoltService)
+			vs.ServiceLock.Lock()
+			vs.UpdateInProgress = true
+			metadata := &MigrateServiceMetadata{
+				NewVnetID: msr.NewVnetID,
+				RequestID: msr.ID,
+			}
+			vs.Metadata = metadata
+			vs.ServiceLock.Unlock()
+
+			//vpv flows will be removed when last service is removed from it and
+			// new vpv flows will be installed when new service is added
+			if vs.UsHSIAFlowsApplied {
+				vpv.DelTrapFlows()
+				vs.DelHsiaFlows()
+				logger.Info(ctx, "Remove Service Flows Triggered", log.Fields{"Service": srv, "US": vs.UsHSIAFlowsApplied, "DS": vs.DsHSIAFlowsApplied})
+			} else {
+				vs.updateVnetProfile(msr.DeviceID)
+			}
+		} else {
+			logger.Warnw(ctx, "Migrate Service Failed: Service Not Found", log.Fields{"Service": srv, "Vnet": msr.OldVnetID})
+		}
+	}
+}
+
+//AddMigratingServices - store msr info to device obj
+func (d *VoltDevice) AddMigratingServices(msr *MigrateServicesRequest) {
+
+	var msrMap *util.ConcurrentMap
+	if msrMapIntf, ok := d.MigratingServices.Get(msr.OldVnetID); !ok {
+		msrMap = util.NewConcurrentMap()
+	} else {
+		msrMap = msrMapIntf.(*util.ConcurrentMap)
+	}
+
+	msrMap.Set(msr.ID, msr)
+	logger.Infow(ctx, "1: MsrListLen", log.Fields{"Len": msrMap.Length(), "Vnet": msr.OldVnetID})
+
+	d.MigratingServices.Set(msr.OldVnetID, msrMap)
+	logger.Infow(ctx, "1: DeviceMsr", log.Fields{"Device": d.Name, "Vnet": msr.OldVnetID, "Len": d.MigratingServices.Length()})
+
+}
+
+//getMigrateServicesRequest - fetches msr info from device
+func (va *VoltApplication) getMigrateServicesRequest(deviceID string, oldVnetID string, requestID string) *MigrateServicesRequest {
+	if vd := va.GetDevice(deviceID); vd != nil {
+		logger.Infow(ctx, "2: DeviceMsr", log.Fields{"Device": deviceID, "Vnet": oldVnetID, "Len": vd.MigratingServices.Length()})
+		if msrListIntf, ok := vd.MigratingServices.Get(oldVnetID); ok {
+			msrList := msrListIntf.(*util.ConcurrentMap)
+			logger.Infow(ctx, "2: MsrListLen", log.Fields{"Len": msrList.Length(), "Vnet": oldVnetID})
+			if msrObj, ok := msrList.Get(requestID); ok {
+				return msrObj.(*MigrateServicesRequest)
+			}
+
+		}
+	}
+	logger.Errorw(ctx, "Device Not Found", log.Fields{"Device": deviceID})
+	return nil
+}
+
+//updateMigrateServicesRequest - Updates the device with updated msr
+func (va *VoltApplication) updateMigrateServicesRequest(deviceID string, oldVnetID string, requestID string, msr *MigrateServicesRequest) {
+	if vd := va.GetDevice(deviceID); vd != nil {
+		if msrList, ok := vd.MigratingServices.Get(oldVnetID); ok {
+			if _, ok := msrList.(*util.ConcurrentMap).Get(requestID); ok {
+				msrList.(*util.ConcurrentMap).Set(requestID, msr)
+			}
+		}
+	}
+}
+
+//updateVnetProfile - Called on flow process completion
+// Removes old service and creates new VPV & service with udpated vnet profile
+func (vs *VoltService) updateVnetProfile(deviceID string) {
+
+	logger.Info(ctx, "Update Vnet Profile Triggering", log.Fields{"Service": vs.Name, "US": vs.UsHSIAFlowsApplied, "DS": vs.DsHSIAFlowsApplied})
+
+	nvs := VoltService{}
+	nvs.VoltServiceCfg = vs.VoltServiceCfg
+	nvs.Device = vs.Device
+	nvs.Ipv4Addr = vs.Ipv4Addr
+	nvs.Ipv6Addr = vs.Ipv6Addr
+	nvs.UsMeterID = vs.UsMeterID
+	nvs.DsMeterID = vs.DsMeterID
+	nvs.AggDsMeterID = vs.AggDsMeterID
+	nvs.UsHSIAFlowsApplied = vs.UsHSIAFlowsApplied
+	nvs.DsHSIAFlowsApplied = vs.DsHSIAFlowsApplied
+	nvs.UsDhcpFlowsApplied = vs.UsDhcpFlowsApplied
+	nvs.DsDhcpFlowsApplied = vs.DsDhcpFlowsApplied
+	nvs.IgmpFlowsApplied = vs.IgmpFlowsApplied
+	nvs.Icmpv6FlowsApplied = vs.Icmpv6FlowsApplied
+	nvs.PendingFlows = vs.PendingFlows
+	nvs.AssociatedFlows = vs.AssociatedFlows
+	nvs.DeleteInProgress = vs.DeleteInProgress
+	nvs.ForceDelete = vs.ForceDelete
+	nvs.BwAvailInfo = vs.BwAvailInfo
+	nvs.UpdateInProgress = vs.UpdateInProgress
+
+	if nvs.DeleteInProgress {
+		logger.Warnw(ctx, "Skipping Service Migration. Service Delete in Progress", log.Fields{"Device": deviceID, "Service": vs.Name, "Vnet": vs.VnetID})
+		return
+	}
+
+	metadata := vs.Metadata.(*MigrateServiceMetadata)
+	oldVnetID := vs.VnetID
+	nvs.VnetID = metadata.NewVnetID
+	id := metadata.RequestID
+	oldSrvName := vs.Name
+
+	if metadata == nil || metadata.NewVnetID == "" {
+		logger.Errorw(ctx, "Migrate Service Metadata not found. Dropping vnet profile update request", log.Fields{"Service": vs.Name, "Vnet": vs.VnetID})
+		return
+	}
+
+	//First add the new service and then only delete the old service
+	// Since if post del service in case of pod crash or reboot, the service data will be lost
+	va := GetApplication()
+	msr := va.getMigrateServicesRequest(deviceID, oldVnetID, id)
+	vnets := strings.Split(metadata.NewVnetID, "-")
+	svlan, _ := strconv.Atoi(vnets[0])
+	nvs.SVlan = of.VlanType(svlan)
+	nvs.UpdateInProgress = false
+	nvs.Metadata = nil
+	nvs.Trigger = ServiceVlanUpdate
+
+	svcName := vs.Port + "-" + strconv.FormatUint(uint64(nvs.SVlan), 10) + "-"
+	svcName = svcName + strconv.FormatUint(uint64(vs.CVlan), 10) + "-"
+	nvs.Name = svcName + strconv.FormatUint(uint64(vs.TechProfileID), 10)
+
+	//TODO:Nav Pass a copy, not the pointer
+	logger.Infow(ctx, "Add New Service Triggering", log.Fields{"Service": nvs.Name, "US": nvs.UsHSIAFlowsApplied, "DS": nvs.DsHSIAFlowsApplied, "DelFlag": nvs.DeleteInProgress})
+	if err := va.AddService(nvs.VoltServiceCfg, &nvs.VoltServiceOper); err != nil {
+		logger.Warnw(ctx, "Add New Service Failed", log.Fields{"Service": nvs.Name, "Error": err})
+	}
+	logger.Infow(ctx, "Add New Service Triggered", log.Fields{"Service": nvs.Name, "US": nvs.UsHSIAFlowsApplied, "DS": nvs.DsHSIAFlowsApplied, "DelFlag": nvs.DeleteInProgress})
+
+	msr.ServicesList[oldSrvName] = true
+	va.updateMigrateServicesRequest(deviceID, oldVnetID, id, msr)
+	msr.WriteToDB()
+
+	logger.Infow(ctx, "Del Old Service Triggering", log.Fields{"Service": oldSrvName, "US": vs.UsHSIAFlowsApplied, "DS": vs.DsHSIAFlowsApplied, "DelFlag": vs.DeleteInProgress})
+	va.DelService(oldSrvName, true, nil, true)
+	logger.Infow(ctx, "Del Old Service Triggered", log.Fields{"Service": oldSrvName, "US": vs.UsHSIAFlowsApplied, "DS": vs.DsHSIAFlowsApplied, "DelFlag": vs.DeleteInProgress})
+	msr.serviceMigrated(oldSrvName)
+}
+
+//serviceMigrated - called on successful service updation
+// Removes the service entry from servicelist and deletes the request on process completion
+func (msr *MigrateServicesRequest) serviceMigrated(serviceName string) {
+
+	msr.MigrateServicesLock.Lock()
+	defer msr.MigrateServicesLock.Unlock()
+
+	delete(msr.ServicesList, serviceName)
+
+	if len(msr.ServicesList) == 0 {
+		_ = db.DelMigrateServicesReq(msr.DeviceID, msr.GetMsrKey())
+		return
+	}
+	msr.WriteToDB()
+	//TODO:Nav - Need for any Response to SubMgr?
+}
+
+//TriggerPendingMigrateServicesReq - trigger pending service request
+func (va *VoltApplication) TriggerPendingMigrateServicesReq(device string) {
+	va.FetchAndProcessAllMigrateServicesReq(device, storeAndProcessMigrateSrvRequest)
+}
+
+//FetchAndProcessAllMigrateServicesReq - fetch all pending migrate services req from DB and process based on provided func
+func (va *VoltApplication) FetchAndProcessAllMigrateServicesReq(device string, msrAction func(*MigrateServicesRequest)) {
+
+	msrList, _ := db.GetAllMigrateServicesReq(device)
+	for _, msr := range msrList {
+		b, ok := msr.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		msr := va.createMigrateServicesFromString(b)
+		msrAction(msr)
+		logger.Warnw(ctx, "Triggering Pending Migrate Services Req", log.Fields{"OldVnet": msr.OldVnetID, "NewVnet": msr.NewVnetID, "Device": device, "PendingProfiles": len(msr.ServicesList)})
+
+	}
+}
+
+// createMigrateServicesFromString to create Service from string
+func (va *VoltApplication) createMigrateServicesFromString(b []byte) *MigrateServicesRequest {
+	var msr MigrateServicesRequest
+	if err := json.Unmarshal(b, &msr); err == nil {
+		logger.Debugw(ctx, "Adding Migrate Services Request From Db", log.Fields{"Vlan": msr.OldVnetID})
+
+	} else {
+		logger.Warn(ctx, "Unmarshal failed")
+	}
+	return &msr
+}
+
+//storeAndProcessMigrateSrvRequest - stores the msr info in device obj and triggers req
+func storeAndProcessMigrateSrvRequest(msr *MigrateServicesRequest) {
+	d := GetApplication().GetDevice(msr.DeviceID)
+	d.AddMigratingServices(msr)
+	msr.ProcessMigrateServicesProfRequest()
+}
+
+//forceUpdateAllServices - force udpate services with new vnet profile
+func forceUpdateAllServices(msr *MigrateServicesRequest) {
+	for srv := range msr.ServicesList {
+		if vsIntf, ok := GetApplication().ServiceByName.Load(srv); ok {
+			vsIntf.(*VoltService).updateVnetProfile(msr.DeviceID)
+		}
+	}
+	_ = db.DelMigrateServicesReq(msr.DeviceID, msr.GetMsrKey())
+}
+
+//DeepEqualServicecfg - checks if the given service cfgs are same
+func (va *VoltApplication) DeepEqualServicecfg(evs *VoltServiceCfg, nvs *VoltServiceCfg) bool {
+	if nvs.Name != evs.Name {
+		return false
+	}
+	if nvs.UniVlan != evs.UniVlan {
+		return false
+	}
+	if nvs.CVlan != evs.CVlan {
+		return false
+	}
+	if nvs.SVlan != evs.SVlan {
+		return false
+	}
+	if nvs.SVlanTpid != 0 && nvs.SVlanTpid != evs.SVlanTpid {
+		return false
+	}
+	if !util.IsPbitSliceSame(nvs.Pbits, evs.Pbits) {
+		return false
+	}
+	if !reflect.DeepEqual(nvs.DsRemarkPbitsMap, evs.DsRemarkPbitsMap) {
+		return false
+	}
+	if nvs.TechProfileID != evs.TechProfileID {
+		return false
+	}
+	if nvs.CircuitID != evs.CircuitID {
+		return false
+	}
+	if !bytes.Equal(nvs.RemoteID, evs.RemoteID) {
+		return false
+	}
+	if nvs.Port != evs.Port {
+		return false
+	}
+	if nvs.PonPort != evs.PonPort {
+		return false
+	}
+	if evs.MacLearning == MacLearningNone && !util.MacAddrsMatch(nvs.MacAddr, evs.MacAddr) {
+		return false
+	}
+	if nvs.IsOption82Disabled != evs.IsOption82Disabled {
+		return false
+	}
+	if nvs.IgmpEnabled != evs.IgmpEnabled {
+		return false
+	}
+	if nvs.McastService != evs.McastService {
+		return false
+	}
+	if nvs.ONTEtherTypeClassification != evs.ONTEtherTypeClassification {
+		return false
+	}
+	if nvs.UsMeterProfile != evs.UsMeterProfile {
+		return false
+	}
+	if nvs.DsMeterProfile != evs.DsMeterProfile {
+		return false
+	}
+	if nvs.AggDsMeterProfile != evs.AggDsMeterProfile {
+		return false
+	}
+	if nvs.VnetID != evs.VnetID {
+		return false
+	}
+	if nvs.MvlanProfileName != evs.MvlanProfileName {
+		return false
+	}
+	if nvs.RemoteIDType != evs.RemoteIDType {
+		return false
+	}
+	if nvs.SchedID != evs.SchedID {
+		return false
+	}
+	if nvs.AllowTransparent != evs.AllowTransparent {
+		return false
+	}
+	if nvs.EnableMulticastKPI != evs.EnableMulticastKPI {
+		return false
+	}
+	if nvs.DataRateAttr != evs.DataRateAttr {
+		return false
+	}
+	if nvs.MinDataRateUs != evs.MinDataRateUs {
+		return false
+	}
+	if nvs.MinDataRateDs != evs.MinDataRateDs {
+		return false
+	}
+	if nvs.MaxDataRateUs != evs.MaxDataRateUs {
+		return false
+	}
+	if nvs.MaxDataRateDs != evs.MaxDataRateDs {
+		return false
+	}
+
+	return true
+}
+
+//TriggerAssociatedFlowDelete - re-trigger service flow delete for pending delete flows
+func (vs *VoltService) TriggerAssociatedFlowDelete() bool {
+
+	//Clear the Flows flag if already set
+	//This case happens only in case of some race condition
+	if vs.UsHSIAFlowsApplied {
+		if err := vs.DelUsHsiaFlows(); err != nil {
+			logger.Errorw(ctx, "DelUsHsiaFlows Failed", log.Fields{"Device": vs.Device, "Service": vs.Name, "Error": err})
+		}
+	}
+
+	if vs.DsHSIAFlowsApplied {
+		if err := vs.DelDsHsiaFlows(); err != nil {
+			logger.Errorw(ctx, "DelDsHsiaFlows Failed", log.Fields{"Device": vs.Device, "Service": vs.Name, "Error": err})
+		}
+	}
+
+	vs.ServiceLock.Lock()
+	cookieList := []uint64{}
+	for cookie := range vs.AssociatedFlows {
+		cookieList = append(cookieList, convertToUInt64(cookie))
+	}
+	vs.ServiceLock.Unlock()
+
+	if len(cookieList) == 0 {
+		return false
+	}
+
+	//Trigger Flow Delete
+	for _, cookie := range cookieList {
+		if vd := GetApplication().GetDevice(vs.Device); vd != nil {
+			flow := &of.VoltFlow{}
+			flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+			subFlow := of.NewVoltSubFlow()
+			subFlow.Cookie = cookie
+			flow.SubFlows[cookie] = subFlow
+			logger.Infow(ctx, "Retriggering Service Delete Flow", log.Fields{"Device": vs.Device, "Service": vs.Name, "Cookie": cookie})
+			if err := vs.DelFlows(vd, flow); err != nil {
+				logger.Errorw(ctx, "DelFlows Failed", log.Fields{"Device": vs.Device, "Service": vs.Name, "Cookie": cookie, "Error": err})
+			}
+		}
+	}
+	return true
+}
+
+//triggerServiceInProgressInd - Indication is generated when Service is not provisioned after add serviec req from NB
+func (vs *VoltService) triggerServiceInProgressInd() {
+}
diff --git a/internal/pkg/application/timer.go b/internal/pkg/application/timer.go
new file mode 100644
index 0000000..1a80cb5
--- /dev/null
+++ b/internal/pkg/application/timer.go
@@ -0,0 +1,70 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"time"
+)
+
+//TimerType - type of timer used
+type TimerType string
+
+const (
+	tickTimer        TimerType = "TickTimer"
+	pendingPoolTimer TimerType = "PendingPoolTimer"
+)
+
+var timerMap = map[TimerType]bool{
+	tickTimer:        false,
+	pendingPoolTimer: false,
+}
+
+var timerChannels = make(map[TimerType](chan bool))
+
+// TimerCfg structure
+type TimerCfg struct {
+	tick time.Duration
+}
+
+// Start to start timer
+func (va *VoltApplication) Start(cfg TimerCfg, timerType TimerType) {
+	if timerMap[timerType] {
+		logger.Warn(ctx, "Duplicate Timer!!! Timer already running")
+		return
+	}
+	timerMap[timerType] = true
+	timerChannels[timerType] = make(chan bool)
+	for {
+		select {
+		case <-time.After(cfg.tick):
+			switch timerType {
+			case tickTimer:
+				va.Tick()
+			case pendingPoolTimer:
+				va.removeExpiredGroups()
+			}
+		case <- timerChannels[timerType]:
+			return
+		}
+	}
+}
+
+// StopTimer to stop timers
+func StopTimer() {
+	for _, ch := range timerChannels {
+		ch <- true
+	}
+}
diff --git a/internal/pkg/application/util.go b/internal/pkg/application/util.go
new file mode 100644
index 0000000..866e12b
--- /dev/null
+++ b/internal/pkg/application/util.go
@@ -0,0 +1,60 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"errors"
+	"strconv"
+
+	"github.com/google/gopacket/layers"
+)
+
+const (
+	//EtherType8100 - EtherType dot1q
+	EtherType8100 uint8 = 0
+	//EtherType88a8 - EtherType dot1ad
+	EtherType88a8 uint8 = 1
+	//EtherType9100 - EtherType dot1ad doubleTag
+	EtherType9100 uint8 = 2
+	//EtherType9200 - EtherType dot1q doubleTag
+	EtherType9200 uint8 = 3
+)
+
+//GetMetadataForL2Protocol - returns metadata value for provide ethertype
+func GetMetadataForL2Protocol(etherType layers.EthernetType) (uint8, error) {
+	switch etherType {
+	case layers.EthernetTypeDot1Q:
+		return EtherType8100, nil
+	case layers.EthernetTypeQinQ:
+		return EtherType88a8, nil
+	case layers.EthernetTypeDot1QDoubleTag:
+		return EtherType9100, nil
+	case layers.EthernetTypeQinQDoubleTag:
+		return EtherType9200, nil
+	default:
+		return 0, errors.New("EtherType not supported")
+	}
+}
+
+func convertToUInt64(data string) uint64 {
+
+	value, err := strconv.ParseUint(data, 10, 64)
+	if err != nil {
+		return 0
+	}
+	return value
+
+}
diff --git a/internal/pkg/application/vnets.go b/internal/pkg/application/vnets.go
new file mode 100644
index 0000000..cf2d1fc
--- /dev/null
+++ b/internal/pkg/application/vnets.go
@@ -0,0 +1,3138 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package application
+
+import (
+	"encoding/json"
+	"errors"
+	"net"
+	infraerrorCodes "voltha-go-controller/internal/pkg/errorcodes"
+	"strconv"
+	"sync"
+	"time"
+
+	"github.com/google/gopacket"
+	"github.com/google/gopacket/layers"
+	"go.uber.org/atomic"
+
+	"voltha-go-controller/internal/pkg/controller"
+	cntlr "voltha-go-controller/internal/pkg/controller"
+	"voltha-go-controller/database"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/util"
+	errorCodes "voltha-go-controller/internal/pkg/errorcodes"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	// ICMPv6ArpGroupID constant
+	ICMPv6ArpGroupID uint32 = 1
+
+	// Radisys vendor id constant
+	Radisys string = "Radisys"
+)
+
+var (
+	//BroadcastMAC - Broadcast MAC Address
+	BroadcastMAC, _ = net.ParseMAC("ff:ff:ff:ff:ff:ff")
+)
+
+// NonZeroMacAddress utility to identify if the MAC address is non-zero.
+// We use zero MAC address as an unset MAC address
+func NonZeroMacAddress(h net.HardwareAddr) bool {
+	for i := 0; i < 6; i++ {
+		if h[i] != 0 {
+			return true
+		}
+	}
+	return false
+}
+
+// VNET package manages the different virtual networks that are part of the
+// the network. In the case of VOLT, the networks can be single tagged or
+// double tagged networks. In addition, the networks may be used for unicast
+// and multicast traffic. The unicast traffic further has two models, the
+// 1:1 and N:1 model. In case of a 1:1 model, the outer tag is same for many
+// subscribers and the inner tag is unique to each subscriber for the same
+// outer tag. The N:1 uses the same inner and outer tags, or for that matter
+// a single tag that can also be shared by subscribers. The VNET implementation
+// manages all these possibilities and the associated configuration.
+
+const (
+	// PbitMatchNone constant
+	PbitMatchNone of.PbitType = 8
+	// PbitMatchAll constant
+	PbitMatchAll of.PbitType = 0xFF
+)
+
+// SVlan        - Value of the outer tag if double tagged or the only tag if single
+//                tagged
+// SVlanTpid    - SVlan Tag Protocol Identifier
+// CVlan        - Value of the inner tag. Set to VlanNone if single tagged
+// DhcpRelay    - Set to true if the DHCP relay is enabled on the virtual network
+// MacLearning  - Set to true if the flows should include MAC address
+// UsDhcpPbit   - The pbit used for US DHCP packets
+// DsDhcpPbit   - The pbit used for DS DHCP packets
+
+// VnetConfig structure
+type VnetConfig struct {
+	Name                       string
+	SVlan                      of.VlanType
+	CVlan                      of.VlanType
+	UniVlan                    of.VlanType
+	SVlanTpid                  layers.EthernetType
+	DhcpRelay                  bool
+	ArpLearning                bool
+	MacLearning                MacLearningType
+	PppoeIa                    bool
+	ONTEtherTypeClassification int
+	VlanControl                VlanControl
+	Encapsulation              string
+	UsDhcpPbit                 []of.PbitType
+	DsDhcpPbit                 []of.PbitType
+	UsIGMPPbit                 []of.PbitType
+	DsIGMPPbit                 []of.PbitType
+	DevicesList                []string //List of serial number of devices on which this vnet is applied
+	AllowTransparent           bool
+	CtrlPktPbitRemark          map[of.PbitType]of.PbitType
+}
+
+// VnetOper structure
+type VnetOper struct {
+	PendingDeleteFlow     map[string]map[string]bool
+	DeleteInProgress      bool
+	PendingDeviceToDelete string
+	VnetLock              sync.RWMutex    `json:"-"`
+	VnetPortLock          sync.RWMutex    `json:"-"`
+	AssociatedPorts       map[string]bool `json:"-"`
+}
+
+// VoltVnet sructure
+type VoltVnet struct {
+	VnetConfig
+	VnetOper
+	Version string
+}
+
+const (
+	// EncapsulationPPPoEIA constant
+	EncapsulationPPPoEIA string = "PPPoE-IA"
+	// EncapsulationPPPoE constant
+	EncapsulationPPPoE string = "PPPoE"
+	// EncapsulationIPoE constant
+	EncapsulationIPoE string = "IPoE"
+)
+
+// NewVoltVnet is constructor for the VNET structure
+func NewVoltVnet(cfg VnetConfig) *VoltVnet {
+	var vv VoltVnet
+	vv.VnetConfig = cfg
+	if vv.PendingDeleteFlow == nil {
+		vv.PendingDeleteFlow = make(map[string]map[string]bool)
+	}
+	vv.DeleteInProgress = false
+	if cfg.Encapsulation == EncapsulationPPPoEIA {
+		vv.PppoeIa = true
+	}
+	vv.AssociatedPorts = make(map[string]bool)
+	return &vv
+}
+
+//associatePortToVnet - associate a port to Vnet
+func (vv *VoltVnet) associatePortToVnet(port string) {
+	vv.VnetPortLock.Lock()
+	if vv.AssociatedPorts == nil {
+		vv.AssociatedPorts = make(map[string]bool)
+	}
+	vv.AssociatedPorts[port] = true
+	vv.VnetPortLock.Unlock()
+}
+
+//disassociatePortFromVnet - disassociate a port from Vnet and return true if the association map is empty
+func (vv *VoltVnet) disassociatePortFromVnet(device string, port string) {
+	vv.VnetPortLock.Lock()
+	delete(vv.AssociatedPorts, port)
+	logger.Infow(ctx, "Disassociated Port from Vnet", log.Fields{"Device": device, "Port": port, "Vnet": vv.Name, "PendingDeleteFlow": vv.PendingDeleteFlow, "AssociatedPorts": vv.AssociatedPorts, "DeleteFlag": vv.DeleteInProgress})
+	vv.VnetPortLock.Unlock()
+
+	if vv.DeleteInProgress {
+		if !vv.isAssociatedPortsPresent() {
+			if len(vv.PendingDeleteFlow[device]) == 0 {
+				logger.Warnw(ctx, "Deleting Vnet", log.Fields{"Name": vv.Name})
+				GetApplication().deleteVnetConfig(vv)
+				_ = db.DelVnet(vv.Name)
+			} else {
+				logger.Warnw(ctx, "Skipping Del Vnet", log.Fields{"Name": vv.Name, "PendingDeleteFlow": vv.PendingDeleteFlow})
+			}
+		} else {
+			vv.VnetPortLock.RLock()
+			logger.Warnw(ctx, "Skipping Del Vnet", log.Fields{"Name": vv.Name, "AssociatedPorts": vv.AssociatedPorts})
+			vv.VnetPortLock.RUnlock()
+		}
+	}
+}
+
+func (vv *VoltVnet) isAssociatedPortsPresent() bool {
+	vv.VnetPortLock.RLock()
+	defer vv.VnetPortLock.RUnlock()
+	return len(vv.AssociatedPorts) != 0
+}
+
+// WriteToDb commit the VNET to the database
+func (vv *VoltVnet) WriteToDb() {
+
+	if vv.DeleteInProgress {
+		logger.Warnw(ctx, "Skipping Redis Update for Vnet, Vnet delete in progress", log.Fields{"Vnet": vv.Name})
+		return
+	}
+	vv.ForceWriteToDb()
+}
+
+//ForceWriteToDb force commit a vnet to the DB
+func (vv *VoltVnet) ForceWriteToDb() {
+	vv.VnetPortLock.RLock()
+	defer vv.VnetPortLock.RUnlock()
+	vv.Version = database.PresentVersionMap[database.VnetPath]
+	logger.Debugw(ctx, "Updating VNET....", log.Fields{"vnet": vv})
+	if b, err := json.Marshal(vv); err == nil {
+		if err:= db.PutVnet(vv.Name, string(b)); err != nil {
+			logger.Warnw(ctx, "Add Vnet to DB failed", log.Fields{"vnet name": vv.Name, "Error": err})
+		}
+	}
+}
+
+// VnetKey creates the key using the two VLAN tags
+// We append the two VLAN tags to create a single key
+func VnetKey(otag of.VlanType, itag of.VlanType, utag of.VlanType) string {
+	return strconv.Itoa(int(otag)) + "-" + strconv.Itoa(int(itag)) + "-" + strconv.Itoa(int(utag))
+}
+
+// GetVnet get VNET configuration related functionality associated with VOLT application
+func (va *VoltApplication) GetVnet(otag of.VlanType, itag of.VlanType, utag of.VlanType) *VoltVnet {
+	// When matching VNET, it is expected to match first just the outer
+	// tag, and then the combination to make sure there is no conflict
+	// for the new configuration.
+	if vnet, ok := va.VnetsByTag.Load(VnetKey(otag, of.VlanNone, utag)); ok {
+		return vnet.(*VoltVnet)
+	}
+	if vnet, ok := va.VnetsByTag.Load(VnetKey(otag, itag, utag)); ok {
+		return vnet.(*VoltVnet)
+	}
+	return nil
+}
+
+// The VNET may also be assigned name for easier references. For now,
+// the VNET is mainly identified by the two VLANs.
+
+// GetVnetByName to get vnet by name
+func (va *VoltApplication) GetVnetByName(name string) *VoltVnet {
+	if vnet, ok := va.VnetsByName.Load(name); ok {
+		return vnet.(*VoltVnet)
+	}
+	return nil
+}
+
+// storeVnetConfig to store vnet config
+func (va *VoltApplication) storeVnetConfig(cfg VnetConfig, vv *VoltVnet) {
+
+	var vnetMap *util.ConcurrentMap
+
+	va.VnetsByTag.Store(VnetKey(cfg.SVlan, cfg.CVlan, cfg.UniVlan), vv)
+	va.VnetsByName.Store(cfg.Name, vv)
+
+	if vnetMapIntf, ok := va.VnetsBySvlan.Get(vv.SVlan); !ok {
+		vnetMap = util.NewConcurrentMap()
+	} else {
+		vnetMap = vnetMapIntf.(*util.ConcurrentMap)
+	}
+	vnetMap.Set(vv, true)
+	va.VnetsBySvlan.Set(vv.SVlan, vnetMap)
+}
+
+// deleteVnetConfig to delete vnet config
+func (va *VoltApplication) deleteVnetConfig(vnet *VoltVnet) {
+	va.VnetsByTag.Delete(VnetKey(vnet.SVlan, vnet.CVlan, vnet.UniVlan))
+	va.VnetsByName.Delete(vnet.Name)
+
+	if vnetMapIntf, ok := va.VnetsBySvlan.Get(vnet.SVlan); ok {
+		vnetMap := vnetMapIntf.(*util.ConcurrentMap)
+		vnetMap.Remove(vnet)
+		va.VnetsBySvlan.Set(vnet.SVlan, vnetMap)
+	}
+}
+
+// AddVnet to add a VNET to the list of VNETs configured.
+func (va *VoltApplication) AddVnet(cfg VnetConfig, oper *VnetOper) error {
+
+	AppMutex.VnetMutex.Lock()
+	var vv *VoltVnet
+	devicesToHandle := []string{}
+	vv = va.GetVnetByName(cfg.Name)
+	if vv != nil {
+		//Could be for another OLT or could be case of backup-restore
+		for _, serialNum := range cfg.DevicesList {
+			if isDeviceInList(serialNum, vv.DevicesList) {
+				//This is backup restore scenario, just update the profile
+				logger.Info(ctx, "Add Vnet : Profile Name already exists with OLT, update-the-profile")
+				continue
+			}
+			devicesToHandle = append(devicesToHandle, serialNum)
+		}
+		if len(devicesToHandle) == 0 {
+			logger.Debug(ctx, "Ignoring Duplicate VNET by name ", log.Fields{"Vnet": cfg.Name})
+			AppMutex.VnetMutex.Unlock()
+			return nil
+		}
+	}
+
+	if vv == nil {
+		vv = NewVoltVnet(cfg)
+		if oper != nil {
+			vv.PendingDeleteFlow = oper.PendingDeleteFlow
+			vv.DeleteInProgress = oper.DeleteInProgress
+			vv.AssociatedPorts = oper.AssociatedPorts
+			vv.PendingDeviceToDelete = oper.PendingDeviceToDelete
+		}
+		devicesToHandle = append(devicesToHandle, cfg.DevicesList...)
+	} else {
+		vv.DevicesList = append(vv.DevicesList, devicesToHandle...)
+	}
+
+	va.storeVnetConfig(cfg, vv)
+	vv.WriteToDb()
+
+	logger.Infow(ctx, "Added VNET TO DB", log.Fields{"cfg": cfg, "devicesToHandle": devicesToHandle})
+
+	//va.PushDevFlowForVlan(vv)
+	AppMutex.VnetMutex.Unlock()
+	return nil
+}
+
+// DelVnet to delete a VNET from the list of VNETs configured
+func (va *VoltApplication) DelVnet(name, deviceSerialNum string) error {
+	logger.Infow(ctx, "Deleting Vnet", log.Fields{"Vnet": name})
+	AppMutex.VnetMutex.Lock()
+	if vnetIntf, ok := va.VnetsByName.Load(name); ok {
+		vnet := vnetIntf.(*VoltVnet)
+		//Delete from mvp list
+		vnet.DevicesList = util.RemoveFromSlice(vnet.DevicesList, deviceSerialNum)
+
+		va.DeleteDevFlowForVlanFromDevice(vnet, deviceSerialNum)
+		if len(vnet.DevicesList) == 0 {
+			vnet.DeleteInProgress = true
+			vnet.PendingDeviceToDelete = deviceSerialNum
+			vnet.ForceWriteToDb()
+			vnet.VnetPortLock.RLock()
+			if len(vnet.PendingDeleteFlow) == 0 && !vnet.isAssociatedPortsPresent() {
+				logger.Warnw(ctx, "Deleting Vnet", log.Fields{"Name": vnet.Name, "AssociatedPorts": vnet.AssociatedPorts, "PendingDelFlows": vnet.PendingDeleteFlow})
+				va.deleteVnetConfig(vnet)
+				_ = db.DelVnet(vnet.Name)
+			} else {
+				logger.Warnw(ctx, "Skipping Del Vnet", log.Fields{"Name": vnet.Name, "AssociatedPorts": vnet.AssociatedPorts, "PendingDelFlows": vnet.PendingDeleteFlow})
+			}
+			vnet.VnetPortLock.RUnlock()
+		} else {
+			//Update the devicelist in db
+			vnet.WriteToDb()
+		}
+	}
+	//TODO: if no vnets are present on device remove icmpv6 group from device
+	AppMutex.VnetMutex.Unlock()
+	return nil
+}
+
+// UpdateVnet to update the VNET with associated service count
+func (va *VoltApplication) UpdateVnet(vv *VoltVnet) error {
+	va.storeVnetConfig(vv.VnetConfig, vv)
+	vv.WriteToDb()
+	logger.Infow(ctx, "Updated VNET TO DB", log.Fields{"vv": vv.VnetConfig})
+	return nil
+}
+
+// ------------------------------------------------------------
+// Manifestation of a VNET on a port is handled below
+// ------------------------------------------------------------
+//
+// The VNET on a port handles everything that is done for a VNET
+// such as DHCP relay state machine, MAC addresses, IP addresses
+// learnt, so on.
+
+// DhcpStatus type
+type DhcpStatus uint8
+
+const (
+	// DhcpStatusNone constant
+	DhcpStatusNone DhcpStatus = 0
+	// DhcpStatusAcked constant
+	DhcpStatusAcked DhcpStatus = 1
+	// DhcpStatusNacked constant
+	DhcpStatusNacked DhcpStatus = 2
+	// EthTypeNone constant
+	EthTypeNone int = 0
+	// EthTypeIPoE constant
+	EthTypeIPoE int = 1
+	// EthTypePPPoE constant
+	EthTypePPPoE int = 2
+)
+
+// VoltPortVnet structure
+type VoltPortVnet struct {
+	Device                     string
+	Port                       string
+	PonPort                    uint32
+	VnetName                   string
+	SVlan                      of.VlanType
+	CVlan                      of.VlanType
+	UniVlan                    of.VlanType
+	SVlanTpid                  layers.EthernetType
+	DhcpRelay                  bool
+	ArpRelay                   bool
+	PppoeIa                    bool
+	MacLearning                MacLearningType
+	DhcpStatus                 DhcpStatus
+	DhcpExpiryTime             time.Time
+	Dhcp6ExpiryTime            time.Time
+	FlowsApplied               bool
+	services                   sync.Map
+	servicesCount              *atomic.Uint64
+	Ipv4Addr                   net.IP
+	Ipv6Addr                   net.IP
+	MacAddr                    net.HardwareAddr
+	LearntMacAddr              net.HardwareAddr
+	CircuitID                  []byte //Will not be used
+	RemoteID                   []byte //Will not be used
+	IsOption82Disabled         bool   //Will not be used
+	RelayState                 DhcpRelayState
+	PPPoeState                 PppoeIaState
+	RelayStatev6               Dhcpv6RelayState
+	IgmpEnabled                bool
+	IgmpFlowsApplied           bool
+	McastService               bool
+	ONTEtherTypeClassification int
+	VlanControl                VlanControl
+	MvlanProfileName           string
+	Version                    string
+	McastTechProfileID         uint16
+	McastPbit                  of.PbitType
+	McastUsMeterID             uint32
+	AllowTransparent           bool
+	VpvLock                    sync.Mutex `json:"-"`
+	SchedID                    int
+	DHCPv6DUID                 [MaxLenDhcpv6DUID]byte
+	PendingFlowLock            sync.RWMutex `json:"-"`
+	PendingDeleteFlow          map[string]bool
+	DeleteInProgress           bool
+	Blocked                    bool
+	DhcpPbit                   of.PbitType
+}
+
+//VlanControl vlan control type
+type VlanControl uint8
+
+const (
+	// None constant
+	// ONU and OLT will passthrough UNIVLAN as is to BNG
+	None VlanControl = iota
+
+	// ONUCVlanOLTSVlan constant
+	// Tagged traffic, ONU will replace UNIVLAN with CVLAN and OLT will add SVLAN
+	// Untagged traffic, ONU will add CVLAN and OLT will add SVLAN
+	ONUCVlanOLTSVlan
+
+	// OLTCVlanOLTSVlan constant
+	// Tagged traffic, ONU will passthrough UNIVLAN as is to OLT and
+	// OLT will replace UNIVLAN with CVLAN and add SVLAN
+	OLTCVlanOLTSVlan
+
+	// ONUCVlan constant
+	// Tagged traffic, ONU will replace UNIVLAN with CVLAN
+	// Untagged traffic, ONU will add CVLAN
+	ONUCVlan
+
+	// OLTSVlan constant
+	// UnTagged traffic, OLT will add the SVLAN
+	OLTSVlan
+)
+
+// NewVoltPortVnet is constructor for VoltPortVnet
+func NewVoltPortVnet(vnet *VoltVnet) *VoltPortVnet {
+	var vpv VoltPortVnet
+
+	vpv.VnetName = vnet.Name
+	vpv.SVlan = vnet.SVlan
+	vpv.CVlan = vnet.CVlan
+	vpv.UniVlan = vnet.UniVlan
+	vpv.SVlanTpid = vnet.SVlanTpid
+	vpv.DhcpRelay = vnet.DhcpRelay
+	vpv.DhcpStatus = DhcpStatusNone
+	vpv.PPPoeState = PppoeIaStateNone
+	vpv.ArpRelay = vnet.ArpLearning
+	vpv.PppoeIa = vnet.PppoeIa
+	vpv.VlanControl = vnet.VlanControl
+	vpv.ONTEtherTypeClassification = vnet.ONTEtherTypeClassification
+	vpv.AllowTransparent = vnet.AllowTransparent
+	vpv.FlowsApplied = false
+	vpv.IgmpEnabled = false
+	vpv.MacAddr, _ = net.ParseMAC("00:00:00:00:00:00")
+	vpv.LearntMacAddr, _ = net.ParseMAC("00:00:00:00:00:00")
+	// for OLTCVLAN SVLAN=CVLAN, UNIVLAN can differ.
+	if vpv.VlanControl == ONUCVlan {
+		vpv.CVlan = vpv.SVlan
+	}
+	// for OLTSVLAN  CVLAN=UNIVLAN , SVLAN can differ,
+	// hence assigning UNIVLAN to CVLAN, so that ONU will transparently forward the packet.
+	if vpv.VlanControl == OLTSVlan {
+		vpv.CVlan = vpv.UniVlan
+	}
+	vpv.servicesCount = atomic.NewUint64(0)
+	vpv.SchedID = 0
+	vpv.PendingDeleteFlow = make(map[string]bool)
+	vpv.DhcpPbit = vnet.UsDhcpPbit[0]
+	return &vpv
+}
+
+func (vpv *VoltPortVnet) setDevice(device string) {
+
+	if vpv.Device != device && vpv.Device != "" {
+		GetApplication().DisassociateVpvsFromDevice(device, vpv)
+		//TEMP:
+		vpv.printAssociatedVPVs(false)
+	}
+
+	logger.Infow(ctx, "Associating VPV and Device", log.Fields{"Device": vpv.Device, "Port": vpv.Port, "SVlan": vpv.SVlan})
+
+	vpv.Device = device
+	GetApplication().AssociateVpvsToDevice(device, vpv)
+	//TEMP:
+	vpv.printAssociatedVPVs(true)
+}
+
+//TODO - Nav - Temp
+func (vpv *VoltPortVnet) printAssociatedVPVs(add bool) {
+	logger.Infow(ctx, "Start----Printing all associated VPV", log.Fields{"Device": vpv.Device, "Add": add})
+	if vMap := GetApplication().GetAssociatedVpvsForDevice(vpv.Device, vpv.SVlan); vMap != nil {
+		vMap.Range(func(key, value interface{}) bool {
+			vpvEntry := key.(*VoltPortVnet)
+			logger.Infow(ctx, "Associated VPVs", log.Fields{"SVlan": vpvEntry.SVlan, "CVlan": vpvEntry.CVlan, "UniVlan": vpvEntry.UniVlan})
+			return true
+		})
+	}
+	logger.Infow(ctx, "End----Printing all associated VPV", log.Fields{"Device": vpv.Device, "Add": add})
+
+}
+
+// GetCircuitID : The interface to be satisfied by the VoltPortVnet to be a DHCP relay
+// session is implemented below. The main functions still remain in
+// the service.go file.
+func (vpv *VoltPortVnet) GetCircuitID() []byte {
+	return []byte(vpv.CircuitID)
+}
+
+// GetRemoteID to get remote id
+func (vpv *VoltPortVnet) GetRemoteID() []byte {
+	return []byte(vpv.RemoteID)
+}
+
+// GetDhcpState to get dhcp state
+func (vpv *VoltPortVnet) GetDhcpState() DhcpRelayState {
+	return vpv.RelayState
+}
+
+// SetDhcpState to set the dhcp state
+func (vpv *VoltPortVnet) SetDhcpState(state DhcpRelayState) {
+	vpv.RelayState = state
+}
+
+// GetPppoeIaState to get pppoeia state
+func (vpv *VoltPortVnet) GetPppoeIaState() PppoeIaState {
+	return vpv.PPPoeState
+}
+
+// SetPppoeIaState to set pppoeia state
+func (vpv *VoltPortVnet) SetPppoeIaState(state PppoeIaState) {
+	vpv.PPPoeState = state
+}
+
+// GetDhcpv6State to get dhcpv6 state
+func (vpv *VoltPortVnet) GetDhcpv6State() Dhcpv6RelayState {
+	return vpv.RelayStatev6
+}
+
+// SetDhcpv6State to set dhcpv6 state
+func (vpv *VoltPortVnet) SetDhcpv6State(state Dhcpv6RelayState) {
+	vpv.RelayStatev6 = state
+}
+
+// DhcpResultInd for dhcp result indication
+func (vpv *VoltPortVnet) DhcpResultInd(res *layers.DHCPv4) {
+	vpv.ProcessDhcpResult(res)
+}
+
+// Dhcpv6ResultInd for dhcpv6 result indication
+func (vpv *VoltPortVnet) Dhcpv6ResultInd(ipv6Addr net.IP, leaseTime uint32) {
+	vpv.ProcessDhcpv6Result(ipv6Addr, leaseTime)
+}
+
+// GetNniVlans to get nni vlans
+func (vpv *VoltPortVnet) GetNniVlans() (uint16, uint16) {
+	switch vpv.VlanControl {
+	case ONUCVlanOLTSVlan,
+		OLTCVlanOLTSVlan:
+		return uint16(vpv.SVlan), uint16(vpv.CVlan)
+	case ONUCVlan,
+		None:
+		return uint16(vpv.SVlan), uint16(of.VlanNone)
+	case OLTSVlan:
+		return uint16(vpv.SVlan), uint16(of.VlanNone)
+	}
+	return uint16(of.VlanNone), uint16(of.VlanNone)
+}
+
+// GetService to get service
+func (vpv *VoltPortVnet) GetService(name string) (*VoltService, bool) {
+	service, ok := vpv.services.Load(name)
+	if ok {
+		return service.(*VoltService), ok
+	}
+	return nil, ok
+}
+
+// AddService to add service
+func (vpv *VoltPortVnet) AddService(service *VoltService) {
+	vpv.services.Store(service.Name, service)
+	vpv.servicesCount.Inc()
+	logger.Infow(ctx, "Service added/updated to VPV", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "Service": service.Name, "Count": vpv.servicesCount.Load()})
+}
+
+// DelService to delete service
+func (vpv *VoltPortVnet) DelService(service *VoltService) {
+	vpv.services.Delete(service.Name)
+	vpv.servicesCount.Dec()
+
+	// If the only Igmp Enabled service is removed, remove the Igmp trap flow along with it
+	if service.IgmpEnabled {
+		if err := vpv.DelIgmpFlows(); err != nil {
+			statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+			vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+		}
+
+		vpv.IgmpEnabled = false
+	}
+	logger.Infow(ctx, "Service deleted from VPV", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "Service": service.Name, "Count": vpv.servicesCount.Load()})
+}
+
+// ProcessDhcpResult to process dhcp results
+func (vpv *VoltPortVnet) ProcessDhcpResult(res *layers.DHCPv4) {
+	msgType := DhcpMsgType(res)
+	if msgType == layers.DHCPMsgTypeAck {
+		vpv.ProcessDhcpSuccess(res)
+	} else if msgType == layers.DHCPMsgTypeNak {
+		vpv.DhcpStatus = DhcpStatusNacked
+	}
+	vpv.WriteToDb()
+}
+
+// ProcessDhcpSuccess : Learn the IPv4 address allocated to the services and update the
+// the services with the same. This also calls for adding flows
+// for the services as the DHCP procedure is completed
+func (vpv *VoltPortVnet) ProcessDhcpSuccess(res *layers.DHCPv4) {
+	vpv.DhcpStatus = DhcpStatusAcked
+	vpv.Ipv4Addr, _ = GetIpv4Addr(res)
+	logger.Infow(ctx, "Received IPv4 Address", log.Fields{"IP Address": vpv.Ipv4Addr.String()})
+	logger.Infow(ctx, "Services Configured", log.Fields{"Count": vpv.servicesCount.Load()})
+
+	vpv.services.Range(vpv.updateIPv4AndProvisionFlows)
+	vpv.ProcessDhcpv4Options(res)
+}
+
+// ProcessDhcpv4Options : Currently we process lease time and store the validity of the
+// IP address allocated.
+func (vpv *VoltPortVnet) ProcessDhcpv4Options(res *layers.DHCPv4) {
+	for _, o := range res.Options {
+		switch o.Type {
+		case layers.DHCPOptLeaseTime:
+			leasetime := GetIPv4LeaseTime(o)
+			vpv.DhcpExpiryTime = time.Now().Add((time.Duration(leasetime) * time.Second))
+			logger.Infow(ctx, "Lease Expiry Set", log.Fields{"Time": vpv.DhcpExpiryTime})
+		}
+	}
+}
+
+// ProcessDhcpv6Result : Read the IPv6 address allocated to the device and store it on the
+// VNET. The same IPv6 address is also passed to the services. When a
+// service is fetched all the associated information such as MAC address,
+// IPv4 address and IPv6 addresses can be provided.
+func (vpv *VoltPortVnet) ProcessDhcpv6Result(ipv6Addr net.IP, leaseTime uint32) {
+	// TODO: Status based hanlding of flows
+	vpv.Dhcp6ExpiryTime = time.Now().Add((time.Duration(leaseTime) * time.Second))
+	vpv.Ipv6Addr = ipv6Addr
+
+	vpv.services.Range(vpv.updateIPv6AndProvisionFlows)
+	vpv.WriteToDb()
+}
+
+// AddSvcUsMeterToDevice to add service upstream meter info to device
+func AddSvcUsMeterToDevice(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	logger.Info(ctx, "Adding upstream meter profile to device", log.Fields{"ServiceName": svc.Name})
+	if device, _ := GetApplication().GetDeviceFromPort(svc.Port); device != nil {
+		GetApplication().AddMeterToDevice(svc.Port, device.Name, svc.UsMeterID, 0)
+		return true
+	}
+	logger.Errorw(ctx, "Dropping US Meter request: Device not found", log.Fields{"Service": svc})
+	return false
+}
+
+// PushFlowsForPortVnet - triggers flow construction and push for provided VPV
+func (vpv *VoltPortVnet) PushFlowsForPortVnet(d *VoltDevice) {
+
+	vp := d.GetPort(vpv.Port)
+
+	//Ignore if UNI port is not found or not UP
+	if vp == nil || vp.State != PortStateUp {
+		logger.Warnw(ctx, "Ignoring Vlan UP Ind for VPV: Port Not Found/Ready", log.Fields{"Port": vp})
+		return
+	}
+
+	if vpv.PonPort != 0xFF && vpv.PonPort != vp.PonPort {
+		logger.Errorw(ctx, "UNI port discovered on wrong PON Port. Dropping Flow Configuration for VPV", log.Fields{"Device": d.Name, "Port": vpv.Port, "DetectedPon": vp.PonPort, "ExpectedPon": vpv.PonPort, "Vnet": vpv.VnetName})
+		return
+	}
+
+	//Disable the flag so that flows can be pushed again
+	// vpv.IgmpFlowsApplied = false
+	// vpv.DsFlowsApplied = false
+	// vpv.UsFlowsApplied = false
+	vpv.VpvLock.Lock()
+	vpv.PortUpInd(d, vpv.Port)
+	vpv.VpvLock.Unlock()
+}
+
+// PortUpInd : When a port transistions to UP state, the indication is passed
+// on to this module via the application. We read the VNET configuration
+// again here to apply the latest configuration if the configuration
+// changed. Thus, a reboot of ONT forces the new configuration to get
+// applied.
+func (vpv *VoltPortVnet) PortUpInd(device *VoltDevice, port string) {
+
+	if vpv.DeleteInProgress {
+		logger.Errorw(ctx, "Ignoring VPV Port UP Ind, VPV deleteion In-Progress", log.Fields{"Device": device, "Port": port, "Vnet": vpv.VnetName})
+		return
+	}
+	vpv.setDevice(device.Name)
+	logger.Infow(ctx, "Port UP Ind, pushing flows for the port", log.Fields{"Device": device, "Port": port, "VnetDhcp": vpv.DhcpRelay, "McastService": vpv.McastService})
+
+	nni, _ := GetApplication().GetNniPort(device.Name)
+	if nni == "" {
+		logger.Warnw(ctx, "Ignoring Vnet Port UP indication: NNI is unavailable", log.Fields{"Port": vpv.Port, "Device": device.Name})
+		return
+	}
+
+	if vp := device.GetPort(port); vp != nil {
+
+		if vpv.PonPort != 0xFF && vpv.PonPort != vp.PonPort {
+			logger.Errorw(ctx, "UNI port discovered on wrong PON Port. Dropping Flow Config for VPV", log.Fields{"Device": device.Name, "Port": port, "DetectedPon": vp.PonPort, "ExpectedPon": vpv.PonPort, "Vnet": vpv.VnetName})
+			return
+		}
+	}
+
+	if vpv.Blocked {
+		logger.Errorw(ctx, "VPV Bocked for Processing. Ignoring flow push request", log.Fields{"Port": vpv.Port, "Vnet": vpv.VnetName})
+		return
+	}
+
+	if vpv.DhcpRelay || vpv.ArpRelay || vpv.PppoeIa {
+		// If MAC Learning is true if no MAC is configured, push DS/US DHCP, US HSIA flows without MAC.
+		// DS HSIA flows are installed after learning the MAC.
+		logger.Infow(ctx, "Port Up - Trap Flows", log.Fields{"Device": device.Name, "Port": port})
+		// no HSIA flows for multicast service
+		if !vpv.McastService {
+			vpv.services.Range(AddUsHsiaFlows)
+		}
+		vpv.AddTrapFlows()
+		if vpv.MacLearning == MacLearningNone || NonZeroMacAddress(vpv.MacAddr) {
+			logger.Infow(ctx, "Port Up - DS Flows", log.Fields{"Device": device.Name, "Port": port})
+			// US & DS DHCP, US HSIA flows are already installed
+			// install only DS HSIA flow here.
+			// no HSIA flows for multicast service
+			if !vpv.McastService {
+				vpv.services.Range(AddDsHsiaFlows)
+			}
+		}
+
+	} else {
+		// DHCP relay is not configured. This implies that the service must use
+		// 1:1 and does not require MAC learning. In a completely uncommon but
+		// plausible case, the MAC address can be learnt from N:1 without DHCP
+		// relay by configuring any unknown MAC address to be reported. This
+		// however is not seen as a real use case.
+		logger.Infow(ctx, "Port Up - Service Flows", log.Fields{"Device": device.Name, "Port": port})
+		if !vpv.McastService {
+			vpv.services.Range(AddUsHsiaFlows)
+		}
+		vpv.AddTrapFlows()
+		if !vpv.McastService {
+			vpv.services.Range(AddDsHsiaFlows)
+		}
+	}
+
+	// Process IGMP proxy - install IGMP trap rules before DHCP trap rules
+	if vpv.IgmpEnabled {
+		logger.Infow(ctx, "Port Up - IGMP Flows", log.Fields{"Device": device.Name, "Port": port})
+		vpv.services.Range(AddSvcUsMeterToDevice)
+		if err := vpv.AddIgmpFlows(); err != nil {
+			statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+			vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+		}
+
+		if vpv.McastService {
+			vpv.services.Range(PostAccessConfigSuccessInd)
+		}
+	}
+
+	vpv.WriteToDb()
+}
+
+// PortDownInd : When the port status changes to down, we delete all configured flows
+// The same indication is also passed to the services enqueued for them
+// to take appropriate actions
+func (vpv *VoltPortVnet) PortDownInd(device string, port string) {
+
+	logger.Infow(ctx, "VPV Port DOWN Ind, deleting all flows for services",
+		log.Fields{"service count": vpv.servicesCount.Load()})
+
+	//vpv.services.Range(DelAllFlows)
+	vpv.DelTrapFlows()
+	vpv.DelHsiaFlows()
+	vpv.WriteToDb()
+	vpv.ClearServiceCounters()
+}
+
+// SetMacAddr : The MAC address is set when a MAC address is learnt through the
+// packets received from the network. Currently, DHCP packets are
+// only packets we learn the MAC address from
+func (vpv *VoltPortVnet) SetMacAddr(addr net.HardwareAddr) {
+
+	//Store Learnt MAC address and return if MACLearning is not enabled
+	vpv.LearntMacAddr = addr
+	if vpv.MacLearning == MacLearningNone || !NonZeroMacAddress(addr) ||
+		(NonZeroMacAddress(vpv.MacAddr) && vpv.MacLearning == Learn) {
+		return
+	}
+
+	// Compare the two MAC addresses to see if it is same
+	// If they are same, we just return. If not, we perform
+	// actions to address the change in MAC address
+	//if NonZeroMacAddress(vpv.MacAddr) && !util.MacAddrsMatch(vpv.MacAddr, addr) {
+	if !util.MacAddrsMatch(vpv.MacAddr, addr) {
+		expectedPort := GetApplication().GetMacInPortMap(addr)
+		if expectedPort != "" && expectedPort != vpv.Port {
+			logger.Errorw(ctx, "mac-learnt-from-different-port-ignoring-setmacaddr",
+				log.Fields{"ExpectedPort": expectedPort, "ReceivedPort": vpv.Port, "LearntMacAdrr": vpv.MacAddr, "NewMacAdrr": addr.String()})
+			return
+		}
+		if NonZeroMacAddress(vpv.MacAddr) {
+			logger.Warnw(ctx, "MAC Address Changed. Remove old flows (if added) and re-add with updated MAC", log.Fields{"UpdatedMAC": addr})
+
+			// The newly learnt MAC address is different than earlier one.
+			// The existing MAC based HSIA flows need to be undone as the device
+			// may have been changed
+			// Atleast one HSIA flow should be present in adapter to retain the TP and GEM
+			// hence delete one after the other
+			vpv.services.Range(DelUsHsiaFlows)
+			vpv.MacAddr = addr
+			vpv.services.Range(vpv.setLearntMAC)
+			vpv.services.Range(AddUsHsiaFlows)
+			vpv.services.Range(DelDsHsiaFlows)
+			GetApplication().DeleteMacInPortMap(vpv.MacAddr)
+		} else {
+			vpv.MacAddr = addr
+			vpv.services.Range(vpv.setLearntMAC)
+			logger.Infow(ctx, "MAC Address learnt from DHCP or ARP", log.Fields{"Learnt MAC": addr.String(), "Port": vpv.Port})
+		}
+		GetApplication().UpdateMacInPortMap(vpv.MacAddr, vpv.Port)
+	} else {
+		logger.Infow(ctx, "Leant MAC Address is same", log.Fields{"Learnt MAC": addr.String(), "Port": vpv.Port})
+	}
+
+	_, err := GetApplication().GetDeviceFromPort(vpv.Port)
+	if err != nil {
+		logger.Warnw(ctx, "Not pushing Service Flows: Error Getting Device", log.Fields{"Reason": err.Error()})
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+		return
+	}
+	// Ds Hsia flows has to be pushed
+	if vpv.FlowsApplied {
+		// no HSIA flows for multicast service
+		if !vpv.McastService {
+			vpv.services.Range(AddDsHsiaFlows)
+		}
+	}
+	vpv.WriteToDb()
+}
+
+// MatchesVlans : If the VNET matches both S and C VLANs, return true. Else, return false
+func (vpv *VoltPortVnet) MatchesVlans(svlan of.VlanType, cvlan of.VlanType, univlan of.VlanType) bool {
+	if vpv.SVlan != svlan || vpv.CVlan != cvlan || vpv.UniVlan != univlan {
+		return false
+	}
+	return true
+}
+
+// MatchesCvlan : If the VNET matches CVLAN, return true. Else, return false
+func (vpv *VoltPortVnet) MatchesCvlan(cvlan []of.VlanType) bool {
+	if len(cvlan) != 1 && !vpv.AllowTransparent {
+		return false
+	}
+	if vpv.CVlan != cvlan[0] {
+		return false
+	}
+	return true
+}
+
+// MatchesPriority : If the VNET matches priority of the incoming packet with any service, return true. Else, return false
+func (vpv *VoltPortVnet) MatchesPriority(priority uint8) *VoltService {
+
+	var service *VoltService
+	pbitFound := false
+	matchpbitsFunc := func(key, value interface{}) bool {
+		svc := value.(*VoltService)
+		for _, pbit := range svc.Pbits {
+			if uint8(pbit) == priority {
+				logger.Infow(ctx, "Pbit match found with service",
+					log.Fields{"Pbit": priority, "serviceName": svc.Name})
+				pbitFound = true
+				service = svc
+				return false //Returning false to stop the Range loop
+			}
+		}
+		return true
+	}
+	_ = pbitFound
+	vpv.services.Range(matchpbitsFunc)
+	return service
+}
+
+// GetRemarkedPriority : If the VNET matches priority of the incoming packet with any service, return true. Else, return false
+func (vpv *VoltPortVnet) GetRemarkedPriority(priority uint8) uint8 {
+
+	dsPbit := uint8(0)
+	matchpbitsFunc := func(key, value interface{}) bool {
+		svc := value.(*VoltService)
+		if remarkPbit, ok := svc.DsRemarkPbitsMap[int(priority)]; ok {
+			logger.Infow(ctx, "Pbit match found with service",
+				log.Fields{"Pbit": priority, "serviceName": svc.Name, "remarkPbit": remarkPbit})
+			dsPbit = uint8(remarkPbit)
+			return false //Returning false to stop the Range loop
+		}
+		// When no remarking info is available, remark the incoming pbit
+		// to highest pbit configured for the subscriber (across all subservices associated)
+		svcPbit := uint8(svc.Pbits[0])
+		if svcPbit > dsPbit {
+			dsPbit = svcPbit
+		}
+		return true
+	}
+	vpv.services.Range(matchpbitsFunc)
+	logger.Debugw(ctx, "Remarked Pbit Value", log.Fields{"Incoming": priority, "Remarked": dsPbit})
+	return dsPbit
+}
+
+// AddSvc adds a service on the VNET on a port. The addition is
+// triggered when NB requests for service addition
+func (vpv *VoltPortVnet) AddSvc(svc *VoltService) {
+
+	//vpv.services = append(vpv.services, svc)
+	vpv.AddService(svc)
+	logger.Debugw(ctx, "Added Service to VPV", log.Fields{"Num of SVCs": vpv.servicesCount.Load(), "SVC": svc})
+
+	// Learn the circuit-id and remote-id from the service
+	// TODO: There must be a better way of doing this. This
+	// may be explored
+	if svc.IgmpEnabled {
+		vpv.IgmpEnabled = true
+	}
+	// first time service activation MacLearning will have default value as None.
+	// to handle reciliency if anythng other then None we should retain it .
+	if svc.MacLearning == MacLearningNone {
+		if !vpv.DhcpRelay && !vpv.ArpRelay {
+			svc.MacLearning = MacLearningNone
+		} else if vpv.MacLearning == Learn {
+			svc.MacLearning = Learn
+		} else if vpv.MacLearning == ReLearn {
+			svc.MacLearning = ReLearn
+		}
+	}
+
+	//TODO: Temp Change - Need to address MAC Learning flow issues completely
+	if (svc.MacLearning == Learn || svc.MacLearning == ReLearn) && NonZeroMacAddress(vpv.MacAddr) {
+		svc.MacAddr = vpv.MacAddr
+	} else if vpv.servicesCount.Load() == 1 {
+		vpv.MacAddr = svc.MacAddr
+	}
+
+	vpv.MacLearning = svc.MacLearning
+	vpv.PonPort = svc.PonPort
+	logger.Debugw(ctx, "Added MAC to VPV", log.Fields{"MacLearning": vpv.MacLearning, "VPV": vpv})
+	//Reconfigure Vlans based on Vlan Control type
+	svc.VlanControl = vpv.VlanControl
+	// for OLTCVLAN SVLAN=CVLAN, UNIVLAN can differ.
+	if vpv.VlanControl == ONUCVlan {
+		svc.CVlan = svc.SVlan
+	}
+	// for OLTSVLAN  CVLAN=UNIVLAN , SVLAN can differ,
+	// hence assigning UNIVLAN to CVLAN, so that ONU will transparently forward the packet.
+	if vpv.VlanControl == OLTSVlan {
+		svc.CVlan = svc.UniVlan
+	}
+	if svc.McastService {
+		vpv.McastService = true
+		vpv.McastTechProfileID = svc.TechProfileID
+		//Assumption: Only one Pbit for mcast service
+		vpv.McastPbit = svc.Pbits[0]
+		vpv.McastUsMeterID = svc.UsMeterID
+		vpv.SchedID = svc.SchedID
+	}
+	svc.ONTEtherTypeClassification = vpv.ONTEtherTypeClassification
+	svc.AllowTransparent = vpv.AllowTransparent
+	svc.SVlanTpid = vpv.SVlanTpid
+
+	//Ensure configuring the mvlan profile only once
+	//One subscriber cannot have multiple mvlan profiles. Only the first configuration is valid
+	if svc.MvlanProfileName != "" {
+		if vpv.MvlanProfileName == "" {
+			vpv.MvlanProfileName = svc.MvlanProfileName
+		} else {
+			logger.Warnw(ctx, "Mvlan Profile already configured for subscriber. Ignoring new Mvlan", log.Fields{"Existing Mvlan": vpv.MvlanProfileName, "New Mvlan": svc.MvlanProfileName})
+		}
+	}
+
+	_, err := GetApplication().GetDeviceFromPort(vpv.Port)
+	if err != nil {
+		logger.Warnw(ctx, "Not pushing Service Flows: Error Getting Device", log.Fields{"Reason": err.Error()})
+		//statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		//TODO-COMM: 		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+		return
+	}
+
+	//Push Service Flows if DHCP relay is not configured
+	//or already DHCP flows are configured for the VPV
+	//to which the serivce is associated
+	if vpv.FlowsApplied {
+		if NonZeroMacAddress(vpv.MacAddr) || svc.MacLearning == MacLearningNone {
+			svc.AddHsiaFlows()
+		} else {
+			if err:= svc.AddUsHsiaFlows(); err != nil {
+				logger.Warnw(ctx, "Add US hsia flow failed", log.Fields{"service": svc.Name, "Error": err})
+			}
+		}
+	}
+
+	//Assumption: Igmp will be enabled only for one service and SubMgr ensure the same
+	// When already the port is UP and provisioned a service without igmp, then trap flows for subsequent
+	// service with Igmp Enabled needs to be installed
+	if svc.IgmpEnabled && vpv.FlowsApplied {
+		logger.Infow(ctx, "Add Service - IGMP Flows", log.Fields{"Device": vpv.Device, "Port": vpv.Port})
+		if err := vpv.AddIgmpFlows(); err != nil {
+			statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+			vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+		}
+
+		if vpv.McastService {
+			//For McastService, send Service Activated indication once IGMP US flow is pushed
+			vpv.services.Range(PostAccessConfigSuccessInd)
+		}
+	}
+	vpv.WriteToDb()
+}
+
+// setLearntMAC to set learnt mac
+func (vpv *VoltPortVnet) setLearntMAC(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	svc.SetMacAddr(vpv.MacAddr)
+	svc.WriteToDb()
+	return true
+}
+
+// PostAccessConfigSuccessInd for posting access config success indication
+func PostAccessConfigSuccessInd(key, value interface{}) bool {
+	return true
+}
+
+// updateIPv4AndProvisionFlows to update ipv4 and provisional flows
+func (vpv *VoltPortVnet) updateIPv4AndProvisionFlows(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	logger.Info(ctx, "Updating Ipv4 address for service", log.Fields{"ServiceName": svc.Name})
+	svc.SetIpv4Addr(vpv.Ipv4Addr)
+	svc.WriteToDb()
+
+	return true
+}
+
+// updateIPv6AndProvisionFlows to update ipv6 and provisional flow
+func (vpv *VoltPortVnet) updateIPv6AndProvisionFlows(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	svc.SetIpv6Addr(vpv.Ipv6Addr)
+	svc.WriteToDb()
+
+	return true
+}
+
+// AddUsHsiaFlows to add upstream hsia flows
+func AddUsHsiaFlows(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	if err:= svc.AddUsHsiaFlows(); err != nil {
+		logger.Warnw(ctx, "Add US hsia flow failed", log.Fields{"service": svc.Name, "Error": err})
+	}
+	return true
+}
+
+// AddDsHsiaFlows to add downstream hsia flows
+func AddDsHsiaFlows(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	if err:= svc.AddDsHsiaFlows(); err != nil {
+		logger.Warnw(ctx, "Add DS hsia flow failed", log.Fields{"service": svc.Name, "Error": err})
+	}
+	return true
+}
+
+// ClearFlagsInService to clear the flags used in service
+func ClearFlagsInService(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	svc.ServiceLock.Lock()
+	svc.IgmpFlowsApplied = false
+	svc.DsDhcpFlowsApplied = false
+	svc.DsHSIAFlowsApplied = false
+	svc.Icmpv6FlowsApplied = false
+	svc.UsHSIAFlowsApplied = false
+	svc.UsDhcpFlowsApplied = false
+	svc.PendingFlows = make(map[string]bool)
+	svc.AssociatedFlows = make(map[string]bool)
+	svc.ServiceLock.Unlock()
+	svc.WriteToDb()
+	logger.Debugw(ctx, "Cleared Flow Flags for service", log.Fields{"name": svc.Name})
+	return true
+}
+
+// DelDsHsiaFlows to delete hsia flows
+func DelDsHsiaFlows(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	if err:= svc.DelDsHsiaFlows(); err != nil {
+		logger.Warnw(ctx, "Delete DS hsia flow failed", log.Fields{"service": svc.Name, "Error": err})
+	}
+	return true
+}
+
+// DelUsHsiaFlows to delete upstream hsia flows
+func DelUsHsiaFlows(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	if err:= svc.DelUsHsiaFlows(); err != nil {
+		logger.Warnw(ctx, "Delete US hsia flow failed", log.Fields{"service": svc.Name, "Error": err})
+	}
+	return true
+}
+
+// ClearServiceCounters to clear the service counters
+func ClearServiceCounters(key, value interface{}) bool {
+	svc := value.(*VoltService)
+	//Delete the per service counter too
+	GetApplication().ServiceCounters.Delete(svc.Name)
+	if svc.IgmpEnabled && svc.EnableMulticastKPI {
+		_ = db.DelAllServiceChannelCounter(svc.Name)
+	}
+	return true
+}
+
+//AddTrapFlows - Adds US & DS Trap flows
+func (vpv *VoltPortVnet) AddTrapFlows() {
+
+	if !vpv.FlowsApplied || vgcRebooted {
+		if vpv.DhcpRelay {
+			if err := vpv.AddUsDhcpFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+			if err := vpv.AddDsDhcpFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+			logger.Infow(ctx, "ICMPv6 MC Group modification will not be triggered to rwcore for ",
+				log.Fields{"port": vpv.Port})
+			//vpv.updateICMPv6McGroup(true)
+		} else if vpv.ArpRelay {
+			if err := vpv.AddUsArpFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+			logger.Info(ctx, "ARP trap rules not added in downstream direction")
+
+		} else if vpv.PppoeIa {
+			if err := vpv.AddUsPppoeFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+			if err := vpv.AddDsPppoeFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+		}
+		vpv.FlowsApplied = true
+		vpv.WriteToDb()
+	}
+}
+
+//DelTrapFlows - Removes all US & DS DHCP, IGMP trap flows.
+func (vpv *VoltPortVnet) DelTrapFlows() {
+
+	// Delete HSIA & DHCP flows before deleting IGMP flows
+	if vpv.FlowsApplied || vgcRebooted {
+		if vpv.DhcpRelay {
+			if err:= vpv.DelUsDhcpFlows(); err != nil {
+				logger.Warnw(ctx, "Delete US hsia flow failed", log.Fields{"port": vpv.Port, "SVlan": vpv.SVlan, "CVlan": vpv.CVlan,
+					"UniVlan": vpv.UniVlan, "Error": err})
+			}
+			logger.Infow(ctx, "ICMPv6 MC Group modification will not be triggered to rwcore  for ",
+				log.Fields{"port": vpv.Port})
+			if err := vpv.DelDsDhcpFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+			//vpv.updateICMPv6McGroup(false)
+		} else if vpv.ArpRelay {
+			if err := vpv.DelUsArpFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+		} else if vpv.PppoeIa {
+			if err := vpv.DelUsPppoeFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+			if err := vpv.DelDsPppoeFlows(); err != nil {
+				statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+				vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+			}
+		}
+		vpv.FlowsApplied = false
+		vpv.WriteToDb()
+	}
+	if err:= vpv.DelIgmpFlows(); err != nil {
+		logger.Warnw(ctx, "Delete igmp flow failed", log.Fields{"port": vpv.Port, "SVlan": vpv.SVlan, "CVlan": vpv.CVlan,
+			"UniVlan": vpv.UniVlan, "Error": err})
+	}
+}
+
+// DelHsiaFlows deletes the service flows
+func (vpv *VoltPortVnet) DelHsiaFlows() {
+	// no HSIA flows for multicast service
+	if !vpv.McastService {
+		vpv.services.Range(DelUsHsiaFlows)
+		vpv.services.Range(DelDsHsiaFlows)
+	}
+}
+
+//ClearServiceCounters - Removes all igmp counters for a service
+func (vpv *VoltPortVnet) ClearServiceCounters() {
+	//send flows deleted indication to submgr
+	vpv.services.Range(ClearServiceCounters)
+}
+
+// AddUsDhcpFlows pushes the DHCP flows to the VOLTHA via the controller
+func (vpv *VoltPortVnet) AddUsDhcpFlows() error {
+	var vd *VoltDevice
+	device := vpv.Device
+
+	if vd = GetApplication().GetDevice(device); vd != nil {
+		if vd.State != controller.DeviceStateUP {
+			logger.Errorw(ctx, "Skipping US DHCP Flow Push - Device state DOWN", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+			return nil
+		}
+	} else {
+		logger.Errorw(ctx, "US DHCP Flow Push Failed- Device not found", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+		return errorCodes.ErrDeviceNotFound
+	}
+
+	flows, err := vpv.BuildUsDhcpFlows()
+	if err == nil {
+		logger.Debugw(ctx, "Adding US DHCP flows", log.Fields{"Device": device})
+		if err1 := vpv.PushFlows(vd, flows); err1 != nil {
+			//push ind here ABHI
+			statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err1)
+			vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+		}
+	} else {
+		logger.Errorw(ctx, "US DHCP Flow Add Failed", log.Fields{"Reason": err.Error(), "Device": device})
+		//push ind here ABHI
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+
+	}
+	/*
+	flows, err = vpv.BuildUsDhcp6Flows()
+	if err == nil {
+		logger.Debugw(ctx, "Adding US DHCP6 flows", log.Fields{"Device": device})
+		if err1 := vpv.PushFlows(vd, flows); err1 != nil {
+			//pussh ind here ABHI
+			statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err1)
+			vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+
+		}
+	} else {
+		logger.Errorw(ctx, "US DHCP6 Flow Add Failed", log.Fields{"Reason": err.Error(), "Device": device})
+		//push ind here ABHI
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+
+	}*/
+	return nil
+}
+
+// AddDsDhcpFlows function pushes the DHCP flows to the VOLTHA via the controller
+func (vpv *VoltPortVnet) AddDsDhcpFlows() error {
+
+	var vd *VoltDevice
+	device := vpv.Device
+
+	if vd = GetApplication().GetDevice(device); vd != nil {
+		if vd.State != controller.DeviceStateUP {
+			logger.Errorw(ctx, "Skipping DS DHCP Flow Push - Device state DOWN", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+			return nil
+		}
+	} else {
+		logger.Errorw(ctx, "DS DHCP Flow Push Failed- Device not found", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+		return errorCodes.ErrDeviceNotFound
+	}
+	if GetApplication().GetVendorID() != Radisys && vd.GlobalDhcpFlowAdded {
+		return nil
+	}
+
+	flows, err := vpv.BuildDsDhcpFlows()
+	if err == nil {
+		if err1 := vpv.PushFlows(vd, flows); err1 != nil {
+			//push ind here and procced
+			statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err1)
+			vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+
+		}
+	} else {
+		logger.Errorw(ctx, "DS DHCP Flow Add Failed", log.Fields{"Reason": err.Error()})
+		//send ind here and proceed
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+
+	}
+	/*
+	flows, err = vpv.BuildDsDhcp6Flows()
+	if err == nil {
+		if err1 := vpv.PushFlows(vd, flows); err1 != nil {
+			//push ind and proceed
+			statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err1)
+			vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+
+		}
+	} else {
+		logger.Errorw(ctx, "DS DHCP6 Flow Add Failed", log.Fields{"Reason": err.Error()})
+		//Send ind here and proceed
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+
+	}*/
+	if GetApplication().GetVendorID() != Radisys {
+		vd.GlobalDhcpFlowAdded = true
+	}
+	return nil
+}
+
+// DelDhcpFlows deletes both US & DS DHCP flows applied for this Vnet instantiated on the port
+func (vpv *VoltPortVnet) DelDhcpFlows() {
+	if err := vpv.DelUsDhcpFlows(); err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+	}
+
+	if err := vpv.DelDsDhcpFlows(); err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+	}
+}
+
+// DelUsDhcpFlows delete the DHCP flows applied for this Vnet instantiated on the port
+// Write the status of the VPV to the DB once the delete is scheduled
+// for dispatch
+func (vpv *VoltPortVnet) DelUsDhcpFlows() error {
+	device, err := GetApplication().GetDeviceFromPort(vpv.Port)
+	if err != nil {
+		return err
+	}
+
+	err = vpv.delDhcp4Flows(device)
+	if err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+	}
+	/*
+	err = vpv.delDhcp6Flows(device)
+	if err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+	}*/
+	return nil
+}
+
+func (vpv *VoltPortVnet) delDhcp4Flows(device *VoltDevice) error {
+	flows, err := vpv.BuildUsDhcpFlows()
+	if err == nil {
+		return vpv.RemoveFlows(device, flows)
+	}
+	logger.Errorw(ctx, "US DHCP Flow Delete Failed", log.Fields{"Reason": err.Error()})
+	return err
+}
+/*
+func (vpv *VoltPortVnet) delDhcp6Flows(device *VoltDevice) error {
+	flows, err := vpv.BuildUsDhcp6Flows()
+	if err == nil {
+		return vpv.RemoveFlows(device, flows)
+	}
+	logger.Errorw(ctx, "US DHCP6 Flow Delete Failed", log.Fields{"Reason": err.Error()})
+	return err
+
+}*/
+
+// DelDsDhcpFlows delete the DHCP flows applied for this Vnet instantiated on the port
+// Write the status of the VPV to the DB once the delete is scheduled
+// for dispatch
+func (vpv *VoltPortVnet) DelDsDhcpFlows() error {
+	device, err := GetApplication().GetDeviceFromPort(vpv.Port)
+	if err != nil {
+		return err
+	}
+	err = vpv.delDsDhcp4Flows(device)
+	if err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+	}
+	/*
+	err = vpv.delDsDhcp6Flows(device)
+	if err != nil {
+		statusCode, statusMessage := infraerrorCodes.GetErrorInfo(err)
+		vpv.FlowInstallFailure("VGC processing failure", statusCode, statusMessage)
+	}*/
+	return nil
+}
+
+func (vpv *VoltPortVnet) delDsDhcp4Flows(device *VoltDevice) error {
+	flows, err := vpv.BuildDsDhcpFlows()
+	if err == nil {
+		return vpv.RemoveFlows(device, flows)
+	}
+	logger.Errorw(ctx, "DS DHCP Flow Delete Failed", log.Fields{"Reason": err.Error()})
+	return err
+}
+
+/*
+func (vpv *VoltPortVnet) delDsDhcp6Flows(device *VoltDevice) error {
+	flows, err := vpv.BuildDsDhcp6Flows()
+	if err == nil {
+		return vpv.RemoveFlows(device, flows)
+	}
+	logger.Errorw(ctx, "DS DHCP6 Flow Delete Failed", log.Fields{"Reason": err.Error()})
+	return err
+}*/
+
+// AddUsArpFlows pushes the ARP flows to the VOLTHA via the controller
+func (vpv *VoltPortVnet) AddUsArpFlows() error {
+
+	var vd *VoltDevice
+	device := vpv.Device
+	if vd = GetApplication().GetDevice(device); vd != nil {
+		if vd.State != controller.DeviceStateUP {
+			logger.Errorw(ctx, "Skipping US ARP Flow Push - Device state DOWN", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+			return nil
+		}
+	} else {
+		logger.Errorw(ctx, "US ARP Flow Push Failed- Device not found", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+		return errorCodes.ErrDeviceNotFound
+	}
+
+	flows, err := vpv.BuildUsArpFlows()
+	if err == nil {
+		logger.Debugw(ctx, "Adding US ARP flows", log.Fields{"Device": device})
+		if err1 := vpv.PushFlows(vd, flows); err1 != nil {
+			return err1
+		}
+	} else {
+		logger.Errorw(ctx, "US ARP Flow Add Failed", log.Fields{"Reason": err.Error(), "Device": device})
+		return err
+	}
+	return nil
+}
+
+// DelUsArpFlows delete the ARP flows applied for this Vnet instantiated on the port
+// Write the status of the VPV to the DB once the delete is scheduled
+// for dispatch
+func (vpv *VoltPortVnet) DelUsArpFlows() error {
+	device, err := GetApplication().GetDeviceFromPort(vpv.Port)
+	if err != nil {
+		return err
+	}
+	flows, err := vpv.BuildUsArpFlows()
+	if err == nil {
+		return vpv.RemoveFlows(device, flows)
+	}
+	logger.Errorw(ctx, "US ARP Flow Delete Failed", log.Fields{"Reason": err.Error()})
+	return err
+}
+
+// AddUsPppoeFlows pushes the PPPoE flows to the VOLTHA via the controller
+func (vpv *VoltPortVnet) AddUsPppoeFlows() error {
+	logger.Debugw(ctx, "Adding US PPPoE flows", log.Fields{"STAG": vpv.SVlan, "CTAG": vpv.CVlan, "Device": vpv.Device})
+
+	var vd *VoltDevice
+	device := vpv.Device
+
+	if vd = GetApplication().GetDevice(device); vd != nil {
+		if vd.State != controller.DeviceStateUP {
+			logger.Errorw(ctx, "Skipping US PPPoE Flow Push - Device state DOWN", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+			return nil
+		}
+	} else {
+		logger.Errorw(ctx, "US PPPoE Flow Push Failed- Device not found", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+		return errorCodes.ErrDeviceNotFound
+	}
+
+	if flows, err := vpv.BuildUsPppoeFlows(); err == nil {
+		logger.Debugw(ctx, "Adding US PPPoE flows", log.Fields{"Device": device})
+
+		if err1 := vpv.PushFlows(vd, flows); err1 != nil {
+			return err1
+		}
+	} else {
+		logger.Errorw(ctx, "US PPPoE Flow Add Failed", log.Fields{"Reason": err.Error(), "Device": device})
+		return err
+	}
+	return nil
+}
+
+// AddDsPppoeFlows to add downstream pppoe flows
+func (vpv *VoltPortVnet) AddDsPppoeFlows() error {
+	logger.Debugw(ctx, "Adding DS PPPoE flows", log.Fields{"STAG": vpv.SVlan, "CTAG": vpv.CVlan, "Device": vpv.Device})
+	var vd *VoltDevice
+	device := vpv.Device
+
+	if vd = GetApplication().GetDevice(device); vd != nil {
+		if vd.State != controller.DeviceStateUP {
+			logger.Errorw(ctx, "Skipping DS PPPoE Flow Push - Device state DOWN", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+			return nil
+		}
+	} else {
+		logger.Errorw(ctx, "DS PPPoE Flow Push Failed- Device not found", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan, "device": device})
+		return errorCodes.ErrDeviceNotFound
+	}
+
+	flows, err := vpv.BuildDsPppoeFlows()
+	if err == nil {
+
+		if err1 := vpv.PushFlows(vd, flows); err1 != nil {
+			return err1
+		}
+	} else {
+		logger.Errorw(ctx, "DS PPPoE Flow Add Failed", log.Fields{"Reason": err.Error()})
+		return err
+	}
+	return nil
+}
+
+// DelUsPppoeFlows delete the PPPoE flows applied for this Vnet instantiated on the port
+// Write the status of the VPV to the DB once the delete is scheduled
+// for dispatch
+func (vpv *VoltPortVnet) DelUsPppoeFlows() error {
+	logger.Debugw(ctx, "Deleting US PPPoE flows", log.Fields{"STAG": vpv.SVlan, "CTAG": vpv.CVlan, "Device": vpv.Device})
+	device, err := GetApplication().GetDeviceFromPort(vpv.Port)
+	if err != nil {
+		return err
+	}
+	flows, err := vpv.BuildUsPppoeFlows()
+	if err == nil {
+		return vpv.RemoveFlows(device, flows)
+	}
+	logger.Errorw(ctx, "US PPPoE Flow Delete Failed", log.Fields{"Reason": err.Error()})
+	return err
+}
+
+// DelDsPppoeFlows delete the PPPoE flows applied for this Vnet instantiated on the port
+// Write the status of the VPV to the DB once the delete is scheduled
+// for dispatch
+func (vpv *VoltPortVnet) DelDsPppoeFlows() error {
+	logger.Debugw(ctx, "Deleting DS PPPoE flows", log.Fields{"STAG": vpv.SVlan, "CTAG": vpv.CVlan, "Device": vpv.Device})
+	device, err := GetApplication().GetDeviceFromPort(vpv.Port)
+	if err != nil {
+		return err
+	}
+	flows, err := vpv.BuildDsPppoeFlows()
+	if err == nil {
+		return vpv.RemoveFlows(device, flows)
+	}
+	logger.Errorw(ctx, "DS PPPoE Flow Delete Failed", log.Fields{"Reason": err.Error()})
+	return err
+}
+
+// AddIgmpFlows function pushes the IGMP flows to the VOLTHA via the controller
+func (vpv *VoltPortVnet) AddIgmpFlows() error {
+
+	if !vpv.IgmpFlowsApplied || vgcRebooted {
+		if vpv.MvlanProfileName == "" {
+			logger.Info(ctx, "Mvlan Profile not configured. Ignoring Igmp trap flow")
+			return nil
+		}
+		device, err := GetApplication().GetDeviceFromPort(vpv.Port)
+		if err != nil {
+			logger.Errorw(ctx, "Error getting device from port", log.Fields{"Port": vpv.Port, "Reason": err.Error()})
+			return err
+		} else if device.State != controller.DeviceStateUP {
+			logger.Warnw(ctx, "Device state Down. Ignoring US IGMP Flow Push", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan, "UNIVlan": vpv.UniVlan})
+			return nil
+		}
+		flows, err := vpv.BuildIgmpFlows()
+		if err == nil {
+			for cookie := range flows.SubFlows {
+				if vd := GetApplication().GetDevice(device.Name); vd != nil {
+					cookie := strconv.FormatUint(cookie, 10)
+					fe := &FlowEvent{
+						eType:     EventTypeUsIgmpFlowAdded,
+						cookie:    cookie,
+						eventData: vpv,
+					}
+					vd.RegisterFlowAddEvent(cookie, fe)
+				}
+			}
+			if err1 := cntlr.GetController().AddFlows(vpv.Port, device.Name, flows); err1 != nil {
+				return err1
+			}
+		} else {
+			logger.Errorw(ctx, "IGMP Flow Add Failed", log.Fields{"Reason": err.Error()})
+			return err
+		}
+		vpv.IgmpFlowsApplied = true
+		vpv.WriteToDb()
+	}
+	return nil
+}
+
+// DelIgmpFlows delete the IGMP flows applied for this Vnet instantiated on the port
+// Write the status of the VPV to the DB once the delete is scheduled
+// for dispatch
+func (vpv *VoltPortVnet) DelIgmpFlows() error {
+
+	if vpv.IgmpFlowsApplied || vgcRebooted {
+		device, err := GetApplication().GetDeviceFromPort(vpv.Port)
+		if err != nil {
+			logger.Errorw(ctx, "Error getting device from port", log.Fields{"Port": vpv.Port, "Reason": err.Error()})
+			return err
+		}
+		flows, err := vpv.BuildIgmpFlows()
+		if err == nil {
+			if err1 := vpv.RemoveFlows(device, flows); err1 != nil {
+				return err1
+			}
+		} else {
+			logger.Errorw(ctx, "IGMP Flow Add Failed", log.Fields{"Reason": err.Error()})
+			return err
+		}
+		vpv.IgmpFlowsApplied = false
+		vpv.WriteToDb()
+	}
+	return nil
+}
+
+// BuildUsDhcpFlows builds the US DHCP relay flows for a subscriber
+// The flows included by this function cover US only as the DS is
+// created either automatically by the VOLTHA or at the device level
+// earlier
+func (vpv *VoltPortVnet) BuildUsDhcpFlows() (*of.VoltFlow, error) {
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+
+	logger.Infow(ctx, "Building US DHCP flow", log.Fields{"Port": vpv.Port})
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+
+	if GetApplication().GetVendorID() == Radisys {
+		if err := vpv.setUsMatchVlan(subFlow); err != nil {
+			return nil, err
+		}
+	} else {
+		subFlow.SetMatchVlan(vpv.UniVlan)
+		subFlow.SetSetVlan(vpv.CVlan)
+	}
+	subFlow.SetUdpv4Match()
+	subFlow.SrcPort = 68
+	subFlow.DstPort = 67
+	uniport, err := GetApplication().GetPortID(vpv.Port)
+	if err != nil {
+		logger.Errorw(ctx, "Failed to fetch uni port from vpv", log.Fields{"error": err, "port": vpv.Port})
+		return nil, err
+	}
+	subFlow.SetInPort(uniport)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+	subFlow.SetReportToController()
+
+	// Set techprofile, meterid of first service
+	vpv.services.Range(func(key, value interface{}) bool {
+		svc := value.(*VoltService)
+		writemetadata := uint64(svc.TechProfileID) << 32
+		subFlow.SetWriteMetadata(writemetadata)
+		subFlow.SetMeterID(svc.UsMeterID)
+		return false
+	})
+
+	subFlow.SetPcp(vpv.DhcpPbit)
+	// metadata := uint64(uniport)
+	// subFlow.SetWriteMetadata(metadata)
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata := uint64(allowTransparent)<<56 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+
+	//| 12-bit cvlan | 4 bits empty | <32-bits uniport>| 16-bits dhcp mask or flow mask |
+	subFlow.Cookie = uint64(vpv.CVlan)<<52 | uint64(uniport)<<16 | of.DhcpArpFlowMask | of.UsFlowMask
+	subFlow.Priority = of.DhcpFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built US DHCP flow ", log.Fields{"cookie": subFlow.Cookie, "flow": flow})
+	return flow, nil
+}
+
+// BuildDsDhcpFlows to build the downstream dhcp flows
+func (vpv *VoltPortVnet) BuildDsDhcpFlows() (*of.VoltFlow, error) {
+
+	logger.Infow(ctx, "Building DS DHCP flow", log.Fields{"Port": vpv.Port, "ML": vpv.MacLearning, "Mac": vpv.MacAddr})
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+	// If dhcp trap rule is global rule, No need to match on vlan
+	if GetApplication().GetVendorID() == Radisys {
+		vpv.setDsMatchVlan(subFlow)
+	}
+	subFlow.SetUdpv4Match()
+	subFlow.SrcPort = 67
+	subFlow.DstPort = 68
+	uniport, _ := GetApplication().GetPortID(vpv.Port)
+	nni, err := GetApplication().GetNniPort(vpv.Device)
+	if err != nil {
+		return nil, err
+	}
+	nniport, err := GetApplication().GetPortID(nni)
+	if err != nil {
+		return nil, err
+	}
+	subFlow.SetInPort(nniport)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+	// metadata := uint64(uniport)
+	// subFlow.SetWriteMetadata(metadata)
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata := uint64(allowTransparent)<<56 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+	subFlow.SetReportToController()
+	//| 12-bit cvlan | 4 bits empty | <32-bits uniport>| 16-bits dhcp mask or flow mask |
+	subFlow.Cookie = uint64(vpv.CVlan)<<52 | uint64(uniport)<<16 | of.DhcpArpFlowMask | of.DsFlowMask
+	subFlow.Priority = of.DhcpFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built DS DHCP flow ", log.Fields{"cookie": subFlow.Cookie, "Flow": flow})
+
+	return flow, nil
+}
+
+// BuildUsDhcp6Flows to trap the DHCPv6 packets to be reported to the
+// application.
+func (vpv *VoltPortVnet) BuildUsDhcp6Flows() (*of.VoltFlow, error) {
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+
+	logger.Infow(ctx, "Building US DHCPv6 flow", log.Fields{"Port": vpv.Port})
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+
+	subFlow.SetMatchVlan(vpv.UniVlan)
+	subFlow.SetSetVlan(vpv.CVlan)
+	subFlow.SetUdpv6Match()
+	subFlow.SrcPort = 546
+	subFlow.DstPort = 547
+	uniport, err := GetApplication().GetPortID(vpv.Port)
+	if err != nil {
+		return nil, err
+	}
+	// Set techprofile, meterid of first service
+	vpv.services.Range(func(key, value interface{}) bool {
+		svc := value.(*VoltService)
+		writemetadata := uint64(svc.TechProfileID) << 32
+		subFlow.SetWriteMetadata(writemetadata)
+		subFlow.SetMeterID(svc.UsMeterID)
+		return false
+	})
+	subFlow.SetInPort(uniport)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+	//subFlow.SetMeterId(vpv.UsDhcpMeterId)
+	// metadata := uint64(uniport)
+	// subFlow.SetWriteMetadata(metadata)
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata := uint64(allowTransparent)<<56 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+	subFlow.SetReportToController()
+	//| 12-bit cvlan | 4 bits empty | <32-bits uniport>| 16-bits dhcp mask or flow mask |
+	subFlow.Cookie = uint64(vpv.CVlan)<<52 | uint64(uniport)<<16 | of.Dhcpv6FlowMask | of.UsFlowMask
+	subFlow.Priority = of.DhcpFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built US DHCPv6 flow", log.Fields{"cookie": subFlow.Cookie, "flow": flow})
+	return flow, nil
+}
+
+// BuildDsDhcp6Flows to trap the DHCPv6 packets to be reported to the
+// application.
+func (vpv *VoltPortVnet) BuildDsDhcp6Flows() (*of.VoltFlow, error) {
+	logger.Infow(ctx, "Building DS DHCPv6 flow", log.Fields{"Port": vpv.Port, "ML": vpv.MacLearning, "Mac": vpv.MacAddr})
+
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+
+	vpv.setDsMatchVlan(subFlow)
+	subFlow.SetUdpv6Match()
+	subFlow.SrcPort = 547
+	subFlow.DstPort = 547
+	uniport, _ := GetApplication().GetPortID(vpv.Port)
+	nni, err := GetApplication().GetNniPort(vpv.Device)
+	if err != nil {
+		return nil, err
+	}
+	nniport, err := GetApplication().GetPortID(nni)
+	if err != nil {
+		return nil, err
+	}
+	subFlow.SetInPort(nniport)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+	// metadata := uint64(uniport)
+	// subFlow.SetWriteMetadata(metadata)
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata := uint64(allowTransparent)<<56 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+	subFlow.SetReportToController()
+	//| 12-bit cvlan | 4 bits empty | <32-bits uniport>| 16-bits dhcp mask or flow mask |
+	subFlow.Cookie = uint64(vpv.CVlan)<<52 | uint64(uniport)<<16 | of.Dhcpv6FlowMask | of.DsFlowMask
+	subFlow.Priority = of.DhcpFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built DS DHCPv6 flow", log.Fields{"cookie": subFlow.Cookie, "flow": flow})
+	return flow, nil
+}
+
+// BuildUsArpFlows builds the US ARP relay flows for a subscriber
+// The flows included by this function cover US only as the DS is
+// created either automatically by the VOLTHA or at the device level
+// earlier
+func (vpv *VoltPortVnet) BuildUsArpFlows() (*of.VoltFlow, error) {
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+
+	logger.Infow(ctx, "Building US ARP flow", log.Fields{"Port": vpv.Port})
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+
+	if vpv.MacLearning == MacLearningNone && NonZeroMacAddress(vpv.MacAddr) {
+		subFlow.SetMatchSrcMac(vpv.MacAddr)
+	}
+
+	subFlow.SetMatchDstMac(BroadcastMAC)
+	if err := vpv.setUsMatchVlan(subFlow); err != nil {
+		return nil, err
+	}
+	subFlow.SetArpMatch()
+	uniport, err := GetApplication().GetPortID(vpv.Port)
+	if err != nil {
+		return nil, err
+	}
+	subFlow.SetInPort(uniport)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+	subFlow.SetReportToController()
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata := uint64(uniport)
+	subFlow.SetWriteMetadata(metadata)
+	metadata = uint64(allowTransparent)<<56 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+	subFlow.Cookie = uint64(vpv.CVlan)<<52 | uint64(uniport)<<32 | of.DhcpArpFlowMask | of.UsFlowMask
+	subFlow.Priority = of.ArpFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built US ARP flow ", log.Fields{"cookie": subFlow.Cookie, "flow": flow})
+	return flow, nil
+}
+
+// setUsMatchVlan to set upstream match vlan
+func (vpv *VoltPortVnet) setUsMatchVlan(flow *of.VoltSubFlow) error {
+	switch vpv.VlanControl {
+	case None:
+		flow.SetMatchVlan(vpv.SVlan)
+	case ONUCVlanOLTSVlan:
+		flow.SetMatchVlan(vpv.CVlan)
+	case OLTCVlanOLTSVlan:
+		flow.SetMatchVlan(vpv.UniVlan)
+		//flow.SetSetVlan(vpv.CVlan)
+	case ONUCVlan:
+		flow.SetMatchVlan(vpv.SVlan)
+	case OLTSVlan:
+		flow.SetMatchVlan(vpv.UniVlan)
+		//flow.SetSetVlan(vpv.SVlan)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+	return nil
+}
+
+// BuildUsPppoeFlows to build upstream pppoe flows
+func (vpv *VoltPortVnet) BuildUsPppoeFlows() (*of.VoltFlow, error) {
+
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	logger.Infow(ctx, "Building US PPPoE flow", log.Fields{"Port": vpv.Port})
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+
+	if vpv.MacLearning == MacLearningNone && NonZeroMacAddress(vpv.MacAddr) {
+		subFlow.SetMatchSrcMac(vpv.MacAddr)
+	}
+
+	if err := vpv.setUsMatchVlan(subFlow); err != nil {
+		return nil, err
+	}
+	subFlow.SetPppoeDiscoveryMatch()
+	uniport, err := GetApplication().GetPortID(vpv.Port)
+	if err != nil {
+		return nil, err
+	}
+	subFlow.SetInPort(uniport)
+	subFlow.SetReportToController()
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata := uint64(uniport)
+	subFlow.SetWriteMetadata(metadata)
+
+	metadata = uint64(allowTransparent)<<56 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+
+	//| 12-bit cvlan | 4 bits empty | <32-bits uniport>| 16-bits pppoe mask or flow mask |
+	subFlow.Cookie = uint64(vpv.CVlan)<<52 | uint64(uniport)<<16 | of.PppoeFlowMask | of.UsFlowMask
+	subFlow.Priority = of.PppoeFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built US PPPoE flow ", log.Fields{"cookie": subFlow.Cookie, "flow": flow})
+	return flow, nil
+}
+
+// BuildDsPppoeFlows to build downstream pppoe flows
+func (vpv *VoltPortVnet) BuildDsPppoeFlows() (*of.VoltFlow, error) {
+
+	logger.Infow(ctx, "Building DS PPPoE flow", log.Fields{"Port": vpv.Port, "ML": vpv.MacLearning, "Mac": vpv.MacAddr})
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+
+	vpv.setDsMatchVlan(subFlow)
+	subFlow.SetPppoeDiscoveryMatch()
+
+	if NonZeroMacAddress(vpv.MacAddr) {
+		subFlow.SetMatchDstMac(vpv.MacAddr)
+	}
+
+	uniport, _ := GetApplication().GetPortID(vpv.Port)
+	nni, err := GetApplication().GetNniPort(vpv.Device)
+	if err != nil {
+		return nil, err
+	}
+	nniport, err := GetApplication().GetPortID(nni)
+	if err != nil {
+		return nil, err
+	}
+	subFlow.SetInPort(nniport)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+	metadata := uint64(uniport)
+	subFlow.SetWriteMetadata(metadata)
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata = uint64(allowTransparent)<<56 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+	subFlow.SetReportToController()
+	//| 12-bit cvlan | 4 bits empty | <32-bits uniport>| 16-bits dhcp mask or flow mask |
+	subFlow.Cookie = uint64(vpv.CVlan)<<52 | uint64(uniport)<<16 | of.PppoeFlowMask | of.DsFlowMask
+	subFlow.Priority = of.PppoeFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built DS DHCP flow ", log.Fields{"cookie": subFlow.Cookie, "Flow": flow})
+	return flow, nil
+}
+
+// setDsMatchVlan to set downstream match vlan
+func (vpv *VoltPortVnet) setDsMatchVlan(flow *of.VoltSubFlow) {
+	switch vpv.VlanControl {
+	case None:
+		flow.SetMatchVlan(vpv.SVlan)
+	case ONUCVlanOLTSVlan,
+		OLTCVlanOLTSVlan,
+		ONUCVlan,
+		OLTSVlan:
+		flow.SetMatchVlan(vpv.SVlan)
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+	}
+}
+
+// BuildIgmpFlows builds the US IGMP flows for a subscriber. IGMP requires flows only
+// in the US direction.
+func (vpv *VoltPortVnet) BuildIgmpFlows() (*of.VoltFlow, error) {
+	logger.Infow(ctx, "Building US IGMP Flow", log.Fields{"Port": vpv.Port})
+	mvp := GetApplication().GetMvlanProfileByName(vpv.MvlanProfileName)
+	if mvp == nil {
+		return nil, errors.New("Mvlan Profile configured not found")
+	}
+	mvlan := mvp.GetUsMatchVlan()
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	subFlow := of.NewVoltSubFlow()
+	subFlow.SetTableID(0)
+
+        if GetApplication().GetVendorID() == Radisys {
+                if err := vpv.setUsMatchVlan(subFlow); err != nil {
+                        return nil, err
+                }
+        } else {
+                subFlow.SetMatchVlan(vpv.UniVlan)
+                subFlow.SetSetVlan(vpv.CVlan)
+        }
+
+	uniport, err := GetApplication().GetPortID(vpv.Port)
+	if err != nil {
+		return nil, err
+	}
+	subFlow.SetInPort(uniport)
+	// PortName and PortID to be used for validation of port before flow pushing
+	flow.PortID = uniport
+	flow.PortName = vpv.Port
+
+	if vpv.MacLearning == MacLearningNone && NonZeroMacAddress(vpv.MacAddr) {
+		subFlow.SetMatchSrcMac(vpv.MacAddr)
+	}
+	logger.Infow(ctx, "Mvlan", log.Fields{"mvlan": mvlan})
+	//metadata := uint64(mvlan)
+
+	if vpv.McastService {
+		metadata := uint64(vpv.McastUsMeterID)
+		metadata = metadata | uint64(vpv.McastTechProfileID)<<32
+		subFlow.SetMatchPbit(vpv.McastPbit)
+		subFlow.SetMeterID(vpv.McastUsMeterID)
+		subFlow.SetWriteMetadata(metadata)
+	} else {
+		// Set techprofile, meterid of first service
+		vpv.services.Range(func(key, value interface{}) bool {
+			svc := value.(*VoltService)
+			writemetadata := uint64(svc.TechProfileID) << 32
+			subFlow.SetWriteMetadata(writemetadata)
+			subFlow.SetMeterID(svc.UsMeterID)
+			return false
+		})
+	}
+
+	allowTransparent := 0
+	if vpv.AllowTransparent {
+		allowTransparent = 1
+	}
+	metadata := uint64(allowTransparent)<<56 | uint64(vpv.SchedID)<<40 | uint64(vpv.ONTEtherTypeClassification)<<36 | uint64(vpv.VlanControl)<<32 | uint64(vpv.UniVlan)<<16 | uint64(vpv.CVlan)
+	subFlow.SetTableMetadata(metadata)
+	subFlow.SetIgmpMatch()
+	subFlow.SetReportToController()
+	//| 16 bits empty | <32-bits uniport>| 16-bits igmp mask or flow mask |
+	subFlow.Cookie = uint64(uniport)<<16 | of.IgmpFlowMask | of.UsFlowMask
+	subFlow.Priority = of.IgmpFlowPriority
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "Built US IGMP flow ", log.Fields{"cookie": subFlow.Cookie, "flow": flow})
+	return flow, nil
+}
+
+// WriteToDb for writing to database
+func (vpv *VoltPortVnet) WriteToDb() {
+	if vpv.DeleteInProgress {
+		logger.Warnw(ctx, "Skipping Redis Update for VPV, VPV delete in progress", log.Fields{"Vnet": vpv.VnetName, "Port": vpv.Port})
+		return
+	}
+	vpv.ForceWriteToDb()
+}
+
+//ForceWriteToDb force commit a VPV to the DB
+func (vpv *VoltPortVnet) ForceWriteToDb() {
+	vpv.PendingFlowLock.RLock()
+	defer vpv.PendingFlowLock.RUnlock()
+	vpv.Version = database.PresentVersionMap[database.VpvPath]
+	if b, err := json.Marshal(vpv); err == nil {
+		if err := db.PutVpv(vpv.Port, uint16(vpv.SVlan), uint16(vpv.CVlan), uint16(vpv.UniVlan), string(b)); err != nil {
+			logger.Warnw(ctx, "VPV write to DB failed", log.Fields{"port": vpv.Port, "SVlan": vpv.SVlan, "CVlan": vpv.CVlan,
+				"UniVlan": vpv.UniVlan, "Error": err})
+		}
+	}
+}
+
+// DelFromDb for deleting from database
+func (vpv *VoltPortVnet) DelFromDb() {
+	logger.Debugw(ctx, "Deleting VPV from DB", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan})
+	_ = db.DelVpv(vpv.Port, uint16(vpv.SVlan), uint16(vpv.CVlan), uint16(vpv.UniVlan))
+}
+
+// ClearAllServiceFlags to clear all service flags
+func (vpv *VoltPortVnet) ClearAllServiceFlags() {
+	vpv.services.Range(ClearFlagsInService)
+}
+
+// ClearAllVpvFlags to clear all vpv flags
+func (vpv *VoltPortVnet) ClearAllVpvFlags() {
+	vpv.PendingFlowLock.Lock()
+	vpv.FlowsApplied = false
+	vpv.IgmpFlowsApplied = false
+	vpv.PendingDeleteFlow = make(map[string]bool)
+	vpv.PendingFlowLock.Unlock()
+	vpv.WriteToDb()
+	logger.Debugw(ctx, "Cleared Flow Flags for VPV",
+		log.Fields{"device": vpv.Device, "port": vpv.Port,
+			"svlan": vpv.SVlan, "cvlan": vpv.CVlan, "univlan": vpv.UniVlan})
+}
+
+// CreateVpvFromString to create vpv from string
+func (va *VoltApplication) CreateVpvFromString(b []byte, hash string) {
+	var vpv VoltPortVnet
+	if err := json.Unmarshal(b, &vpv); err == nil {
+		vnetsByPortsSliceIntf, ok := va.VnetsByPort.Load(vpv.Port)
+		if !ok {
+			va.VnetsByPort.Store(vpv.Port, []*VoltPortVnet{})
+			vnetsByPortsSliceIntf = []*VoltPortVnet{}
+		}
+		vpv.servicesCount = atomic.NewUint64(0)
+		vnetsByPortsSlice := vnetsByPortsSliceIntf.([]*VoltPortVnet)
+		vnetsByPortsSlice = append(vnetsByPortsSlice, &vpv)
+		va.VnetsByPort.Store(vpv.Port, vnetsByPortsSlice)
+		va.UpdateMacInPortMap(vpv.MacAddr, vpv.Port)
+		if vnet := va.GetVnetByName(vpv.VnetName); vnet != nil {
+			vnet.associatePortToVnet(vpv.Port)
+		}
+
+		if vpv.DeleteInProgress {
+			va.VoltPortVnetsToDelete[&vpv] = true
+			logger.Warnw(ctx, "VPV (restored) to be deleted", log.Fields{"Port": vpv.Port, "Vnet": vpv.VnetName})
+		}
+		logger.Debugw(ctx, "Added VPV from string", log.Fields{"port": vpv.Port, "svlan": vpv.SVlan, "cvlan": vpv.CVlan, "univlan": vpv.UniVlan})
+	}
+}
+
+// RestoreVpvsFromDb to restore vpvs from database
+func (va *VoltApplication) RestoreVpvsFromDb() {
+	// VNETS must be learnt first
+	vpvs, _ := db.GetVpvs()
+	for hash, vpv := range vpvs {
+		b, ok := vpv.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		va.CreateVpvFromString(b, hash)
+	}
+}
+
+// GetVnetByPort : VNET related functionality of VOLT Application here on.
+// Get the VNET from a port. The port identity is passed as device and port identities in string.
+// The identity of the VNET is the SVLAN and the CVLAN. Only if the both match the VLAN
+// is assumed to have matched. TODO: 1:1 should be treated differently and needs to be addressed
+func (va *VoltApplication) GetVnetByPort(port string, svlan of.VlanType, cvlan of.VlanType, univlan of.VlanType) *VoltPortVnet {
+	if _, ok := va.VnetsByPort.Load(port); !ok {
+		return nil
+	}
+	vpvs, _ := va.VnetsByPort.Load(port)
+	for _, vpv := range vpvs.([]*VoltPortVnet) {
+		if vpv.MatchesVlans(svlan, cvlan, univlan) {
+			return vpv
+		}
+	}
+	return nil
+}
+
+// AddVnetToPort to add vnet to port
+func (va *VoltApplication) AddVnetToPort(port string, vvnet *VoltVnet, vs *VoltService) *VoltPortVnet {
+	// The VNET is not on the port and is to be added
+	logger.Debugw(ctx, "Adding VNET to Port", log.Fields{"Port": port, "VNET": vvnet.Name})
+	vpv := NewVoltPortVnet(vvnet)
+	vpv.MacLearning = vvnet.MacLearning
+	vpv.Port = port
+	vvnet.associatePortToVnet(port)
+	if _, ok := va.VnetsByPort.Load(port); !ok {
+		va.VnetsByPort.Store(port, []*VoltPortVnet{})
+	}
+	vpvsIntf, _ := va.VnetsByPort.Load(port)
+	vpvs := vpvsIntf.([]*VoltPortVnet)
+	vpvs = append(vpvs, vpv)
+	va.VnetsByPort.Store(port, vpvs)
+	va.UpdateMacInPortMap(vpv.MacAddr, vpv.Port)
+
+	vpv.VpvLock.Lock()
+	defer vpv.VpvLock.Unlock()
+
+	// Add the service that is causing the VNET to be added to the port
+	vpv.AddSvc(vs)
+
+	// Process the PORT UP if the port is already up
+	d, err := va.GetDeviceFromPort(port)
+	if err == nil {
+		vpv.setDevice(d.Name)
+		p := d.GetPort(port)
+		if p != nil {
+
+			if vs.PonPort != 0xFF && vs.PonPort != p.PonPort {
+				logger.Errorw(ctx, "UNI port discovered on wrong PON Port. Dropping Flow Push for VPV", log.Fields{"Device": d.Name, "Port": port, "DetectedPon": p.PonPort, "ExpectedPon": vs.PonPort, "Vnet": vpv.VnetName})
+			} else {
+				logger.Infow(ctx, "Checking UNI port state", log.Fields{"State": p.State})
+				if d.State == controller.DeviceStateUP && p.State == PortStateUp {
+					vpv.PortUpInd(d, port)
+				}
+			}
+		}
+	}
+	vpv.WriteToDb()
+	return vpv
+}
+
+// DelVnetFromPort for deleting vnet from port
+func (va *VoltApplication) DelVnetFromPort(port string, vpv *VoltPortVnet) {
+
+	//Delete DHCP Session
+	delDhcpSessions(vpv.LearntMacAddr, vpv.SVlan, vpv.CVlan, vpv.DHCPv6DUID)
+
+	//Delete PPPoE session
+	delPppoeIaSessions(vpv.LearntMacAddr, vpv.SVlan, vpv.CVlan)
+
+	//Delete Mac from MacPortMap
+	va.DeleteMacInPortMap(vpv.MacAddr)
+
+	//Delete VPV
+	vpvsIntf, ok := va.VnetsByPort.Load(port)
+	if !ok {
+		return
+	}
+	vpvs := vpvsIntf.([]*VoltPortVnet)
+	for i, lvpv := range vpvs {
+		if lvpv == vpv {
+			logger.Debugw(ctx, "Deleting VPV from port", log.Fields{"Port": vpv.Port, "SVLAN": vpv.SVlan, "CVLAN": vpv.CVlan,
+				"UNIVLAN": vpv.UniVlan})
+
+			vpvs = append(vpvs[0:i], vpvs[i+1:]...)
+
+			vpv.DeleteInProgress = true
+			vpv.ForceWriteToDb()
+
+			va.VnetsByPort.Store(port, vpvs)
+			vpv.DelTrapFlows()
+			vpv.DelHsiaFlows()
+			va.DisassociateVpvsFromDevice(vpv.Device, vpv)
+			vpv.PendingFlowLock.RLock()
+			if len(vpv.PendingDeleteFlow) == 0 {
+				vpv.DelFromDb()
+			}
+			if vnet := va.GetVnetByName(vpv.VnetName); vnet != nil {
+				vnet.disassociatePortFromVnet(vpv.Device, vpv.Port)
+			}
+			vpv.PendingFlowLock.RUnlock()
+			return
+		}
+	}
+}
+
+// RestoreVnetsFromDb to restore vnet from port
+func (va *VoltApplication) RestoreVnetsFromDb() {
+	// VNETS must be learnt first
+	vnets, _ := db.GetVnets()
+	for _, net := range vnets {
+		b, ok := net.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		var vnet VoltVnet
+		err := json.Unmarshal(b, &vnet)
+		if err != nil {
+			logger.Warn(ctx, "Unmarshal of VNET failed")
+			continue
+		}
+		logger.Debugw(ctx, "Retrieved VNET", log.Fields{"VNET": vnet.VnetConfig})
+		if err := va.AddVnet(vnet.VnetConfig, &vnet.VnetOper); err != nil {
+			logger.Warnw(ctx, "Add Vnet Failed", log.Fields{"Config": vnet.VnetConfig, "Error": err})
+		}
+
+		if vnet.DeleteInProgress {
+			va.VnetsToDelete[vnet.Name] = true
+			logger.Warnw(ctx, "Vnet (restored) to be deleted", log.Fields{"Vnet": vnet.Name})
+		}
+
+	}
+}
+
+// GetServiceFromCvlan : Locate a service based on the packet received. The packet contains VLANs that
+// are used as the key to locate the service. If more than one service is on the
+// same port (essentially a UNI of ONU), the services must be separated by different
+// CVLANs
+func (va *VoltApplication) GetServiceFromCvlan(device, port string, vlans []of.VlanType, priority uint8) *VoltService {
+	// Fetch the device first to make sure the device exists
+	dIntf, ok := va.DevicesDisc.Load(device)
+	if !ok {
+		return nil
+	}
+	d := dIntf.(*VoltDevice)
+
+	// If the port is NNI port, the services dont exist on it. The svc then
+	// must be obtained from a different context and is not included here
+	if port == d.NniPort {
+		return nil
+	}
+
+	//To return the matched service
+	var service *VoltService
+
+	// This is an access port and the port should have all the associated
+	// services which can be uniquely identified by the VLANs in the packet
+	vnets, ok := va.VnetsByPort.Load(port)
+
+	if !ok {
+		logger.Debugw(ctx, "No Vnets for port", log.Fields{"Port": port})
+		return nil
+	}
+	logger.Debugw(ctx, "Matching for VLANs", log.Fields{"VLANs": vlans, "Priority": priority})
+	for _, vnet := range vnets.([]*VoltPortVnet) {
+		logger.Infow(ctx, "Vnet", log.Fields{"Vnet": vnet})
+		switch vnet.VlanControl {
+		case ONUCVlanOLTSVlan:
+			service = vnet.MatchesPriority(priority)
+			if vnet.MatchesCvlan(vlans) && service != nil {
+				return service
+			}
+		case ONUCVlan,
+			None:
+			service = vnet.MatchesPriority(priority)
+			// In case of DHCP Flow - cvlan == VlanNone
+			// In case of HSIA Flow - cvlan == Svlan
+			if len(vlans) == 1 && (vlans[0] == vnet.SVlan || vlans[0] == of.VlanNone) && service != nil {
+				return service
+			}
+		case OLTCVlanOLTSVlan,
+			OLTSVlan:
+			service = vnet.MatchesPriority(priority)
+			if len(vlans) == 1 && vlans[0] == vnet.UniVlan && service != nil {
+				return service
+			}
+		default:
+			logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vnet.VlanControl})
+		}
+	}
+	return nil
+}
+
+// GetVnetFromFields : Locate a service based on the packet received. The packet contains VLANs that
+// are used as the key to locate the service. If more than one service is on the
+// same port (essentially a UNI of ONU), the services must be separated by different
+// CVLANs
+func (va *VoltApplication) GetVnetFromFields(device string, port string, vlans []of.VlanType, priority uint8) (*VoltPortVnet, *VoltService) {
+	// Fetch the device first to make sure the device exists
+	dIntf, ok := va.DevicesDisc.Load(device)
+	if !ok {
+		return nil, nil
+	}
+	d := dIntf.(*VoltDevice)
+
+	// If the port is NNI port, the services dont exist on it. The svc then
+	// must be obtained from a different context and is not included here
+	if port == d.NniPort {
+		return nil, nil
+	}
+
+	//To return the matched service
+	var service *VoltService
+
+	// This is an access port and the port should have all the associated
+	// services which can be uniquely identified by the VLANs in the packet
+	if vnets, ok := va.VnetsByPort.Load(port); ok {
+		logger.Debugw(ctx, "Matching for VLANs", log.Fields{"VLANs": vlans, "Priority": priority})
+		for _, vnet := range vnets.([]*VoltPortVnet) {
+			logger.Infow(ctx, "Vnet", log.Fields{"Vnet": vnet})
+			switch vnet.VlanControl {
+			case ONUCVlanOLTSVlan:
+				service = vnet.MatchesPriority(priority)
+				if vnet.MatchesCvlan(vlans) && service != nil {
+					return vnet, service
+				}
+			case ONUCVlan,
+				None:
+				service = vnet.MatchesPriority(priority)
+				if (len(vlans) == 1 || vnet.AllowTransparent) && vlans[0] == vnet.SVlan && service != nil {
+					return vnet, service
+				}
+			case OLTCVlanOLTSVlan,
+				OLTSVlan:
+				service = vnet.MatchesPriority(priority)
+				if (len(vlans) == 1 || vnet.AllowTransparent) && vlans[0] == vnet.UniVlan && service != nil {
+					return vnet, service
+				}
+			default:
+				logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vnet.VlanControl})
+			}
+		}
+	}
+	return nil, nil
+}
+
+// GetVnetFromPkt : Locate a service based on the packet received. The packet contains VLANs that
+// are used as the key to locate the service. If more than one service is on the
+// same port (essentially a UNI of ONU), the services must be separated by different
+// CVLANs
+func (va *VoltApplication) GetVnetFromPkt(device string, port string, pkt gopacket.Packet) (*VoltPortVnet, *VoltService) {
+	vlans := GetVlans(pkt)
+	priority := GetPriority(pkt)
+	return va.GetVnetFromFields(device, port, vlans, priority)
+}
+
+// PushDevFlowForVlan to push icmpv6 flows for vlan
+func (va *VoltApplication) PushDevFlowForVlan(vnet *VoltVnet) {
+	logger.Infow(ctx, "PushDevFlowForVlan", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan})
+	pushflow := func(key interface{}, value interface{}) bool {
+		device := value.(*VoltDevice)
+		if !isDeviceInList(device.SerialNum, vnet.DevicesList) {
+			logger.Info(ctx, "Device not present in vnet device list", log.Fields{"Device": device.SerialNum})
+			return true
+		}
+		if device.State != controller.DeviceStateUP {
+			logger.Errorw(ctx, "Push Dev Flows Failed - Device state DOWN", log.Fields{"Port": device.NniPort, "Vlan": vnet.SVlan, "device": device})
+			return true
+		}
+		if applied, ok := device.VlanPortStatus.Load(uint16(vnet.SVlan)); !ok || !applied.(bool) {
+			logger.Errorw(ctx, "Push Dev Flows Failed - Vlan not enabled yet", log.Fields{"Port": device.NniPort, "Vlan": vnet.SVlan})
+			return true
+		}
+
+		if vnetListIntf, ok := device.ConfiguredVlanForDeviceFlows.Get(VnetKey(vnet.SVlan, vnet.CVlan, 0)); ok {
+			vnetList := vnetListIntf.(*util.ConcurrentMap)
+			vnetList.Set(vnet.Name, true)
+			device.ConfiguredVlanForDeviceFlows.Set(VnetKey(vnet.SVlan, vnet.CVlan, 0), vnetList)
+			logger.Infow(ctx, "Flow already pushed for these Vlans. Adding profile to list", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan, "vnetList-len": vnetList.Length()})
+			return true
+		}
+		logger.Debugw(ctx, "Configuring Dev Flows Group for device ", log.Fields{"Device": device})
+		err := ProcessIcmpv6McGroup(device.Name, false)
+		if err != nil {
+			logger.Warnw(ctx, "Configuring Dev Flows Group for device failed ", log.Fields{"Device": device.Name, "err": err})
+			return true
+		}
+		if portID, err := va.GetPortID(device.NniPort); err == nil {
+			if state, _ := cntlr.GetController().GetPortState(device.Name, device.NniPort); state != cntlr.PortStateUp {
+				logger.Warnw(ctx, "Skipping Dev Flow Configuration - Port Down", log.Fields{"Device": device})
+				return true
+			}
+
+			//Pushing ICMPv6 Flow
+			flow := BuildICMPv6Flow(portID, vnet)
+			err = cntlr.GetController().AddFlows(device.NniPort, device.Name, flow)
+			if err != nil {
+				logger.Warnw(ctx, "Configuring ICMPv6 Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+				return true
+			}
+			logger.Infow(ctx, "ICMPv6 Flow Added to Queue", log.Fields{"flow": flow})
+
+			// Pushing ARP Flow
+			flow = BuildDSArpFlow(portID, vnet)
+			err = cntlr.GetController().AddFlows(device.NniPort, device.Name, flow)
+			if err != nil {
+				logger.Warnw(ctx, "Configuring ARP Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+				return true
+			}
+			logger.Infow(ctx, "ARP Flow Added to Queue", log.Fields{"flow": flow})
+
+			vnetList := util.NewConcurrentMap()
+			vnetList.Set(vnet.Name, true)
+			device.ConfiguredVlanForDeviceFlows.Set(VnetKey(vnet.SVlan, vnet.CVlan, 0), vnetList)
+		}
+		return true
+	}
+	va.DevicesDisc.Range(pushflow)
+}
+
+// PushDevFlowForDevice to push icmpv6 flows for device
+func (va *VoltApplication) PushDevFlowForDevice(device *VoltDevice) {
+	logger.Infow(ctx, "PushDevFlowForDevice", log.Fields{"device": device})
+
+	logger.Debugw(ctx, "Configuring ICMPv6 Group for device ", log.Fields{"Device": device.Name})
+	err := ProcessIcmpv6McGroup(device.Name, false)
+	if err != nil {
+		logger.Warnw(ctx, "Configuring ICMPv6 Group for device failed ", log.Fields{"Device": device.Name, "err": err})
+		return
+	}
+	pushicmpv6 := func(key, value interface{}) bool {
+		vnet := value.(*VoltVnet)
+		if vnetListIntf, ok := device.ConfiguredVlanForDeviceFlows.Get(VnetKey(vnet.SVlan, vnet.CVlan, 0)); ok {
+			vnetList := vnetListIntf.(*util.ConcurrentMap)
+			vnetList.Set(vnet.Name, true)
+			device.ConfiguredVlanForDeviceFlows.Set(VnetKey(vnet.SVlan, vnet.CVlan, 0), vnetList)
+			logger.Infow(ctx, "Flow already pushed for these Vlans. Adding profile to list", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan, "vnetList-len": vnetList.Length()})
+			return true
+		}
+		nniPortID, err := va.GetPortID(device.NniPort)
+		if err != nil {
+			logger.Errorw(ctx, "Push ICMPv6 Failed - Failed to get NNI Port Id", log.Fields{"Port": device.NniPort, "Reason": err.Error})
+		}
+		if applied, ok := device.VlanPortStatus.Load(uint16(vnet.SVlan)); !ok || !applied.(bool) {
+			logger.Warnw(ctx, "Push ICMPv6 Failed - Vlan not enabled yet", log.Fields{"Port": device.NniPort, "Vlan": vnet.SVlan})
+			return true
+		}
+		flow := BuildICMPv6Flow(nniPortID, vnet)
+		err = cntlr.GetController().AddFlows(device.NniPort, device.Name, flow)
+		if err != nil {
+			logger.Warnw(ctx, "Configuring ICMPv6 Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+			return true
+		}
+		logger.Infow(ctx, "ICMP Flow Added to Queue", log.Fields{"flow": flow})
+
+		flow = BuildDSArpFlow(nniPortID, vnet)
+		err = cntlr.GetController().AddFlows(device.NniPort, device.Name, flow)
+		if err != nil {
+			logger.Warnw(ctx, "Configuring ARP Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+			return true
+		}
+		logger.Infow(ctx, "ARP Flow Added to Queue", log.Fields{"flow": flow})
+
+		vnetList := util.NewConcurrentMap()
+		vnetList.Set(vnet.Name, true)
+		device.ConfiguredVlanForDeviceFlows.Set(VnetKey(vnet.SVlan, vnet.CVlan, 0), vnetList)
+		return true
+	}
+	va.VnetsByName.Range(pushicmpv6)
+}
+
+// DeleteDevFlowForVlan to delete icmpv6 flow for vlan
+func (va *VoltApplication) DeleteDevFlowForVlan(vnet *VoltVnet) {
+	logger.Infow(ctx, "DeleteDevFlowForVlan", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan})
+	delflows := func(key interface{}, value interface{}) bool {
+		device := value.(*VoltDevice)
+
+		if vnetListIntf, ok := device.ConfiguredVlanForDeviceFlows.Get(VnetKey(vnet.SVlan, vnet.CVlan, 0)); ok {
+			vnetList := vnetListIntf.(*util.ConcurrentMap)
+			vnetList.Remove(vnet.Name)
+			device.ConfiguredVlanForDeviceFlows.Set(VnetKey(vnet.SVlan, vnet.CVlan, 0), vnetList)
+			if vnetList.Length() != 0 {
+				logger.Warnw(ctx, "Similar VNet associated to diff service. Not removing ICMPv6 flow", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan, "vnetList-len": vnetList.Length()})
+				return true
+			}
+		}
+		if portID, err := va.GetPortID(device.NniPort); err == nil {
+			if state, _ := cntlr.GetController().GetPortState(device.Name, device.NniPort); state != cntlr.PortStateUp {
+				logger.Warnw(ctx, "Skipping ICMPv6 Flow Deletion - Port Down", log.Fields{"Device": device})
+				return true
+			}
+			//Pushing ICMPv6 Flow
+			flow := BuildICMPv6Flow(portID, vnet)
+			flow.ForceAction = true
+			err := vnet.RemoveFlows(device, flow)
+			if err != nil {
+				logger.Warnw(ctx, "De-Configuring ICMPv6 Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+				return true
+			}
+			logger.Infow(ctx, "ICMPv6 Flow Delete Added to Queue", log.Fields{"flow": flow})
+
+			//Pushing ARP Flow
+			flow = BuildDSArpFlow(portID, vnet)
+			flow.ForceAction = true
+			err = vnet.RemoveFlows(device, flow)
+			if err != nil {
+				logger.Warnw(ctx, "De-Configuring ARP Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+				return true
+			}
+			logger.Infow(ctx, "ARP Flow Delete Added to Queue", log.Fields{"flow": flow})
+
+			device.ConfiguredVlanForDeviceFlows.Remove(VnetKey(vnet.SVlan, vnet.CVlan, 0))
+		}
+		return true
+	}
+	va.DevicesDisc.Range(delflows)
+}
+
+// DeleteDevFlowForDevice to delete icmpv6 flow for device
+func (va *VoltApplication) DeleteDevFlowForDevice(device *VoltDevice) {
+	logger.Infow(ctx, "DeleteDevFlowForDevice", log.Fields{"Device": device})
+	delicmpv6 := func(key, value interface{}) bool {
+		vnet := value.(*VoltVnet)
+		if vnetListIntf, ok := device.ConfiguredVlanForDeviceFlows.Get(VnetKey(vnet.SVlan, vnet.CVlan, 0)); ok {
+			vnetList := vnetListIntf.(*util.ConcurrentMap)
+			vnetList.Remove(vnet.Name)
+			device.ConfiguredVlanForDeviceFlows.Set(VnetKey(vnet.SVlan, vnet.CVlan, 0), vnetList)
+			if vnetList.Length() != 0 {
+				logger.Warnw(ctx, "Similar VNet associated to diff service. Not removing ICMPv6 flow", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan, "vnetList-len": vnetList.Length()})
+				return true
+			}
+		} else {
+			logger.Warnw(ctx, "ICMPv6 Flow map entry not found for Vnet", log.Fields{"Vnet": vnet.VnetConfig})
+			return true
+		}
+		nniPortID, err := va.GetPortID(device.NniPort)
+		if err != nil {
+			logger.Errorw(ctx, "Delete ICMPv6 Failed - Failed to get NNI Port Id", log.Fields{"Port": device.NniPort, "Reason": err.Error})
+		}
+		flow := BuildICMPv6Flow(nniPortID, vnet)
+		flow.ForceAction = true
+		err = vnet.RemoveFlows(device, flow)
+		if err != nil {
+			logger.Warnw(ctx, "De-Configuring ICMPv6 Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+			return true
+		}
+
+		flow = BuildDSArpFlow(nniPortID, vnet)
+		flow.ForceAction = true
+		err = vnet.RemoveFlows(device, flow)
+		if err != nil {
+			logger.Warnw(ctx, "De-Configuring ARP Flow for device failed ", log.Fields{"Device": device.Name, "err": err})
+			return true
+		}
+
+		device.ConfiguredVlanForDeviceFlows.Remove(VnetKey(vnet.SVlan, vnet.CVlan, 0))
+		logger.Infow(ctx, "ICMP Flow Delete Added to Queue", log.Fields{"flow": flow})
+		return true
+	}
+	va.VnetsByName.Range(delicmpv6)
+	logger.Debugw(ctx, "De-Configuring ICMPv6 Group for device ", log.Fields{"Device": device.Name})
+	err := ProcessIcmpv6McGroup(device.Name, true)
+	if err != nil {
+		logger.Warnw(ctx, "De-Configuring ICMPv6 Group on device failed ", log.Fields{"Device": device.Name, "err": err})
+		return
+	}
+}
+
+// DeleteDevFlowForVlanFromDevice to delete icmpv6 flow for vlan from device
+func (va *VoltApplication) DeleteDevFlowForVlanFromDevice(vnet *VoltVnet, deviceSerialNum string) {
+	logger.Infow(ctx, "DeleteDevFlowForVlanFromDevice", log.Fields{"Device-serialNum": deviceSerialNum, "SVlan": vnet.SVlan, "CVlan": vnet.CVlan})
+	delflows := func(key interface{}, value interface{}) bool {
+		device := value.(*VoltDevice)
+		if device.SerialNum != deviceSerialNum {
+			return true
+		}
+		if vnetListIntf, ok := device.ConfiguredVlanForDeviceFlows.Get(VnetKey(vnet.SVlan, vnet.CVlan, 0)); ok {
+			vnetList := vnetListIntf.(*util.ConcurrentMap)
+			vnetList.Remove(vnet.Name)
+			device.ConfiguredVlanForDeviceFlows.Set(VnetKey(vnet.SVlan, vnet.CVlan, 0), vnetList)
+			if vnetList.Length() != 0 {
+				logger.Warnw(ctx, "Similar VNet associated to diff service. Not removing ICMPv6 flow", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan, "vnetList-len": vnetList.Length()})
+				return true
+			}
+		} else if !vgcRebooted && len(vnet.DevicesList) != 0 {
+			// Return only in-case of non-reboot/delete scenario. Else, the flows need to be force removed
+			// DeviceList check is there to avoid dangling flow in-case of pod restart during service de-activation.
+			// The step will be as follow:
+			// 1. Deact Service
+			// 2. Pod Reboot
+			// 3. Pending Delete Service triggered
+			// 4. Del Service Ind followed by DelVnet req from NB
+			// 5. If Vlan status response is awaited, the ConfiguredVlanForDeviceFlows cache will not have flow info
+			// hence the flow will not be cleared
+			logger.Warnw(ctx, "Dev Flow map entry not found for Vnet", log.Fields{"PodReboot": vgcRebooted, "VnetDeleteInProgress": vnet.DeleteInProgress})
+			return true
+		}
+		if portID, err := va.GetPortID(device.NniPort); err == nil {
+			if state, _ := cntlr.GetController().GetPortState(device.Name, device.NniPort); state != cntlr.PortStateUp {
+				logger.Warnw(ctx, "Skipping ICMPv6 Flow Deletion - Port Down", log.Fields{"Device": device})
+				return false
+			}
+			flow := BuildICMPv6Flow(portID, vnet)
+			flow.ForceAction = true
+			if err := vnet.RemoveFlows(device, flow); err != nil {
+				logger.Warnw(ctx, "Delete Flow Failed", log.Fields{"Device": device, "Flow": flow, "Error": err})
+			}
+			logger.Infow(ctx, "ICMP Flow Delete Added to Queue", log.Fields{"flow": flow})
+
+			flow = BuildDSArpFlow(portID, vnet)
+			flow.ForceAction = true
+			if err := vnet.RemoveFlows(device, flow); err != nil {
+				logger.Warnw(ctx, "Delete Flow Failed", log.Fields{"Device": device, "Flow": flow, "Error": err})
+			}
+			logger.Infow(ctx, "ARP Flow Delete Added to Queue", log.Fields{"flow": flow})
+			device.ConfiguredVlanForDeviceFlows.Remove(VnetKey(vnet.SVlan, vnet.CVlan, 0))
+		}
+		return false
+	}
+	va.DevicesDisc.Range(delflows)
+}
+
+// BuildICMPv6Flow to Build DS flow for ICMPv6
+func BuildICMPv6Flow(inport uint32, vnet *VoltVnet) *of.VoltFlow {
+	logger.Info(ctx, "Building ICMPv6 MC Flow", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan})
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	subFlow := of.NewVoltSubFlow()
+
+	subFlow.SetICMPv6Match()
+	subFlow.SetMatchVlan(vnet.SVlan)
+	subFlow.SetInPort(inport)
+	subFlow.SetPopVlan()
+	subFlow.SetOutGroup(ICMPv6ArpGroupID)
+	subFlow.Cookie = uint64(vnet.CVlan)<<48 | uint64(vnet.SVlan)<<32 | of.IgmpFlowMask | of.DsFlowMask
+	subFlow.Priority = of.McFlowPriority
+	var metadata uint64
+	if vnet.VlanControl == None {
+		metadata = uint64(ONUCVlan)<<32 | uint64(vnet.CVlan)
+	} else {
+		metadata = uint64(vnet.VlanControl)<<32 | uint64(vnet.CVlan)
+	}
+	subFlow.SetTableMetadata(metadata)
+	metadata = uint64(vnet.setPbitRemarking())
+
+	logger.Infow(ctx, "ICMPv6 Pbit Remarking", log.Fields{"RemarkPbit": metadata})
+	subFlow.SetWriteMetadata(metadata)
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	return flow
+}
+
+//BuildDSArpFlow Builds DS flow for ARP
+func BuildDSArpFlow(inport uint32, vnet *VoltVnet) *of.VoltFlow {
+	logger.Info(ctx, "Building ARP MC Flow", log.Fields{"SVlan": vnet.SVlan, "CVlan": vnet.CVlan})
+
+	flow := &of.VoltFlow{}
+	flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	subFlow := of.NewVoltSubFlow()
+
+	BcastMAC, _ := net.ParseMAC("FF:FF:FF:FF:FF:FF")
+	subFlow.SetArpMatch()
+	subFlow.SetMatchDstMac(BcastMAC)
+	subFlow.SetMatchVlan(vnet.SVlan)
+	subFlow.SetInPort(inport)
+	subFlow.SetPopVlan()
+	subFlow.SetOutGroup(ICMPv6ArpGroupID)
+
+	subFlow.Cookie = uint64(vnet.CVlan)<<48 | uint64(vnet.SVlan)<<32 | of.DsArpFlowMask | of.DsFlowMask
+	subFlow.Priority = of.McFlowPriority
+
+	var metadata uint64
+	if vnet.VlanControl == None {
+		metadata = uint64(ONUCVlan)<<32 | uint64(vnet.CVlan)
+	} else {
+		metadata = uint64(vnet.VlanControl)<<32 | uint64(vnet.CVlan)
+	}
+	subFlow.SetTableMetadata(metadata)
+	metadata = uint64(vnet.setPbitRemarking())
+	subFlow.SetWriteMetadata(metadata)
+
+	flow.SubFlows[subFlow.Cookie] = subFlow
+	logger.Infow(ctx, "ARP Pbit Remarking", log.Fields{"RemarkPbit": metadata})
+	return flow
+}
+
+// setPbitRemarking to set Pbit remarking
+func (vv *VoltVnet) setPbitRemarking() uint32 {
+
+	// 	                              Remarkable
+	// 	         Remarked Pbit          Pbit
+	// |-----------------------------| |------|
+	// |7| |6| |5| |4| |3| |2| |1| |0| 76543210
+	// 000 000 000 000 000 000 000 000 00000000
+
+	// Eg:
+	// For 6:3 & 7:1
+	// 001 011 000 000 000 000 000 000 11000000
+
+	var remarkable uint8
+	var remarked uint32
+	for refPbit, remarkPbit := range vv.CtrlPktPbitRemark {
+		remarkable = remarkable | 1<<refPbit
+		remarked = remarked | uint32(remarkPbit)<<(refPbit*3)
+	}
+	return remarked<<8 | uint32(remarkable)
+}
+
+// ProcessIcmpv6McGroup to add icmpv6 multicast group
+func ProcessIcmpv6McGroup(device string, delete bool) error {
+
+	logger.Info(ctx, "Creating ICMPv6 MC Group")
+	va := GetApplication()
+	vd := va.GetDevice(device)
+	group := &of.Group{}
+	group.GroupID = ICMPv6ArpGroupID
+	group.Device = device
+	if delete {
+		if !vd.icmpv6GroupAdded {
+			logger.Info(ctx, "ICMPv6 MC Group is already deleted. Ignoring  icmpv6 group Delete")
+			return nil //TODO
+		}
+		vd.icmpv6GroupAdded = false
+		group.Command = of.GroupCommandDel
+		group.ForceAction = true
+	} else {
+		if vd.icmpv6GroupAdded {
+			logger.Info(ctx, "ICMPv6 MC Group is already added. Ignoring icmpv6 group Add")
+			return nil //TODO
+		}
+		vd.icmpv6GroupAdded = true
+		group.Command = of.GroupCommandAdd
+		receivers := GetApplication().GetIcmpv6Receivers(device)
+		group.Buckets = append(group.Buckets, receivers...)
+	}
+	logger.Infow(ctx, "ICMPv6 MC Group Action", log.Fields{"Device": device, "Delete": delete})
+	port, _ := GetApplication().GetNniPort(device)
+	err := cntlr.GetController().GroupUpdate(port, device, group)
+	return err
+}
+
+//isVlanMatching - checks is vlans matches with vpv based on vlan control
+func (vpv *VoltPortVnet) isVlanMatching(cvlan of.VlanType, svlan of.VlanType) bool {
+
+	switch vpv.VlanControl {
+	case ONUCVlanOLTSVlan,
+		OLTCVlanOLTSVlan:
+		if vpv.SVlan == svlan && vpv.CVlan == cvlan {
+			return true
+		}
+	case ONUCVlan,
+		OLTSVlan,
+		None:
+		if vpv.SVlan == svlan {
+			return true
+		}
+	default:
+		logger.Errorw(ctx, "Invalid Vlan Control Option", log.Fields{"Value": vpv.VlanControl})
+	}
+	return false
+}
+
+//PushFlows - Triggers flow addition after registering for flow indication event
+func (vpv *VoltPortVnet) PushFlows(device *VoltDevice, flow *of.VoltFlow) error {
+
+	for cookie := range flow.SubFlows {
+		cookie := strconv.FormatUint(cookie, 10)
+		fe := &FlowEvent{
+			eType:     EventTypeControlFlowAdded,
+			cookie:    cookie,
+			eventData: vpv,
+		}
+		device.RegisterFlowAddEvent(cookie, fe)
+	}
+	return cntlr.GetController().AddFlows(vpv.Port, device.Name, flow)
+}
+
+//FlowInstallFailure - Process flow failure indication and triggers HSIA failure for all associated services
+func (vpv *VoltPortVnet) FlowInstallFailure(cookie string, errorCode uint32, errReason string) {
+
+	sendFlowFailureInd := func(key, value interface{}) bool {
+		//svc := value.(*VoltService)
+		//TODO-COMM: svc.triggerServiceFailureInd(errorCode, errReason)
+		return true
+	}
+	logger.Errorw(ctx, "Control Flow Add Failure Notification", log.Fields{"uniPort": vpv.Port, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason})
+	vpv.services.Range(sendFlowFailureInd)
+}
+
+//RemoveFlows - Triggers flow deletion after registering for flow indication event
+func (vpv *VoltPortVnet) RemoveFlows(device *VoltDevice, flow *of.VoltFlow) error {
+
+	vpv.PendingFlowLock.Lock()
+	defer vpv.PendingFlowLock.Unlock()
+
+	for cookie := range flow.SubFlows {
+		cookie := strconv.FormatUint(cookie, 10)
+		fe := &FlowEvent{
+			eType:     EventTypeControlFlowRemoved,
+			device:    device.Name,
+			cookie:    cookie,
+			eventData: vpv,
+		}
+		device.RegisterFlowDelEvent(cookie, fe)
+		vpv.PendingDeleteFlow[cookie] = true
+	}
+	return cntlr.GetController().DelFlows(vpv.Port, device.Name, flow)
+}
+
+//CheckAndDeleteVpv - remove VPV from DB is there are no pending flows to be removed
+func (vpv *VoltPortVnet) CheckAndDeleteVpv() {
+	vpv.PendingFlowLock.RLock()
+	defer vpv.PendingFlowLock.RUnlock()
+	if !vpv.DeleteInProgress {
+		return
+	}
+	if len(vpv.PendingDeleteFlow) == 0 && !vpv.FlowsApplied {
+		logger.Infow(ctx, "All Flows removed for VPV. Triggering VPV Deletion from DB", log.Fields{"VPV Port": vpv.Port, "Device": vpv.Device, "Vnet": vpv.VnetName})
+		vpv.DelFromDb()
+		logger.Infow(ctx, "Deleted VPV from DB/Cache successfully", log.Fields{"VPV Port": vpv.Port, "Device": vpv.Device, "Vnet": vpv.VnetName})
+	}
+}
+
+//FlowRemoveSuccess - Process flow success indication
+func (vpv *VoltPortVnet) FlowRemoveSuccess(cookie string, device string) {
+	vpv.PendingFlowLock.Lock()
+	logger.Infow(ctx, "VPV Flow Remove Success Notification", log.Fields{"Port": vpv.Port, "Cookie": cookie, "Device": device})
+
+	delete(vpv.PendingDeleteFlow, cookie)
+	vpv.PendingFlowLock.Unlock()
+	vpv.CheckAndDeleteVpv()
+	vpv.WriteToDb()
+}
+
+//FlowRemoveFailure - Process flow failure indication and triggers Del HSIA failure for all associated services
+func (vpv *VoltPortVnet) FlowRemoveFailure(cookie string, device string, errorCode uint32, errReason string) {
+	vpv.PendingFlowLock.Lock()
+
+	logger.Errorw(ctx, "VPV Flow Remove Failure Notification", log.Fields{"Port": vpv.Port, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason, "Device": device})
+
+	sendFlowFailureInd := func(key, value interface{}) bool {
+		svc := value.(*VoltService)
+		svc.triggerServiceFailureInd(errorCode, errReason)
+		return true
+	}
+	logger.Errorw(ctx, "Control Flow Del Failure Notification", log.Fields{"uniPort": vpv.Port, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason})
+	vpv.services.Range(sendFlowFailureInd)
+
+	if vpv.DeleteInProgress {
+		delete(vpv.PendingDeleteFlow, cookie)
+		vpv.PendingFlowLock.Unlock()
+		vpv.CheckAndDeleteVpv()
+	} else {
+		vpv.PendingFlowLock.Unlock()
+		vpv.WriteToDb()
+	}
+}
+
+//RemoveFlows - Triggers flow deletion after registering for flow indication event
+func (vv *VoltVnet) RemoveFlows(device *VoltDevice, flow *of.VoltFlow) error {
+
+	vv.VnetLock.Lock()
+	defer vv.VnetLock.Unlock()
+
+	var flowMap map[string]bool
+	var ok bool
+
+	for cookie := range flow.SubFlows {
+		cookie := strconv.FormatUint(cookie, 10)
+		fe := &FlowEvent{
+			eType:     EventTypeDeviceFlowRemoved,
+			device:    device.Name,
+			cookie:    cookie,
+			eventData: vv,
+		}
+		device.RegisterFlowDelEvent(cookie, fe)
+		if flowMap, ok = vv.PendingDeleteFlow[device.Name]; !ok {
+			flowMap = make(map[string]bool)
+		}
+		flowMap[cookie] = true
+		vv.PendingDeleteFlow[device.Name] = flowMap
+	}
+	vv.WriteToDb()
+	return cntlr.GetController().DelFlows(device.NniPort, device.Name, flow)
+}
+
+//CheckAndDeleteVnet - remove Vnet from DB is there are no pending flows to be removed
+func (vv *VoltVnet) CheckAndDeleteVnet(device string) {
+	if !vv.DeleteInProgress {
+		return
+	}
+	vv.VnetPortLock.RLock()
+	if len(vv.PendingDeleteFlow[device]) == 0 && !vv.isAssociatedPortsPresent() {
+		logger.Warnw(ctx, "Deleting Vnet : All flows removed", log.Fields{"Name": vv.Name, "AssociatedPorts": vv.AssociatedPorts, "Device": device})
+		GetApplication().deleteVnetConfig(vv)
+		_ = db.DelVnet(vv.Name)
+		logger.Infow(ctx, "Deleted Vnet from DB/Cache successfully", log.Fields{"Device": device, "Vnet": vv.Name})
+	} else {
+		logger.Warnw(ctx, "Skipping Del Vnet", log.Fields{"Name": vv.Name, "AssociatedPorts": vv.AssociatedPorts, "PendingDelFlows": vv.PendingDeleteFlow[device]})
+	}
+	vv.VnetPortLock.RUnlock()
+}
+
+//FlowRemoveSuccess - Process flow success indication
+func (vv *VoltVnet) FlowRemoveSuccess(cookie string, device string) {
+	vv.VnetLock.Lock()
+	defer vv.VnetLock.Unlock()
+
+	logger.Infow(ctx, "Vnet Flow Remove Success Notification", log.Fields{"VnetProfile": vv.Name, "Cookie": cookie, "Device": device})
+
+	if _, ok := vv.PendingDeleteFlow[device]; ok {
+		delete(vv.PendingDeleteFlow[device], cookie)
+	}
+
+	//Check and update success for pending disable request
+	if d := GetApplication().GetDevice(device); d != nil {
+		_, present := d.ConfiguredVlanForDeviceFlows.Get(VnetKey(vv.SVlan, vv.CVlan, 0))
+		if !present && len(vv.PendingDeleteFlow[device]) == 0 {
+			vv.CheckAndDeleteVnet(device)
+		}
+	}
+	vv.WriteToDb()
+}
+
+//FlowRemoveFailure - Process flow failure indication
+func (vv *VoltVnet) FlowRemoveFailure(cookie string, device string, errorCode uint32, errReason string) {
+
+	vv.VnetLock.Lock()
+	defer vv.VnetLock.Unlock()
+
+	if flowMap, ok := vv.PendingDeleteFlow[device]; ok {
+		if _, ok := flowMap[cookie]; ok {
+			logger.Errorw(ctx, "Device Flow Remove Failure Notification", log.Fields{"Vnet": vv.Name, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason, "Device": device})
+
+			if vv.DeleteInProgress {
+				delete(vv.PendingDeleteFlow[device], cookie)
+				vv.CheckAndDeleteVnet(device)
+			}
+			return
+		}
+	}
+	logger.Errorw(ctx, "Device Flow Remove Failure Notification for Unknown cookie", log.Fields{"Vnet": vv.Name, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason})
+}
+
+//IgmpFlowInstallFailure - Process flow failure indication and triggers HSIA failure for Igmp enabled services
+func (vpv *VoltPortVnet) IgmpFlowInstallFailure(cookie string, errorCode uint32, errReason string) {
+
+	//Note: Current implementation supports only for single service with Igmp Enabled for a subscriber
+	//When multiple Igmp-suported service enabled, comment "return false"
+
+	sendFlowFailureInd := func(key, value interface{}) bool {
+		svc := value.(*VoltService)
+		if svc.IgmpEnabled {
+			svc.triggerServiceFailureInd(errorCode, errReason)
+			return false
+		}
+		return true
+	}
+	logger.Errorw(ctx, "US IGMP Flow Failure Notification", log.Fields{"uniPort": vpv.Port, "Cookie": cookie, "ErrorCode": errorCode, "ErrorReason": errReason})
+	vpv.services.Range(sendFlowFailureInd)
+}
+
+// GetMatchingMcastService to get matching multicast service
+func (va *VoltApplication) GetMatchingMcastService(port string, device string, cvlan of.VlanType) *VoltService {
+
+	var service *VoltService
+	dIntf, ok := va.DevicesDisc.Load(device)
+	if !ok {
+		return nil
+	}
+	d := dIntf.(*VoltDevice)
+
+	// If the port is NNI port, the services dont exist on it. The svc then
+	// must be obtained from a different context and is not included here
+	if port == d.NniPort {
+		return nil
+	}
+
+	// This is an access port and the port should have all the associated
+	// services which can be uniquely identified by the VLANs in the packet
+	vnets, ok := va.VnetsByPort.Load(port)
+
+	if !ok {
+		logger.Debugw(ctx, "No Vnets for port", log.Fields{"Port": port})
+		return nil
+	}
+	logger.Debugw(ctx, "Matching for VLANs", log.Fields{"VLANs": cvlan})
+	getMcastService := func(key, value interface{}) bool {
+		srv := value.(*VoltService)
+		if srv.IgmpEnabled {
+			service = srv
+
+			//TODO: Current implementation supports only for single service with Igmp Enabled
+			//FIX-ME:  When multiple service suports Igmp, update of logic required
+			return false
+		}
+		return true
+	}
+
+	for _, vpv := range vnets.([]*VoltPortVnet) {
+		if vpv.CVlan == cvlan {
+			vpv.services.Range(getMcastService)
+			if service != nil {
+				break
+			}
+		}
+	}
+	return service
+}
+
+//TriggerAssociatedFlowDelete - Re-trigger delete for pending delete flows
+func (vv *VoltVnet) TriggerAssociatedFlowDelete(device string) bool {
+	vv.VnetLock.Lock()
+	cookieList := []uint64{}
+	flowMap := vv.PendingDeleteFlow[device]
+
+	for cookie := range flowMap {
+		cookieList = append(cookieList, convertToUInt64(cookie))
+	}
+	vv.VnetLock.Unlock()
+
+	if len(cookieList) == 0 {
+		return false
+	}
+
+	for _, cookie := range cookieList {
+		if vd := GetApplication().GetDevice(device); vd != nil {
+			flow := &of.VoltFlow{}
+			flow.SubFlows = make(map[uint64]*of.VoltSubFlow)
+			subFlow := of.NewVoltSubFlow()
+			subFlow.Cookie = cookie
+			flow.SubFlows[cookie] = subFlow
+			logger.Infow(ctx, "Retriggering Vnet Delete Flow", log.Fields{"Device": device, "Vnet": vv.Name, "Cookie": cookie})
+			if err := vv.RemoveFlows(vd, flow); err != nil {
+				logger.Warnw(ctx, "Vnet Delete Flow Failed", log.Fields{"Device": device, "Vnet": vv.Name, "Cookie": cookie, "Error": err})
+			}
+		}
+	}
+	return true
+}
diff --git a/internal/pkg/controller/addflows.go b/internal/pkg/controller/addflows.go
new file mode 100644
index 0000000..0364341
--- /dev/null
+++ b/internal/pkg/controller/addflows.go
@@ -0,0 +1,217 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	infraerror "voltha-go-controller/internal/pkg/errorcodes"
+	infraerrorcode "voltha-go-controller/internal/pkg/errorcodes/service"
+	"time"
+
+	"voltha-go-controller/internal/pkg/of"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	//MaxRetryCount - Maximum retry attempts on failure
+	MaxRetryCount int = 1
+)
+
+// AddFlowsTask structure
+type AddFlowsTask struct {
+	taskID    uint8
+	ctx       context.Context
+	flow      *of.VoltFlow
+	device    *Device
+	timestamp string
+}
+
+// NewAddFlowsTask is constructor for AddFlowsTask
+func NewAddFlowsTask(ctx context.Context, flow *of.VoltFlow, device *Device) *AddFlowsTask {
+	var aft AddFlowsTask
+	aft.device = device
+	aft.flow = flow
+	aft.ctx = ctx
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	aft.timestamp = tstamp
+	return &aft
+}
+
+// Name to add flow task
+func (aft *AddFlowsTask) Name() string {
+	for _, flow := range aft.flow.SubFlows {
+		logger.Infow(ctx, "Flow Cookies", log.Fields{"Cookie": flow.Cookie})
+	}
+	return "Add Flows Task"
+}
+
+// TaskID to return task ID
+func (aft *AddFlowsTask) TaskID() uint8 {
+	return aft.taskID
+}
+
+// Timestamp to return timestamp
+func (aft *AddFlowsTask) Timestamp() string {
+	return aft.timestamp
+}
+
+// Stop to stop the add flow task
+func (aft *AddFlowsTask) Stop() {
+}
+
+// Start to start adding flow task
+func (aft *AddFlowsTask) Start(ctx context.Context, taskID uint8) error {
+	var err error
+	aft.taskID = taskID
+	aft.ctx = ctx
+	flowsToProcess := make(map[uint64]*of.VoltSubFlow)
+	flowsPresent := 0
+	// First add/delete the flows first locally before passing them to actual device
+	for _, flow := range aft.flow.SubFlows {
+		logger.Infow(ctx, "Flow Mod Request", log.Fields{"Cookie": flow.Cookie, "Oper": aft.flow.Command, "Port": aft.flow.PortID})
+		if aft.flow.Command == of.CommandAdd {
+			flow.State = of.FlowAddPending
+			if err := aft.device.AddFlow(flow); err != nil {
+				logger.Warnw(ctx, "Add Flow Error", log.Fields{"Cookie": flow.Cookie, "Reason": err.Error()})
+
+				// If flow already exists in cache, check for flow state
+				// If Success: Trigger success FLow Indication
+				// if Failure: Continue process, so add-retry happens
+				if err.Error() == ErrDuplicateFlow {
+					dbFlow, _ := aft.device.GetFlow(flow.Cookie)
+					if dbFlow.State == of.FlowAddSuccess {
+						aft.device.triggerFlowNotification(flow.Cookie, aft.flow.Command, of.BwAvailDetails{}, nil)
+						flowsPresent++
+					}
+				}
+			}
+			flowsToProcess[flow.Cookie] = flow
+		} else {
+			dbFlow, ok := aft.device.GetFlow(flow.Cookie)
+			if !ok {
+				logger.Warnw(ctx, "Delete Flow Error: Flow Does not Exist", log.Fields{"Cookie": flow.Cookie, "Device": aft.device.ID})
+			} else {
+				// dbFlow.State = of.FlowDelPending
+				// aft.device.AddFlowToDb(dbFlow)
+				flowsToProcess[flow.Cookie] = dbFlow
+			}
+			aft.device.triggerFlowNotification(flow.Cookie, aft.flow.Command, of.BwAvailDetails{}, nil)
+		}
+	}
+
+	if flowsPresent == len(aft.flow.SubFlows) {
+		logger.Warn(ctx, "All Flows already present in database. Skipping Flow Push to SB")
+	}
+
+	// PortName and PortID are used for validation of PortID, whether it is still valid and associated with old PortName or
+	// PortID got assigned to another PortName. If the condition met, skip these flow update to voltha core
+	if aft.flow.PortName != "" && aft.flow.PortID != 0 {
+		portName, _ := aft.device.GetPortName(aft.flow.PortID)
+		if aft.flow.PortName != portName && portName != "" {
+			for _, flow := range aft.flow.SubFlows {
+				logger.Errorw(ctx, "Skip Flow Update", log.Fields{"Reason": "Port Deleted", "PortName": aft.flow.PortName, "PortNo": aft.flow.PortID, "Cookie": flow.Cookie, "Operation": aft.flow.Command})
+				if aft.flow.Command == of.CommandDel {
+					aft.device.triggerFlowNotification(flow.Cookie, aft.flow.Command, of.BwAvailDetails{}, nil)
+				}
+			}
+			return nil
+		}
+	}
+
+	if !aft.device.isSBOperAllowed(aft.flow.ForceAction) {
+		for _, flow := range aft.flow.SubFlows {
+			logger.Errorw(ctx, "Skipping Flow Table Update", log.Fields{"Reason": "Device State not UP", "State": aft.device.State, "Cookie": flow.Cookie, "Operation": aft.flow.Command})
+		}
+		return nil
+	}
+
+	flows := of.ProcessVoltFlow(aft.device.ID, aft.flow.Command, flowsToProcess)
+	for _, flow := range flows {
+		attempt := 0
+		if vc := aft.device.VolthaClient(); vc != nil {
+			for {
+				if _, err = vc.UpdateLogicalDeviceFlowTable(aft.ctx, flow); err != nil {
+					logger.Errorw(ctx, "Update Flow Table Failed", log.Fields{"Cookie": flow.GetFlowMod().Cookie, "Reason": err.Error(), "Operation": aft.flow.Command})
+					statusCode, _ := infraerror.GetErrorInfo(err)
+
+					// Retry on flow delete failure once.
+					// Do NOT retry incase of failure with reason: Entry Not Found
+					if aft.flow.Command == of.CommandDel && statusCode != uint32(infraerrorcode.ErrNotExists) {
+						if attempt != MaxRetryCount {
+							logger.Errorw(ctx, "Retrying Flow Delete", log.Fields{"Cookie": flow.GetFlowMod().Cookie, "Attempt": attempt})
+							attempt++
+							continue
+						}
+						logger.Errorw(ctx, "Flow Delete failed even aft max retries", log.Fields{"Flow": flow, "Attempt": attempt})
+					}
+				}
+				break
+			}
+			aft.device.triggerFlowNotification(flow.FlowMod.Cookie, aft.flow.Command, of.BwAvailDetails{}, nil)
+
+		} else {
+			logger.Errorw(ctx, "Update Flow Table Failed: Voltha Client Unavailable", log.Fields{"Flow": flow})
+		}
+	}
+	return nil
+}
+
+func isFlowOperSuccess(statusCode uint32, oper of.Command) bool {
+	volthaErrorCode := infraerrorcode.ErrorCode(statusCode)
+
+	if volthaErrorCode == infraerrorcode.ErrOk {
+		return true
+	}
+
+	if oper == of.CommandAdd && volthaErrorCode == infraerrorcode.ErrAlreadyExists {
+		return true
+
+	} else if oper == of.CommandDel && volthaErrorCode == infraerrorcode.ErrNotExists {
+		return true
+	}
+	return false
+}
+
+// func getBwAvailInfo(bwAvailInfo []*voltha.ResponseMsg) of.BwAvailDetails {
+// 	var bwInfo of.BwAvailDetails
+// 	// convert the bw details sent from olt to a struct
+// 	// received msg format:
+// 	// additional_data[Data{ResponseMsg
+// 	//{"key":"prevBW","value":"111111"},
+// 	//{"key":"presentBW","value":"10000"}]
+// 	if len(bwAvailInfo) > 1 {
+// 		prevBwResp := bwAvailInfo[0]
+// 		if prevBwResp.Key == of.PrevBwInfo {
+// 			_, err := strconv.Atoi(prevBwResp.Val)
+// 			if err == nil {
+// 				bwInfo.PrevBw = prevBwResp.Val
+// 			}
+// 		}
+
+// 		presentBwResp := bwAvailInfo[1]
+// 		if presentBwResp.Key == of.PresentBwInfo {
+// 			_, err := strconv.Atoi(prevBwResp.Val)
+// 			if err == nil {
+// 				bwInfo.PresentBw = presentBwResp.Val
+// 			}
+// 		}
+// 		if bwInfo.PresentBw == bwInfo.PrevBw {
+// 			return of.BwAvailDetails{}
+// 		}
+// 		logger.Infow(ctx, "Bandwidth-consumed-info", log.Fields{"BwConsumed": bwInfo})
+// 	}
+// 	return bwInfo
+// }
diff --git a/internal/pkg/controller/auditdevice.go b/internal/pkg/controller/auditdevice.go
new file mode 100644
index 0000000..ea00185
--- /dev/null
+++ b/internal/pkg/controller/auditdevice.go
@@ -0,0 +1,210 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	"time"
+
+	"voltha-go-controller/internal/pkg/tasks"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+)
+
+// AuditEventType type
+type AuditEventType uint8
+
+const (
+	// AuditEventDeviceDisc constant
+	AuditEventDeviceDisc AuditEventType = 0
+	// AuditEventDeviceStateChange constant
+	AuditEventDeviceStateChange AuditEventType = 1
+)
+
+const (
+	// NNIPortID NNI port id
+	NNIPortID uint32 = 0x1000000
+)
+
+// AuditDevice structure
+type AuditDevice struct {
+	taskID    uint8
+	ctx       context.Context
+	device    *Device
+	stop      bool
+	timestamp string
+	event     AuditEventType
+}
+
+// NewAuditDevice is constructor for AuditDevice
+func NewAuditDevice(device *Device, event AuditEventType) *AuditDevice {
+	var ad AuditDevice
+	ad.device = device
+	ad.stop = false
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	ad.timestamp = tstamp
+	ad.event = event
+	return &ad
+}
+
+// Name returns the task name
+func (ad *AuditDevice) Name() string {
+	return "Device Audit Task"
+}
+
+// TaskID returns the task id
+func (ad *AuditDevice) TaskID() uint8 {
+	return ad.taskID
+}
+
+// Timestamp returns the timestamp for the task
+func (ad *AuditDevice) Timestamp() string {
+	return ad.timestamp
+}
+
+// Stop to stop the task
+func (ad *AuditDevice) Stop() {
+	ad.stop = true
+}
+
+// Start to start the task
+func (ad *AuditDevice) Start(ctx context.Context, taskID uint8) error {
+	logger.Warnw(ctx, "Audit Device Task Triggered", log.Fields{"Context": ctx, "taskId": taskID, "Device": ad.device.ID})
+	ad.taskID = taskID
+	ad.ctx = ctx
+
+	if ad.stop {
+		logger.Errorw(ctx, "Audit Device Task Cancelled", log.Fields{"Context": ad.ctx, "Task": ad.taskID})
+		return tasks.ErrTaskCancelError
+	}
+
+	ofpps, err := ad.device.VolthaClient().ListLogicalDevicePorts(ad.ctx, &common.ID{Id: ad.device.ID})
+	if err != nil {
+		return err
+	}
+
+	// Compute the difference between the ports received and ports at VGC
+	// First build a map of all the received ports under missing ports. We
+	// will eliminate the ports that are in the device from the missing ports
+	// so that the elements remaining are missing ports. The ones that are
+	// not in missing ports are added to excess ports which should be deleted
+	// from the VGC.
+	missingPorts := make(map[uint32]*ofp.OfpPort)
+	for _, ofpp := range ofpps.Items {
+		missingPorts[ofpp.OfpPort.PortNo] = ofpp.OfpPort
+	}
+
+	var excessPorts []uint32
+	GetController().SetAuditFlags(ad.device)
+
+	processPortState := func(id uint32, vgcPort *DevicePort) {
+		logger.Debugw(ctx, "Process Port State Ind", log.Fields{"Port No": vgcPort.ID, "Port Name": vgcPort.Name})
+
+		if ofpPort, ok := missingPorts[id]; ok {
+			if ((vgcPort.State == PortStateDown) && (ofpPort.State == uint32(ofp.OfpPortState_OFPPS_LIVE))) || ((vgcPort.State == PortStateUp) && (ofpPort.State != uint32(ofp.OfpPortState_OFPPS_LIVE))) {
+				// This port exists in the received list and the map at
+				// VGC. This is common so delete it
+				logger.Infow(ctx, "Port State Mismatch", log.Fields{"Port": vgcPort.ID, "OfpPort": ofpPort.PortNo, "ReceivedState": ofpPort.State, "CurrentState": vgcPort.State})
+				ad.device.ProcessPortState(ofpPort.PortNo, ofpPort.State)
+			} else {
+				//To ensure the flows are in sync with port status and no mismatch due to reboot,
+				// repush/delete flows based on current port status
+				logger.Infow(ctx, "Port State Processing", log.Fields{"Port": vgcPort.ID, "OfpPort": ofpPort.PortNo, "ReceivedState": ofpPort.State, "CurrentState": vgcPort.State})
+				ad.device.ProcessPortStateAfterReboot(ofpPort.PortNo, ofpPort.State)
+			}
+			delete(missingPorts, id)
+		} else {
+			// This port is missing from the received list. This is an
+			// excess port at VGC. This must be added to excess ports
+			excessPorts = append(excessPorts, id)
+		}
+		logger.Debugw(ctx, "Processed Port State Ind", log.Fields{"Port No": vgcPort.ID, "Port Name": vgcPort.Name})
+
+	}
+
+	// 1st process the NNI port before all other ports so that the device state can be updated.
+	if vgcPort, ok := ad.device.PortsByID[NNIPortID]; ok {
+		logger.Info(ctx, "Processing NNI port state")
+		processPortState(NNIPortID, vgcPort)
+	}
+
+	for id, vgcPort := range ad.device.PortsByID {
+		if id == NNIPortID {
+			//NNI port already processed
+			continue
+		}
+		if ad.stop {
+			break
+		}
+		processPortState(id, vgcPort)
+	}
+	GetController().ResetAuditFlags(ad.device)
+
+	if ad.stop {
+		logger.Errorw(ctx, "Audit Device Task Cancelled", log.Fields{"Context": ad.ctx, "Task": ad.taskID})
+		return tasks.ErrTaskCancelError
+	}
+	ad.AddMissingPorts(missingPorts)
+	ad.DelExcessPorts(excessPorts)
+	ad.device.deviceAuditInProgress = false
+	logger.Warnw(ctx, "Audit Device Task Completed", log.Fields{"Context": ctx, "taskId": taskID, "Device": ad.device.ID})
+	return nil
+}
+
+// AddMissingPorts to add the missing ports
+func (ad *AuditDevice) AddMissingPorts(mps map[uint32]*ofp.OfpPort) {
+	logger.Debugw(ctx, "Device Audit - Add Missing Ports", log.Fields{"NumPorts": len(mps)})
+
+	addMissingPort := func(mp *ofp.OfpPort) {
+		logger.Debugw(ctx, "Process Port Add Ind", log.Fields{"Port No": mp.PortNo, "Port Name": mp.Name})
+
+		// Error is ignored as it only drops duplicate ports
+		logger.Infow(ctx, "Calling AddPort", log.Fields{"No": mp.PortNo, "Name": mp.Name})
+		if err := ad.device.AddPort(mp.PortNo, mp.Name); err != nil {
+			logger.Warnw(ctx, "AddPort Failed", log.Fields{"No": mp.PortNo, "Name": mp.Name, "Reason": err})
+		}
+		if mp.State == uint32(ofp.OfpPortState_OFPPS_LIVE) {
+			ad.device.ProcessPortState(mp.PortNo, mp.State)
+		}
+		logger.Debugw(ctx, "Processed Port Add Ind", log.Fields{"Port No": mp.PortNo, "Port Name": mp.Name})
+
+	}
+
+	// 1st process the NNI port before all other ports so that the flow provisioning for UNIs can be enabled
+	if mp, ok := mps[NNIPortID]; ok {
+		logger.Info(ctx, "Adding Missing NNI port")
+		addMissingPort(mp)
+	}
+
+	for portNo, mp := range mps {
+		if portNo != NNIPortID {
+			addMissingPort(mp)
+		}
+	}
+}
+
+// DelExcessPorts to delete the excess ports
+func (ad *AuditDevice) DelExcessPorts(eps []uint32) {
+	logger.Debugw(ctx, "Device Audit - Delete Excess Ports", log.Fields{"NumPorts": len(eps)})
+	for _, id := range eps {
+		// Now delete the port from the device @ VGC
+		logger.Infow(ctx, "Device Audit - Deleting Port", log.Fields{"PortId": id})
+		if err := ad.device.DelPort(id); err != nil {
+			logger.Warnw(ctx, "DelPort Failed", log.Fields{"PortId": id, "Reason": err})
+		}
+	}
+}
diff --git a/internal/pkg/controller/audittables.go b/internal/pkg/controller/audittables.go
new file mode 100644
index 0000000..486d560
--- /dev/null
+++ b/internal/pkg/controller/audittables.go
@@ -0,0 +1,532 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	"strconv"
+	"time"
+
+	"voltha-go-controller/internal/pkg/intf"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/tasks"
+	"voltha-go-controller/internal/pkg/util"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+var (
+	rcvdGroups  map[uint32]*ofp.OfpGroupDesc
+	groupsToAdd []*of.Group
+	groupsToMod []*of.Group
+)
+
+// AuditTablesTask structure
+type AuditTablesTask struct {
+	taskID    uint8
+	ctx       context.Context
+	device    *Device
+	stop      bool
+	timestamp string
+}
+
+// NewAuditTablesTask is constructor for AuditTablesTask
+func NewAuditTablesTask(device *Device) *AuditTablesTask {
+	var att AuditTablesTask
+	att.device = device
+	att.stop = false
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	att.timestamp = tstamp
+	return &att
+}
+
+// Name returns name of the task
+func (att *AuditTablesTask) Name() string {
+	return "Audit Table Task"
+}
+
+// TaskID to return task id of the task
+func (att *AuditTablesTask) TaskID() uint8 {
+	return att.taskID
+}
+
+// Timestamp to return timestamp for the task
+func (att *AuditTablesTask) Timestamp() string {
+	return att.timestamp
+}
+
+// Stop to stop the task
+func (att *AuditTablesTask) Stop() {
+	att.stop = true
+}
+
+// Start is called by the framework and is responsible for implementing
+// the actual task.
+func (att *AuditTablesTask) Start(ctx context.Context, taskID uint8) error {
+	logger.Warnw(ctx, "Audit Table Task Triggered", log.Fields{"Context": ctx, "taskId": taskID, "Device": att.device.ID})
+	att.taskID = taskID
+	att.ctx = ctx
+	var errInfo error
+	var err error
+
+	// Audit the meters
+	if err = att.AuditMeters(); err != nil {
+		logger.Errorw(ctx, "Audit Meters Failed", log.Fields{"Reason": err.Error()})
+		errInfo = err
+	}
+
+	// Audit the Groups
+	if rcvdGroups, err = att.AuditGroups(); err != nil {
+		logger.Errorw(ctx, "Audit Groups Failed", log.Fields{"Reason": err.Error()})
+		errInfo = err
+	}
+
+	// Audit the flows
+	if err = att.AuditFlows(); err != nil {
+		logger.Errorw(ctx, "Audit Flows Failed", log.Fields{"Reason": err.Error()})
+		errInfo = err
+	}
+
+	// Triggering deletion of excess groups from device after the corresponding flows are removed
+	// to avoid flow dependency error during group deletion
+	logger.Infow(ctx, "Excess Groups", log.Fields{"Groups": rcvdGroups})
+	att.DelExcessGroups(rcvdGroups)
+	logger.Warnw(ctx, "Audit Table Task Completed", log.Fields{"Context": ctx, "taskId": taskID, "Device": att.device.ID})
+	return errInfo
+
+}
+
+// AuditMeters : Audit the meters which includes fetching the existing meters at the
+// voltha and identifying the delta between the ones held here and the
+// ones held at VOLTHA. The delta must be cleaned up to keep both the
+// components in sync
+func (att *AuditTablesTask) AuditMeters() error {
+
+	if att.stop {
+		return tasks.ErrTaskCancelError
+	}
+	var vc voltha.VolthaServiceClient
+	if vc = att.device.VolthaClient(); vc == nil {
+		logger.Error(ctx, "Fetch Device Meters Failed: Voltha Client Unavailable")
+		return nil
+	}
+
+	//-----------------------------
+	// Perform the audit of meters
+	// Fetch the meters
+	ms, err := vc.ListLogicalDeviceMeters(att.ctx, &voltha.ID{Id: att.device.ID})
+	if err != nil {
+		logger.Warnw(ctx, "Audit of flows failed", log.Fields{"Reason": err.Error()})
+		return err
+	}
+
+	// Build the map for easy and faster processing
+	rcvdMeters := make(map[uint32]*ofp.OfpMeterStats)
+	for _, m := range ms.Items {
+		rcvdMeters[m.Stats.MeterId] = m.Stats
+	}
+
+	// Verify all meters that are in the controller but not in the device
+	missingMeters := []*of.Meter{}
+	for _, meter := range att.device.meters {
+
+		if att.stop {
+			break
+		}
+		logger.Debugw(ctx, "Auditing Meter", log.Fields{"Id": meter.ID})
+
+		if _, ok := rcvdMeters[meter.ID]; ok {
+			// The meter exists in the device too. Just remove it from
+			// the received meters
+			delete(rcvdMeters, meter.ID)
+		} else {
+			// The flow exists at the controller but not at the device
+			// Push the flow to the device
+			logger.Debugw(ctx, "Adding Meter To Missing Meters", log.Fields{"Id": meter.ID})
+			missingMeters = append(missingMeters, meter)
+		}
+	}
+	if !att.stop {
+		att.AddMissingMeters(missingMeters)
+		att.DelExcessMeters(rcvdMeters)
+	} else {
+		err = tasks.ErrTaskCancelError
+	}
+	return err
+}
+
+// AddMissingMeters adds the missing meters detected by AuditMeters
+func (att *AuditTablesTask) AddMissingMeters(meters []*of.Meter) {
+	logger.Debugw(ctx, "Adding missing meters", log.Fields{"Number": len(meters)})
+	for _, meter := range meters {
+		meterMod, err := of.MeterUpdate(att.device.ID, of.MeterCommandAdd, meter)
+		if err != nil {
+			logger.Errorw(ctx, "Update Meter Table Failed", log.Fields{"Reason": err.Error()})
+			continue
+		}
+		if vc := att.device.VolthaClient(); vc != nil {
+			if _, err = vc.UpdateLogicalDeviceMeterTable(att.ctx, meterMod); err != nil {
+				logger.Errorw(ctx, "Update Meter Table Failed", log.Fields{"Reason": err.Error()})
+			}
+		} else {
+			logger.Error(ctx, "Update Meter Table Failed: Voltha Client Unavailable")
+		}
+	}
+}
+
+// DelExcessMeters to delete excess meters
+func (att *AuditTablesTask) DelExcessMeters(meters map[uint32]*ofp.OfpMeterStats) {
+	logger.Debugw(ctx, "Deleting Excess Meters", log.Fields{"Number": len(meters)})
+	for _, meter := range meters {
+		meterMod := &ofp.OfpMeterMod{}
+		meterMod.Command = ofp.OfpMeterModCommand_OFPMC_DELETE
+		meterMod.MeterId = meter.MeterId
+		meterUpd := &ofp.MeterModUpdate{Id: att.device.ID, MeterMod: meterMod}
+		if vc := att.device.VolthaClient(); vc != nil {
+			if _, err := vc.UpdateLogicalDeviceMeterTable(att.ctx, meterUpd); err != nil {
+				logger.Errorw(ctx, "Update Meter Table Failed", log.Fields{"Reason": err.Error()})
+			}
+		} else {
+			logger.Error(ctx, "Update Meter Table Failed: Voltha Client Unavailable")
+		}
+	}
+}
+
+// AuditFlows audit the flows which includes fetching the existing meters at the
+// voltha and identifying the delta between the ones held here and the
+// ones held at VOLTHA. The delta must be cleaned up to keep both the
+// components in sync
+func (att *AuditTablesTask) AuditFlows() error {
+
+	if att.stop {
+		return tasks.ErrTaskCancelError
+	}
+
+	var vc voltha.VolthaServiceClient
+	if vc = att.device.VolthaClient(); vc == nil {
+		logger.Error(ctx, "Flow Audit Failed: Voltha Client Unavailable")
+		return nil
+	}
+
+	// ---------------------------------
+	// Perform the audit of flows first
+	// Retrieve the flows from the device
+	f, err := vc.ListLogicalDeviceFlows(att.ctx, &common.ID{Id: att.device.ID})
+	if err != nil {
+		logger.Warnw(ctx, "Audit of flows failed", log.Fields{"Reason": err.Error()})
+		return err
+	}
+
+	defaultSuccessFlowStatus := intf.FlowStatus{
+		Device:      att.device.ID,
+		FlowModType: of.CommandAdd,
+		Status:      0,
+		Reason:      "",
+	}
+
+	// Build the map for easy and faster processing
+	rcvdFlows := make(map[uint64]*ofp.OfpFlowStats)
+	flowsToAdd := &of.VoltFlow{}
+	flowsToAdd.SubFlows = make(map[uint64]*of.VoltSubFlow)
+	for _, flow := range f.Items {
+		rcvdFlows[flow.Cookie] = flow
+	}
+
+	att.device.flowLock.Lock()
+	// Verify all flows that are in the controller but not in the device
+	for _, flow := range att.device.flows {
+
+		if att.stop {
+			break
+		}
+
+		logger.Debugw(ctx, "Auditing Flow", log.Fields{"Cookie": flow.Cookie})
+		if _, ok := rcvdFlows[flow.Cookie]; ok {
+			// The flow exists in the device too. Just remove it from
+			// the received flows & trigger flow success indication unless
+			// the flow in del failure/pending state
+
+			if flow.State != of.FlowDelFailure && flow.State != of.FlowDelPending {
+				delete(rcvdFlows, flow.Cookie)
+			}
+			defaultSuccessFlowStatus.Cookie = strconv.FormatUint(flow.Cookie, 10)
+
+			logger.Infow(ctx, "Triggering Internal Flow Notification", log.Fields{"Flow Status": defaultSuccessFlowStatus})
+			GetController().ProcessFlowModResultIndication(defaultSuccessFlowStatus)
+		} else {
+			// The flow exists at the controller but not at the device
+			// Push the flow to the device
+			logger.Debugw(ctx, "Adding Flow To Missing Flows", log.Fields{"Cookie": flow.Cookie})
+			flowsToAdd.SubFlows[flow.Cookie] = flow
+		}
+	}
+	att.device.flowLock.Unlock()
+
+	if !att.stop {
+		//  The flows remaining in the received flows are the excess flows at
+		// the device. Delete those flows
+		att.DelExcessFlows(rcvdFlows)
+		// Add the flows missing at the device
+		att.AddMissingFlows(flowsToAdd)
+	} else {
+		err = tasks.ErrTaskCancelError
+	}
+	return err
+}
+
+// AddMissingFlows : The flows missing from the device are reinstalled att the audit
+// The flows are added into a VoltFlow structure.
+func (att *AuditTablesTask) AddMissingFlows(mflow *of.VoltFlow) {
+	logger.Debugw(ctx, "Add Missing Flows", log.Fields{"Number": len(mflow.SubFlows)})
+	mflow.Command = of.CommandAdd
+	ofFlows := of.ProcessVoltFlow(att.device.ID, mflow.Command, mflow.SubFlows)
+	var vc voltha.VolthaServiceClient
+	var bwConsumedInfo of.BwAvailDetails
+	if vc = att.device.VolthaClient(); vc == nil {
+		logger.Error(ctx, "Update Flow Table Failed: Voltha Client Unavailable")
+		return
+	}
+	for _, flow := range ofFlows {
+		var dbFlow *of.VoltSubFlow
+		var present bool
+		if flow.FlowMod != nil {
+			if dbFlow, present = att.device.GetFlow(flow.FlowMod.Cookie); !present {
+				logger.Warn(ctx, "Flow Removed from DB. Ignoring Add Missing Flow", log.Fields{"Device": att.device.ID, "Cookie": flow.FlowMod.Cookie})
+				continue
+			}
+		}
+		var err error
+		if _, err = vc.UpdateLogicalDeviceFlowTable(att.ctx, flow); err != nil {
+			logger.Errorw(ctx, "Update Flow Table Failed", log.Fields{"Reason": err.Error()})
+		}
+		att.device.triggerFlowResultNotification(flow.FlowMod.Cookie, dbFlow, of.CommandAdd, bwConsumedInfo, err)
+	}
+}
+
+// DelExcessFlows delete the excess flows held at the VOLTHA
+func (att *AuditTablesTask) DelExcessFlows(flows map[uint64]*ofp.OfpFlowStats) {
+	logger.Debugw(ctx, "Deleting Excess Flows", log.Fields{"Number of Flows": len(flows)})
+
+	var vc voltha.VolthaServiceClient
+	if vc = att.device.VolthaClient(); vc == nil {
+		logger.Error(ctx, "Delete Excess Flows Failed: Voltha Client Unavailable")
+		return
+	}
+
+	// Let's cycle through the flows to delete the excess flows
+	for _, flow := range flows {
+
+		if _, present := att.device.GetFlow(flow.Cookie); present {
+			logger.Warn(ctx, "Flow Present in DB. Ignoring Delete Excess Flow", log.Fields{"Device": att.device.ID, "Cookie": flow.Cookie})
+			continue
+		}
+
+		logger.Debugw(ctx, "Deleting Flow", log.Fields{"Cookie": flow.Cookie})
+		// Create the flowMod structure and fill it out
+		flowMod := &ofp.OfpFlowMod{}
+		flowMod.Cookie = flow.Cookie
+		flowMod.TableId = flow.TableId
+		flowMod.Command = ofp.OfpFlowModCommand_OFPFC_DELETE_STRICT
+		flowMod.IdleTimeout = flow.IdleTimeout
+		flowMod.HardTimeout = flow.HardTimeout
+		flowMod.Priority = flow.Priority
+		flowMod.BufferId = of.DefaultBufferID
+		flowMod.OutPort = of.DefaultOutPort
+		flowMod.OutGroup = of.DefaultOutGroup
+		flowMod.Flags = flow.Flags
+		flowMod.Match = flow.Match
+		flowMod.Instructions = flow.Instructions
+
+		// Create FlowTableUpdate
+		flowUpdate := &ofp.FlowTableUpdate{
+			Id:      att.device.ID,
+			FlowMod: flowMod,
+		}
+
+		var err error
+		if _, err = vc.UpdateLogicalDeviceFlowTable(att.ctx, flowUpdate); err != nil {
+			logger.Errorw(ctx, "Flow Audit Delete Failed", log.Fields{"Reason": err.Error()})
+		}
+		att.device.triggerFlowResultNotification(flow.Cookie, nil, of.CommandDel, of.BwAvailDetails{}, err)
+	}
+}
+
+// AuditGroups audit the groups which includes fetching the existing groups at the
+// voltha and identifying the delta between the ones held here and the
+// ones held at VOLTHA. The delta must be cleaned up to keep both the
+// components in sync
+func (att *AuditTablesTask) AuditGroups() (map[uint32]*ofp.OfpGroupDesc, error) {
+
+	// Build the map for easy and faster processing
+	rcvdGroups = make(map[uint32]*ofp.OfpGroupDesc)
+
+	if att.stop {
+		return rcvdGroups, tasks.ErrTaskCancelError
+	}
+
+	var vc voltha.VolthaServiceClient
+	if vc = att.device.VolthaClient(); vc == nil {
+		logger.Error(ctx, "Group Audit Failed: Voltha Client Unavailable")
+		return rcvdGroups, nil
+	}
+
+	// ---------------------------------
+	// Perform the audit of groups first
+	// Retrieve the groups from the device
+	g, err := vc.ListLogicalDeviceFlowGroups(att.ctx, &common.ID{Id: att.device.ID})
+	if err != nil {
+		logger.Warnw(ctx, "Audit of groups failed", log.Fields{"Reason": err.Error()})
+		return rcvdGroups, err
+	}
+
+	groupsToAdd = []*of.Group{}
+	groupsToMod = []*of.Group{}
+	for _, group := range g.Items {
+		rcvdGroups[group.Desc.GroupId] = group.Desc
+	}
+	logger.Infow(ctx, "Received Groups", log.Fields{"Groups": rcvdGroups})
+
+	// Verify all groups that are in the controller but not in the device
+	att.device.groups.Range(att.compareGroupEntries)
+
+	if !att.stop {
+		// Add the groups missing at the device
+		logger.Infow(ctx, "Missing Groups", log.Fields{"Groups": groupsToAdd})
+		att.AddMissingGroups(groupsToAdd)
+
+		// Update groups with group member mismatch
+		logger.Infow(ctx, "Modify Groups", log.Fields{"Groups": groupsToMod})
+		att.UpdateMismatchGroups(groupsToMod)
+
+		// Note: Excess groups will be deleted after ensuring the connected
+		// flows are also removed as part fo audit flows
+	} else {
+		err = tasks.ErrTaskCancelError
+	}
+	// The groups remaining in the received groups are the excess groups at
+	// the device
+	return rcvdGroups, err
+}
+
+// compareGroupEntries to compare the group entries
+func (att *AuditTablesTask) compareGroupEntries(key, value interface{}) bool {
+
+	if att.stop {
+		return false
+	}
+
+	groupID := key.(uint32)
+	dbGroup := value.(*of.Group)
+	logger.Debugw(ctx, "Auditing Group", log.Fields{"Groupid": groupID})
+	if rcvdGrp, ok := rcvdGroups[groupID]; ok {
+		// The group exists in the device too.
+		// Compare the group members and add to modify list if required
+		compareGroupMembers(dbGroup, rcvdGrp)
+		delete(rcvdGroups, groupID)
+	} else {
+		// The group exists at the controller but not at the device
+		// Push the group to the device
+		logger.Debugw(ctx, "Adding Group To Missing Groups", log.Fields{"GroupId": groupID})
+		groupsToAdd = append(groupsToAdd, value.(*of.Group))
+	}
+	return true
+}
+
+func compareGroupMembers(refGroup *of.Group, rcvdGroup *ofp.OfpGroupDesc) {
+
+	portList := []uint32{}
+	refPortList := []uint32{}
+
+	//Collect port list from response Group Mod structure
+	//If PON is configured even for one group, then only PON shall be considered for compared for all groups
+	for _, bucket := range rcvdGroup.Buckets {
+		for _, actionBucket := range bucket.Actions {
+			if actionBucket.Type == ofp.OfpActionType_OFPAT_OUTPUT {
+				action := actionBucket.GetOutput()
+				portList = append(portList, action.Port)
+			}
+		}
+	}
+
+	refPortList = append(refPortList, refGroup.Buckets...)
+
+	//Is port list differs, trigger group update
+	if !util.IsSliceSame(refPortList, portList) {
+		groupsToMod = append(groupsToMod, refGroup)
+	}
+}
+
+//AddMissingGroups - addmissing groups to Voltha
+func (att *AuditTablesTask) AddMissingGroups(groupList []*of.Group) {
+	att.PushGroups(groupList, of.GroupCommandAdd)
+}
+
+//UpdateMismatchGroups - updates mismatched groups to Voltha
+func (att *AuditTablesTask) UpdateMismatchGroups(groupList []*of.Group) {
+	att.PushGroups(groupList, of.GroupCommandMod)
+}
+
+// PushGroups - The groups missing/to be updated in the device are reinstalled att the audit
+func (att *AuditTablesTask) PushGroups(groupList []*of.Group, grpCommand of.GroupCommand) {
+	logger.Debugw(ctx, "Pushing Groups", log.Fields{"Number": len(groupList), "Command": grpCommand})
+
+	var vc voltha.VolthaServiceClient
+	if vc = att.device.VolthaClient(); vc == nil {
+		logger.Error(ctx, "Update Group Table Failed: Voltha Client Unavailable")
+		return
+	}
+	for _, group := range groupList {
+		group.Command = grpCommand
+		groupUpdate := of.CreateGroupTableUpdate(group)
+		if _, err := vc.UpdateLogicalDeviceFlowGroupTable(att.ctx, groupUpdate); err != nil {
+			logger.Errorw(ctx, "Update Group Table Failed", log.Fields{"Reason": err.Error()})
+		}
+	}
+}
+
+// DelExcessGroups - Delete the excess groups held at the VOLTHA
+func (att *AuditTablesTask) DelExcessGroups(groups map[uint32]*ofp.OfpGroupDesc) {
+	logger.Debugw(ctx, "Deleting Excess Groups", log.Fields{"Number of Groups": len(groups)})
+
+	var vc voltha.VolthaServiceClient
+	if vc = att.device.VolthaClient(); vc == nil {
+		logger.Error(ctx, "Delete Excess Groups Failed: Voltha Client Unavailable")
+		return
+	}
+
+	// Let's cycle through the groups to delete the excess groups
+	for _, groupDesc := range groups {
+		logger.Debugw(ctx, "Deleting Group", log.Fields{"GroupId": groupDesc.GroupId})
+		group := &of.Group{}
+		group.Device = att.device.ID
+		group.GroupID = groupDesc.GroupId
+
+		//Group Members should be deleted before triggered group delete
+		group.Command = of.GroupCommandMod
+		groupUpdate := of.CreateGroupTableUpdate(group)
+		if _, err := vc.UpdateLogicalDeviceFlowGroupTable(att.ctx, groupUpdate); err != nil {
+			logger.Errorw(ctx, "Update Group Table Failed", log.Fields{"Reason": err.Error()})
+		}
+
+		group.Command = of.GroupCommandDel
+		groupUpdate = of.CreateGroupTableUpdate(group)
+		if _, err := vc.UpdateLogicalDeviceFlowGroupTable(att.ctx, groupUpdate); err != nil {
+			logger.Errorw(ctx, "Update Group Table Failed", log.Fields{"Reason": err.Error()})
+		}
+	}
+}
diff --git a/internal/pkg/controller/changeevent.go b/internal/pkg/controller/changeevent.go
new file mode 100644
index 0000000..95f6b07
--- /dev/null
+++ b/internal/pkg/controller/changeevent.go
@@ -0,0 +1,92 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	"errors"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+)
+
+// ChangeEventTask structure
+type ChangeEventTask struct {
+	taskID    uint8
+	ctx       context.Context
+	event     *ofp.ChangeEvent
+	device    *Device
+	timestamp string
+}
+
+// NewChangeEventTask is constructor for ChangeEventTask
+func NewChangeEventTask(ctx context.Context, event *ofp.ChangeEvent, device *Device) *ChangeEventTask {
+	var cet ChangeEventTask
+	cet.device = device
+	cet.event = event
+	cet.ctx = ctx
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	cet.timestamp = tstamp
+	return &cet
+}
+
+// Name returns the name of the task
+func (cet *ChangeEventTask) Name() string {
+	return "Change Event Task"
+}
+
+// TaskID to return task id of the task
+func (cet *ChangeEventTask) TaskID() uint8 {
+	return cet.taskID
+}
+
+// Timestamp to return timestamp for the task
+func (cet *ChangeEventTask) Timestamp() string {
+	return cet.timestamp
+}
+
+// Stop to stop the task
+func (cet *ChangeEventTask) Stop() {
+}
+
+// Start to start the Change event task
+func (cet *ChangeEventTask) Start(ctx context.Context, taskID uint8) error {
+	cet.taskID = taskID
+	cet.ctx = ctx
+
+	if status, ok := cet.event.Event.(*ofp.ChangeEvent_PortStatus); ok {
+		portNo := status.PortStatus.Desc.PortNo
+		portName := status.PortStatus.Desc.Name
+		state := status.PortStatus.Desc.State
+		logger.Infow(ctx, "Process Port Change Event", log.Fields{"Port No": portNo, "Port Name": portName, "State": state, "Reason": status.PortStatus.Reason})
+		if status.PortStatus.Reason == ofp.OfpPortReason_OFPPR_ADD {
+			_ = cet.device.AddPort(portNo, portName)
+			if state == uint32(ofp.OfpPortState_OFPPS_LIVE) {
+				cet.device.ProcessPortState(portNo, state)
+			}
+		} else if status.PortStatus.Reason == ofp.OfpPortReason_OFPPR_DELETE {
+			if err := cet.device.DelPort(portNo); err != nil {
+				logger.Warnw(ctx, "DelPort Failed", log.Fields{"Port No": portNo, "Error": err})
+			}
+		} else if status.PortStatus.Reason == ofp.OfpPortReason_OFPPR_MODIFY {
+			cet.device.ProcessPortUpdate(portName, portNo, state)
+		}
+		logger.Infow(ctx, "Processed Port Change Event", log.Fields{"Port No": portNo, "Port Name": portName, "State": state, "Reason": status.PortStatus.Reason})
+		return nil
+	}
+	return errors.New("Invalid message received")
+}
diff --git a/internal/pkg/controller/controller.go b/internal/pkg/controller/controller.go
new file mode 100644
index 0000000..ae34133
--- /dev/null
+++ b/internal/pkg/controller/controller.go
@@ -0,0 +1,523 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package controller
+
+import (
+	"context"
+	"errors"
+	"sync"
+	"time"
+
+	"encoding/hex"
+
+	"voltha-go-controller/database"
+	errorCodes "voltha-go-controller/internal/pkg/errorcodes"
+	"voltha-go-controller/internal/pkg/intf"
+	"voltha-go-controller/internal/pkg/of"
+	"voltha-go-controller/internal/pkg/tasks"
+	"voltha-go-controller/internal/pkg/util"
+	"voltha-go-controller/internal/pkg/vpagent"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+var ctx = context.TODO()
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
+
+var db database.DBIntf
+
+var deviceTableSyncDuration = 15 * time.Minute
+
+//SetDeviceTableSyncDuration - sets interval between device table sync up activity
+//  duration - in minutes
+func SetDeviceTableSyncDuration(duration int) {
+	deviceTableSyncDuration = time.Duration(duration) * time.Minute
+}
+
+// VoltController structure
+type VoltController struct {
+	rebootLock              sync.Mutex
+	rebootInProgressDevices map[string]string
+	devices                 map[string]*Device
+	deviceLock              sync.RWMutex
+	vagent                  map[string]*vpagent.VPAgent
+	ctx                     context.Context
+	app                     intf.App
+	RebootFlow              bool
+	BlockedDeviceList       *util.ConcurrentMap
+	deviceTaskQueue         *util.ConcurrentMap
+}
+
+var vcontroller *VoltController
+
+// NewController is the constructor for VoltController
+func NewController(ctx context.Context, app intf.App) intf.IVPClientAgent {
+	var controller VoltController
+
+	controller.rebootInProgressDevices = make(map[string]string)
+	controller.devices = make(map[string]*Device)
+	controller.deviceLock = sync.RWMutex{}
+	controller.ctx = ctx
+	controller.app = app
+	controller.BlockedDeviceList = util.NewConcurrentMap()
+	controller.deviceTaskQueue = util.NewConcurrentMap()
+	db = database.GetDatabase()
+	vcontroller = &controller
+	return &controller
+}
+
+// AddDevice to add device
+func (v *VoltController) AddDevice(config *intf.VPClientCfg) intf.IVPClient {
+
+	d := NewDevice(config.DeviceID, config.SerialNum, config.VolthaClient, config.SouthBoundID)
+	v.devices[config.DeviceID] = d
+	v.app.AddDevice(d.ID, d.SerialNum, config.SouthBoundID)
+
+	d.RestoreMetersFromDb()
+	d.RestoreGroupsFromDb()
+	d.RestoreFlowsFromDb()
+	d.RestorePortsFromDb()
+	d.ConnectInd(context.TODO(), intf.DeviceDisc)
+	d.packetOutChannel = config.PacketOutChannel
+
+	logger.Warnw(ctx, "Added device", log.Fields{"Device": config.DeviceID, "SerialNo": d.SerialNum, "State": d.State})
+
+	return d
+}
+
+// DelDevice to delete device
+func (v *VoltController) DelDevice(id string) {
+	d, ok := v.devices[id]
+	if ok {
+		delete(v.devices, id)
+		d.Delete()
+	}
+	v.app.DelDevice(id)
+	d.cancel() // To stop the device tables sync routine
+	logger.Warnw(ctx, "Deleted device", log.Fields{"Device": id})
+}
+
+//AddControllerTask - add task to controller queue
+func (v *VoltController) AddControllerTask(device string, task tasks.Task) {
+	var taskQueueIntf interface{}
+	var taskQueue *tasks.Tasks
+	var found bool
+	if taskQueueIntf, found = v.deviceTaskQueue.Get(device); !found {
+		taskQueue = tasks.NewTasks(context.TODO())
+		v.deviceTaskQueue.Set(device, taskQueue)
+	} else {
+		taskQueue = taskQueueIntf.(*tasks.Tasks)
+	}
+	taskQueue.AddTask(task)
+	logger.Warnw(ctx, "Task Added to Controller Task List", log.Fields{"Len": taskQueue.NumPendingTasks(), "Total": taskQueue.TotalTasks()})
+}
+
+//AddNewDevice - called when new device is discovered. This will be
+//processed as part of controller queue
+func (v *VoltController) AddNewDevice(config *intf.VPClientCfg) {
+	adt := NewAddDeviceTask(config)
+	v.AddControllerTask(config.DeviceID, adt)
+}
+
+// GetDevice to get device info
+func (v *VoltController) GetDevice(id string) (*Device, error) {
+	d, ok := v.devices[id]
+	if ok {
+		return d, nil
+	}
+	return nil, errorCodes.ErrDeviceNotFound
+}
+
+// IsRebootInProgressForDevice to check if reboot is in progress for the device
+func (v *VoltController) IsRebootInProgressForDevice(device string) bool {
+	v.rebootLock.Lock()
+	defer v.rebootLock.Unlock()
+	_, ok := v.rebootInProgressDevices[device]
+	return ok
+}
+
+// SetRebootInProgressForDevice to set reboot in progress for the device
+func (v *VoltController) SetRebootInProgressForDevice(device string) bool {
+	v.rebootLock.Lock()
+	defer v.rebootLock.Unlock()
+	_, ok := v.rebootInProgressDevices[device]
+	if ok {
+		return true
+	}
+	v.rebootInProgressDevices[device] = device
+	logger.Warnw(ctx, "Setted Reboot-In-Progress flag", log.Fields{"Device": device})
+
+	d, err := v.GetDevice(device)
+	if err == nil {
+		d.ResetCache()
+	} else {
+		logger.Errorw(ctx, "Failed to get device", log.Fields{"Device": device, "Error": err})
+	}
+
+	return true
+}
+
+// ReSetRebootInProgressForDevice to reset reboot in progress for the device
+func (v *VoltController) ReSetRebootInProgressForDevice(device string) bool {
+	v.rebootLock.Lock()
+	defer v.rebootLock.Unlock()
+	_, ok := v.rebootInProgressDevices[device]
+	if !ok {
+		return true
+	}
+	delete(v.rebootInProgressDevices, device)
+	logger.Warnw(ctx, "Resetted Reboot-In-Progress flag", log.Fields{"Device": device})
+	return true
+}
+
+// DeviceRebootInd is device reboot indication
+func (v *VoltController) DeviceRebootInd(dID string, srNo string, sbID string) {
+	v.app.DeviceRebootInd(dID, srNo, sbID)
+	_ = db.DelAllRoutesForDevice(dID)
+	_ = db.DelAllGroup(dID)
+	_ = db.DelAllMeter(dID)
+	_ = db.DelAllPONCounters(dID)
+}
+
+// DeviceDisableInd is device deactivation indication
+func (v *VoltController) DeviceDisableInd(dID string) {
+	v.app.DeviceDisableInd(dID)
+}
+
+//TriggerPendingProfileDeleteReq - trigger pending profile delete requests
+func (v *VoltController) TriggerPendingProfileDeleteReq(device string) {
+	v.app.TriggerPendingProfileDeleteReq(device)
+}
+
+//TriggerPendingMigrateServicesReq - trigger pending services migration requests
+func (v *VoltController) TriggerPendingMigrateServicesReq(device string) {
+	v.app.TriggerPendingMigrateServicesReq(device)
+}
+
+// SetAuditFlags to set the audit flags
+func (v *VoltController) SetAuditFlags(device *Device) {
+	v.app.SetRebootFlag(true)
+	device.auditInProgress = true
+}
+
+// ResetAuditFlags to reset the audit flags
+func (v *VoltController) ResetAuditFlags(device *Device) {
+	v.app.SetRebootFlag(false)
+	device.auditInProgress = false
+}
+
+//ProcessFlowModResultIndication - send flow mod result notification
+func (v *VoltController) ProcessFlowModResultIndication(flowStatus intf.FlowStatus) {
+	v.app.ProcessFlowModResultIndication(flowStatus)
+}
+
+// AddVPAgent to add the vpagent
+func (v *VoltController) AddVPAgent(vep string, vpa *vpagent.VPAgent) {
+	v.vagent[vep] = vpa
+}
+
+// VPAgent to get vpagent info
+func (v *VoltController) VPAgent(vep string) (*vpagent.VPAgent, error) {
+	vpa, ok := v.vagent[vep]
+	if ok {
+		return vpa, nil
+	}
+	return nil, errors.New("VPA Not Registered")
+}
+
+// PacketOutReq for packet out request
+func (v *VoltController) PacketOutReq(device string, inport string, outport string, pkt []byte, isCustomPkt bool) error {
+	logger.Debugw(ctx, "Packet Out Req", log.Fields{"Device": device, "OutPort": outport})
+	d, err := v.GetDevice(device)
+	if err != nil {
+		return err
+	}
+	logger.Debugw(ctx, "Packet Out Pkt", log.Fields{"Pkt": hex.EncodeToString(pkt)})
+	return d.PacketOutReq(inport, outport, pkt, isCustomPkt)
+}
+
+// AddFlows to add flows
+func (v *VoltController) AddFlows(port string, device string, flow *of.VoltFlow) error {
+	d, err := v.GetDevice(device)
+	if err != nil {
+		logger.Errorw(ctx, "Device Not Found", log.Fields{"Device": device})
+		return err
+	}
+	devPort := d.GetPortByName(port)
+	if devPort == nil {
+		logger.Errorw(ctx, "Port Not Found", log.Fields{"Device": device})
+		return errorCodes.ErrPortNotFound
+	}
+	if d.ctx == nil {
+		//FIXME: Application should know the context before it could submit task. Handle at application level
+		logger.Errorw(ctx, "Context is missing. AddFlow Operation Not added to Task", log.Fields{"Device": device})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+
+	var isMigrationRequired bool
+	if flow.MigrateCookie {
+		// flow migration to new cookie must be done only during the audit. Migration for all subflows must be done if
+		// atlease one subflow with old cookie found in the device.
+		for _, subFlow := range flow.SubFlows {
+			if isMigrationRequired = d.IsFlowPresentWithOldCookie(subFlow); isMigrationRequired {
+				break
+			}
+		}
+	}
+
+	if isMigrationRequired {
+		// In this case, the flow is updated in local cache and db here.
+		// Actual flow deletion and addition at voltha will happen during flow tables audit.
+		for _, subFlow := range flow.SubFlows {
+			logger.Debugw(ctx, "Cookie Migration Required", log.Fields{"OldCookie": subFlow.OldCookie, "NewCookie": subFlow.Cookie})
+			if err := d.DelFlowWithOldCookie(subFlow); err != nil {
+				logger.Errorw(ctx, "Delete flow with old cookie failed", log.Fields{"Error": err, "OldCookie": subFlow.OldCookie})
+			}
+			if err := d.AddFlow(subFlow); err != nil {
+				logger.Errorw(ctx, "Flow Add Failed", log.Fields{"Error": err, "Cookie": subFlow.Cookie})
+			}
+		}
+	} else {
+		flow.Command = of.CommandAdd
+		d.UpdateFlows(flow, devPort)
+		for cookie := range flow.SubFlows {
+			logger.Debugw(ctx, "Flow Add added to queue", log.Fields{"Cookie": cookie, "Device": device, "Port": port})
+		}
+	}
+	return nil
+}
+
+// DelFlows to delete flows
+func (v *VoltController) DelFlows(port string, device string, flow *of.VoltFlow) error {
+	d, err := v.GetDevice(device)
+	if err != nil {
+		logger.Errorw(ctx, "Device Not Found", log.Fields{"Device": device})
+		return err
+	}
+	devPort := d.GetPortByName(port)
+	if devPort == nil {
+		logger.Errorw(ctx, "Port Not Found", log.Fields{"Device": device})
+		return errorCodes.ErrPortNotFound
+	}
+	if d.ctx == nil {
+		//FIXME: Application should know the context before it could submit task. Handle at application level
+		logger.Errorw(ctx, "Context is missing. DelFlow Operation Not added to Task", log.Fields{"Device": device})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+
+	var isMigrationRequired bool
+	if flow.MigrateCookie {
+		// flow migration to new cookie must be done only during the audit. Migration for all subflows must be done if
+		// atlease one subflow with old cookie found in the device.
+		for _, subFlow := range flow.SubFlows {
+			if isMigrationRequired = d.IsFlowPresentWithOldCookie(subFlow); isMigrationRequired {
+				break
+			}
+		}
+	}
+
+	if isMigrationRequired {
+		// In this case, the flow is deleted from local cache and db here.
+		// Actual flow deletion at voltha will happen during flow tables audit.
+		for _, subFlow := range flow.SubFlows {
+			logger.Debugw(ctx, "Old Cookie delete Required", log.Fields{"OldCookie": subFlow.OldCookie})
+			if err := d.DelFlowWithOldCookie(subFlow); err != nil {
+				logger.Errorw(ctx, "DelFlowWithOldCookie failed", log.Fields{"OldCookie": subFlow.OldCookie, "Error": err})
+			}
+		}
+	} else {
+		flow.Command = of.CommandDel
+		d.UpdateFlows(flow, devPort)
+		for cookie := range flow.SubFlows {
+			logger.Debugw(ctx, "Flow Del added to queue", log.Fields{"Cookie": cookie, "Device": device, "Port": port})
+		}
+	}
+	return nil
+}
+
+// GroupUpdate for group update
+func (v *VoltController) GroupUpdate(port string, device string, group *of.Group) error {
+	d, err := v.GetDevice(device)
+	if err != nil {
+		logger.Errorw(ctx, "Device Not Found", log.Fields{"Device": device})
+		return err
+	}
+
+	devPort := d.GetPortByName(port)
+	if devPort == nil {
+		logger.Errorw(ctx, "Port Not Found", log.Fields{"Device": device})
+		return errorCodes.ErrPortNotFound
+	}
+
+	if d.ctx == nil {
+		//FIXME: Application should know the context before it could submit task. Handle at application level
+		logger.Errorw(ctx, "Context is missing. GroupMod Operation Not added to task", log.Fields{"Device": device})
+		return errorCodes.ErrInvalidParamInRequest
+	}
+
+	d.UpdateGroup(group, devPort)
+	return nil
+}
+
+// ModMeter to get mod meter info
+func (v *VoltController) ModMeter(port string, device string, command of.MeterCommand, meter *of.Meter) error {
+	d, err := v.GetDevice(device)
+	if err != nil {
+		logger.Errorw(ctx, "Device Not Found", log.Fields{"Device": device})
+		return err
+	}
+
+	devPort := d.GetPortByName(port)
+	if devPort == nil {
+		logger.Errorw(ctx, "Port Not Found", log.Fields{"Device": device})
+		return errorCodes.ErrPortNotFound
+	}
+
+	d.ModMeter(command, meter, devPort)
+	return nil
+}
+
+// PortAddInd for port add indication
+func (v *VoltController) PortAddInd(device string, id uint32, name string) {
+	v.app.PortAddInd(device, id, name)
+}
+
+// PortDelInd for port delete indication
+func (v *VoltController) PortDelInd(device string, port string) {
+	v.app.PortDelInd(device, port)
+}
+
+// PortUpdateInd for port update indication
+func (v *VoltController) PortUpdateInd(device string, name string, id uint32) {
+	v.app.PortUpdateInd(device, name, id)
+}
+
+// PortUpInd for port up indication
+func (v *VoltController) PortUpInd(device string, port string) {
+	v.app.PortUpInd(device, port)
+}
+
+// PortDownInd for port down indication
+func (v *VoltController) PortDownInd(device string, port string) {
+	v.app.PortDownInd(device, port)
+}
+
+// DeviceUpInd for device up indication
+func (v *VoltController) DeviceUpInd(device string) {
+	v.app.DeviceUpInd(device)
+}
+
+// DeviceDownInd for device down indication
+func (v *VoltController) DeviceDownInd(device string) {
+	v.app.DeviceDownInd(device)
+}
+
+// PacketInInd for packet in indication
+func (v *VoltController) PacketInInd(device string, port string, data []byte) {
+	v.app.PacketInInd(device, port, data)
+}
+
+// GetPortState to get port status
+func (v *VoltController) GetPortState(device string, name string) (PortState, error) {
+	d, err := v.GetDevice(device)
+	if err != nil {
+		logger.Errorw(ctx, "Device Not Found", log.Fields{"Device": device})
+		return PortStateDown, err
+	}
+	return d.GetPortState(name)
+}
+
+// UpdateMvlanProfiles for update mvlan profiles
+func (v *VoltController) UpdateMvlanProfiles(device string) {
+	v.app.UpdateMvlanProfilesForDevice(device)
+}
+
+// GetController to get controller
+func GetController() *VoltController {
+	return vcontroller
+}
+
+/*
+// PostIndication to post indication
+func (v *VoltController) PostIndication(device string, task interface{}) error {
+	var srvTask AddServiceIndTask
+	var portTask AddPortIndTask
+	var taskCommon tasks.Task
+	var isSvcTask bool
+
+	switch data := task.(type) {
+	case *AddServiceIndTask:
+		srvTask = *data
+		taskCommon = data
+		isSvcTask = true
+	case *AddPortIndTask:
+		portTask = *data
+		taskCommon = data
+	}
+
+	d, err := v.GetDevice(device)
+	if err != nil {
+		logger.Errorw(ctx, "Device Not Found", log.Fields{"Device": device})
+		//It means device itself it not present so just post the indication directly
+		if isSvcTask {
+			msgbus.PostAccessConfigInd(srvTask.result, d.SerialNum, srvTask.indicationType, srvTask.serviceName, 0, srvTask.reason, srvTask.trigger, srvTask.portState)
+		} else {
+			msgbus.ProcessPortInd(portTask.indicationType, d.SerialNum, portTask.portName, portTask.accessConfig, portTask.serviceList)
+		}
+		return err
+	}
+	if taskCommon != nil {
+		d.AddTask(taskCommon)
+	}
+	return nil
+}
+*/
+
+// GetTaskList to get the task list
+func (v *VoltController) GetTaskList(device string) []tasks.Task {
+	d, err := v.GetDevice(device)
+	if err != nil || d.ctx == nil {
+		logger.Errorw(ctx, "Device Not Connected/Found", log.Fields{"Device": device, "Dev Obj": d})
+		return []tasks.Task{}
+	}
+	return d.GetTaskList()
+
+}
+
+// AddBlockedDevices to add devices to blocked devices list
+func (v *VoltController) AddBlockedDevices(deviceSerialNumber string) {
+	v.BlockedDeviceList.Set(deviceSerialNumber, deviceSerialNumber)
+}
+
+// DelBlockedDevices to remove device from blocked device list
+func (v *VoltController) DelBlockedDevices(deviceSerialNumber string) {
+	v.BlockedDeviceList.Remove(deviceSerialNumber)
+}
+
+// IsBlockedDevice to check if device is blocked
+func (v *VoltController) IsBlockedDevice(deviceSerialNumber string) bool {
+	_, ifPresent := v.BlockedDeviceList.Get(deviceSerialNumber)
+	return ifPresent
+}
diff --git a/internal/pkg/controller/controllertasks.go b/internal/pkg/controller/controllertasks.go
new file mode 100644
index 0000000..bd06ffb
--- /dev/null
+++ b/internal/pkg/controller/controllertasks.go
@@ -0,0 +1,76 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package controller
+
+import (
+	"context"
+	"time"
+
+	"voltha-go-controller/internal/pkg/intf"
+	"voltha-go-controller/internal/pkg/vpagent"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// AddDeviceTask structure
+type AddDeviceTask struct {
+	taskID    uint8
+	ctx       context.Context
+	config    *intf.VPClientCfg
+	timestamp string
+}
+
+// NewAddDeviceTask is the constructor for AddDeviceTask
+func NewAddDeviceTask(config *intf.VPClientCfg) *AddDeviceTask {
+	var adt AddDeviceTask
+	adt.config = config
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	adt.timestamp = tstamp
+	return &adt
+}
+
+// Name returns name of the task
+func (adt *AddDeviceTask) Name() string {
+	return "Add Device Task"
+}
+
+// TaskID returns task Id of the task
+func (adt *AddDeviceTask) TaskID() uint8 {
+	return adt.taskID
+}
+
+// Timestamp returns time stamp for the task
+func (adt *AddDeviceTask) Timestamp() string {
+	return adt.timestamp
+}
+
+// Stop to stop the task
+func (adt *AddDeviceTask) Stop() {
+}
+
+// Start to start the task
+func (adt *AddDeviceTask) Start(ctx context.Context, taskID uint8) error {
+	adt.taskID = taskID
+	adt.ctx = ctx
+
+	logger.Infow(ctx, "Add Device Task Triggered", log.Fields{"Device": adt.config.DeviceID, "SerialNum": adt.config.SerialNum})
+
+	device := GetController().AddDevice(adt.config)
+	vpagent.GetVPAgent().AddClientToClientMap(adt.config.DeviceID, device)
+	logger.Infow(ctx, "Add Device Task Completed", log.Fields{"Device": adt.config.DeviceID, "SerialNum": adt.config.SerialNum})
+
+	return nil
+}
diff --git a/internal/pkg/controller/device.go b/internal/pkg/controller/device.go
new file mode 100644
index 0000000..aa7bd2c
--- /dev/null
+++ b/internal/pkg/controller/device.go
@@ -0,0 +1,1042 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	"encoding/json"
+	"errors"
+	infraerror "voltha-go-controller/internal/pkg/errorcodes"
+	"strconv"
+	"sync"
+	"time"
+
+	"voltha-go-controller/database"
+	"voltha-go-controller/internal/pkg/holder"
+	"voltha-go-controller/internal/pkg/intf"
+	"voltha-go-controller/internal/pkg/of"
+	//"voltha-go-controller/internal/pkg/vpagent"
+	"voltha-go-controller/internal/pkg/tasks"
+	"voltha-go-controller/internal/pkg/util"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+// PortState type
+type PortState string
+
+const (
+	// PortStateDown constant
+	PortStateDown PortState = "DOWN"
+	// PortStateUp constant
+	PortStateUp PortState = "UP"
+	// DefaultMaxFlowQueues constant
+	DefaultMaxFlowQueues = 67
+	//ErrDuplicateFlow - indicates flow already exists in DB
+	ErrDuplicateFlow string = "Duplicate Flow"
+)
+
+// DevicePort structure
+type DevicePort struct {
+	tasks.Tasks
+	Name    string
+	ID      uint32
+	State   PortState
+	Version string
+}
+
+// NewDevicePort is the constructor for DevicePort
+func NewDevicePort(id uint32, name string) *DevicePort {
+	var port DevicePort
+
+	port.ID = id
+	port.Name = name
+	port.State = PortStateDown
+	return &port
+}
+
+// UniIDFlowQueue structure which maintains flows in queue.
+type UniIDFlowQueue struct {
+	tasks.Tasks
+	ID uint32
+}
+
+// NewUniIDFlowQueue is the constructor for UniIDFlowQueue.
+func NewUniIDFlowQueue(id uint32) *UniIDFlowQueue {
+	var flowQueue UniIDFlowQueue
+	flowQueue.ID = id
+	return &flowQueue
+}
+
+// DeviceState type
+type DeviceState string
+
+const (
+
+	// DeviceStateUNKNOWN constant
+	DeviceStateUNKNOWN DeviceState = "UNKNOWN"
+	// DeviceStateINIT constant
+	DeviceStateINIT DeviceState = "INIT"
+	// DeviceStateUP constant
+	DeviceStateUP DeviceState = "UP"
+	// DeviceStateDOWN constant
+	DeviceStateDOWN DeviceState = "DOWN"
+	// DeviceStateREBOOTED constant
+	DeviceStateREBOOTED DeviceState = "REBOOTED"
+	// DeviceStateDISABLED constant
+	DeviceStateDISABLED DeviceState = "DISABLED"
+	// DeviceStateDELETED constant
+	DeviceStateDELETED DeviceState = "DELETED"
+)
+
+// Device structure
+type Device struct {
+	tasks.Tasks
+	ID                    string
+	SerialNum             string
+	State                 DeviceState
+	PortsByID             map[uint32]*DevicePort
+	PortsByName           map[string]*DevicePort
+	portLock              sync.RWMutex
+	vclientHolder         *holder.VolthaServiceClientHolder
+	ctx                   context.Context
+	cancel                context.CancelFunc
+	packetOutChannel      chan *ofp.PacketOut
+	flows                 map[uint64]*of.VoltSubFlow
+	flowLock              sync.RWMutex
+	meters                map[uint32]*of.Meter
+	meterLock             sync.RWMutex
+	groups                sync.Map //map[uint32]*of.Group -> [GroupId : Group]
+	auditInProgress       bool
+	flowQueueLock         sync.RWMutex
+	flowHash              uint32
+	flowQueue             map[uint32]*UniIDFlowQueue // key is hash ID generated and value is UniIDFlowQueue.
+	deviceAuditInProgress bool
+	SouthBoundID          string
+}
+
+// NewDevice is the constructor for Device
+func NewDevice(id string, slno string, vclientHldr *holder.VolthaServiceClientHolder, southBoundID string) *Device {
+	var device Device
+	device.ID = id
+	device.SerialNum = slno
+	device.State = DeviceStateDOWN
+	device.PortsByID = make(map[uint32]*DevicePort)
+	device.PortsByName = make(map[string]*DevicePort)
+	device.vclientHolder = vclientHldr
+	device.flows = make(map[uint64]*of.VoltSubFlow)
+	device.meters = make(map[uint32]*of.Meter)
+	device.flowQueue = make(map[uint32]*UniIDFlowQueue)
+	//Get the flowhash from db and update the flowhash variable in the device.
+	device.SouthBoundID = southBoundID
+	flowHash, err := db.GetFlowHash(id)
+	if err != nil {
+		device.flowHash = DefaultMaxFlowQueues
+	} else {
+		var hash uint32
+		err = json.Unmarshal([]byte(flowHash), &hash)
+		if err != nil {
+			logger.Error(ctx, "Failed to unmarshall flowhash")
+		} else {
+			device.flowHash = hash
+		}
+	}
+	logger.Infow(ctx, "Flow hash for device", log.Fields{"Deviceid": id, "hash": device.flowHash})
+	return &device
+}
+
+// ResetCache to reset cache
+func (d *Device) ResetCache() {
+	logger.Warnw(ctx, "Resetting flows, meters and groups cache", log.Fields{"Device": d.ID})
+	d.flows = make(map[uint64]*of.VoltSubFlow)
+	d.meters = make(map[uint32]*of.Meter)
+	d.groups = sync.Map{}
+}
+
+// GetFlow - Get the flow from device obj
+func (d *Device) GetFlow(cookie uint64) (*of.VoltSubFlow, bool) {
+	d.flowLock.RLock()
+	defer d.flowLock.RUnlock()
+	logger.Infow(ctx, "Get Flow", log.Fields{"Cookie": cookie})
+	flow, ok := d.flows[cookie]
+	return flow, ok
+}
+
+// AddFlow - Adds the flow to the device and also to the database
+func (d *Device) AddFlow(flow *of.VoltSubFlow) error {
+	d.flowLock.Lock()
+	defer d.flowLock.Unlock()
+	logger.Infow(ctx, "AddFlow to device", log.Fields{"Cookie": flow.Cookie})
+	if _, ok := d.flows[flow.Cookie]; ok {
+		return errors.New(ErrDuplicateFlow)
+	}
+	d.flows[flow.Cookie] = flow
+	d.AddFlowToDb(flow)
+	return nil
+}
+
+// AddFlowToDb is the utility to add the flow to the device
+func (d *Device) AddFlowToDb(flow *of.VoltSubFlow) {
+	if b, err := json.Marshal(flow); err == nil {
+		if err = db.PutFlow(d.ID, flow.Cookie, string(b)); err != nil {
+			logger.Errorw(ctx, "Write Flow to DB failed", log.Fields{"device": d.ID, "cookie": flow.Cookie, "Reason": err})
+		}
+	}
+}
+
+// DelFlow - Deletes the flow from the device and the database
+func (d *Device) DelFlow(flow *of.VoltSubFlow) error {
+	d.flowLock.Lock()
+	defer d.flowLock.Unlock()
+	if _, ok := d.flows[flow.Cookie]; ok {
+		delete(d.flows, flow.Cookie)
+		d.DelFlowFromDb(flow.Cookie)
+		return nil
+	}
+	return errors.New("Flow does not Exist")
+}
+
+// DelFlowFromDb is utility to delete the flow from the device
+func (d *Device) DelFlowFromDb(flowID uint64) {
+	_ = db.DelFlow(d.ID, flowID)
+}
+
+// IsFlowPresentWithOldCookie is to check whether there is any flow with old cookie.
+func (d *Device) IsFlowPresentWithOldCookie(flow *of.VoltSubFlow) bool {
+	d.flowLock.RLock()
+	defer d.flowLock.RUnlock()
+	if _, ok := d.flows[flow.Cookie]; ok {
+		return false
+	} else if flow.OldCookie != 0 && flow.Cookie != flow.OldCookie {
+		if _, ok := d.flows[flow.OldCookie]; ok {
+			logger.Infow(ctx, "Flow present with old cookie", log.Fields{"OldCookie": flow.OldCookie})
+			return true
+		}
+	}
+	return false
+}
+
+// DelFlowWithOldCookie is to delete flow with old cookie.
+func (d *Device) DelFlowWithOldCookie(flow *of.VoltSubFlow) error {
+	d.flowLock.Lock()
+	defer d.flowLock.Unlock()
+	if _, ok := d.flows[flow.OldCookie]; ok {
+		logger.Infow(ctx, "Flow was added before vgc upgrade. Trying to delete with old cookie",
+			log.Fields{"OldCookie": flow.OldCookie})
+		delete(d.flows, flow.OldCookie)
+		d.DelFlowFromDb(flow.OldCookie)
+		return nil
+	}
+	return errors.New("Flow does not Exist")
+}
+
+// RestoreFlowsFromDb to restore flows from database
+func (d *Device) RestoreFlowsFromDb() {
+	flows, _ := db.GetFlows(d.ID)
+	for _, flow := range flows {
+		b, ok := flow.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		d.CreateFlowFromString(b)
+	}
+}
+
+// CreateFlowFromString to create flow from string
+func (d *Device) CreateFlowFromString(b []byte) {
+	var flow of.VoltSubFlow
+	if err := json.Unmarshal(b, &flow); err == nil {
+		if _, ok := d.flows[flow.Cookie]; !ok {
+			logger.Debugw(ctx, "Adding Flow From Db", log.Fields{"Cookie": flow.Cookie})
+			d.flows[flow.Cookie] = &flow
+		} else {
+			logger.Warnw(ctx, "Duplicate Flow", log.Fields{"Cookie": flow.Cookie})
+		}
+	} else {
+		logger.Warn(ctx, "Unmarshal failed")
+	}
+}
+
+// ----------------------------------------------------------
+// Database related functionality
+// Group operations at the device which include update and delete
+
+// UpdateGroupEntry - Adds/Updates the group to the device and also to the database
+func (d *Device) UpdateGroupEntry(group *of.Group) {
+
+	logger.Infow(ctx, "Update Group to device", log.Fields{"ID": group.GroupID})
+	d.groups.Store(group.GroupID, group)
+	d.AddGroupToDb(group)
+}
+
+// AddGroupToDb - Utility to add the group to the device DB
+func (d *Device) AddGroupToDb(group *of.Group) {
+	if b, err := json.Marshal(group); err == nil {
+		logger.Infow(ctx, "Adding Group to DB", log.Fields{"grp": group, "Json": string(b)})
+		if err = db.PutGroup(d.ID, group.GroupID, string(b)); err != nil {
+			logger.Errorw(ctx, "Write Group to DB failed", log.Fields{"device": d.ID, "groupID": group.GroupID, "Reason": err})
+		}
+	}
+}
+
+// DelGroupEntry - Deletes the group from the device and the database
+func (d *Device) DelGroupEntry(group *of.Group) {
+
+	if _, ok := d.groups.Load(group.GroupID); ok {
+		d.groups.Delete(group.GroupID)
+		d.DelGroupFromDb(group.GroupID)
+	}
+}
+
+// DelGroupFromDb - Utility to delete the Group from the device
+func (d *Device) DelGroupFromDb(groupID uint32) {
+	_ = db.DelGroup(d.ID, groupID)
+}
+
+//RestoreGroupsFromDb - restores all groups from DB
+func (d *Device) RestoreGroupsFromDb() {
+	logger.Info(ctx, "Restoring Groups")
+	groups, _ := db.GetGroups(d.ID)
+	for _, group := range groups {
+		b, ok := group.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		d.CreateGroupFromString(b)
+	}
+}
+
+//CreateGroupFromString - Forms group struct from json string
+func (d *Device) CreateGroupFromString(b []byte) {
+	var group of.Group
+	if err := json.Unmarshal(b, &group); err == nil {
+		if _, ok := d.groups.Load(group.GroupID); !ok {
+			logger.Debugw(ctx, "Adding Group From Db", log.Fields{"GroupId": group.GroupID})
+			d.groups.Store(group.GroupID, &group)
+		} else {
+			logger.Warnw(ctx, "Duplicate Group", log.Fields{"GroupId": group.GroupID})
+		}
+	} else {
+		logger.Warn(ctx, "Unmarshal failed")
+	}
+}
+
+// AddMeter to add meter
+func (d *Device) AddMeter(meter *of.Meter) error {
+	d.meterLock.Lock()
+	defer d.meterLock.Unlock()
+	if _, ok := d.meters[meter.ID]; ok {
+		return errors.New("Duplicate Meter")
+	}
+	d.meters[meter.ID] = meter
+	go d.AddMeterToDb(meter)
+	return nil
+}
+
+// GetMeter to get meter
+func (d *Device) GetMeter(id uint32) (*of.Meter, error) {
+	d.meterLock.RLock()
+	defer d.meterLock.RUnlock()
+	if m, ok := d.meters[id]; ok {
+		return m, nil
+	}
+	return nil, errors.New("Meter Not Found")
+}
+
+// DelMeter to delete meter
+func (d *Device) DelMeter(meter *of.Meter) bool {
+	d.meterLock.Lock()
+	defer d.meterLock.Unlock()
+	if _, ok := d.meters[meter.ID]; ok {
+		delete(d.meters, meter.ID)
+		go d.DelMeterFromDb(meter.ID)
+		return true
+	}
+	return false
+}
+
+// AddMeterToDb is utility to add the Group to the device
+func (d *Device) AddMeterToDb(meter *of.Meter) {
+	if b, err := json.Marshal(meter); err == nil {
+		if err = db.PutDeviceMeter(d.ID, meter.ID, string(b)); err != nil {
+			logger.Errorw(ctx, "Write Meter to DB failed", log.Fields{"device": d.ID, "meterID": meter.ID, "Reason": err})
+		}
+	}
+}
+
+// DelMeterFromDb to delete meter from db
+func (d *Device) DelMeterFromDb(id uint32) {
+	_ = db.DelDeviceMeter(d.ID, id)
+}
+
+// RestoreMetersFromDb to restore meters from db
+func (d *Device) RestoreMetersFromDb() {
+	meters, _ := db.GetDeviceMeters(d.ID)
+	for _, meter := range meters {
+		b, ok := meter.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		d.CreateMeterFromString(b)
+	}
+}
+
+// CreateMeterFromString to create meter from string
+func (d *Device) CreateMeterFromString(b []byte) {
+	var meter of.Meter
+	if err := json.Unmarshal(b, &meter); err == nil {
+		if _, ok := d.meters[meter.ID]; !ok {
+			logger.Debugw(ctx, "Adding Meter From Db", log.Fields{"ID": meter.ID})
+			d.meters[meter.ID] = &meter
+		} else {
+			logger.Warnw(ctx, "Duplicate Meter", log.Fields{"ID": meter.ID})
+		}
+	} else {
+		logger.Warn(ctx, "Unmarshal failed")
+	}
+}
+
+// VolthaClient to get voltha client
+func (d *Device) VolthaClient() voltha.VolthaServiceClient {
+	return d.vclientHolder.Get()
+}
+
+// AddPort to add the port as requested by the device/VOLTHA
+// Inform the application if the port is successfully added
+func (d *Device) AddPort(id uint32, name string) error {
+	d.portLock.Lock()
+	defer d.portLock.Unlock()
+
+	if _, ok := d.PortsByID[id]; ok {
+		return errors.New("Duplicate port")
+	}
+	if _, ok := d.PortsByName[name]; ok {
+		return errors.New("Duplicate port")
+	}
+
+	p := NewDevicePort(id, name)
+	d.PortsByID[id] = p
+	d.PortsByName[name] = p
+	d.WritePortToDb(p)
+	GetController().PortAddInd(d.ID, p.ID, p.Name)
+	logger.Infow(ctx, "Added Port", log.Fields{"Device": d.ID, "Port": id})
+	return nil
+}
+
+// DelPort to delete the port as requested by the device/VOLTHA
+// Inform the application if the port is successfully deleted
+func (d *Device) DelPort(id uint32) error {
+
+	p := d.GetPortByID(id)
+	if p == nil {
+		return errors.New("Unknown Port")
+	}
+	if p.State == PortStateUp {
+		GetController().PortDownInd(d.ID, p.Name)
+	}
+	d.portLock.Lock()
+	defer d.portLock.Unlock()
+
+	GetController().PortDelInd(d.ID, p.Name)
+	delete(d.PortsByID, p.ID)
+	delete(d.PortsByName, p.Name)
+	d.DelPortFromDb(p.ID)
+	logger.Infow(ctx, "Deleted Port", log.Fields{"Device": d.ID, "Port": id})
+	return nil
+}
+
+// UpdatePortByName is utility to update the port by Name
+func (d *Device) UpdatePortByName(name string, port uint32) {
+	d.portLock.Lock()
+	defer d.portLock.Unlock()
+
+	p, ok := d.PortsByName[name]
+	if !ok {
+		return
+	}
+	delete(d.PortsByID, p.ID)
+	p.ID = port
+	d.PortsByID[port] = p
+	d.WritePortToDb(p)
+	GetController().PortUpdateInd(d.ID, p.Name, p.ID)
+	logger.Infow(ctx, "Updated Port", log.Fields{"Device": d.ID, "Port": p.ID, "PortName": name})
+}
+
+// GetPortName to get the name of the port by its id
+func (d *Device) GetPortName(id uint32) (string, error) {
+	d.portLock.RLock()
+	defer d.portLock.RUnlock()
+
+	if p, ok := d.PortsByID[id]; ok {
+		return p.Name, nil
+	}
+	logger.Errorw(ctx, "Port not found", log.Fields{"port": id})
+	return "", errors.New("Unknown Port ID")
+}
+
+// GetPortByID is utility to retrieve the port by ID
+func (d *Device) GetPortByID(id uint32) *DevicePort {
+	d.portLock.RLock()
+	defer d.portLock.RUnlock()
+
+	p, ok := d.PortsByID[id]
+	if ok {
+		return p
+	}
+	return nil
+}
+
+// GetPortByName is utility to retrieve the port by Name
+func (d *Device) GetPortByName(name string) *DevicePort {
+	d.portLock.RLock()
+	defer d.portLock.RUnlock()
+
+	p, ok := d.PortsByName[name]
+	if ok {
+		return p
+	}
+	return nil
+}
+
+// GetPortState to get the state of the port by name
+func (d *Device) GetPortState(name string) (PortState, error) {
+	d.portLock.RLock()
+	defer d.portLock.RUnlock()
+
+	if p, ok := d.PortsByName[name]; ok {
+		return p.State, nil
+	}
+	return PortStateDown, errors.New("Unknown Port ID")
+}
+
+// GetPortID to get the port-id by the port name
+func (d *Device) GetPortID(name string) (uint32, error) {
+	d.portLock.RLock()
+	defer d.portLock.RUnlock()
+
+	if p, ok := d.PortsByName[name]; ok {
+		return p.ID, nil
+	}
+	return 0, errors.New("Unknown Port ID")
+
+}
+
+// WritePortToDb to add the port to the database
+func (d *Device) WritePortToDb(port *DevicePort) {
+	port.Version = database.PresentVersionMap[database.DevicePortPath]
+	if b, err := json.Marshal(port); err == nil {
+		if err = db.PutPort(d.ID, port.ID, string(b)); err != nil {
+			logger.Errorw(ctx, "Write port to DB failed", log.Fields{"device": d.ID, "port": port.ID, "Reason": err})
+		}
+	}
+}
+
+// DelPortFromDb to delete port from database
+func (d *Device) DelPortFromDb(id uint32) {
+	_ = db.DelPort(d.ID, id)
+}
+
+// RestorePortsFromDb to restore ports from database
+func (d *Device) RestorePortsFromDb() {
+	ports, _ := db.GetPorts(d.ID)
+	for _, port := range ports {
+		b, ok := port.Value.([]byte)
+		if !ok {
+			logger.Warn(ctx, "The value type is not []byte")
+			continue
+		}
+		d.CreatePortFromString(b)
+	}
+}
+
+// CreatePortFromString to create port from string
+func (d *Device) CreatePortFromString(b []byte) {
+	var port DevicePort
+	if err := json.Unmarshal(b, &port); err == nil {
+		if _, ok := d.PortsByID[port.ID]; !ok {
+			logger.Debugw(ctx, "Adding Port From Db", log.Fields{"ID": port.ID})
+			d.PortsByID[port.ID] = &port
+			d.PortsByName[port.Name] = &port
+			GetController().PortAddInd(d.ID, port.ID, port.Name)
+		} else {
+			logger.Warnw(ctx, "Duplicate Port", log.Fields{"ID": port.ID})
+		}
+	} else {
+		logger.Warn(ctx, "Unmarshal failed")
+	}
+}
+
+// Delete : OLT Delete functionality yet to be implemented. IDeally all of the
+// resources should have been removed by this time. It is an error
+// scenario if the OLT has resources associated with it.
+func (d *Device) Delete() {
+	d.StopAll()
+}
+
+// Stop to stop the task
+func (d *Device) Stop() {
+}
+
+// ConnectInd is called when the connection between VGC and the VOLTHA is
+// restored. This will perform audit of the device post reconnection
+func (d *Device) ConnectInd(ctx context.Context, discType intf.DiscoveryType) {
+	logger.Warnw(ctx, "Audit Upon Connection Establishment", log.Fields{"Device": d.ID, "State": d.State})
+	ctx1, cancel := context.WithCancel(ctx)
+	d.cancel = cancel
+	d.ctx = ctx1
+	d.Tasks.Initialize(ctx1)
+
+	logger.Warnw(ctx, "Device State change Ind: UP", log.Fields{"Device": d.ID})
+	d.State = DeviceStateUP
+	GetController().DeviceUpInd(d.ID)
+
+	logger.Warnw(ctx, "Device State change Ind: UP, trigger Audit Tasks", log.Fields{"Device": d.ID})
+	t := NewAuditDevice(d, AuditEventDeviceDisc)
+	d.Tasks.AddTask(t)
+
+	t1 := NewAuditTablesTask(d)
+	d.Tasks.AddTask(t1)
+
+	t2 := NewPendingProfilesTask(d)
+	d.Tasks.AddTask(t2)
+
+	go d.synchronizeDeviceTables()
+}
+
+func (d *Device) synchronizeDeviceTables() {
+
+	tick := time.NewTicker(deviceTableSyncDuration)
+loop:
+	for {
+		select {
+		case <-d.ctx.Done():
+			logger.Warnw(d.ctx, "Context Done. Cancelling Periodic Audit", log.Fields{"Context": ctx, "Device": d.ID, "DeviceSerialNum": d.SerialNum})
+			break loop
+		case <-tick.C:
+			t1 := NewAuditTablesTask(d)
+			d.Tasks.AddTask(t1)
+		}
+	}
+	tick.Stop()
+}
+
+// DeviceUpInd is called when the logical device state changes to UP. This will perform audit of the device post reconnection
+func (d *Device) DeviceUpInd() {
+	logger.Warnw(ctx, "Device State change Ind: UP", log.Fields{"Device": d.ID})
+	d.State = DeviceStateUP
+	GetController().DeviceUpInd(d.ID)
+
+	logger.Warnw(ctx, "Device State change Ind: UP, trigger Audit Tasks", log.Fields{"Device": d.ID})
+	t := NewAuditDevice(d, AuditEventDeviceDisc)
+	d.Tasks.AddTask(t)
+
+	t1 := NewAuditTablesTask(d)
+	d.Tasks.AddTask(t1)
+
+	t2 := NewPendingProfilesTask(d)
+	d.Tasks.AddTask(t2)
+}
+
+// DeviceDownInd is called when the logical device state changes to Down.
+func (d *Device) DeviceDownInd() {
+	logger.Warnw(ctx, "Device State change Ind: Down", log.Fields{"Device": d.ID})
+	d.State = DeviceStateDOWN
+	GetController().DeviceDownInd(d.ID)
+}
+
+// DeviceRebootInd is called when the logical device is rebooted.
+func (d *Device) DeviceRebootInd() {
+	logger.Warnw(ctx, "Device State change Ind: Rebooted", log.Fields{"Device": d.ID})
+
+	if d.State == DeviceStateREBOOTED {
+		d.State = DeviceStateREBOOTED
+		logger.Warnw(ctx, "Ignoring Device State change Ind: REBOOT, Device Already in REBOOT state", log.Fields{"Device": d.ID, "SeralNo": d.SerialNum})
+		return
+	}
+
+	d.State = DeviceStateREBOOTED
+	GetController().SetRebootInProgressForDevice(d.ID)
+	GetController().DeviceRebootInd(d.ID, d.SerialNum, d.SouthBoundID)
+	d.ReSetAllPortStates()
+}
+
+// DeviceDisabledInd is called when the logical device is disabled
+func (d *Device) DeviceDisabledInd() {
+	logger.Warnw(ctx, "Device State change Ind: Disabled", log.Fields{"Device": d.ID})
+	d.State = DeviceStateDISABLED
+	GetController().DeviceDisableInd(d.ID)
+}
+
+//ReSetAllPortStates - Set all logical device port status to DOWN
+func (d *Device) ReSetAllPortStates() {
+	logger.Warnw(ctx, "Resetting all Ports State to DOWN", log.Fields{"Device": d.ID, "State": d.State})
+
+	d.portLock.Lock()
+	defer d.portLock.Unlock()
+
+	for _, port := range d.PortsByID {
+		if port.State != PortStateDown {
+			logger.Infow(ctx, "Resetting Port State to DOWN", log.Fields{"Device": d.ID, "Port": port})
+			GetController().PortDownInd(d.ID, port.Name)
+			port.State = PortStateDown
+			d.WritePortToDb(port)
+		}
+	}
+}
+
+//ReSetAllPortStatesInDb - Set all logical device port status to DOWN in DB and skip indication to application
+func (d *Device) ReSetAllPortStatesInDb() {
+	logger.Warnw(ctx, "Resetting all Ports State to DOWN In DB", log.Fields{"Device": d.ID, "State": d.State})
+
+	d.portLock.Lock()
+	defer d.portLock.Unlock()
+
+	for _, port := range d.PortsByID {
+		if port.State != PortStateDown {
+			logger.Infow(ctx, "Resetting Port State to DOWN and Write to DB", log.Fields{"Device": d.ID, "Port": port})
+			port.State = PortStateDown
+			d.WritePortToDb(port)
+		}
+	}
+}
+
+// ProcessPortUpdate deals with the change in port id (ONU movement) and taking action
+// to update only when the port state is DOWN
+func (d *Device) ProcessPortUpdate(portName string, port uint32, state uint32) {
+	if p := d.GetPortByName(portName); p != nil {
+		if p.ID != port {
+			logger.Infow(ctx, "Port ID update indication", log.Fields{"Port": p.Name, "Old PortID": p.ID, "New Port ID": port})
+			if p.State != PortStateDown {
+				logger.Errorw(ctx, "Port ID update failed. Port State UP", log.Fields{"Port": p})
+				return
+			}
+			d.UpdatePortByName(portName, port)
+			logger.Errorw(ctx, "Port ID Updated", log.Fields{"Port": p})
+		}
+		d.ProcessPortState(port, state)
+	}
+}
+
+// ***Operations Performed on Port state Transitions***
+//
+// |-----------------------------------------------------------------------------|
+// |  State             |   Action                                               |
+// |--------------------|--------------------------------------------------------|
+// | UP                 | UNI - Trigger Flow addition for service configured     |
+// |                    | NNI - Trigger Flow addition for vnets & mvlan profiles |
+// |                    |                                                        |
+// | DOWN               | UNI - Trigger Flow deletion for service configured     |
+// |                    | NNI - Trigger Flow deletion for vnets & mvlan profiles |
+// |                    |                                                        |
+// |-----------------------------------------------------------------------------|
+//
+
+// ProcessPortState deals with the change in port status and taking action
+// based on the new state and the old state
+func (d *Device) ProcessPortState(port uint32, state uint32) {
+	if d.State != DeviceStateUP && !util.IsNniPort(port) {
+		logger.Warnw(ctx, "Ignore Port State Processing - Device not UP", log.Fields{"Device": d.ID, "Port": port, "DeviceState": d.State})
+		return
+	}
+	if p := d.GetPortByID(port); p != nil {
+		logger.Infow(ctx, "Port State Processing", log.Fields{"Received": state, "Current": p.State})
+
+		// Avoid blind initialization as the current tasks in the queue will be lost
+		// Eg: Service Del followed by Port Down - The flows will be dangling
+		// Eg: NNI Down followed by NNI UP - Mcast data flows will be dangling
+		p.Tasks.CheckAndInitialize(d.ctx)
+		if state == uint32(ofp.OfpPortState_OFPPS_LIVE) && p.State == PortStateDown {
+			// Transition from DOWN to UP
+			logger.Infow(ctx, "Port State Change to UP", log.Fields{"Device": d.ID, "Port": port})
+			GetController().PortUpInd(d.ID, p.Name)
+			p.State = PortStateUp
+			d.WritePortToDb(p)
+		} else if (state != uint32(ofp.OfpPortState_OFPPS_LIVE)) && (p.State != PortStateDown) {
+			// Transition from UP to Down
+			logger.Infow(ctx, "Port State Change to Down", log.Fields{"Device": d.ID, "Port": port})
+			GetController().PortDownInd(d.ID, p.Name)
+			p.State = PortStateDown
+			d.WritePortToDb(p)
+		} else {
+			logger.Warnw(ctx, "Dropping Port Ind: No Change in Port State", log.Fields{"PortName": p.Name, "ID": port, "Device": d.ID, "PortState": p.State, "IncomingState": state})
+		}
+	}
+}
+
+// ProcessPortStateAfterReboot - triggers the port state indication to sort out configu mismatch due to reboot
+func (d *Device) ProcessPortStateAfterReboot(port uint32, state uint32) {
+	if d.State != DeviceStateUP && !util.IsNniPort(port) {
+		logger.Warnw(ctx, "Ignore Port State Processing - Device not UP", log.Fields{"Device": d.ID, "Port": port, "DeviceState": d.State})
+		return
+	}
+	if p := d.GetPortByID(port); p != nil {
+		logger.Infow(ctx, "Port State Processing after Reboot", log.Fields{"Received": state, "Current": p.State})
+		p.Tasks.Initialize(d.ctx)
+		if p.State == PortStateUp {
+			logger.Infow(ctx, "Port State: UP", log.Fields{"Device": d.ID, "Port": port})
+			GetController().PortUpInd(d.ID, p.Name)
+		} else if p.State == PortStateDown {
+			logger.Infow(ctx, "Port State: Down", log.Fields{"Device": d.ID, "Port": port})
+			GetController().PortDownInd(d.ID, p.Name)
+		}
+	}
+}
+
+// ChangeEvent : Change event brings in ports related changes such as addition/deletion
+// or modification where the port status change up/down is indicated to the
+// controller
+func (d *Device) ChangeEvent(event *ofp.ChangeEvent) error {
+	cet := NewChangeEventTask(d.ctx, event, d)
+	d.AddTask(cet)
+	return nil
+}
+
+// PacketIn handle the incoming packet-in and deliver to the application for the
+// actual processing
+func (d *Device) PacketIn(pkt *ofp.PacketIn) {
+	logger.Debugw(ctx, "Received a Packet-In", log.Fields{"Device": d.ID})
+	if pkt.PacketIn.Reason != ofp.OfpPacketInReason_OFPR_ACTION {
+		logger.Warnw(ctx, "Unsupported PacketIn Reason", log.Fields{"Reason": pkt.PacketIn.Reason})
+		return
+	}
+	data := pkt.PacketIn.Data
+	port := PacketInGetPort(pkt.PacketIn)
+	if pName, err := d.GetPortName(port); err == nil {
+		GetController().PacketInInd(d.ID, pName, data)
+	} else {
+		logger.Warnw(ctx, "Unknown Port", log.Fields{"Reason": err.Error()})
+	}
+}
+
+// PacketInGetPort to get the port on which the packet-in is reported
+func PacketInGetPort(pkt *ofp.OfpPacketIn) uint32 {
+	for _, field := range pkt.Match.OxmFields {
+		if field.OxmClass == ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC {
+			if ofbField, ok := field.Field.(*ofp.OfpOxmField_OfbField); ok {
+				if ofbField.OfbField.Type == ofp.OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT {
+					if port, ok := ofbField.OfbField.Value.(*ofp.OfpOxmOfbField_Port); ok {
+						return port.Port
+					}
+				}
+			}
+		}
+	}
+	return 0
+}
+
+// PacketOutReq receives the packet out request from the application via the
+// controller. The interface from the application uses name as the identity.
+func (d *Device) PacketOutReq(outport string, inport string, data []byte, isCustomPkt bool) error {
+	inp, err := d.GetPortID(inport)
+	if err != nil {
+		return errors.New("Unknown inport")
+	}
+	outp, err1 := d.GetPortID(outport)
+	if err1 != nil {
+		return errors.New("Unknown outport")
+	}
+	logger.Debugw(ctx, "Sending packet out", log.Fields{"Device": d.ID, "Inport": inport, "Outport": outport})
+	return d.SendPacketOut(outp, inp, data, isCustomPkt)
+}
+
+// SendPacketOut is responsible for building the OF structure and send the
+// packet-out to the VOLTHA
+func (d *Device) SendPacketOut(outport uint32, inport uint32, data []byte, isCustomPkt bool) error {
+	pout := &ofp.PacketOut{}
+	pout.Id = d.ID
+	opout := &ofp.OfpPacketOut{}
+	pout.PacketOut = opout
+	opout.InPort = inport
+	opout.Data = data
+	opout.Actions = []*ofp.OfpAction{
+		{
+			Type: ofp.OfpActionType_OFPAT_OUTPUT,
+			Action: &ofp.OfpAction_Output{
+				Output: &ofp.OfpActionOutput{
+					Port:   outport,
+					MaxLen: 65535,
+				},
+			},
+		},
+	}
+	d.packetOutChannel <- pout
+	return nil
+}
+
+// UpdateFlows receives the flows in the form that is implemented
+// in the VGC and transforms them to the OF format. This is handled
+// as a port of the task that is enqueued to do the same.
+func (d *Device) UpdateFlows(flow *of.VoltFlow, devPort *DevicePort) {
+	t := NewAddFlowsTask(d.ctx, flow, d)
+	logger.Debugw(ctx, "Port Context", log.Fields{"Ctx": devPort.GetContext()})
+	// check if port isNni , if yes flows will be added to device port queues.
+	if util.IsNniPort(devPort.ID) {
+		// Adding the flows to device port queues.
+		devPort.AddTask(t)
+		return
+	}
+	// If the flowHash is enabled then add the flows to the flowhash generated queues.
+	flowQueue := d.getAndAddFlowQueueForUniID(uint32(devPort.ID))
+	if flowQueue != nil {
+		logger.Debugw(ctx, "flowHashQId", log.Fields{"uniid": devPort.ID, "flowhash": flowQueue.ID})
+		flowQueue.AddTask(t)
+		logger.Debugw(ctx, "Tasks Info", log.Fields{"uniid": devPort.ID, "flowhash": flowQueue.ID, "Total": flowQueue.TotalTasks(), "Pending": flowQueue.NumPendingTasks()})
+	} else {
+		//FlowThrotling disabled, add to the device port queue
+		devPort.AddTask(t)
+		return
+	}
+}
+
+// UpdateGroup to update group info
+func (d *Device) UpdateGroup(group *of.Group, devPort *DevicePort) {
+	task := NewModGroupTask(d.ctx, group, d)
+	logger.Debugw(ctx, "NNI Port Context", log.Fields{"Ctx": devPort.GetContext()})
+	devPort.AddTask(task)
+}
+
+// ModMeter for mod meter task
+func (d *Device) ModMeter(command of.MeterCommand, meter *of.Meter, devPort *DevicePort) {
+	if command == of.MeterCommandAdd {
+		if _, err := d.GetMeter(meter.ID); err == nil {
+			logger.Debugw(ctx, "Meter already added", log.Fields{"ID": meter.ID})
+			return
+		}
+	}
+	t := NewModMeterTask(d.ctx, command, meter, d)
+	devPort.AddTask(t)
+}
+
+func (d *Device) getAndAddFlowQueueForUniID(id uint32) *UniIDFlowQueue {
+	d.flowQueueLock.RLock()
+	//If flowhash is 0 that means flowhash throttling is disabled, return nil
+	if d.flowHash == 0 {
+		d.flowQueueLock.RUnlock()
+		return nil
+	}
+	flowHashID := id % uint32(d.flowHash)
+	if value, found := d.flowQueue[uint32(flowHashID)]; found {
+		d.flowQueueLock.RUnlock()
+		return value
+	}
+	d.flowQueueLock.RUnlock()
+	logger.Debugw(ctx, "Flow queue not found creating one", log.Fields{"uniid": id, "hash": flowHashID})
+
+	return d.addFlowQueueForUniID(id)
+}
+
+func (d *Device) addFlowQueueForUniID(id uint32) *UniIDFlowQueue {
+
+	d.flowQueueLock.Lock()
+	defer d.flowQueueLock.Unlock()
+	flowHashID := id % uint32(d.flowHash)
+	flowQueue := NewUniIDFlowQueue(uint32(flowHashID))
+	flowQueue.Tasks.Initialize(d.ctx)
+	d.flowQueue[flowHashID] = flowQueue
+	return flowQueue
+}
+
+// SetFlowHash sets the device flow hash and writes to the DB.
+func (d *Device) SetFlowHash(hash uint32) {
+	d.flowQueueLock.Lock()
+	defer d.flowQueueLock.Unlock()
+
+	d.flowHash = hash
+	d.writeFlowHashToDB()
+}
+
+func (d *Device) writeFlowHashToDB() {
+	hash, err := json.Marshal(d.flowHash)
+	if err != nil {
+		logger.Errorw(ctx, "failed to marshal flow hash", log.Fields{"hash": d.flowHash})
+		return
+	}
+	if err := db.PutFlowHash(d.ID, string(hash)); err != nil {
+		logger.Errorw(ctx, "Failed to add flow hash to DB", log.Fields{"device": d.ID, "hash": d.flowHash})
+	}
+}
+
+//isSBOperAllowed - determins if the SB operation is allowed based on device state & force flag
+func (d *Device) isSBOperAllowed(forceAction bool) bool {
+
+	if d.State == DeviceStateUP {
+		return true
+	}
+
+	if d.State == DeviceStateDISABLED && forceAction {
+		return true
+	}
+
+	return false
+}
+
+func (d *Device) triggerFlowNotification(cookie uint64, oper of.Command, bwDetails of.BwAvailDetails, err error) {
+	flow, _ := d.GetFlow(cookie)
+	d.triggerFlowResultNotification(cookie, flow, oper, bwDetails, err)
+}
+
+func (d *Device) triggerFlowResultNotification(cookie uint64, flow *of.VoltSubFlow, oper of.Command, bwDetails of.BwAvailDetails, err error) {
+
+	statusCode, statusMsg := infraerror.GetErrorInfo(err)
+	success := isFlowOperSuccess(statusCode, oper)
+
+	updateFlow := func(cookie uint64, state int, reason string) {
+		if dbFlow, ok := d.GetFlow(cookie); ok {
+			dbFlow.State = uint8(state)
+			dbFlow.ErrorReason = reason
+			d.AddFlowToDb(dbFlow)
+		}
+	}
+
+	//Update flow results
+	// Add - Update Success or Failure status with reason
+	// Del - Delete entry from DB on success else update error reason
+	if oper == of.CommandAdd {
+		state := of.FlowAddSuccess
+		reason := ""
+		if !success {
+			state = of.FlowAddFailure
+			reason = statusMsg
+		}
+		updateFlow(cookie, state, reason)
+		logger.Debugw(ctx, "Updated Flow to DB", log.Fields{"Cookie": cookie, "State": state})
+	} else {
+		if success && flow != nil {
+			if err := d.DelFlow(flow); err != nil {
+				logger.Warnw(ctx, "Delete Flow Error", log.Fields{"Cookie": flow.Cookie, "Reason": err.Error()})
+			}
+		} else if !success {
+			updateFlow(cookie, of.FlowDelFailure, statusMsg)
+		}
+	}
+
+	flowResult := intf.FlowStatus{
+		Cookie:         strconv.FormatUint(cookie, 10),
+		Device:         d.ID,
+		FlowModType:    oper,
+		Flow:           flow,
+		Status:         statusCode,
+		Reason:         statusMsg,
+		AdditionalData: bwDetails,
+	}
+
+	logger.Infow(ctx, "Sending Flow Notification", log.Fields{"Cookie": cookie, "Error Code": statusCode, "FlowOp": oper})
+	GetController().ProcessFlowModResultIndication(flowResult)
+}
diff --git a/internal/pkg/controller/modgroup.go b/internal/pkg/controller/modgroup.go
new file mode 100644
index 0000000..49da920
--- /dev/null
+++ b/internal/pkg/controller/modgroup.go
@@ -0,0 +1,133 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	"time"
+
+	infraerror "voltha-go-controller/internal/pkg/errorcodes"
+	infraerrorcode "voltha-go-controller/internal/pkg/errorcodes/service"
+
+	"voltha-go-controller/internal/pkg/of"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"google.golang.org/grpc/codes"
+)
+
+//ModGroupTask - Group Modification Task
+type ModGroupTask struct {
+	taskID    uint8
+	ctx       context.Context
+	group     *of.Group
+	device    *Device
+	timestamp string
+}
+
+//NewModGroupTask - Initializes new group task
+func NewModGroupTask(ctx context.Context, group *of.Group, device *Device) *ModGroupTask {
+	var grp ModGroupTask
+	grp.device = device
+	grp.group = group
+	grp.ctx = ctx
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	grp.timestamp = tstamp
+	return &grp
+}
+
+//Name - Name of task
+func (grp *ModGroupTask) Name() string {
+	return "Group Mod Task"
+}
+
+//TaskID - Task id
+func (grp *ModGroupTask) TaskID() uint8 {
+	return grp.taskID
+}
+
+// Timestamp to return timestamp of the task
+func (grp *ModGroupTask) Timestamp() string {
+	return grp.timestamp
+}
+
+//Stop - task stop
+func (grp *ModGroupTask) Stop() {
+}
+
+//Start - task start
+func (grp *ModGroupTask) Start(ctx context.Context, taskID uint8) error {
+	var err error
+	grp.taskID = taskID
+	grp.ctx = ctx
+	i := 0
+
+	processGroupModResult := func(err error) bool {
+
+		statusCode, statusMsg := infraerror.GetErrorInfo(err)
+
+		if infraerrorcode.ErrorCode(statusCode) != infraerrorcode.ErrOk {
+
+			if grp.group.Command == of.GroupCommandAdd && (codes.Code(statusCode) == codes.AlreadyExists) {
+				logger.Warnw(ctx, "Update Group Table Failed - Ignoring since Group Already exists",
+					log.Fields{"groupId": grp.group.GroupID, "groupOp": grp.group.Command, "Status": statusCode, "errorReason": statusMsg})
+				return true
+			}
+			logger.Errorw(ctx, "Update Group Table Failed",
+				log.Fields{"groupId": grp.group.GroupID, "groupOp": grp.group.Command, "Status": statusCode, "errorReason": statusMsg})
+			return false
+		}
+		logger.Infow(ctx, "Group Mod Result", log.Fields{"groupID": grp.group.GroupID, "Error Code": statusCode})
+		return true
+
+	}
+
+	if grp.group.Command != of.GroupCommandDel {
+		grp.group.State = of.GroupOperPending
+		grp.device.UpdateGroupEntry(grp.group)
+	} else {
+		grp.device.DelGroupEntry(grp.group)
+	}
+
+	if !grp.device.isSBOperAllowed(grp.group.ForceAction) {
+		logger.Errorw(ctx, "Skipping Group Table Update", log.Fields{"Reason": "Device State not UP", "State": grp.device.State, "GroupID": grp.group.GroupID, "Operation": grp.group.Command})
+		return nil
+	}
+
+	groupUpdate := of.CreateGroupTableUpdate(grp.group)
+	if vc := grp.device.VolthaClient(); vc != nil {
+
+		//Retry on group mod failure
+		//Retry attempts = 3
+		//Delay between retry = 100ms. Total Possible Delay = 200ms
+		for {
+			logger.Infow(ctx, "Group Mod Triggered", log.Fields{"GroupId": grp.group.GroupID, "Attempt": i})
+			_, err = vc.UpdateLogicalDeviceFlowGroupTable(grp.ctx, groupUpdate)
+			if isSuccess := processGroupModResult(err); isSuccess {
+				break
+			}
+			i++
+			if i < 3 {
+				time.Sleep(100 * time.Millisecond)
+				continue
+			}
+			logger.Errorw(ctx, "Update Group Table Failed on all 3 attempts. Dropping request", log.Fields{"GroupId": grp.group.GroupID, "Bucket": grp.group.Buckets})
+			break
+
+		}
+		return err
+	}
+	logger.Error(ctx, "Update Group Flow Table Failed: Voltha Client Unavailable")
+	return nil
+}
diff --git a/internal/pkg/controller/modmeter.go b/internal/pkg/controller/modmeter.go
new file mode 100644
index 0000000..04b1e04
--- /dev/null
+++ b/internal/pkg/controller/modmeter.go
@@ -0,0 +1,124 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	"time"
+
+	"voltha-go-controller/internal/pkg/of"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// ModMeterTask structure
+type ModMeterTask struct {
+	taskID    uint8
+	ctx       context.Context
+	command   of.MeterCommand
+	meter     *of.Meter
+	device    *Device
+	timestamp string
+}
+
+// NewModMeterTask is the constructor for ModMeterTask
+func NewModMeterTask(ctx context.Context, command of.MeterCommand, meter *of.Meter, device *Device) *ModMeterTask {
+	var mmt ModMeterTask
+	mmt.device = device
+	mmt.meter = meter
+	mmt.ctx = ctx
+	mmt.command = command
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	mmt.timestamp = tstamp
+	return &mmt
+}
+
+// Name returns name of the task
+func (mmt *ModMeterTask) Name() string {
+	return "Add Flows Task"
+}
+
+// TaskID returns task Id of the task
+func (mmt *ModMeterTask) TaskID() uint8 {
+	return mmt.taskID
+}
+
+// Timestamp returns time stamp for the task
+func (mmt *ModMeterTask) Timestamp() string {
+	return mmt.timestamp
+}
+
+// Stop to stop the task
+func (mmt *ModMeterTask) Stop() {
+}
+
+// Start to start the task
+func (mmt *ModMeterTask) Start(ctx context.Context, taskID uint8) error {
+	mmt.taskID = taskID
+	mmt.ctx = ctx
+
+	//Temp commenting Sync response handling
+	//triggerMeterNotification := func(err error) {
+
+	// 	statusCode, statusMsg := infraerror.GetErrorInfo(err)
+
+	// 	if mmt.command == of.MeterCommandAdd && infraerrorcode.ErrorCode(statusCode) != infraerrorcode.ErrOk {
+	// 		mmt.meter.State = of.MeterOperFailure
+	// 		mmt.meter.ErrorReason = statusMsg
+
+	// 		logger.Errorw(ctx, "Update Meter Table Failed",
+	// 			log.Fields{"meterId": mmt.meter.ID, "meterOp": mmt.command, "Status": statusCode, "errorReason": statusMsg})
+	// 		go mmt.device.AddMeterToDb(mmt.meter)
+	// 	} else {
+	// 		log.Infow("Meter Mod Result", log.Fields{"meterID": mmt.meter.ID, "Error Code": statusCode})
+	// 	}
+	// }
+
+	// First add/delete the flows first locally before passing them to actual device
+	if mmt.command == of.MeterCommandAdd {
+		mmt.meter.State = of.MeterOperPending
+		if err := mmt.device.AddMeter(mmt.meter); err != nil {
+			// Meter already exists so we dont have to do anything here
+			return nil
+		}
+	} else {
+		if !mmt.device.DelMeter(mmt.meter) {
+			// Meter doesn't exist so we dont have to do anything here
+			return nil
+		}
+	}
+
+	if mmt.device.State != DeviceStateUP {
+		logger.Errorw(ctx, "Update Meter Table Failed: Device State DOWN", log.Fields{"Reason": "Device State DOWN", "Meter": mmt.meter.ID})
+		return nil
+	}
+	meterMod, err := of.MeterUpdate(mmt.device.ID, mmt.command, mmt.meter)
+	if err != nil {
+		logger.Errorw(ctx, "Update Meter Table Failed", log.Fields{"Reason": err.Error()})
+		return err
+	}
+
+	if vc := mmt.device.VolthaClient(); vc != nil {
+
+		if _, err = vc.UpdateLogicalDeviceMeterTable(mmt.ctx, meterMod); err != nil {
+			logger.Errorw(ctx, "Update Meter Table Failed", log.Fields{"Reason": err.Error()})
+		}
+		//triggerMeterNotification(err)
+		return err
+	}
+
+	logger.Error(ctx, "Update Meter Table Failed: Voltha Client Unavailable")
+	return nil
+}
diff --git a/internal/pkg/controller/pendingprofiles.go b/internal/pkg/controller/pendingprofiles.go
new file mode 100644
index 0000000..6258f36
--- /dev/null
+++ b/internal/pkg/controller/pendingprofiles.go
@@ -0,0 +1,98 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package controller
+
+import (
+	"context"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// PendingProfilesTask structure
+type PendingProfilesTask struct {
+	taskID uint8
+	ctx    context.Context
+	device *Device
+	ts     string
+}
+
+// NewPendingProfilesTask is constructor for PendingProfilesTask
+func NewPendingProfilesTask(device *Device) *PendingProfilesTask {
+	var ppt PendingProfilesTask
+	ppt.device = device
+	ppt.ts = (time.Now()).Format(time.RFC3339Nano)
+	return &ppt
+}
+
+// Name returns name of the task
+func (ppt *PendingProfilesTask) Name() string {
+	return "Pending Profiles Task"
+}
+
+// TaskID returns task id of the task
+func (ppt *PendingProfilesTask) TaskID() uint8 {
+	return ppt.taskID
+}
+
+// Timestamp returns timestamp of the task
+func (ppt *PendingProfilesTask) Timestamp() string {
+	return ppt.ts
+}
+
+// Stop to stop the task
+func (ppt *PendingProfilesTask) Stop() {
+}
+
+// Start is called by the framework and is responsible for implementing
+// the actual task.
+func (ppt *PendingProfilesTask) Start(ctx context.Context, taskID uint8) error {
+	logger.Warnw(ctx, "Pending Profiles Task Triggered", log.Fields{"Context": ctx, "taskID": taskID, "Device": ppt.device.ID})
+	ppt.taskID = taskID
+	ppt.ctx = ctx
+	var errInfo error
+
+	GetController().SetAuditFlags(ppt.device)
+
+	//Trigger Pending Service Delete Tasks
+	logger.Warnw(ctx, "Pending Service Delete Task Triggered", log.Fields{"Device": ppt.device.ID})
+	GetController().TriggerPendingProfileDeleteReq(ppt.device.ID)
+	logger.Warnw(ctx, "Pending Service Delete Task Completed", log.Fields{"Device": ppt.device.ID})
+
+	//Trigger Pending Migrate Services Tasks
+	logger.Warnw(ctx, "Pending Migrate Services Task Triggered", log.Fields{"Device": ppt.device.ID})
+	GetController().TriggerPendingMigrateServicesReq(ppt.device.ID)
+	logger.Warnw(ctx, "Pending Migrate Services Task Completed", log.Fields{"Device": ppt.device.ID})
+
+	GetController().ResetAuditFlags(ppt.device)
+
+	// Updating Mvlan Profile
+	logger.Warnw(ctx, "Pending Update Mvlan Task Triggered", log.Fields{"Device": ppt.device.ID})
+	if err := ppt.UpdateMvlanProfiles(); err != nil {
+		logger.Errorw(ctx, "Update Mvlan Profile Failed", log.Fields{"Reason": err.Error()})
+		errInfo = err
+	}
+	logger.Warnw(ctx, "Pending Update Mvlan Task Completed", log.Fields{"Device": ppt.device.ID})
+
+	logger.Warnw(ctx, "Pending Profiles Task Completed", log.Fields{"Context": ctx, "taskID": taskID, "Device": ppt.device.ID})
+	return errInfo
+}
+
+// UpdateMvlanProfiles to update the mvlan profiles
+func (ppt *PendingProfilesTask) UpdateMvlanProfiles() error {
+	GetController().UpdateMvlanProfiles(ppt.device.ID)
+	return nil
+}
diff --git a/internal/pkg/controller/utils.go b/internal/pkg/controller/utils.go
new file mode 100644
index 0000000..c07ac59
--- /dev/null
+++ b/internal/pkg/controller/utils.go
@@ -0,0 +1,283 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package controller
+
+import (
+	"fmt"
+	"strings"
+	"sync"
+)
+
+var mu sync.Mutex
+var xid uint32 = 1
+
+// GetXid to get xid
+func GetXid() uint32 {
+	mu.Lock()
+	defer mu.Unlock()
+	xid++
+	return xid
+}
+
+// PadString for padding of string
+func PadString(value string, padSize int) string {
+	size := len(value)
+	nullsNeeded := padSize - size
+	null := fmt.Sprintf("%c", '\000')
+	padded := strings.Repeat(null, nullsNeeded)
+	return fmt.Sprintf("%s%s", value, padded)
+}
+
+/*
+// extractAction for extract action
+func extractAction(action ofp.IAction) *openflow_13.OfpAction {
+	var ofpAction openflow_13.OfpAction
+	switch action.GetType() {
+	case ofp.OFPATOutput:
+		var outputAction openflow_13.OfpAction_Output
+		loxiOutputAction := action.(*ofp.ActionOutput)
+		var output openflow_13.OfpActionOutput
+		output.Port = uint32(loxiOutputAction.GetPort())
+		/*
+			var maxLen uint16
+			maxLen = loxiOutputAction.GetMaxLen()
+			output.MaxLen = uint32(maxLen)
+
+		*/
+/*
+		output.MaxLen = 0
+		outputAction.Output = &output
+		ofpAction.Action = &outputAction
+		ofpAction.Type = openflow_13.OfpActionType_OFPAT_OUTPUT
+	case ofp.OFPATCopyTtlOut: //CopyTtltOut
+	case ofp.OFPATCopyTtlIn: //CopyTtlIn
+	case ofp.OFPATSetMplsTtl: //SetMplsTtl
+	case ofp.OFPATDecMplsTtl: //DecMplsTtl
+	case ofp.OFPATPushVLAN: //PushVlan
+		var pushVlan openflow_13.OfpAction_Push
+		loxiPushAction := action.(*ofp.ActionPushVlan)
+		var push openflow_13.OfpActionPush
+		push.Ethertype = uint32(loxiPushAction.Ethertype) //TODO This should be available in the fields
+		pushVlan.Push = &push
+		ofpAction.Type = openflow_13.OfpActionType_OFPAT_PUSH_VLAN
+		ofpAction.Action = &pushVlan
+	case ofp.OFPATPopVLAN: //PopVlan
+		ofpAction.Type = openflow_13.OfpActionType_OFPAT_POP_VLAN
+	case ofp.OFPATPushMpls: //PushMpls
+	case ofp.OFPATPopMpls: //PopMpls
+	case ofp.OFPATSetQueue: //SetQueue
+	case ofp.OFPATGroup: //ActionGroup
+	case ofp.OFPATSetNwTtl: //SetNwTtl
+	case ofp.OFPATDecNwTtl: //DecNwTtl
+	case ofp.OFPATSetField: //SetField
+		ofpAction.Type = openflow_13.OfpActionType_OFPAT_SET_FIELD
+		var ofpActionForSetField openflow_13.OfpAction_SetField
+		var ofpActionSetField openflow_13.OfpActionSetField
+		var ofpOxmField openflow_13.OfpOxmField
+		ofpOxmField.OxmClass = openflow_13.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+		var ofpOxmFieldForOfbField openflow_13.OfpOxmField_OfbField
+		var ofpOxmOfbField openflow_13.OfpOxmOfbField
+		loxiSetField := action.(*ofp.ActionSetField)
+		oxmName := loxiSetField.Field.GetOXMName()
+		switch oxmName {
+		//TODO handle set field sith other fields
+		case "vlan_vid":
+			ofpOxmOfbField.Type = openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID
+			var vlanVid openflow_13.OfpOxmOfbField_VlanVid
+			var VlanVid = loxiSetField.Field.GetOXMValue().(uint16)
+			vlanVid.VlanVid = uint32(VlanVid)
+
+			ofpOxmOfbField.Value = &vlanVid
+		}
+		ofpOxmFieldForOfbField.OfbField = &ofpOxmOfbField
+		ofpOxmField.Field = &ofpOxmFieldForOfbField
+		ofpActionSetField.Field = &ofpOxmField
+		ofpActionForSetField.SetField = &ofpActionSetField
+		ofpAction.Action = &ofpActionForSetField
+
+	case ofp.OFPATPushPbb: //PushPbb
+	case ofp.OFPATPopPbb: //PopPbb
+	case ofp.OFPATExperimenter: //Experimenter
+
+	}
+	return &ofpAction
+
+}
+
+// parseOxm for parsing OxmOfb field
+func parseOxm(ofbField *openflow_13.OfpOxmOfbField, DeviceID string) (goloxi.IOxm, uint16) {
+	switch ofbField.Type {
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT:
+		ofpInPort := ofp.NewOxmInPort()
+		val := ofbField.GetValue().(*openflow_13.OfpOxmOfbField_Port)
+		ofpInPort.Value = ofp.Port(val.Port)
+		return ofpInPort, 4
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE:
+		ofpEthType := ofp.NewOxmEthType()
+		val := ofbField.GetValue().(*openflow_13.OfpOxmOfbField_EthType)
+		ofpEthType.Value = ofp.EthernetType(val.EthType)
+		return ofpEthType, 2
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT:
+		ofpInPhyPort := ofp.NewOxmInPhyPort()
+		val := ofbField.GetValue().(*openflow_13.OfpOxmOfbField_PhysicalPort)
+		ofpInPhyPort.Value = ofp.Port(val.PhysicalPort)
+		return ofpInPhyPort, 4
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO:
+		ofpIPProto := ofp.NewOxmIpProto()
+		val := ofbField.GetValue().(*openflow_13.OfpOxmOfbField_IpProto)
+		ofpIPProto.Value = ofp.IpPrototype(val.IpProto)
+		return ofpIPProto, 1
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC:
+		ofpUDPSrc := ofp.NewOxmUdpSrc()
+		val := ofbField.GetValue().(*openflow_13.OfpOxmOfbField_UdpSrc)
+		ofpUDPSrc.Value = uint16(val.UdpSrc)
+		return ofpUDPSrc, 2
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST:
+		ofpUDPDst := ofp.NewOxmUdpDst()
+		val := ofbField.GetValue().(*openflow_13.OfpOxmOfbField_UdpDst)
+		ofpUDPDst.Value = uint16(val.UdpDst)
+		return ofpUDPDst, 2
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID:
+		ofpVlanVid := ofp.NewOxmVlanVid()
+		val := ofbField.GetValue()
+		if val == nil {
+			ofpVlanVid.Value = uint16(0)
+			return ofpVlanVid, 2
+		}
+		vlanID := val.(*openflow_13.OfpOxmOfbField_VlanVid)
+		if ofbField.HasMask {
+			ofpVlanVidMasked := ofp.NewOxmVlanVidMasked()
+			valMask := ofbField.GetMask()
+			vlanMask := valMask.(*openflow_13.OfpOxmOfbField_VlanVidMask)
+			if vlanID.VlanVid == 4096 && vlanMask.VlanVidMask == 4096 {
+				ofpVlanVidMasked.Value = uint16(vlanID.VlanVid)
+				ofpVlanVidMasked.ValueMask = uint16(vlanMask.VlanVidMask)
+			} else {
+				ofpVlanVidMasked.Value = uint16(vlanID.VlanVid) | 0x1000
+				ofpVlanVidMasked.ValueMask = uint16(vlanMask.VlanVidMask)
+
+			}
+			return ofpVlanVidMasked, 4
+		}
+		ofpVlanVid.Value = uint16(vlanID.VlanVid) | 0x1000
+		return ofpVlanVid, 2
+	case openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_METADATA:
+		ofpMetadata := ofp.NewOxmMetadata()
+		val := ofbField.GetValue().(*openflow_13.OfpOxmOfbField_TableMetadata)
+		ofpMetadata.Value = val.TableMetadata
+		return ofpMetadata, 8
+	default:
+	}
+	return nil, 0
+}
+
+// parseInstructions for parsing of instructions
+func parseInstructions(ofpInstruction *openflow_13.OfpInstruction, DeviceID string) (ofp.IInstruction, uint16) {
+	instType := ofpInstruction.Type
+	data := ofpInstruction.GetData()
+	switch instType {
+	case ofp.OFPITWriteMetadata:
+		instruction := ofp.NewInstructionWriteMetadata()
+		instruction.Len = 24
+		metadata := data.(*openflow_13.OfpInstruction_WriteMetadata).WriteMetadata
+		instruction.Metadata = uint64(metadata.Metadata)
+		return instruction, 24
+	case ofp.OFPITMeter:
+		instruction := ofp.NewInstructionMeter()
+		instruction.Len = 8
+		meter := data.(*openflow_13.OfpInstruction_Meter).Meter
+		instruction.MeterId = meter.MeterId
+		return instruction, 8
+	case ofp.OFPITGotoTable:
+		instruction := ofp.NewInstructionGotoTable()
+		instruction.Len = 8
+		gotoTable := data.(*openflow_13.OfpInstruction_GotoTable).GotoTable
+		instruction.TableId = uint8(gotoTable.TableId)
+		return instruction, 8
+	case ofp.OFPITApplyActions:
+		instruction := ofp.NewInstructionApplyActions()
+		var instructionSize uint16
+		instructionSize = 8
+		//ofpActions := ofpInstruction.GetActions().Actions
+		var actions []goloxi.IAction
+		for _, ofpAction := range ofpInstruction.GetActions().Actions {
+			action, actionSize := parseAction(ofpAction, DeviceID)
+			actions = append(actions, action)
+			instructionSize += actionSize
+
+		}
+		instruction.Actions = actions
+		instruction.SetLen(instructionSize)
+		return instruction, instructionSize
+	}
+	//shouldn't have reached here :<
+	return nil, 0
+}
+
+// parseAction for parsing of actions
+func parseAction(ofpAction *openflow_13.OfpAction, DeviceID string) (goloxi.IAction, uint16) {
+	switch ofpAction.Type {
+	case openflow_13.OfpActionType_OFPAT_OUTPUT:
+		ofpOutputAction := ofpAction.GetOutput()
+		outputAction := ofp.NewActionOutput()
+		outputAction.Port = ofp.Port(ofpOutputAction.Port)
+		outputAction.MaxLen = uint16(ofpOutputAction.MaxLen)
+		outputAction.Len = 16
+		return outputAction, 16
+	case openflow_13.OfpActionType_OFPAT_PUSH_VLAN:
+		ofpPushVlanAction := ofp.NewActionPushVlan()
+		ofpPushVlanAction.Ethertype = uint16(ofpAction.GetPush().Ethertype)
+		ofpPushVlanAction.Len = 8
+		return ofpPushVlanAction, 8
+	case openflow_13.OfpActionType_OFPAT_POP_VLAN:
+		ofpPopVlanAction := ofp.NewActionPopVlan()
+		ofpPopVlanAction.Len = 8
+		return ofpPopVlanAction, 8
+	case openflow_13.OfpActionType_OFPAT_SET_FIELD:
+		ofpActionSetField := ofpAction.GetSetField()
+		setFieldAction := ofp.NewActionSetField()
+
+		iOxm, _ := parseOxm(ofpActionSetField.GetField().GetOfbField(), DeviceID)
+		setFieldAction.Field = iOxm
+		setFieldAction.Len = 16
+		return setFieldAction, 16
+	default:
+	}
+	return nil, 0
+}
+
+// parsePortStats for parsing of port stats
+func parsePortStats(port *voltha.LogicalPort) *ofp.PortStatsEntry {
+	stats := port.OfpPortStats
+	port.OfpPort.GetPortNo()
+	var entry ofp.PortStatsEntry
+	entry.SetPortNo(ofp.Port(port.OfpPort.GetPortNo()))
+	entry.SetRxPackets(stats.GetRxPackets())
+	entry.SetTxPackets(stats.GetTxPackets())
+	entry.SetRxBytes(stats.GetRxBytes())
+	entry.SetTxBytes(stats.GetTxBytes())
+	entry.SetRxDropped(stats.GetRxDropped())
+	entry.SetTxDropped(stats.GetTxDropped())
+	entry.SetRxErrors(stats.GetRxErrors())
+	entry.SetTxErrors(stats.GetTxErrors())
+	entry.SetRxFrameErr(stats.GetRxFrameErr())
+	entry.SetRxOverErr(stats.GetRxOverErr())
+	entry.SetRxCrcErr(stats.GetRxCrcErr())
+	entry.SetCollisions(stats.GetCollisions())
+	entry.SetDurationSec(stats.GetDurationSec())
+	entry.SetDurationNsec(stats.GetDurationNsec())
+	return &entry
+}*/
diff --git a/internal/pkg/errorcodes/errorcodes.go b/internal/pkg/errorcodes/errorcodes.go
new file mode 100644
index 0000000..4bb0490
--- /dev/null
+++ b/internal/pkg/errorcodes/errorcodes.go
@@ -0,0 +1,333 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+// Package errorcodes provides constants that are commonly used by RWcore and adapters.
+package errorcodes
+
+import (
+	"net/http"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+// NBErrorCode represents the error code for the error.
+type NBErrorCode int
+
+const (
+        // VolthaErrorMessageFormat represents the format in which the Voltha accepts the errors.
+        VolthaErrorMessageFormat = "code = %d, desc = %s"
+)
+
+// List of error messages returned to Voltha.
+var (
+	// ErrUnimplementedRPC is returned when the RPC is not implemented
+	ErrUnimplementedRPC = status.Errorf(codes.Unimplemented, VolthaErrorMessageFormat, UnsupportedOperation, "Operation not implemented")
+	// ErrOperationNotSupported is returned when the operation is not supported
+	ErrOperationNotSupported = status.Errorf(codes.Unimplemented, VolthaErrorMessageFormat, UnsupportedOperation, "Operation not supported")
+
+	// ErrFailedRequest is returned when the component fails to send any request to other component
+	ErrFailedRequest = status.Errorf(codes.Internal, VolthaErrorMessageFormat, UnsuccessfulOperation, "Failed to send request")
+
+	// ErrFailedToEncodeConfig is returned when the data json marshal fails
+	ErrFailedToEncodeConfig = status.Errorf(codes.Internal, VolthaErrorMessageFormat, MessageEncodeFailed, "Failed to encode data")
+	// ErrFailedToDecodeConfig is returned when the data json unmarshal fails
+	ErrFailedToDecodeConfig = status.Errorf(codes.Internal, VolthaErrorMessageFormat, MessageDecodeFailed, "Failed to decode data")
+
+	// ErrFailedToUpdateDB is returned when update of data in KV store fails
+	ErrFailedToUpdateDB = status.Errorf(codes.Internal, VolthaErrorMessageFormat, DBOperationFailed, "Failed to update DB")
+	// ErrFailedToGetFromDB is returned when get data from KV store fails
+	ErrFailedToGetFromDB = status.Errorf(codes.Internal, VolthaErrorMessageFormat, DBOperationFailed, "Failed to fetch from DB")
+	// ErrFailedToDeleteFromDB is returned when delete data from KV store fails
+	ErrFailedToDeleteFromDB = status.Errorf(codes.Internal, VolthaErrorMessageFormat, DBOperationFailed, "Failed to delete from DB")
+
+	// ErrDeviceNotFound is returned when the handler for the device is not present in VOLTHA
+	ErrDeviceNotFound = status.Errorf(codes.NotFound, VolthaErrorMessageFormat, ResourceNotFound, "Device not found")
+	// ErrDeviceNotReachable is returned when the connection between adapter and agent is broken
+	ErrDeviceNotReachable = status.Errorf(codes.Unavailable, VolthaErrorMessageFormat, DeviceUnreachable, "Device is not reachable")
+	// ErrWrongDevice is returned when the received request has wrong device (parent/child)
+	ErrWrongDevice = status.Errorf(codes.FailedPrecondition, VolthaErrorMessageFormat, PrerequisiteNotMet, "Wrong device in request")
+	// ErrDeviceNotEnabledAndUp is returned when the state of the device is neither enabled nor active
+	ErrDeviceNotEnabledAndUp = status.Errorf(codes.FailedPrecondition, VolthaErrorMessageFormat, ResourceInImproperState, "Device is not enabled and up")
+	// ErrDeviceDeleted is returned when the state of the device is DELETED
+	ErrDeviceDeleted = status.Errorf(codes.FailedPrecondition, VolthaErrorMessageFormat, ResourceInImproperState, "Device is deleted")
+
+	// ErrPortNotFound is returned when the port is not present in VOLTHA
+	ErrPortNotFound = status.Errorf(codes.NotFound, VolthaErrorMessageFormat, ResourceNotFound, "Port not found")
+	// ErrPortIsInInvalidState is returned when the port is in an invalid state
+	ErrPortIsInInvalidState = status.Errorf(codes.Internal, VolthaErrorMessageFormat, ResourceInImproperState, "Port is in an invalid state")
+
+	// ErrInvalidParamInRequest is returned when the request contains invalid configuration
+	ErrInvalidParamInRequest = status.Errorf(codes.InvalidArgument, VolthaErrorMessageFormat, InvalidArgument, "Received invalid configuration in request")
+
+	// ErrImageNotRegistered is returned when the image is not registered
+	ErrImageNotRegistered = status.Errorf(codes.FailedPrecondition, VolthaErrorMessageFormat, PrerequisiteNotMet, "Image is not registered")
+	// ErrImageDownloadInProgress is returned when the image download is in progress
+	ErrImageDownloadInProgress = status.Errorf(codes.FailedPrecondition, VolthaErrorMessageFormat, MethodNotAllowed, "Image download is in progress")
+)
+
+// ConvertToVolthaErrorFormat converts the error to Voltha error format
+func ConvertToVolthaErrorFormat(err error) error {
+	st, ok := status.FromError(err)
+	if !ok {
+		return err
+	}
+	return status.Errorf(st.Code(), VolthaErrorMessageFormat, GrpcToVolthaErrorCodeMap[st.Code()], st.Message())
+}
+
+const (
+        //Success is returned when there is no error - 0
+        Success NBErrorCode = iota
+        //InvalidURL is returned when the URL specified for the request is invalid - 1
+        InvalidURL
+        //MissingArgument is returned when the mandatory/conditionally mandatory argument is missing - 2
+        MissingArgument
+        //RequestTimeout is returned when the request timed out. - 3
+        RequestTimeout
+        //ResourceAlreadyExists is returned when the resource already exists and create for the same is not allowed - 4
+        ResourceAlreadyExists
+        //ResourceInImproperState is returned when the resource is in improper state to process the request. - 5
+        ResourceInImproperState
+        //DeviceUnreachable is returned when the device is not reachable - 6
+        DeviceUnreachable
+        //OperationAlreadyInProgress is returned when the requested operation is already in progress - 7
+        OperationAlreadyInProgress
+        //InvalidConfig is returned when the configuration provided is invalid - 8
+        InvalidConfig
+        //ResourceNotFound is returned when the resource is not found - 9
+        ResourceNotFound
+        //MethodNotAllowed is returned when the requested method is not allowed - 10
+        MethodNotAllowed
+        //ResourceInUse is returned when the resource is in use, the delete of the resource is not allowed when in use - 11
+        ResourceInUse
+        //JobIDNotFound is returned when the Job ID not found - 12
+        JobIDNotFound
+        //JobIDAlreadyInUse is returned when the Job ID already in use - 13
+        JobIDAlreadyInUse
+        //PeerUnreachable is returned when the peer is unreachable -14
+        PeerUnreachable
+        //InvalidPatchOperation is returned when the parameter(s) mentioned in the patch operation are invalid - 15
+        InvalidPatchOperation
+        //OLTUnreachable is returned when the OLT is not reachable - 16
+        OLTUnreachable
+        //PrerequisiteNotMet is returned when the required prerequisite is not met to execute the requested procedure - 17
+        PrerequisiteNotMet
+        //MessageEncodeFailed is returned when Message encoding failed - 18
+        MessageEncodeFailed
+        //MessageDecodeFailed is returned when Message decoding failed - 19
+        MessageDecodeFailed
+        //ONTInternalError is returned when Internal error is reported by the ONT - 20
+        ONTInternalError
+        //OLTInternalError is returned when Internal error is reported by the OLT - 21
+        OLTInternalError
+        //VolthaInternalError is returned when Internal error occurred at Voltha - 22
+        VolthaInternalError
+        //ConfigMismatch is returned when the configuration does not match - 23
+        ConfigMismatch
+        //DBOperationFailed is returned when the database operation failed for the key - 24
+        DBOperationFailed
+        //ResourceLimitExceeded is returned when the resource limit exceeded the allowed limit - 25
+        ResourceLimitExceeded
+        //UndefinedEnv is returned when the required environment variable is not defined - 26
+        UndefinedEnv
+        //InvalidArgument is returned when the argument provided is invalid - 27
+        InvalidArgument
+        //InvalidPayload is returned when the configuration payload is invalid - 28
+        InvalidPayload
+        //DuplicateKey is returned when the duplicate entry for the key - 29
+        DuplicateKey
+        //DuplicateValue is returned when the duplicate entry for the value - 30
+        DuplicateValue
+        //UnsupportedOperation is returned when the request operation is not supported - 31
+        UnsupportedOperation
+        //UserUnauthorized is returned when the user is unauthorized to perform the requested operation - 32
+        UserUnauthorized
+        //LiveKPISubscriptionExists is returned when the live KPI subscription exists already for the requested resource - 33
+        LiveKPISubscriptionExists
+        //UnsuccessfulOperation is returned when the requested operation is unsuccessful - 34
+        UnsuccessfulOperation
+        //ResourceInDisabledStateAlready is returned when the resource is in disabled state already - 35
+        ResourceInDisabledStateAlready
+        //ResourceInEnabledStateAlready is returned when the resource is in enabled state already - 36
+        ResourceInEnabledStateAlready
+        //ResourceNotDiscoveredYet is returned when the resource is not discovered yet - 37
+        ResourceNotDiscoveredYet
+        //HighDiskUtilization is returned when the disk utilization is high, consider the disk cleanup. - 38
+        HighDiskUtilization
+        //KafkaError is returned when there is a kafka error - 39
+        KafkaError
+        //ResourceBusy is returned when the component/resource is busy. - 40
+        ResourceBusy
+        // UnsupportedParameter is returned when un supported field is provided in request. -41
+        UnsupportedParameter
+        //JobIDAlreadyExists is returned when the Job ID is already there in DB. -42
+        JobIDAlreadyExists
+        //LiveKPISubscriptionNotFound is returned when the live KPI subscription not found for the requested resource. -42
+        LiveKPISubscriptionNotFound
+        // HostUnreachable is returned when failed to establish the SFTP connection. -44
+        HostUnreachable
+        // DHCPServerUnreachable is returned when dhcp server is unreachable. -45
+        DHCPServerUnreachable
+        // SessionExpired is returned when user session is expired/timeout - 46
+        SessionExpired
+        // AccessDenied is returned when user operation is forbidden - 47
+        AccessDenied
+        // PasswordUpdateRequired is returned when password for the user is about to expire - 48
+        PasswordUpdateRequired
+        // InvalidMessageHeader is returned when token in security request is invalid/nil - 49
+        InvalidMessageHeader
+        // UserAccountBlocked is returned when user account gets blocked after multiple invalid attempts - 50
+        UserAccountBlocked
+        // UserAccountExpired is returned when user account gets expired - 51
+        UserAccountExpired
+        // UserAccountDormant is returned when user account gets dormant - 52
+        UserAccountDormant
+        // InvalidCredentials is returned when credentials are invalid in login request - 53
+        InvalidCredentials
+        // ConcurrentAccessFromMultipleIPs when multiple sessions gets established from same ip - 54
+        ConcurrentAccessFromMultipleIPs
+        // KPIThresholdCrossed when KPI threshold is crossed - 55
+        KPIThresholdCrossed
+        // ONTUnreachable is returned when the ONT is not reachable - 56
+        ONTUnreachable
+        // ResourceUnreachable is returned when the resource is not reachable -57
+        ResourceUnreachable
+        // ONTProcessingError is returned when onu returns processing error for omci message - 58
+        ONTProcessingError
+        // ONTResourceBusy is returned when onu returns device busy error for omci message - 59
+        ONTResourceBusy
+        // ONTMEInstanceExists is returned when onu returns OMCI ME instance exists error for omci message - 60
+        ONTMEInstanceExists
+        // ONTUnknownMEInstance is returned when onu returns OMCI ME Unknown Instance error for omci message - 61
+        ONTUnknownMEInstance
+        // JoinUnsuccessful is returned when an IGMP Join request is unsuccessful - 62
+        JoinUnsuccessful
+        // QueryExpired is returned when there is no response to IGMP Queries from the controller - 63
+        QueryExpired
+        // AvailableBwValidationErr is returned when requested bandwidth is not available on the pon port - 64
+        AvailableBwValidationErr
+)
+
+//NBErrorCodeMap converts error code to error description string
+var NBErrorCodeMap = map[NBErrorCode]string{
+        Success:                         "Success",
+        InvalidURL:                      "INVALID_URL",
+        RequestTimeout:                  "REQUEST_TIMEOUT",
+        MissingArgument:                 "MISSING_ARGUMENT",
+        ResourceAlreadyExists:           "RESOURCE_ALREADY_EXISTS",
+        ResourceInImproperState:         "RESOURCE_IN_IMPROPER_STATE",
+        DeviceUnreachable:               "DEVICE_UNREACHABLE",
+        OperationAlreadyInProgress:      "OPERATION_ALREADY_IN_PROGRESS",
+        InvalidConfig:                   "INVALID_CONFIG",
+        ResourceNotFound:                "RESOURCE_NOT_FOUND",
+        MethodNotAllowed:                "METHOD_NOT_ALLOWED",
+        ResourceInUse:                   "RESOURCE_IN_USE",
+        JobIDNotFound:                   "JOB_ID_NOT_FOUND",
+        JobIDAlreadyInUse:               "JOB_ID_ALREADY_IN_USE",
+        PeerUnreachable:                 "PEER_UNREACHABLE",
+        InvalidPatchOperation:           "INVALID_PATCH_OPERATION",
+        OLTUnreachable:                  "OLT_UNREACHABLE",
+        PrerequisiteNotMet:              "PREREQUISITE_NOT_MET",
+        MessageEncodeFailed:             "MESSAGE_ENCODE_FAILED",
+        MessageDecodeFailed:             "MESSAGE_DECODE_FAILED",
+        ONTInternalError:                "ONT_INTERNAL_ERROR",
+        OLTInternalError:                "OLT_INTERNAL_ERROR",
+        VolthaInternalError:               "Voltha_INTERNAL_ERROR",
+        ConfigMismatch:                  "CONFIG_MISMATCH",
+        DBOperationFailed:               "DB_OPERATION_FAILED",
+        ResourceLimitExceeded:           "RESOURCE_LIMIT_EXCEEDED",
+        UndefinedEnv:                    "UNDEFINED_ENV",
+        InvalidArgument:                 "INVALID_ARGUMENT",
+        InvalidPayload:                  "INVALID_PAYLOAD",
+        DuplicateKey:                    "DUPLICATE_KEY",
+        DuplicateValue:                  "DUPLICATE_VALUE",
+        UnsupportedOperation:            "UNSUPPORTED_OPERATION",
+        UserUnauthorized:                "USER_UNAUTHORIZED",
+        LiveKPISubscriptionExists:       "LIVE_KPI_SUBSCRIPTION_EXISTS",
+        UnsuccessfulOperation:           "UNSUCCESSFUL_OPERATION",
+        ResourceInDisabledStateAlready:  "RESOURCE_IN_DISABLED_STATE_ALREADY",
+        ResourceInEnabledStateAlready:   "RESOURCE_IN_ENABLED_STATE_ALREADY",
+        ResourceNotDiscoveredYet:        "RESOURCE_NOT_DISCOVERED_YET",
+        HighDiskUtilization:             "HIGH_DISK_UTILIZATION",
+        KafkaError:                      "KAFKA_ERROR",
+        LiveKPISubscriptionNotFound:     "LIVE_KPI_SUBSCRIPTION_NOT_FOUND",
+        ResourceBusy:                    "RESOURCE_BUSY",
+        UnsupportedParameter:            "UNSUPPORTED_PARAMETER",
+        JobIDAlreadyExists:              "JOB_ID_ALREADY_EXISTS",
+        HostUnreachable:                 "HOST_UNREACHABLE",
+        DHCPServerUnreachable:           "DHCP_SERVER_UNREACHABLE",
+        InvalidMessageHeader:            "INVALID_MESSAGE_HEADER",
+        SessionExpired:                  "SESSION_EXPIRED",
+        AccessDenied:                    "ACCESS_DENIED",
+        PasswordUpdateRequired:          "PASSWORD_UPDATE_REQUIRED",
+        InvalidCredentials:              "INVALID_CREDENTIALS",
+        UserAccountBlocked:              "USER_ACCOUNT_BLOCKED",
+        UserAccountExpired:              "USER_ACCOUNT_EXPIRED",
+        ConcurrentAccessFromMultipleIPs: "CONCURRENT_ACCESS_FROM_MULTIPLE_IPS",
+        KPIThresholdCrossed:             "KPI_THRESHOLD_CROSSED",
+        ONTUnreachable:                  "ONT_UNREACHABLE",
+        ONTProcessingError:              "ONT_PROCESSING_ERROR",
+        ONTResourceBusy:                 "ONT_RESOURCE_BUSY",
+        ONTMEInstanceExists:             "ONT_ME_INSTANCE_ALREADY_EXISTS",
+        ONTUnknownMEInstance:            "ONT_UNKNOWN_ME_INSTANCE",
+        JoinUnsuccessful:                "JOIN_UNSUCCESSFUL",
+        QueryExpired:                    "QUERY_EXPIRED",
+}
+
+// GrpcToVolthaErrorCodeMap contains mapping of grpc error code coming from OpenOLT-Agent to Voltha error codes.
+var GrpcToVolthaErrorCodeMap = map[codes.Code]NBErrorCode{
+        codes.OK:                 Success,
+        codes.Canceled:           UnsuccessfulOperation,
+        codes.Unknown:            OLTInternalError,
+        codes.InvalidArgument:    InvalidArgument,
+        codes.DeadlineExceeded:   RequestTimeout,
+        codes.NotFound:           ResourceNotFound,
+        codes.AlreadyExists:      ResourceAlreadyExists,
+        codes.PermissionDenied:   UserUnauthorized,
+        codes.ResourceExhausted:  ResourceLimitExceeded,
+        codes.FailedPrecondition: PrerequisiteNotMet,
+        codes.Aborted:            UnsuccessfulOperation,
+        codes.OutOfRange:         InvalidArgument,
+        codes.Unimplemented:      UnsupportedOperation,
+        codes.Internal:           OLTInternalError,
+        codes.Unavailable:        ResourceBusy,
+        codes.DataLoss:           OLTInternalError,
+        codes.Unauthenticated:    UserUnauthorized,
+}
+
+// HTTPStatusCodeToVolthaErrorCodeMap contains mapping of http status code coming from VGC to Voltha error codes.
+var HTTPStatusCodeToVolthaErrorCodeMap = map[int]NBErrorCode{
+        http.StatusOK:                  Success,
+        http.StatusCreated:             Success,
+        http.StatusAccepted:            Success,
+        http.StatusBadRequest:          InvalidPayload,
+        http.StatusConflict:            ResourceInImproperState,
+        http.StatusInternalServerError: VolthaInternalError,
+}
+
+// GetErrorInfo - parses the error details from err structure response from voltha
+// Return statusCode (uint32) - Error code [0 - Success]
+//        status Msg (string) - Error Msg
+func GetErrorInfo(err error) (uint32, string) {
+        var statusCode uint32
+        var statusMsg string
+        if status, _ := status.FromError(err); status != nil {
+                statusCode = uint32(status.Code())
+                statusMsg = status.Message()
+        } else {
+                statusCode = 0
+        }
+        return statusCode, statusMsg
+}
+
diff --git a/internal/pkg/errorcodes/service/errors.go b/internal/pkg/errorcodes/service/errors.go
new file mode 100644
index 0000000..a25278d
--- /dev/null
+++ b/internal/pkg/errorcodes/service/errors.go
@@ -0,0 +1,122 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+// Package service provides constants.
+package service
+
+const (
+	errorCodeStartRange = 1000
+)
+
+const (
+	// VolthaErrorMessageFormat represents the format in which the Voltha accepts the errors.
+	VolthaErrorMessageFormat = "code = %d, desc = %s"
+)
+
+// ErrorCode is Enum of error type
+type ErrorCode int
+
+//ErrorAction is Enum for error action
+type ErrorAction int
+
+const (
+	//ErrOk is returned when request is successful
+	ErrOk ErrorCode = 0
+	//ErrInProgress is returned when operation is in progress
+	ErrInProgress ErrorCode = iota + errorCodeStartRange
+	//ErrInvalidParm is returned when parameter is wrong
+	ErrInvalidParm
+	//ErrResourceUnavailable is returned when no free resources are available
+	ErrResourceUnavailable
+	//ErrAlreadyExists is returned when entry already exists
+	ErrAlreadyExists
+	//ErrNotExists is returned when entry does not exists
+	ErrNotExists
+	//ErrInvalidOperation is returned when invalid operation is performed
+	ErrInvalidOperation
+	//ErrDeviceNotConnected is returned when there is no connection with the target system
+	ErrDeviceNotConnected
+	//ErrTimeout is returned when operation times out
+	ErrTimeout
+	//ErrResourceBusy is returned when resource is busy
+	ErrResourceBusy
+	//ErrInternal is returned when Errors happened internally
+	ErrInternal
+	//ErrIo is returned when there is I/O error
+	ErrIo
+	//ErrMandatoryParmIsMissing is returned when mandatory parameter is missing
+	ErrMandatoryParmIsMissing
+	//ErrBadState is returned when object is in bad state
+	ErrBadState
+	//ErrOnuInternal is returned when ONT internal failure occurs
+	ErrOnuInternal
+	//ErrElanNotCreated is returned when ELAN is not created
+	ErrElanNotCreated
+	//ErrOltInternal is returned when OLT internal failure occurs
+	ErrOltInternal
+)
+
+//ErrorCodeMap converts error code to error description string
+var ErrorCodeMap = map[ErrorCode]string{
+	ErrOk:                     "Success",
+	ErrInProgress:             "Operation is in progress",
+	ErrInvalidParm:            "Invalid parameter",
+	ErrResourceUnavailable:    "No free resource available",
+	ErrAlreadyExists:          "Entry already exists",
+	ErrNotExists:              "Entry does not exists",
+	ErrInvalidOperation:       "Invalid Operation",
+	ErrDeviceNotConnected:     "No connection with the target system",
+	ErrTimeout:                "Operation timed out",
+	ErrResourceBusy:           "Resource Busy",
+	ErrInternal:               "Internal Error",
+	ErrIo:                     "I/O Error",
+	ErrMandatoryParmIsMissing: "Mandatory parameter is missing",
+	ErrBadState:               "Object is in bad state",
+	ErrOnuInternal:            "ONT internal error",
+	ErrElanNotCreated:         "ELAN not created",
+	ErrOltInternal:            "OLT internal error",
+}
+
+const (
+	//Retry is returned if subservice reactivation is required
+	Retry ErrorAction = iota
+	//Quiet is returned if no action has to be taken
+	Quiet
+	//Deactivate is returned if subservice has to be deactivated
+	Deactivate
+	//Invalid is returned when invalid error is received from vgc
+	Invalid
+)
+
+//RetryErrorCodeMap consists of errors that requires service activation retry
+var RetryErrorCodeMap = map[ErrorCode]ErrorAction{
+	ErrOk:                     Quiet,
+	ErrInProgress:             Deactivate,
+	ErrInvalidParm:            Deactivate,
+	ErrResourceUnavailable:    Deactivate,
+	ErrAlreadyExists:          Quiet,
+	ErrNotExists:              Quiet,
+	ErrInvalidOperation:       Deactivate,
+	ErrDeviceNotConnected:     Quiet,
+	ErrTimeout:                Retry,
+	ErrResourceBusy:           Retry,
+	ErrInternal:               Deactivate,
+	ErrIo:                     Retry,
+	ErrMandatoryParmIsMissing: Deactivate,
+	ErrBadState:               Deactivate,
+	ErrOnuInternal:            Retry,
+	ErrElanNotCreated:         Retry,
+	ErrOltInternal:            Deactivate,
+}
diff --git a/internal/pkg/holder/doc.go b/internal/pkg/holder/doc.go
new file mode 100644
index 0000000..0564693
--- /dev/null
+++ b/internal/pkg/holder/doc.go
@@ -0,0 +1,32 @@
+/*
+ *   Copyright 2020-present Open Networking Foundation
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+// The implementation of the open flow agent (vpagent) uses a GRPC connection
+// to VOLTHA accross several implementaton packages including the vpagent and
+// openflow packages. This GRPC connection is shared through packages and type
+// instances via injection.
+//
+// As the vpagent executes within a micro-service environment it is
+// possible that the GRPC connection is reset (re-established). When the connection
+// is re-established, because injection is used, the new value needs to be
+// re-injected across the implementation.
+//
+// To help simply the re-injection or value change scenario a holder for the
+// GRPC connection is established so that the reference to the holder can
+// stay [sic] consistant over the lifetime of the vpagent while the underlying
+// GRPC connection can change without walking the entire runtime structure.
+
+package holder
diff --git a/internal/pkg/holder/holder.go b/internal/pkg/holder/holder.go
new file mode 100644
index 0000000..95c1483
--- /dev/null
+++ b/internal/pkg/holder/holder.go
@@ -0,0 +1,56 @@
+/*
+ *   Copyright 2020-present Open Networking Foundation
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package holder
+
+import (
+	"sync"
+
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+// VolthaServiceClientHolder provides a consistent (voluntarily unmutable) reference
+// point for a mutable value that represents a GRPC service interface to
+// VOLTHA
+type VolthaServiceClientHolder struct {
+	mutex           sync.RWMutex
+	volthaSvcClient voltha.VolthaServiceClient
+}
+
+// VolthaServiceClientReference structure
+type VolthaServiceClientReference struct {
+}
+
+// Clear sets the held value to nil (not set)
+func (h *VolthaServiceClientHolder) Clear() {
+	h.mutex.Lock()
+	defer h.mutex.Unlock()
+	h.volthaSvcClient = nil
+}
+
+// Set assigns the value being held to the specified value
+func (h *VolthaServiceClientHolder) Set(client voltha.VolthaServiceClient) {
+	h.mutex.Lock()
+	defer h.mutex.Unlock()
+	h.volthaSvcClient = client
+}
+
+// Get returns the currently held value
+func (h *VolthaServiceClientHolder) Get() voltha.VolthaServiceClient {
+	h.mutex.RLock()
+	defer h.mutex.RUnlock()
+	return h.volthaSvcClient
+}
diff --git a/internal/pkg/intf/appif.go b/internal/pkg/intf/appif.go
new file mode 100644
index 0000000..76ea4d4
--- /dev/null
+++ b/internal/pkg/intf/appif.go
@@ -0,0 +1,37 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package intf
+
+// App Interface
+type App interface {
+	PortAddInd(string, uint32, string)
+	PortDelInd(string, string)
+	PortUpdateInd(string, string, uint32)
+	PacketInInd(string, string, []byte)
+	PortUpInd(string, string)
+	PortDownInd(string, string)
+	AddDevice(string, string, string)
+	DeviceUpInd(string)
+	DeviceDownInd(string)
+	DelDevice(string)
+	SetRebootFlag(bool)
+	ProcessFlowModResultIndication(FlowStatus)
+	DeviceRebootInd(string, string, string)
+	DeviceDisableInd(string)
+	UpdateMvlanProfilesForDevice(string)
+	TriggerPendingProfileDeleteReq(string)
+	TriggerPendingMigrateServicesReq(string)
+}
diff --git a/internal/pkg/intf/flowerror.go b/internal/pkg/intf/flowerror.go
new file mode 100644
index 0000000..04ca588
--- /dev/null
+++ b/internal/pkg/intf/flowerror.go
@@ -0,0 +1,31 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package intf
+
+import (
+	"voltha-go-controller/internal/pkg/of"
+)
+
+// FlowStatus structure
+type FlowStatus struct {
+	Device         string
+	Cookie         string
+	FlowModType    of.Command
+	Flow           *of.VoltSubFlow
+	Status         uint32
+	Reason         string
+	AdditionalData of.BwAvailDetails
+}
diff --git a/internal/pkg/intf/taskif.go b/internal/pkg/intf/taskif.go
new file mode 100644
index 0000000..5427d4f
--- /dev/null
+++ b/internal/pkg/intf/taskif.go
@@ -0,0 +1,29 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package intf
+
+import (
+	"context"
+	"voltha-go-controller/internal/pkg/tasks"
+)
+
+/*Tasks interface is responsible for creating the tasks
+and executing them as well. For now, it is assumed that
+one task run at a time though interface doesn't force it.*/
+type Tasks interface {
+	AddTask(tasks.Task)
+	Initialize(cxt context.Context)
+}
diff --git a/internal/pkg/intf/vpagent.go b/internal/pkg/intf/vpagent.go
new file mode 100644
index 0000000..f255d13
--- /dev/null
+++ b/internal/pkg/intf/vpagent.go
@@ -0,0 +1,68 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package intf
+
+import (
+	"context"
+
+	"voltha-go-controller/internal/pkg/holder"
+
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+)
+
+// VPClientCfg structure
+type VPClientCfg struct {
+	DeviceID         string
+	SerialNum        string
+	SouthBoundID     string
+	VolthaClient     *holder.VolthaServiceClientHolder
+	PacketOutChannel chan *ofp.PacketOut
+}
+
+// DiscoveryType type
+type DiscoveryType uint8
+
+const (
+	// DeviceDisc constant
+	DeviceDisc DiscoveryType = 0
+	// DeviceReDisc constant
+	DeviceReDisc DiscoveryType = 1
+)
+
+// IVPClient interface
+type IVPClient interface {
+	ChangeEvent(*ofp.ChangeEvent) error
+	PacketIn(*ofp.PacketIn)
+	ConnectInd(cxt context.Context, DiscType DiscoveryType)
+	Stop()
+}
+
+// VPAgent interface
+type VPAgent interface {
+	receiveOltRebootNoti(ctx context.Context)
+	handleOltRebootNoti(ctx context.Context)
+}
+
+// IVPClientAgent interface
+type IVPClientAgent interface {
+	AddNewDevice(cfg *VPClientCfg)
+	DelDevice(id string)
+	IsRebootInProgressForDevice(device string) bool
+	// RebootInd(string, string, string)
+	IsBlockedDevice(string) bool
+	AddBlockedDevices(string)
+	DelBlockedDevices(string)
+}
diff --git a/internal/pkg/of/flows.go b/internal/pkg/of/flows.go
new file mode 100644
index 0000000..2cc84ac
--- /dev/null
+++ b/internal/pkg/of/flows.go
@@ -0,0 +1,1016 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package of
+
+import (
+	"context"
+	"net"
+	"strconv"
+
+	"github.com/google/gopacket/layers"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	//"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+// PbitType type
+type PbitType uint16
+
+// TODO: Port related constants - OF specifies a different value
+// for controller. Need to make sure this is correct
+const (
+	ControllerPort uint32   = 0xfffffffd
+	PbitMatchNone  PbitType = 8
+	PbitMatchAll   PbitType = 0xFF
+)
+
+var logger log.CLogger
+
+var ctx = context.TODO()
+
+// ----------------------------------------------------------
+// Cookie related specifications and utilities
+// ----------------------------------------------------------
+// Though the openflow does not utilize cookies as unique identities of
+// flows, we use cookies as identities in the application. The same
+// may also be used in the VOLTHA if so desired to reduce the complexity
+// of management of flows. In terms of how the cookie is set and is
+// ensured to be unique is:
+// Cookie is a 64 bit value. The first 32 bits are set to the port-id
+// All rules set at the device level are associated with NNI. All other
+// flows, both ingress and egress are associated with the access port.
+// The last 32 bits are used to uniquely identifies flows of a port.
+const (
+	// The flow masks are used to set the MSB of the lower
+	// 32 bits of cookie
+
+	// UsFlowMask constant
+	UsFlowMask uint64 = 0x8000
+	// DsFlowMask constant
+	DsFlowMask uint64 = 0x0000
+
+	// Flow types used to divide the available cookie value range
+	// Each type is allocated 256 flow identities which are plenty
+	// for the known use cases.
+
+	// DhcpArpFlowMask constant
+	DhcpArpFlowMask uint64 = 0x0100
+	// PppoeFlowMask constant
+	PppoeFlowMask uint64 = 0x0100
+	// HsiaFlowMask constant
+	HsiaFlowMask uint64 = 0x0200
+	// DsArpFlowMask constant
+	DsArpFlowMask uint64 = 0x0300
+	// IgmpFlowMask constant
+	IgmpFlowMask uint64 = 0x0400
+	// Dhcpv6FlowMask constant
+	Dhcpv6FlowMask uint64 = 0x0800
+
+	// Flow priorities - Higher the value, higher the priority
+
+	// DhcpFlowPriority constant
+	DhcpFlowPriority uint32 = 5000
+	// ArpFlowPriority constant
+	ArpFlowPriority uint32 = 5000
+	// IgmpFlowPriority constant
+	IgmpFlowPriority uint32 = 5000
+	// McFlowPriority constant
+	McFlowPriority uint32 = 5000
+	// PppoeFlowPriority constant
+	PppoeFlowPriority uint32 = 5000
+	// HsiaFlowPriority constant
+	HsiaFlowPriority uint32 = 100
+)
+
+// CookieSetPort to set port
+func CookieSetPort(cookie uint64, port uint32) uint64 {
+	return cookie | (uint64(port) << 32)
+}
+
+// CookieGetPort to get port
+func CookieGetPort(cookie uint64) uint32 {
+	return uint32(cookie >> 32)
+}
+
+// -------------------------------------------------------
+// The flow match and action related definitions follow
+// -------------------------------------------------------
+// The Ethernet types listed below serve our requirement. We may extend
+// the list as we identify more use cases to be supported.
+
+// EtherType type
+type EtherType uint16
+
+const (
+	// EtherTypeAny constant
+	EtherTypeAny EtherType = 0x0000 // Needs assertion
+	// EtherTypeIpv4 constant
+	EtherTypeIpv4 EtherType = 0x0800
+	// EtherTypeIpv6 constant
+	EtherTypeIpv6 EtherType = 0x86DD
+	// EtherTypePppoeDiscovery constant
+	EtherTypePppoeDiscovery EtherType = 0x8863
+	// EtherTypePppoeSession constant
+	EtherTypePppoeSession EtherType = 0x8864
+	// EtherTypeArp constant
+	EtherTypeArp EtherType = 0x0806
+)
+
+// VLAN related definitions
+// VLANs can take a value between 1 and 4095. VLAN 0 is used to set just
+// the PCP bytes. VLAN 4097 is being used to represent "no VLAN"
+// 4096 is being used to represent "any vlan"
+
+// VlanType type
+type VlanType uint16
+
+const (
+	// VlanAny constant
+	VlanAny VlanType = 0x1000
+	// VlanNone constant
+	VlanNone VlanType = 0x1001
+)
+
+func (vlan *VlanType) String() string {
+	return strconv.Itoa(int(*vlan))
+}
+
+// IP Protocol defintions
+// IP protocol 0xff is reserved and we are using the reserved value to
+// represent that match is not needed.
+
+// IPProtocol type
+type IPProtocol uint8
+
+const (
+	// IPProtocolIgnore constant
+	IPProtocolIgnore IPProtocol = 0xff
+	// IPProtocolTCP constant
+	IPProtocolTCP IPProtocol = 0x06
+	// IPProtocolUDP constant
+	IPProtocolUDP IPProtocol = 0x11
+	// IPProtocolIgmp constant
+	IPProtocolIgmp IPProtocol = 0x02
+	// IPProtocolIcmpv6 constant
+	IPProtocolIcmpv6 IPProtocol = 0x3A
+)
+
+// The following structure is included in each flow which further is
+// used to create a flow. The match structure is used to specify the
+// match rules encoded into the flow.
+
+// Match structure
+type Match struct {
+	InPort        uint32
+	MatchVlan     VlanType
+	SrcMacMatch   bool
+	SrcMacAddr    net.HardwareAddr
+	SrcMacMask    net.HardwareAddr
+	DstMacMatch   bool
+	DstMacAddr    net.HardwareAddr
+	DstMacMask    net.HardwareAddr
+	MatchPbits    bool
+	Pbits         PbitType
+	L3Protocol    EtherType
+	SrcIpv4Match  bool
+	SrcIpv4Addr   net.IP
+	DstIpv4Match  bool
+	DstIpv4Addr   net.IP
+	L4Protocol    IPProtocol
+	SrcPort       uint16
+	DstPort       uint16
+	TableMetadata uint64
+}
+
+// Reset to be used when a Match is created. It sets the values to
+// defaults which results is no match rules at all and thus when
+// applied on a port, match all packets. The match rules must be
+// set before use.
+func (m *Match) Reset() {
+	m.MatchVlan = VlanNone
+	m.SrcMacMatch = false
+	m.DstMacMatch = false
+	m.MatchPbits = false
+	m.L3Protocol = EtherTypeAny
+	m.L4Protocol = IPProtocolIgnore
+	m.SrcPort = 0
+	m.DstPort = 0
+	m.TableMetadata = 0
+}
+
+// SetInPort to set in port
+func (m *Match) SetInPort(port uint32) {
+	m.InPort = port
+}
+
+// SetMatchVlan to set match vlan
+func (m *Match) SetMatchVlan(vlan VlanType) {
+	m.MatchVlan = vlan
+}
+
+// SetPppoeDiscoveryMatch to set L3 protocol
+func (m *Match) SetPppoeDiscoveryMatch() {
+	m.L3Protocol = EtherTypePppoeDiscovery
+}
+
+// SetTableMetadata to set table metadata
+func (m *Match) SetTableMetadata(metadata uint64) {
+	m.TableMetadata = metadata
+}
+
+// SetMatchSrcMac to set source mac address
+func (m *Match) SetMatchSrcMac(mac net.HardwareAddr) {
+	m.SrcMacMatch = true
+	m.SrcMacAddr = mac
+}
+
+// SetMatchDstMac to set destination mac address
+func (m *Match) SetMatchDstMac(mac net.HardwareAddr) {
+	m.DstMacMatch = true
+	m.DstMacAddr = mac
+}
+
+// SetMatchPbit to set pbits
+func (m *Match) SetMatchPbit(pbit PbitType) {
+	m.MatchPbits = true
+	m.Pbits = pbit
+}
+
+// SetMatchSrcIpv4 to set source ipv4 address
+func (m *Match) SetMatchSrcIpv4(ip net.IP) {
+	m.SrcIpv4Match = true
+	m.SrcIpv4Addr = ip
+}
+
+// SetMatchDstIpv4 to set destination ipv4 address
+func (m *Match) SetMatchDstIpv4(ip net.IP) {
+	m.DstIpv4Match = true
+	m.DstIpv4Addr = ip
+}
+
+// SetArpMatch to set L3 protocol as Arp
+func (m *Match) SetArpMatch() {
+	m.L3Protocol = EtherTypeArp
+}
+
+// SetICMPv6Match to set L3 and L4 protocol as IPV6 and ICMPv6
+func (m *Match) SetICMPv6Match() {
+	m.L3Protocol = EtherTypeIpv6
+	m.L4Protocol = IPProtocolIcmpv6
+}
+
+// SetUdpv4Match to set L3 and L4 protocol as IPv4 and UDP
+func (m *Match) SetUdpv4Match() {
+	m.L3Protocol = EtherTypeIpv4
+	m.L4Protocol = IPProtocolUDP
+}
+
+// SetIgmpMatch to set L3 and L4 protocol as IPv4 and Igmp
+func (m *Match) SetIgmpMatch() {
+	m.L3Protocol = EtherTypeIpv4
+	m.L4Protocol = IPProtocolIgmp
+}
+
+// SetUdpv6Match to set L3 and L4 protocol as IPv6 and UDP
+func (m *Match) SetUdpv6Match() {
+	m.L3Protocol = EtherTypeIpv6
+	m.L4Protocol = IPProtocolUDP
+}
+
+// SetIpv4Match to set L3 as IPv4
+func (m *Match) SetIpv4Match() {
+	m.L3Protocol = EtherTypeIpv4
+}
+
+// OutputType type
+type OutputType uint8
+
+const (
+	// OutputTypeDrop constant
+	OutputTypeDrop OutputType = 1
+	// OutputTypeToController constant
+	OutputTypeToController OutputType = 2
+	// OutputTypeToNetwork constant
+	OutputTypeToNetwork OutputType = 3
+	// OutputTypeGoToTable constant
+	OutputTypeGoToTable OutputType = 4
+	// OutputTypeToGroup constant
+	OutputTypeToGroup OutputType = 5
+)
+
+const (
+	// FlowAddSuccess constant
+	FlowAddSuccess = 0
+	// FlowAddFailure constant
+	FlowAddFailure = 1
+	// FlowAddPending constant
+	FlowAddPending = 2
+	// FlowDelPending constant
+	FlowDelPending = 3
+	// FlowDelFailure constant
+	FlowDelFailure = 4
+)
+
+// Action structure
+type Action struct {
+	Output      OutputType
+	PushVlan    []VlanType
+	EtherType   layers.EthernetType
+	SetVlan     VlanType
+	RemoveVlan  int
+	OutPort     uint32
+	GoToTableID uint32
+	Metadata    uint64
+	MeterID     uint32
+	Pcp         PbitType
+}
+
+const (
+	// PbitNone constant
+	PbitNone PbitType = 8
+)
+
+// Reset the action structure
+func (a *Action) Reset() {
+	a.Output = OutputTypeDrop
+	a.PushVlan = make([]VlanType, 0)
+	a.SetVlan = VlanNone
+	a.RemoveVlan = 0
+	a.Metadata = 0
+	a.MeterID = 0
+	a.Pcp = PbitNone
+}
+
+// SetReportToController for set action to report to controller
+func (a *Action) SetReportToController() {
+	a.Output = OutputTypeToController
+	a.OutPort = ControllerPort
+}
+
+// SetPushVlan for set action to push to vlan
+func (a *Action) SetPushVlan(vlan VlanType, etherType layers.EthernetType) {
+	a.PushVlan = append(a.PushVlan, vlan)
+	a.EtherType = etherType
+}
+
+// SetSetVlan to set SetVlan
+func (a *Action) SetSetVlan(vlan VlanType) {
+	a.SetVlan = vlan
+}
+
+// SetPopVlan to set remove vlan counter
+func (a *Action) SetPopVlan() {
+	a.RemoveVlan++
+}
+
+// SetMeterID to set meter id
+func (a *Action) SetMeterID(meterID uint32) {
+	a.MeterID = meterID
+}
+
+// SetWriteMetadata to set metadata
+func (a *Action) SetWriteMetadata(metadata uint64) {
+	a.Metadata = metadata
+}
+
+// SetPcp to set pcp
+func (a *Action) SetPcp(pcp PbitType) {
+	a.Pcp = pcp
+}
+
+// GetWriteMetadata returns metadata
+func (a *Action) GetWriteMetadata() uint64 {
+	return a.Metadata
+}
+
+// SetOutPort to set output port
+func (a *Action) SetOutPort(port uint32) {
+	a.Output = OutputTypeToNetwork
+	a.OutPort = port
+}
+
+// SetOutGroup to set output group
+func (a *Action) SetOutGroup(group uint32) {
+	a.Output = OutputTypeToGroup
+	a.OutPort = group
+}
+
+// SetGoToTable to set GoToTableID
+func (a *Action) SetGoToTable(table uint32) {
+	a.Output = OutputTypeGoToTable
+	a.GoToTableID = table
+}
+
+// VoltSubFlow structure
+type VoltSubFlow struct {
+	Cookie     uint64
+	CookieMask uint64
+	// OldCookie is used in vgc upgrade when there is cookie generation logic change.
+	OldCookie   uint64
+	TableID     uint32
+	Priority    uint32
+	State       uint8
+	ErrorReason string
+	Match
+	Action
+}
+
+// NewVoltSubFlow is constructor for VoltSubFlow
+func NewVoltSubFlow() *VoltSubFlow {
+	var sf VoltSubFlow
+	sf.Match.Reset()
+	sf.Action.Reset()
+	return &sf
+}
+
+// SetTableID to set table id
+func (sf *VoltSubFlow) SetTableID(tableID uint32) {
+	sf.TableID = tableID
+}
+
+// Command type
+type Command uint8
+
+const (
+	// CommandAdd constant
+	CommandAdd Command = 0
+	// CommandDel constant
+	CommandDel Command = 1
+)
+
+// VoltFlow : Definition of a flow
+type VoltFlow struct {
+	Command       Command
+	SubFlows      map[uint64]*VoltSubFlow
+	ForceAction   bool
+	MigrateCookie bool
+	// PortName and PortID to be used for validation of port before flow pushing
+	PortName string
+	PortID   uint32
+}
+
+const (
+	// PrevBwInfo indicates the string returned by core for bandwidth consumed before creating scheduler
+	PrevBwInfo string = "prevBW"
+	// PresentBwInfo indicates the string returned by core for bandwidth consumed after creating scheduler
+	PresentBwInfo string = "presentBW"
+)
+
+// BwAvailDetails consists of bw consumtion details at olt
+type BwAvailDetails struct {
+	PrevBw    string
+	PresentBw string
+}
+
+// -------------------------------------------------------------------
+// OPENFLOW Implementation of flows
+//
+// The flows constructed using the above structures is translated to
+// the VOLTHA OpenFlow GRPC structures. The code below is used to
+// construct the VOLTHA OF GRPC structures.
+const (
+	// DefaultMeterID constant
+	DefaultMeterID uint32 = 0x1
+	// DefaultBufferID constant
+	DefaultBufferID uint32 = 0xffffffff
+	// DefaultOutPort constant
+	DefaultOutPort uint32 = 0xffffffff
+	// DefaultOutGroup constant
+	DefaultOutGroup uint32 = 0xffffffff
+	// DefaultFlags constant
+	DefaultFlags uint32 = 0x1
+)
+
+// NewInportMatch for inport info
+func NewInportMatch(port uint32) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_Port{Port: port}
+	return &entry
+}
+
+// NewTableMetadataMatch for table metadata
+func NewTableMetadataMatch(metadata uint64) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_METADATA
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_TableMetadata{TableMetadata: metadata}
+	return &entry
+}
+
+// NewSrcMacAddrMatch for source mac address info
+func NewSrcMacAddrMatch(addr []byte) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_EthSrc{EthSrc: addr}
+	return &entry
+}
+
+// NewDstMacAddrMatch for destination mac address info
+func NewDstMacAddrMatch(addr []byte) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_EthDst{EthDst: addr}
+	return &entry
+}
+
+// NewVlanMatch for vlan info
+func NewVlanMatch(vlan uint16) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_VlanVid{VlanVid: uint32(vlan&0x0fff + 0x1000)}
+	mf.Mask = &ofp.OfpOxmOfbField_VlanVidMask{VlanVidMask: uint32(0x1000)}
+	return &entry
+}
+
+// NewPcpMatch for pcp info
+func NewPcpMatch(pbits PbitType) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_VlanPcp{VlanPcp: uint32(pbits)}
+	return &entry
+}
+
+// NewEthTypeMatch for eth type info
+func NewEthTypeMatch(l3proto uint16) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_EthType{EthType: uint32(l3proto)}
+	return &entry
+}
+
+// ipv4ToUint to convert ipv4 to uint
+func ipv4ToUint(ip net.IP) uint32 {
+	result := uint32(0)
+	addr := ip.To4()
+	if addr == nil {
+		logger.Warnw(ctx, "Invalid Group Addr", log.Fields{"IP": ip})
+		return 0
+	}
+	result = result + uint32(addr[0])<<24
+	result = result + uint32(addr[1])<<16
+	result = result + uint32(addr[2])<<8
+	result = result + uint32(addr[3])
+	return result
+}
+
+// NewIpv4SrcMatch for ipv4 source address
+func NewIpv4SrcMatch(ip net.IP) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_Ipv4Src{Ipv4Src: ipv4ToUint(ip)}
+	return &entry
+}
+
+// NewIpv4DstMatch for ipv4 destination address
+func NewIpv4DstMatch(ip net.IP) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_Ipv4Dst{Ipv4Dst: ipv4ToUint(ip)}
+	return &entry
+}
+
+// NewIPProtoMatch for ip proto info
+func NewIPProtoMatch(l4proto uint16) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_IpProto{IpProto: uint32(l4proto)}
+	return &entry
+}
+
+// NewUDPSrcMatch for source udp info
+func NewUDPSrcMatch(port uint16) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_UdpSrc{UdpSrc: uint32(port)}
+	return &entry
+}
+
+// NewUDPDstMatch for destination udp info
+func NewUDPDstMatch(port uint16) *ofp.OfpOxmField {
+	var entry ofp.OfpOxmField
+	var mf ofp.OfpOxmOfbField
+	entry.OxmClass = ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC
+	entry.Field = &ofp.OfpOxmField_OfbField{OfbField: &mf}
+	mf.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST
+	mf.HasMask = false
+	mf.Value = &ofp.OfpOxmOfbField_UdpDst{UdpDst: uint32(port)}
+	return &entry
+}
+
+// NewMeterIDInstruction for meter id instructions
+func NewMeterIDInstruction(meterID uint32) *ofp.OfpInstruction {
+	var meter ofp.OfpInstruction
+	meter.Type = uint32(ofp.OfpInstructionType_OFPIT_METER)
+	meter.Data = &ofp.OfpInstruction_Meter{
+		Meter: &ofp.OfpInstructionMeter{
+			MeterId: meterID,
+		},
+	}
+	return &meter
+}
+
+// NewGoToTableInstruction for go to table instructions
+func NewGoToTableInstruction(table uint32) *ofp.OfpInstruction {
+	var gotoTable ofp.OfpInstruction
+	gotoTable.Type = uint32(ofp.OfpInstructionType_OFPIT_GOTO_TABLE)
+	gotoTable.Data = &ofp.OfpInstruction_GotoTable{
+		GotoTable: &ofp.OfpInstructionGotoTable{
+			TableId: table,
+		},
+	}
+	return &gotoTable
+}
+
+// NewPopVlanInstruction for pop vlan instructions
+func NewPopVlanInstruction() *ofp.OfpInstruction {
+	var removeTag ofp.OfpInstruction
+	var actions ofp.OfpInstructionActions
+	removeTag.Type = uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS)
+	removeTag.Data = &ofp.OfpInstruction_Actions{
+		Actions: &actions,
+	}
+	action := flows.PopVlan()
+	actions.Actions = append(actions.Actions, action)
+	return &removeTag
+}
+
+// NewWriteMetadataInstruction for write metadata instructions
+func NewWriteMetadataInstruction(metadata uint64) *ofp.OfpInstruction {
+	var md ofp.OfpInstruction
+	md.Type = uint32(ofp.OfpInstructionType_OFPIT_WRITE_METADATA)
+	md.Data = &ofp.OfpInstruction_WriteMetadata{WriteMetadata: &ofp.OfpInstructionWriteMetadata{Metadata: metadata}}
+	return &md
+}
+
+// NewPopVlanAction for pop vlan action
+func NewPopVlanAction() *ofp.OfpAction {
+	return flows.PopVlan()
+}
+
+// NewPushVlanInstruction for push vlan instructions
+func NewPushVlanInstruction(vlan uint16, etherType uint32) *ofp.OfpInstruction {
+	var addTag ofp.OfpInstruction
+	var actions ofp.OfpInstructionActions
+	addTag.Type = uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS)
+	addTag.Data = &ofp.OfpInstruction_Actions{
+		Actions: &actions,
+	}
+	pushAction := flows.PushVlan(etherType)
+	actions.Actions = append(actions.Actions, pushAction)
+	var setField ofp.OfpOxmOfbField
+	setField.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID
+	setField.HasMask = false
+	setField.Value = &ofp.OfpOxmOfbField_VlanVid{
+		VlanVid: uint32(vlan&0x0fff + 0x1000),
+	}
+	setAction := flows.SetField(&setField)
+	actions.Actions = append(actions.Actions, setAction)
+	return &addTag
+}
+
+// NewPushVlanAction for push vlan action
+func NewPushVlanAction(etherType uint32) *ofp.OfpAction {
+	pushAction := flows.PushVlan(etherType)
+	return pushAction
+}
+
+// NewSetVlanAction for set vlan action
+func NewSetVlanAction(vlan uint16) *ofp.OfpAction {
+	var setField ofp.OfpOxmOfbField
+	setField.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID
+	setField.HasMask = false
+	and := (vlan & 0xfff)
+	or := and + 0x1000
+	v := uint32(vlan&0x0fff + 0x1000)
+	logger.Infow(ctx, "Vlan Construction", log.Fields{"Vlan": vlan, "vlan&0x0fff": and, "OR": or, "final": v})
+	setField.Value = &ofp.OfpOxmOfbField_VlanVid{
+		VlanVid: uint32(vlan&0x0fff + 0x1000),
+	}
+	setAction := flows.SetField(&setField)
+	return setAction
+}
+
+// NewSetPcpAction for set pcap action
+func NewSetPcpAction(pbits PbitType) *ofp.OfpAction {
+	var setField ofp.OfpOxmOfbField
+	setField.Type = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP
+	setField.HasMask = false
+	setField.Value = &ofp.OfpOxmOfbField_VlanPcp{VlanPcp: uint32(pbits)}
+	setAction := flows.SetField(&setField)
+	return setAction
+}
+
+// NewOutputInstruction for output instructions
+func NewOutputInstruction(port uint32) *ofp.OfpInstruction {
+	var outport ofp.OfpInstruction
+	var actions ofp.OfpInstructionActions
+	outport.Type = uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS)
+	outport.Data = &ofp.OfpInstruction_Actions{
+		Actions: &actions,
+	}
+	action := flows.Output(port, 65535)
+	actions.Actions = append(actions.Actions, action)
+	return &outport
+}
+
+// NewOutputAction for output action
+func NewOutputAction(port uint32) *ofp.OfpAction {
+	return flows.Output(port, 65535)
+}
+
+// NewGroupOutputInstruction for group output instructions
+func NewGroupOutputInstruction(group uint32) *ofp.OfpInstruction {
+	var outgroup ofp.OfpInstruction
+	var actions ofp.OfpInstructionActions
+	outgroup.Type = uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS)
+	outgroup.Data = &ofp.OfpInstruction_Actions{
+		Actions: &actions,
+	}
+	action := flows.Group(group)
+	actions.Actions = append(actions.Actions, action)
+	return &outgroup
+}
+
+// NewGroupAction for group action
+func NewGroupAction(group uint32) *ofp.OfpAction {
+	return flows.Group(group)
+}
+
+// CreateMatchAndActions to create match list and action
+func CreateMatchAndActions(f *VoltSubFlow) ([]*ofp.OfpOxmField, []*ofp.OfpInstruction) {
+	logger.Debug(ctx, "Create Match and Action called")
+
+	// Return values declared here
+	var matchList []*ofp.OfpOxmField
+	var instructions []*ofp.OfpInstruction
+
+	// Construct the match rules
+	// Add match in port
+	if f.Match.InPort != 0 {
+		entry := NewInportMatch(uint32(f.Match.InPort))
+		matchList = append(matchList, entry)
+	}
+
+	// Add table metadata match
+	if f.Match.TableMetadata != 0 {
+		entry := NewTableMetadataMatch(uint64(f.Match.TableMetadata))
+		matchList = append(matchList, entry)
+	}
+
+	// Add Src MAC address match
+	if f.SrcMacMatch {
+		entry := NewSrcMacAddrMatch(f.SrcMacAddr)
+		matchList = append(matchList, entry)
+	}
+
+	// Add Src MAC address match
+	if f.DstMacMatch {
+		entry := NewDstMacAddrMatch(f.DstMacAddr)
+		matchList = append(matchList, entry)
+	}
+
+	// Add VLAN tag match
+	if f.MatchVlan != VlanNone {
+		entry := NewVlanMatch(uint16(f.MatchVlan))
+		matchList = append(matchList, entry)
+	}
+
+	if f.MatchPbits {
+		entry := NewPcpMatch(f.Pbits)
+		matchList = append(matchList, entry)
+	}
+
+	// Add EtherType match
+	if f.L3Protocol != EtherTypeAny {
+		entry := NewEthTypeMatch(uint16(f.L3Protocol))
+		matchList = append(matchList, entry)
+	}
+
+	// Add the Src IPv4 addr match
+	if f.SrcIpv4Match {
+		entry := NewIpv4SrcMatch(f.SrcIpv4Addr)
+		matchList = append(matchList, entry)
+	}
+
+	// Add the Dst IPv4 addr match
+	if f.DstIpv4Match {
+		entry := NewIpv4DstMatch(f.DstIpv4Addr)
+		matchList = append(matchList, entry)
+	}
+
+	// Add IP protocol match
+	if f.L4Protocol != IPProtocolIgnore {
+		entry := NewIPProtoMatch(uint16(f.L4Protocol))
+		matchList = append(matchList, entry)
+	}
+
+	// Add UDP Source port match
+	if f.SrcPort != 0 {
+		entry := NewUDPSrcMatch(uint16(f.SrcPort))
+		matchList = append(matchList, entry)
+	}
+
+	// Add UDP Dest port match
+	if f.DstPort != 0 {
+		entry := NewUDPDstMatch(uint16(f.DstPort))
+		matchList = append(matchList, entry)
+	}
+
+	// Construct the instructions
+	// Add a GOTO table action
+	if f.Output == OutputTypeGoToTable {
+		instruction := NewGoToTableInstruction(f.GoToTableID)
+		instructions = append(instructions, instruction)
+	}
+
+	// Add the meter instruction
+	if f.MeterID != 0 {
+		instruction := NewMeterIDInstruction(f.MeterID)
+		instructions = append(instructions, instruction)
+	}
+
+	// Add the metadata instruction
+	if f.Metadata != 0 {
+		instruction := NewWriteMetadataInstruction(f.Metadata)
+		instructions = append(instructions, instruction)
+	}
+
+	// The below are all apply actions. All of these could be combined into
+	// a single instruction.
+	{
+		var instruction ofp.OfpInstruction
+		var actions ofp.OfpInstructionActions
+		instruction.Type = uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS)
+		instruction.Data = &ofp.OfpInstruction_Actions{
+			Actions: &actions,
+		}
+
+		// Apply action of popping the VLAN
+		if f.RemoveVlan != 0 {
+			for i := 0; i < f.RemoveVlan; i++ {
+				action := NewPopVlanAction()
+				actions.Actions = append(actions.Actions, action)
+			}
+		}
+
+		if f.SetVlan != VlanNone {
+			action := NewSetVlanAction(uint16(f.SetVlan))
+			actions.Actions = append(actions.Actions, action)
+		}
+
+		if f.Pcp != PbitNone {
+			action := NewSetPcpAction(f.Pcp)
+			actions.Actions = append(actions.Actions, action)
+		}
+
+		// Add the VLAN PUSH
+		if len(f.PushVlan) != 0 {
+			action := NewPushVlanAction(uint32(f.EtherType))
+			actions.Actions = append(actions.Actions, action)
+			for _, vlan := range f.PushVlan {
+				action = NewSetVlanAction(uint16(vlan))
+				actions.Actions = append(actions.Actions, action)
+			}
+		}
+
+		if f.Action.Output == OutputTypeToController {
+			action := NewOutputAction(0xfffffffd)
+			actions.Actions = append(actions.Actions, action)
+		} else if f.Action.Output == OutputTypeToNetwork {
+			action := NewOutputAction(f.OutPort)
+			actions.Actions = append(actions.Actions, action)
+		} else if f.Action.Output == OutputTypeToGroup {
+			action := NewGroupAction(f.OutPort)
+			actions.Actions = append(actions.Actions, action)
+		}
+		instructions = append(instructions, &instruction)
+	}
+
+	return matchList, instructions
+}
+
+// CreateFlow to create flow
+func CreateFlow(device string, command ofp.OfpFlowModCommand, matches []*ofp.OfpOxmField,
+	instructions []*ofp.OfpInstruction, sf *VoltSubFlow) *ofp.FlowTableUpdate {
+	flowUpdate := ofp.FlowTableUpdate{
+		Id: device,
+		FlowMod: &ofp.OfpFlowMod{
+			Cookie:      sf.Cookie,
+			CookieMask:  sf.CookieMask,
+			TableId:     sf.TableID,
+			Command:     command,
+			IdleTimeout: uint32(0),
+			HardTimeout: uint32(0),
+			Priority:    sf.Priority,
+			BufferId:    DefaultBufferID,
+			OutPort:     DefaultOutPort,
+			OutGroup:    DefaultOutGroup,
+			Flags:       DefaultFlags,
+			Match: &ofp.OfpMatch{
+				Type:      ofp.OfpMatchType_OFPMT_OXM,
+				OxmFields: matches,
+			},
+
+			Instructions: instructions,
+		},
+	}
+	return &flowUpdate
+}
+
+// Processing logic for the VOLT flows. The VOLT flows are different from
+// the normal openflows. Each VOLT flow may break into multiple flows.
+// The order of processing:
+// 1. If the flow has to match more than one VLAN tag, it is broken into
+//    more than one flow.
+// 2. When more than one flow is creatd, the higher layer processing is
+//    broken into the second flow. The first flow includes only the
+//    the processing of first VLAN tag.
+// 3. The a sinle flow is created, the first flow has all the match criteria
+//    and action.
+
+// ProcessVoltFlow to process volt flow
+func ProcessVoltFlow(device string, operation Command, subFlow map[uint64]*VoltSubFlow) []*ofp.FlowTableUpdate {
+	var flows []*ofp.FlowTableUpdate
+	var command ofp.OfpFlowModCommand
+	if operation == CommandAdd {
+		command = ofp.OfpFlowModCommand_OFPFC_ADD
+	} else {
+		command = ofp.OfpFlowModCommand_OFPFC_DELETE_STRICT
+	}
+	for _, sf := range subFlow {
+		logger.Debugw(ctx, "Flow Construction for", log.Fields{"Flow": sf})
+		match, instruction := CreateMatchAndActions(sf)
+		flow := CreateFlow(device, command, match, instruction, sf)
+		logger.Debugw(ctx, "Flow Constructed", log.Fields{"Flow": flow})
+		flows = append(flows, flow)
+	}
+	return flows
+}
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/internal/pkg/of/group.go b/internal/pkg/of/group.go
new file mode 100644
index 0000000..0a5863a
--- /dev/null
+++ b/internal/pkg/of/group.go
@@ -0,0 +1,148 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package of
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+//	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+// The commands on groups avialable. Add is not expected to be used.
+// The mod is used for both create and update. The delete is used to
+// delete the group
+
+// GroupCommand type
+type GroupCommand ofp.OfpGroupModCommand
+
+const (
+	// GroupCommandAdd constant
+	GroupCommandAdd GroupCommand = 0
+	// GroupCommandMod constant
+	GroupCommandMod GroupCommand = 1
+	// GroupCommandDel constant
+	GroupCommandDel GroupCommand = 2
+)
+
+const (
+	// GroupOperSuccess constant
+	GroupOperSuccess = 0
+	// GroupOperFailure constant
+	GroupOperFailure = 1
+	// GroupOperPending constant
+	GroupOperPending = 2
+)
+
+// The group modification record to be used by the controller
+// to create a group. This is prepared by application and passed
+// on to the controller
+
+// Group structure
+type Group struct {
+	Device           string
+	Command          GroupCommand `json:"-"`
+	GroupID          uint32
+	Buckets          []uint32
+	SetVlan          VlanType
+	IsPonVlanPresent bool
+	State            uint8
+	ErrorReason      string
+	ForceAction      bool
+}
+
+// CreateGroupTableUpdate creates the logical group flow table update
+// This is used by controller for building the final outgoing
+// structure towards the VOLTHA
+func CreateGroupTableUpdate(g *Group) *ofp.FlowGroupTableUpdate {
+	logger.Debugw(ctx, "Group Construction", log.Fields{"Group": g})
+	groupUpdate := &ofp.FlowGroupTableUpdate{
+		Id: g.Device,
+		GroupMod: &ofp.OfpGroupMod{
+			Command: ofp.OfpGroupModCommand(g.Command),
+			Type:    ofp.OfpGroupType_OFPGT_ALL,
+			GroupId: g.GroupID,
+		},
+	}
+	logger.Debugw(ctx, "Adding Receivers", log.Fields{"Num": len(g.Buckets)})
+
+	// Since OLT doesnt support setvlan action during update, adding setVlan action
+	// during group creation itself even when bucketlist is empty
+	if len(g.Buckets) == 0 && g.IsPonVlanPresent {
+		bucket := &ofp.OfpBucket{}
+		bucket.Weight = 0
+		bucket.Actions = []*ofp.OfpAction{
+			{
+				Type: ofp.OfpActionType_OFPAT_SET_FIELD,
+				Action: &ofp.OfpAction_SetField{
+					SetField: &ofp.OfpActionSetField{
+						Field: &ofp.OfpOxmField{
+							Field: &ofp.OfpOxmField_OfbField{
+								OfbField: &ofp.OfpOxmOfbField{
+									Type: ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID,
+									Value: &ofp.OfpOxmOfbField_VlanVid{
+										VlanVid: uint32(g.SetVlan),
+									},
+								},
+							},
+						},
+					},
+				},
+			},
+		}
+		groupUpdate.GroupMod.Buckets = append(groupUpdate.GroupMod.Buckets, bucket)
+	}
+
+	for _, pon := range g.Buckets {
+		bucket := &ofp.OfpBucket{}
+		bucket.Weight = 0
+		bucket.Actions = []*ofp.OfpAction{
+			{
+				Type: ofp.OfpActionType_OFPAT_OUTPUT,
+				Action: &ofp.OfpAction_Output{
+					Output: &ofp.OfpActionOutput{
+						Port:   pon,
+						MaxLen: 65535,
+					},
+				},
+			},
+		}
+		if g.IsPonVlanPresent {
+			setVlanAction := &ofp.OfpAction{
+
+				Type: ofp.OfpActionType_OFPAT_SET_FIELD,
+				Action: &ofp.OfpAction_SetField{
+					SetField: &ofp.OfpActionSetField{
+						Field: &ofp.OfpOxmField{
+							Field: &ofp.OfpOxmField_OfbField{
+								OfbField: &ofp.OfpOxmOfbField{
+									Type: ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID,
+									Value: &ofp.OfpOxmOfbField_VlanVid{
+										VlanVid: uint32(g.SetVlan),
+									},
+								},
+							},
+						},
+					},
+				},
+			}
+			bucket.Actions = append(bucket.Actions, setVlanAction)
+		}
+		groupUpdate.GroupMod.Buckets = append(groupUpdate.GroupMod.Buckets, bucket)
+	}
+
+	logger.Debugw(ctx, "Group Constructed", log.Fields{"Group": groupUpdate})
+	return groupUpdate
+}
diff --git a/internal/pkg/of/meter.go b/internal/pkg/of/meter.go
new file mode 100644
index 0000000..dc3be2f
--- /dev/null
+++ b/internal/pkg/of/meter.go
@@ -0,0 +1,99 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package of
+
+import (
+	// "github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	// "github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+// MeterCommand :  Meters and bands as stored by VOLT application
+type MeterCommand uint32
+
+const (
+	// MeterCommandAdd constant
+	MeterCommandAdd MeterCommand = 1
+	// MeterCommandDel constant
+	MeterCommandDel MeterCommand = 2
+)
+
+const (
+	// MeterOperSuccess constant
+	MeterOperSuccess = 0
+	// MeterOperFailure constant
+	MeterOperFailure = 1
+	// MeterOperPending constant
+	MeterOperPending = 2
+)
+
+// Band structure
+type Band struct {
+	Type      uint32
+	Rate      uint32
+	BurstSize uint32
+}
+
+// Meter structure
+type Meter struct {
+	ID          uint32
+	Bands       []Band
+	State       uint8
+	ErrorReason string
+}
+
+// NewMeter is constructor for Meter
+func NewMeter(id uint32) *Meter {
+	var vm Meter
+	vm.ID = id
+	return &vm
+}
+
+// AddBand to add band info to meter
+func (vm *Meter) AddBand(rate uint32, bs uint32) {
+	vb := Band{Rate: rate, BurstSize: bs}
+	vm.Bands = append(vm.Bands, vb)
+}
+
+// MeterUpdate for conversion of VOLT to OF for meters and bands
+func MeterUpdate(deviceID string, c MeterCommand, m *Meter) (*ofp.MeterModUpdate, error) {
+	mmu := &ofp.MeterModUpdate{Id: deviceID}
+	mmu.MeterMod = &ofp.OfpMeterMod{
+		MeterId: m.ID,
+	}
+	if c == MeterCommandAdd {
+		mmu.MeterMod.Command = ofp.OfpMeterModCommand_OFPMC_ADD
+		mmu.MeterMod.Flags = 5
+		for _, b := range m.Bands {
+			AddBand(mmu, b)
+		}
+	} else {
+		mmu.MeterMod.Command = ofp.OfpMeterModCommand_OFPMC_DELETE
+	}
+	return mmu, nil
+}
+
+// AddBand to add band info
+func AddBand(mmu *ofp.MeterModUpdate, b Band) {
+	band := &ofp.OfpMeterBandHeader{}
+	band.Type = ofp.OfpMeterBandType_OFPMBT_DROP
+	band.Rate = b.Rate
+	band.BurstSize = b.BurstSize
+	// band.Data = &ofp.OfpMeterBandHeader_Drop{
+	// 	Drop: &ofp.OfpMeterBandDrop{},
+	// }
+	mmu.MeterMod.Bands = append(mmu.MeterMod.Bands, band)
+}
diff --git a/internal/pkg/tasks/task_intf.go b/internal/pkg/tasks/task_intf.go
new file mode 100644
index 0000000..bd9b256
--- /dev/null
+++ b/internal/pkg/tasks/task_intf.go
@@ -0,0 +1,33 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package tasks
+
+import (
+	"context"
+)
+
+// Each task must support this interface to be exercised
+// by the implementation to execute the tasks similarly
+// across all tasks
+
+// Task interface
+type Task interface {
+	TaskID() uint8
+	Name() string
+	Timestamp() string
+	Start(context.Context, uint8) error
+	Stop()
+}
diff --git a/internal/pkg/tasks/tasks.go b/internal/pkg/tasks/tasks.go
new file mode 100644
index 0000000..414c4f2
--- /dev/null
+++ b/internal/pkg/tasks/tasks.go
@@ -0,0 +1,309 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package tasks
+
+import (
+	"context"
+	"errors"
+	"sync"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+var (
+	// ErrCxtCancelError error
+	ErrCxtCancelError = errors.New("Context Cancelled")
+	// ErrTaskCancelError error
+	ErrTaskCancelError = errors.New("Task Cancelled")
+
+	ctx = context.TODO()
+)
+
+// TaskSet implements a set of dependent tasks into a single unit. The
+// tasks are added in the order they are expected to be executed. If any
+// of the tasks fails, the remaining tasks are not executed.
+
+// TaskSet structure
+type TaskSet struct {
+	name      string
+	taskID    uint8
+	timestamp string
+	queued    []Task
+}
+
+// NewTaskSet is constructor for TaskSet
+func NewTaskSet(name string) *TaskSet {
+	var ts TaskSet
+	ts.name = name
+	tstamp := (time.Now()).Format(time.RFC3339Nano)
+	ts.timestamp = tstamp
+	return &ts
+}
+
+// Name to return name of the task
+func (ts *TaskSet) Name() string {
+	return ts.name
+}
+
+// TaskID to return task id of the task
+func (ts *TaskSet) TaskID() uint8 {
+	return ts.taskID
+}
+
+// Timestamp to return timestamp for the task
+func (ts *TaskSet) Timestamp() string {
+	return ts.timestamp
+}
+
+// AddTask to add task
+func (ts *TaskSet) AddTask(task Task) {
+	logger.Debugw(ctx, "Adding Task to TaskSet", log.Fields{"SetName": ts.name, "TaskName": task.Name()})
+	ts.queued = append(ts.queued, task)
+}
+
+// Start to start the task
+func (ts *TaskSet) Start(ctx context.Context, taskID uint8) error {
+	logger.Debug(ctx, "Starting Execution TaskSet", log.Fields{"SetName": ts.name})
+	ts.taskID = taskID
+	for len(ts.queued) != 0 {
+		task := ts.queued[0]
+		logger.Infow(ctx, "Starting Execution of task", log.Fields{"TaskName": task.Name()})
+		err := task.Start(ctx, ts.taskID)
+		if err != nil {
+			return err
+		}
+		task = ts.popTask()
+		logger.Infow(ctx, "Execution of task completed", log.Fields{"TaskName": task.Name()})
+	}
+	logger.Debug(ctx, "Exiting Execution of TaskSet")
+	return nil
+}
+
+// popTask is used internally to remove the task that is
+// is just completed.
+func (ts *TaskSet) popTask() Task {
+	var task Task
+	task, ts.queued = ts.queued[0], ts.queued[1:]
+	return task
+}
+
+// Stop is used internally to remove the task that is
+// is just completed.
+func (ts *TaskSet) Stop() {
+	var task Task
+	// Stop all the tasks and clean up
+	for size := len(ts.queued); size > 0; size = len(ts.queued) {
+		// Pop out the first task and clean up resources
+		task, ts.queued = ts.queued[0], ts.queued[1:]
+		task.Stop()
+	}
+}
+
+//***************************************************************************
+// Task Execution Environment
+// ------------------------------
+// The section below helps create an execution environment for tasks
+// of a single ONU. Addition and in sequence execution of tasks is
+// the main goal.
+
+// queued - holds tasks yet to be executed and the current in progress
+// taskID - This variable is used to generate unique task id for each task
+// currentTask - This holds the value of task being executed
+// timout - This variable sets the timeout value for all of the messages
+// stop - This provides a way of stopping the execution of next task
+
+// Tasks structure
+type Tasks struct {
+	queued      []Task
+	taskID      uint8
+	stop        bool
+	totalTasks  uint16
+	failedTasks uint16
+	lock        sync.RWMutex
+	ctx         context.Context
+}
+
+// NewTasks is constructor for Tasks
+func NewTasks(ctx context.Context) *Tasks {
+	var ts Tasks
+	ts.taskID = 0xff
+	ts.stop = false
+	ts.queued = []Task{}
+	ts.totalTasks = 0
+	ts.failedTasks = 0
+	ts.ctx = ctx
+	return &ts
+}
+
+// Initialize is used to initialize the embedded tasks structure within
+// each ONU.
+func (ts *Tasks) Initialize(ctx context.Context) {
+
+	//Send signal to stop any task which are being executed
+	ts.StopAll()
+	ts.taskID = 0xff
+	ts.ctx = ctx
+}
+
+// CheckAndInitialize is used to initialize the embedded tasks structure within
+// NNI and resets taskID only when there are no pending tasks
+func (ts *Tasks) CheckAndInitialize(ctx context.Context) {
+
+	ts.lock.Lock()
+	logger.Infow(ctx, "Queued Tasks", log.Fields{"Count": len(ts.queued)})
+	if len(ts.queued) == 0 {
+		ts.lock.Unlock()
+		ts.Initialize(ctx)
+		return
+	}
+	ts.ctx = ctx
+	ts.lock.Unlock()
+}
+
+// getNewTaskId generates a unique task-id for each new task. The
+// transaction-ids are generated for the task-ids.
+func (ts *Tasks) getNewTaskID() uint8 {
+	ts.taskID++
+	return ts.taskID
+}
+
+// GetContext to get context of the task
+func (ts *Tasks) GetContext() context.Context {
+	return ts.ctx
+}
+
+// AddTask adds a task and executes it if there is no task
+// pending execution. The execution happens on a seperate thread.
+// The tasks are maintained per ONU. This structure is instantiated
+// one per ONU
+func (ts *Tasks) AddTask(task Task) {
+	ts.lock.Lock()
+	defer ts.lock.Unlock()
+
+	// logger.Infow(ctx, "Adding Task", log.Fields{"TaskName": task.Name()})
+	ts.queued = append(ts.queued, task)
+	if ts.queued[0] == task {
+		go ts.executeTasks()
+	}
+}
+
+// TotalTasks returns the total number of tasks completed by the
+// the execution of the tasks.
+func (ts *Tasks) TotalTasks() uint16 {
+	return ts.totalTasks
+}
+
+// StopAll stops the execution of the tasks and cleans up
+// everything associated with the tasks
+func (ts *Tasks) StopAll() {
+	ts.lock.Lock()
+	defer ts.lock.Unlock()
+
+	ts.stop = true
+	logger.Infow(ctx, "Stopping all tasks in queue", log.Fields{"TaskCount": len(ts.queued)})
+
+	if len(ts.queued) > 0 {
+		ts.queued = ts.queued[:1]
+		logger.Warnw(ctx, "Skipping Current Task", log.Fields{"Task": ts.queued[0].Name()})
+	}
+	ts.stop = false
+}
+
+// popTask is used internally to remove the task that is
+// is just completed.
+func (ts *Tasks) popTask() (Task, int) {
+	ts.lock.Lock()
+	defer ts.lock.Unlock()
+
+	var task Task
+	queueLen := len(ts.queued)
+	if queueLen > 0 {
+		task = ts.queued[0]
+		ts.queued = append(ts.queued[:0], ts.queued[0+1:]...)
+	} else {
+		logger.Errorw(ctx, "Trying to remove task from empty Task List", log.Fields{"#task ": queueLen})
+	}
+
+	return task, len(ts.queued)
+}
+
+// NumPendingTasks returns the count of tasks that are either in progress or
+// yet to be executed. The first in the list is the in progress
+// task.
+func (ts *Tasks) NumPendingTasks() uint16 {
+	return uint16(len(ts.queued))
+}
+
+// GetTaskList returns the list of tasks that are either in progress or
+// yet to be executed. The first in the list is the in progress
+// task.
+func (ts *Tasks) GetTaskList() []Task {
+	taskList := []Task{}
+	return append(taskList, ts.queued...)
+}
+
+// CurrentTask returns the task that is currently running. This can be
+// used for verifying upon unforseen failures for debugging from
+// with the code
+func (ts *Tasks) CurrentTask() Task {
+	return ts.queued[0]
+}
+
+// executeTasks executes the pending tasks one by one. The tasks are attempted
+// one after another to avoid two tasks simultaneously operating on the
+// same ONU.
+func (ts *Tasks) executeTasks() {
+	// logger.Debug(ctx, "Starting Execution of tasks")
+	for (len(ts.queued) != 0) && (!ts.stop) {
+		task := ts.queued[0]
+		taskID := ts.getNewTaskID()
+		// logger.Infow(ctx, "Starting Execution of task", log.Fields{"TaskName": task.Name()})
+		ts.totalTasks++
+
+		err := task.Start(ts.ctx, taskID)
+		if err == ErrTaskCancelError {
+			logger.Warnw(ctx, "Previous task cancelled. Exiting current task queue execution thread", log.Fields{"TaskCount": len(ts.queued)})
+			return
+		}
+		_, pending := ts.popTask()
+
+		if err != nil {
+			ts.failedTasks++
+		}
+		if err == ErrCxtCancelError {
+			// TODO - This needs correction
+			ts.StopAll()
+			return
+		}
+
+		if pending == 0 {
+			break
+		}
+	}
+	// logger.Debug(ctx, "Exiting Execution of tasks")
+}
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/internal/pkg/types/common.go b/internal/pkg/types/common.go
new file mode 100755
index 0000000..8dbeb0b
--- /dev/null
+++ b/internal/pkg/types/common.go
@@ -0,0 +1,55 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+// Package common provides constants.
+package common
+import (
+	"errors"
+)
+
+//AdminState represents Status of an VLAN:ENABLE/DISABLE
+type AdminState string
+
+var (
+	// ErrEntryNotFound is the error when the key doesn't exist in the KVStore
+	ErrEntryNotFound = errors.New("Entry not found")
+)
+// DeviceState refers to the state of device
+type DeviceState string
+
+// Device State constants
+const (
+	DeviceStateDOWN DeviceState = "DOWN"
+	DeviceStateUP   DeviceState = "UP"
+)
+
+// Status represents the status of the request sent to the device manager.
+type Status string
+
+//LogLevel  represents the type of the OLT's LOG
+type LogLevel int
+
+const (
+        // CRITICAL represents log level type of the OLT.
+        CRITICAL LogLevel = iota
+        // ERROR represents log level type of the OLT.
+        ERROR
+        // WARNING represents log level type of the OLT.
+        WARNING
+        // INFO represents log level type of the OLT.
+        INFO
+        // DEBUG represents log level type of the OLT.
+        DEBUG
+)
diff --git a/internal/pkg/types/multicast.go b/internal/pkg/types/multicast.go
new file mode 100755
index 0000000..6b6aac8
--- /dev/null
+++ b/internal/pkg/types/multicast.go
@@ -0,0 +1,104 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package common
+
+// IGMPConfig identifies the IGMP Configuration parameters.
+type IGMPConfig struct {
+	// ProfileID represents IGMP profile ID
+	ProfileID string `json:"ProfileID"`
+	// ProfileName represents IGMP profile Name
+	ProfileName string `json:"ProfileName"`
+	// UnsolicitedTimeOut represents unsolicited timeout.
+	UnsolicitedTimeOut int `json:"UnsolicitedTimeOut"`
+	// MaxResp represents IGMP max response time.
+	MaxResp int `json:"MaxResp"`
+	// KeepAliveInterval represents IGMP keep alive interval.
+	KeepAliveInterval int `json:"KeepAliveInterval"`
+	// KeepAliveCount represents IGMP keep alive count.
+	KeepAliveCount int `json:"KeepAliveCount"`
+	// LastQueryInterval represents IGMP last query interval.
+	LastQueryInterval int `json:"LastQueryInterval"`
+	// LastQueryCount represents IGMP last query count.
+	LastQueryCount int `json:"LastQueryCount"`
+	// FastLeave represents IGMP fast leave enabled or not.
+	FastLeave *bool `json:"FastLeave"`
+	// PeriodicQuery represents IGMP period query interval.
+	PeriodicQuery *bool `json:"PeriodicQuery"`
+	// IgmpCos represents IGMP COS value(0-7).
+	IgmpCos int `json:"IgmpCos"`
+	// WithRAUpLink represents IGMP RA uplink.
+	WithRAUpLink *bool `json:"withRAUpLink"`
+	// WithRADownLink represents IGMP RA downlink.
+	WithRADownLink *bool `json:"withRADownLink"`
+	// IgmpVerToServer represents IGMP version.
+	IgmpVerToServer string `json:"igmpVerToServer"`
+	// IgmpSourceIP represents IGMP src ip.
+	IgmpSourceIP string `json:"igmpSourceIp"`
+}
+
+//MulticastSrcListMode represents mode of source list
+type MulticastSrcListMode string
+
+const (
+	//Include refers to MulticastSrcListMode as include
+	Include MulticastSrcListMode = "include"
+	//Exclude refers to MulticastSrcListMode as exclude
+	Exclude MulticastSrcListMode = "exclude"
+	// StaticGroup refes to the static group name
+	StaticGroup string = "static"
+	// IsStaticYes refes to the static flag value yes
+	IsStaticYes string = "yes"
+	// IsStaticNo refes to the static flag value no
+	IsStaticNo string = "no"
+)
+
+// MulticastGroupProxy identifies source specific multicast(SSM) config.
+type MulticastGroupProxy struct {
+	// Mode represents source list include/exclude
+	Mode MulticastSrcListMode `json:"Mode"`
+	// SourceList represents list of multicast server IP addresses.
+	SourceList []string `json:"SourceList"`
+	// IsStatic flag indicating if the group is a "static" group
+	IsStatic string `json:"IsStatic,omitempty"`
+}
+
+// MVLANProfile identifies the MVLAN profile.
+type MVLANProfile struct {
+	// VLANID represents the Multicast VLAN ID.
+	VLANID int `json:"VLANID"`
+	// ProfileID represents Multicast profile ID
+	ProfileID string `json:"ProfileID"`
+	// ProfileName represents Multicast profile Name
+	ProfileName string `json:"ProfileName"`
+	// PonVLAN represents the vlan, where mcast traffic will be translated at OLT
+	PonVLAN int `json:"PonVLAN"`
+	// Groups represents the MVLAN group information. Key will be group name and value as array of multicast channel IPs.
+	Groups map[string][]string `json:"Groups"`
+	// Proxy represents multicast group proxy info. Key will be group name and value as proxy info
+	Proxy map[string]MulticastGroupProxy `json:"Proxy"`
+	//IsChannelBasedGroup represents if the group is channel based
+	IsChannelBasedGroup bool `json:"IsChannelBasedGroup"`
+	// ActiveIgmpChannelsPerSubscriber represents maximum igmp channels per subscriber can use
+	// Default : 3
+	ActiveIgmpChannelsPerSubscriber int `json:"ActiveIgmpChannelsPerSubscriber"`
+}
+
+// McastConfig the structure for multicast config
+type McastConfig struct {
+	MVLANProfileID string
+	IGMPProfileID  string
+	IGMPSrcIP      string
+}
diff --git a/internal/pkg/util/concurrentmap.go b/internal/pkg/util/concurrentmap.go
new file mode 100644
index 0000000..f16bb29
--- /dev/null
+++ b/internal/pkg/util/concurrentmap.go
@@ -0,0 +1,87 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package util
+
+import (
+	"sync"
+
+	"go.uber.org/atomic"
+)
+
+// ConcurrentMap implements a wrapper on top of SyncMap so that the count is also maintained
+type ConcurrentMap struct {
+	syncMap sync.Map
+	count   *atomic.Uint64
+	MapLock sync.RWMutex
+}
+
+// NewConcurrentMap - Initializes new ConcurentMap Object
+func NewConcurrentMap() *ConcurrentMap {
+	var cm ConcurrentMap
+	cm.count = atomic.NewUint64(0)
+	return &cm
+}
+
+// Get - Gets return the value store in the sync map
+//If value is present, the result will be true else false
+func (cm *ConcurrentMap) Get(key interface{}) (value interface{}, result bool) {
+	return cm.syncMap.Load(key)
+}
+
+// Set - Store the value in sync map against the key provided
+func (cm *ConcurrentMap) Set(key, value interface{}) {
+	if cm.count == nil {
+		cm.count = atomic.NewUint64(0)
+	}
+	_, exists := cm.syncMap.Load(key)
+	cm.syncMap.Store(key, value)
+	if !exists {
+		cm.count.Inc()
+	}
+}
+
+// Remove - Removes the key-value pair from the sync map
+func (cm *ConcurrentMap) Remove(key interface{}) bool {
+
+	if _, ok := cm.syncMap.Load(key); ok {
+		cm.syncMap.Delete(key)
+		cm.count.Dec()
+		return true
+	}
+	return false
+}
+
+// Range calls f sequentially for each key and value present in the sync map.
+// If f returns false, range stops the iteration.
+//
+// Range does not necessarily correspond to any consistent snapshot of the Sync Map's
+// contents: no key will be visited more than once, but if the value for any key
+// is stored or deleted concurrently, Range may reflect any mapping for that key
+// from any point during the Range call.
+//
+// Range may be O(N) with the number of elements in the sync map even if f returns
+// false after a constant number of calls.
+func (cm *ConcurrentMap) Range(f func(key, value interface{}) bool) {
+	cm.syncMap.Range(f)
+}
+
+// Length - return the no of entries present in the map
+func (cm *ConcurrentMap) Length() uint64 {
+	if cm.count == nil {
+		return 0
+	}
+	return cm.count.Load()
+}
diff --git a/internal/pkg/util/envutils/envutils.go b/internal/pkg/util/envutils/envutils.go
new file mode 100644
index 0000000..2c7d2ca
--- /dev/null
+++ b/internal/pkg/util/envutils/envutils.go
@@ -0,0 +1,123 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+// Package envutils provides the env parsing utility functions
+package envutils
+
+import (
+	"fmt"
+	"os"
+	"strconv"
+)
+
+// common constants
+const (
+	// common environment variables
+
+	KafkaAdapterHost     = "KAFKA_ADAPTER_HOST"
+	KafkaAdapterPort     = "KAFKA_ADAPTER_PORT"
+	KafkaClusterHost     = "KAFKA_CLUSTER_HOST"
+	KafkaClusterPort     = "KAFKA_CLUSTER_PORT"
+	KvStoreType          = "KV_STORE_TYPE"
+	KvStoreTimeout       = "KV_STORE_TIMEOUT"
+	KvStoreHost          = "KV_STORE_HOST"
+	KvStorePort          = "KV_STORE_PORT"
+	AdapterTopic         = "ADAPTER_TOPIC"
+	CoreTopic            = "CORE_TOPIC"
+	EventTopic           = "EVENT_TOPIC"
+	LogLevel             = "LOG_LEVEL"
+	OnuNumber            = "ONU_NUMBER"
+	Banner               = "BANNER"
+	DisplayVersionOnly   = "DISPLAY_VERSION_ONLY"
+	ProbeHost            = "PROBE_HOST"
+	ProbePort            = "PROBE_PORT"
+	LiveProbeInterval    = "LIVE_PROBE_INTERVAL"
+	NotLiveProbeInterval = "NOT_LIVE_PROBE_INTERVAL"
+	VolthaHost           = "VOLTHA_HOST"
+	VolthaPort           = "VOLTHA_PORT"
+	HostName             = "HOST_NAME"
+
+	// openolt adapter environment variables
+
+	HeartbeatCheckInterval      = "HEARTBEAT_CHECK_INTERVAL"
+	HeartbeatFailReportInterval = "HEARTBEAT_FAIL_REPORT_INTERVAL"
+	GrpcTimeoutInterval         = "GRPC_TIMEOUT_INTERVAL"
+
+	// rwcore environment variables
+
+	RWCoreEndpoint            = "RW_CORE_ENDPOINT"
+	GrpcHost                  = "GRPC_HOST"
+	GrpcPort                  = "GRPC_PORT"
+	AffinityRouterTopic       = "AFFINITY_ROUTER_TOPIC"
+	InCompetingMode           = "IN_COMPETING_MODE"
+	KVTxnKeyDelTime           = "KV_TXN_KEY_DEL_TIME"
+	KVStoreDataPrefix         = "KV_STORE_DATA_PREFIX"
+	LongRunningRequestTimeout = "LONG_RUNNING_REQ_TIMEOUT"
+	DefaultRequestTimeout     = "DEFAULT_REQ_TIMEOUT"
+	DefaultCoreTimeout        = "DEFAULT_CORE_TIMEOUT"
+	CoreBindingKey            = "CORE_BINDING_KEY"
+	CorePairTopic             = "CORE_PAIR_TOPIC"
+	MaxConnectionRetries      = "MAX_CONNECTION_RETRIES"
+	ConnectionRetryInterval   = "CONNECTION_RETRY_INTERVAL"
+
+	// vgc environment variables
+
+	DeviceListRefreshInterval = "DEVICE_LIST_REFRESH_INTERVAL" // in seconds
+	CPUProfile                = "CPU_PROFILE"
+	MemProfile                = "MEM_PROFILE"
+	VendorID                  = "VENDOR_ID"
+    DeviceSyncDuration        = "DEVICE_SYNC_DURATION"
+	// openonu environment variables
+
+	OmciPacketCapture = "SAVE_OMCI_PACKET_CAPTURE"
+)
+
+// ParseStringEnvVariable reads the environment variable and returns env as string
+func ParseStringEnvVariable(envVarName string, defaultVal string) string {
+	envValue := os.Getenv(envVarName)
+	if envValue == "" {
+		fmt.Println("Environment variable " + envVarName + " undefined")
+		return defaultVal
+	}
+	return envValue
+}
+
+// ParseIntEnvVariable reads the environment variable and returns env as int64
+func ParseIntEnvVariable(envVarName string, defaultVal int64) int64 {
+	envValue := os.Getenv(envVarName)
+	if envValue == "" {
+		fmt.Println("Environment variable "+envVarName+" undefined", envVarName)
+		return defaultVal
+	}
+	returnVal, err := strconv.Atoi(envValue)
+	if err != nil {
+		fmt.Println("Unable to convert string to integer environment variable")
+		return defaultVal
+	}
+	return int64(returnVal)
+}
+
+// ParseBoolEnvVariable reads the environment variable and returns env as boolean
+func ParseBoolEnvVariable(envVarName string, defaultVal bool) bool {
+	envValue := os.Getenv(envVarName)
+	if envValue == "" {
+		fmt.Println("Environment variable " + envVarName + " undefined")
+		return defaultVal
+	}
+	if envValue == "true" || envValue == "True" {
+		return true
+	}
+	return false
+}
diff --git a/internal/pkg/util/utils.go b/internal/pkg/util/utils.go
new file mode 100644
index 0000000..0711053
--- /dev/null
+++ b/internal/pkg/util/utils.go
@@ -0,0 +1,148 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package util
+
+import (
+	"encoding/binary"
+	"net"
+	"strings"
+
+	"voltha-go-controller/internal/pkg/of"
+)
+
+// RemoveFromSlice to remove particular value from given slice.
+func RemoveFromSlice(s []string, value string) []string {
+	i := 0
+	for i = 0; i < len(s); i++ {
+		if s[i] == value {
+			break
+		}
+	}
+	if i != len(s) {
+		//It means value is found in the slice
+		s[len(s)-1], s[i] = s[i], s[len(s)-1]
+		return s[:len(s)-1]
+	}
+	return s
+}
+
+//IsSliceSame - check and return true if the two slices are identical
+func IsSliceSame(ref, rcvd []uint32) bool {
+
+	var found bool
+	if len(ref) != len(rcvd) {
+		return false
+	}
+
+	for _, refEntry := range ref {
+		found = false
+
+		for _, rcvdEntry := range rcvd {
+			if refEntry == rcvdEntry {
+				found = true
+				break
+			}
+		}
+		if !found {
+			return false
+		}
+	}
+	return true
+}
+
+//IsPbitSliceSame - check and return true if the two slices are identical
+func IsPbitSliceSame(ref, rcvd []of.PbitType) bool {
+
+	var found bool
+	if len(ref) != len(rcvd) {
+		return false
+	}
+
+	for _, refEntry := range ref {
+		found = false
+
+		for _, rcvdEntry := range rcvd {
+			if refEntry == rcvdEntry {
+				found = true
+				break
+			}
+		}
+		if !found {
+			return false
+		}
+	}
+	return true
+}
+
+// IsNniPort is to check if given port is Nni Port.
+func IsNniPort(id uint32) bool {
+	return (id >= 0x100000)
+}
+
+// Uint32ToByte to convert uint32 to byte
+func Uint32ToByte(value uint32) []byte {
+	byteValue := make([]byte, 4)
+	binary.BigEndian.PutUint32(byteValue[0:4], value)
+	return byteValue
+}
+
+// IP2LongConv convert ip address to integer value.
+func IP2LongConv(ip net.IP) uint32 {
+	if len(ip) == 16 {
+		return binary.BigEndian.Uint32(ip[12:16])
+	}
+	return binary.BigEndian.Uint32(ip)
+}
+
+// Long2ipConv convert integer to ip address.
+func Long2ipConv(nn uint32) net.IP {
+	ip := make(net.IP, 4)
+	binary.BigEndian.PutUint32(ip, nn)
+	return ip
+}
+
+// GetExpIPList converts list or range of IPs to expanded IP list
+func GetExpIPList(ips []string) []net.IP {
+	ipList := []net.IP{}
+
+	for _, ipOrRange := range ips {
+		if strings.Contains(ipOrRange, "-") {
+			var splits = strings.Split(ipOrRange, "-")
+			ipStart := IP2LongConv(net.ParseIP(splits[0]))
+			ipEnd := IP2LongConv(net.ParseIP(splits[1]))
+
+			for i := ipStart; i <= ipEnd; i++ {
+				ipList = append(ipList, Long2ipConv(i))
+			}
+		} else {
+			ipList = append(ipList, net.ParseIP(ipOrRange))
+		}
+	}
+	return ipList
+}
+
+// MacAddrsMatch for comparison of MAC addresses and return true if MAC addresses matches
+func MacAddrsMatch(addr1 net.HardwareAddr, addr2 net.HardwareAddr) bool {
+	if len(addr1) != len(addr2) {
+		return false
+	}
+	for i := 0; i < len(addr1); i++ {
+		if addr1[i] != addr2[i] {
+			return false
+		}
+	}
+	return true
+}
diff --git a/internal/pkg/vpagent/changeEvent.go b/internal/pkg/vpagent/changeEvent.go
new file mode 100644
index 0000000..3d86675
--- /dev/null
+++ b/internal/pkg/vpagent/changeEvent.go
@@ -0,0 +1,106 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package vpagent
+
+import (
+	"context"
+	"io"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"google.golang.org/grpc"
+)
+
+func (vpa *VPAgent) receiveChangeEvents(ctx context.Context) {
+	logger.Debug(ctx, "receive-change-events-started")
+	// If we exit, assume disconnected
+	defer func() {
+		vpa.events <- vpaEventVolthaDisconnected
+		logger.Debug(ctx, "receive-change-events-finished")
+	}()
+	if vpa.volthaClient == nil {
+		logger.Error(ctx, "no-voltha-connection")
+		return
+	}
+	opt := grpc.EmptyCallOption{}
+	streamCtx, streamDone := context.WithCancel(context.Background())
+	defer streamDone()
+	vServiceClient := vpa.volthaClient.Get()
+	if vServiceClient == nil {
+		logger.Error(ctx, "Failed to get Voltha Service Client")
+		return
+	}
+
+	stream, err := vServiceClient.ReceiveChangeEvents(streamCtx, &empty.Empty{}, opt)
+	if err != nil {
+		logger.Errorw(ctx, "Unable to establish Receive Change Event Stream",
+			log.Fields{"error": err})
+		return
+	}
+
+top:
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Errorw(ctx, "Context Done", log.Fields{"Context": ctx})
+			break top
+		default:
+			ce, err := stream.Recv()
+			if err == io.EOF {
+				logger.Infow(ctx, "EOF for receiveChangeEvents stream, reconnecting", log.Fields{"err": err})
+				stream, err = vServiceClient.ReceiveChangeEvents(streamCtx, &empty.Empty{}, opt)
+				if err != nil {
+					logger.Errorw(ctx, "Unable to establish Receive Change Event Stream",
+						log.Fields{"error": err})
+					return
+				}
+				continue
+			}
+			if isConnCanceled(err) {
+				logger.Errorw(ctx, "error receiving change event",
+					log.Fields{"error": err})
+				break top
+			} else if err != nil {
+				logger.Infow(ctx, "Ignoring unhandled error", log.Fields{"err": err})
+				continue
+			}
+			vpa.changeEventChannel <- ce
+			logger.Debug(ctx, "receive-change-event-queued")
+		}
+	}
+}
+
+func (vpa *VPAgent) handleChangeEvents(ctx context.Context) {
+	logger.Debug(ctx, "handle-change-event-started")
+
+top:
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Errorw(ctx, "Context Done", log.Fields{"Context": ctx})
+			break top
+		case changeEvent := <-vpa.changeEventChannel:
+			logger.Debugw(ctx, "Change Event", log.Fields{"Device": changeEvent.Id})
+			if vpc := vpa.getVPClient(changeEvent.Id); vpc != nil {
+				if err:= vpc.ChangeEvent(changeEvent); err != nil {
+					logger.Errorw(ctx, "error handling Change Event", log.Fields{"Error": err, "Device": changeEvent.Id})
+				}
+			}
+		}
+	}
+
+	logger.Debug(ctx, "handle-change-event-finsihed")
+}
diff --git a/internal/pkg/vpagent/common.go b/internal/pkg/vpagent/common.go
new file mode 100644
index 0000000..d32e0a6
--- /dev/null
+++ b/internal/pkg/vpagent/common.go
@@ -0,0 +1,63 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+// Package vpagent Common Logger initialization
+package vpagent
+
+import (
+	"context"
+	"strings"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
+
+// IsConnCanceled returns true, if error is from a closed gRPC connection.
+// ref. https://github.com/grpc/grpc-go/pull/1854
+func isConnCanceled(err error) bool {
+	if err == nil {
+		return false
+	}
+	// >= gRPC v1.23.x
+	s, ok := status.FromError(err)
+	if ok {
+		// connection is canceled or server has already closed the connection
+		return s.Code() == codes.Canceled || s.Message() == "transport is closing"
+	}
+
+	e, ok := status.FromError(err)
+	if ok {
+		// connection is canceled or server has already closed the connection
+		return e.Code() == codes.Canceled || e.Message() == "all SubConns are in TransientFailure"
+	}
+
+	// >= gRPC v1.10.x
+	if err == context.Canceled {
+		return true
+	}
+
+	// <= gRPC v1.7.x returns 'errors.New("grpc: the client connection is closing")'
+	return strings.Contains(err.Error(), "grpc: the client connection is closing")
+}
diff --git a/internal/pkg/vpagent/connection.go b/internal/pkg/vpagent/connection.go
new file mode 100644
index 0000000..607f7f3
--- /dev/null
+++ b/internal/pkg/vpagent/connection.go
@@ -0,0 +1,88 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package vpagent
+
+import (
+	"context"
+	"errors"
+	"time"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc"
+)
+
+//GrpcMaxSize Max size of grpc message
+const GrpcMaxSize int = 17455678
+
+func (vpa *VPAgent) establishConnectionToVoltha(ctx context.Context, p *probe.Probe) error {
+	if p != nil {
+		p.UpdateStatus(ctx, "voltha", probe.ServiceStatusPreparing)
+	}
+
+	if vpa.volthaConnection != nil {
+		vpa.volthaConnection.Close()
+	}
+
+	vpa.volthaConnection = nil
+	vpa.volthaClient.Clear()
+	try := 1
+	for vpa.ConnectionMaxRetries == 0 || try < vpa.ConnectionMaxRetries {
+		conn, err := grpc.Dial(vpa.VolthaAPIEndPoint, grpc.WithInsecure(), grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(GrpcMaxSize)))
+		if err == nil {
+			svc := voltha.NewVolthaServiceClient(conn)
+			if svc != nil {
+				if _, err = svc.GetVoltha(context.Background(), &empty.Empty{}); err == nil {
+					logger.Debugw(ctx, "Established connection to Voltha",
+						log.Fields{
+							"VolthaApiEndPoint": vpa.VolthaAPIEndPoint,
+						})
+					vpa.volthaConnection = conn
+					vpa.volthaClient.Set(svc)
+					if p != nil {
+						p.UpdateStatus(ctx, "voltha", probe.ServiceStatusRunning)
+					}
+					vpa.events <- vpaEventVolthaConnected
+					return nil
+				}
+			}
+		}
+		logger.Warnw(ctx, "Failed to connect to voltha",
+			log.Fields{
+				"VolthaApiEndPoint": vpa.VolthaAPIEndPoint,
+				"error":             err.Error(),
+			})
+		if vpa.ConnectionMaxRetries == 0 || try < vpa.ConnectionMaxRetries {
+			if vpa.ConnectionMaxRetries != 0 {
+				try++
+			}
+			time.Sleep(vpa.ConnectionRetryDelay)
+		}
+	}
+	if p != nil {
+		p.UpdateStatus(ctx, "voltha", probe.ServiceStatusFailed)
+	}
+	return errors.New("failed-to-connect-to-voltha")
+}
+
+// CloseConnectionToVoltha closes the grpc connection to VOLTHA
+func (vpa *VPAgent) CloseConnectionToVoltha() {
+	//Close the grpc connection to voltha
+	logger.Debug(ctx, "Closing voltha grpc connection")
+	vpa.volthaConnection.Close()
+}
diff --git a/internal/pkg/vpagent/packetIn.go b/internal/pkg/vpagent/packetIn.go
new file mode 100644
index 0000000..4a2f7c3
--- /dev/null
+++ b/internal/pkg/vpagent/packetIn.go
@@ -0,0 +1,96 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package vpagent
+
+import (
+	"context"
+	"io"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"google.golang.org/grpc"
+)
+
+func (vpa *VPAgent) receivePacketsIn(ctx context.Context) {
+	logger.Debug(ctx, "receive-packets-in-started")
+	// If we exit, assume disconnected
+	defer func() {
+		vpa.events <- vpaEventVolthaDisconnected
+		logger.Debug(ctx, "receive-packets-in-finished")
+	}()
+	if vpa.volthaClient == nil {
+		logger.Error(ctx, "no-voltha-connection")
+		return
+	}
+	opt := grpc.EmptyCallOption{}
+	streamCtx, streamDone := context.WithCancel(context.Background())
+	defer streamDone()
+	stream, err := vpa.volthaClient.Get().ReceivePacketsIn(streamCtx, &empty.Empty{}, opt)
+	if err != nil {
+		logger.Errorw(ctx, "Unable to establish Receive PacketIn Stream",
+			log.Fields{"error": err})
+		return
+	}
+
+top:
+
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Errorw(ctx, "Context Done", log.Fields{"Context": ctx})
+			break top
+		default:
+			pkt, err := stream.Recv()
+			if err == io.EOF {
+				logger.Infow(ctx, "EOF for receivePacketsIn stream, reconnecting", log.Fields{"err": err})
+				stream, err = vpa.volthaClient.Get().ReceivePacketsIn(streamCtx, &empty.Empty{}, opt)
+				if err != nil {
+					logger.Errorw(ctx, "Unable to establish Receive PacketIn Stream",
+						log.Fields{"error": err})
+					return
+				}
+				continue
+			}
+
+			if isConnCanceled(err) {
+				logger.Errorw(ctx, "error receiving packet",
+					log.Fields{"error": err})
+				break top
+			} else if err != nil {
+				logger.Infow(ctx, "Ignoring unhandled error", log.Fields{"err": err})
+				continue
+			}
+			vpa.packetInChannel <- pkt
+		}
+	}
+}
+
+func (vpa *VPAgent) handlePacketsIn(ctx context.Context) {
+	logger.Debug(ctx, "handle-packets-in-started")
+top:
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Errorw(ctx, "Context Done", log.Fields{"Context": ctx})
+			break top
+		case packet := <-vpa.packetInChannel:
+			if vpc := vpa.getVPClient(packet.Id); vpc != nil {
+				vpc.PacketIn(packet)
+			}
+		}
+	}
+	logger.Debug(ctx, "handle-packets-in-finished")
+}
diff --git a/internal/pkg/vpagent/packetOut.go b/internal/pkg/vpagent/packetOut.go
new file mode 100644
index 0000000..501bdea
--- /dev/null
+++ b/internal/pkg/vpagent/packetOut.go
@@ -0,0 +1,61 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package vpagent
+
+import (
+	"context"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"google.golang.org/grpc"
+)
+
+func (vpa *VPAgent) streamPacketOut(ctx context.Context) {
+	logger.Debug(ctx, "packet-out-started")
+	// If we exit, assume disconnected
+	defer func() {
+		vpa.events <- vpaEventVolthaDisconnected
+		logger.Debug(ctx, "packet-out-finished")
+	}()
+	if vpa.volthaClient == nil {
+		logger.Error(ctx, "no-voltha-connection")
+		return
+	}
+	opt := grpc.EmptyCallOption{}
+	streamCtx, streamDone := context.WithCancel(context.Background())
+	outClient, err := vpa.volthaClient.Get().StreamPacketsOut(streamCtx, opt)
+	defer streamDone()
+	if err != nil {
+		logger.Errorw(ctx, "streamPacketOut Error creating packetout stream ", log.Fields{"error": err})
+		return
+	}
+top:
+	for {
+		select {
+		case <-ctx.Done():
+			break top
+		case ofPacketOut := <-vpa.packetOutChannel:
+			if logger.V(log.DebugLevel) {
+				logger.Debug(ctx, "streamPacketOut Receive PacketOut from Channel")
+			}
+			if err := outClient.Send(ofPacketOut); err != nil {
+				logger.Errorw(ctx, "packet-out-send-error",
+					log.Fields{"error": err.Error()})
+				break top
+			}
+			logger.Debug(ctx, "packet-out-send")
+		}
+	}
+}
diff --git a/internal/pkg/vpagent/refresh.go b/internal/pkg/vpagent/refresh.go
new file mode 100644
index 0000000..c06e5cd
--- /dev/null
+++ b/internal/pkg/vpagent/refresh.go
@@ -0,0 +1,141 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package vpagent
+
+import (
+	"context"
+	"time"
+
+	"voltha-go-controller/internal/pkg/intf"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+func (vpa *VPAgent) synchronizeDeviceList(ctx context.Context) {
+	// Send reconnection indication to the devices already known
+	for _, vpc := range vpa.clientMap {
+		vpc.ConnectInd(context.TODO(), intf.DeviceReDisc)
+	}
+
+	// Refresh once to get everything started
+	vpa.refreshDeviceList()
+
+	tick := time.NewTicker(vpa.DeviceListRefreshInterval)
+loop:
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Errorw(ctx, "Context Done", log.Fields{"Context": ctx})
+			break loop
+		case <-tick.C:
+			vpa.refreshDeviceList()
+		}
+	}
+	tick.Stop()
+}
+
+func (vpa *VPAgent) refreshDeviceList() {
+	// If we exit, assume disconnected
+	if vpa.volthaClient == nil {
+		logger.Error(ctx, "no-voltha-connection")
+		vpa.events <- vpaEventVolthaDisconnected
+		return
+	}
+	deviceList, err := vpa.volthaClient.Get().ListLogicalDevices(context.Background(), &empty.Empty{})
+	if err != nil {
+		logger.Errorw(ctx, "vpagent failed to query device list from voltha",
+			log.Fields{"error": err})
+		vpa.events <- vpaEventVolthaDisconnected
+		return
+	}
+
+	var toAdd []int
+	var toDel []string
+	var deviceIDMap = make(map[string]string)
+	for index, d := range deviceList.Items {
+		deviceID := d.Id
+		deviceIDMap[deviceID] = deviceID
+		if vpa.clientMap[deviceID] == nil {
+			toAdd = append(toAdd, index)
+		}
+	}
+	for key := range vpa.clientMap {
+		deviceID, ok := deviceIDMap[key]
+		if !ok || (ok && deviceID == "") {
+			toDel = append(toDel, key)
+		}
+	}
+	logger.Debugw(ctx, "Device Refresh", log.Fields{"ToAdd": toAdd, "ToDel": toDel})
+	for i := 0; i < len(toAdd); i++ {
+		device := deviceList.Items[toAdd[i]]
+		serialNum := device.Desc.SerialNum
+		// If the blocked device list contain device serial number, do not add VPClient.
+		if vpa.VPClientAgent.IsBlockedDevice(serialNum) {
+			logger.Debugw(ctx, "Device Serial Number is present in the blocked device list", log.Fields{"device-serial-number": serialNum})
+		} else {
+			vpa.addVPClient(device) // client is started in addVPClient
+		}
+	}
+
+	for i := 0; i < len(toDel); i++ {
+		vpa.VPClientAgent.DelDevice(toDel[i])
+		vpa.mapLock.Lock()
+		delete(vpa.clientMap, toDel[i])
+		vpa.mapLock.Unlock()
+	}
+}
+
+func (vpa *VPAgent) addVPClient(device *voltha.LogicalDevice) intf.IVPClient {
+	logger.Warnw(ctx, "GrpcClient addClient called ", log.Fields{"device-id": device.Id})
+	vpa.mapLock.Lock()
+	defer vpa.mapLock.Unlock()
+	var serialNum = "Unknown"
+	if device.Desc != nil {
+		serialNum = device.Desc.SerialNum
+	}
+	vpc := vpa.clientMap[device.Id]
+	if vpc == nil {
+		vpa.VPClientAgent.AddNewDevice(&intf.VPClientCfg{
+			DeviceID:         device.Id,
+			SerialNum:        serialNum,
+			SouthBoundID:     device.RootDeviceId,
+			VolthaClient:     vpa.volthaClient,
+			PacketOutChannel: vpa.packetOutChannel,
+		})
+
+	}
+	logger.Debugw(ctx, "Finished with addClient", log.Fields{"deviceID": device.Id})
+	return vpc
+}
+
+//AddClientToClientMap - called by controller once device obj is created
+func (vpa *VPAgent) AddClientToClientMap(deviceID string, vpc intf.IVPClient) {
+	vpa.mapLock.Lock()
+	defer vpa.mapLock.Unlock()
+
+	if vpc != nil {
+		vpa.clientMap[deviceID] = vpc
+	}
+}
+
+func (vpa *VPAgent) getVPClient(deviceID string) intf.IVPClient {
+	if vpc, ok := vpa.clientMap[deviceID]; ok {
+		return vpc
+	}
+	return nil
+}
diff --git a/internal/pkg/vpagent/volthaprotoagent.go b/internal/pkg/vpagent/volthaprotoagent.go
new file mode 100644
index 0000000..04e3d08
--- /dev/null
+++ b/internal/pkg/vpagent/volthaprotoagent.go
@@ -0,0 +1,237 @@
+/*
+* Copyright 2022-present Open Networking Foundation
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package vpagent
+
+import (
+	"context"
+	"sync"
+	"time"
+
+	"voltha-go-controller/database"
+	"voltha-go-controller/internal/pkg/holder"
+	"voltha-go-controller/internal/pkg/intf"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc"
+)
+
+var logger log.CLogger
+var ctx = context.TODO()
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
+
+type vpaEvent byte
+type vpaState byte
+
+var db database.DBIntf
+
+const (
+	vpaEventStart = vpaEvent(iota)
+	vpaEventVolthaConnected
+	vpaEventVolthaDisconnected
+	vpaEventError
+
+	vpaStateConnected = vpaState(iota)
+	vpaStateConnecting
+	vpaStateDisconnected
+)
+
+var vpAgent *VPAgent
+
+// VPAgent structure
+type VPAgent struct {
+	VolthaAPIEndPoint         string
+	DeviceListRefreshInterval time.Duration
+	ConnectionMaxRetries      int
+	ConnectionRetryDelay      time.Duration
+
+	volthaConnection *grpc.ClientConn
+	volthaClient     *holder.VolthaServiceClientHolder
+	mapLock          sync.Mutex
+	clientMap        map[string]intf.IVPClient
+	events           chan vpaEvent
+
+	packetInChannel    chan *ofp.PacketIn
+	packetOutChannel   chan *ofp.PacketOut
+	changeEventChannel chan *ofp.ChangeEvent
+	VPClientAgent      intf.IVPClientAgent
+}
+
+// NewVPAgent is constructor for VPAgent
+func NewVPAgent(config *VPAgent) (*VPAgent, error) {
+	vpa := VPAgent{
+		VolthaAPIEndPoint:         config.VolthaAPIEndPoint,
+		DeviceListRefreshInterval: config.DeviceListRefreshInterval,
+		ConnectionMaxRetries:      config.ConnectionMaxRetries,
+		ConnectionRetryDelay:      config.ConnectionRetryDelay,
+		VPClientAgent:             config.VPClientAgent,
+		volthaClient:              &holder.VolthaServiceClientHolder{},
+		packetInChannel:           make(chan *ofp.PacketIn),
+		// customPacketIndChannel:    make(chan *voltha.CustomPacketIn),
+		packetOutChannel:   make(chan *ofp.PacketOut),
+		changeEventChannel: make(chan *ofp.ChangeEvent),
+		// ofpCommandNotiChannel:     make(chan *voltha.OfpCmdRespNotification),
+		// oltRebootNotiChannel:      make(chan *voltha.OltRebootNotification),
+		clientMap: make(map[string]intf.IVPClient),
+		events:    make(chan vpaEvent, 100),
+	}
+
+	if vpa.DeviceListRefreshInterval <= 0 {
+		logger.Warnw(ctx, "device list refresh internal not valid, setting to default",
+			log.Fields{
+				"value":   vpa.DeviceListRefreshInterval.String(),
+				"default": (10 * time.Second).String()})
+		vpa.DeviceListRefreshInterval = 1 * time.Minute
+	}
+
+	if vpa.ConnectionRetryDelay <= 0 {
+		logger.Warnw(ctx, "connection retry delay not value, setting to default",
+			log.Fields{
+				"value":   vpa.ConnectionRetryDelay.String(),
+				"default": (3 * time.Second).String()})
+		vpa.ConnectionRetryDelay = 3 * time.Second
+	}
+
+	if db == nil {
+		db = database.GetDatabase()
+	}
+	vpAgent = &vpa
+	return &vpa, nil
+}
+
+//GetVPAgent - returns vpAgent object
+func GetVPAgent() *VPAgent {
+	return vpAgent
+}
+
+// VolthaSvcClient for Voltha Svc client
+func (vpa *VPAgent) VolthaSvcClient() voltha.VolthaServiceClient {
+	return vpa.volthaClient.Get()
+}
+
+// Run - make the inital connection to voltha and kicks off io streams
+func (vpa *VPAgent) Run(ctx context.Context) {
+
+	logger.Debugw(ctx, "Starting GRPC - VOLTHA client",
+		log.Fields{
+			"voltha-endpoint": vpa.VolthaAPIEndPoint})
+
+	// If the context contains a k8s probe then register services
+	p := probe.GetProbeFromContext(ctx)
+	if p != nil {
+		p.RegisterService(ctx, "voltha")
+	}
+
+	vpa.events <- vpaEventStart
+
+	/*
+	 * Two sub-contexts are created here for different purposes so we can
+	 * control the lifecyle of processing loops differently.
+	 *
+	 * volthaCtx -  controls those processes that rely on the GRPC
+	 *              GRPCconnection to voltha and will be restarted when the
+	 *              GRPC connection is interrupted.
+	 * hdlCtx    -  controls those processes that listen to channels and
+	 *              process each message. these will likely never be
+	 *              stopped until the vpagent is stopped.
+	 */
+	var volthaCtx, hdlCtx context.Context
+	var volthaDone, hdlDone func()
+	state := vpaStateDisconnected
+
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Errorw(ctx, "Context Done", log.Fields{"Context": ctx})
+			if volthaDone != nil {
+				volthaDone()
+			}
+			if hdlDone != nil {
+				hdlDone()
+			}
+			return
+		case event := <-vpa.events:
+			switch event {
+			case vpaEventStart:
+				logger.Debug(ctx, "vpagent-voltha-start-event")
+
+				// Start the loops that process messages
+				hdlCtx, hdlDone = context.WithCancel(context.Background())
+				go vpa.handlePacketsIn(hdlCtx)
+				go vpa.handleChangeEvents(hdlCtx)
+
+				// Kick off process to attempt to establish
+				// connection to voltha
+				state = vpaStateConnecting
+				go func() {
+					if err := vpa.establishConnectionToVoltha(hdlCtx, p); err != nil {
+						logger.Fatalw(ctx, "voltha-connection-failed", log.Fields{"error": err})
+					}
+				}()
+
+			case vpaEventVolthaConnected:
+				logger.Debug(ctx, "vpagent-voltha-connect-event")
+
+				// Start the loops that poll from voltha
+				if state != vpaStateConnected {
+					state = vpaStateConnected
+					volthaCtx, volthaDone = context.WithCancel(context.Background())
+					go vpa.receiveChangeEvents(volthaCtx)
+					go vpa.receivePacketsIn(volthaCtx)
+					go vpa.streamPacketOut(volthaCtx)
+					go vpa.synchronizeDeviceList(volthaCtx)
+				}
+
+			case vpaEventVolthaDisconnected:
+				if p != nil {
+					p.UpdateStatus(ctx, "voltha", probe.ServiceStatusNotReady)
+				}
+				logger.Debug(ctx, "vpagent-voltha-disconnect-event")
+				if state == vpaStateConnected {
+					state = vpaStateDisconnected
+					vpa.volthaClient.Clear()
+					volthaDone()
+					volthaDone = nil
+				}
+				if state != vpaStateConnecting {
+					state = vpaStateConnecting
+					go func() {
+						hdlCtx, hdlDone = context.WithCancel(context.Background())
+						if err := vpa.establishConnectionToVoltha(hdlCtx, p); err != nil {
+							logger.Fatalw(ctx, "voltha-connection-failed", log.Fields{"error": err})
+						}
+					}()
+				}
+
+			case vpaEventError:
+				logger.Debug(ctx, "vpagent-error-event")
+			default:
+				logger.Fatalw(ctx, "vpagent-unknown-event",
+					log.Fields{"event": event})
+			}
+		}
+	}
+}