[VOL-2576] Improve route calculation

This commit changes the way device routes are calculated. It
replaces the device graph method. The graph method relies on the
shortest path calculation which is quite resource intensive. For
instance, generating the routes for a PON network with 1 OLT having
8 PON ports, 64 ONUs per Port and 4 UNIs per ONUs took 96 secs to
generate the 4096 routes.  The new method creates the routes from
the devices data with no middle step.  Generating routes for the
above topology now takes 4ms.

Change-Id: I32bffe06d12ad0fea94002a39f217547dc55cdbf
diff --git a/rw_core/core/logical_device_agent.go b/rw_core/core/logical_device_agent.go
index 7274e9f..857776f 100644
--- a/rw_core/core/logical_device_agent.go
+++ b/rw_core/core/logical_device_agent.go
@@ -21,6 +21,7 @@
 	"encoding/hex"
 	"errors"
 	"fmt"
+	"github.com/opencord/voltha-go/rw_core/route"
 	"reflect"
 	"sync"
 	"time"
@@ -28,7 +29,6 @@
 	"github.com/gogo/protobuf/proto"
 	"github.com/opencord/voltha-go/db/model"
 	fd "github.com/opencord/voltha-go/rw_core/flowdecomposition"
-	"github.com/opencord/voltha-go/rw_core/graph"
 	coreutils "github.com/opencord/voltha-go/rw_core/utils"
 	fu "github.com/opencord/voltha-lib-go/v3/pkg/flows"
 	"github.com/opencord/voltha-lib-go/v3/pkg/log"
@@ -47,7 +47,7 @@
 	ldeviceMgr         *LogicalDeviceManager
 	clusterDataProxy   *model.Proxy
 	exitChannel        chan int
-	deviceGraph        *graph.DeviceGraph
+	deviceRoutes       *route.DeviceRoutes
 	flowProxy          *model.Proxy
 	groupProxy         *model.Proxy
 	meterProxy         *model.Proxy
@@ -55,7 +55,7 @@
 	portProxies        map[string]*model.Proxy
 	portProxiesLock    sync.RWMutex
 	lockLogicalDevice  sync.RWMutex
-	lockDeviceGraph    sync.RWMutex
+	lockDeviceRoutes   sync.RWMutex
 	logicalPortsNo     map[uint32]bool //value is true for NNI port
 	lockLogicalPortsNo sync.RWMutex
 	flowDecomposer     *fd.FlowDecomposer
@@ -78,7 +78,7 @@
 	agent.portProxies = make(map[string]*model.Proxy)
 	agent.portProxiesLock = sync.RWMutex{}
 	agent.lockLogicalPortsNo = sync.RWMutex{}
-	agent.lockDeviceGraph = sync.RWMutex{}
+	agent.lockDeviceRoutes = sync.RWMutex{}
 	agent.logicalPortsNo = make(map[uint32]bool)
 	agent.defaultTimeout = timeout
 	return &agent
@@ -205,9 +205,13 @@
 		return status.Error(codes.Internal, "logical-device-proxy-null")
 	}
 
-	// Setup the device graph - run it in its own routine
+	// Setup the device routes. Building routes may fail if the pre-conditions are not satisfied (e.g. no PON ports present)
 	if loadFromdB {
-		go agent.generateDeviceGraph(context.Background())
+		go func() {
+			if err := agent.buildRoutes(context.Background()); err != nil {
+				log.Warn("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+			}
+		}()
 	}
 	return nil
 }
@@ -344,10 +348,10 @@
 		}
 		agent.addLogicalPortToMap(port.PortNo, false)
 	} else {
-		// Update the device graph to ensure all routes on the logical device have been calculated
-		if err = agent.updateRoutes(ctx, device, port); err != nil {
-			log.Errorw("failed-to-update-routes", log.Fields{"deviceId": device.Id, "port": port, "error": err})
-			return err
+		// Update the device routes to ensure all routes on the logical device have been calculated
+		if err = agent.buildRoutes(ctx); err != nil {
+			// Not an error - temporary state
+			log.Warnw("failed-to-update-routes", log.Fields{"device-id": device.Id, "port": port, "error": err})
 		}
 	}
 	return nil
@@ -530,17 +534,21 @@
 	return nil
 }
 
-//generateDeviceGraphIfNeeded generates the device graph if the logical device has been updated since the last time
+//generateDeviceRoutesIfNeeded generates the device routes if the logical device has been updated since the last time
 //that device graph was generated.
-func (agent *LogicalDeviceAgent) generateDeviceGraphIfNeeded(ctx context.Context) error {
+func (agent *LogicalDeviceAgent) generateDeviceRoutesIfNeeded(ctx context.Context) error {
+	agent.lockDeviceRoutes.Lock()
+	defer agent.lockDeviceRoutes.Unlock()
+
 	ld := agent.GetLogicalDevice()
-	agent.lockDeviceGraph.Lock()
-	defer agent.lockDeviceGraph.Unlock()
-	if agent.deviceGraph != nil && agent.deviceGraph.IsUpToDate(ld) {
+
+	if agent.deviceRoutes != nil && agent.deviceRoutes.IsUpToDate(ld) {
 		return nil
 	}
 	log.Debug("Generation of device graph required")
-	agent.generateDeviceGraph(ctx)
+	if err := agent.buildRoutes(ctx); err != nil {
+		return err
+	}
 	return nil
 }
 
@@ -550,7 +558,7 @@
 	if flow == nil {
 		return nil
 	}
-	if err := agent.generateDeviceGraphIfNeeded(ctx); err != nil {
+	if err := agent.generateDeviceRoutesIfNeeded(ctx); err != nil {
 		return err
 	}
 	switch flow.GetCommand() {
@@ -575,9 +583,10 @@
 	if groupMod == nil {
 		return nil
 	}
-	if err := agent.generateDeviceGraphIfNeeded(ctx); err != nil {
+	if err := agent.generateDeviceRoutesIfNeeded(ctx); err != nil {
 		return err
 	}
+
 	switch groupMod.GetCommand() {
 	case ofp.OfpGroupModCommand_OFPGC_ADD:
 		return agent.groupAdd(ctx, groupMod)
@@ -596,7 +605,7 @@
 	if meterMod == nil {
 		return nil
 	}
-	if err := agent.generateDeviceGraphIfNeeded(ctx); err != nil {
+	if err := agent.generateDeviceRoutesIfNeeded(ctx); err != nil {
 		return err
 	}
 	switch meterMod.GetCommand() {
@@ -857,7 +866,10 @@
 			log.Error("Meter-referred-in-flows-not-present")
 			return err
 		}
-		deviceRules := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: updatedFlows}, *lDevice.FlowGroups)
+		deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: updatedFlows}, *lDevice.FlowGroups)
+		if err != nil {
+			return err
+		}
 		log.Debugw("rules", log.Fields{"rules": deviceRules.String()})
 
 		if err := agent.addDeviceFlowsAndGroups(ctx, deviceRules, &flowMetadata); err != nil {
@@ -973,7 +985,10 @@
 			log.Error("Meter-referred-in-flows-not-present")
 			return errors.New("Meter-referred-in-flows-not-present")
 		}
-		deviceRules := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: toDelete}, ofp.FlowGroups{Items: flowGroups})
+		deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: toDelete}, ofp.FlowGroups{Items: flowGroups})
+		if err != nil {
+			return err
+		}
 		log.Debugw("rules", log.Fields{"rules": deviceRules.String()})
 
 		if err := agent.deleteDeviceFlowsAndGroups(ctx, deviceRules, &flowMetadata); err != nil {
@@ -1112,7 +1127,10 @@
 			log.Error("meter-referred-in-flows-not-present")
 			return err
 		}
-		deviceRules := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: flowsToDelete}, ofp.FlowGroups{Items: flowGroups})
+		deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: flowsToDelete}, ofp.FlowGroups{Items: flowGroups})
+		if err != nil {
+			return err
+		}
 		log.Debugw("rules", log.Fields{"rules": deviceRules.String()})
 
 		if err := agent.deleteDeviceFlowsAndGroups(ctx, deviceRules, &flowMetadata); err != nil {
@@ -1203,7 +1221,10 @@
 		groupsChanged = true
 	}
 	if flowsChanged || groupsChanged {
-		deviceRules := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: flows}, ofp.FlowGroups{Items: groups})
+		deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: flows}, ofp.FlowGroups{Items: groups})
+		if err != nil {
+			return err
+		}
 		log.Debugw("rules", log.Fields{"rules": deviceRules.String()})
 
 		if err := agent.updateDeviceFlowsAndGroups(ctx, deviceRules, nil); err != nil {
@@ -1292,8 +1313,16 @@
 			log.Errorw("logical-device-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
 			return err
 		}
-		// Reset the logical device graph
-		go agent.generateDeviceGraph(context.Background())
+
+		// Remove the logical port from cache
+		agent.deleteLogicalPortsFromMap([]uint32{lPort.DevicePortNo})
+
+		// Reset the logical device routes
+		go func() {
+			if err := agent.buildRoutes(context.Background()); err != nil {
+				log.Warn("device-routes-not-ready", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "error": err})
+			}
+		}()
 	}
 	return nil
 }
@@ -1304,20 +1333,31 @@
 	defer agent.lockLogicalDevice.Unlock()
 
 	logicalDevice := agent.getLogicalDeviceWithoutLock()
-	updatedLPorts := []*voltha.LogicalPort{}
+	lPortstoKeep := []*voltha.LogicalPort{}
+	lPortsNoToDelete := []uint32{}
 	for _, logicalPort := range logicalDevice.Ports {
 		if logicalPort.DeviceId != deviceID {
-			updatedLPorts = append(updatedLPorts, logicalPort)
+			lPortstoKeep = append(lPortstoKeep, logicalPort)
+		} else {
+			lPortsNoToDelete = append(lPortsNoToDelete, logicalPort.DevicePortNo)
 		}
 	}
