[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/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
+}