VOL-2867 - Replaced coreif interface package with smaller per-package interfaces.
Also moved state transition logic into its own package.
Also removed unused interfaces & mocks.
Change-Id: I849741853620684e6ceafe6e098a9c4f64fbdc6f
diff --git a/rw_core/core/device/agent.go b/rw_core/core/device/agent.go
index b6bd0ec..03cc3b8 100755
--- a/rw_core/core/device/agent.go
+++ b/rw_core/core/device/agent.go
@@ -663,15 +663,15 @@
}
logger.Debugw(ctx, "updated-device-in-store", log.Fields{"device-id: ": agent.deviceID})
- previousState := getDeviceStates(agent.device)
+ prevDevice := agent.device
// update the device
agent.device = device
// release lock before processing transition
agent.requestQueue.RequestComplete()
- if err := agent.deviceMgr.processTransition(log.WithSpanFromContext(context.Background(), ctx), device, previousState); err != nil {
- logger.Errorw(ctx, "failed-process-transition", log.Fields{"device-id": device.Id, "previousAdminState": previousState.Admin, "currentAdminState": device.AdminState})
+ if err := agent.deviceMgr.stateTransitions.ProcessTransition(log.WithSpanFromContext(context.Background(), ctx), device, prevDevice); err != nil {
+ logger.Errorw(ctx, "failed-process-transition", log.Fields{"device-id": device.Id, "previousAdminState": prevDevice.AdminState, "currentAdminState": device.AdminState})
}
return nil
}
diff --git a/rw_core/core/device/logical_agent.go b/rw_core/core/device/logical_agent.go
index ecc9008..8de602d 100644
--- a/rw_core/core/device/logical_agent.go
+++ b/rw_core/core/device/logical_agent.go
@@ -74,7 +74,7 @@
ldProxy: ldProxy,
ldeviceMgr: ldeviceMgr,
deviceRoutes: route.NewDeviceRoutes(id, deviceID, deviceMgr.listDevicePorts),
- flowDecomposer: fd.NewFlowDecomposer(deviceMgr),
+ flowDecomposer: fd.NewFlowDecomposer(deviceMgr.getDeviceReadOnly),
defaultTimeout: defaultTimeout,
requestQueue: coreutils.NewRequestQueue(),
diff --git a/rw_core/core/device/manager.go b/rw_core/core/device/manager.go
index 0afddbe..3b98cf8 100755
--- a/rw_core/core/device/manager.go
+++ b/rw_core/core/device/manager.go
@@ -19,8 +19,6 @@
import (
"context"
"errors"
- "reflect"
- "runtime"
"sync"
"time"
@@ -29,6 +27,7 @@
"github.com/opencord/voltha-go/rw_core/core/adapter"
"github.com/opencord/voltha-go/rw_core/core/device/event"
"github.com/opencord/voltha-go/rw_core/core/device/remote"
+ "github.com/opencord/voltha-go/rw_core/core/device/state"
"github.com/opencord/voltha-go/rw_core/utils"
"github.com/opencord/voltha-lib-go/v3/pkg/kafka"
"github.com/opencord/voltha-lib-go/v3/pkg/log"
@@ -50,7 +49,7 @@
adapterMgr *adapter.Manager
logicalDeviceMgr *LogicalManager
kafkaICProxy kafka.InterContainerProxy
- stateTransitions *TransitionMap
+ stateTransitions *state.TransitionMap
dbPath *model.Path
dProxy *model.Proxy
coreInstanceID string
@@ -72,7 +71,7 @@
defaultTimeout: defaultCoreTimeout,
deviceLoadingInProgress: make(map[string][]chan int),
}
- deviceMgr.stateTransitions = NewTransitionMap(deviceMgr)
+ deviceMgr.stateTransitions = state.NewTransitionMap(deviceMgr)
logicalDeviceMgr := &LogicalManager{
Manager: event.NewManager(),
@@ -291,7 +290,7 @@
func (dMgr *Manager) stopManagingDevice(ctx context.Context, id string) {
logger.Infow(ctx, "stopManagingDevice", log.Fields{"device-id": id})
if dMgr.IsDeviceInCache(id) { // Proceed only if an agent is present for this device
- if root, _ := dMgr.IsRootDevice(id); root {
+ if device, err := dMgr.getDeviceReadOnly(ctx, id); err == nil && device.Root {
// stop managing the logical device
_ = dMgr.logicalDeviceMgr.stopManagingLogicalDeviceWithDeviceID(ctx, id)
}
@@ -434,16 +433,6 @@
return exist
}
-// IsRootDevice returns true if root device is found in the map
-func (dMgr *Manager) IsRootDevice(id string) (bool, error) {
- dMgr.lockRootDeviceMap.RLock()
- defer dMgr.lockRootDeviceMap.RUnlock()
- if exist := dMgr.rootDevices[id]; exist {
- return dMgr.rootDevices[id], nil
- }
- return false, nil
-}
-
// ListDevices retrieves the latest devices from the data model
func (dMgr *Manager) ListDevices(ctx context.Context, _ *empty.Empty) (*voltha.Devices, error) {
logger.Debug(ctx, "ListDevices")
@@ -1076,33 +1065,6 @@
return childDevice, nil
}
-func (dMgr *Manager) processTransition(ctx context.Context, device *voltha.Device, previousState *deviceState) error {
- // This will be triggered on every state update
- logger.Debugw(ctx, "state-transition", log.Fields{
- "device": device.Id,
- "prev-admin-state": previousState.Admin,
- "prev-oper-state": previousState.Operational,
- "prev-conn-state": previousState.Connection,
- "curr-admin-state": device.AdminState,
- "curr-oper-state": device.OperStatus,
- "curr-conn-state": device.ConnectStatus,
- })
- handlers := dMgr.stateTransitions.GetTransitionHandler(ctx, device, previousState)
- if handlers == nil {
- logger.Debugw(ctx, "no-op-transition", log.Fields{"device-id": device.Id})
- return nil
- }
- logger.Debugw(ctx, "handler-found", log.Fields{"num-expectedHandlers": len(handlers), "isParent": device.Root, "current-data": device, "previous-state": previousState})
- for _, handler := range handlers {
- logger.Debugw(ctx, "running-handler", log.Fields{"handler": funcName(handler)})
- if err := handler(ctx, device); err != nil {
- logger.Warnw(ctx, "handler-failed", log.Fields{"handler": funcName(handler), "error": err})
- return err
- }
- }
- return nil
-}
-
func (dMgr *Manager) packetOut(ctx context.Context, deviceID string, outPort uint32, packet *ofp.OfpPacketOut) error {
logger.Debugw(ctx, "packetOut", log.Fields{"device-id": deviceID, "outPort": outPort})
if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
@@ -1494,12 +1456,6 @@
return nil
}
-func funcName(f interface{}) string {
- p := reflect.ValueOf(f).Pointer()
- rf := runtime.FuncForPC(p)
- return rf.Name()
-}
-
// UpdateDeviceAttribute updates value of particular device attribute
func (dMgr *Manager) UpdateDeviceAttribute(ctx context.Context, deviceID string, attribute string, value interface{}) {
if agent, ok := dMgr.deviceAgents.Load(deviceID); ok {
diff --git a/rw_core/core/device/state/common.go b/rw_core/core/device/state/common.go
new file mode 100644
index 0000000..5353059
--- /dev/null
+++ b/rw_core/core/device/state/common.go
@@ -0,0 +1,33 @@
+/*
+ * 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 core Common Logger initialization
+package state
+
+import (
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+ // Setup this package so that it's log level can be modified at run time
+ var err error
+ logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "state"})
+ if err != nil {
+ panic(err)
+ }
+}
diff --git a/rw_core/core/device/state_transitions.go b/rw_core/core/device/state/transitions.go
similarity index 75%
rename from rw_core/core/device/state_transitions.go
rename to rw_core/core/device/state/transitions.go
index 3800c2f..d69d949 100644
--- a/rw_core/core/device/state_transitions.go
+++ b/rw_core/core/device/state/transitions.go
@@ -14,11 +14,14 @@
* limitations under the License.
*/
-package device
+package state
import (
"context"
- "github.com/opencord/voltha-go/rw_core/coreif"
+ "reflect"
+ "runtime"
+
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
"github.com/opencord/voltha-protos/v3/go/voltha"
)
@@ -67,186 +70,200 @@
Operational voltha.OperStatus_Types
}
-// TransitionHandler function type which takes the current and previous device info as input parameter
-type TransitionHandler func(context.Context, *voltha.Device) error
+// transitionHandler function type which takes the current and previous device info as input parameter
+type transitionHandler func(context.Context, *voltha.Device) error
-// Transition represent transition related attributes
-type Transition struct {
+// transition represent transition related attributes
+type transition struct {
deviceType deviceType
previousState deviceState
currentState deviceState
- handlers []TransitionHandler
+ handlers []transitionHandler
}
// TransitionMap represent map of transitions and device manager
type TransitionMap struct {
- transitions []Transition
- dMgr coreif.DeviceManager
+ transitions []transition
+ dMgr DeviceManager
+}
+
+// DeviceManager represents a generic device manager
+type DeviceManager interface {
+ NotifyInvalidTransition(ctx context.Context, curr *voltha.Device) error
+ CreateLogicalDevice(ctx context.Context, curr *voltha.Device) error
+ SetupUNILogicalPorts(ctx context.Context, curr *voltha.Device) error
+ DeleteLogicalDevice(ctx context.Context, curr *voltha.Device) error
+ DeleteLogicalPorts(ctx context.Context, curr *voltha.Device) error
+ DeleteAllChildDevices(ctx context.Context, curr *voltha.Device) error
+ RunPostDeviceDelete(ctx context.Context, curr *voltha.Device) error
+ ChildDeviceLost(ctx context.Context, curr *voltha.Device) error
+ DeleteAllLogicalPorts(ctx context.Context, curr *voltha.Device) error
+ DeleteAllDeviceFlows(ctx context.Context, curr *voltha.Device) error
}
// NewTransitionMap creates transition map
-func NewTransitionMap(dMgr coreif.DeviceManager) *TransitionMap {
+func NewTransitionMap(dMgr DeviceManager) *TransitionMap {
var transitionMap TransitionMap
transitionMap.dMgr = dMgr
- transitionMap.transitions = make([]Transition, 0)
+ transitionMap.transitions = make([]transition, 0)
transitionMap.transitions = append(
transitionMap.transitions,
- Transition{
+ transition{
deviceType: parent,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE},
- handlers: []TransitionHandler{dMgr.CreateLogicalDevice}})
+ handlers: []transitionHandler{dMgr.CreateLogicalDevice}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: child,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
- handlers: []TransitionHandler{}})
+ handlers: []transitionHandler{}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: child,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE},
- handlers: []TransitionHandler{dMgr.SetupUNILogicalPorts}})
+ handlers: []transitionHandler{dMgr.SetupUNILogicalPorts}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: child,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_DISCOVERED},
- handlers: []TransitionHandler{}})
+ handlers: []transitionHandler{}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: child,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE},
- handlers: []TransitionHandler{dMgr.SetupUNILogicalPorts}})
+ handlers: []transitionHandler{dMgr.SetupUNILogicalPorts}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DELETED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.RunPostDeviceDelete}})
+ handlers: []transitionHandler{dMgr.RunPostDeviceDelete}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: parent,
previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DELETED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.RunPostDeviceDelete}})
+ handlers: []transitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.RunPostDeviceDelete}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: parent,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.DeleteAllDeviceFlows}})
+ handlers: []transitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.DeleteAllDeviceFlows}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: parent,
previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.DeleteAllDeviceFlows}})
+ handlers: []transitionHandler{dMgr.DeleteAllLogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteLogicalDevice, dMgr.DeleteAllDeviceFlows}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: parent,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_ACTIVE},
- handlers: []TransitionHandler{dMgr.CreateLogicalDevice}})
+ handlers: []transitionHandler{dMgr.CreateLogicalDevice}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: parent,
previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNREACHABLE, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_REACHABLE, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.CreateLogicalDevice}})
+ handlers: []transitionHandler{dMgr.CreateLogicalDevice}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: child,
previousState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DELETED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.ChildDeviceLost, dMgr.DeleteLogicalPorts, dMgr.RunPostDeviceDelete}})
+ handlers: []transitionHandler{dMgr.ChildDeviceLost, dMgr.DeleteLogicalPorts, dMgr.RunPostDeviceDelete}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: child,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DELETED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.ChildDeviceLost, dMgr.DeleteLogicalPorts, dMgr.RunPostDeviceDelete}})
+ handlers: []transitionHandler{dMgr.ChildDeviceLost, dMgr.DeleteLogicalPorts, dMgr.RunPostDeviceDelete}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: parent,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVE},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_ACTIVATING},
currentState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_ENABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: child,
previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_UNKNOWN, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_PREPROVISIONED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
transitionMap.transitions = append(transitionMap.transitions,
- Transition{
+ transition{
deviceType: any,
previousState: deviceState{Admin: voltha.AdminState_DOWNLOADING_IMAGE, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
currentState: deviceState{Admin: voltha.AdminState_DISABLED, Connection: voltha.ConnectStatus_UNKNOWN, Operational: voltha.OperStatus_UNKNOWN},
- handlers: []TransitionHandler{dMgr.NotifyInvalidTransition}})
+ handlers: []transitionHandler{dMgr.NotifyInvalidTransition}})
return &transitionMap
}
-func getDeviceStates(device *voltha.Device) *deviceState {
- return &deviceState{Admin: device.AdminState, Connection: device.ConnectStatus, Operational: device.OperStatus}
+func getDeviceStates(device *voltha.Device) deviceState {
+ return deviceState{Admin: device.AdminState, Connection: device.ConnectStatus, Operational: device.OperStatus}
}
// isMatched matches a state transition. It returns whether there is a match and if there is whether it is an exact match
-func getHandler(previous *deviceState, current *deviceState, transition *Transition) ([]TransitionHandler, *match) {
+func getHandler(previous deviceState, current deviceState, transition *transition) ([]transitionHandler, *match) {
m := &match{}
// Do we have an exact match?
- if *previous == transition.previousState && *current == transition.currentState {
+ if previous == transition.previousState && current == transition.currentState {
return transition.handlers, &match{admin: currPrevStateMatch, oper: currPrevStateMatch, conn: currPrevStateMatch}
}
@@ -296,27 +313,28 @@
return transition.handlers, m
}
-// GetTransitionHandler returns transition handler & a flag that's set if the transition is invalid
-func (tMap *TransitionMap) GetTransitionHandler(ctx context.Context, device *voltha.Device, pState *deviceState) []TransitionHandler {
+// getTransitionHandler returns transition handler & a flag that's set if the transition is invalid
+func (tMap *TransitionMap) getTransitionHandler(ctx context.Context, cDevice, pDevice *voltha.Device) []transitionHandler {
//1. Get the previous and current set of states
- cState := getDeviceStates(device)
+ cState := getDeviceStates(cDevice)
+ pState := getDeviceStates(pDevice)
// Do nothing is there are no states change
- if *pState == *cState {
+ if pState == cState {
return nil
}
//logger.Infow(ctx, "deviceType", log.Fields{"device": pDevice})
deviceType := parent
- if !device.Root {
+ if !cDevice.Root {
logger.Info(ctx, "device is child")
deviceType = child
}
- logger.Infof(ctx, "deviceType:%d-deviceId:%s-previous:%v-current:%v", deviceType, device.Id, pState, cState)
+ logger.Infof(ctx, "deviceType:%d-deviceId:%s-previous:%v-current:%v", deviceType, cDevice.Id, pState, cState)
//2. Go over transition array to get the right transition
- var currentMatch []TransitionHandler
- var tempHandler []TransitionHandler
+ var currentMatch []transitionHandler
+ var tempHandler []transitionHandler
var m *match
bestMatch := &match{}
for _, aTransition := range tMap.transitions {
@@ -336,3 +354,34 @@
}
return currentMatch
}
+
+func (tMap *TransitionMap) ProcessTransition(ctx context.Context, device, prevDevice *voltha.Device) error {
+ // This will be triggered on every state update
+ logger.Debugw(ctx, "state-transition", log.Fields{
+ "device": device.Id,
+ "prev-admin-state": prevDevice.AdminState,
+ "prev-oper-state": prevDevice.OperStatus,
+ "prev-conn-state": prevDevice.ConnectStatus,
+ "curr-admin-state": device.AdminState,
+ "curr-oper-state": device.OperStatus,
+ "curr-conn-state": device.ConnectStatus,
+ })
+ handlers := tMap.getTransitionHandler(ctx, device, prevDevice)
+ if handlers == nil {
+ logger.Debugw(ctx, "no-op-transition", log.Fields{"deviceId": device.Id})
+ return nil
+ }
+ logger.Debugw(ctx, "handler-found", log.Fields{"num-expectedHandlers": len(handlers), "isParent": device.Root, "current-data": device, "previous-data": prevDevice})
+ for _, handler := range handlers {
+ logger.Debugw(ctx, "running-handler", log.Fields{"handler": funcName(handler)})
+ if err := handler(ctx, device); err != nil {
+ logger.Warnw(ctx, "handler-failed", log.Fields{"handler": funcName(handler), "error": err})
+ return err
+ }
+ }
+ return nil
+}
+
+func funcName(f interface{}) string {
+ return runtime.FuncForPC(reflect.ValueOf(f).Pointer()).Name()
+}
diff --git a/rw_core/core/device/state/transitions_test.go b/rw_core/core/device/state/transitions_test.go
new file mode 100644
index 0000000..39d7d0b
--- /dev/null
+++ b/rw_core/core/device/state/transitions_test.go
@@ -0,0 +1,329 @@
+/*
+ * Copyright 2019-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 state
+
+import (
+ "context"
+ "fmt"
+ "reflect"
+ "testing"
+
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "github.com/stretchr/testify/assert"
+)
+
+var transitionMap *TransitionMap
+var tdm DeviceManager
+
+type testDeviceManager struct {
+ DeviceManager
+}
+
+func newTestDeviceManager() *testDeviceManager {
+ return &testDeviceManager{}
+}
+
+func init() {
+ tdm = newTestDeviceManager()
+ transitionMap = NewTransitionMap(tdm)
+}
+
+func getDevice(root bool, admin voltha.AdminState_Types, conn voltha.ConnectStatus_Types, oper voltha.OperStatus_Types) *voltha.Device {
+ return &voltha.Device{
+ Id: "test",
+ Root: root,
+ AdminState: admin,
+ ConnectStatus: conn,
+ OperStatus: oper,
+ }
+}
+
+func assertInvalidTransition(t *testing.T, device, prevDevice *voltha.Device) {
+ handlers := transitionMap.getTransitionHandler(context.Background(), device, prevDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.NotifyInvalidTransition).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+}
+
+func assertNoOpTransition(t *testing.T, device, prevDevice *voltha.Device) {
+ handlers := transitionMap.getTransitionHandler(context.Background(), device, prevDevice)
+ assert.Equal(t, 0, len(handlers))
+}
+
+func TestValidTransitions(t *testing.T) {
+ ctx := context.Background()
+ previousDevice := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
+ handlers := transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
+ device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
+ device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_DISCOVERED)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
+ device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
+ device = getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_FAILED)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 3, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.ChildDeviceLost).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteLogicalPorts).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[2]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+ device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 4, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllLogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteLogicalDevice).Pointer() == reflect.ValueOf(handlers[2]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllDeviceFlows).Pointer() == reflect.ValueOf(handlers[3]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 4, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllLogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteLogicalDevice).Pointer() == reflect.ValueOf(handlers[2]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllDeviceFlows).Pointer() == reflect.ValueOf(handlers[3]).Pointer())
+
+ previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 1, len(handlers))
+ assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
+
+ var deleteDeviceTest = struct {
+ previousDevices []*voltha.Device
+ devices []*voltha.Device
+ expectedParentHandlers []transitionHandler
+ expectedChildHandlers []transitionHandler
+ }{
+ previousDevices: []*voltha.Device{
+ getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_FAILED),
+ getDevice(false, voltha.AdminState_UNKNOWN, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
+ getDevice(false, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
+ getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
+ getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE),
+ getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN),
+ getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN),
+ },
+ devices: []*voltha.Device{
+ getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
+ getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN),
+ getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_FAILED),
+ },
+ expectedParentHandlers: []transitionHandler{
+ tdm.DeleteAllLogicalPorts,
+ tdm.DeleteAllChildDevices,
+ tdm.DeleteLogicalDevice,
+ tdm.RunPostDeviceDelete,
+ },
+ expectedChildHandlers: []transitionHandler{
+ tdm.ChildDeviceLost,
+ tdm.DeleteLogicalPorts,
+ tdm.RunPostDeviceDelete,
+ },
+ }
+
+ testName := "delete-parent-device-post-provisioning"
+ for _, previousDevice := range deleteDeviceTest.previousDevices {
+ for _, device := range deleteDeviceTest.devices {
+ device.Root = true
+ t.Run(testName, func(t *testing.T) {
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 4, len(handlers))
+ for idx, expHandler := range deleteDeviceTest.expectedParentHandlers {
+ assert.True(t, reflect.ValueOf(expHandler).Pointer() == reflect.ValueOf(handlers[idx]).Pointer())
+ }
+ })
+ }
+ }
+
+ testName = "delete-child-device"
+ for _, previousDevice := range deleteDeviceTest.previousDevices {
+ for _, device := range deleteDeviceTest.devices {
+ device.Root = false
+ t.Run(testName, func(t *testing.T) {
+ handlers = transitionMap.getTransitionHandler(ctx, device, previousDevice)
+ assert.Equal(t, 3, len(handlers))
+ for idx, expHandler := range deleteDeviceTest.expectedChildHandlers {
+ assert.True(t, reflect.ValueOf(expHandler).Pointer() == reflect.ValueOf(handlers[idx]).Pointer())
+ }
+ })
+ }
+ }
+}
+
+func TestInvalidTransitions(t *testing.T) {
+ previousDevice := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
+ device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ assertInvalidTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertInvalidTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_UNKNOWN, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertInvalidTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertInvalidTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_UNKNOWN, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertInvalidTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertInvalidTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertInvalidTransition(t, device, previousDevice)
+}
+
+func TestNoOpTransitions(t *testing.T) {
+ previousDevice := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertNoOpTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertNoOpTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertNoOpTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
+ assertNoOpTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
+ device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
+ assertNoOpTransition(t, device, previousDevice)
+
+ previousDevice = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
+ device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING)
+ assertNoOpTransition(t, device, previousDevice)
+}
+
+func TestMatch(t *testing.T) {
+ best := &match{admin: currPrevStateMatch, oper: currPrevStateMatch, conn: currPrevStateMatch}
+ m := &match{admin: currStateOnlyMatch, oper: currWildcardMatch, conn: currWildcardMatch}
+ fmt.Println(m.isBetterMatch(best), m.toInt(), best.toInt())
+}
diff --git a/rw_core/core/device/state_transitions_test.go b/rw_core/core/device/state_transitions_test.go
deleted file mode 100644
index 458d178..0000000
--- a/rw_core/core/device/state_transitions_test.go
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * Copyright 2019-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 device
-
-import (
- "context"
- "fmt"
- "reflect"
- "testing"
-
- "github.com/opencord/voltha-go/rw_core/coreif"
- "github.com/opencord/voltha-go/rw_core/mocks"
- "github.com/opencord/voltha-protos/v3/go/voltha"
- "github.com/stretchr/testify/assert"
-)
-
-var transitionMap *TransitionMap
-var tdm coreif.DeviceManager
-
-type testDeviceManager struct {
- mocks.DeviceManager
-}
-
-func newTestDeviceManager() *testDeviceManager {
- return &testDeviceManager{}
-}
-
-func init() {
- tdm = newTestDeviceManager()
- transitionMap = NewTransitionMap(tdm)
-}
-
-func getDevice(root bool, admin voltha.AdminState_Types, conn voltha.ConnectStatus_Types, oper voltha.OperStatus_Types) *voltha.Device {
- return &voltha.Device{
- Id: "test",
- Root: root,
- AdminState: admin,
- ConnectStatus: conn,
- OperStatus: oper,
- }
-}
-
-func getDeviceState(admin voltha.AdminState_Types, conn voltha.ConnectStatus_Types, oper voltha.OperStatus_Types) *deviceState {
- return &deviceState{
- Admin: admin,
- Connection: conn,
- Operational: oper,
- }
-}
-
-func assertInvalidTransition(t *testing.T, device *voltha.Device, previousState *deviceState) {
- handlers := transitionMap.GetTransitionHandler(context.Background(), device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.NotifyInvalidTransition).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-}
-
-func assertNoOpTransition(t *testing.T, device *voltha.Device, previousState *deviceState) {
- handlers := transitionMap.GetTransitionHandler(context.Background(), device, previousState)
- assert.Equal(t, 0, len(handlers))
-}
-
-func TestValidTransitions(t *testing.T) {
- ctx := context.Background()
- previousState := getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
- handlers := transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
- device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
- device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_DISCOVERED)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_DISCOVERED)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVATING)
- device = getDevice(false, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.SetupUNILogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
- device = getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_FAILED)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 3, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.ChildDeviceLost).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.DeleteLogicalPorts).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.RunPostDeviceDelete).Pointer() == reflect.ValueOf(handlers[2]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
- device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 4, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.DeleteAllLogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.DeleteLogicalDevice).Pointer() == reflect.ValueOf(handlers[2]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.DeleteAllDeviceFlows).Pointer() == reflect.ValueOf(handlers[3]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 4, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.DeleteAllLogicalPorts).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.DeleteLogicalDevice).Pointer() == reflect.ValueOf(handlers[2]).Pointer())
- assert.True(t, reflect.ValueOf(tdm.DeleteAllDeviceFlows).Pointer() == reflect.ValueOf(handlers[3]).Pointer())
-
- previousState = getDeviceState(voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN)
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 1, len(handlers))
- assert.True(t, reflect.ValueOf(tdm.CreateLogicalDevice).Pointer() == reflect.ValueOf(handlers[0]).Pointer())
-
- var deleteDeviceTest = struct {
- previousStates []*deviceState
- devices []*voltha.Device
- expectedParentHandlers []TransitionHandler
- expectedChildHandlers []TransitionHandler
- }{
- previousStates: []*deviceState{
- getDeviceState(voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_FAILED),
- getDeviceState(voltha.AdminState_UNKNOWN, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
- getDeviceState(voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
- getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
- getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE),
- getDeviceState(voltha.AdminState_DISABLED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN),
- getDeviceState(voltha.AdminState_DISABLED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN),
- },
- devices: []*voltha.Device{
- getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN),
- getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_UNKNOWN),
- getDevice(false, voltha.AdminState_DELETED, voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_FAILED),
- },
- expectedParentHandlers: []TransitionHandler{
- tdm.DeleteAllLogicalPorts,
- tdm.DeleteAllChildDevices,
- tdm.DeleteLogicalDevice,
- tdm.RunPostDeviceDelete,
- },
- expectedChildHandlers: []TransitionHandler{
- tdm.ChildDeviceLost,
- tdm.DeleteLogicalPorts,
- tdm.RunPostDeviceDelete,
- },
- }
-
- testName := "delete-parent-device-post-provisioning"
- for _, previousState := range deleteDeviceTest.previousStates {
- for _, device := range deleteDeviceTest.devices {
- device.Root = true
- t.Run(testName, func(t *testing.T) {
- handlers = transitionMap.GetTransitionHandler(ctx, device, previousState)
- assert.Equal(t, 4, len(handlers))
- for idx, expHandler := range deleteDeviceTest.expectedParentHandlers {
- assert.True(t, reflect.ValueOf(expHandler).Pointer() == reflect.ValueOf(handlers[idx]).Pointer())
- }
- })
- }
- }
-
- testName = "delete-child-device"
- for _, deviceState := range deleteDeviceTest.previousStates {
- for _, device := range deleteDeviceTest.devices {
- device.Root = false
- t.Run(testName, func(t *testing.T) {
- handlers = transitionMap.GetTransitionHandler(ctx, device, deviceState)
- assert.Equal(t, 3, len(handlers))
- for idx, expHandler := range deleteDeviceTest.expectedChildHandlers {
- assert.True(t, reflect.ValueOf(expHandler).Pointer() == reflect.ValueOf(handlers[idx]).Pointer())
- }
- })
- }
- }
-}
-
-func TestInvalidTransitions(t *testing.T) {
- previousState := getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVE)
- device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- assertInvalidTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device = getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertInvalidTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_UNKNOWN, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertInvalidTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertInvalidTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_UNKNOWN, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertInvalidTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertInvalidTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_DOWNLOADING_IMAGE, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertInvalidTransition(t, device, previousState)
-}
-
-func TestNoOpTransitions(t *testing.T) {
- previousState := getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device := getDevice(true, voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertNoOpTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertNoOpTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(true, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertNoOpTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_ENABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- device = getDevice(false, voltha.AdminState_DISABLED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_UNKNOWN)
- assertNoOpTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_UNKNOWN, voltha.OperStatus_ACTIVATING)
- device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
- assertNoOpTransition(t, device, previousState)
-
- previousState = getDeviceState(voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_DISCOVERED)
- device = getDevice(false, voltha.AdminState_PREPROVISIONED, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING)
- assertNoOpTransition(t, device, previousState)
-}
-
-func TestMatch(t *testing.T) {
- best := &match{admin: currPrevStateMatch, oper: currPrevStateMatch, conn: currPrevStateMatch}
- m := &match{admin: currStateOnlyMatch, oper: currWildcardMatch, conn: currWildcardMatch}
- fmt.Println(m.isBetterMatch(best), m.toInt(), best.toInt())
-}
diff --git a/rw_core/coreif/adapter_manager_if.go b/rw_core/coreif/adapter_manager_if.go
deleted file mode 100644
index 7f661cf..0000000
--- a/rw_core/coreif/adapter_manager_if.go
+++ /dev/null
@@ -1,31 +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 coreif
-
-import (
- "context"
-
- "github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-// AdapterManager represent adapter manager related methods
-type AdapterManager interface {
- ListAdapters(ctx context.Context) (*voltha.Adapters, error)
- GetAdapterName(deviceType string) (string, error)
- GetDeviceType(deviceType string) *voltha.DeviceType
- RegisterAdapter(adapter *voltha.Adapter, deviceTypes *voltha.DeviceTypes) *voltha.CoreInstance
-}
diff --git a/rw_core/coreif/core_if.go b/rw_core/coreif/core_if.go
deleted file mode 100644
index 6454135..0000000
--- a/rw_core/coreif/core_if.go
+++ /dev/null
@@ -1,43 +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.
- */
-/*
-Defines a DeviceManager Interface - Used for unit testing of the flow decomposer only at this
-time.
-*/
-
-package coreif
-
-import (
- "context"
-
- "github.com/opencord/voltha-go/db/model"
- "github.com/opencord/voltha-go/rw_core/config"
- "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
-)
-
-// Core represent core methods
-type Core interface {
- Start(ctx context.Context)
- Stop(ctx context.Context)
- GetKafkaInterContainerProxy() *kafka.InterContainerProxy
- GetConfig() *config.RWCoreFlags
- GetInstanceId() string
- GetClusterDataProxy() *model.Proxy
- GetAdapterManager() AdapterManager
- StartGRPCService(ctx context.Context)
- GetDeviceManager() DeviceManager
- GetLogicalDeviceManager() LogicalDeviceManager
-}
diff --git a/rw_core/coreif/device_manager_if.go b/rw_core/coreif/device_manager_if.go
deleted file mode 100644
index 4d23e5c..0000000
--- a/rw_core/coreif/device_manager_if.go
+++ /dev/null
@@ -1,43 +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.
- */
-/*
-Defines a DeviceManager Interface - Used for unit testing of the flow decomposer only at this
-time.
-*/
-
-package coreif
-
-import (
- "context"
- "github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-// DeviceManager represents a generic device manager
-type DeviceManager interface {
- GetDevice(context.Context, *voltha.ID) (*voltha.Device, error)
- IsRootDevice(string) (bool, error)
- NotifyInvalidTransition(ctx context.Context, curr *voltha.Device) error
- CreateLogicalDevice(ctx context.Context, curr *voltha.Device) error
- SetupUNILogicalPorts(ctx context.Context, curr *voltha.Device) error
- DisableAllChildDevices(ctx context.Context, curr *voltha.Device) error
- DeleteLogicalDevice(ctx context.Context, curr *voltha.Device) error
- DeleteLogicalPorts(ctx context.Context, curr *voltha.Device) error
- DeleteAllChildDevices(ctx context.Context, curr *voltha.Device) error
- RunPostDeviceDelete(ctx context.Context, curr *voltha.Device) error
- ChildDeviceLost(ctx context.Context, curr *voltha.Device) error
- DeleteAllLogicalPorts(ctx context.Context, curr *voltha.Device) error
- DeleteAllDeviceFlows(ctx context.Context, curr *voltha.Device) error
-}
diff --git a/rw_core/coreif/logical_device_agent_if.go b/rw_core/coreif/logical_device_agent_if.go
deleted file mode 100644
index f3c8644..0000000
--- a/rw_core/coreif/logical_device_agent_if.go
+++ /dev/null
@@ -1,35 +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.
- */
-/*
- Defines a logicalDeviceAgent Interface - Used for unit testing of the flow decomposer only at this
- time.
-*/
-
-package coreif
-
-import (
- "context"
-
- "github.com/opencord/voltha-go/rw_core/route"
-)
-
-// LogicalDeviceAgent represents a generic agent
-type LogicalDeviceAgent interface {
- GetDeviceRoutes() *route.DeviceRoutes
- GetWildcardInputPorts(ctx context.Context, excludePort uint32) map[uint32]struct{}
- GetRoute(ctx context.Context, ingressPortNo uint32, egressPortNo uint32) ([]route.Hop, error)
- GetNNIPorts() map[uint32]struct{}
-}
diff --git a/rw_core/coreif/logical_device_manager_if.go b/rw_core/coreif/logical_device_manager_if.go
deleted file mode 100644
index 02068eb..0000000
--- a/rw_core/coreif/logical_device_manager_if.go
+++ /dev/null
@@ -1,46 +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.
- */
-/*
-Defines a DeviceManager Interface - Used for unit testing of the flow decomposer only at this
-time.
-*/
-
-package coreif
-
-import (
- "context"
-
- "github.com/opencord/voltha-protos/v3/go/openflow_13"
- "github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-// LogicalDeviceManager represent logical device manager related methods
-type LogicalDeviceManager interface {
- GetLogicalPort(lPortID *voltha.LogicalPortId) (*voltha.LogicalPort, error)
- EnableLogicalPort(ctx context.Context, id *voltha.LogicalPortId, ch chan interface{})
- DisableLogicalPort(ctx context.Context, id *voltha.LogicalPortId, ch chan interface{})
- UpdateFlowTable(ctx context.Context, id string, flow *openflow_13.OfpFlowMod, ch chan interface{})
- UpdateMeterTable(ctx context.Context, id string, meter *openflow_13.OfpMeterMod, ch chan interface{})
- UpdateGroupTable(ctx context.Context, id string, groupMod *openflow_13.OfpGroupMod, ch chan interface{})
- GetLogicalDevice(id string) (*voltha.LogicalDevice, error)
- ListManagedLogicalDevices() (*voltha.LogicalDevices, error)
- ListLogicalDevices() (*voltha.LogicalDevices, error)
- ListLogicalDeviceFlows(ctx context.Context, id string) (*openflow_13.Flows, error)
- ListLogicalDeviceFlowGroups(ctx context.Context, id string) (*openflow_13.FlowGroups, error)
- ListLogicalDevicePorts(ctx context.Context, id string) (*voltha.LogicalPorts, error)
- ListLogicalDeviceMeters(ctx context.Context, id string) (*openflow_13.Meters, error)
- PacketOut(packet *openflow_13.PacketOut) error
-}
diff --git a/rw_core/flowdecomposition/flow_decomposer.go b/rw_core/flowdecomposition/flow_decomposer.go
index 142c381..2b4f194 100644
--- a/rw_core/flowdecomposition/flow_decomposer.go
+++ b/rw_core/flowdecomposition/flow_decomposer.go
@@ -21,7 +21,6 @@
"fmt"
"github.com/gogo/protobuf/proto"
- "github.com/opencord/voltha-go/rw_core/coreif"
"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"
@@ -33,18 +32,26 @@
// FlowDecomposer represent flow decomposer attribute
type FlowDecomposer struct {
- deviceMgr coreif.DeviceManager
+ getDevice GetDeviceFunc
+}
+
+// DeviceManager represents a generic device manager
+type GetDeviceFunc func(context.Context, string) (*voltha.Device, error)
+
+type LogicalDeviceAgent interface {
+ GetDeviceRoutes() *route.DeviceRoutes
+ GetWildcardInputPorts(ctx context.Context, excludePort uint32) map[uint32]struct{}
+ GetRoute(ctx context.Context, ingressPortNo uint32, egressPortNo uint32) ([]route.Hop, error)
+ GetNNIPorts() map[uint32]struct{}
}
// NewFlowDecomposer creates flow decomposer instance
-func NewFlowDecomposer(deviceMgr coreif.DeviceManager) *FlowDecomposer {
- var decomposer FlowDecomposer
- decomposer.deviceMgr = deviceMgr
- return &decomposer
+func NewFlowDecomposer(getDevice GetDeviceFunc) *FlowDecomposer {
+ return &FlowDecomposer{getDevice: getDevice}
}
//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 map[uint64]*ofp.OfpFlowStats, groups map[uint32]*ofp.OfpGroupEntry) (*fu.DeviceRules, error) {
+func (fd *FlowDecomposer) DecomposeRules(ctx context.Context, agent LogicalDeviceAgent, flows map[uint64]*ofp.OfpFlowStats, groups map[uint32]*ofp.OfpGroupEntry) (*fu.DeviceRules, error) {
deviceRules := *fu.NewDeviceRules()
devicesToUpdate := make(map[string]string)
@@ -63,8 +70,7 @@
}
// Handles special case of any controller-bound flow for a parent device
-func (fd *FlowDecomposer) updateOutputPortForControllerBoundFlowForParentDevide(flow *ofp.OfpFlowStats,
- dr *fu.DeviceRules) (*fu.DeviceRules, error) {
+func (fd *FlowDecomposer) updateOutputPortForControllerBoundFlowForParentDevide(ctx context.Context, dr *fu.DeviceRules) (*fu.DeviceRules, error) {
EAPOL := fu.EthType(0x888e)
IGMP := fu.IpProto(2)
UDP := fu.IpProto(17)
@@ -72,7 +78,7 @@
newDeviceRules := dr.Copy()
// Check whether we are dealing with a parent device
for deviceID, fg := range dr.GetRules() {
- if root, _ := fd.deviceMgr.IsRootDevice(deviceID); root {
+ if device, err := fd.getDevice(ctx, deviceID); err == nil && device.Root {
newDeviceRules.ClearFlows(deviceID)
for i := 0; i < fg.Flows.Len(); i++ {
f := fg.GetFlow(i)
@@ -103,7 +109,7 @@
}
//processControllerBoundFlow decomposes trap flows
-func (fd *FlowDecomposer) processControllerBoundFlow(ctx context.Context, agent coreif.LogicalDeviceAgent, path []route.Hop,
+func (fd *FlowDecomposer) processControllerBoundFlow(ctx context.Context, agent LogicalDeviceAgent, path []route.Hop,
inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats) (*fu.DeviceRules, error) {
logger.Debugw(ctx, "trap-flow", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo, "flow": flow})
@@ -285,7 +291,7 @@
}
// processDownstreamFlowWithNextTable decomposes downstream flows containing next table ID instructions
-func (fd *FlowDecomposer) processDownstreamFlowWithNextTable(ctx context.Context, agent coreif.LogicalDeviceAgent, path []route.Hop,
+func (fd *FlowDecomposer) processDownstreamFlowWithNextTable(ctx context.Context, agent LogicalDeviceAgent, path []route.Hop,
inPortNo uint32, outPortNo uint32, flow *ofp.OfpFlowStats) (*fu.DeviceRules, error) {
logger.Debugw(ctx, "decomposing-olt-flow-in-downstream-flow-with-next-table", log.Fields{"inPortNo": inPortNo, "outPortNo": outPortNo})
deviceRules := fu.NewDeviceRules()
@@ -446,7 +452,7 @@
}
// 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,
+func (fd *FlowDecomposer) decomposeFlow(ctx context.Context, agent LogicalDeviceAgent, flow *ofp.OfpFlowStats,
groupMap map[uint32]*ofp.OfpGroupEntry) (*fu.DeviceRules, error) {
inPortNo := fu.GetInPort(flow)
@@ -487,7 +493,7 @@
} else {
var ingressDevice *voltha.Device
var err error
- if ingressDevice, err = fd.deviceMgr.GetDevice(ctx, &voltha.ID{Id: path[0].DeviceID}); err != nil {
+ if ingressDevice, err = fd.getDevice(ctx, path[0].DeviceID); err != nil {
// This can happen in a race condition where a device is deleted right after we obtain a
// route involving the device (GetRoute() above). Handle it as a no route event as well.
return deviceRules, fmt.Errorf("get-device-error :%v :%w", err, route.ErrNoRoute)
@@ -518,6 +524,6 @@
return deviceRules, status.Errorf(codes.Aborted, "unknown downstream flow %v", *flow)
}
}
- deviceRules, err = fd.updateOutputPortForControllerBoundFlowForParentDevide(flow, deviceRules)
+ deviceRules, err = fd.updateOutputPortForControllerBoundFlowForParentDevide(ctx, deviceRules)
return deviceRules, err
}
diff --git a/rw_core/flowdecomposition/flow_decomposer_test.go b/rw_core/flowdecomposition/flow_decomposer_test.go
index ea560e2..6f8c80c 100644
--- a/rw_core/flowdecomposition/flow_decomposer_test.go
+++ b/rw_core/flowdecomposition/flow_decomposer_test.go
@@ -20,7 +20,7 @@
"errors"
"testing"
- "github.com/opencord/voltha-go/rw_core/mocks"
+ "github.com/opencord/voltha-go/rw_core/core/device/state"
"github.com/opencord/voltha-go/rw_core/route"
fu "github.com/opencord/voltha-lib-go/v3/pkg/flows"
ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
@@ -31,7 +31,7 @@
)
type testDeviceManager struct {
- mocks.DeviceManager
+ state.DeviceManager
devices map[string]*voltha.Device
devicePorts map[string]map[uint32]*voltha.Port
}
@@ -397,7 +397,9 @@
tfd.deviceRoutes.RootPorts = make(map[uint32]uint32)
tfd.deviceRoutes.RootPorts[10] = 10
- tfd.fd = NewFlowDecomposer(tfd.dMgr)
+ tfd.fd = NewFlowDecomposer(func(ctx context.Context, deviceID string) (*voltha.Device, error) {
+ return tfd.dMgr.GetDevice(ctx, &voltha.ID{Id: deviceID})
+ })
return &tfd
}
diff --git a/rw_core/mocks/device_manager.go b/rw_core/mocks/device_manager.go
deleted file mode 100644
index e83f935..0000000
--- a/rw_core/mocks/device_manager.go
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Copyright 2019-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 mocks
-
-import (
- "context"
-
- "github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-// DeviceManager represents a mock of a device manager that implements the coreif/DeviceManager interface. It provides
-// default behaviors. For non-default behavior, another implementation of the coreif/DeviceManager interface must be
-// used.
-type DeviceManager struct {
-}
-
-// getDevice -
-func (dm *DeviceManager) GetDevice(ctx context.Context, deviceID *voltha.ID) (*voltha.Device, error) {
- return nil, nil
-}
-
-// IsRootDevice -
-func (dm *DeviceManager) IsRootDevice(deviceID string) (bool, error) {
- return false, nil
-}
-
-// NotifyInvalidTransition -
-func (dm *DeviceManager) NotifyInvalidTransition(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// CreateLogicalDevice -
-func (dm *DeviceManager) CreateLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// SetupUNILogicalPorts -
-func (dm *DeviceManager) SetupUNILogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// DisableAllChildDevices -
-func (dm *DeviceManager) DisableAllChildDevices(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// DeleteLogicalDevice -
-func (dm *DeviceManager) DeleteLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// DeleteLogicalPorts -
-func (dm *DeviceManager) DeleteLogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// DeleteAllChildDevices -
-func (dm *DeviceManager) DeleteAllChildDevices(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// DeleteAllLogicalPorts -
-func (dm *DeviceManager) DeleteAllLogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// DeleteAllDeviceFlows -
-func (dm *DeviceManager) DeleteAllDeviceFlows(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// RunPostDeviceDelete -
-func (dm *DeviceManager) RunPostDeviceDelete(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
-
-// childDeviceLost -
-func (dm *DeviceManager) ChildDeviceLost(ctx context.Context, cDevice *voltha.Device) error {
- return nil
-}
diff --git a/rw_core/mocks/device_manager_test.go b/rw_core/mocks/device_manager_test.go
deleted file mode 100644
index e92d62f..0000000
--- a/rw_core/mocks/device_manager_test.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2019-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 mocks
-
-import (
- "testing"
-
- "github.com/opencord/voltha-go/rw_core/coreif"
-)
-
-func TestDeviceManagerImplementsDeviceManagerIf(t *testing.T) {
- deviceMgr := &DeviceManager{}
-
- if _, ok := interface{}(deviceMgr).(coreif.DeviceManager); !ok {
- t.Error("Device manager does not implement the coreif.DeviceManager interface")
- }
-}