-	logicalDevice.Ports = updatedLPorts
-	log.Debugw("updated-logical-ports", log.Fields{"ports": updatedLPorts})
+	logicalDevice.Ports = lPortstoKeep
+
+	log.Debugw("updated-logical-ports", log.Fields{"ports": lPortstoKeep})
 	if err := agent.updateLogicalDeviceWithoutLock(ctx, logicalDevice); err != nil {
 		log.Errorw("logical-device-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
 		return err
 	}
-	// Reset the logical device graph
-	go agent.generateDeviceGraph(context.Background())
+	// Remove the port from the cached logical ports set
+	agent.deleteLogicalPortsFromMap(lPortsNoToDelete)
+
+	// Reset the logical device routes
+	go func() {
+		if err := agent.buildRoutes(context.Background()); err != nil {
+			log.Warn("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+		}
+	}()
 
 	return nil
 }
@@ -1364,22 +1404,21 @@
 	return status.Errorf(codes.NotFound, "Port %s on Logical Device %s", lPortID, agent.logicalDeviceID)
 }
 
-func (agent *LogicalDeviceAgent) getPreCalculatedRoute(ingress, egress uint32) []graph.RouteHop {
-	log.Debugw("ROUTE", log.Fields{"len": len(agent.deviceGraph.Routes)})
-	for routeLink, route := range agent.deviceGraph.Routes {
+func (agent *LogicalDeviceAgent) getPreCalculatedRoute(ingress, egress uint32) ([]route.Hop, error) {
+	log.Debugw("ROUTE", log.Fields{"len": len(agent.deviceRoutes.Routes)})
+	for routeLink, route := range agent.deviceRoutes.Routes {
 		log.Debugw("ROUTELINKS", log.Fields{"ingress": ingress, "egress": egress, "routelink": routeLink})
 		if ingress == routeLink.Ingress && egress == routeLink.Egress {
-			return route
+			return route, nil
 		}
 	}
-	log.Warnw("no-route", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "ingress": ingress, "egress": egress})
-	return nil
+	return nil, status.Errorf(codes.FailedPrecondition, "no route from:%d to:%d", ingress, egress)
 }
 
 // GetRoute returns route
-func (agent *LogicalDeviceAgent) GetRoute(ingressPortNo uint32, egressPortNo uint32) []graph.RouteHop {
+func (agent *LogicalDeviceAgent) GetRoute(ctx context.Context, ingressPortNo uint32, egressPortNo uint32) ([]route.Hop, error) {
 	log.Debugw("getting-route", log.Fields{"ingress-port": ingressPortNo, "egress-port": egressPortNo})
-	routes := make([]graph.RouteHop, 0)
+	routes := make([]route.Hop, 0)
 
 	// Note: A port value of 0 is equivalent to a nil port
 
@@ -1388,30 +1427,29 @@
 		log.Debugw("controller-flow", log.Fields{"ingressPortNo": ingressPortNo, "egressPortNo": egressPortNo, "logicalPortsNo": agent.logicalPortsNo})
 		if agent.isNNIPort(ingressPortNo) {
 			//This is a trap on the NNI Port
-			if len(agent.deviceGraph.Routes) == 0 {
+			if len(agent.deviceRoutes.Routes) == 0 {
 				// If there are no routes set (usually when the logical device has only NNI port(s), then just return an
 				// route with same IngressHop and EgressHop
-				hop := graph.RouteHop{DeviceID: agent.rootDeviceID, Ingress: ingressPortNo, Egress: ingressPortNo}
+				hop := route.Hop{DeviceID: agent.rootDeviceID, Ingress: ingressPortNo, Egress: ingressPortNo}
 				routes = append(routes, hop)
 				routes = append(routes, hop)
-				return routes
+				return routes, nil
 			}
 			//Return a 'half' route to make the flow decomposer logic happy
-			for routeLink, route := range agent.deviceGraph.Routes {
+			for routeLink, path := range agent.deviceRoutes.Routes {
 				if agent.isNNIPort(routeLink.Egress) {
-					routes = append(routes, graph.RouteHop{}) // first hop is set to empty
-					routes = append(routes, route[1])
-					return routes
+					routes = append(routes, route.Hop{}) // first hop is set to empty
+					routes = append(routes, path[1])
+					return routes, nil
 				}
 			}
-			log.Warnw("no-upstream-route", log.Fields{"ingressPortNo": ingressPortNo, "egressPortNo": egressPortNo, "logicalPortsNo": agent.logicalPortsNo})
-			return nil
+			return nil, status.Errorf(codes.FailedPrecondition, "no upstream route from:%d to:%d", ingressPortNo, egressPortNo)
 		}
 		//treat it as if the output port is the first NNI of the OLT
 		var err error
 		if egressPortNo, err = agent.getFirstNNIPort(); err != nil {
 			log.Warnw("no-nni-port", log.Fields{"error": err})
-			return nil
+			return nil, err
 		}
 	}
 	//If ingress port is not specified (nil), it may be a wildcarded
@@ -1420,27 +1458,25 @@
 	//hop is filled, the first hop is nil
 	if ingressPortNo == 0 && agent.isNNIPort(egressPortNo) {
 		// We can use the 2nd hop of any upstream route, so just find the first upstream:
-		for routeLink, route := range agent.deviceGraph.Routes {
+		for routeLink, path := range agent.deviceRoutes.Routes {
 			if agent.isNNIPort(routeLink.Egress) {
-				routes = append(routes, graph.RouteHop{}) // first hop is set to empty
-				routes = append(routes, route[1])
-				return routes
+				routes = append(routes, route.Hop{}) // first hop is set to empty
+				routes = append(routes, path[1])
+				return routes, nil
 			}
 		}
-		log.Warnw("no-upstream-route", log.Fields{"ingressPortNo": ingressPortNo, "egressPortNo": egressPortNo, "logicalPortsNo": agent.logicalPortsNo})
-		return nil
+		return nil, status.Errorf(codes.FailedPrecondition, "no upstream route from:%d to:%d", ingressPortNo, egressPortNo)
 	}
 	//If egress port is not specified (nil), we can also can return a "half" route
 	if egressPortNo == 0 {
-		for routeLink, route := range agent.deviceGraph.Routes {
+		for routeLink, path := range agent.deviceRoutes.Routes {
 			if routeLink.Ingress == ingressPortNo {
-				routes = append(routes, route[0])
-				routes = append(routes, graph.RouteHop{})
-				return routes
+				routes = append(routes, path[0])
+				routes = append(routes, route.Hop{})
+				return routes, nil
 			}
 		}
-		log.Warnw("no-downstream-route", log.Fields{"ingressPortNo": ingressPortNo, "egressPortNo": egressPortNo, "logicalPortsNo": agent.logicalPortsNo})
-		return nil
+		return nil, status.Errorf(codes.FailedPrecondition, "no downstream route from:%d to:%d", ingressPortNo, egressPortNo)
 	}
 	//	Return the pre-calculated route
 	return agent.getPreCalculatedRoute(ingressPortNo, egressPortNo)
@@ -1464,53 +1500,47 @@
 	return lPorts
 }
 
-// GetDeviceGraph returns device graph
-func (agent *LogicalDeviceAgent) GetDeviceGraph() *graph.DeviceGraph {
-	return agent.deviceGraph
+// GetDeviceRoutes returns device graph
+func (agent *LogicalDeviceAgent) GetDeviceRoutes() *route.DeviceRoutes {
+	return agent.deviceRoutes
 }
 
-//updateRoutes rebuilds the device graph if not done already
-func (agent *LogicalDeviceAgent) updateRoutes(ctx context.Context, device *voltha.Device, port *voltha.Port) error {
-	log.Debugf("updateRoutes", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "device": device.Id, "port": port})
+//rebuildRoutes rebuilds the device routes
+func (agent *LogicalDeviceAgent) buildRoutes(ctx context.Context) error {
+	log.Debugf("building-routes", log.Fields{"logical-device-id": agent.logicalDeviceID})
 	agent.lockLogicalDevice.Lock()
 	defer agent.lockLogicalDevice.Unlock()
-	if agent.deviceGraph == nil {
-		agent.deviceGraph = graph.NewDeviceGraph(agent.logicalDeviceID, agent.deviceMgr.GetDevice)
+	if agent.deviceRoutes == nil {
+		agent.deviceRoutes = route.NewDeviceRoutes(agent.logicalDeviceID, agent.deviceMgr.GetDevice)
 	}
 	// Get all the logical ports on that logical device
 	lDevice := agent.getLogicalDeviceWithoutLock()
 
-	//TODO:  Find a better way to refresh only missing routes
-	agent.deviceGraph.ComputeRoutes(ctx, lDevice.Ports)
-	agent.deviceGraph.Print()
+	if err := agent.deviceRoutes.ComputeRoutes(ctx, lDevice.Ports); err != nil {
+		return err
+	}
+	if err := agent.deviceRoutes.Print(); err != nil {
+		return err
+	}
+
 	return nil
 }
 
-//updateDeviceGraph updates the device graph if not done already and setup the default rules as well
-func (agent *LogicalDeviceAgent) updateDeviceGraph(ctx context.Context, lp *voltha.LogicalPort) {
-	log.Debugf("updateDeviceGraph", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+//updateRoutes updates the device routes
+func (agent *LogicalDeviceAgent) updateRoutes(ctx context.Context, lp *voltha.LogicalPort) error {
+	log.Debugw("updateRoutes", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
 	agent.lockLogicalDevice.Lock()
 	defer agent.lockLogicalDevice.Unlock()
-	if agent.deviceGraph == nil {
-		agent.deviceGraph = graph.NewDeviceGraph(agent.logicalDeviceID, agent.deviceMgr.GetDevice)
+	if agent.deviceRoutes == nil {
+		agent.deviceRoutes = route.NewDeviceRoutes(agent.logicalDeviceID, agent.deviceMgr.GetDevice)
 	}
-	agent.deviceGraph.AddPort(ctx, lp)
-	agent.deviceGraph.Print()
-}
-
-//generateDeviceGraph regenerates the device graph
-func (agent *LogicalDeviceAgent) generateDeviceGraph(ctx context.Context) {
-	log.Debugw("generateDeviceGraph", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
-	agent.lockLogicalDevice.Lock()
-	defer agent.lockLogicalDevice.Unlock()
-	// Get the latest logical device
-	ld := agent.getLogicalDeviceWithoutLock()
-	log.Debugw("generating-graph", log.Fields{"lDeviceId": agent.logicalDeviceID, "lPorts": len(ld.Ports)})
-	if agent.deviceGraph == nil {
-		agent.deviceGraph = graph.NewDeviceGraph(agent.logicalDeviceID, agent.deviceMgr.GetDevice)
+	if err := agent.deviceRoutes.AddPort(ctx, lp, agent.logicalDevice.Ports); err != nil {
+		return err
 	}
-	agent.deviceGraph.ComputeRoutes(ctx, ld.Ports)
-	agent.deviceGraph.Print()
+	if err := agent.deviceRoutes.Print(); err != nil {
+		return err
+	}
+	return nil
 }
 
 // diff go over two lists of logical ports and return what's new, what's changed and what's removed.
@@ -1649,9 +1679,13 @@
 		return false, err
 	}
 
-	// Update the device graph with this new logical port
+	// Update the device routes with this new logical port
 	clonedLP := (proto.Clone(lp)).(*voltha.LogicalPort)
-	go agent.updateDeviceGraph(context.Background(), clonedLP)
+	go func() {
+		if err := agent.updateRoutes(context.Background(), clonedLP); err != nil {
+			log.Warn("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "logical-port": lp.OfpPort.PortNo, "error": err})
+		}
+	}()
 
 	return true, nil
 }
@@ -1716,7 +1750,13 @@
 	}
 	// Update the device graph with this new logical port
 	clonedLP := (proto.Clone(portCap.Port)).(*voltha.LogicalPort)
-	go agent.updateDeviceGraph(context.Background(), clonedLP)
+
+	go func() {
+		if err := agent.updateRoutes(context.Background(), clonedLP); err != nil {
+			log.Warn("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+		}
+	}()
+
 	return true, nil
 }
 
@@ -1752,6 +1792,14 @@
 	}
 }
 
+func (agent *LogicalDeviceAgent) deleteLogicalPortsFromMap(portsNo []uint32) {
+	agent.lockLogicalPortsNo.Lock()
+	defer agent.lockLogicalPortsNo.Unlock()
+	for _, pNo := range portsNo {
+		delete(agent.logicalPortsNo, pNo)
+	}
+}
+
 func (agent *LogicalDeviceAgent) addLogicalPortsToMap(lps []*voltha.LogicalPort) {
 	agent.lockLogicalPortsNo.Lock()
 	defer agent.lockLogicalPortsNo.Unlock()
diff --git a/rw_core/coreif/logical_device_agent_if.go b/rw_core/coreif/logical_device_agent_if.go
index 734e024..0670bdf 100644
--- a/rw_core/coreif/logical_device_agent_if.go
+++ b/rw_core/coreif/logical_device_agent_if.go
@@ -21,15 +21,16 @@
 package coreif
 
 import (
-	"github.com/opencord/voltha-go/rw_core/graph"
+	"context"
+	"github.com/opencord/voltha-go/rw_core/route"
 	"github.com/opencord/voltha-protos/v3/go/voltha"
 )
 
 // LogicalDeviceAgent represents a generic agent
 type LogicalDeviceAgent interface {
 	GetLogicalDevice() *voltha.LogicalDevice
-	GetDeviceGraph() *graph.DeviceGraph
+	GetDeviceRoutes() *route.DeviceRoutes
 	GetWildcardInputPorts(excludePort ...uint32) []uint32
-	GetRoute(ingressPortNo uint32, egressPortNo uint32) []graph.RouteHop
+	GetRoute(ctx context.Context, ingressPortNo uint32, egressPortNo uint32) ([]route.Hop, error)
 	GetNNIPorts() []uint32
 }
diff --git a/rw_core/flowdecomposition/flow_decomposer.go b/rw_core/flowdecomposition/flow_decomposer.go
index 5819705..9e996dc 100644
--- a/rw_core/flowdecomposition/flow_decomposer.go
+++ b/rw_core/flowdecomposition/flow_decomposer.go
@@ -20,11 +20,13 @@
 	"context"
 	"github.com/gogo/protobuf/proto"
 	"github.com/opencord/voltha-go/rw_core/coreif"
-	"github.com/opencord/voltha-go/rw_core/graph"
+	"github.com/opencord/voltha-go/rw_core/route"
 	fu "github.com/opencord/voltha-lib-go/v3/pkg/flows"
 	"github.com/opencord/voltha-lib-go/v3/pkg/log"
 	ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
 	"github.com/opencord/voltha-protos/v3/go/voltha"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
 )
 
 func init() {
@@ -47,7 +49,7 @@
 }
 
 //DecomposeRules decomposes per-device flows and flow-groups from the flows and groups defined on a logical device
-func (fd *FlowDecomposer) DecomposeRules(ctx context.Context, agent coreif.LogicalDeviceAgent, flows ofp.Flows, groups ofp.FlowGroups) *fu.DeviceRules {
+func (fd *FlowDecomposer) DecomposeRules(ctx context.Context, agent coreif.LogicalDeviceAgent, flows ofp.Flows, groups ofp.FlowGroups) (*fu.DeviceRules, error) {
 	deviceRules := *fu.NewDeviceRules()
 	devicesToUpdate := make(map[string]string)
 
@@ -56,16 +58,18 @@
 		groupMap[groupEntry.Desc.GroupId] = groupEntry
 	}
 
-	var decomposedRules *fu.DeviceRules
 	for _, flow := range flows.Items {
-		decomposedRules = fd.decomposeFlow(ctx, agent, flow, groupMap)
+		decomposedRules, err := fd.decomposeFlow(ctx, agent, flow, groupMap)
+		if err != nil {
+			return nil, err
+		}
 		for deviceID, flowAndGroups := range decomposedRules.Rules {
 			deviceRules.CreateEntryIfNotExist(deviceID)
 			deviceRules.Rules[deviceID].AddFrom(flowAndGroups)
 			devicesToUpdate[deviceID] = deviceID
 		}
 	}
-	return deviceRules.FilterRules(devicesToUpdate)
+	return deviceRules.FilterRules(devicesToUpdate), nil
 }
 
 // Handles special case of any controller-bound flow for a parent device
@@ -106,7 +110,7 @@
 }
 
 //processControllerBoundFlow decomposes trap flows
-func (fd *FlowDecomposer) processControllerBoundFlow(agent coreif.LogicalDeviceAgent, route []graph.RouteHop,
+func (fd *FlowDecomposer) processControllerBoundFlow(ctx context.Context, agent coreif.LogicalDeviceAgent, path []route.Hop,
 	inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats) *fu.DeviceRules {
 
 	log.Debugw("trap-flow", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo, "flow": flow})
@@ -114,11 +118,11 @@
 	meterID := fu.GetMeterIdFromFlow(flow)
 	metadataFromwriteMetadata := fu.GetMetadataFromWriteMetadataAction(flow)
 
-	ingressHop := route[0]
-	egressHop := route[1]
+	ingressHop := path[0]
+	egressHop := path[1]
 
 	//case of packet_in from NNI port rule
-	if agent.GetDeviceGraph().IsRootPort(inPortNo) {
+	if agent.GetDeviceRoutes().IsRootPort(inPortNo) {
 		// Trap flow for NNI port
 		log.Debug("trap-nni")
 
@@ -212,8 +216,8 @@
 // upstream needs to get Q-in-Q treatment and that this is expressed via two flow rules, the first using the
 // goto-statement. We also assume that the inner tag is applied at the ONU, while the outer tag is
 // applied at the OLT
-func (fd *FlowDecomposer) processUpstreamNonControllerBoundFlow(agent coreif.LogicalDeviceAgent,
-	route []graph.RouteHop, inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats) *fu.DeviceRules {
+func (fd *FlowDecomposer) processUpstreamNonControllerBoundFlow(ctx context.Context,
+	path []route.Hop, inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats) *fu.DeviceRules {
 
 	log.Debugw("upstream-non-controller-bound-flow", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo})
 	deviceRules := fu.NewDeviceRules()
@@ -221,8 +225,8 @@
 	meterID := fu.GetMeterIdFromFlow(flow)
 	metadataFromwriteMetadata := fu.GetMetadataFromWriteMetadataAction(flow)
 
-	ingressHop := route[0]
-	egressHop := route[1]
+	ingressHop := path[0]
+	egressHop := path[1]
 
 	if flow.TableId == 0 && fu.HasNextTable(flow) {
 		log.Debugw("decomposing-onu-flow-in-upstream-has-next-table", log.Fields{"table_id": flow.TableId})
@@ -272,7 +276,7 @@
 }
 
 // processDownstreamFlowWithNextTable decomposes downstream flows containing next table ID instructions
-func (fd *FlowDecomposer) processDownstreamFlowWithNextTable(agent coreif.LogicalDeviceAgent, route []graph.RouteHop,
+func (fd *FlowDecomposer) processDownstreamFlowWithNextTable(ctx context.Context, agent coreif.LogicalDeviceAgent, path []route.Hop,
 	inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats) *fu.DeviceRules {
 	log.Debugw("decomposing-olt-flow-in-downstream-flow-with-next-table", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo})
 	deviceRules := fu.NewDeviceRules()
@@ -289,13 +293,17 @@
 		return deviceRules
 	}
 
-	ingressHop := route[0]
-	egressHop := route[1]
+	ingressHop := path[0]
+	egressHop := path[1]
 	if metadataFromwriteMetadata != 0 {
 		log.Debugw("creating-metadata-flow", log.Fields{"flow": flow})
 		portNumber := fu.GetEgressPortNumberFromWriteMetadata(flow)
 		if portNumber != 0 {
-			recalculatedRoute := agent.GetRoute(inPortNo, portNumber)
+			recalculatedRoute, err := agent.GetRoute(ctx, inPortNo, portNumber)
+			if err != nil {
+				log.Errorw("no-route-double-tag", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo, "metadata": metadataFromwriteMetadata, "error": err})
+				return deviceRules
+			}
 			switch len(recalculatedRoute) {
 			case 0:
 				log.Errorw("no-route-double-tag", log.Fields{"inPortNo": inPortNo, "outPortNo": portNumber, "comment": "deleting-flow", "metadata": metadataFromwriteMetadata})
@@ -304,7 +312,7 @@
 			case 2:
 				log.Debugw("route-found", log.Fields{"ingressHop": ingressHop, "egressHop": egressHop})
 			default:
-				log.Errorw("invalid-route-length", log.Fields{"routeLen": len(route)})
+				log.Errorw("invalid-route-length", log.Fields{"routeLen": len(path)})
 				return deviceRules
 			}
 			ingressHop = recalculatedRoute[0]
@@ -358,13 +366,13 @@
 }
 
 // processUnicastFlow decomposes unicast flows
-func (fd *FlowDecomposer) processUnicastFlow(agent coreif.LogicalDeviceAgent, route []graph.RouteHop,
+func (fd *FlowDecomposer) processUnicastFlow(ctx context.Context, path []route.Hop,
 	inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats) *fu.DeviceRules {
 
 	log.Debugw("decomposing-onu-flow-in-downstream-unicast-flow", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo})
 	deviceRules := fu.NewDeviceRules()
 
-	egressHop := route[1]
+	egressHop := path[1]
 
 	meterID := fu.GetMeterIdFromFlow(flow)
 	metadataFromwriteMetadata := fu.GetMetadataFromWriteMetadataAction(flow)
@@ -389,7 +397,7 @@
 }
 
 // processMulticastFlow decompose multicast flows
-func (fd *FlowDecomposer) processMulticastFlow(agent coreif.LogicalDeviceAgent, route []graph.RouteHop,
+func (fd *FlowDecomposer) processMulticastFlow(ctx context.Context, path []route.Hop,
 	inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats, grpID uint32,
 	groupMap map[uint32]*ofp.OfpGroupEntry) *fu.DeviceRules {
 
@@ -408,71 +416,71 @@
 		return deviceRules
 	}
 
-	deviceRules.CreateEntryIfNotExist(route[0].DeviceID)
+	deviceRules.CreateEntryIfNotExist(path[0].DeviceID)
 	fg := fu.NewFlowsAndGroups()
 	fg.AddFlow(flow)
 	//return the multicast flow without decomposing it
-	deviceRules.AddFlowsAndGroup(route[0].DeviceID, fg)
+	deviceRules.AddFlowsAndGroup(path[0].DeviceID, fg)
 	return deviceRules
 }
 
 // decomposeFlow decomposes a flow for a logical device into flows for each physical device
 func (fd *FlowDecomposer) decomposeFlow(ctx context.Context, agent coreif.LogicalDeviceAgent, flow *ofp.OfpFlowStats,
-	groupMap map[uint32]*ofp.OfpGroupEntry) *fu.DeviceRules {
+	groupMap map[uint32]*ofp.OfpGroupEntry) (*fu.DeviceRules, error) {
 
 	inPortNo := fu.GetInPort(flow)
 	if fu.HasGroup(flow) && inPortNo == 0 {
 		//if no in-port specified for a multicast flow, put NNI port as in-port
-		//so that a valid route can be found for the flow
+		//so that a valid path can be found for the flow
 		nniPorts := agent.GetNNIPorts()
 		if len(nniPorts) > 0 {
 			inPortNo = nniPorts[0]
-			log.Debugw("Assigning NNI port as in-port for the multicast flow", log.Fields{"nni": nniPorts[0], "flow:": flow})
+			log.Debugw("assigning-nni-port-as-in-port-for-multicast-flow", log.Fields{"nni": nniPorts[0], "flow:": flow})
 		}
 	}
 	outPortNo := fu.GetOutPort(flow)
 	deviceRules := fu.NewDeviceRules()
-	route := agent.GetRoute(inPortNo, outPortNo)
+	path, err := agent.GetRoute(ctx, inPortNo, outPortNo)
+	if err != nil {
+		log.Errorw("no-route", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo, "error": err})
+		return deviceRules, err
+	}
 
-	switch len(route) {
+	switch len(path) {
 	case 0:
-		log.Errorw("no-route", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo, "comment": "deleting-flow"})
-		//	TODO: Delete flow
-		return deviceRules
+		return deviceRules, status.Errorf(codes.FailedPrecondition, "no route from:%d to:%d", inPortNo, outPortNo)
 	case 2:
-		log.Debugw("route-found", log.Fields{"ingressHop": route[0], "egressHop": route[1]})
+		log.Debugw("route-found", log.Fields{"ingressHop": path[0], "egressHop": path[1]})
 	default:
-		log.Errorw("invalid-route-length", log.Fields{"routeLen": len(route)})
-		return deviceRules
+		return deviceRules, status.Errorf(codes.Aborted, "invalid route length %d", len(path))
 	}
 
 	// Process controller bound flow
 	if outPortNo != 0 && (outPortNo&0x7fffffff) == uint32(ofp.OfpPortNo_OFPP_CONTROLLER) {
-		deviceRules = fd.processControllerBoundFlow(agent, route, inPortNo, outPortNo, flow)
+		deviceRules = fd.processControllerBoundFlow(ctx, agent, path, inPortNo, outPortNo, flow)
 	} else {
 		var ingressDevice *voltha.Device
 		var err error
-		if ingressDevice, err = fd.deviceMgr.GetDevice(ctx, route[0].DeviceID); err != nil {
-			log.Errorw("ingress-device-not-found", log.Fields{"deviceId": route[0].DeviceID, "flow": flow})
-			return deviceRules
+		if ingressDevice, err = fd.deviceMgr.GetDevice(ctx, path[0].DeviceID); err != nil {
+			return deviceRules, err
 		}
 		isUpstream := !ingressDevice.Root
 		if isUpstream { // Unicast OLT and ONU UL
-			log.Info("processOltAndOnuUpstreamNonControllerBoundUnicastFlows", log.Fields{"flows": flow})
-			deviceRules = fd.processUpstreamNonControllerBoundFlow(agent, route, inPortNo, outPortNo, flow)
+			log.Debug("process-olt-nd-onu-upstream-noncontrollerbound-unicast-flows", log.Fields{"flows": flow})
+			deviceRules = fd.processUpstreamNonControllerBoundFlow(ctx, path, inPortNo, outPortNo, flow)
 		} else if fu.HasNextTable(flow) && flow.TableId == 0 { // Unicast OLT flow DL
-			log.Debugw("processOltDownstreamNonControllerBoundFlowWithNextTable", log.Fields{"flows": flow})
-			deviceRules = fd.processDownstreamFlowWithNextTable(agent, route, inPortNo, outPortNo, flow)
+			log.Debugw("process-olt-downstream-noncontrollerbound-flow-with-nexttable", log.Fields{"flows": flow})
+			deviceRules = fd.processDownstreamFlowWithNextTable(ctx, agent, path, inPortNo, outPortNo, flow)
 		} else if flow.TableId == 1 && outPortNo != 0 { // Unicast ONU flow DL
-			log.Debugw("processOnuDownstreamUnicastFlow", log.Fields{"flows": flow})
-			deviceRules = fd.processUnicastFlow(agent, route, inPortNo, outPortNo, flow)
+			log.Debugw("process-onu-downstream-unicast-flow", log.Fields{"flows": flow})
+			deviceRules = fd.processUnicastFlow(ctx, path, inPortNo, outPortNo, flow)
 		} else if grpID := fu.GetGroup(flow); grpID != 0 && flow.TableId == 0 { //Multicast
-			log.Debugw("processMulticastFlow", log.Fields{"flows": flow})
-			deviceRules = fd.processMulticastFlow(agent, route, inPortNo, outPortNo, flow, grpID, groupMap)
+			log.Debugw("process-multicast-flow", log.Fields{"flows": flow})
+			deviceRules = fd.processMulticastFlow(ctx, path, inPortNo, outPortNo, flow, grpID, groupMap)
 		} else {
-			log.Errorw("unknown-downstream-flow", log.Fields{"flow": *flow})
+			return deviceRules, status.Errorf(codes.Aborted, "unknown downstream flow %v", *flow)
 		}
 	}
 	deviceRules = fd.updateOutputPortForControllerBoundFlowForParentDevide(flow, deviceRules)
-	return deviceRules
+	return deviceRules, nil
 }
diff --git a/rw_core/flowdecomposition/flow_decomposer_test.go b/rw_core/flowdecomposition/flow_decomposer_test.go
index 8e2d9f3..29c1a6a 100644
--- a/rw_core/flowdecomposition/flow_decomposer_test.go
+++ b/rw_core/flowdecomposition/flow_decomposer_test.go
@@ -18,13 +18,15 @@
 import (
 	"context"
 	"errors"
-	"github.com/opencord/voltha-go/rw_core/graph"
 	"github.com/opencord/voltha-go/rw_core/mocks"
+	"github.com/opencord/voltha-go/rw_core/route"
 	fu "github.com/opencord/voltha-lib-go/v3/pkg/flows"
 	"github.com/opencord/voltha-lib-go/v3/pkg/log"
 	ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
 	"github.com/opencord/voltha-protos/v3/go/voltha"
 	"github.com/stretchr/testify/assert"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
 
 	"testing"
 )
@@ -116,9 +118,9 @@
 type testFlowDecomposer struct {
 	dMgr           *testDeviceManager
 	logicalPorts   map[uint32]*voltha.LogicalPort
-	routes         map[graph.OFPortLink][]graph.RouteHop
+	routes         map[route.OFPortLink][]route.Hop
 	defaultRules   *fu.DeviceRules
-	deviceGraph    *graph.DeviceGraph
+	deviceRoutes   *route.DeviceRoutes
 	fd             *FlowDecomposer
 	logicalPortsNo map[uint32]bool
 }
@@ -141,11 +143,11 @@
 	tfd.logicalPortsNo[10] = false
 	tfd.logicalPortsNo[65536] = true // nni
 
-	tfd.routes = make(map[graph.OFPortLink][]graph.RouteHop)
+	tfd.routes = make(map[route.OFPortLink][]route.Hop)
 
 	//DOWNSTREAM ROUTES
 
-	tfd.routes[graph.OFPortLink{Ingress: 10, Egress: 1}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 10, Egress: 1}] = []route.Hop{
 		{
 			DeviceID: "olt",
 			Ingress:  tfd.dMgr.devices["olt"].Ports[1].PortNo,
@@ -158,7 +160,7 @@
 		},
 	}
 
-	tfd.routes[graph.OFPortLink{Ingress: 10, Egress: 2}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 10, Egress: 2}] = []route.Hop{
 		{
 			DeviceID: "olt",
 			Ingress:  tfd.dMgr.devices["olt"].Ports[1].PortNo,
@@ -170,7 +172,7 @@
 			Egress:   tfd.dMgr.devices["onu2"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 10, Egress: 3}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 10, Egress: 3}] = []route.Hop{
 		{
 			DeviceID: "olt",
 			Ingress:  tfd.dMgr.devices["olt"].Ports[1].PortNo,
@@ -182,7 +184,7 @@
 			Egress:   tfd.dMgr.devices["onu3"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 10, Egress: 4}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 10, Egress: 4}] = []route.Hop{
 		{
 			DeviceID: "olt",
 			Ingress:  tfd.dMgr.devices["olt"].Ports[1].PortNo,
@@ -194,7 +196,7 @@
 			Egress:   tfd.dMgr.devices["onu4"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 10, Egress: 10}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 10, Egress: 10}] = []route.Hop{
 		{
 			DeviceID: "olt",
 			Ingress:  tfd.dMgr.devices["olt"].Ports[1].PortNo,
@@ -209,7 +211,7 @@
 
 	//UPSTREAM DATA PLANE
 
-	tfd.routes[graph.OFPortLink{Ingress: 1, Egress: 10}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 1, Egress: 10}] = []route.Hop{
 		{
 			DeviceID: "onu1",
 			Ingress:  tfd.dMgr.devices["onu1"].Ports[1].PortNo,
@@ -221,7 +223,7 @@
 			Egress:   tfd.dMgr.devices["olt"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 2, Egress: 10}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 2, Egress: 10}] = []route.Hop{
 		{
 			DeviceID: "onu2",
 			Ingress:  tfd.dMgr.devices["onu2"].Ports[1].PortNo,
@@ -233,7 +235,7 @@
 			Egress:   tfd.dMgr.devices["olt"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 3, Egress: 10}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 3, Egress: 10}] = []route.Hop{
 		{
 			DeviceID: "onu3",
 			Ingress:  tfd.dMgr.devices["onu3"].Ports[1].PortNo,
@@ -245,7 +247,7 @@
 			Egress:   tfd.dMgr.devices["olt"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 4, Egress: 10}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 4, Egress: 10}] = []route.Hop{
 		{
 			DeviceID: "onu4",
 			Ingress:  tfd.dMgr.devices["onu4"].Ports[1].PortNo,
@@ -261,7 +263,7 @@
 	//UPSTREAM NEXT TABLE BASED
 
 	// openflow port 0 means absence of a port - go/protobuf interpretation
-	tfd.routes[graph.OFPortLink{Ingress: 1, Egress: 0}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 1, Egress: 0}] = []route.Hop{
 		{
 			DeviceID: "onu1",
 			Ingress:  tfd.dMgr.devices["onu1"].Ports[1].PortNo,
@@ -273,7 +275,7 @@
 			Egress:   tfd.dMgr.devices["olt"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 2, Egress: 0}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 2, Egress: 0}] = []route.Hop{
 		{
 			DeviceID: "onu2",
 			Ingress:  tfd.dMgr.devices["onu2"].Ports[1].PortNo,
@@ -285,7 +287,7 @@
 			Egress:   tfd.dMgr.devices["olt"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 3, Egress: 0}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 3, Egress: 0}] = []route.Hop{
 		{
 			DeviceID: "onu3",
 			Ingress:  tfd.dMgr.devices["onu3"].Ports[1].PortNo,
@@ -297,7 +299,7 @@
 			Egress:   tfd.dMgr.devices["olt"].Ports[1].PortNo,
 		},
 	}
-	tfd.routes[graph.OFPortLink{Ingress: 4, Egress: 0}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 4, Egress: 0}] = []route.Hop{
 		{
 			DeviceID: "onu4",
 			Ingress:  tfd.dMgr.devices["onu4"].Ports[1].PortNo,
@@ -312,7 +314,7 @@
 
 	// DOWNSTREAM NEXT TABLE BASED
 
-	tfd.routes[graph.OFPortLink{Ingress: 10, Egress: 0}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 10, Egress: 0}] = []route.Hop{
 		{
 			DeviceID: "olt",
 			Ingress:  tfd.dMgr.devices["olt"].Ports[1].PortNo,
@@ -321,7 +323,7 @@
 		{}, // 2nd hop is not known yet
 	}
 
-	tfd.routes[graph.OFPortLink{Ingress: 0, Egress: 10}] = []graph.RouteHop{
+	tfd.routes[route.OFPortLink{Ingress: 0, Egress: 10}] = []route.Hop{
 		{}, // 1st hop is wildcard
 		{
 			DeviceID: "olt",
@@ -390,9 +392,9 @@
 	tfd.defaultRules.AddFlowsAndGroup("onu4", fg)
 
 	//Set up the device graph - flow decomposer uses it only to verify whether a port is a root port.
-	tfd.deviceGraph = graph.NewDeviceGraph("ldid", tfd.getDeviceHelper)
-	tfd.deviceGraph.RootPorts = make(map[uint32]uint32)
-	tfd.deviceGraph.RootPorts[10] = 10
+	tfd.deviceRoutes = route.NewDeviceRoutes("ldid", tfd.getDeviceHelper)
+	tfd.deviceRoutes.RootPorts = make(map[uint32]uint32)
+	tfd.deviceRoutes.RootPorts[10] = 10
 
 	tfd.fd = NewFlowDecomposer(tfd.dMgr)
 
@@ -411,8 +413,8 @@
 	return nil
 }
 
-func (tfd *testFlowDecomposer) GetDeviceGraph() *graph.DeviceGraph {
-	return tfd.deviceGraph
+func (tfd *testFlowDecomposer) GetDeviceRoutes() *route.DeviceRoutes {
+	return tfd.deviceRoutes
 }
 
 func (tfd *testFlowDecomposer) GetAllDefaultRules() *fu.DeviceRules {
@@ -433,8 +435,8 @@
 	return lPorts
 }
 
-func (tfd *testFlowDecomposer) GetRoute(ingressPortNo uint32, egressPortNo uint32) []graph.RouteHop {
-	var portLink graph.OFPortLink
+func (tfd *testFlowDecomposer) GetRoute(ctx context.Context, ingressPortNo uint32, egressPortNo uint32) ([]route.Hop, error) {
+	var portLink route.OFPortLink
 	if egressPortNo == 0 {
 		portLink.Egress = 0
 	} else if egressPortNo&0x7fffffff == uint32(ofp.OfpPortNo_OFPP_CONTROLLER) {
@@ -449,10 +451,10 @@
 	}
 	for key, val := range tfd.routes {
 		if key.Ingress == portLink.Ingress && key.Egress == portLink.Egress {
-			return val
+			return val, nil
 		}
 	}
-	return nil
+	return nil, status.Errorf(codes.FailedPrecondition, "no route from:%d to:%d", ingressPortNo, egressPortNo)
 }
 
 func (tfd *testFlowDecomposer) GetNNIPorts() []uint32 {
@@ -484,7 +486,8 @@
 	groups := ofp.FlowGroups{}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
 
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	onu1FlowAndGroup := deviceRules.Rules["onu1"]
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.Equal(t, 1, onu1FlowAndGroup.Flows.Len())
@@ -546,7 +549,8 @@
 	groups := ofp.FlowGroups{}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
 
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	onu1FlowAndGroup := deviceRules.Rules["onu1"]
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.Equal(t, 1, onu1FlowAndGroup.Flows.Len())
@@ -607,7 +611,8 @@
 	groups := ofp.FlowGroups{}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
 
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	onu1FlowAndGroup := deviceRules.Rules["onu1"]
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.Equal(t, 1, onu1FlowAndGroup.Flows.Len())
@@ -668,7 +673,8 @@
 	groups := ofp.FlowGroups{}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
 
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	onu1FlowAndGroup := deviceRules.Rules["onu1"]
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.Equal(t, 1, onu1FlowAndGroup.Flows.Len())
@@ -732,7 +738,8 @@
 	flows := ofp.Flows{Items: []*ofp.OfpFlowStats{fu.MkFlowStat(fa)}}
 	groups := ofp.FlowGroups{}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	onu1FlowAndGroup := deviceRules.Rules["onu1"]
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.Nil(t, onu1FlowAndGroup)
@@ -794,7 +801,8 @@
 	groups := ofp.FlowGroups{}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
 
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	onu1FlowAndGroup := deviceRules.Rules["onu1"]
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.NotNil(t, onu1FlowAndGroup)
@@ -896,8 +904,8 @@
 	groups := ofp.FlowGroups{}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
 
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
-
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	onu1FlowAndGroup := deviceRules.Rules["onu1"]
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.Equal(t, 1, onu1FlowAndGroup.Flows.Len())
@@ -983,7 +991,8 @@
 	groups := ofp.FlowGroups{Items: []*ofp.OfpGroupEntry{fu.MkGroupStat(ga)}}
 	tfd := newTestFlowDecomposer(newTestDeviceManager())
 
-	deviceRules := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	deviceRules, err := tfd.fd.DecomposeRules(context.Background(), tfd, flows, groups)
+	assert.Nil(t, err)
 	oltFlowAndGroup := deviceRules.Rules["olt"]
 	assert.Equal(t, 1, oltFlowAndGroup.Flows.Len())
 	assert.Equal(t, 0, oltFlowAndGroup.Groups.Len())
diff --git a/rw_core/graph/device_graph.go b/rw_core/graph/device_graph.go
deleted file mode 100644
index 630fb2f..0000000
--- a/rw_core/graph/device_graph.go
+++ /dev/null
@@ -1,511 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package graph
-
-import (
-	"context"
-	"fmt"
-	"strconv"
-	"strings"
-	"sync"
-
-	"github.com/gyuho/goraph"
-	"github.com/opencord/voltha-lib-go/v3/pkg/log"
-	"github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-func init() {
-	_, err := log.AddPackage(log.JSON, log.WarnLevel, nil)
-	if err != nil {
-		log.Errorw("unable-to-register-package-to-the-log-map", log.Fields{"error": err})
-	}
-}
-
-// RouteHop represent route hop attributes
-type RouteHop struct {
-	DeviceID string
-	Ingress  uint32
-	Egress   uint32
-}
-
-// OFPortLink represent of port link attributes
-type OFPortLink struct {
-	Ingress uint32
-	Egress  uint32
-}
-
-type ofPortLinkToPath struct {
-	link OFPortLink
-	path []RouteHop
-}
-
-// GetDeviceFunc returns device function
-type GetDeviceFunc func(ctx context.Context, id string) (*voltha.Device, error)
-
-// DeviceGraph represent device graph attributes
-type DeviceGraph struct {
-	logicalDeviceID    string
-	GGraph             goraph.Graph
-	getDeviceFromModel GetDeviceFunc
-	logicalPorts       []*voltha.LogicalPort
-	rootPortsString    map[string]uint32
-	nonRootPortsString map[string]uint32
-	RootPorts          map[uint32]uint32
-	rootPortsLock      sync.RWMutex
-	Routes             map[OFPortLink][]RouteHop
-	graphBuildLock     sync.RWMutex
-	boundaryPorts      map[string]uint32
-	boundaryPortsLock  sync.RWMutex
-	cachedDevices      map[string]*voltha.Device
-	cachedDevicesLock  sync.RWMutex
-	devicesAdded       map[string]string
-	portsAdded         map[string]string
-}
-
-// NewDeviceGraph creates device graph instance
-func NewDeviceGraph(logicalDeviceID string, getDevice GetDeviceFunc) *DeviceGraph {
-	var dg DeviceGraph
-	dg.logicalDeviceID = logicalDeviceID
-	dg.GGraph = goraph.NewGraph()
-	dg.getDeviceFromModel = getDevice
-	dg.graphBuildLock = sync.RWMutex{}
-	dg.cachedDevicesLock = sync.RWMutex{}
-	dg.rootPortsLock = sync.RWMutex{}
-	dg.devicesAdded = make(map[string]string)
-	dg.portsAdded = make(map[string]string)
-	dg.rootPortsString = make(map[string]uint32)
-	dg.nonRootPortsString = make(map[string]uint32)
-	dg.RootPorts = make(map[uint32]uint32)
-	dg.boundaryPorts = make(map[string]uint32)
-	dg.Routes = make(map[OFPortLink][]RouteHop)
-	dg.cachedDevices = make(map[string]*voltha.Device)
-	log.Debug("new device graph created ...")
-	return &dg
-}
-
-//IsRootPort returns true if the port is a root port on a logical device
-func (dg *DeviceGraph) IsRootPort(port uint32) bool {
-	dg.rootPortsLock.RLock()
-	defer dg.rootPortsLock.RUnlock()
-	_, exist := dg.RootPorts[port]
-	return exist
-}
-
-//GetDeviceNodeIds retrieves all the nodes in the device graph
-func (dg *DeviceGraph) GetDeviceNodeIds() map[string]string {
-	dg.graphBuildLock.RLock()
-	defer dg.graphBuildLock.RUnlock()
-	nodeIds := make(map[string]string)
-	nodesMap := dg.GGraph.GetNodes()
-	for id, node := range nodesMap {
-		if len(strings.Split(node.String(), ":")) != 2 { // not port node
-			nodeIds[id.String()] = id.String()
-		}
-	}
-	return nodeIds
-}
-
-//ComputeRoutes creates a device graph from the logical ports and then calculates all the routes
-//between the logical ports.  This will clear up the graph and routes if there were any.
-func (dg *DeviceGraph) ComputeRoutes(ctx context.Context, lps []*voltha.LogicalPort) {
-	if dg == nil || len(lps) == 0 {
-		return
-	}
-	dg.graphBuildLock.Lock()
-	defer dg.graphBuildLock.Unlock()
-
-	// Clear the graph
-	dg.reset()
-
-	dg.logicalPorts = lps
-
-	// Set the root, non-root ports and boundary ports
-	for _, lp := range lps {
-		portID := concatDeviceIDPortID(lp.DeviceId, lp.DevicePortNo)
-		if lp.RootPort {
-			dg.rootPortsString[portID] = lp.OfpPort.PortNo
-			dg.RootPorts[lp.OfpPort.PortNo] = lp.OfpPort.PortNo
-		} else {
-			dg.nonRootPortsString[portID] = lp.OfpPort.PortNo
-		}
-		dg.boundaryPorts[portID] = lp.OfpPort.PortNo
-	}
-
-	// Build the graph
-	var device *voltha.Device
-	for _, logicalPort := range dg.logicalPorts {
-		device, _ = dg.getDevice(ctx, logicalPort.DeviceId, false)
-		dg.GGraph = dg.addDevice(ctx, device, dg.GGraph, &dg.devicesAdded, &dg.portsAdded, dg.boundaryPorts)
-	}
-
-	dg.Routes = dg.buildRoutes()
-}
-
-// AddPort adds a port to the graph.  If the graph is empty it will just invoke ComputeRoutes function
-func (dg *DeviceGraph) AddPort(ctx context.Context, lp *voltha.LogicalPort) {
-	log.Debugw("Addport", log.Fields{"logicalPort": lp})
-	//  If the graph does not exist invoke ComputeRoutes.
-	if len(dg.boundaryPorts) == 0 {
-		dg.ComputeRoutes(ctx, []*voltha.LogicalPort{lp})
-		return
-	}
-
-	dg.graphBuildLock.Lock()
-	defer dg.graphBuildLock.Unlock()
-
-	portID := concatDeviceIDPortID(lp.DeviceId, lp.DevicePortNo)
-
-	//	If the port is already part of the boundary ports, do nothing
-	if dg.portExist(portID) {
-		return
-	}
-	// Add the port to the set of boundary ports
-	dg.boundaryPorts[portID] = lp.OfpPort.PortNo
-
-	// Add the device where this port is located to the device graph. If the device is already added then
-	// only the missing port will be added
-	device, _ := dg.getDevice(ctx, lp.DeviceId, false)
-	dg.GGraph = dg.addDevice(ctx, device, dg.GGraph, &dg.devicesAdded, &dg.portsAdded, dg.boundaryPorts)
-
-	if lp.RootPort {
-		// Compute the route from this root port to all non-root ports
-		dg.rootPortsString[portID] = lp.OfpPort.PortNo
-		dg.RootPorts[lp.OfpPort.PortNo] = lp.OfpPort.PortNo
-		dg.Routes = dg.buildPathsToAllNonRootPorts(lp)
-	} else {
-		// Compute the route from this port to all root ports
-		dg.nonRootPortsString[portID] = lp.OfpPort.PortNo
-		dg.Routes = dg.buildPathsToAllRootPorts(lp)
-	}
-
-	dg.Print()
-}
-
-// Print prints routes
-func (dg *DeviceGraph) Print() error {
-	log.Debugw("Print", log.Fields{"graph": dg.logicalDeviceID, "boundaryPorts": dg.boundaryPorts})
-	if level, err := log.GetPackageLogLevel(); err == nil && level == log.DebugLevel {
-		output := ""
-		routeNumber := 1
-		for k, v := range dg.Routes {
-			key := fmt.Sprintf("LP:%d->LP:%d", k.Ingress, k.Egress)
-			val := ""
-			for _, i := range v {
-				val += fmt.Sprintf("{%d->%s->%d},", i.Ingress, i.DeviceID, i.Egress)
-			}
-			val = val[:len(val)-1]
-			output += fmt.Sprintf("%d:{%s=>%s}   ", routeNumber, key, fmt.Sprintf("[%s]", val))
-			routeNumber++
-		}
-		if len(dg.Routes) == 0 {
-			log.Debugw("no-routes-found", log.Fields{"lDeviceId": dg.logicalDeviceID, "Graph": dg.GGraph.String()})
-		} else {
-			log.Debugw("graph_routes", log.Fields{"lDeviceId": dg.logicalDeviceID, "Routes": output})
-		}
-	}
-	return nil
-}
-
-// IsUpToDate returns true if device is up to date
-func (dg *DeviceGraph) IsUpToDate(ld *voltha.LogicalDevice) bool {
-	if ld != nil {
-		if len(dg.boundaryPorts) != len(ld.Ports) {
-			return false
-		}
-		var portID string
-		var val uint32
-		var exist bool
-		for _, lp := range ld.Ports {
-			portID = concatDeviceIDPortID(lp.DeviceId, lp.DevicePortNo)
-			if val, exist = dg.boundaryPorts[portID]; !exist || val != lp.OfpPort.PortNo {
-				return false
-			}
-		}
-		return true
-	}
-	return len(dg.boundaryPorts) == 0
-}
-
-//getDevice returns the device either from the local cache (default) or from the model.
-//TODO: Set a cache timeout such that we do not use invalid data.  The full device lifecycle should also
-//be taken in consideration
-func (dg *DeviceGraph) getDevice(ctx context.Context, id string, useCache bool) (*voltha.Device, error) {
-	if useCache {
-		dg.cachedDevicesLock.RLock()
-		if d, exist := dg.cachedDevices[id]; exist {
-			dg.cachedDevicesLock.RUnlock()
-			//log.Debugw("getDevice - returned from cache", log.Fields{"deviceId": id})
-			return d, nil
-		}
-		dg.cachedDevicesLock.RUnlock()
-	}
-	//	Not cached
-	d, err := dg.getDeviceFromModel(ctx, id)
-	if err != nil {
-		log.Errorw("device-not-found", log.Fields{"deviceId": id, "error": err})
-		return nil, err
-	}
-	// cache it
-	dg.cachedDevicesLock.Lock()
-	dg.cachedDevices[id] = d
-	dg.cachedDevicesLock.Unlock()
-	//log.Debugw("getDevice - returned from model", log.Fields{"deviceId": id})
-	return d, nil
-}
-
-// addDevice adds a device to a device graph and setup edges that represent the device connections to its peers
-func (dg *DeviceGraph) addDevice(ctx context.Context, device *voltha.Device, g goraph.Graph, devicesAdded *map[string]string, portsAdded *map[string]string,
-	boundaryPorts map[string]uint32) goraph.Graph {
-
-	if device == nil {
-		return g
-	}
-
-	log.Debugw("Adding-device", log.Fields{"deviceId": device.Id, "ports": device.Ports})
-
-	if _, exist := (*devicesAdded)[device.Id]; !exist {
-		g.AddNode(goraph.NewNode(device.Id))
-		(*devicesAdded)[device.Id] = device.Id
-	}
-
-	var portID string
-	var peerPortID string
-	for _, port := range device.Ports {
-		portID = concatDeviceIDPortID(device.Id, port.PortNo)
-		if _, exist := (*portsAdded)[portID]; !exist {
-			(*portsAdded)[portID] = portID
-			g.AddNode(goraph.NewNode(portID))
-			err := g.AddEdge(goraph.StringID(device.Id), goraph.StringID(portID), 1)
-			if err != nil {
-				log.Errorw("unable-to-add-edge", log.Fields{"error": err})
-			}
-			err = g.AddEdge(goraph.StringID(portID), goraph.StringID(device.Id), 1)
-			if err != nil {
-				log.Errorw("unable-to-add-edge", log.Fields{"error": err})
-			}
-		}
-		for _, peer := range port.Peers {
-			if _, exist := (*devicesAdded)[peer.DeviceId]; !exist {
-				d, _ := dg.getDevice(ctx, peer.DeviceId, true)
-				g = dg.addDevice(ctx, d, g, devicesAdded, portsAdded, boundaryPorts)
-			}
-			peerPortID = concatDeviceIDPortID(peer.DeviceId, peer.PortNo)
-			err := g.AddEdge(goraph.StringID(portID), goraph.StringID(peerPortID), 1)
-			if err != nil {
-				log.Errorw("unable-to-add-edge", log.Fields{"error": err})
-			}
-			err = g.AddEdge(goraph.StringID(peerPortID), goraph.StringID(portID), 1)
-			if err != nil {
-				log.Errorw("unable-to-add-edge", log.Fields{"error": err})
-			}
-		}
-	}
-	return g
-}
-
-//portExist returns true if the port ID is already part of the boundary ports map.
-func (dg *DeviceGraph) portExist(id string) bool {
-	dg.boundaryPortsLock.RLock()
-	defer dg.boundaryPortsLock.RUnlock()
-	_, exist := dg.boundaryPorts[id]
-	return exist
-}
-
-// buildPathsToAllRootPorts builds all the paths from the non-root logical port to all root ports
-// on the logical device
-func (dg *DeviceGraph) buildPathsToAllRootPorts(lp *voltha.LogicalPort) map[OFPortLink][]RouteHop {
-	paths := dg.Routes
-	source := concatDeviceIDPortID(lp.DeviceId, lp.DevicePortNo)
-	sourcePort := lp.OfpPort.PortNo
-	ch := make(chan *ofPortLinkToPath)
-	numBuildRequest := 0
-	for target, targetPort := range dg.rootPortsString {
-		go dg.buildRoute(source, target, sourcePort, targetPort, ch)
-		numBuildRequest++
-	}
-	responseReceived := 0
-forloop:
-	for {
-		if responseReceived == numBuildRequest {
-			break
-		}
-		res, ok := <-ch
-		if !ok {
-			log.Debug("channel closed")
-			break forloop
-		}
-		if res != nil && len(res.path) > 0 {
-			paths[res.link] = res.path
-			paths[OFPortLink{Ingress: res.link.Egress, Egress: res.link.Ingress}] = getReverseRoute(res.path)
-		}
-		responseReceived++
-	}
-	return paths
-}
-
-// buildPathsToAllNonRootPorts builds all the paths from the root logical port to all non-root ports
-// on the logical device
-func (dg *DeviceGraph) buildPathsToAllNonRootPorts(lp *voltha.LogicalPort) map[OFPortLink][]RouteHop {
-	paths := dg.Routes
-	source := concatDeviceIDPortID(lp.DeviceId, lp.DevicePortNo)
-	sourcePort := lp.OfpPort.PortNo
-	ch := make(chan *ofPortLinkToPath)
-	numBuildRequest := 0
-	for target, targetPort := range dg.nonRootPortsString {
-		go dg.buildRoute(source, target, sourcePort, targetPort, ch)
-		numBuildRequest++
-	}
-	responseReceived := 0
-forloop:
-	for {
-		if responseReceived == numBuildRequest {
-			break
-		}
-		res, ok := <-ch
-		if !ok {
-			log.Debug("channel closed")
-			break forloop
-		}
-		if res != nil && len(res.path) > 0 {
-			paths[res.link] = res.path
-			paths[OFPortLink{Ingress: res.link.Egress, Egress: res.link.Ingress}] = getReverseRoute(res.path)
-		}
-		responseReceived++
-	}
-	return paths
-}
-
-//buildRoute builds a route between a source and a target logical port
-func (dg *DeviceGraph) buildRoute(sourceID, targetID string, sourcePort, targetPort uint32, ch chan *ofPortLinkToPath) {
-	var pathIds []goraph.ID
-	path := make([]RouteHop, 0)
-	var err error
-	var hop RouteHop
-	var result *ofPortLinkToPath
-
-	if sourceID == targetID {
-		ch <- result
-		return
-	}
-	//Ignore Root - Root Routes
-	if dg.IsRootPort(sourcePort) && dg.IsRootPort(targetPort) {
-		ch <- result
-		return
-	}
-
-	//Ignore non-Root - non-Root Routes
-	if !dg.IsRootPort(sourcePort) && !dg.IsRootPort(targetPort) {
-		ch <- result
-		return
-	}
-
-	if pathIds, _, err = goraph.Dijkstra(dg.GGraph, goraph.StringID(sourceID), goraph.StringID(targetID)); err != nil {
-		log.Errorw("no-path", log.Fields{"sourceId": sourceID, "targetId": targetID, "error": err})
-		ch <- result
-		return
-	}
-	if len(pathIds)%3 != 0 {
-		ch <- result
-		return
-	}
-	var deviceID string
-	var ingressPort uint32
-	var egressPort uint32
-	for i := 0; i < len(pathIds); i = i + 3 {
-		if deviceID, ingressPort, err = splitIntoDeviceIDPortID(pathIds[i].String()); err != nil {
-			log.Errorw("id-error", log.Fields{"sourceId": sourceID, "targetId": targetID, "error": err})
-			break
-		}
-		if _, egressPort, err = splitIntoDeviceIDPortID(pathIds[i+2].String()); err != nil {
-			log.Errorw("id-error", log.Fields{"sourceId": sourceID, "targetId": targetID, "error": err})
-			break
-		}
-		hop = RouteHop{Ingress: ingressPort, DeviceID: deviceID, Egress: egressPort}
-		path = append(path, hop)
-	}
-	result = &ofPortLinkToPath{link: OFPortLink{Ingress: sourcePort, Egress: targetPort}, path: path}
-	ch <- result
-}
-
-//buildRoutes build all routes between all the ports on the logical device
-func (dg *DeviceGraph) buildRoutes() map[OFPortLink][]RouteHop {
-	paths := make(map[OFPortLink][]RouteHop)
-	ch := make(chan *ofPortLinkToPath)
-	numBuildRequest := 0
-	for source, sourcePort := range dg.boundaryPorts {
-		for target, targetPort := range dg.boundaryPorts {
-			go dg.buildRoute(source, target, sourcePort, targetPort, ch)
-			numBuildRequest++
-		}
-	}
-	responseReceived := 0
-forloop:
-	for {
-		if responseReceived == numBuildRequest {
-			break
-		}
-		res, ok := <-ch
-		if !ok {
-			log.Debug("channel closed")
-			break forloop
-		}
-		if res != nil && len(res.path) > 0 {
-			paths[res.link] = res.path
-		}
-		responseReceived++
-	}
-	return paths
-}
-
-// reset cleans up the device graph
-func (dg *DeviceGraph) reset() {
-	dg.devicesAdded = make(map[string]string)
-	dg.portsAdded = make(map[string]string)
-	dg.rootPortsString = make(map[string]uint32)
-	dg.nonRootPortsString = make(map[string]uint32)
-	dg.RootPorts = make(map[uint32]uint32)
-	dg.boundaryPorts = make(map[string]uint32)
-	dg.Routes = make(map[OFPortLink][]RouteHop)
-	dg.cachedDevices = make(map[string]*voltha.Device)
-}
-
-//concatDeviceIdPortId formats a portid using the device id and the port number
-func concatDeviceIDPortID(deviceID string, portNo uint32) string {
-	return fmt.Sprintf("%s:%d", deviceID, portNo)
-}
-
-// splitIntoDeviceIdPortId extracts the device id and port number from the portId
-func splitIntoDeviceIDPortID(id string) (string, uint32, error) {
-	result := strings.Split(id, ":")
-	if len(result) != 2 {
-		return "", 0, fmt.Errorf("invalid-id-%s", id)
-	}
-	temp, err := strconv.ParseInt(result[1], 10, 32)
-	if err != nil {
-		return "", 0, fmt.Errorf("invalid-id-%s-%s", id, err.Error())
-	}
-	return result[0], uint32(temp), nil
-}
-
-//getReverseRoute returns the reverse of the route
-func getReverseRoute(route []RouteHop) []RouteHop {
-	reverse := make([]RouteHop, len(route))
-	for i, j := 0, len(route)-1; j >= 0; i, j = i+1, j-1 {
-		reverse[i].DeviceID, reverse[i].Ingress, reverse[i].Egress = route[j].DeviceID, route[j].Egress, route[j].Ingress
-	}
-	return reverse
-}
diff --git a/rw_core/graph/device_graph_test.go b/rw_core/graph/device_graph_test.go
deleted file mode 100644
index 85a730f..0000000
--- a/rw_core/graph/device_graph_test.go
+++ /dev/null
@@ -1,365 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package graph
-
-import (
-	"context"
-	"errors"
-	"fmt"
-	"github.com/opencord/voltha-protos/v3/go/openflow_13"
-	"github.com/opencord/voltha-protos/v3/go/voltha"
-	"github.com/stretchr/testify/assert"
-	"math/rand"
-	"strconv"
-	"strings"
-	"sync"
-	"testing"
-	"time"
-)
-
-var (
-	ld              voltha.LogicalDevice
-	olt             voltha.Device
-	onus            map[int][]voltha.Device
-	logicalDeviceID string
-	oltDeviceID     string
-	numCalled       int
-	lock            sync.RWMutex
-)
-
-func init() {
-	logicalDeviceID = "ld"
-	oltDeviceID = "olt"
-	lock = sync.RWMutex{}
-}
-
-func setupDevices(numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu int) {
-	// Create the OLT and add the NNI ports
-	olt = voltha.Device{Id: oltDeviceID, ParentId: logicalDeviceID}
-	olt.Ports = make([]*voltha.Port, 0)
-	for nniPort := 1; nniPort < numNNIPort+1; nniPort++ {
-		p := voltha.Port{PortNo: uint32(nniPort), DeviceId: oltDeviceID, Type: voltha.Port_ETHERNET_NNI}
-		olt.Ports = append(olt.Ports, &p)
-	}
-
-	// Create the ONUs and associate them with the OLT
-	onus = make(map[int][]voltha.Device)
-	for pPortNo := numNNIPort + 1; pPortNo < numPonPortOnOlt+numNNIPort+1; pPortNo++ {
-		onusOnPon := make([]voltha.Device, 0)
-		var onu voltha.Device
-		oltPeerPort := uint32(pPortNo)
-		oltPonPort := voltha.Port{PortNo: uint32(pPortNo), DeviceId: oltDeviceID, Type: voltha.Port_PON_OLT}
-		oltPonPort.Peers = make([]*voltha.Port_PeerPort, 0)
-		for i := 0; i < numOnuPerOltPonPort; i++ {
-			id := fmt.Sprintf("%d-onu-%d", pPortNo, i)
-			onu = voltha.Device{Id: id, ParentId: oltDeviceID, ParentPortNo: uint32(pPortNo)}
-			ponPort := voltha.Port{PortNo: 1, DeviceId: onu.Id, Type: voltha.Port_PON_ONU}
-			ponPort.Peers = make([]*voltha.Port_PeerPort, 0)
-			peerPort := voltha.Port_PeerPort{DeviceId: oltDeviceID, PortNo: oltPeerPort}
-			ponPort.Peers = append(ponPort.Peers, &peerPort)
-			onu.Ports = make([]*voltha.Port, 0)
-			onu.Ports = append(onu.Ports, &ponPort)
-			for j := 2; j < numUniPerOnu+2; j++ {
-				uniPort := voltha.Port{PortNo: uint32(j), DeviceId: onu.Id, Type: voltha.Port_ETHERNET_UNI}
-				onu.Ports = append(onu.Ports, &uniPort)
-			}
-			onusOnPon = append(onusOnPon, onu)
-			oltPeerPort := voltha.Port_PeerPort{DeviceId: onu.Id, PortNo: 1}
-			oltPonPort.Peers = append(oltPonPort.Peers, &oltPeerPort)
-		}
-		onus[pPortNo] = onusOnPon
-		olt.Ports = append(olt.Ports, &oltPonPort)
-	}
-
-	// Create the logical device
-	ld = voltha.LogicalDevice{Id: logicalDeviceID}
-	ld.Ports = make([]*voltha.LogicalPort, 0)
-	ofpPortNo := 1
-	var id string
-	//Add olt NNI ports
-	for i, port := range olt.Ports {
-		if port.Type == voltha.Port_ETHERNET_NNI {
-			id = fmt.Sprintf("nni-%d", i)
-			lp := voltha.LogicalPort{Id: id, DeviceId: olt.Id, DevicePortNo: port.PortNo, OfpPort: &openflow_13.OfpPort{PortNo: uint32(ofpPortNo)}, RootPort: true}
-			ld.Ports = append(ld.Ports, &lp)
-			ofpPortNo = ofpPortNo + 1
-		}
-	}
-	//Add onu UNI ports
-	for _, onusOnPort := range onus {
-		for _, onu := range onusOnPort {
-			for j, port := range onu.Ports {
-				if port.Type == voltha.Port_ETHERNET_UNI {
-					id = fmt.Sprintf("%s:uni-%d", onu.Id, j)
-					lp := voltha.LogicalPort{Id: id, DeviceId: onu.Id, DevicePortNo: port.PortNo, OfpPort: &openflow_13.OfpPort{PortNo: uint32(ofpPortNo)}, RootPort: false}
-					ld.Ports = append(ld.Ports, &lp)
-					ofpPortNo = ofpPortNo + 1
-				}
-			}
-		}
-	}
-}
-
-func GetDeviceHelper(_ context.Context, id string) (*voltha.Device, error) {
-	lock.Lock()
-	numCalled++
-	lock.Unlock()
-	if id == "olt" {
-		return &olt, nil
-	}
-	// Extract the olt pon port from the id ("<ponport>-onu-<onu number>")
-	res := strings.Split(id, "-")
-	if len(res) == 3 {
-		if ponPort, err := strconv.Atoi(res[0]); err == nil {
-			for _, onu := range onus[ponPort] {
-				if onu.Id == id {
-					return &onu, nil
-				}
-			}
-
-		}
-	}
-	return nil, errors.New("Not-found")
-}
-
-func TestGetRoutesOneShot(t *testing.T) {
-	numNNIPort := 1
-	numPonPortOnOlt := 1
-	numOnuPerOltPonPort := 64
-	numUniPerOnu := 1
-
-	setupDevices(numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu)
-	getDevice := GetDeviceHelper
-
-	fmt.Println(fmt.Sprintf("Test: Computing all routes. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d", len(ld.Ports), numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
-	// Create a device graph and computes Routes
-	start := time.Now()
-	dg := NewDeviceGraph(logicalDeviceID, getDevice)
-	dg.ComputeRoutes(context.Background(), ld.Ports)
-	assert.NotNil(t, dg.GGraph)
-	fmt.Println(fmt.Sprintf("Total Time:%dms  Total Routes:%d", time.Since(start)/time.Millisecond, len(dg.Routes)))
-	assert.EqualValues(t, (2 * numNNIPort * numPonPortOnOlt * numOnuPerOltPonPort * numUniPerOnu), len(dg.Routes))
-}
-
-func TestGetRoutesPerPort(t *testing.T) {
-	numNNIPort := 1
-	numPonPortOnOlt := 1
-	numOnuPerOltPonPort := 64
-	numUniPerOnu := 1
-
-	setupDevices(numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu)
-	getDevice := GetDeviceHelper
-
-	fmt.Println(fmt.Sprintf("Test: Compute routes per port. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d", len(ld.Ports), numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
-
-	// Create a device graph and computes Routes
-	start := time.Now()
-	var pt time.Time
-	dg := NewDeviceGraph(logicalDeviceID, getDevice)
-	for k, lp := range ld.Ports {
-		if k == len(ld.Ports)-1 {
-			pt = time.Now()
-		}
-		dg.AddPort(context.Background(), lp)
-	}
-	assert.NotNil(t, dg.GGraph)
-	fmt.Println(fmt.Sprintf("Total Time:%dms.  Total Routes:%d. LastPort_Time:%dms", time.Since(start)/time.Millisecond, len(dg.Routes), time.Since(pt)/time.Millisecond))
-	assert.EqualValues(t, (2 * numNNIPort * numPonPortOnOlt * numOnuPerOltPonPort * numUniPerOnu), len(dg.Routes))
-}
-
-func TestGetRoutesPerPortMultipleUNIs(t *testing.T) {
-	numNNIPort := 1
-	numPonPortOnOlt := 1
-	numOnuPerOltPonPort := 64
-	numUniPerOnu := 5
-
-	setupDevices(numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu)
-	getDevice := GetDeviceHelper
-
-	fmt.Println(fmt.Sprintf("Test: Compute routes per port - multiple UNIs. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d", len(ld.Ports), numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
-
-	// Create a device graph and computes Routes
-	start := time.Now()
-	var pt time.Time
-	dg := NewDeviceGraph(logicalDeviceID, getDevice)
-	for k, lp := range ld.Ports {
-		if k == len(ld.Ports)-1 {
-			pt = time.Now()
-		}
-		dg.AddPort(context.Background(), lp)
-	}
-	assert.NotNil(t, dg.GGraph)
-	fmt.Println(fmt.Sprintf("Total Time:%dms.  Total Routes:%d. LastPort_Time:%dms", time.Since(start)/time.Millisecond, len(dg.Routes), time.Since(pt)/time.Millisecond))
-	assert.EqualValues(t, (2 * numNNIPort * numPonPortOnOlt * numOnuPerOltPonPort * numUniPerOnu), len(dg.Routes))
-}
-
-func TestGetRoutesPerPortNoUNI(t *testing.T) {
-	numNNIPort := 1
-	numPonPortOnOlt := 1
-	numOnuPerOltPonPort := 1
-	numUniPerOnu := 0
-
-	setupDevices(numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu)
-	getDevice := GetDeviceHelper
-	assert.EqualValues(t, 1, len(ld.Ports))
-
-	fmt.Println(fmt.Sprintf("Test: Compute routes per port - no UNI. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d", len(ld.Ports), numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
-
-	// Create a device graph and computes Routes
-	start := time.Now()
-	var pt time.Time
-	dg := NewDeviceGraph(logicalDeviceID, getDevice)
-	for k, lp := range ld.Ports {
-		if k == len(ld.Ports)-1 {
-			pt = time.Now()
-		}
-		dg.AddPort(context.Background(), lp)
-	}
-	assert.NotNil(t, dg.GGraph)
-	fmt.Println(fmt.Sprintf("Total Time:%dms.  Total Routes:%d. LastPort_Time:%dms", time.Since(start)/time.Millisecond, len(dg.Routes), time.Since(pt)/time.Millisecond))
-	assert.EqualValues(t, 0, len(dg.Routes))
-}
-
-func TestGetRoutesPerPortNoONU(t *testing.T) {
-	numNNIPort := 1
-	numPonPortOnOlt := 1
-	numOnuPerOltPonPort := 0
-	numUniPerOnu := 0
-
-	setupDevices(numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu)
-	getDevice := GetDeviceHelper
-	assert.EqualValues(t, 1, len(ld.Ports))
-
-	fmt.Println(fmt.Sprintf("Test: Compute routes per port - no ONU. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d", len(ld.Ports), numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
-
-	// Create a device graph and computes Routes
-	start := time.Now()
-	var pt time.Time
-	dg := NewDeviceGraph(logicalDeviceID, getDevice)
-	for k, lp := range ld.Ports {
-		if k == len(ld.Ports)-1 {
-			pt = time.Now()
-		}
-		dg.AddPort(context.Background(), lp)
-	}
-	assert.NotNil(t, dg.GGraph)
-	fmt.Println(fmt.Sprintf("Total Time:%dms.  Total Routes:%d. LastPort_Time:%dms", time.Since(start)/time.Millisecond, len(dg.Routes), time.Since(pt)/time.Millisecond))
-	assert.EqualValues(t, 0, len(dg.Routes))
-}
-
-func TestGetRoutesPerPortNoNNI(t *testing.T) {
-	numNNIPort := 0
-	numPonPortOnOlt := 1
-	numOnuPerOltPonPort := 1
-	numUniPerOnu := 1
-
-	setupDevices(numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu)
-	getDevice := GetDeviceHelper
-	assert.EqualValues(t, 1, len(ld.Ports))
-
-	fmt.Println(fmt.Sprintf("Test: Compute routes per port - no NNI. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d", len(ld.Ports), numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
-
-	// Create a device graph and computes Routes
-	start := time.Now()
-	var pt time.Time
-	dg := NewDeviceGraph(logicalDeviceID, getDevice)
-	for k, lp := range ld.Ports {
-		if k == len(ld.Ports)-1 {
-			pt = time.Now()
-		}
-		dg.AddPort(context.Background(), lp)
-	}
-	assert.NotNil(t, dg.GGraph)
-	fmt.Println(fmt.Sprintf("Total Time:%dms.  Total Routes:%d. LastPort_Time:%dms", time.Since(start)/time.Millisecond, len(dg.Routes), time.Since(pt)/time.Millisecond))
-	assert.EqualValues(t, 0, len(dg.Routes))
-}
-
-func TestReverseRoute(t *testing.T) {
-	// Test the typical use case - 2 hops in a route
-	route := make([]RouteHop, 2)
-	route[0].DeviceID = "d1"
-	route[0].Ingress = 1
-	route[0].Egress = 2
-	route[1].DeviceID = "d2"
-	route[1].Ingress = 10
-	route[1].Egress = 15
-
-	reverseRoute := getReverseRoute(route)
-	assert.Equal(t, 2, len(reverseRoute))
-	assert.Equal(t, "d2", reverseRoute[0].DeviceID)
-	assert.Equal(t, "d1", reverseRoute[1].DeviceID)
-	assert.Equal(t, uint32(15), reverseRoute[0].Ingress)
-	assert.Equal(t, uint32(10), reverseRoute[0].Egress)
-	assert.Equal(t, uint32(2), reverseRoute[1].Ingress)
-	assert.Equal(t, uint32(1), reverseRoute[1].Egress)
-
-	fmt.Println("Reverse of two hops successful.")
-
-	//Test 3 hops in a route
-	route = make([]RouteHop, 3)
-	route[0].DeviceID = "d1"
-	route[0].Ingress = 1
-	route[0].Egress = 2
-	route[1].DeviceID = "d2"
-	route[1].Ingress = 10
-	route[1].Egress = 15
-	route[2].DeviceID = "d3"
-	route[2].Ingress = 20
-	route[2].Egress = 25
-	reverseRoute = getReverseRoute(route)
-	assert.Equal(t, 3, len(reverseRoute))
-	assert.Equal(t, "d3", reverseRoute[0].DeviceID)
-	assert.Equal(t, "d2", reverseRoute[1].DeviceID)
-	assert.Equal(t, "d1", reverseRoute[2].DeviceID)
-	assert.Equal(t, uint32(25), reverseRoute[0].Ingress)
-	assert.Equal(t, uint32(20), reverseRoute[0].Egress)
-	assert.Equal(t, uint32(15), reverseRoute[1].Ingress)
-	assert.Equal(t, uint32(10), reverseRoute[1].Egress)
-	assert.Equal(t, uint32(2), reverseRoute[2].Ingress)
-	assert.Equal(t, uint32(1), reverseRoute[2].Egress)
-
-	fmt.Println("Reverse of three hops successful.")
-
-	// Test any number of hops in a route
-	numRoutes := rand.Intn(100)
-	route = make([]RouteHop, numRoutes)
-	deviceIds := make([]string, numRoutes)
-	ingressNos := make([]uint32, numRoutes)
-	egressNos := make([]uint32, numRoutes)
-	for i := 0; i < numRoutes; i++ {
-		deviceIds[i] = fmt.Sprintf("d-%d", i)
-		ingressNos[i] = rand.Uint32()
-		egressNos[i] = rand.Uint32()
-	}
-	for i := 0; i < numRoutes; i++ {
-		route[i].DeviceID = deviceIds[i]
-		route[i].Ingress = ingressNos[i]
-		route[i].Egress = egressNos[i]
-	}
-	reverseRoute = getReverseRoute(route)
-	assert.Equal(t, numRoutes, len(reverseRoute))
-	for i, j := 0, numRoutes-1; j >= 0; i, j = i+1, j-1 {
-		assert.Equal(t, deviceIds[j], reverseRoute[i].DeviceID)
-		assert.Equal(t, egressNos[j], reverseRoute[i].Ingress)
-		assert.Equal(t, ingressNos[j], reverseRoute[i].Egress)
-	}
-
-	fmt.Println(fmt.Sprintf("Reverse of %d hops successful.", numRoutes))
-
-	reverseOfReverse := getReverseRoute(reverseRoute)
-	assert.Equal(t, route, reverseOfReverse)
-	fmt.Println("Reverse of reverse successful.")
-}
diff --git a/rw_core/route/device_route.go b/rw_core/route/device_route.go
new file mode 100644
index 0000000..b80c20c
--- /dev/null
+++ b/rw_core/route/device_route.go
@@ -0,0 +1,403 @@
+/*
+ * 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 route
+
+import (
+	"context"
+	"fmt"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+	"github.com/opencord/voltha-protos/v3/go/voltha"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+	"sync"
+)
+
+func init() {
+	_, err := log.AddPackage(log.JSON, log.WarnLevel, nil)
+	if err != nil {
+		log.Fatalw("unable-to-register-package-to-the-log-map", log.Fields{"error": err})
+	}
+}
+
+// Hop represent a route hop
+type Hop struct {
+	DeviceID string
+	Ingress  uint32
+	Egress   uint32
+}
+
+// PathID is the identification of a route between two logical ports
+type PathID struct {
+	Ingress uint32
+	Egress  uint32
+}
+
+type OFPortLink struct {
+	Ingress uint32
+	Egress  uint32
+}
+
+// GetDeviceFunc returns device function
+type GetDeviceFunc func(ctx context.Context, id string) (*voltha.Device, error)
+
+// DeviceRoutes represent the set of routes between logical ports of a logical device
+type DeviceRoutes struct {
+	logicalDeviceID     string
+	getDeviceFromModel  GetDeviceFunc
+	logicalPorts        []*voltha.LogicalPort
+	RootPorts           map[uint32]uint32
+	rootPortsLock       sync.RWMutex
+	Routes              map[PathID][]Hop
+	routeBuildLock      sync.RWMutex
+	devicesPonPorts     map[string][]*voltha.Port
+	devicesPonPortsLock sync.RWMutex
+}
+
+// NewDeviceRoutes creates device graph instance
+func NewDeviceRoutes(logicalDeviceID string, getDevice GetDeviceFunc) *DeviceRoutes {
+	var dr DeviceRoutes
+	dr.logicalDeviceID = logicalDeviceID
+	dr.getDeviceFromModel = getDevice
+	dr.RootPorts = make(map[uint32]uint32)
+	dr.Routes = make(map[PathID][]Hop)
+	dr.devicesPonPorts = make(map[string][]*voltha.Port)
+	log.Debug("new device routes created ...")
+	return &dr
+}
+
+//IsRootPort returns true if the port is a root port on a logical device
+func (dr *DeviceRoutes) IsRootPort(port uint32) bool {
+	dr.rootPortsLock.RLock()
+	defer dr.rootPortsLock.RUnlock()
+	_, exist := dr.RootPorts[port]
+	return exist
+}
+
+//ComputeRoutes calculates all the routes between the logical ports.  This will clear up any existing route
+func (dr *DeviceRoutes) ComputeRoutes(ctx context.Context, lps []*voltha.LogicalPort) error {
+	dr.routeBuildLock.Lock()
+	defer dr.routeBuildLock.Unlock()
+
+	log.Debugw("computing-all-routes", log.Fields{"len-logical-ports": len(lps)})
+	var err error
+	defer func() {
+		// On error, clear the routes - any flow request or a port add/delete will trigger the rebuild
+		if err != nil {
+			dr.reset()
+		}
+	}()
+
+	if len(lps) < 2 {
+		return status.Error(codes.FailedPrecondition, "not-enough-logical-ports")
+	}
+
+	dr.reset()
+	dr.logicalPorts = append(dr.logicalPorts, lps...)
+
+	// Setup the physical ports to logical ports map, the nni ports as well as the root ports map
+	physPortToLogicalPortMap := make(map[string]uint32)
+	nniPorts := make([]*voltha.LogicalPort, 0)
+	for _, lp := range lps {
+		physPortToLogicalPortMap[concatDeviceIDPortID(lp.DeviceId, lp.DevicePortNo)] = lp.OfpPort.PortNo
+		if lp.RootPort {
+			nniPorts = append(nniPorts, lp)
+			dr.RootPorts[lp.OfpPort.PortNo] = lp.OfpPort.PortNo
+		}
+	}
+	if len(nniPorts) == 0 {
+		err = status.Error(codes.FailedPrecondition, "no nni port")
+		return err
+	}
+	var rootDevice *voltha.Device
+	var childDevice *voltha.Device
+	var copyFromNNIPort *voltha.LogicalPort
+	for idx, nniPort := range nniPorts {
+		if idx == 0 {
+			copyFromNNIPort = nniPort
+		} else if len(dr.Routes) > 0 {
+			dr.copyFromExistingNNIRoutes(nniPort, copyFromNNIPort)
+			return nil
+		}
+		// Get root device
+		rootDevice, err = dr.getDevice(ctx, nniPort.DeviceId)
+		if err != nil {
+			return err
+		}
+		if len(rootDevice.Ports) == 0 {
+			err = status.Errorf(codes.FailedPrecondition, "no-port-%s", rootDevice.Id)
+			return err
+		}
+		for _, rootDevicePort := range rootDevice.Ports {
+			if rootDevicePort.Type == voltha.Port_PON_OLT {
+				for _, rootDevicePeer := range rootDevicePort.Peers {
+					childDevice, err = dr.getDevice(ctx, rootDevicePeer.DeviceId)
+					if err != nil {
+						return err
+					}
+					childPonPorts := dr.getDevicePonPorts(childDevice.Id, nniPort.DeviceId)
+					if len(childPonPorts) < 1 {
+						err = status.Errorf(codes.Aborted, "no-child-pon-port-%s", childDevice.Id)
+						return err
+					}
+					// We use the first PON port on the ONU whose parent is the root device.
+					childPonPort := childPonPorts[0].PortNo
+					for _, childDevicePort := range childDevice.Ports {
+						if childDevicePort.Type == voltha.Port_ETHERNET_UNI {
+							childLogicalPort, exist := physPortToLogicalPortMap[concatDeviceIDPortID(childDevice.Id, childDevicePort.PortNo)]
+							if !exist {
+								// This can happen if this logical port has not been created yet for that device
+								continue
+							}
+							dr.Routes[PathID{Ingress: nniPort.OfpPort.PortNo, Egress: childLogicalPort}] = []Hop{
+								{DeviceID: rootDevice.Id, Ingress: nniPort.DevicePortNo, Egress: rootDevicePort.PortNo},
+								{DeviceID: childDevice.Id, Ingress: childPonPort, Egress: childDevicePort.PortNo},
+							}
+							dr.Routes[PathID{Ingress: childLogicalPort, Egress: nniPort.OfpPort.PortNo}] = getReverseRoute(
+								dr.Routes[PathID{Ingress: nniPort.OfpPort.PortNo, Egress: childLogicalPort}])
+						}
+					}
+				}
+			}
+		}
+	}
+	return nil
+}
+
+// verifyPrecondition verify whether the preconditions are met to proceed with addition of the new logical port
+func (dr *DeviceRoutes) addPortAndVerifyPrecondition(lp *voltha.LogicalPort) error {
+	var exist, nniLogicalPortExist, uniLogicalPortExist bool
+	for _, existingLogicalPort := range dr.logicalPorts {
+		nniLogicalPortExist = nniLogicalPortExist || existingLogicalPort.RootPort
+		uniLogicalPortExist = uniLogicalPortExist || !existingLogicalPort.RootPort
+		exist = exist || existingLogicalPort.OfpPort.PortNo == lp.OfpPort.PortNo
+		if nniLogicalPortExist && uniLogicalPortExist && exist {
+			break
+		}
+	}
+	if !exist {
+		dr.logicalPorts = append(dr.logicalPorts, lp)
+		nniLogicalPortExist = nniLogicalPortExist || lp.RootPort
+		uniLogicalPortExist = uniLogicalPortExist || !lp.RootPort
+	}
+
+	// If we do not have both NNI and UNI ports then return an error
+	if !(nniLogicalPortExist && uniLogicalPortExist) {
+		fmt.Println("errors", nniLogicalPortExist, uniLogicalPortExist)
+		return status.Error(codes.FailedPrecondition, "no-uni-and-nni-ports-combination")
+	}
+	return nil
+}
+
+// AddPort augments the current set of routes with new routes corresponding to the logical port "lp".  If the routes have
+// not been built yet then use logical port "lps" to compute all current routes (lps includes lp)
+func (dr *DeviceRoutes) AddPort(ctx context.Context, lp *voltha.LogicalPort, lps []*voltha.LogicalPort) error {
+	log.Debugw("add-port-to-routes", log.Fields{"port": lp, "len-logical-ports": len(lps)})
+
+	dr.routeBuildLock.Lock()
+	if len(dr.Routes) == 0 {
+		dr.routeBuildLock.Unlock()
+		return dr.ComputeRoutes(ctx, lps)
+	}
+
+	// A set of routes exists
+	if err := dr.addPortAndVerifyPrecondition(lp); err != nil {
+		dr.reset()
+		dr.routeBuildLock.Unlock()
+		return err
+	}
+
+	defer dr.routeBuildLock.Unlock()
+	// Update the set of root ports, if applicable
+	if lp.RootPort {
+		dr.RootPorts[lp.OfpPort.PortNo] = lp.OfpPort.PortNo
+	}
+
+	var copyFromNNIPort *voltha.LogicalPort
+	// Setup the physical ports to logical ports map
+	nniPorts := make([]*voltha.LogicalPort, 0)
+	for _, lport := range dr.logicalPorts {
+		if lport.RootPort {
+			nniPorts = append(nniPorts, lport)
+			if copyFromNNIPort == nil && lport.OfpPort.PortNo != lp.OfpPort.PortNo {
+				copyFromNNIPort = lport
+			}
+		}
+	}
+
+	if copyFromNNIPort == nil {
+		// Trying to add the same NNI port.  Just return
+		return nil
+	}
+
+	// Adding NNI Port?   If we are here we already have an NNI port with a set of routes.  Just copy the existing
+	// routes using an existing NNI port
+	if lp.RootPort {
+		dr.copyFromExistingNNIRoutes(lp, copyFromNNIPort)
+		return nil
+	}
+
+	// Adding a UNI port
+	for _, nniPort := range nniPorts {
+		childPonPorts := dr.getDevicePonPorts(lp.DeviceId, nniPort.DeviceId)
+		if len(childPonPorts) == 0 || len(childPonPorts[0].Peers) == 0 {
+			// Ports may not have been cached yet - get the device info which sets the PON port cache
+			if _, err := dr.getDevice(ctx, lp.DeviceId); err != nil {
+				dr.reset()
+				return err
+			}
+			childPonPorts = dr.getDevicePonPorts(lp.DeviceId, nniPort.DeviceId)
+			if len(childPonPorts) == 0 || len(childPonPorts[0].Peers) == 0 {
+				dr.reset()
+				return status.Errorf(codes.FailedPrecondition, "no-pon-ports-%s", lp.DeviceId)
+			}
+		}
+		// We use the first PON port on the child device
+		childPonPort := childPonPorts[0]
+		dr.Routes[PathID{Ingress: nniPort.OfpPort.PortNo, Egress: lp.OfpPort.PortNo}] = []Hop{
+			{DeviceID: nniPort.DeviceId, Ingress: nniPort.DevicePortNo, Egress: childPonPort.Peers[0].PortNo},
+			{DeviceID: lp.DeviceId, Ingress: childPonPort.PortNo, Egress: lp.DevicePortNo},
+		}
+		dr.Routes[PathID{Ingress: lp.OfpPort.PortNo, Egress: nniPort.OfpPort.PortNo}] = getReverseRoute(
+			dr.Routes[PathID{Ingress: nniPort.OfpPort.PortNo, Egress: lp.OfpPort.PortNo}])
+	}
+	return nil
+}
+
+// Print prints routes
+func (dr *DeviceRoutes) Print() error {
+	log.Debugw("Print", log.Fields{"logical-device-id": dr.logicalDeviceID, "logical-ports": dr.logicalPorts})
+	if log.V(log.DebugLevel) {
+		output := ""
+		routeNumber := 1
+		for k, v := range dr.Routes {
+			key := fmt.Sprintf("LP:%d->LP:%d", k.Ingress, k.Egress)
+			val := ""
+			for _, i := range v {
+				val += fmt.Sprintf("{%d->%s->%d},", i.Ingress, i.DeviceID, i.Egress)
+			}
+			val = val[:len(val)-1]
+			output += fmt.Sprintf("%d:{%s=>%s}   ", routeNumber, key, fmt.Sprintf("[%s]", val))
+			routeNumber++
+		}
+		if len(dr.Routes) == 0 {
+			log.Debugw("no-routes-found", log.Fields{"logical-device-id": dr.logicalDeviceID})
+		} else {
+			log.Debugw("graph_routes", log.Fields{"lDeviceId": dr.logicalDeviceID, "Routes": output})
+		}
+	}
+	return nil
+}
+
+// IsUpToDate returns true if device is up to date
+func (dr *DeviceRoutes) IsUpToDate(ld *voltha.LogicalDevice) bool {
+	dr.routeBuildLock.Lock()
+	defer dr.routeBuildLock.Unlock()
+	numNNI, numUNI := 0, 0
+	if ld != nil {
+		if len(dr.logicalPorts) != len(ld.Ports) {
+			return false
+		}
+		numNNI = len(dr.RootPorts)
+		numUNI = len(ld.Ports) - numNNI
+	}
+	return len(dr.Routes) == numNNI*numUNI*2
+}
+
+// getDevicePonPorts returns all the PON ports of a device whose peer device ID is peerDeviceID
+func (dr *DeviceRoutes) getDevicePonPorts(deviceID string, peerDeviceID string) []*voltha.Port {
+	dr.devicesPonPortsLock.RLock()
+	defer dr.devicesPonPortsLock.RUnlock()
+	ponPorts := make([]*voltha.Port, 0)
+	ports, exist := dr.devicesPonPorts[deviceID]
+	if !exist {
+		return ponPorts
+	}
+	//fmt.Println("getDevicePonPorts", deviceID, peerDeviceID, ports)
+	for _, port := range ports {
+		for _, peer := range port.Peers {
+			if peer.DeviceId == peerDeviceID {
+				ponPorts = append(ponPorts, port)
+			}
+		}
+	}
+	return ponPorts
+}
+
+//getDevice returns the from the model and updates the PON ports map of that device.
+func (dr *DeviceRoutes) getDevice(ctx context.Context, deviceID string) (*voltha.Device, error) {
+	device, err := dr.getDeviceFromModel(ctx, deviceID)
+	if err != nil {
+		log.Errorw("device-not-found", log.Fields{"deviceId": deviceID, "error": err})
+		return nil, err
+	}
+	dr.devicesPonPortsLock.Lock()
+	defer dr.devicesPonPortsLock.Unlock()
+	for _, port := range device.Ports {
+		if port.Type == voltha.Port_PON_ONU || port.Type == voltha.Port_PON_OLT {
+			dr.devicesPonPorts[device.Id] = append(dr.devicesPonPorts[device.Id], port)
+		}
+	}
+	return device, nil
+}
+
+//copyFromExistingNNIRoutes copies routes from an existing set of NNI routes
+func (dr *DeviceRoutes) copyFromExistingNNIRoutes(newNNIPort *voltha.LogicalPort, copyFromNNIPort *voltha.LogicalPort) {
+	updatedRoutes := make(map[PathID][]Hop)
+	for key, val := range dr.Routes {
+		if key.Ingress == copyFromNNIPort.OfpPort.PortNo {
+			updatedRoutes[PathID{Ingress: newNNIPort.OfpPort.PortNo, Egress: key.Egress}] = []Hop{
+				{DeviceID: newNNIPort.DeviceId, Ingress: newNNIPort.DevicePortNo, Egress: val[0].Egress},
+				val[1],
+			}
+		}
+		if key.Egress == copyFromNNIPort.OfpPort.PortNo {
+			updatedRoutes[PathID{Ingress: key.Ingress, Egress: newNNIPort.OfpPort.PortNo}] = []Hop{
+				val[0],
+				{DeviceID: newNNIPort.DeviceId, Ingress: val[1].Ingress, Egress: newNNIPort.DevicePortNo},
+			}
+		}
+		updatedRoutes[key] = val
+	}
+	dr.Routes = updatedRoutes
+}
+
+// reset cleans up the device graph
+func (dr *DeviceRoutes) reset() {
+	dr.rootPortsLock.Lock()
+	dr.RootPorts = make(map[uint32]uint32)
+	dr.rootPortsLock.Unlock()
+	// Do not numGetDeviceCalledLock Routes, logicalPorts  as the callee function already holds its numGetDeviceCalledLock.
+	dr.Routes = make(map[PathID][]Hop)
+	dr.logicalPorts = make([]*voltha.LogicalPort, 0)
+	dr.devicesPonPortsLock.Lock()
+	dr.devicesPonPorts = make(map[string][]*voltha.Port)
+	dr.devicesPonPortsLock.Unlock()
+}
+
+//concatDeviceIdPortId formats a portid using the device id and the port number
+func concatDeviceIDPortID(deviceID string, portNo uint32) string {
+	return fmt.Sprintf("%s:%d", deviceID, portNo)
+}
+
+//getReverseRoute returns the reverse of the route
+func getReverseRoute(route []Hop) []Hop {
+	reverse := make([]Hop, len(route))
+	for i, j := 0, len(route)-1; j >= 0; i, j = i+1, j-1 {
+		reverse[i].DeviceID, reverse[i].Ingress, reverse[i].Egress = route[j].DeviceID, route[j].Egress, route[j].Ingress
+	}
+	return reverse
+}
diff --git a/rw_core/route/device_route_test.go b/rw_core/route/device_route_test.go
new file mode 100644
index 0000000..1f90ecd
--- /dev/null
+++ b/rw_core/route/device_route_test.go
@@ -0,0 +1,490 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package route
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"github.com/opencord/voltha-protos/v3/go/openflow_13"
+	"github.com/opencord/voltha-protos/v3/go/voltha"
+	"github.com/stretchr/testify/assert"
+	"math/rand"
+	"reflect"
+	"strings"
+	"sync"
+	"testing"
+	"time"
+)
+
+const (
+	logicalDeviceID = "ld"
+	oltDeviceID     = "olt"
+)
+
+//portRegistration is a message sent from an OLT device to a logical device to create a logical port
+type portRegistration struct {
+	port     *voltha.Port
+	rootPort bool
+}
+
+//onuRegistration is a message sent from an ONU device to an OLT device to register an ONU
+type onuRegistration struct {
+	onu      *voltha.Device
+	oltPonNo uint32
+	onuPonNo uint32
+}
+
+type logicalDeviceManager struct {
+	logicalDevice   *voltha.LogicalDevice
+	deviceRoutes    *DeviceRoutes
+	ldChnl          chan portRegistration
+	numLogicalPorts int
+	done            chan struct{}
+}
+
+func newLogicalDeviceManager(ld *voltha.LogicalDevice, ch chan portRegistration, totalLogicalPorts int, done chan struct{}) *logicalDeviceManager {
+	return &logicalDeviceManager{
+		logicalDevice:   ld,
+		ldChnl:          ch,
+		numLogicalPorts: totalLogicalPorts,
+		done:            done,
+	}
+}
+
+func (ldM *logicalDeviceManager) start(getDevice GetDeviceFunc, buildRoutes bool) {
+	ldM.deviceRoutes = NewDeviceRoutes(ldM.logicalDevice.Id, getDevice)
+	ofpPortNo := uint32(1)
+	for portReg := range ldM.ldChnl {
+		if portReg.port == nil {
+			// End of registration - exit loop
+			break
+		}
+		lp := &voltha.LogicalPort{
+			Id:           portReg.port.Label,
+			OfpPort:      &openflow_13.OfpPort{PortNo: ofpPortNo},
+			DeviceId:     portReg.port.DeviceId,
+			DevicePortNo: portReg.port.PortNo,
+			RootPort:     portReg.rootPort,
+		}
+		ldM.logicalDevice.Ports = append(ldM.logicalDevice.Ports, lp)
+		if buildRoutes {
+			err := ldM.deviceRoutes.AddPort(context.Background(), lp, ldM.logicalDevice.Ports)
+			if err != nil && !strings.Contains(err.Error(), "code = FailedPrecondition") {
+				fmt.Println("(Error when adding port:", lp, len(ldM.logicalDevice.Ports), err)
+			}
+		}
+		ofpPortNo++
+	}
+	// Inform the caller we are now done
+	ldM.done <- struct{}{}
+}
+
+type oltManager struct {
+	olt              *voltha.Device
+	logicalDeviceMgr *logicalDeviceManager
+	numNNIPort       int
+	numPonPortOnOlt  int
+	oltChnl          chan onuRegistration
+}
+
+func newOltManager(oltDeviceID string, ldMgr *logicalDeviceManager, numNNIPort int, numPonPortOnOlt int, ch chan onuRegistration) *oltManager {
+	return &oltManager{
+		olt:              &voltha.Device{Id: oltDeviceID, ParentId: ldMgr.logicalDevice.Id, Root: true},
+		logicalDeviceMgr: ldMgr,
+		numNNIPort:       numNNIPort,
+		numPonPortOnOlt:  numPonPortOnOlt,
+		oltChnl:          ch,
+	}
+}
+
+func (oltM *oltManager) start() {
+	oltM.olt.Ports = make([]*voltha.Port, 0)
+	// Setup the OLT nni ports and trigger the nni ports creation
+	for nniPort := 1; nniPort < oltM.numNNIPort+1; nniPort++ {
+		p := &voltha.Port{Label: fmt.Sprintf("nni-%d", nniPort), PortNo: uint32(nniPort), DeviceId: oltM.olt.Id, Type: voltha.Port_ETHERNET_NNI}
+		oltM.olt.Ports = append(oltM.olt.Ports, p)
+		oltM.logicalDeviceMgr.ldChnl <- portRegistration{port: p, rootPort: true}
+	}
+
+	// Create OLT pon ports
+	for ponPort := oltM.numNNIPort + 1; ponPort < oltM.numPonPortOnOlt+oltM.numNNIPort+1; ponPort++ {
+		p := voltha.Port{PortNo: uint32(ponPort), DeviceId: oltM.olt.Id, Type: voltha.Port_PON_OLT}
+		oltM.olt.Ports = append(oltM.olt.Ports, &p)
+	}
+
+	// Wait for onu registration
+	for onuReg := range oltM.oltChnl {
+		if onuReg.onu == nil {
+			// All onu has registered - exit the loop
+			break
+		}
+		oltM.registerOnu(onuReg.onu, onuReg.oltPonNo, onuReg.onuPonNo)
+	}
+	// Inform the logical device manager we are done
+	oltM.logicalDeviceMgr.ldChnl <- portRegistration{port: nil}
+}
+
+func (oltM *oltManager) registerOnu(onu *voltha.Device, oltPonNo uint32, onuPonNo uint32) {
+	// Update the olt pon peers
+	for _, port := range oltM.olt.Ports {
+		if port.Type == voltha.Port_PON_OLT && port.PortNo == oltPonNo {
+			port.Peers = append(port.Peers, &voltha.Port_PeerPort{DeviceId: onu.Id, PortNo: onuPonNo})
+		}
+	}
+	// For each uni port on the ONU trigger the creation of a logical port
+	for _, port := range onu.Ports {
+		if port.Type == voltha.Port_ETHERNET_UNI {
+			oltM.logicalDeviceMgr.ldChnl <- portRegistration{port: port, rootPort: false}
+		}
+	}
+}
+
+type onuManager struct {
+	oltMgr                  *oltManager
+	numOnus                 int
+	numUnisPerOnu           int
+	startingUniPortNo       int
+	numGetDeviceInvoked     int
+	numGetDeviceInvokedLock sync.RWMutex
+	deviceLock              sync.RWMutex
+	onus                    []*voltha.Device
+}
+
+func newOnuManager(oltMgr *oltManager, numOnus int, numUnisPerOnu int, startingUniPortNo int) *onuManager {
+	return &onuManager{
+		oltMgr:            oltMgr,
+		numOnus:           numOnus,
+		numUnisPerOnu:     numUnisPerOnu,
+		startingUniPortNo: startingUniPortNo,
+		onus:              make([]*voltha.Device, 0),
+	}
+}
+
+func (onuM *onuManager) start(startingOltPeerPortNo int, numPonPortOnOlt int) {
+	var wg sync.WaitGroup
+	for oltPonNo := startingOltPeerPortNo; oltPonNo < startingOltPeerPortNo+numPonPortOnOlt; oltPonNo++ {
+		for i := 0; i < onuM.numOnus; i++ {
+			wg.Add(1)
+			go func(idx int, oltPonNum int) {
+				var onu *voltha.Device
+				defer wg.Done()
+				id := fmt.Sprintf("%d-onu-%d", oltPonNum, idx)
+				onu = &voltha.Device{Id: id, ParentId: onuM.oltMgr.olt.Id, ParentPortNo: uint32(oltPonNum)}
+				ponPort := &voltha.Port{Label: fmt.Sprintf("%s:pon-%d", onu.Id, idx), PortNo: 1, DeviceId: onu.Id, Type: voltha.Port_PON_ONU}
+				ponPort.Peers = make([]*voltha.Port_PeerPort, 0)
+				peerPort := voltha.Port_PeerPort{DeviceId: onuM.oltMgr.olt.Id, PortNo: uint32(oltPonNum)}
+				ponPort.Peers = append(ponPort.Peers, &peerPort)
+				onu.Ports = make([]*voltha.Port, 0)
+				onu.Ports = append(onu.Ports, ponPort)
+				for j := onuM.startingUniPortNo; j < onuM.numUnisPerOnu+onuM.startingUniPortNo; j++ {
+					uniPort := &voltha.Port{Label: fmt.Sprintf("%s:uni-%d", onu.Id, j), PortNo: uint32(j), DeviceId: onu.Id, Type: voltha.Port_ETHERNET_UNI}
+					onu.Ports = append(onu.Ports, uniPort)
+				}
+				onuM.deviceLock.Lock()
+				onuM.onus = append(onuM.onus, onu)
+				onuM.deviceLock.Unlock()
+				onuM.oltMgr.oltChnl <- onuRegistration{
+					onu:      onu,
+					oltPonNo: uint32(oltPonNum),
+					onuPonNo: 1,
+				}
+			}(i, oltPonNo)
+		}
+	}
+	wg.Wait()
+	//send an empty device to indicate the end of onu registration
+	onuM.oltMgr.oltChnl <- onuRegistration{
+		onu:      nil,
+		oltPonNo: 0,
+		onuPonNo: 1,
+	}
+}
+
+func (onuM *onuManager) getOnu(deviceID string) *voltha.Device {
+	onuM.deviceLock.Lock()
+	defer onuM.deviceLock.Unlock()
+	for _, onu := range onuM.onus {
+		if onu.Id == deviceID {
+			return onu
+		}
+	}
+	return nil
+}
+
+func (onuM *onuManager) GetDeviceHelper(_ context.Context, id string) (*voltha.Device, error) {
+	onuM.numGetDeviceInvokedLock.Lock()
+	onuM.numGetDeviceInvoked++
+	onuM.numGetDeviceInvokedLock.Unlock()
+	if id == oltDeviceID {
+		return onuM.oltMgr.olt, nil
+	}
+	if onu := onuM.getOnu(id); onu != nil {
+		return onu, nil
+	}
+	return nil, errors.New("not-found")
+}
+
+func TestDeviceRoutes_ComputeRoutes(t *testing.T) {
+	numNNIPort := 2
+	numPonPortOnOlt := 8
+	numOnuPerOltPonPort := 32
+	numUniPerOnu := 4
+	done := make(chan struct{})
+
+	fmt.Println(fmt.Sprintf("Test: Computing all routes. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d",
+		numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
+
+	// Create all the devices and logical device before computing the routes in one go
+	ld := &voltha.LogicalDevice{Id: logicalDeviceID}
+	ldMgrChnl := make(chan portRegistration, numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu)
+	ldMgr := newLogicalDeviceManager(ld, ldMgrChnl, numNNIPort+numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, done)
+	oltMgrChnl := make(chan onuRegistration, numPonPortOnOlt*numOnuPerOltPonPort)
+	oltMgr := newOltManager(oltDeviceID, ldMgr, numNNIPort, numPonPortOnOlt, oltMgrChnl)
+	onuMgr := newOnuManager(oltMgr, numOnuPerOltPonPort, numUniPerOnu, 2)
+	getDevice := onuMgr.GetDeviceHelper
+	// Start the managers.  Only the devices are created.  No routes will be built.
+	go ldMgr.start(getDevice, false)
+	go oltMgr.start()
+	go onuMgr.start(numNNIPort+1, numPonPortOnOlt)
+
+	// Wait for all the devices to be created
+	<-done
+	close(oltMgrChnl)
+	close(ldMgrChnl)
+
+	// Computes the routes
+	start := time.Now()
+	err := ldMgr.deviceRoutes.ComputeRoutes(context.TODO(), ldMgr.logicalDevice.Ports)
+	assert.Nil(t, err)
+
+	// Validate the routes are up to date
+	assert.True(t, ldMgr.deviceRoutes.IsUpToDate(ld))
+
+	// Validate the expected number of routes
+	assert.EqualValues(t, 2*numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, len(ldMgr.deviceRoutes.Routes))
+
+	// Validate the root ports
+	for _, port := range ldMgr.logicalDevice.Ports {
+		assert.Equal(t, port.RootPort, ldMgr.deviceRoutes.IsRootPort(port.OfpPort.PortNo))
+	}
+	fmt.Println(fmt.Sprintf("Total Time:%dms, Total Routes:%d NumGetDeviceInvoked:%d", time.Since(start)/time.Millisecond, len(ldMgr.deviceRoutes.Routes), onuMgr.numGetDeviceInvoked))
+}
+
+func TestDeviceRoutes_AddPort(t *testing.T) {
+	numNNIPort := 2
+	numPonPortOnOlt := 8
+	numOnuPerOltPonPort := 32
+	numUniPerOnu := 4
+	done := make(chan struct{})
+
+	fmt.Println(fmt.Sprintf("Test: Computing all routes. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d",
+		numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
+
+	start := time.Now()
+	// Create all the devices and logical device before computing the routes in one go
+	ld := &voltha.LogicalDevice{Id: logicalDeviceID}
+	ldMgrChnl := make(chan portRegistration, numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu)
+	ldMgr := newLogicalDeviceManager(ld, ldMgrChnl, numNNIPort+numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, done)
+	oltMgrChnl := make(chan onuRegistration, numPonPortOnOlt*numOnuPerOltPonPort)
+	oltMgr := newOltManager(oltDeviceID, ldMgr, numNNIPort, numPonPortOnOlt, oltMgrChnl)
+	onuMgr := newOnuManager(oltMgr, numOnuPerOltPonPort, numUniPerOnu, 2)
+	getDevice := onuMgr.GetDeviceHelper
+	// Start the managers and trigger the routes to be built as the logical ports become available
+	go ldMgr.start(getDevice, true)
+	go oltMgr.start()
+	go onuMgr.start(numNNIPort+1, numPonPortOnOlt)
+
+	// Wait for all the devices to be created and routes created
+	<-done
+	close(oltMgrChnl)
+	close(ldMgrChnl)
+
+	ldMgr.deviceRoutes.Print()
+
+	// Validate the routes are up to date
+	assert.True(t, ldMgr.deviceRoutes.IsUpToDate(ld))
+
+	// Validate the expected number of routes
+	assert.EqualValues(t, 2*numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, len(ldMgr.deviceRoutes.Routes))
+
+	// Validate the root ports
+	for _, port := range ldMgr.logicalDevice.Ports {
+		assert.Equal(t, port.RootPort, ldMgr.deviceRoutes.IsRootPort(port.OfpPort.PortNo))
+	}
+
+	fmt.Println(fmt.Sprintf("Total Time:%dms, Total Routes:%d NumGetDeviceInvoked:%d", time.Since(start)/time.Millisecond, len(ldMgr.deviceRoutes.Routes), onuMgr.numGetDeviceInvoked))
+}
+
+func TestDeviceRoutes_compareRoutesGeneration(t *testing.T) {
+	numNNIPort := 2
+	numPonPortOnOlt := 8
+	numOnuPerOltPonPort := 32
+	numUniPerOnu := 4
+	done := make(chan struct{})
+
+	fmt.Println(fmt.Sprintf("Test: Computing all routes. LogicalPorts:%d,  NNI:%d, Pon/OLT:%d, ONU/Pon:%d, Uni/Onu:%d",
+		numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, numNNIPort, numPonPortOnOlt, numOnuPerOltPonPort, numUniPerOnu))
+
+	// Create all the devices and logical device before computing the routes in one go
+	ld1 := &voltha.LogicalDevice{Id: logicalDeviceID}
+	ldMgrChnl1 := make(chan portRegistration, numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu)
+	ldMgr1 := newLogicalDeviceManager(ld1, ldMgrChnl1, numNNIPort+numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, done)
+	oltMgrChnl1 := make(chan onuRegistration, numPonPortOnOlt*numOnuPerOltPonPort)
+	oltMgr1 := newOltManager(oltDeviceID, ldMgr1, numNNIPort, numPonPortOnOlt, oltMgrChnl1)
+	onuMgr1 := newOnuManager(oltMgr1, numOnuPerOltPonPort, numUniPerOnu, 2)
+	getDevice := onuMgr1.GetDeviceHelper
+	// Start the managers.  Only the devices are created.  No routes will be built.
+	go ldMgr1.start(getDevice, false)
+	go oltMgr1.start()
+	go onuMgr1.start(numNNIPort+1, numPonPortOnOlt)
+
+	// Wait for all the devices to be created
+	<-done
+	close(oltMgrChnl1)
+	close(ldMgrChnl1)
+
+	err := ldMgr1.deviceRoutes.ComputeRoutes(context.TODO(), ldMgr1.logicalDevice.Ports)
+	assert.Nil(t, err)
+
+	routesGeneratedAllAtOnce := ldMgr1.deviceRoutes.Routes
+
+	done = make(chan struct{})
+	// Create all the devices and logical device before computing the routes in one go
+	ld2 := &voltha.LogicalDevice{Id: logicalDeviceID}
+	ldMgrChnl2 := make(chan portRegistration, numNNIPort*numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu)
+	ldMgr2 := newLogicalDeviceManager(ld2, ldMgrChnl2, numNNIPort+numPonPortOnOlt*numOnuPerOltPonPort*numUniPerOnu, done)
+	oltMgrChnl2 := make(chan onuRegistration, numPonPortOnOlt*numOnuPerOltPonPort)
+	oltMgr2 := newOltManager(oltDeviceID, ldMgr2, numNNIPort, numPonPortOnOlt, oltMgrChnl2)
+	onuMgr2 := newOnuManager(oltMgr2, numOnuPerOltPonPort, numUniPerOnu, 2)
+	// Start the managers.  Only the devices are created.  No routes will be built.
+	go ldMgr2.start(getDevice, true)
+	go oltMgr2.start()
+	go onuMgr2.start(numNNIPort+1, numPonPortOnOlt)
+
+	// Wait for all the devices to be created
+	<-done
+	close(oltMgrChnl2)
+	close(ldMgrChnl2)
+
+	routesGeneratedPerPort := ldMgr1.deviceRoutes.Routes
+	assert.True(t, isEqual(routesGeneratedAllAtOnce, routesGeneratedPerPort))
+}
+
+func TestDeviceRoutes_reverseRoute(t *testing.T) {
+	// Test the typical use case - 2 hops in a route
+	route := make([]Hop, 2)
+	route[0].DeviceID = "d1"
+	route[0].Ingress = 1
+	route[0].Egress = 2
+	route[1].DeviceID = "d2"
+	route[1].Ingress = 10
+	route[1].Egress = 15
+
+	reverseRoute := getReverseRoute(route)
+	assert.Equal(t, 2, len(reverseRoute))
+	assert.Equal(t, "d2", reverseRoute[0].DeviceID)
+	assert.Equal(t, "d1", reverseRoute[1].DeviceID)
+	assert.Equal(t, uint32(15), reverseRoute[0].Ingress)
+	assert.Equal(t, uint32(10), reverseRoute[0].Egress)
+	assert.Equal(t, uint32(2), reverseRoute[1].Ingress)
+	assert.Equal(t, uint32(1), reverseRoute[1].Egress)
+
+	fmt.Println("Reverse of two hops successful.")
+
+	//Test 3 hops in a route
+	route = make([]Hop, 3)
+	route[0].DeviceID = "d1"
+	route[0].Ingress = 1
+	route[0].Egress = 2
+	route[1].DeviceID = "d2"
+	route[1].Ingress = 10
+	route[1].Egress = 15
+	route[2].DeviceID = "d3"
+	route[2].Ingress = 20
+	route[2].Egress = 25
+	reverseRoute = getReverseRoute(route)
+	assert.Equal(t, 3, len(reverseRoute))
+	assert.Equal(t, "d3", reverseRoute[0].DeviceID)
+	assert.Equal(t, "d2", reverseRoute[1].DeviceID)
+	assert.Equal(t, "d1", reverseRoute[2].DeviceID)
+	assert.Equal(t, uint32(25), reverseRoute[0].Ingress)
+	assert.Equal(t, uint32(20), reverseRoute[0].Egress)
+	assert.Equal(t, uint32(15), reverseRoute[1].Ingress)
+	assert.Equal(t, uint32(10), reverseRoute[1].Egress)
+	assert.Equal(t, uint32(2), reverseRoute[2].Ingress)
+	assert.Equal(t, uint32(1), reverseRoute[2].Egress)
+
+	fmt.Println("Reverse of three hops successful.")
+
+	// Test any number of hops in a route
+	numRoutes := rand.Intn(100)
+	route = make([]Hop, numRoutes)
+	deviceIds := make([]string, numRoutes)
+	ingressNos := make([]uint32, numRoutes)
+	egressNos := make([]uint32, numRoutes)
+	for i := 0; i < numRoutes; i++ {
+		deviceIds[i] = fmt.Sprintf("d-%d", i)
+		ingressNos[i] = rand.Uint32()
+		egressNos[i] = rand.Uint32()
+	}
+	for i := 0; i < numRoutes; i++ {
+		route[i].DeviceID = deviceIds[i]
+		route[i].Ingress = ingressNos[i]
+		route[i].Egress = egressNos[i]
+	}
+	reverseRoute = getReverseRoute(route)
+	assert.Equal(t, numRoutes, len(reverseRoute))
+	for i, j := 0, numRoutes-1; j >= 0; i, j = i+1, j-1 {
+		assert.Equal(t, deviceIds[j], reverseRoute[i].DeviceID)
+		assert.Equal(t, egressNos[j], reverseRoute[i].Ingress)
+		assert.Equal(t, ingressNos[j], reverseRoute[i].Egress)
+	}
+
+	fmt.Println(fmt.Sprintf("Reverse of %d hops successful.", numRoutes))
+
+	reverseOfReverse := getReverseRoute(reverseRoute)
+	assert.Equal(t, route, reverseOfReverse)
+	fmt.Println("Reverse of reverse successful.")
+}
+
+func isEqual(routes1 map[PathID][]Hop, routes2 map[PathID][]Hop) bool {
+	if routes1 == nil && routes2 == nil {
+		return true
+	}
+	if (routes1 == nil && routes2 != nil) || (routes2 == nil && routes1 != nil) {
+		return false
+	}
+	if len(routes1) != len(routes2) {
+		return false
+	}
+	for routeID1, routeHop1 := range routes1 {
+		found := false
+		for routeID2, routeHop2 := range routes2 {
+			if routeID1 == routeID2 {
+				if !reflect.DeepEqual(routeHop1, routeHop2) {
+					return false
+				}
+				found = true
+				break
+			}
+		}
+		if !found {
+			return false
+		}
+	}
+	return true
+}