VOL-2909 - Disaggregating rw_core/core/.
This breaks the core package into logical components. (adapter manager, adapter proxy, devices, nbi/api), as well as the "core" which aggregates all these.
Change-Id: I257ac64024a1cf3efe3f5d89d508e60e6e681fb1
diff --git a/rw_core/core/device/agent.go b/rw_core/core/device/agent.go
new file mode 100755
index 0000000..8d18e10
--- /dev/null
+++ b/rw_core/core/device/agent.go
@@ -0,0 +1,1704 @@
+/*
+ * 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 device
+
+import (
+ "context"
+ "encoding/hex"
+ "errors"
+ "fmt"
+ "github.com/golang/protobuf/ptypes"
+ "github.com/opencord/voltha-go/rw_core/core/adapter"
+ "github.com/opencord/voltha-go/rw_core/core/device/remote"
+ "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+ "reflect"
+ "sync"
+ "time"
+
+ "github.com/gogo/protobuf/proto"
+ "github.com/opencord/voltha-go/db/model"
+ coreutils "github.com/opencord/voltha-go/rw_core/utils"
+ fu "github.com/opencord/voltha-lib-go/v3/pkg/flows"
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+ ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+ ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+)
+
+// Agent represents device agent attributes
+type Agent struct {
+ deviceID string
+ parentID string
+ deviceType string
+ isRootdevice bool
+ adapterProxy *remote.AdapterProxy
+ adapterMgr *adapter.Manager
+ deviceMgr *Manager
+ clusterDataProxy *model.Proxy
+ exitChannel chan int
+ device *voltha.Device
+ requestQueue *coreutils.RequestQueue
+ defaultTimeout time.Duration
+ startOnce sync.Once
+ stopOnce sync.Once
+ stopped bool
+}
+
+//newAgent creates a new device agent. The device will be initialized when start() is called.
+func newAgent(ap *remote.AdapterProxy, device *voltha.Device, deviceMgr *Manager, cdProxy *model.Proxy, timeout time.Duration) *Agent {
+ var agent Agent
+ agent.adapterProxy = ap
+ if device.Id == "" {
+ agent.deviceID = coreutils.CreateDeviceID()
+ } else {
+ agent.deviceID = device.Id
+ }
+
+ agent.isRootdevice = device.Root
+ agent.parentID = device.ParentId
+ agent.deviceType = device.Type
+ agent.deviceMgr = deviceMgr
+ agent.adapterMgr = deviceMgr.adapterMgr
+ agent.exitChannel = make(chan int, 1)
+ agent.clusterDataProxy = cdProxy
+ agent.defaultTimeout = timeout
+ agent.device = proto.Clone(device).(*voltha.Device)
+ agent.requestQueue = coreutils.NewRequestQueue()
+ return &agent
+}
+
+// start() saves the device to the data model and registers for callbacks on that device if deviceToCreate!=nil.
+// Otherwise, it will load the data from the dB and setup the necessary callbacks and proxies. Returns the device that
+// was started.
+func (agent *Agent) start(ctx context.Context, deviceToCreate *voltha.Device) (*voltha.Device, error) {
+ needToStart := false
+ if agent.startOnce.Do(func() { needToStart = true }); !needToStart {
+ return agent.getDevice(ctx)
+ }
+ var startSucceeded bool
+ defer func() {
+ if !startSucceeded {
+ if err := agent.stop(ctx); err != nil {
+ logger.Errorw("failed-to-cleanup-after-unsuccessful-start", log.Fields{"device-id": agent.deviceID, "error": err})
+ }
+ }
+ }()
+
+ var device *voltha.Device
+ if deviceToCreate == nil {
+ // Load the existing device
+ device := &voltha.Device{}
+ have, err := agent.clusterDataProxy.Get(ctx, "devices/"+agent.deviceID, device)
+ if err != nil {
+ return nil, err
+ } else if !have {
+ return nil, status.Errorf(codes.NotFound, "device-%s", agent.deviceID)
+ }
+
+ agent.deviceType = device.Adapter
+ agent.device = proto.Clone(device).(*voltha.Device)
+
+ logger.Infow("device-loaded-from-dB", log.Fields{"device-id": agent.deviceID})
+ } else {
+ // Create a new device
+ // Assumption is that AdminState, FlowGroups, and Flows are unitialized since this
+ // is a new device, so populate them here before passing the device to clusterDataProxy.AddWithId.
+ // agent.deviceId will also have been set during newAgent().
+ device = (proto.Clone(deviceToCreate)).(*voltha.Device)
+ device.Id = agent.deviceID
+ device.AdminState = voltha.AdminState_PREPROVISIONED
+ device.FlowGroups = &ofp.FlowGroups{Items: nil}
+ device.Flows = &ofp.Flows{Items: nil}
+ if !deviceToCreate.GetRoot() && deviceToCreate.ProxyAddress != nil {
+ // Set the default vlan ID to the one specified by the parent adapter. It can be
+ // overwritten by the child adapter during a device update request
+ device.Vlan = deviceToCreate.ProxyAddress.ChannelId
+ }
+
+ // Add the initial device to the local model
+ if err := agent.clusterDataProxy.AddWithID(ctx, "devices", agent.deviceID, device); err != nil {
+ return nil, status.Errorf(codes.Aborted, "failed-adding-device-%s: %s", agent.deviceID, err)
+ }
+ agent.device = device
+ }
+
+ startSucceeded = true
+ logger.Debugw("device-agent-started", log.Fields{"device-id": agent.deviceID})
+
+ return agent.getDevice(ctx)
+}
+
+// stop stops the device agent. Not much to do for now
+func (agent *Agent) stop(ctx context.Context) error {
+ needToStop := false
+ if agent.stopOnce.Do(func() { needToStop = true }); !needToStop {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logger.Infow("stopping-device-agent", log.Fields{"deviceId": agent.deviceID, "parentId": agent.parentID})
+
+ // Remove the device from the KV store
+ if err := agent.clusterDataProxy.Remove(ctx, "devices/"+agent.deviceID); err != nil {
+ return err
+ }
+
+ close(agent.exitChannel)
+
+ agent.stopped = true
+
+ logger.Infow("device-agent-stopped", log.Fields{"device-id": agent.deviceID, "parent-id": agent.parentID})
+
+ return nil
+}
+
+// Load the most recent state from the KVStore for the device.
+func (agent *Agent) reconcileWithKVStore(ctx context.Context) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ logger.Warnw("request-aborted", log.Fields{"device-id": agent.deviceID, "error": err})
+ return
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debug("reconciling-device-agent-devicetype")
+ // TODO: context timeout
+ device := &voltha.Device{}
+ if have, err := agent.clusterDataProxy.Get(ctx, "devices/"+agent.deviceID, device); err != nil {
+ logger.Errorw("kv-get-failed", log.Fields{"device-id": agent.deviceID, "error": err})
+ return
+ } else if !have {
+ return // not found in kv
+ }
+
+ agent.deviceType = device.Adapter
+ agent.device = device
+ logger.Debugw("reconciled-device-agent-devicetype", log.Fields{"device-id": agent.deviceID, "type": agent.deviceType})
+}
+
+// onSuccess is a common callback for scenarios where we receive a nil response following a request to an adapter
+// and the only action required is to publish a successful result on kafka
+func (agent *Agent) onSuccess(rpc string, response interface{}, reqArgs ...interface{}) {
+ logger.Debugw("response successful", log.Fields{"rpc": rpc, "device-id": agent.deviceID})
+ // TODO: Post success message onto kafka
+}
+
+// onFailure is a common callback for scenarios where we receive an error response following a request to an adapter
+// and the only action required is to publish the failed result on kafka
+func (agent *Agent) onFailure(rpc string, response interface{}, reqArgs ...interface{}) {
+ if res, ok := response.(error); ok {
+ logger.Errorw("rpc-failed", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "error": res, "args": reqArgs})
+ } else {
+ logger.Errorw("rpc-failed-invalid-error", log.Fields{"rpc": rpc, "device-id": agent.deviceID, "args": reqArgs})
+ }
+ // TODO: Post failure message onto kafka
+}
+
+func (agent *Agent) waitForAdapterResponse(ctx context.Context, cancel context.CancelFunc, rpc string, ch chan *kafka.RpcResponse,
+ onSuccess coreutils.ResponseCallback, onFailure coreutils.ResponseCallback, reqArgs ...interface{}) {
+ defer cancel()
+ select {
+ case rpcResponse, ok := <-ch:
+ if !ok {
+ onFailure(rpc, status.Errorf(codes.Aborted, "channel-closed"), reqArgs)
+ } else if rpcResponse.Err != nil {
+ onFailure(rpc, rpcResponse.Err, reqArgs)
+ } else {
+ onSuccess(rpc, rpcResponse.Reply, reqArgs)
+ }
+ case <-ctx.Done():
+ onFailure(rpc, ctx.Err(), reqArgs)
+ }
+}
+
+// getDevice returns the device data from cache
+func (agent *Agent) getDevice(ctx context.Context) (*voltha.Device, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ return proto.Clone(agent.device).(*voltha.Device), nil
+}
+
+// getDeviceWithoutLock is a helper function to be used ONLY by any device agent function AFTER it has acquired the device lock.
+func (agent *Agent) getDeviceWithoutLock() *voltha.Device {
+ return proto.Clone(agent.device).(*voltha.Device)
+}
+
+// enableDevice activates a preprovisioned or a disable device
+func (agent *Agent) enableDevice(ctx context.Context) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logger.Debugw("enableDevice", log.Fields{"device-id": agent.deviceID})
+
+ cloned := agent.getDeviceWithoutLock()
+
+ // First figure out which adapter will handle this device type. We do it at this stage as allow devices to be
+ // pre-provisioned with the required adapter not registered. At this stage, since we need to communicate
+ // with the adapter then we need to know the adapter that will handle this request
+ adapterName, err := agent.adapterMgr.GetAdapterType(cloned.Type)
+ if err != nil {
+ return err
+ }
+ cloned.Adapter = adapterName
+
+ if cloned.AdminState == voltha.AdminState_ENABLED {
+ logger.Warnw("device-already-enabled", log.Fields{"device-id": agent.deviceID})
+ err = status.Error(codes.FailedPrecondition, fmt.Sprintf("cannot-enable-an-already-enabled-device: %s ", cloned.Id))
+ return err
+ }
+
+ if cloned.AdminState == voltha.AdminState_DELETED {
+ // This is a temporary state when a device is deleted before it gets removed from the model.
+ err = status.Error(codes.FailedPrecondition, fmt.Sprintf("cannot-enable-a-deleted-device: %s ", cloned.Id))
+ return err
+ }
+
+ previousAdminState := cloned.AdminState
+
+ // Update the Admin State and set the operational state to activating before sending the request to the
+ // Adapters
+ if err := agent.updateDeviceStateInStoreWithoutLock(ctx, cloned, voltha.AdminState_ENABLED, cloned.ConnectStatus, voltha.OperStatus_ACTIVATING); err != nil {
+ return err
+ }
+
+ // Adopt the device if it was in pre-provision state. In all other cases, try to re-enable it.
+ device := proto.Clone(cloned).(*voltha.Device)
+ var ch chan *kafka.RpcResponse
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ if previousAdminState == voltha.AdminState_PREPROVISIONED {
+ ch, err = agent.adapterProxy.AdoptDevice(subCtx, device)
+ } else {
+ ch, err = agent.adapterProxy.ReEnableDevice(subCtx, device)
+ }
+ if err != nil {
+ cancel()
+ return err
+ }
+ // Wait for response
+ go agent.waitForAdapterResponse(subCtx, cancel, "enableDevice", ch, agent.onSuccess, agent.onFailure)
+ return nil
+}
+
+func (agent *Agent) waitForAdapterFlowResponse(ctx context.Context, cancel context.CancelFunc, ch chan *kafka.RpcResponse, response coreutils.Response) {
+ defer cancel()
+ select {
+ case rpcResponse, ok := <-ch:
+ if !ok {
+ response.Error(status.Errorf(codes.Aborted, "channel-closed"))
+ } else if rpcResponse.Err != nil {
+ response.Error(rpcResponse.Err)
+ } else {
+ response.Done()
+ }
+ case <-ctx.Done():
+ response.Error(ctx.Err())
+ }
+}
+
+//deleteFlowWithoutPreservingOrder removes a flow specified by index from the flows slice. This function will
+//panic if the index is out of range.
+func deleteFlowWithoutPreservingOrder(flows []*ofp.OfpFlowStats, index int) []*ofp.OfpFlowStats {
+ flows[index] = flows[len(flows)-1]
+ flows[len(flows)-1] = nil
+ return flows[:len(flows)-1]
+}
+
+//deleteGroupWithoutPreservingOrder removes a group specified by index from the groups slice. This function will
+//panic if the index is out of range.
+func deleteGroupWithoutPreservingOrder(groups []*ofp.OfpGroupEntry, index int) []*ofp.OfpGroupEntry {
+ groups[index] = groups[len(groups)-1]
+ groups[len(groups)-1] = nil
+ return groups[:len(groups)-1]
+}
+
+func flowsToUpdateToDelete(newFlows, existingFlows []*ofp.OfpFlowStats) (updatedNewFlows, flowsToDelete, updatedAllFlows []*ofp.OfpFlowStats) {
+ // Process flows
+ for _, flow := range existingFlows {
+ if idx := fu.FindFlows(newFlows, flow); idx == -1 {
+ updatedAllFlows = append(updatedAllFlows, flow)
+ } else {
+ // We have a matching flow (i.e. the following field matches: "TableId", "Priority", "Flags", "Cookie",
+ // "Match". If this is an exact match (i.e. all other fields matches as well) then this flow will be
+ // ignored. Otherwise, the previous flow will be deleted and the new one added
+ if proto.Equal(newFlows[idx], flow) {
+ // Flow already exist, remove it from the new flows but keep it in the updated flows slice
+ newFlows = deleteFlowWithoutPreservingOrder(newFlows, idx)
+ updatedAllFlows = append(updatedAllFlows, flow)
+ } else {
+ // Minor change to flow, delete old and add new one
+ flowsToDelete = append(flowsToDelete, flow)
+ }
+ }
+ }
+ updatedAllFlows = append(updatedAllFlows, newFlows...)
+ return newFlows, flowsToDelete, updatedAllFlows
+}
+
+func groupsToUpdateToDelete(newGroups, existingGroups []*ofp.OfpGroupEntry) (updatedNewGroups, groupsToDelete, updatedAllGroups []*ofp.OfpGroupEntry) {
+ for _, group := range existingGroups {
+ if idx := fu.FindGroup(newGroups, group.Desc.GroupId); idx == -1 { // does not exist now
+ updatedAllGroups = append(updatedAllGroups, group)
+ } else {
+ // Follow same logic as flows
+ if proto.Equal(newGroups[idx], group) {
+ // Group already exist, remove it from the new groups
+ newGroups = deleteGroupWithoutPreservingOrder(newGroups, idx)
+ updatedAllGroups = append(updatedAllGroups, group)
+ } else {
+ // Minor change to group, delete old and add new one
+ groupsToDelete = append(groupsToDelete, group)
+ }
+ }
+ }
+ updatedAllGroups = append(updatedAllGroups, newGroups...)
+ return newGroups, groupsToDelete, updatedAllGroups
+}
+
+func (agent *Agent) addFlowsAndGroupsToAdapter(ctx context.Context, newFlows []*ofp.OfpFlowStats, newGroups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
+ logger.Debugw("add-flows-groups-to-adapters", log.Fields{"device-id": agent.deviceID, "flows": newFlows, "groups": newGroups, "flow-metadata": flowMetadata})
+
+ if (len(newFlows) | len(newGroups)) == 0 {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": newFlows, "groups": newGroups})
+ return coreutils.DoneResponse(), nil
+ }
+
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return coreutils.DoneResponse(), err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ device := agent.getDeviceWithoutLock()
+ dType := agent.adapterMgr.GetDeviceType(device.Type)
+ if dType == nil {
+ return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+ }
+
+ existingFlows := proto.Clone(device.Flows).(*voltha.Flows)
+ existingGroups := proto.Clone(device.FlowGroups).(*ofp.FlowGroups)
+
+ // Process flows
+ newFlows, flowsToDelete, updatedAllFlows := flowsToUpdateToDelete(newFlows, existingFlows.Items)
+
+ // Process groups
+ newGroups, groupsToDelete, updatedAllGroups := groupsToUpdateToDelete(newGroups, existingGroups.Items)
+
+ // Sanity check
+ if (len(updatedAllFlows) | len(flowsToDelete) | len(updatedAllGroups) | len(groupsToDelete)) == 0 {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": newFlows, "groups": newGroups})
+ return coreutils.DoneResponse(), nil
+ }
+
+ // store the changed data
+ device.Flows = &voltha.Flows{Items: updatedAllFlows}
+ device.FlowGroups = &voltha.FlowGroups{Items: updatedAllGroups}
+ if err := agent.updateDeviceWithoutLock(ctx, device); err != nil {
+ return coreutils.DoneResponse(), status.Errorf(codes.Internal, "failure-updating-device-%s", agent.deviceID)
+ }
+
+ // Send update to adapters
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ response := coreutils.NewResponse()
+ if !dType.AcceptsAddRemoveFlowUpdates {
+ if len(updatedAllGroups) != 0 && reflect.DeepEqual(existingGroups.Items, updatedAllGroups) && len(updatedAllFlows) != 0 && reflect.DeepEqual(existingFlows.Items, updatedAllFlows) {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": newFlows, "groups": newGroups})
+ cancel()
+ return coreutils.DoneResponse(), nil
+ }
+ rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, &voltha.Flows{Items: updatedAllFlows}, &voltha.FlowGroups{Items: updatedAllGroups}, flowMetadata)
+ if err != nil {
+ cancel()
+ return coreutils.DoneResponse(), err
+ }
+ go agent.waitForAdapterFlowResponse(subCtx, cancel, rpcResponse, response)
+ } else {
+ flowChanges := &ofp.FlowChanges{
+ ToAdd: &voltha.Flows{Items: newFlows},
+ ToRemove: &voltha.Flows{Items: flowsToDelete},
+ }
+ groupChanges := &ofp.FlowGroupChanges{
+ ToAdd: &voltha.FlowGroups{Items: newGroups},
+ ToRemove: &voltha.FlowGroups{Items: groupsToDelete},
+ ToUpdate: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
+ }
+ rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
+ if err != nil {
+ cancel()
+ return coreutils.DoneResponse(), err
+ }
+ go agent.waitForAdapterFlowResponse(subCtx, cancel, rpcResponse, response)
+ }
+ return response, nil
+}
+
+//addFlowsAndGroups adds the "newFlows" and "newGroups" from the existing flows/groups and sends the update to the
+//adapters
+func (agent *Agent) addFlowsAndGroups(ctx context.Context, newFlows []*ofp.OfpFlowStats, newGroups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) error {
+ response, err := agent.addFlowsAndGroupsToAdapter(ctx, newFlows, newGroups, flowMetadata)
+ if err != nil {
+ return err
+ }
+ if errs := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, response); errs != nil {
+ logger.Warnw("no-adapter-response", log.Fields{"device-id": agent.deviceID, "result": errs})
+ return status.Errorf(codes.Aborted, "flow-failure-device-%s", agent.deviceID)
+ }
+ return nil
+}
+
+func (agent *Agent) deleteFlowsAndGroupsFromAdapter(ctx context.Context, flowsToDel []*ofp.OfpFlowStats, groupsToDel []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
+ logger.Debugw("delete-flows-groups-from-adapter", log.Fields{"device-id": agent.deviceID, "flows": flowsToDel, "groups": groupsToDel})
+
+ if (len(flowsToDel) | len(groupsToDel)) == 0 {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": flowsToDel, "groups": groupsToDel})
+ return coreutils.DoneResponse(), nil
+ }
+
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return coreutils.DoneResponse(), err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ device := agent.getDeviceWithoutLock()
+ dType := agent.adapterMgr.GetDeviceType(device.Type)
+ if dType == nil {
+ return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+ }
+
+ existingFlows := proto.Clone(device.Flows).(*voltha.Flows)
+ existingGroups := proto.Clone(device.FlowGroups).(*ofp.FlowGroups)
+
+ var flowsToKeep []*ofp.OfpFlowStats
+ var groupsToKeep []*ofp.OfpGroupEntry
+
+ // Process flows
+ for _, flow := range existingFlows.Items {
+ if idx := fu.FindFlows(flowsToDel, flow); idx == -1 {
+ flowsToKeep = append(flowsToKeep, flow)
+ }
+ }
+
+ // Process groups
+ for _, group := range existingGroups.Items {
+ if fu.FindGroup(groupsToDel, group.Desc.GroupId) == -1 { // does not exist now
+ groupsToKeep = append(groupsToKeep, group)
+ }
+ }
+
+ logger.Debugw("deleteFlowsAndGroups",
+ log.Fields{
+ "device-id": agent.deviceID,
+ "flows-to-del": len(flowsToDel),
+ "flows-to-keep": len(flowsToKeep),
+ "groups-to-del": len(groupsToDel),
+ "groups-to-keep": len(groupsToKeep),
+ })
+
+ // Sanity check
+ if (len(flowsToKeep) | len(flowsToDel) | len(groupsToKeep) | len(groupsToDel)) == 0 {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows-to-del": flowsToDel, "groups-to-del": groupsToDel})
+ return coreutils.DoneResponse(), nil
+ }
+
+ // store the changed data
+ device.Flows = &voltha.Flows{Items: flowsToKeep}
+ device.FlowGroups = &voltha.FlowGroups{Items: groupsToKeep}
+ if err := agent.updateDeviceWithoutLock(ctx, device); err != nil {
+ return coreutils.DoneResponse(), status.Errorf(codes.Internal, "failure-updating-%s", agent.deviceID)
+ }
+
+ // Send update to adapters
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ response := coreutils.NewResponse()
+ if !dType.AcceptsAddRemoveFlowUpdates {
+ if len(groupsToKeep) != 0 && reflect.DeepEqual(existingGroups.Items, groupsToKeep) && len(flowsToKeep) != 0 && reflect.DeepEqual(existingFlows.Items, flowsToKeep) {
+ logger.Debugw("nothing-to-update", log.Fields{"deviceId": agent.deviceID, "flowsToDel": flowsToDel, "groupsToDel": groupsToDel})
+ cancel()
+ return coreutils.DoneResponse(), nil
+ }
+ rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, &voltha.Flows{Items: flowsToKeep}, &voltha.FlowGroups{Items: groupsToKeep}, flowMetadata)
+ if err != nil {
+ cancel()
+ return coreutils.DoneResponse(), err
+ }
+ go agent.waitForAdapterFlowResponse(subCtx, cancel, rpcResponse, response)
+ } else {
+ flowChanges := &ofp.FlowChanges{
+ ToAdd: &voltha.Flows{Items: []*ofp.OfpFlowStats{}},
+ ToRemove: &voltha.Flows{Items: flowsToDel},
+ }
+ groupChanges := &ofp.FlowGroupChanges{
+ ToAdd: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
+ ToRemove: &voltha.FlowGroups{Items: groupsToDel},
+ ToUpdate: &voltha.FlowGroups{Items: []*ofp.OfpGroupEntry{}},
+ }
+ rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
+ if err != nil {
+ cancel()
+ return coreutils.DoneResponse(), err
+ }
+ go agent.waitForAdapterFlowResponse(subCtx, cancel, rpcResponse, response)
+ }
+ return response, nil
+}
+
+//deleteFlowsAndGroups removes the "flowsToDel" and "groupsToDel" from the existing flows/groups and sends the update to the
+//adapters
+func (agent *Agent) deleteFlowsAndGroups(ctx context.Context, flowsToDel []*ofp.OfpFlowStats, groupsToDel []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) error {
+ response, err := agent.deleteFlowsAndGroupsFromAdapter(ctx, flowsToDel, groupsToDel, flowMetadata)
+ if err != nil {
+ return err
+ }
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, response); res != nil {
+ return status.Errorf(codes.Aborted, "errors-%s", res)
+ }
+ return nil
+}
+
+func (agent *Agent) updateFlowsAndGroupsToAdapter(ctx context.Context, updatedFlows []*ofp.OfpFlowStats, updatedGroups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) (coreutils.Response, error) {
+ logger.Debugw("updateFlowsAndGroups", log.Fields{"device-id": agent.deviceID, "flows": updatedFlows, "groups": updatedGroups})
+
+ if (len(updatedFlows) | len(updatedGroups)) == 0 {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": updatedFlows, "groups": updatedGroups})
+ return coreutils.DoneResponse(), nil
+ }
+
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return coreutils.DoneResponse(), err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ device := agent.getDeviceWithoutLock()
+ if device.OperStatus != voltha.OperStatus_ACTIVE || device.ConnectStatus != voltha.ConnectStatus_REACHABLE || device.AdminState != voltha.AdminState_ENABLED {
+ return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "invalid device states")
+ }
+ dType := agent.adapterMgr.GetDeviceType(device.Type)
+ if dType == nil {
+ return coreutils.DoneResponse(), status.Errorf(codes.FailedPrecondition, "non-existent-device-type-%s", device.Type)
+ }
+
+ existingFlows := proto.Clone(device.Flows).(*voltha.Flows)
+ existingGroups := proto.Clone(device.FlowGroups).(*ofp.FlowGroups)
+
+ if len(updatedGroups) != 0 && reflect.DeepEqual(existingGroups.Items, updatedGroups) && len(updatedFlows) != 0 && reflect.DeepEqual(existingFlows.Items, updatedFlows) {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": updatedFlows, "groups": updatedGroups})
+ return coreutils.DoneResponse(), nil
+ }
+
+ logger.Debugw("updating-flows-and-groups",
+ log.Fields{
+ "device-id": agent.deviceID,
+ "updated-flows": updatedFlows,
+ "updated-groups": updatedGroups,
+ })
+
+ // store the updated data
+ device.Flows = &voltha.Flows{Items: updatedFlows}
+ device.FlowGroups = &voltha.FlowGroups{Items: updatedGroups}
+ if err := agent.updateDeviceWithoutLock(ctx, device); err != nil {
+ return coreutils.DoneResponse(), status.Errorf(codes.Internal, "failure-updating-%s", agent.deviceID)
+ }
+
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ response := coreutils.NewResponse()
+ // Process bulk flow update differently than incremental update
+ if !dType.AcceptsAddRemoveFlowUpdates {
+ rpcResponse, err := agent.adapterProxy.UpdateFlowsBulk(subCtx, device, &voltha.Flows{Items: updatedFlows}, &voltha.FlowGroups{Items: updatedGroups}, nil)
+ if err != nil {
+ cancel()
+ return coreutils.DoneResponse(), err
+ }
+ go agent.waitForAdapterFlowResponse(subCtx, cancel, rpcResponse, response)
+ } else {
+ var flowsToAdd []*ofp.OfpFlowStats
+ var flowsToDelete []*ofp.OfpFlowStats
+ var groupsToAdd []*ofp.OfpGroupEntry
+ var groupsToDelete []*ofp.OfpGroupEntry
+
+ // Process flows
+ for _, flow := range updatedFlows {
+ if idx := fu.FindFlows(existingFlows.Items, flow); idx == -1 {
+ flowsToAdd = append(flowsToAdd, flow)
+ }
+ }
+ for _, flow := range existingFlows.Items {
+ if idx := fu.FindFlows(updatedFlows, flow); idx != -1 {
+ flowsToDelete = append(flowsToDelete, flow)
+ }
+ }
+
+ // Process groups
+ for _, g := range updatedGroups {
+ if fu.FindGroup(existingGroups.Items, g.Desc.GroupId) == -1 { // does not exist now
+ groupsToAdd = append(groupsToAdd, g)
+ }
+ }
+ for _, group := range existingGroups.Items {
+ if fu.FindGroup(updatedGroups, group.Desc.GroupId) != -1 { // does not exist now
+ groupsToDelete = append(groupsToDelete, group)
+ }
+ }
+
+ logger.Debugw("updating-flows-and-groups",
+ log.Fields{
+ "device-id": agent.deviceID,
+ "flows-to-add": flowsToAdd,
+ "flows-to-delete": flowsToDelete,
+ "groups-to-add": groupsToAdd,
+ "groups-to-delete": groupsToDelete,
+ })
+
+ // Sanity check
+ if (len(flowsToAdd) | len(flowsToDelete) | len(groupsToAdd) | len(groupsToDelete) | len(updatedGroups)) == 0 {
+ logger.Debugw("nothing-to-update", log.Fields{"device-id": agent.deviceID, "flows": updatedFlows, "groups": updatedGroups})
+ cancel()
+ return coreutils.DoneResponse(), nil
+ }
+
+ flowChanges := &ofp.FlowChanges{
+ ToAdd: &voltha.Flows{Items: flowsToAdd},
+ ToRemove: &voltha.Flows{Items: flowsToDelete},
+ }
+ groupChanges := &ofp.FlowGroupChanges{
+ ToAdd: &voltha.FlowGroups{Items: groupsToAdd},
+ ToRemove: &voltha.FlowGroups{Items: groupsToDelete},
+ ToUpdate: &voltha.FlowGroups{Items: updatedGroups},
+ }
+ rpcResponse, err := agent.adapterProxy.UpdateFlowsIncremental(subCtx, device, flowChanges, groupChanges, flowMetadata)
+ if err != nil {
+ cancel()
+ return coreutils.DoneResponse(), err
+ }
+ go agent.waitForAdapterFlowResponse(subCtx, cancel, rpcResponse, response)
+ }
+
+ return response, nil
+}
+
+//updateFlowsAndGroups replaces the existing flows and groups with "updatedFlows" and "updatedGroups" respectively. It
+//also sends the updates to the adapters
+func (agent *Agent) updateFlowsAndGroups(ctx context.Context, updatedFlows []*ofp.OfpFlowStats, updatedGroups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) error {
+ response, err := agent.updateFlowsAndGroupsToAdapter(ctx, updatedFlows, updatedGroups, flowMetadata)
+ if err != nil {
+ return err
+ }
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, response); res != nil {
+ return status.Errorf(codes.Aborted, "errors-%s", res)
+ }
+ return nil
+}
+
+//deleteAllFlows deletes all flows in the device table
+func (agent *Agent) deleteAllFlows(ctx context.Context) error {
+ logger.Debugw("deleteAllFlows", log.Fields{"deviceId": agent.deviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ device := agent.getDeviceWithoutLock()
+ // purge all flows on the device by setting it to nil
+ device.Flows = &ofp.Flows{Items: nil}
+ if err := agent.updateDeviceWithoutLock(ctx, device); err != nil {
+ // The caller logs the error
+ return err
+ }
+ return nil
+}
+
+//disableDevice disable a device
+func (agent *Agent) disableDevice(ctx context.Context) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("disableDevice", log.Fields{"device-id": agent.deviceID})
+
+ cloned := agent.getDeviceWithoutLock()
+
+ if cloned.AdminState == voltha.AdminState_DISABLED {
+ logger.Debugw("device-already-disabled", log.Fields{"id": agent.deviceID})
+ return nil
+ }
+ if cloned.AdminState == voltha.AdminState_PREPROVISIONED ||
+ cloned.AdminState == voltha.AdminState_DELETED {
+ return status.Errorf(codes.FailedPrecondition, "deviceId:%s, invalid-admin-state:%s", agent.deviceID, cloned.AdminState)
+ }
+
+ // Update the Admin State and operational state before sending the request out
+ if err := agent.updateDeviceStateInStoreWithoutLock(ctx, cloned, voltha.AdminState_DISABLED, cloned.ConnectStatus, voltha.OperStatus_UNKNOWN); err != nil {
+ return err
+ }
+
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.DisableDevice(subCtx, proto.Clone(cloned).(*voltha.Device))
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "disableDevice", ch, agent.onSuccess, agent.onFailure)
+
+ return nil
+}
+
+func (agent *Agent) rebootDevice(ctx context.Context) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("rebootDevice", log.Fields{"device-id": agent.deviceID})
+
+ device := agent.getDeviceWithoutLock()
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.RebootDevice(subCtx, device)
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "rebootDevice", ch, agent.onSuccess, agent.onFailure)
+ return nil
+}
+
+func (agent *Agent) deleteDevice(ctx context.Context) error {
+ logger.Debugw("deleteDevice", log.Fields{"device-id": agent.deviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ cloned := agent.getDeviceWithoutLock()
+
+ previousState := cloned.AdminState
+
+ // No check is required when deleting a device. Changing the state to DELETE will trigger the removal of this
+ // device by the state machine
+ if err := agent.updateDeviceStateInStoreWithoutLock(ctx, cloned, voltha.AdminState_DELETED, cloned.ConnectStatus, cloned.OperStatus); err != nil {
+ return err
+ }
+
+ // If the device was in pre-prov state (only parent device are in that state) then do not send the request to the
+ // adapter
+ if previousState != ic.AdminState_PREPROVISIONED {
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.DeleteDevice(subCtx, cloned)
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "deleteDevice", ch, agent.onSuccess, agent.onFailure)
+ }
+ return nil
+}
+
+func (agent *Agent) setParentID(ctx context.Context, device *voltha.Device, parentID string) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logger.Debugw("setParentId", log.Fields{"device-id": device.Id, "parent-id": parentID})
+
+ cloned := agent.getDeviceWithoutLock()
+ cloned.ParentId = parentID
+ // Store the device
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, ""); err != nil {
+ return err
+ }
+
+ return nil
+}
+
+func (agent *Agent) updatePmConfigs(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("updatePmConfigs", log.Fields{"device-id": pmConfigs.Id})
+
+ cloned := agent.getDeviceWithoutLock()
+ cloned.PmConfigs = proto.Clone(pmConfigs).(*voltha.PmConfigs)
+ // Store the device
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, ""); err != nil {
+ return err
+ }
+ // Send the request to the adapter
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.UpdatePmConfigs(subCtx, cloned, pmConfigs)
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "updatePmConfigs", ch, agent.onSuccess, agent.onFailure)
+ return nil
+}
+
+func (agent *Agent) initPmConfigs(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("initPmConfigs", log.Fields{"device-id": pmConfigs.Id})
+
+ cloned := agent.getDeviceWithoutLock()
+ cloned.PmConfigs = proto.Clone(pmConfigs).(*voltha.PmConfigs)
+ updateCtx := context.WithValue(ctx, model.RequestTimestamp, time.Now().UnixNano())
+ return agent.updateDeviceInStoreWithoutLock(updateCtx, cloned, false, "")
+}
+
+func (agent *Agent) listPmConfigs(ctx context.Context) (*voltha.PmConfigs, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("listPmConfigs", log.Fields{"device-id": agent.deviceID})
+
+ return agent.getDeviceWithoutLock().PmConfigs, nil
+}
+
+func (agent *Agent) downloadImage(ctx context.Context, img *voltha.ImageDownload) (*voltha.OperationResp, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logger.Debugw("downloadImage", log.Fields{"device-id": agent.deviceID})
+
+ device := agent.getDeviceWithoutLock()
+
+ if device.AdminState != voltha.AdminState_ENABLED {
+ return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, expected-admin-state:%s", agent.deviceID, voltha.AdminState_ENABLED)
+ }
+ // Save the image
+ clonedImg := proto.Clone(img).(*voltha.ImageDownload)
+ clonedImg.DownloadState = voltha.ImageDownload_DOWNLOAD_REQUESTED
+ cloned := proto.Clone(device).(*voltha.Device)
+ if cloned.ImageDownloads == nil {
+ cloned.ImageDownloads = []*voltha.ImageDownload{clonedImg}
+ } else {
+ if device.AdminState != voltha.AdminState_ENABLED {
+ logger.Debugw("device-not-enabled", log.Fields{"id": agent.deviceID})
+ return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, expected-admin-state:%s", agent.deviceID, voltha.AdminState_ENABLED)
+ }
+ // Save the image
+ clonedImg := proto.Clone(img).(*voltha.ImageDownload)
+ clonedImg.DownloadState = voltha.ImageDownload_DOWNLOAD_REQUESTED
+ if device.ImageDownloads == nil {
+ device.ImageDownloads = []*voltha.ImageDownload{clonedImg}
+ } else {
+ device.ImageDownloads = append(device.ImageDownloads, clonedImg)
+ }
+ if err := agent.updateDeviceStateInStoreWithoutLock(ctx, cloned, voltha.AdminState_DOWNLOADING_IMAGE, device.ConnectStatus, device.OperStatus); err != nil {
+ return nil, err
+ }
+
+ // Send the request to the adapter
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.DownloadImage(ctx, cloned, clonedImg)
+ if err != nil {
+ cancel()
+ return nil, err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "downloadImage", ch, agent.onSuccess, agent.onFailure)
+ }
+ return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
+}
+
+// isImageRegistered is a helper method to figure out if an image is already registered
+func isImageRegistered(img *voltha.ImageDownload, device *voltha.Device) bool {
+ for _, image := range device.ImageDownloads {
+ if image.Id == img.Id && image.Name == img.Name {
+ return true
+ }
+ }
+ return false
+}
+
+func (agent *Agent) cancelImageDownload(ctx context.Context, img *voltha.ImageDownload) (*voltha.OperationResp, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logger.Debugw("cancelImageDownload", log.Fields{"device-id": agent.deviceID})
+
+ device := agent.getDeviceWithoutLock()
+
+ // Verify whether the Image is in the list of image being downloaded
+ if !isImageRegistered(img, device) {
+ return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, image-not-registered:%s", agent.deviceID, img.Name)
+ }
+
+ // Update image download state
+ for _, image := range device.ImageDownloads {
+ if image.Id == img.Id && image.Name == img.Name {
+ image.DownloadState = voltha.ImageDownload_DOWNLOAD_CANCELLED
+ }
+ }
+
+ if device.AdminState == voltha.AdminState_DOWNLOADING_IMAGE {
+ // Set the device to Enabled
+ if err := agent.updateDeviceStateInStoreWithoutLock(ctx, device, voltha.AdminState_ENABLED, device.ConnectStatus, device.OperStatus); err != nil {
+ return nil, err
+ }
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.CancelImageDownload(subCtx, device, img)
+ if err != nil {
+ cancel()
+ return nil, err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "cancelImageDownload", ch, agent.onSuccess, agent.onFailure)
+ }
+ return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
+}
+
+func (agent *Agent) activateImage(ctx context.Context, img *voltha.ImageDownload) (*voltha.OperationResp, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("activateImage", log.Fields{"device-id": agent.deviceID})
+ cloned := agent.getDeviceWithoutLock()
+
+ // Verify whether the Image is in the list of image being downloaded
+ if !isImageRegistered(img, cloned) {
+ return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, image-not-registered:%s", agent.deviceID, img.Name)
+ }
+
+ if cloned.AdminState == voltha.AdminState_DOWNLOADING_IMAGE {
+ return nil, status.Errorf(codes.FailedPrecondition, "device-id:%s, device-in-downloading-state:%s", agent.deviceID, img.Name)
+ }
+ // Update image download state
+ for _, image := range cloned.ImageDownloads {
+ if image.Id == img.Id && image.Name == img.Name {
+ image.ImageState = voltha.ImageDownload_IMAGE_ACTIVATING
+ }
+ }
+ // Set the device to downloading_image
+ if err := agent.updateDeviceStateInStoreWithoutLock(ctx, cloned, voltha.AdminState_DOWNLOADING_IMAGE, cloned.ConnectStatus, cloned.OperStatus); err != nil {
+ return nil, err
+ }
+
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.ActivateImageUpdate(subCtx, proto.Clone(cloned).(*voltha.Device), img)
+ if err != nil {
+ cancel()
+ return nil, err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "activateImageUpdate", ch, agent.onSuccess, agent.onFailure)
+
+ // The status of the AdminState will be changed following the update_download_status response from the adapter
+ // The image name will also be removed from the device list
+ return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
+}
+
+func (agent *Agent) revertImage(ctx context.Context, img *voltha.ImageDownload) (*voltha.OperationResp, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("revertImage", log.Fields{"device-id": agent.deviceID})
+
+ cloned := agent.getDeviceWithoutLock()
+
+ // Verify whether the Image is in the list of image being downloaded
+ if !isImageRegistered(img, cloned) {
+ return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, image-not-registered:%s", agent.deviceID, img.Name)
+ }
+
+ if cloned.AdminState != voltha.AdminState_ENABLED {
+ return nil, status.Errorf(codes.FailedPrecondition, "deviceId:%s, device-not-enabled-state:%s", agent.deviceID, img.Name)
+ }
+ // Update image download state
+ for _, image := range cloned.ImageDownloads {
+ if image.Id == img.Id && image.Name == img.Name {
+ image.ImageState = voltha.ImageDownload_IMAGE_REVERTING
+ }
+ }
+
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, ""); err != nil {
+ return nil, err
+ }
+
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.RevertImageUpdate(subCtx, proto.Clone(cloned).(*voltha.Device), img)
+ if err != nil {
+ cancel()
+ return nil, err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "revertImageUpdate", ch, agent.onSuccess, agent.onFailure)
+
+ return &voltha.OperationResp{Code: voltha.OperationResp_OPERATION_SUCCESS}, nil
+}
+
+func (agent *Agent) getImageDownloadStatus(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+ logger.Debugw("getImageDownloadStatus", log.Fields{"device-id": agent.deviceID})
+
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ device := agent.getDeviceWithoutLock()
+ ch, err := agent.adapterProxy.GetImageDownloadStatus(ctx, device, img)
+ agent.requestQueue.RequestComplete()
+ if err != nil {
+ return nil, err
+ }
+ // Wait for the adapter response
+ rpcResponse, ok := <-ch
+ if !ok {
+ return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
+ }
+ if rpcResponse.Err != nil {
+ return nil, rpcResponse.Err
+ }
+ // Successful response
+ imgDownload := &voltha.ImageDownload{}
+ if err := ptypes.UnmarshalAny(rpcResponse.Reply, imgDownload); err != nil {
+ return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
+ }
+ return imgDownload, nil
+}
+
+func (agent *Agent) updateImageDownload(ctx context.Context, img *voltha.ImageDownload) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("updating-image-download", log.Fields{"device-id": agent.deviceID, "img": img})
+
+ cloned := agent.getDeviceWithoutLock()
+
+ // Update the image as well as remove it if the download was cancelled
+ clonedImages := make([]*voltha.ImageDownload, len(cloned.ImageDownloads))
+ for _, image := range cloned.ImageDownloads {
+ if image.Id == img.Id && image.Name == img.Name {
+ if image.DownloadState != voltha.ImageDownload_DOWNLOAD_CANCELLED {
+ clonedImages = append(clonedImages, img)
+ }
+ }
+ }
+ cloned.ImageDownloads = clonedImages
+ // Set the Admin state to enabled if required
+ if (img.DownloadState != voltha.ImageDownload_DOWNLOAD_REQUESTED &&
+ img.DownloadState != voltha.ImageDownload_DOWNLOAD_STARTED) ||
+ (img.ImageState != voltha.ImageDownload_IMAGE_ACTIVATING) {
+ return agent.updateDeviceStateInStoreWithoutLock(ctx, cloned, voltha.AdminState_ENABLED, cloned.ConnectStatus, cloned.OperStatus)
+ }
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) getImageDownload(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("getImageDownload", log.Fields{"device-id": agent.deviceID})
+
+ cloned := agent.getDeviceWithoutLock()
+ for _, image := range cloned.ImageDownloads {
+ if image.Id == img.Id && image.Name == img.Name {
+ return image, nil
+ }
+ }
+ return nil, status.Errorf(codes.NotFound, "image-not-found:%s", img.Name)
+}
+
+func (agent *Agent) listImageDownloads(ctx context.Context, deviceID string) (*voltha.ImageDownloads, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("listImageDownloads", log.Fields{"device-id": agent.deviceID})
+
+ return &voltha.ImageDownloads{Items: agent.getDeviceWithoutLock().ImageDownloads}, nil
+}
+
+// getPorts retrieves the ports information of the device based on the port type.
+func (agent *Agent) getPorts(ctx context.Context, portType voltha.Port_PortType) *voltha.Ports {
+ logger.Debugw("getPorts", log.Fields{"device-id": agent.deviceID, "port-type": portType})
+ ports := &voltha.Ports{}
+ if device, _ := agent.deviceMgr.GetDevice(ctx, agent.deviceID); device != nil {
+ for _, port := range device.Ports {
+ if port.Type == portType {
+ ports.Items = append(ports.Items, port)
+ }
+ }
+ }
+ return ports
+}
+
+// getSwitchCapability retrieves the switch capability of a parent device
+func (agent *Agent) getSwitchCapability(ctx context.Context) (*ic.SwitchCapability, error) {
+ logger.Debugw("getSwitchCapability", log.Fields{"device-id": agent.deviceID})
+
+ cloned, err := agent.getDevice(ctx)
+ if err != nil {
+ return nil, err
+ }
+ ch, err := agent.adapterProxy.GetOfpDeviceInfo(ctx, cloned)
+ if err != nil {
+ return nil, err
+ }
+
+ // Wait for adapter response
+ rpcResponse, ok := <-ch
+ if !ok {
+ return nil, status.Errorf(codes.Aborted, "channel-closed")
+ }
+ if rpcResponse.Err != nil {
+ return nil, rpcResponse.Err
+ }
+ // Successful response
+ switchCap := &ic.SwitchCapability{}
+ if err := ptypes.UnmarshalAny(rpcResponse.Reply, switchCap); err != nil {
+ return nil, err
+ }
+ return switchCap, nil
+}
+
+// getPortCapability retrieves the port capability of a device
+func (agent *Agent) getPortCapability(ctx context.Context, portNo uint32) (*ic.PortCapability, error) {
+ logger.Debugw("getPortCapability", log.Fields{"device-id": agent.deviceID})
+ device, err := agent.getDevice(ctx)
+ if err != nil {
+ return nil, err
+ }
+ ch, err := agent.adapterProxy.GetOfpPortInfo(ctx, device, portNo)
+ if err != nil {
+ return nil, err
+ }
+ // Wait for adapter response
+ rpcResponse, ok := <-ch
+ if !ok {
+ return nil, status.Errorf(codes.Aborted, "channel-closed")
+ }
+ if rpcResponse.Err != nil {
+ return nil, rpcResponse.Err
+ }
+ // Successful response
+ portCap := &ic.PortCapability{}
+ if err := ptypes.UnmarshalAny(rpcResponse.Reply, portCap); err != nil {
+ return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
+ }
+ return portCap, nil
+}
+
+func (agent *Agent) onPacketFailure(rpc string, response interface{}, args ...interface{}) {
+ // packet data is encoded in the args param as the first parameter
+ var packet []byte
+ if len(args) >= 1 {
+ if pkt, ok := args[0].([]byte); ok {
+ packet = pkt
+ }
+ }
+ var errResp error
+ if err, ok := response.(error); ok {
+ errResp = err
+ }
+ logger.Warnw("packet-out-error", log.Fields{
+ "device-id": agent.deviceID,
+ "error": errResp,
+ "packet": hex.EncodeToString(packet),
+ })
+}
+
+func (agent *Agent) packetOut(ctx context.Context, outPort uint32, packet *ofp.OfpPacketOut) error {
+ // If deviceType=="" then we must have taken ownership of this device.
+ // Fixes VOL-2226 where a core would take ownership and have stale data
+ if agent.deviceType == "" {
+ agent.reconcileWithKVStore(ctx)
+ }
+ // Send packet to adapter
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.PacketOut(subCtx, agent.deviceType, agent.deviceID, outPort, packet)
+ if err != nil {
+ cancel()
+ return nil
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "packetOut", ch, agent.onSuccess, agent.onPacketFailure, packet.Data)
+ return nil
+}
+
+// updatePartialDeviceData updates a subset of a device that an Adapter can update.
+// TODO: May need a specific proto to handle only a subset of a device that can be changed by an adapter
+func (agent *Agent) mergeDeviceInfoFromAdapter(device *voltha.Device) (*voltha.Device, error) {
+ cloned := agent.getDeviceWithoutLock()
+ cloned.Root = device.Root
+ cloned.Vendor = device.Vendor
+ cloned.Model = device.Model
+ cloned.SerialNumber = device.SerialNumber
+ cloned.MacAddress = device.MacAddress
+ cloned.Vlan = device.Vlan
+ cloned.Reason = device.Reason
+ return cloned, nil
+}
+
+func (agent *Agent) updateDeviceUsingAdapterData(ctx context.Context, device *voltha.Device) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("updateDeviceUsingAdapterData", log.Fields{"device-id": device.Id})
+
+ updatedDevice, err := agent.mergeDeviceInfoFromAdapter(device)
+ if err != nil {
+ return status.Errorf(codes.Internal, "%s", err.Error())
+ }
+ cloned := proto.Clone(updatedDevice).(*voltha.Device)
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) updateDeviceWithoutLock(ctx context.Context, device *voltha.Device) error {
+ logger.Debugw("updateDevice", log.Fields{"deviceId": device.Id})
+ //cloned := proto.Clone(device).(*voltha.Device)
+ cloned := device
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) updateDeviceStatus(ctx context.Context, operStatus voltha.OperStatus_Types, connStatus voltha.ConnectStatus_Types) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ cloned := agent.getDeviceWithoutLock()
+
+ newConnStatus, newOperStatus := cloned.ConnectStatus, cloned.OperStatus
+ // Ensure the enums passed in are valid - they will be invalid if they are not set when this function is invoked
+ if s, ok := voltha.ConnectStatus_Types_value[connStatus.String()]; ok {
+ logger.Debugw("updateDeviceStatus-conn", log.Fields{"ok": ok, "val": s})
+ newConnStatus = connStatus
+ }
+ if s, ok := voltha.OperStatus_Types_value[operStatus.String()]; ok {
+ logger.Debugw("updateDeviceStatus-oper", log.Fields{"ok": ok, "val": s})
+ newOperStatus = operStatus
+ }
+ logger.Debugw("updateDeviceStatus", log.Fields{"deviceId": cloned.Id, "operStatus": cloned.OperStatus, "connectStatus": cloned.ConnectStatus})
+ // Store the device
+ return agent.updateDeviceStateInStoreWithoutLock(ctx, cloned, cloned.AdminState, newConnStatus, newOperStatus)
+}
+
+func (agent *Agent) updatePortsOperState(ctx context.Context, operStatus voltha.OperStatus_Types) error {
+ logger.Debugw("updatePortsOperState", log.Fields{"device-id": agent.deviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ cloned := agent.getDeviceWithoutLock()
+ for _, port := range cloned.Ports {
+ port.OperStatus = operStatus
+ }
+ // Store the device
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) updatePortState(ctx context.Context, portType voltha.Port_PortType, portNo uint32, operStatus voltha.OperStatus_Types) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ // Work only on latest data
+ // TODO: Get list of ports from device directly instead of the entire device
+ cloned := agent.getDeviceWithoutLock()
+
+ // Ensure the enums passed in are valid - they will be invalid if they are not set when this function is invoked
+ if _, ok := voltha.Port_PortType_value[portType.String()]; !ok {
+ return status.Errorf(codes.InvalidArgument, "%s", portType)
+ }
+ for _, port := range cloned.Ports {
+ if port.Type == portType && port.PortNo == portNo {
+ port.OperStatus = operStatus
+ }
+ }
+ logger.Debugw("portStatusUpdate", log.Fields{"deviceId": cloned.Id})
+ // Store the device
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) deleteAllPorts(ctx context.Context) error {
+ logger.Debugw("deleteAllPorts", log.Fields{"deviceId": agent.deviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ cloned := agent.getDeviceWithoutLock()
+
+ if cloned.AdminState != voltha.AdminState_DISABLED && cloned.AdminState != voltha.AdminState_DELETED {
+ err := status.Error(codes.FailedPrecondition, fmt.Sprintf("invalid-state-%v", cloned.AdminState))
+ logger.Warnw("invalid-state-removing-ports", log.Fields{"state": cloned.AdminState, "error": err})
+ return err
+ }
+ if len(cloned.Ports) == 0 {
+ logger.Debugw("no-ports-present", log.Fields{"deviceId": agent.deviceID})
+ return nil
+ }
+
+ cloned.Ports = []*voltha.Port{}
+ logger.Debugw("portStatusUpdate", log.Fields{"deviceId": cloned.Id})
+ // Store the device
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) addPort(ctx context.Context, port *voltha.Port) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("addPort", log.Fields{"deviceId": agent.deviceID})
+
+ cloned := agent.getDeviceWithoutLock()
+ updatePort := false
+ if cloned.Ports == nil {
+ // First port
+ logger.Debugw("addPort-first-port-to-add", log.Fields{"deviceId": agent.deviceID})
+ cloned.Ports = make([]*voltha.Port, 0)
+ } else {
+ for _, p := range cloned.Ports {
+ if p.Type == port.Type && p.PortNo == port.PortNo {
+ if p.Label == "" && p.Type == voltha.Port_PON_OLT {
+ //Creation of OLT PON port is being processed after a default PON port was created. Just update it.
+ logger.Infow("update-pon-port-created-by-default", log.Fields{"default-port": p, "port-to-add": port})
+ p.Label = port.Label
+ p.OperStatus = port.OperStatus
+ updatePort = true
+ break
+ }
+ logger.Debugw("port already exists", log.Fields{"port": port})
+ return nil
+ }
+ }
+ }
+ if !updatePort {
+ cp := proto.Clone(port).(*voltha.Port)
+ // Set the admin state of the port to ENABLE
+ cp.AdminState = voltha.AdminState_ENABLED
+ cloned.Ports = append(cloned.Ports, cp)
+ }
+ // Store the device
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) addPeerPort(ctx context.Context, peerPort *voltha.Port_PeerPort) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("adding-peer-peerPort", log.Fields{"device-id": agent.deviceID, "peer-peerPort": peerPort})
+
+ cloned := agent.getDeviceWithoutLock()
+
+ // Get the peer port on the device based on the peerPort no
+ found := false
+ for _, port := range cloned.Ports {
+ if port.PortNo == peerPort.PortNo { // found peerPort
+ cp := proto.Clone(peerPort).(*voltha.Port_PeerPort)
+ port.Peers = append(port.Peers, cp)
+ logger.Debugw("found-peer", log.Fields{"device-id": agent.deviceID, "portNo": peerPort.PortNo, "deviceId": agent.deviceID})
+ found = true
+ break
+ }
+ }
+ if !found && agent.isRootdevice {
+ // An ONU PON port has been created before the corresponding creation of the OLT PON port. Create the OLT PON port
+ // with default values which will be updated once the OLT PON port creation is processed.
+ ponPort := &voltha.Port{
+ PortNo: peerPort.PortNo,
+ Type: voltha.Port_PON_OLT,
+ AdminState: voltha.AdminState_ENABLED,
+ DeviceId: agent.deviceID,
+ Peers: []*voltha.Port_PeerPort{proto.Clone(peerPort).(*voltha.Port_PeerPort)},
+ }
+ cloned.Ports = append(cloned.Ports, ponPort)
+ logger.Infow("adding-default-pon-port", log.Fields{"device-id": agent.deviceID, "peer": peerPort, "pon-port": ponPort})
+ }
+ // Store the device
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+// TODO: A generic device update by attribute
+func (agent *Agent) updateDeviceAttribute(ctx context.Context, name string, value interface{}) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ logger.Warnw("request-aborted", log.Fields{"device-id": agent.deviceID, "name": name, "error": err})
+ return
+ }
+ defer agent.requestQueue.RequestComplete()
+ if value == nil {
+ return
+ }
+
+ cloned := agent.getDeviceWithoutLock()
+ updated := false
+ s := reflect.ValueOf(cloned).Elem()
+ if s.Kind() == reflect.Struct {
+ // exported field
+ f := s.FieldByName(name)
+ if f.IsValid() && f.CanSet() {
+ switch f.Kind() {
+ case reflect.String:
+ f.SetString(value.(string))
+ updated = true
+ case reflect.Uint32:
+ f.SetUint(uint64(value.(uint32)))
+ updated = true
+ case reflect.Bool:
+ f.SetBool(value.(bool))
+ updated = true
+ }
+ }
+ }
+ logger.Debugw("update-field-status", log.Fields{"deviceId": cloned.Id, "name": name, "updated": updated})
+ // Save the data
+
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, ""); err != nil {
+ logger.Warnw("attribute-update-failed", log.Fields{"attribute": name, "value": value})
+ }
+}
+
+func (agent *Agent) simulateAlarm(ctx context.Context, simulatereq *voltha.SimulateAlarmRequest) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("simulateAlarm", log.Fields{"id": agent.deviceID})
+
+ cloned := agent.getDeviceWithoutLock()
+
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.SimulateAlarm(subCtx, cloned, simulatereq)
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "simulateAlarm", ch, agent.onSuccess, agent.onFailure)
+ return nil
+}
+
+func (agent *Agent) updateDeviceStateInStoreWithoutLock(
+ ctx context.Context,
+ device *voltha.Device,
+ adminState voltha.AdminState_Types,
+ connectStatus voltha.ConnectStatus_Types,
+ operStatus voltha.OperStatus_Types,
+) error {
+ previousState := getDeviceStates(device)
+ device.AdminState, device.ConnectStatus, device.OperStatus = adminState, connectStatus, operStatus
+
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, device, false, ""); err != nil {
+ return err
+ }
+
+ // process state transition in its own thread
+ go func() {
+ if err := agent.deviceMgr.processTransition(context.Background(), device, previousState); err != nil {
+ log.Errorw("failed-process-transition", log.Fields{"deviceId": device.Id, "previousAdminState": previousState.Admin, "currentAdminState": device.AdminState})
+ }
+ }()
+ return nil
+}
+
+//This is an update operation to model without Lock.This function must never be invoked by another function unless the latter holds a lock on the device.
+// It is an internal helper function.
+func (agent *Agent) updateDeviceInStoreWithoutLock(ctx context.Context, device *voltha.Device, strict bool, txid string) error {
+ if agent.stopped {
+ return errors.New("device agent stopped")
+ }
+
+ updateCtx := context.WithValue(ctx, model.RequestTimestamp, time.Now().UnixNano())
+ if err := agent.clusterDataProxy.Update(updateCtx, "devices/"+agent.deviceID, device); err != nil {
+ return status.Errorf(codes.Internal, "failed-update-device:%s: %s", agent.deviceID, err)
+ }
+ logger.Debugw("updated-device-in-store", log.Fields{"deviceId: ": agent.deviceID})
+
+ agent.device = proto.Clone(device).(*voltha.Device)
+ return nil
+}
+
+func (agent *Agent) updateDeviceReason(ctx context.Context, reason string) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ cloned := agent.getDeviceWithoutLock()
+ cloned.Reason = reason
+ logger.Debugw("updateDeviceReason", log.Fields{"deviceId": cloned.Id, "reason": cloned.Reason})
+ // Store the device
+ return agent.updateDeviceInStoreWithoutLock(ctx, cloned, false, "")
+}
+
+func (agent *Agent) disablePort(ctx context.Context, Port *voltha.Port) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("disablePort", log.Fields{"device-id": agent.deviceID, "port-no": Port.PortNo})
+ var cp *voltha.Port
+ // Get the most up to date the device info
+ device := agent.getDeviceWithoutLock()
+ for _, port := range device.Ports {
+ if port.PortNo == Port.PortNo {
+ port.AdminState = voltha.AdminState_DISABLED
+ cp = proto.Clone(port).(*voltha.Port)
+ break
+
+ }
+ }
+ if cp == nil {
+ return status.Errorf(codes.InvalidArgument, "%v", Port.PortNo)
+ }
+
+ if cp.Type != voltha.Port_PON_OLT {
+ return status.Errorf(codes.InvalidArgument, "Disabling of Port Type %v unimplemented", cp.Type)
+ }
+ // Store the device
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, device, false, ""); err != nil {
+ logger.Debugw("updateDeviceInStoreWithoutLock error ", log.Fields{"device-id": agent.deviceID, "port-no": Port.PortNo, "error": err})
+ return err
+ }
+
+ //send request to adapter
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.DisablePort(ctx, device, cp)
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "disablePort", ch, agent.onSuccess, agent.onFailure)
+ return nil
+}
+
+func (agent *Agent) enablePort(ctx context.Context, Port *voltha.Port) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debugw("enablePort", log.Fields{"device-id": agent.deviceID, "port-no": Port.PortNo})
+
+ var cp *voltha.Port
+ // Get the most up to date the device info
+ device := agent.getDeviceWithoutLock()
+ for _, port := range device.Ports {
+ if port.PortNo == Port.PortNo {
+ port.AdminState = voltha.AdminState_ENABLED
+ cp = proto.Clone(port).(*voltha.Port)
+ break
+ }
+ }
+
+ if cp == nil {
+ return status.Errorf(codes.InvalidArgument, "%v", Port.PortNo)
+ }
+
+ if cp.Type != voltha.Port_PON_OLT {
+ return status.Errorf(codes.InvalidArgument, "Enabling of Port Type %v unimplemented", cp.Type)
+ }
+ // Store the device
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, device, false, ""); err != nil {
+ logger.Debugw("updateDeviceInStoreWithoutLock error ", log.Fields{"device-id": agent.deviceID, "port-no": Port.PortNo, "error": err})
+ return err
+ }
+ //send request to adapter
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.EnablePort(ctx, device, cp)
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "enablePort", ch, agent.onSuccess, agent.onFailure)
+ return nil
+}
+
+func (agent *Agent) ChildDeviceLost(ctx context.Context, device *voltha.Device) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logger.Debugw("childDeviceLost", log.Fields{"child-device-id": device.Id, "parent-device-ud": agent.deviceID})
+
+ //Remove the associated peer ports on the parent device
+ parentDevice := agent.getDeviceWithoutLock()
+ var updatedPeers []*voltha.Port_PeerPort
+ for _, port := range parentDevice.Ports {
+ updatedPeers = make([]*voltha.Port_PeerPort, 0)
+ for _, peerPort := range port.Peers {
+ if peerPort.DeviceId != device.Id {
+ updatedPeers = append(updatedPeers, peerPort)
+ }
+ }
+ port.Peers = updatedPeers
+ }
+ if err := agent.updateDeviceInStoreWithoutLock(ctx, parentDevice, false, ""); err != nil {
+ return err
+ }
+
+ //send request to adapter
+ subCtx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ ch, err := agent.adapterProxy.ChildDeviceLost(ctx, agent.deviceType, agent.deviceID, device.ParentPortNo, device.ProxyAddress.OnuId)
+ if err != nil {
+ cancel()
+ return err
+ }
+ go agent.waitForAdapterResponse(subCtx, cancel, "childDeviceLost", ch, agent.onSuccess, agent.onFailure)
+ return nil
+}
+
+func (agent *Agent) startOmciTest(ctx context.Context, omcitestrequest *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+
+ device := agent.getDeviceWithoutLock()
+
+ if device.Adapter == "" {
+ adapterName, err := agent.adapterMgr.GetAdapterType(device.Type)
+ if err != nil {
+ agent.requestQueue.RequestComplete()
+ return nil, err
+ }
+ device.Adapter = adapterName
+ }
+
+ // Send request to the adapter
+ ch, err := agent.adapterProxy.StartOmciTest(ctx, device, omcitestrequest)
+ agent.requestQueue.RequestComplete()
+ if err != nil {
+ return nil, err
+ }
+
+ // Wait for the adapter response
+ rpcResponse, ok := <-ch
+ if !ok {
+ return nil, status.Errorf(codes.Aborted, "channel-closed-device-id-%s", agent.deviceID)
+ }
+ if rpcResponse.Err != nil {
+ return nil, rpcResponse.Err
+ }
+
+ // Unmarshal and return the response
+ testResp := &voltha.TestResponse{}
+ if err := ptypes.UnmarshalAny(rpcResponse.Reply, testResp); err != nil {
+ return nil, status.Errorf(codes.InvalidArgument, "%s", err.Error())
+ }
+ logger.Debugw("Omci_test_Request-Success-device-agent", log.Fields{"testResp": testResp})
+ return testResp, nil
+}
diff --git a/rw_core/core/device/agent_test.go b/rw_core/core/device/agent_test.go
new file mode 100755
index 0000000..ce69599
--- /dev/null
+++ b/rw_core/core/device/agent_test.go
@@ -0,0 +1,538 @@
+/*
+* 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"
+ "github.com/gogo/protobuf/proto"
+ "github.com/opencord/voltha-go/db/model"
+ "github.com/opencord/voltha-go/rw_core/config"
+ "github.com/opencord/voltha-go/rw_core/core/adapter"
+ com "github.com/opencord/voltha-lib-go/v3/pkg/adapters/common"
+ "github.com/opencord/voltha-lib-go/v3/pkg/db"
+ "github.com/opencord/voltha-lib-go/v3/pkg/db/kvstore"
+ "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+ mock_etcd "github.com/opencord/voltha-lib-go/v3/pkg/mocks/etcd"
+ mock_kafka "github.com/opencord/voltha-lib-go/v3/pkg/mocks/kafka"
+ ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "github.com/phayes/freeport"
+ "github.com/stretchr/testify/assert"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+ "math/rand"
+ "sort"
+ "strconv"
+ "strings"
+ "sync"
+ "testing"
+ "time"
+)
+
+type DATest struct {
+ etcdServer *mock_etcd.EtcdServer
+ deviceMgr *Manager
+ logicalDeviceMgr *LogicalManager
+ kmp kafka.InterContainerProxy
+ kClient kafka.Client
+ kvClientPort int
+ oltAdapterName string
+ onuAdapterName string
+ coreInstanceID string
+ defaultTimeout time.Duration
+ maxTimeout time.Duration
+ device *voltha.Device
+ done chan int
+}
+
+func newDATest() *DATest {
+ test := &DATest{}
+ // Start the embedded etcd server
+ var err error
+ test.etcdServer, test.kvClientPort, err = startEmbeddedEtcdServer("voltha.rwcore.da.test", "voltha.rwcore.da.etcd", "error")
+ if err != nil {
+ logger.Fatal(err)
+ }
+ // Create the kafka client
+ test.kClient = mock_kafka.NewKafkaClient()
+ test.oltAdapterName = "olt_adapter_mock"
+ test.onuAdapterName = "onu_adapter_mock"
+ test.coreInstanceID = "rw-da-test"
+ test.defaultTimeout = 5 * time.Second
+ test.maxTimeout = 20 * time.Second
+ test.done = make(chan int)
+ parentID := com.GetRandomString(10)
+ test.device = &voltha.Device{
+ Type: "onu_adapter_mock",
+ ParentId: parentID,
+ ParentPortNo: 1,
+ VendorId: "onu_adapter_mock",
+ Adapter: "onu_adapter_mock",
+ Vlan: 100,
+ Address: nil,
+ ProxyAddress: &voltha.Device_ProxyAddress{
+ DeviceId: parentID,
+ DeviceType: "olt_adapter_mock",
+ ChannelId: 100,
+ ChannelGroupId: 0,
+ ChannelTermination: "",
+ OnuId: 2,
+ },
+ AdminState: voltha.AdminState_PREPROVISIONED,
+ OperStatus: voltha.OperStatus_UNKNOWN,
+ Reason: "All good",
+ ConnectStatus: voltha.ConnectStatus_UNKNOWN,
+ Custom: nil,
+ Ports: []*voltha.Port{
+ {PortNo: 1, Label: "pon-1", Type: voltha.Port_PON_ONU, AdminState: voltha.AdminState_ENABLED,
+ OperStatus: voltha.OperStatus_ACTIVE, Peers: []*voltha.Port_PeerPort{{DeviceId: parentID, PortNo: 1}}},
+ {PortNo: 100, Label: "uni-100", Type: voltha.Port_ETHERNET_UNI, AdminState: voltha.AdminState_ENABLED,
+ OperStatus: voltha.OperStatus_ACTIVE},
+ },
+ }
+ return test
+}
+
+type fakeEventCallbacks struct{}
+
+func (fakeEventCallbacks) SendChangeEvent(_ string, _ *ofp.OfpPortStatus) {}
+func (fakeEventCallbacks) SendPacketIn(_ string, _ string, _ *ofp.OfpPacketIn) {}
+
+func (dat *DATest) startCore(inCompeteMode bool) {
+ cfg := config.NewRWCoreFlags()
+ cfg.CorePairTopic = "rw_core"
+ cfg.DefaultRequestTimeout = dat.defaultTimeout
+ cfg.KVStorePort = dat.kvClientPort
+ cfg.InCompetingMode = inCompeteMode
+ grpcPort, err := freeport.GetFreePort()
+ if err != nil {
+ logger.Fatal("Cannot get a freeport for grpc")
+ }
+ cfg.GrpcPort = grpcPort
+ cfg.GrpcHost = "127.0.0.1"
+ client := setupKVClient(cfg, dat.coreInstanceID)
+ backend := &db.Backend{
+ Client: client,
+ StoreType: cfg.KVStoreType,
+ Host: cfg.KVStoreHost,
+ Port: cfg.KVStorePort,
+ Timeout: cfg.KVStoreTimeout,
+ LivenessChannelInterval: cfg.LiveProbeInterval / 2,
+ PathPrefix: cfg.KVStoreDataPrefix}
+ dat.kmp = kafka.NewInterContainerProxy(
+ kafka.InterContainerHost(cfg.KafkaAdapterHost),
+ kafka.InterContainerPort(cfg.KafkaAdapterPort),
+ kafka.MsgClient(dat.kClient),
+ kafka.DefaultTopic(&kafka.Topic{Name: cfg.CoreTopic}),
+ kafka.DeviceDiscoveryTopic(&kafka.Topic{Name: cfg.AffinityRouterTopic}))
+
+ endpointMgr := kafka.NewEndpointManager(backend)
+ proxy := model.NewProxy(backend, "/")
+ adapterMgr := adapter.NewAdapterManager(proxy, dat.coreInstanceID, dat.kClient)
+
+ dat.deviceMgr, dat.logicalDeviceMgr = NewDeviceManagers(proxy, adapterMgr, dat.kmp, endpointMgr, cfg.CorePairTopic, dat.coreInstanceID, cfg.DefaultCoreTimeout)
+ dat.logicalDeviceMgr.SetEventCallbacks(fakeEventCallbacks{})
+ if err = dat.kmp.Start(); err != nil {
+ logger.Fatal("Cannot start InterContainerProxy")
+ }
+ if err = adapterMgr.Start(context.Background()); err != nil {
+ logger.Fatal("Cannot start adapterMgr")
+ }
+ dat.deviceMgr.Start(context.Background())
+ dat.logicalDeviceMgr.Start(context.Background())
+}
+
+func (dat *DATest) stopAll() {
+ if dat.kClient != nil {
+ dat.kClient.Stop()
+ }
+ if dat.logicalDeviceMgr != nil {
+ dat.logicalDeviceMgr.Stop(context.Background())
+ }
+ if dat.deviceMgr != nil {
+ dat.deviceMgr.Stop(context.Background())
+ }
+ if dat.kmp != nil {
+ dat.kmp.Stop()
+ }
+ if dat.etcdServer != nil {
+ stopEmbeddedEtcdServer(dat.etcdServer)
+ }
+}
+
+//startEmbeddedEtcdServer creates and starts an Embedded etcd server locally.
+func startEmbeddedEtcdServer(configName, storageDir, logLevel string) (*mock_etcd.EtcdServer, int, error) {
+ kvClientPort, err := freeport.GetFreePort()
+ if err != nil {
+ return nil, 0, err
+ }
+ peerPort, err := freeport.GetFreePort()
+ if err != nil {
+ return nil, 0, err
+ }
+ etcdServer := mock_etcd.StartEtcdServer(mock_etcd.MKConfig(configName, kvClientPort, peerPort, storageDir, logLevel))
+ if etcdServer == nil {
+ return nil, 0, status.Error(codes.Internal, "Embedded server failed to start")
+ }
+ return etcdServer, kvClientPort, nil
+}
+
+func stopEmbeddedEtcdServer(server *mock_etcd.EtcdServer) {
+ if server != nil {
+ server.Stop()
+ }
+}
+
+func setupKVClient(cf *config.RWCoreFlags, coreInstanceID string) kvstore.Client {
+ addr := cf.KVStoreHost + ":" + strconv.Itoa(cf.KVStorePort)
+ client, err := kvstore.NewEtcdClient(addr, cf.KVStoreTimeout)
+ if err != nil {
+ panic("no kv client")
+ }
+ return client
+}
+
+func (dat *DATest) createDeviceAgent(t *testing.T) *Agent {
+ deviceMgr := dat.deviceMgr
+ clonedDevice := proto.Clone(dat.device).(*voltha.Device)
+ deviceAgent := newAgent(deviceMgr.adapterProxy, clonedDevice, deviceMgr, deviceMgr.clusterDataProxy, deviceMgr.defaultTimeout)
+ d, err := deviceAgent.start(context.TODO(), clonedDevice)
+ assert.Nil(t, err)
+ assert.NotNil(t, d)
+ deviceMgr.addDeviceAgentToMap(deviceAgent)
+ return deviceAgent
+}
+
+func (dat *DATest) updateDeviceConcurrently(t *testing.T, da *Agent, globalWG *sync.WaitGroup) {
+ originalDevice, err := da.getDevice(context.Background())
+ assert.Nil(t, err)
+ assert.NotNil(t, originalDevice)
+ var localWG sync.WaitGroup
+
+ // Update device routine
+ var (
+ root = false
+ vendor = "onu_adapter_mock"
+ model = "go-mock"
+ serialNumber = com.GetRandomSerialNumber()
+ macAddress = strings.ToUpper(com.GetRandomMacAddress())
+ vlan = rand.Uint32()
+ reason = "testing concurrent device update"
+ portToAdd = &voltha.Port{PortNo: 101, Label: "uni-101", Type: voltha.Port_ETHERNET_UNI, AdminState: voltha.AdminState_ENABLED,
+ OperStatus: voltha.OperStatus_ACTIVE}
+ )
+ localWG.Add(1)
+ go func() {
+ deviceToUpdate := proto.Clone(originalDevice).(*voltha.Device)
+ deviceToUpdate.Root = root
+ deviceToUpdate.Vendor = vendor
+ deviceToUpdate.Model = model
+ deviceToUpdate.SerialNumber = serialNumber
+ deviceToUpdate.MacAddress = macAddress
+ deviceToUpdate.Vlan = vlan
+ deviceToUpdate.Reason = reason
+ err := da.updateDeviceUsingAdapterData(context.Background(), deviceToUpdate)
+ assert.Nil(t, err)
+ localWG.Done()
+ }()
+
+ // Update the device status routine
+ localWG.Add(1)
+ go func() {
+ err := da.updateDeviceStatus(context.Background(), voltha.OperStatus_ACTIVE, voltha.ConnectStatus_REACHABLE)
+ assert.Nil(t, err)
+ localWG.Done()
+ }()
+
+ // Add a port routine
+ localWG.Add(1)
+ go func() {
+ err := da.addPort(context.Background(), portToAdd)
+ assert.Nil(t, err)
+ localWG.Done()
+ }()
+
+ // wait for go routines to be done
+ localWG.Wait()
+
+ expectedChange := proto.Clone(originalDevice).(*voltha.Device)
+ expectedChange.OperStatus = voltha.OperStatus_ACTIVE
+ expectedChange.ConnectStatus = voltha.ConnectStatus_REACHABLE
+ expectedChange.Ports = append(expectedChange.Ports, portToAdd)
+ expectedChange.Root = root
+ expectedChange.Vendor = vendor
+ expectedChange.Model = model
+ expectedChange.SerialNumber = serialNumber
+ expectedChange.MacAddress = macAddress
+ expectedChange.Vlan = vlan
+ expectedChange.Reason = reason
+
+ updatedDevice, _ := da.getDevice(context.Background())
+ assert.NotNil(t, updatedDevice)
+ assert.True(t, proto.Equal(expectedChange, updatedDevice))
+
+ globalWG.Done()
+}
+
+func TestConcurrentDevices(t *testing.T) {
+ for i := 0; i < 2; i++ {
+ da := newDATest()
+ assert.NotNil(t, da)
+ defer da.stopAll()
+
+ // Start the Core
+ da.startCore(false)
+
+ var wg sync.WaitGroup
+ numConCurrentDeviceAgents := 20
+ for i := 0; i < numConCurrentDeviceAgents; i++ {
+ wg.Add(1)
+ a := da.createDeviceAgent(t)
+ go da.updateDeviceConcurrently(t, a, &wg)
+ }
+
+ wg.Wait()
+ }
+}
+
+func isFlowSliceEqual(a, b []*ofp.OfpFlowStats) bool {
+ if len(a) != len(b) {
+ return false
+ }
+ sort.Slice(a, func(i, j int) bool {
+ return a[i].Id < a[j].Id
+ })
+ sort.Slice(b, func(i, j int) bool {
+ return b[i].Id < b[j].Id
+ })
+ for idx := range a {
+ if !proto.Equal(a[idx], b[idx]) {
+ return false
+ }
+ }
+ return true
+}
+
+func isGroupSliceEqual(a, b []*ofp.OfpGroupEntry) bool {
+ if len(a) != len(b) {
+ return false
+ }
+ sort.Slice(a, func(i, j int) bool {
+ return a[i].Desc.GroupId < a[j].Desc.GroupId
+ })
+ sort.Slice(b, func(i, j int) bool {
+ return b[i].Desc.GroupId < b[j].Desc.GroupId
+ })
+ for idx := range a {
+ if !proto.Equal(a[idx], b[idx]) {
+ return false
+ }
+ }
+ return true
+}
+
+func TestFlowsToUpdateToDelete_EmptySlices(t *testing.T) {
+ newFlows := []*ofp.OfpFlowStats{}
+ existingFlows := []*ofp.OfpFlowStats{}
+ expectedNewFlows := []*ofp.OfpFlowStats{}
+ expectedFlowsToDelete := []*ofp.OfpFlowStats{}
+ expectedUpdatedAllFlows := []*ofp.OfpFlowStats{}
+ uNF, fD, uAF := flowsToUpdateToDelete(newFlows, existingFlows)
+ assert.True(t, isFlowSliceEqual(uNF, expectedNewFlows))
+ assert.True(t, isFlowSliceEqual(fD, expectedFlowsToDelete))
+ assert.True(t, isFlowSliceEqual(uAF, expectedUpdatedAllFlows))
+}
+
+func TestFlowsToUpdateToDelete_NoExistingFlows(t *testing.T) {
+ newFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ }
+ existingFlows := []*ofp.OfpFlowStats{}
+ expectedNewFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ }
+ expectedFlowsToDelete := []*ofp.OfpFlowStats{}
+ expectedUpdatedAllFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ }
+ uNF, fD, uAF := flowsToUpdateToDelete(newFlows, existingFlows)
+ assert.True(t, isFlowSliceEqual(uNF, expectedNewFlows))
+ assert.True(t, isFlowSliceEqual(fD, expectedFlowsToDelete))
+ assert.True(t, isFlowSliceEqual(uAF, expectedUpdatedAllFlows))
+}
+
+func TestFlowsToUpdateToDelete_UpdateNoDelete(t *testing.T) {
+ newFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ }
+ existingFlows := []*ofp.OfpFlowStats{
+ {Id: 121, TableId: 1210, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1210000, PacketCount: 0},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 122, TableId: 1220, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1220000, PacketCount: 0},
+ }
+ expectedNewFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ }
+ expectedFlowsToDelete := []*ofp.OfpFlowStats{}
+ expectedUpdatedAllFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ {Id: 121, TableId: 1210, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1210000, PacketCount: 0},
+ {Id: 122, TableId: 1220, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1220000, PacketCount: 0},
+ }
+ uNF, fD, uAF := flowsToUpdateToDelete(newFlows, existingFlows)
+ assert.True(t, isFlowSliceEqual(uNF, expectedNewFlows))
+ assert.True(t, isFlowSliceEqual(fD, expectedFlowsToDelete))
+ assert.True(t, isFlowSliceEqual(uAF, expectedUpdatedAllFlows))
+}
+
+func TestFlowsToUpdateToDelete_UpdateAndDelete(t *testing.T) {
+ newFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 20},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 10, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ {Id: 126, TableId: 1260, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1260000, PacketCount: 0},
+ {Id: 127, TableId: 1270, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1270000, PacketCount: 0},
+ }
+ existingFlows := []*ofp.OfpFlowStats{
+ {Id: 121, TableId: 1210, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1210000, PacketCount: 0},
+ {Id: 122, TableId: 1220, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1220000, PacketCount: 0},
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ }
+ expectedNewFlows := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 20},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 10, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ {Id: 126, TableId: 1260, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1260000, PacketCount: 0},
+ {Id: 127, TableId: 1270, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1270000, PacketCount: 0},
+ }
+ expectedFlowsToDelete := []*ofp.OfpFlowStats{
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ }
+ expectedUpdatedAllFlows := []*ofp.OfpFlowStats{
+ {Id: 121, TableId: 1210, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1210000, PacketCount: 0},
+ {Id: 122, TableId: 1220, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1220000, PacketCount: 0},
+ {Id: 123, TableId: 1230, Priority: 100, IdleTimeout: 0, Flags: 0, Cookie: 1230000, PacketCount: 20},
+ {Id: 124, TableId: 1240, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1240000, PacketCount: 0},
+ {Id: 125, TableId: 1250, Priority: 1000, IdleTimeout: 10, Flags: 0, Cookie: 1250000, PacketCount: 0},
+ {Id: 126, TableId: 1260, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1260000, PacketCount: 0},
+ {Id: 127, TableId: 1270, Priority: 1000, IdleTimeout: 0, Flags: 0, Cookie: 1270000, PacketCount: 0},
+ }
+ uNF, fD, uAF := flowsToUpdateToDelete(newFlows, existingFlows)
+ assert.True(t, isFlowSliceEqual(uNF, expectedNewFlows))
+ assert.True(t, isFlowSliceEqual(fD, expectedFlowsToDelete))
+ assert.True(t, isFlowSliceEqual(uAF, expectedUpdatedAllFlows))
+}
+
+func TestGroupsToUpdateToDelete_EmptySlices(t *testing.T) {
+ newGroups := []*ofp.OfpGroupEntry{}
+ existingGroups := []*ofp.OfpGroupEntry{}
+ expectedNewGroups := []*ofp.OfpGroupEntry{}
+ expectedGroupsToDelete := []*ofp.OfpGroupEntry{}
+ expectedUpdatedAllGroups := []*ofp.OfpGroupEntry{}
+ uNG, gD, uAG := groupsToUpdateToDelete(newGroups, existingGroups)
+ assert.True(t, isGroupSliceEqual(uNG, expectedNewGroups))
+ assert.True(t, isGroupSliceEqual(gD, expectedGroupsToDelete))
+ assert.True(t, isGroupSliceEqual(uAG, expectedUpdatedAllGroups))
+}
+
+func TestGroupsToUpdateToDelete_NoExistingGroups(t *testing.T) {
+ newGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ }
+ existingGroups := []*ofp.OfpGroupEntry{}
+ expectedNewGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ }
+ expectedGroupsToDelete := []*ofp.OfpGroupEntry{}
+ expectedUpdatedAllGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ }
+ uNG, gD, uAG := groupsToUpdateToDelete(newGroups, existingGroups)
+ assert.True(t, isGroupSliceEqual(uNG, expectedNewGroups))
+ assert.True(t, isGroupSliceEqual(gD, expectedGroupsToDelete))
+ assert.True(t, isGroupSliceEqual(uAG, expectedUpdatedAllGroups))
+}
+
+func TestGroupsToUpdateToDelete_UpdateNoDelete(t *testing.T) {
+ newGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ }
+ existingGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 3, GroupId: 30, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 4, GroupId: 40, Buckets: nil}},
+ }
+ expectedNewGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ }
+ expectedGroupsToDelete := []*ofp.OfpGroupEntry{}
+ expectedUpdatedAllGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 3, GroupId: 30, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 4, GroupId: 40, Buckets: nil}},
+ }
+ uNG, gD, uAG := groupsToUpdateToDelete(newGroups, existingGroups)
+ assert.True(t, isGroupSliceEqual(uNG, expectedNewGroups))
+ assert.True(t, isGroupSliceEqual(gD, expectedGroupsToDelete))
+ assert.True(t, isGroupSliceEqual(uAG, expectedUpdatedAllGroups))
+}
+
+func TestGroupsToUpdateToDelete_UpdateWithDelete(t *testing.T) {
+ newGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: []*ofp.OfpBucket{{WatchPort: 10}}}},
+ }
+ existingGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 3, GroupId: 30, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 4, GroupId: 40, Buckets: nil}},
+ }
+ expectedNewGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: []*ofp.OfpBucket{{WatchPort: 10}}}},
+ }
+ expectedGroupsToDelete := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: nil}},
+ }
+ expectedUpdatedAllGroups := []*ofp.OfpGroupEntry{
+ {Desc: &ofp.OfpGroupDesc{Type: 1, GroupId: 10, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 2, GroupId: 20, Buckets: []*ofp.OfpBucket{{WatchPort: 10}}}},
+ {Desc: &ofp.OfpGroupDesc{Type: 3, GroupId: 30, Buckets: nil}},
+ {Desc: &ofp.OfpGroupDesc{Type: 4, GroupId: 40, Buckets: nil}},
+ }
+ uNG, gD, uAG := groupsToUpdateToDelete(newGroups, existingGroups)
+ assert.True(t, isGroupSliceEqual(uNG, expectedNewGroups))
+ assert.True(t, isGroupSliceEqual(gD, expectedGroupsToDelete))
+ assert.True(t, isGroupSliceEqual(uAG, expectedUpdatedAllGroups))
+}
diff --git a/rw_core/core/device/common.go b/rw_core/core/device/common.go
new file mode 100644
index 0000000..f2b8748
--- /dev/null
+++ b/rw_core/core/device/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 device
+
+import (
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+)
+
+var logger log.Logger
+
+func init() {
+ // Setup this package so that it's log level can be modified at run time
+ var err error
+ logger, err = log.AddPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "device"})
+ if err != nil {
+ panic(err)
+ }
+}
diff --git a/rw_core/core/device/logical_agent.go b/rw_core/core/device/logical_agent.go
new file mode 100644
index 0000000..c6e4e73
--- /dev/null
+++ b/rw_core/core/device/logical_agent.go
@@ -0,0 +1,1929 @@
+/*
+ * 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 device
+
+import (
+ "context"
+ "encoding/hex"
+ "errors"
+ "fmt"
+ "reflect"
+ "sync"
+ "time"
+
+ "github.com/gogo/protobuf/proto"
+ "github.com/opencord/voltha-go/db/model"
+ fd "github.com/opencord/voltha-go/rw_core/flowdecomposition"
+ "github.com/opencord/voltha-go/rw_core/route"
+ coreutils "github.com/opencord/voltha-go/rw_core/utils"
+ fu "github.com/opencord/voltha-lib-go/v3/pkg/flows"
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+ ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+ ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+)
+
+// LogicalAgent represent attributes of logical device agent
+type LogicalAgent struct {
+ logicalDeviceID string
+ serialNumber string
+ rootDeviceID string
+ deviceMgr *Manager
+ ldeviceMgr *LogicalManager
+ clusterDataProxy *model.Proxy
+ stopped bool
+ deviceRoutes *route.DeviceRoutes
+ lockDeviceRoutes sync.RWMutex
+ logicalPortsNo map[uint32]bool //value is true for NNI port
+ lockLogicalPortsNo sync.RWMutex
+ flowDecomposer *fd.FlowDecomposer
+ defaultTimeout time.Duration
+ logicalDevice *voltha.LogicalDevice
+ requestQueue *coreutils.RequestQueue
+ startOnce sync.Once
+ stopOnce sync.Once
+}
+
+func newLogicalDeviceAgent(id string, sn string, deviceID string, ldeviceMgr *LogicalManager,
+ deviceMgr *Manager, cdProxy *model.Proxy, timeout time.Duration) *LogicalAgent {
+ var agent LogicalAgent
+ agent.logicalDeviceID = id
+ agent.serialNumber = sn
+ agent.rootDeviceID = deviceID
+ agent.deviceMgr = deviceMgr
+ agent.clusterDataProxy = cdProxy
+ agent.ldeviceMgr = ldeviceMgr
+ agent.flowDecomposer = fd.NewFlowDecomposer(agent.deviceMgr)
+ agent.logicalPortsNo = make(map[uint32]bool)
+ agent.defaultTimeout = timeout
+ agent.requestQueue = coreutils.NewRequestQueue()
+ return &agent
+}
+
+// start creates the logical device and add it to the data model
+func (agent *LogicalAgent) start(ctx context.Context, loadFromDB bool) error {
+ needToStart := false
+ if agent.startOnce.Do(func() { needToStart = true }); !needToStart {
+ return nil
+ }
+
+ logger.Infow("starting-logical_device-agent", log.Fields{"logical-device-id": agent.logicalDeviceID, "load-from-db": loadFromDB})
+
+ var startSucceeded bool
+ defer func() {
+ if !startSucceeded {
+ if err := agent.stop(ctx); err != nil {
+ logger.Errorw("failed-to-cleanup-after-unsuccessful-start", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+ }
+ }
+ }()
+
+ var ld *voltha.LogicalDevice
+ if !loadFromDB {
+ //Build the logical device based on information retrieved from the device adapter
+ var switchCap *ic.SwitchCapability
+ var err error
+ if switchCap, err = agent.deviceMgr.getSwitchCapability(ctx, agent.rootDeviceID); err != nil {
+ return err
+ }
+ ld = &voltha.LogicalDevice{Id: agent.logicalDeviceID, RootDeviceId: agent.rootDeviceID}
+
+ // Create the datapath ID (uint64) using the logical device ID (based on the MAC Address)
+ var datapathID uint64
+ if datapathID, err = coreutils.CreateDataPathID(agent.serialNumber); err != nil {
+ return err
+ }
+ ld.DatapathId = datapathID
+ ld.Desc = (proto.Clone(switchCap.Desc)).(*ofp.OfpDesc)
+ logger.Debugw("Switch-capability", log.Fields{"Desc": ld.Desc, "fromAd": switchCap.Desc})
+ ld.SwitchFeatures = (proto.Clone(switchCap.SwitchFeatures)).(*ofp.OfpSwitchFeatures)
+ ld.Flows = &ofp.Flows{Items: nil}
+ ld.FlowGroups = &ofp.FlowGroups{Items: nil}
+ ld.Ports = []*voltha.LogicalPort{}
+
+ // Save the logical device
+ if err := agent.clusterDataProxy.AddWithID(ctx, "logical_devices", ld.Id, ld); err != nil {
+ logger.Errorw("failed-to-add-logical-device", log.Fields{"logical-device-id": agent.logicalDeviceID})
+ return err
+ }
+ logger.Debugw("logicaldevice-created", log.Fields{"logical-device-id": agent.logicalDeviceID, "root-id": ld.RootDeviceId})
+
+ agent.logicalDevice = proto.Clone(ld).(*voltha.LogicalDevice)
+
+ // Setup the logicalports - internal processing, no need to propagate the client context
+ go func() {
+ err := agent.setupLogicalPorts(context.Background())
+ if err != nil {
+ logger.Errorw("unable-to-setup-logical-ports", log.Fields{"error": err})
+ }
+ }()
+ } else {
+ // load from dB - the logical may not exist at this time. On error, just return and the calling function
+ // will destroy this agent.
+ ld := &voltha.LogicalDevice{}
+ have, err := agent.clusterDataProxy.Get(ctx, "logical_devices/"+agent.logicalDeviceID, ld)
+ if err != nil {
+ return err
+ } else if !have {
+ return status.Errorf(codes.NotFound, "logical_device-%s", agent.logicalDeviceID)
+ }
+
+ // Update the root device Id
+ agent.rootDeviceID = ld.RootDeviceId
+
+ // Update the last data
+ agent.logicalDevice = proto.Clone(ld).(*voltha.LogicalDevice)
+
+ // Setup the local list of logical ports
+ agent.addLogicalPortsToMap(ld.Ports)
+ }
+
+ // Setup the device routes. Building routes may fail if the pre-conditions are not satisfied (e.g. no PON ports present)
+ if loadFromDB {
+ go func() {
+ if err := agent.buildRoutes(context.Background()); err != nil {
+ logger.Warn("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+ }
+ }()
+ }
+ startSucceeded = true
+
+ return nil
+}
+
+// stop stops the logical device agent. This removes the logical device from the data model.
+func (agent *LogicalAgent) stop(ctx context.Context) error {
+ var returnErr error
+ agent.stopOnce.Do(func() {
+ logger.Info("stopping-logical_device-agent")
+
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ // This should never happen - an error is returned only if the agent is stopped and an agent is only stopped once.
+ returnErr = err
+ return
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ //Remove the logical device from the model
+ if err := agent.clusterDataProxy.Remove(ctx, "logical_devices/"+agent.logicalDeviceID); err != nil {
+ returnErr = err
+ } else {
+ logger.Debugw("logicaldevice-removed", log.Fields{"logicaldeviceId": agent.logicalDeviceID})
+ }
+
+ agent.stopped = true
+
+ logger.Info("logical_device-agent-stopped")
+ })
+ return returnErr
+}
+
+// GetLogicalDevice returns the latest logical device data
+func (agent *LogicalAgent) GetLogicalDevice(ctx context.Context) (*voltha.LogicalDevice, error) {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return nil, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ return proto.Clone(agent.logicalDevice).(*voltha.LogicalDevice), nil
+}
+
+// ListLogicalDeviceFlows returns logical device flows
+func (agent *LogicalAgent) ListLogicalDeviceFlows(ctx context.Context) (*ofp.Flows, error) {
+ logger.Debug("ListLogicalDeviceFlows")
+
+ logicalDevice, err := agent.GetLogicalDevice(ctx)
+ if err != nil {
+ return nil, err
+ }
+ if logicalDevice.Flows == nil {
+ return &ofp.Flows{}, nil
+ }
+ return (proto.Clone(logicalDevice.Flows)).(*ofp.Flows), nil
+}
+
+// ListLogicalDeviceMeters returns logical device meters
+func (agent *LogicalAgent) ListLogicalDeviceMeters(ctx context.Context) (*ofp.Meters, error) {
+ logger.Debug("ListLogicalDeviceMeters")
+
+ logicalDevice, err := agent.GetLogicalDevice(ctx)
+ if err != nil {
+ return nil, err
+ }
+ if logicalDevice.Meters == nil {
+ return &ofp.Meters{}, nil
+ }
+ return (proto.Clone(logicalDevice.Meters)).(*ofp.Meters), nil
+}
+
+// ListLogicalDeviceFlowGroups returns logical device flow groups
+func (agent *LogicalAgent) ListLogicalDeviceFlowGroups(ctx context.Context) (*ofp.FlowGroups, error) {
+ logger.Debug("ListLogicalDeviceFlowGroups")
+
+ logicalDevice, err := agent.GetLogicalDevice(ctx)
+ if err != nil {
+ return nil, err
+ }
+ if logicalDevice.FlowGroups == nil {
+ return &ofp.FlowGroups{}, nil
+ }
+ return (proto.Clone(logicalDevice.FlowGroups)).(*ofp.FlowGroups), nil
+}
+
+// ListLogicalDevicePorts returns logical device ports
+func (agent *LogicalAgent) ListLogicalDevicePorts(ctx context.Context) (*voltha.LogicalPorts, error) {
+ logger.Debug("ListLogicalDevicePorts")
+ logicalDevice, err := agent.GetLogicalDevice(ctx)
+ if err != nil {
+ return nil, err
+ }
+ if logicalDevice == nil {
+ return &voltha.LogicalPorts{}, nil
+ }
+ lPorts := make([]*voltha.LogicalPort, 0)
+ lPorts = append(lPorts, logicalDevice.Ports...)
+ return &voltha.LogicalPorts{Items: lPorts}, nil
+}
+
+//updateLogicalDeviceFlowsWithoutLock updates the logical device with the latest flows in the model.
+func (agent *LogicalAgent) updateLogicalDeviceFlowsWithoutLock(ctx context.Context, flows *ofp.Flows) error {
+ ld := agent.getLogicalDeviceWithoutLock()
+
+ logger.Debugw("logical-device-before", log.Fields{"lports": len(ld.Ports)})
+ ld.Flows = flows
+
+ if err := agent.updateLogicalDeviceWithoutLock(ctx, ld); err != nil {
+ logger.Errorw("error-updating-logical-device-with-flows", log.Fields{"error": err})
+ return err
+ }
+ return nil
+}
+
+//updateLogicalDeviceMetersWithoutLock updates the logical device with the meters info
+func (agent *LogicalAgent) updateLogicalDeviceMetersWithoutLock(ctx context.Context, meters *ofp.Meters) error {
+ ld := agent.getLogicalDeviceWithoutLock()
+
+ logger.Debugw("logical-device-before", log.Fields{"lports": len(ld.Ports)})
+ ld.Meters = meters
+
+ if err := agent.updateLogicalDeviceWithoutLock(ctx, ld); err != nil {
+ logger.Errorw("error-updating-logical-device-with-meters", log.Fields{"error": err})
+ return err
+ }
+ return nil
+}
+
+//updateLogicalDeviceFlowGroupsWithoutLock updates the logical device with the flow groups
+func (agent *LogicalAgent) updateLogicalDeviceFlowGroupsWithoutLock(ctx context.Context, flowGroups *ofp.FlowGroups) error {
+ ld := agent.getLogicalDeviceWithoutLock()
+
+ logger.Debugw("logical-device-before", log.Fields{"lports": len(ld.Ports)})
+ ld.FlowGroups = flowGroups
+
+ if err := agent.updateLogicalDeviceWithoutLock(ctx, ld); err != nil {
+ logger.Errorw("error-updating-logical-device-with-flowgroups", log.Fields{"error": err})
+ return err
+ }
+ return nil
+}
+
+// getLogicalDeviceWithoutLock returns a cloned logical device to a function that already holds the agent lock.
+func (agent *LogicalAgent) getLogicalDeviceWithoutLock() *voltha.LogicalDevice {
+ logger.Debug("getLogicalDeviceWithoutLock")
+ return proto.Clone(agent.logicalDevice).(*voltha.LogicalDevice)
+}
+
+func (agent *LogicalAgent) updateLogicalPort(ctx context.Context, device *voltha.Device, port *voltha.Port) error {
+ logger.Debugw("updateLogicalPort", log.Fields{"deviceId": device.Id, "port": port})
+ var err error
+ if port.Type == voltha.Port_ETHERNET_NNI {
+ if _, err = agent.addNNILogicalPort(ctx, device, port); err != nil {
+ return err
+ }
+ agent.addLogicalPortToMap(port.PortNo, true)
+ } else if port.Type == voltha.Port_ETHERNET_UNI {
+ if _, err = agent.addUNILogicalPort(ctx, device, port); err != nil {
+ return err
+ }
+ agent.addLogicalPortToMap(port.PortNo, false)
+ } else {
+ // Update the device routes to ensure all routes on the logical device have been calculated
+ if err = agent.buildRoutes(ctx); err != nil {
+ // Not an error - temporary state
+ logger.Warnw("failed-to-update-routes", log.Fields{"device-id": device.Id, "port": port, "error": err})
+ }
+ }
+ return nil
+}
+
+// setupLogicalPorts is invoked once the logical device has been created and is ready to get ports
+// added to it. While the logical device was being created we could have received requests to add
+// NNI and UNI ports which were discarded. Now is the time to add them if needed
+func (agent *LogicalAgent) setupLogicalPorts(ctx context.Context) error {
+ logger.Infow("setupLogicalPorts", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ // First add any NNI ports which could have been missing
+ if err := agent.setupNNILogicalPorts(ctx, agent.rootDeviceID); err != nil {
+ logger.Errorw("error-setting-up-NNI-ports", log.Fields{"error": err, "deviceId": agent.rootDeviceID})
+ return err
+ }
+
+ // Now, set up the UNI ports if needed.
+ children, err := agent.deviceMgr.GetAllChildDevices(ctx, agent.rootDeviceID)
+ if err != nil {
+ logger.Errorw("error-getting-child-devices", log.Fields{"error": err, "deviceId": agent.rootDeviceID})
+ return err
+ }
+ responses := make([]coreutils.Response, 0)
+ for _, child := range children.Items {
+ response := coreutils.NewResponse()
+ responses = append(responses, response)
+ go func(child *voltha.Device) {
+ if err = agent.setupUNILogicalPorts(context.Background(), child); err != nil {
+ logger.Error("setting-up-UNI-ports-failed", log.Fields{"deviceID": child.Id})
+ response.Error(status.Errorf(codes.Internal, "UNI-ports-setup-failed: %s", child.Id))
+ }
+ response.Done()
+ }(child)
+ }
+ // Wait for completion
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, responses...); res != nil {
+ return status.Errorf(codes.Aborted, "errors-%s", res)
+ }
+ return nil
+}
+
+// setupNNILogicalPorts creates an NNI port on the logical device that represents an NNI interface on a root device
+func (agent *LogicalAgent) setupNNILogicalPorts(ctx context.Context, deviceID string) error {
+ logger.Infow("setupNNILogicalPorts-start", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ // Build the logical device based on information retrieved from the device adapter
+ var err error
+
+ var device *voltha.Device
+ if device, err = agent.deviceMgr.GetDevice(ctx, deviceID); err != nil {
+ logger.Errorw("error-retrieving-device", log.Fields{"error": err, "deviceId": deviceID})
+ return err
+ }
+
+ //Get UNI port number
+ for _, port := range device.Ports {
+ if port.Type == voltha.Port_ETHERNET_NNI {
+ if _, err = agent.addNNILogicalPort(ctx, device, port); err != nil {
+ logger.Errorw("error-adding-UNI-port", log.Fields{"error": err})
+ }
+ agent.addLogicalPortToMap(port.PortNo, true)
+ }
+ }
+ return err
+}
+
+// updatePortState updates the port state of the device
+func (agent *LogicalAgent) updatePortState(ctx context.Context, deviceID string, portNo uint32, operStatus voltha.OperStatus_Types) error {
+ logger.Infow("updatePortState-start", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "portNo": portNo, "state": operStatus})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ // Get the latest logical device info
+ original := agent.getLogicalDeviceWithoutLock()
+ updatedPorts := clonePorts(original.Ports)
+ for _, port := range updatedPorts {
+ if port.DeviceId == deviceID && port.DevicePortNo == portNo {
+ if operStatus == voltha.OperStatus_ACTIVE {
+ port.OfpPort.Config = port.OfpPort.Config & ^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ port.OfpPort.State = uint32(ofp.OfpPortState_OFPPS_LIVE)
+ } else {
+ port.OfpPort.Config = port.OfpPort.Config | uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ port.OfpPort.State = uint32(ofp.OfpPortState_OFPPS_LINK_DOWN)
+ }
+ // Update the logical device
+ if err := agent.updateLogicalDevicePortsWithoutLock(ctx, original, updatedPorts); err != nil {
+ logger.Errorw("error-updating-logical-device", log.Fields{"error": err})
+ return err
+ }
+ return nil
+ }
+ }
+ return status.Errorf(codes.NotFound, "port-%d-not-exist", portNo)
+}
+
+// updatePortsState updates the ports state related to the device
+func (agent *LogicalAgent) updatePortsState(ctx context.Context, device *voltha.Device, state voltha.OperStatus_Types) error {
+ logger.Infow("updatePortsState-start", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ // Get the latest logical device info
+ original := agent.getLogicalDeviceWithoutLock()
+ updatedPorts := clonePorts(original.Ports)
+ for _, port := range updatedPorts {
+ if port.DeviceId == device.Id {
+ if state == voltha.OperStatus_ACTIVE {
+ port.OfpPort.Config = port.OfpPort.Config & ^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ port.OfpPort.State = uint32(ofp.OfpPortState_OFPPS_LIVE)
+ } else {
+ port.OfpPort.Config = port.OfpPort.Config | uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ port.OfpPort.State = uint32(ofp.OfpPortState_OFPPS_LINK_DOWN)
+ }
+ }
+ }
+ // Updating the logical device will trigger the poprt change events to be populated to the controller
+ if err := agent.updateLogicalDevicePortsWithoutLock(ctx, original, updatedPorts); err != nil {
+ logger.Warnw("logical-device-update-failed", log.Fields{"ldeviceId": agent.logicalDeviceID, "error": err})
+ return err
+ }
+ return nil
+}
+
+// setupUNILogicalPorts creates a UNI port on the logical device that represents a child UNI interface
+func (agent *LogicalAgent) setupUNILogicalPorts(ctx context.Context, childDevice *voltha.Device) error {
+ logger.Infow("setupUNILogicalPort", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ // Build the logical device based on information retrieved from the device adapter
+ var err error
+ var added bool
+ //Get UNI port number
+ for _, port := range childDevice.Ports {
+ if port.Type == voltha.Port_ETHERNET_UNI {
+ if added, err = agent.addUNILogicalPort(ctx, childDevice, port); err != nil {
+ logger.Errorw("error-adding-UNI-port", log.Fields{"error": err})
+ }
+ if added {
+ agent.addLogicalPortToMap(port.PortNo, false)
+ }
+ }
+ }
+ return err
+}
+
+// deleteAllLogicalPorts deletes all logical ports associated with this logical device
+func (agent *LogicalAgent) deleteAllLogicalPorts(ctx context.Context) error {
+ logger.Infow("updatePortsState-start", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ // Get the latest logical device info
+ cloned := agent.getLogicalDeviceWithoutLock()
+
+ if err := agent.updateLogicalDevicePortsWithoutLock(ctx, cloned, []*voltha.LogicalPort{}); err != nil {
+ logger.Warnw("logical-device-update-failed", log.Fields{"ldeviceId": agent.logicalDeviceID, "error": err})
+ return err
+ }
+ return nil
+}
+
+// deleteAllUNILogicalPorts deletes all UNI logical ports associated with this parent device
+func (agent *LogicalAgent) deleteAllUNILogicalPorts(ctx context.Context, parentDevice *voltha.Device) error {
+ logger.Debugw("delete-all-uni-logical-ports", log.Fields{"logical-device-id": agent.logicalDeviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ // Get the latest logical device info
+ ld := agent.getLogicalDeviceWithoutLock()
+
+ updateLogicalPorts := []*voltha.LogicalPort{}
+ for _, lport := range ld.Ports {
+ // Save NNI ports only
+ if agent.isNNIPort(lport.DevicePortNo) {
+ updateLogicalPorts = append(updateLogicalPorts, lport)
+ }
+ }
+ if len(updateLogicalPorts) < len(ld.Ports) {
+ // Updating the logical device will trigger the port change events to be populated to the controller
+ if err := agent.updateLogicalDevicePortsWithoutLock(ctx, ld, updateLogicalPorts); err != nil {
+ return err
+ }
+ } else {
+ logger.Debugw("no-change-required", log.Fields{"logical-device-id": agent.logicalDeviceID})
+ }
+ return nil
+}
+
+func clonePorts(ports []*voltha.LogicalPort) []*voltha.LogicalPort {
+ return proto.Clone(&voltha.LogicalPorts{Items: ports}).(*voltha.LogicalPorts).Items
+}
+
+//updateLogicalDevicePortsWithoutLock updates the
+func (agent *LogicalAgent) updateLogicalDevicePortsWithoutLock(ctx context.Context, device *voltha.LogicalDevice, newPorts []*voltha.LogicalPort) error {
+ oldPorts := device.Ports
+ device.Ports = newPorts
+ if err := agent.updateLogicalDeviceWithoutLock(ctx, device); err != nil {
+ return err
+ }
+ agent.portUpdated(oldPorts, newPorts)
+ return nil
+}
+
+//updateLogicalDeviceWithoutLock updates the model with the logical device. It clones the logicaldevice before saving it
+func (agent *LogicalAgent) updateLogicalDeviceWithoutLock(ctx context.Context, logicalDevice *voltha.LogicalDevice) error {
+ if agent.stopped {
+ return errors.New("logical device agent stopped")
+ }
+
+ updateCtx := context.WithValue(ctx, model.RequestTimestamp, time.Now().UnixNano())
+ if err := agent.clusterDataProxy.Update(updateCtx, "logical_devices/"+agent.logicalDeviceID, logicalDevice); err != nil {
+ logger.Errorw("failed-to-update-logical-devices-to-cluster-proxy", log.Fields{"error": err})
+ return err
+ }
+
+ agent.logicalDevice = logicalDevice
+
+ return nil
+}
+
+//generateDeviceRoutesIfNeeded generates the device routes if the logical device has been updated since the last time
+//that device graph was generated.
+func (agent *LogicalAgent) generateDeviceRoutesIfNeeded(ctx context.Context) error {
+ agent.lockDeviceRoutes.Lock()
+ defer agent.lockDeviceRoutes.Unlock()
+
+ ld, err := agent.GetLogicalDevice(ctx)
+ if err != nil {
+ return err
+ }
+
+ if agent.deviceRoutes != nil && agent.deviceRoutes.IsUpToDate(ld) {
+ return nil
+ }
+ logger.Debug("Generation of device route required")
+ if err := agent.buildRoutes(ctx); err != nil {
+ return err
+ }
+ return nil
+}
+
+//updateFlowTable updates the flow table of that logical device
+func (agent *LogicalAgent) updateFlowTable(ctx context.Context, flow *ofp.OfpFlowMod) error {
+ logger.Debug("UpdateFlowTable")
+ if flow == nil {
+ return nil
+ }
+ if err := agent.generateDeviceRoutesIfNeeded(ctx); err != nil {
+ return err
+ }
+ switch flow.GetCommand() {
+ case ofp.OfpFlowModCommand_OFPFC_ADD:
+ return agent.flowAdd(ctx, flow)
+ case ofp.OfpFlowModCommand_OFPFC_DELETE:
+ return agent.flowDelete(ctx, flow)
+ case ofp.OfpFlowModCommand_OFPFC_DELETE_STRICT:
+ return agent.flowDeleteStrict(ctx, flow)
+ case ofp.OfpFlowModCommand_OFPFC_MODIFY:
+ return agent.flowModify(flow)
+ case ofp.OfpFlowModCommand_OFPFC_MODIFY_STRICT:
+ return agent.flowModifyStrict(flow)
+ }
+ return status.Errorf(codes.Internal,
+ "unhandled-command: lDeviceId:%s, command:%s", agent.logicalDeviceID, flow.GetCommand())
+}
+
+//updateGroupTable updates the group table of that logical device
+func (agent *LogicalAgent) updateGroupTable(ctx context.Context, groupMod *ofp.OfpGroupMod) error {
+ logger.Debug("updateGroupTable")
+ if groupMod == nil {
+ return nil
+ }
+ if err := agent.generateDeviceRoutesIfNeeded(ctx); err != nil {
+ return err
+ }
+
+ switch groupMod.GetCommand() {
+ case ofp.OfpGroupModCommand_OFPGC_ADD:
+ return agent.groupAdd(ctx, groupMod)
+ case ofp.OfpGroupModCommand_OFPGC_DELETE:
+ return agent.groupDelete(ctx, groupMod)
+ case ofp.OfpGroupModCommand_OFPGC_MODIFY:
+ return agent.groupModify(ctx, groupMod)
+ }
+ return status.Errorf(codes.Internal,
+ "unhandled-command: lDeviceId:%s, command:%s", agent.logicalDeviceID, groupMod.GetCommand())
+}
+
+// updateMeterTable updates the meter table of that logical device
+func (agent *LogicalAgent) updateMeterTable(ctx context.Context, meterMod *ofp.OfpMeterMod) error {
+ logger.Debug("updateMeterTable")
+ if meterMod == nil {
+ return nil
+ }
+ switch meterMod.GetCommand() {
+ case ofp.OfpMeterModCommand_OFPMC_ADD:
+ return agent.meterAdd(ctx, meterMod)
+ case ofp.OfpMeterModCommand_OFPMC_DELETE:
+ return agent.meterDelete(ctx, meterMod)
+ case ofp.OfpMeterModCommand_OFPMC_MODIFY:
+ return agent.meterModify(ctx, meterMod)
+ }
+ return status.Errorf(codes.Internal,
+ "unhandled-command: lDeviceId:%s, command:%s", agent.logicalDeviceID, meterMod.GetCommand())
+
+}
+
+func (agent *LogicalAgent) meterAdd(ctx context.Context, meterMod *ofp.OfpMeterMod) error {
+ logger.Debugw("meterAdd", log.Fields{"metermod": *meterMod})
+ if meterMod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+ logger.Debug("Acquired logical device lock")
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ var meters []*ofp.OfpMeterEntry
+ if lDevice.Meters != nil && lDevice.Meters.Items != nil {
+ meters = lDevice.Meters.Items
+ }
+ logger.Debugw("Available meters", log.Fields{"meters": meters})
+
+ for _, meter := range meters {
+ if meterMod.MeterId == meter.Config.MeterId {
+ logger.Infow("Meter-already-exists", log.Fields{"meter": *meterMod})
+ return nil
+ }
+ }
+
+ meterEntry := fu.MeterEntryFromMeterMod(meterMod)
+ meters = append(meters, meterEntry)
+ //Update model
+ if err := agent.updateLogicalDeviceMetersWithoutLock(ctx, &ofp.Meters{Items: meters}); err != nil {
+ logger.Errorw("db-meter-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ logger.Debugw("Meter-added-successfully", log.Fields{"Added-meter": meterEntry, "updated-meters": lDevice.Meters})
+ return nil
+}
+
+func (agent *LogicalAgent) meterDelete(ctx context.Context, meterMod *ofp.OfpMeterMod) error {
+ logger.Debug("meterDelete", log.Fields{"meterMod": *meterMod})
+ if meterMod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ var meters []*ofp.OfpMeterEntry
+ var flows []*ofp.OfpFlowStats
+ updatedFlows := make([]*ofp.OfpFlowStats, 0)
+ if lDevice.Meters != nil && lDevice.Meters.Items != nil {
+ meters = lDevice.Meters.Items
+ }
+
+ changedMeter := false
+ changedFow := false
+ logger.Debugw("Available meters", log.Fields{"meters": meters})
+ for index, meter := range meters {
+ if meterMod.MeterId == meter.Config.MeterId {
+ flows = lDevice.Flows.Items
+ changedFow, updatedFlows = agent.getUpdatedFlowsAfterDeletebyMeterID(flows, meterMod.MeterId)
+ meters = append(meters[:index], meters[index+1:]...)
+ logger.Debugw("Meter has been deleted", log.Fields{"meter": meter, "index": index})
+ changedMeter = true
+ break
+ }
+ }
+ if changedMeter {
+ //Update model
+ metersToUpdate := &ofp.Meters{}
+ if lDevice.Meters != nil {
+ metersToUpdate = &ofp.Meters{Items: meters}
+ }
+ if err := agent.updateLogicalDeviceMetersWithoutLock(ctx, metersToUpdate); err != nil {
+ logger.Errorw("db-meter-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ logger.Debug("Meter-deleted-from-DB-successfully", log.Fields{"updatedMeters": metersToUpdate, "no-of-meter": len(metersToUpdate.Items)})
+
+ }
+ if changedFow {
+ //Update model
+ if err := agent.updateLogicalDeviceFlowsWithoutLock(ctx, &ofp.Flows{Items: updatedFlows}); err != nil {
+ logger.Errorw("db-flow-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ logger.Debug("Flows-associated-with-meter-deleted-from-DB-successfully",
+ log.Fields{"updated-no-of-flows": len(updatedFlows), "meter": meterMod.MeterId})
+ }
+ logger.Debugw("meterDelete success", log.Fields{"meterID": meterMod.MeterId})
+ return nil
+}
+
+func (agent *LogicalAgent) meterModify(ctx context.Context, meterMod *ofp.OfpMeterMod) error {
+ logger.Debug("meterModify")
+ if meterMod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ var meters []*ofp.OfpMeterEntry
+ if lDevice.Meters != nil && lDevice.Meters.Items != nil {
+ meters = lDevice.Meters.Items
+ }
+ changedMeter := false
+ for index, meter := range meters {
+ if meterMod.MeterId == meter.Config.MeterId {
+ newmeterEntry := fu.MeterEntryFromMeterMod(meterMod)
+ newmeterEntry.Stats.FlowCount = meter.Stats.FlowCount
+ meters[index] = newmeterEntry
+ changedMeter = true
+ logger.Debugw("Found meter, replaced with new meter", log.Fields{"old meter": meter, "new meter": newmeterEntry})
+ break
+ }
+ }
+ if changedMeter {
+ //Update model
+ metersToUpdate := &ofp.Meters{}
+ if lDevice.Meters != nil {
+ metersToUpdate = &ofp.Meters{Items: meters}
+ }
+ if err := agent.updateLogicalDeviceMetersWithoutLock(ctx, metersToUpdate); err != nil {
+ logger.Errorw("db-meter-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ logger.Debugw("meter-updated-in-DB-successfully", log.Fields{"updated_meters": meters})
+ return nil
+ }
+
+ logger.Errorw("Meter not found ", log.Fields{"meter": meterMod})
+ return fmt.Errorf("no-logical-device-present:%d", meterMod.MeterId)
+
+}
+
+func (agent *LogicalAgent) getUpdatedFlowsAfterDeletebyMeterID(flows []*ofp.OfpFlowStats, meterID uint32) (bool, []*ofp.OfpFlowStats) {
+ logger.Infow("Delete flows matching meter", log.Fields{"meter": meterID})
+ changed := false
+ //updatedFlows := make([]*ofp.OfpFlowStats, 0)
+ for index := len(flows) - 1; index >= 0; index-- {
+ if mID := fu.GetMeterIdFromFlow(flows[index]); mID != 0 && mID == meterID {
+ logger.Debugw("Flow to be deleted", log.Fields{"flow": flows[index], "index": index})
+ flows = append(flows[:index], flows[index+1:]...)
+ changed = true
+ }
+ }
+ return changed, flows
+}
+
+func (agent *LogicalAgent) updateFlowCountOfMeterStats(modCommand *ofp.OfpFlowMod, meters []*ofp.OfpMeterEntry, flow *ofp.OfpFlowStats) bool {
+
+ flowCommand := modCommand.GetCommand()
+ meterID := fu.GetMeterIdFromFlow(flow)
+ logger.Debugw("Meter-id-in-flow-mod", log.Fields{"meterId": meterID})
+ if meterID == 0 {
+ logger.Debugw("No meter present in the flow", log.Fields{"flow": *flow})
+ return false
+ }
+ if meters == nil {
+ logger.Debug("No meters present in logical device")
+ return false
+ }
+ changedMeter := false
+ for _, meter := range meters {
+ if meterID == meter.Config.MeterId { // Found meter in Logicaldevice
+ if flowCommand == ofp.OfpFlowModCommand_OFPFC_ADD {
+ meter.Stats.FlowCount++
+ changedMeter = true
+ } else if flowCommand == ofp.OfpFlowModCommand_OFPFC_DELETE_STRICT {
+ meter.Stats.FlowCount--
+ changedMeter = true
+ }
+ logger.Debugw("Found meter, updated meter flow stats", log.Fields{" meterId": meterID})
+ break
+ }
+ }
+ return changedMeter
+}
+
+//flowAdd adds a flow to the flow table of that logical device
+func (agent *LogicalAgent) flowAdd(ctx context.Context, mod *ofp.OfpFlowMod) error {
+ logger.Debugw("flowAdd", log.Fields{"flow": mod})
+ if mod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ var flows []*ofp.OfpFlowStats
+ var meters []*ofp.OfpMeterEntry
+ var flow *ofp.OfpFlowStats
+ var err error
+
+ if lDevice.Flows != nil && lDevice.Flows.Items != nil {
+ flows = lDevice.Flows.Items
+ }
+
+ if lDevice.Meters != nil && lDevice.Meters.Items != nil {
+ meters = lDevice.Meters.Items
+ }
+ updatedFlows := make([]*ofp.OfpFlowStats, 0)
+ changed := false
+ updated := false
+ checkOverlap := (mod.Flags & uint32(ofp.OfpFlowModFlags_OFPFF_CHECK_OVERLAP)) != 0
+ if checkOverlap {
+ if overlapped := fu.FindOverlappingFlows(flows, mod); len(overlapped) != 0 {
+ // TODO: should this error be notified other than being logged?
+ logger.Warnw("overlapped-flows", log.Fields{"logicaldeviceId": agent.logicalDeviceID})
+ } else {
+ // Add flow
+ flow, err = fu.FlowStatsEntryFromFlowModMessage(mod)
+ if err != nil {
+ return err
+ }
+ flows = append(flows, flow)
+ updatedFlows = append(updatedFlows, flow)
+ changed = true
+ }
+ } else {
+ flow, err = fu.FlowStatsEntryFromFlowModMessage(mod)
+ if err != nil {
+ return err
+ }
+ idx := fu.FindFlows(flows, flow)
+ if idx >= 0 {
+ oldFlow := flows[idx]
+ if (mod.Flags & uint32(ofp.OfpFlowModFlags_OFPFF_RESET_COUNTS)) != 0 {
+ flow.ByteCount = oldFlow.ByteCount
+ flow.PacketCount = oldFlow.PacketCount
+ }
+ if !reflect.DeepEqual(oldFlow, flow) {
+ flows[idx] = flow
+ updatedFlows = append(updatedFlows, flow)
+ changed = true
+ updated = true
+ }
+ } else {
+ flows = append(flows, flow)
+ updatedFlows = append(updatedFlows, flow)
+ changed = true
+ }
+ }
+ logger.Debugw("flowAdd-changed", log.Fields{"changed": changed})
+
+ if changed {
+ var flowMetadata voltha.FlowMetadata
+ if err := agent.GetMeterConfig(updatedFlows, meters, &flowMetadata); err != nil { // This should never happen,meters should be installed before flow arrives
+ logger.Error("Meter-referred-in-flows-not-present")
+ return err
+ }
+ deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: updatedFlows}, *lDevice.FlowGroups)
+ if err != nil {
+ return err
+ }
+ logger.Debugw("rules", log.Fields{"rules": deviceRules.String()})
+
+ // Update model
+ if err := agent.updateLogicalDeviceFlowsWithoutLock(ctx, &ofp.Flows{Items: flows}); err != nil {
+ logger.Errorw("db-flow-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ if !updated {
+ changedMeterStats := agent.updateFlowCountOfMeterStats(mod, meters, flow)
+ metersToUpdate := &ofp.Meters{}
+ if lDevice.Meters != nil {
+ metersToUpdate = &ofp.Meters{Items: meters}
+ }
+ if changedMeterStats {
+ //Update model
+ if err := agent.updateLogicalDeviceMetersWithoutLock(ctx, metersToUpdate); err != nil {
+ logger.Errorw("db-meter-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ logger.Debugw("meter-stats-updated-in-DB-successfully", log.Fields{"updated_meters": meters})
+
+ }
+ }
+ // Send the flows to the devices
+ respChannels := agent.addFlowsAndGroupsToDevices(ctx, deviceRules, &flowMetadata)
+
+ // Create the go routines to wait
+ go func() {
+ // Wait for completion
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChannels...); res != nil {
+ logger.Warnw("failure-to-add-flows", log.Fields{"errors": res, "logical-device-id": agent.logicalDeviceID})
+ // TODO : revert added flow
+ }
+ }()
+ }
+ return nil
+}
+
+// GetMeterConfig returns meter config
+func (agent *LogicalAgent) GetMeterConfig(flows []*ofp.OfpFlowStats, meters []*ofp.OfpMeterEntry, metadata *voltha.FlowMetadata) error {
+ m := make(map[uint32]bool)
+ for _, flow := range flows {
+ if flowMeterID := fu.GetMeterIdFromFlow(flow); flowMeterID != 0 && !m[flowMeterID] {
+ foundMeter := false
+ // Meter is present in the flow , Get from logical device
+ for _, meter := range meters {
+ if flowMeterID == meter.Config.MeterId {
+ metadata.Meters = append(metadata.Meters, meter.Config)
+ logger.Debugw("Found meter in logical device",
+ log.Fields{"meterID": flowMeterID, "meter-band": meter.Config})
+ m[flowMeterID] = true
+ foundMeter = true
+ break
+ }
+ }
+ if !foundMeter {
+ logger.Errorw("Meter-referred-by-flow-is-not-found-in-logicaldevice",
+ log.Fields{"meterID": flowMeterID, "Available-meters": meters, "flow": *flow})
+ return errors.New("Meter-referred-by-flow-is-not-found-in-logicaldevice")
+ }
+ }
+ }
+ logger.Debugw("meter-bands-for-flows", log.Fields{"flows": len(flows), "metadata": metadata})
+ return nil
+
+}
+
+//flowDelete deletes a flow from the flow table of that logical device
+func (agent *LogicalAgent) flowDelete(ctx context.Context, mod *ofp.OfpFlowMod) error {
+ logger.Debug("flowDelete")
+ if mod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ var meters []*ofp.OfpMeterEntry
+ var flows []*ofp.OfpFlowStats
+ var flowGroups []*ofp.OfpGroupEntry
+
+ if lDevice.Flows != nil && lDevice.Flows.Items != nil {
+ flows = lDevice.Flows.Items
+ }
+
+ if lDevice.Meters != nil && lDevice.Meters.Items != nil {
+ meters = lDevice.Meters.Items
+ }
+
+ if lDevice.FlowGroups != nil && lDevice.FlowGroups.Items != nil {
+ flowGroups = lDevice.FlowGroups.Items
+ }
+
+ //build a list of what to keep vs what to delete
+ toKeep := make([]*ofp.OfpFlowStats, 0)
+ toDelete := make([]*ofp.OfpFlowStats, 0)
+ for _, f := range flows {
+ // Check whether the flow and the flowmod matches
+ fs, err := fu.FlowStatsEntryFromFlowModMessage(mod)
+ if err != nil {
+ return err
+ }
+ if fu.FlowMatch(f, fs) {
+ toDelete = append(toDelete, f)
+ continue
+ }
+ // Check wild card match
+ if !fu.FlowMatchesMod(f, mod) {
+ toKeep = append(toKeep, f)
+ } else {
+ toDelete = append(toDelete, f)
+ }
+ }
+
+ logger.Debugw("flowDelete", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "toKeep": len(toKeep), "toDelete": toDelete})
+
+ //Update flows
+ if len(toDelete) > 0 {
+ var flowMetadata voltha.FlowMetadata
+ if err := agent.GetMeterConfig(toDelete, meters, &flowMetadata); err != nil { // This should never happen
+ logger.Error("Meter-referred-in-flows-not-present")
+ return errors.New("Meter-referred-in-flows-not-present")
+ }
+ deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: toDelete}, ofp.FlowGroups{Items: flowGroups})
+ if err != nil {
+ return err
+ }
+ logger.Debugw("rules", log.Fields{"rules": deviceRules.String()})
+
+ if err := agent.updateLogicalDeviceFlowsWithoutLock(ctx, &ofp.Flows{Items: toKeep}); err != nil {
+ logger.Errorw("cannot-update-flows", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+
+ // Update the devices
+ respChnls := agent.deleteFlowsAndGroupsFromDevices(ctx, deviceRules, &flowMetadata)
+
+ // Wait for the responses
+ go func() {
+ // Wait for completion
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+ logger.Errorw("failure-updating-device-flows", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "errors": res})
+ // TODO: Revert the flow deletion
+ }
+ }()
+ }
+
+ //TODO: send announcement on delete
+ return nil
+}
+
+func (agent *LogicalAgent) addFlowsAndGroupsToDevices(ctx context.Context, deviceRules *fu.DeviceRules, flowMetadata *voltha.FlowMetadata) []coreutils.Response {
+ logger.Debugw("send-add-flows-to-device-manager", log.Fields{"logicalDeviceID": agent.logicalDeviceID, "deviceRules": deviceRules, "flowMetadata": flowMetadata})
+
+ responses := make([]coreutils.Response, 0)
+ for deviceID, value := range deviceRules.GetRules() {
+ response := coreutils.NewResponse()
+ responses = append(responses, response)
+ go func(deviceId string, value *fu.FlowsAndGroups) {
+ ctx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ defer cancel()
+ if err := agent.deviceMgr.addFlowsAndGroups(ctx, deviceId, value.ListFlows(), value.ListGroups(), flowMetadata); err != nil {
+ logger.Errorw("flow-add-failed", log.Fields{"deviceID": deviceId, "error": err})
+ response.Error(status.Errorf(codes.Internal, "flow-add-failed: %s", deviceId))
+ }
+ response.Done()
+ }(deviceID, value)
+ }
+ // Return responses (an array of channels) for the caller to wait for a response from the far end.
+ return responses
+}
+
+func (agent *LogicalAgent) deleteFlowsAndGroupsFromDevices(ctx context.Context, deviceRules *fu.DeviceRules, flowMetadata *voltha.FlowMetadata) []coreutils.Response {
+ logger.Debugw("send-delete-flows-to-device-manager", log.Fields{"logicalDeviceID": agent.logicalDeviceID})
+
+ responses := make([]coreutils.Response, 0)
+ for deviceID, value := range deviceRules.GetRules() {
+ response := coreutils.NewResponse()
+ responses = append(responses, response)
+ go func(deviceId string, value *fu.FlowsAndGroups) {
+ ctx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ defer cancel()
+ if err := agent.deviceMgr.deleteFlowsAndGroups(ctx, deviceId, value.ListFlows(), value.ListGroups(), flowMetadata); err != nil {
+ logger.Error("flow-delete-failed", log.Fields{"deviceID": deviceId, "error": err})
+ response.Error(status.Errorf(codes.Internal, "flow-delete-failed: %s", deviceId))
+ }
+ response.Done()
+ }(deviceID, value)
+ }
+ return responses
+}
+
+func (agent *LogicalAgent) updateFlowsAndGroupsOfDevice(ctx context.Context, deviceRules *fu.DeviceRules, flowMetadata *voltha.FlowMetadata) []coreutils.Response {
+ logger.Debugw("send-update-flows-to-device-manager", log.Fields{"logicalDeviceID": agent.logicalDeviceID})
+
+ responses := make([]coreutils.Response, 0)
+ for deviceID, value := range deviceRules.GetRules() {
+ response := coreutils.NewResponse()
+ responses = append(responses, response)
+ go func(deviceId string, value *fu.FlowsAndGroups) {
+ ctx, cancel := context.WithTimeout(context.Background(), agent.defaultTimeout)
+ defer cancel()
+ if err := agent.deviceMgr.updateFlowsAndGroups(ctx, deviceId, value.ListFlows(), value.ListGroups(), flowMetadata); err != nil {
+ logger.Error("flow-update-failed", log.Fields{"deviceID": deviceId, "error": err})
+ response.Error(status.Errorf(codes.Internal, "flow-update-failed: %s", deviceId))
+ }
+ response.Done()
+ }(deviceID, value)
+ }
+ return responses
+}
+
+//flowDeleteStrict deletes a flow from the flow table of that logical device
+func (agent *LogicalAgent) flowDeleteStrict(ctx context.Context, mod *ofp.OfpFlowMod) error {
+ logger.Debug("flowDeleteStrict")
+ if mod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ var meters []*ofp.OfpMeterEntry
+ var flows []*ofp.OfpFlowStats
+ var flowGroups []*ofp.OfpGroupEntry
+ if lDevice.Meters != nil && lDevice.Meters.Items != nil {
+ meters = lDevice.Meters.Items
+ }
+ if lDevice.Flows != nil && lDevice.Flows.Items != nil {
+ flows = lDevice.Flows.Items
+ }
+ if lDevice.FlowGroups != nil && lDevice.FlowGroups.Items != nil {
+ flowGroups = lDevice.FlowGroups.Items
+ }
+
+ changedFlow := false
+ changedMeter := false
+ flow, err := fu.FlowStatsEntryFromFlowModMessage(mod)
+ if err != nil {
+ return err
+ }
+ flowsToDelete := make([]*ofp.OfpFlowStats, 0)
+ idx := fu.FindFlows(flows, flow)
+ if idx >= 0 {
+ changedMeter = agent.updateFlowCountOfMeterStats(mod, meters, flows[idx])
+ flowsToDelete = append(flowsToDelete, flows[idx])
+ flows = append(flows[:idx], flows[idx+1:]...)
+ changedFlow = true
+ } else {
+ return fmt.Errorf("Cannot delete flow - %s", flow)
+ }
+ if changedMeter {
+ //Update model
+ metersToUpdate := &ofp.Meters{}
+ if lDevice.Meters != nil {
+ metersToUpdate = &ofp.Meters{Items: meters}
+ }
+ if err := agent.updateLogicalDeviceMetersWithoutLock(ctx, metersToUpdate); err != nil {
+ logger.Errorw("db-meter-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+
+ }
+ if changedFlow {
+ var flowMetadata voltha.FlowMetadata
+ if err := agent.GetMeterConfig(flowsToDelete, meters, &flowMetadata); err != nil {
+ logger.Error("meter-referred-in-flows-not-present")
+ return err
+ }
+ deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: flowsToDelete}, ofp.FlowGroups{Items: flowGroups})
+ if err != nil {
+ return err
+ }
+ logger.Debugw("rules", log.Fields{"rules": deviceRules.String()})
+
+ if err := agent.updateLogicalDeviceFlowsWithoutLock(ctx, &ofp.Flows{Items: flows}); err != nil {
+ logger.Errorw("cannot-update-flows", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+
+ // Update the devices
+ respChnls := agent.deleteFlowsAndGroupsFromDevices(ctx, deviceRules, &flowMetadata)
+
+ // Wait for completion
+ go func() {
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+ logger.Warnw("failure-deleting-device-flows", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "errors": res})
+ //TODO: Revert flow changes
+ }
+ }()
+ }
+ return nil
+}
+
+//flowModify modifies a flow from the flow table of that logical device
+func (agent *LogicalAgent) flowModify(mod *ofp.OfpFlowMod) error {
+ return errors.New("flowModify not implemented")
+}
+
+//flowModifyStrict deletes a flow from the flow table of that logical device
+func (agent *LogicalAgent) flowModifyStrict(mod *ofp.OfpFlowMod) error {
+ return errors.New("flowModifyStrict not implemented")
+}
+
+func (agent *LogicalAgent) groupAdd(ctx context.Context, groupMod *ofp.OfpGroupMod) error {
+ logger.Debug("groupAdd")
+ if groupMod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ groups := lDevice.FlowGroups.Items
+ if fu.FindGroup(groups, groupMod.GroupId) == -1 {
+ groups = append(groups, fu.GroupEntryFromGroupMod(groupMod))
+
+ deviceRules := fu.NewDeviceRules()
+ deviceRules.CreateEntryIfNotExist(agent.rootDeviceID)
+ fg := fu.NewFlowsAndGroups()
+ fg.AddGroup(fu.GroupEntryFromGroupMod(groupMod))
+ deviceRules.AddFlowsAndGroup(agent.rootDeviceID, fg)
+
+ logger.Debugw("rules", log.Fields{"rules for group-add": deviceRules.String()})
+
+ if err := agent.updateLogicalDeviceFlowGroupsWithoutLock(ctx, &ofp.FlowGroups{Items: groups}); err != nil {
+ logger.Errorw("cannot-update-group", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+
+ // Update the devices
+ respChnls := agent.addFlowsAndGroupsToDevices(ctx, deviceRules, &voltha.FlowMetadata{})
+
+ // Wait for completion
+ go func() {
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+ logger.Warnw("failure-updating-device-flows-groups", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "errors": res})
+ //TODO: Revert flow changes
+ }
+ }()
+ return nil
+ }
+ return fmt.Errorf("Groups %d already present", groupMod.GroupId)
+}
+
+func (agent *LogicalAgent) groupDelete(ctx context.Context, groupMod *ofp.OfpGroupMod) error {
+ logger.Debug("groupDelete")
+ if groupMod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+ groups := lDevice.FlowGroups.Items
+ flows := lDevice.Flows.Items
+ var groupsChanged bool
+ flowsChanged := false
+ groupID := groupMod.GroupId
+ if groupID == uint32(ofp.OfpGroup_OFPG_ALL) {
+ //TODO we must delete all flows that point to this group and
+ //signal controller as requested by flow's flag
+ groups = []*ofp.OfpGroupEntry{}
+ groupsChanged = true
+ } else {
+ idx := fu.FindGroup(groups, groupID)
+ if idx == -1 {
+ return nil // Valid case
+ }
+ flowsChanged, flows = fu.FlowsDeleteByGroupId(flows, groupID)
+ groups = append(groups[:idx], groups[idx+1:]...)
+ groupsChanged = true
+ }
+ if flowsChanged || groupsChanged {
+ deviceRules, err := agent.flowDecomposer.DecomposeRules(ctx, agent, ofp.Flows{Items: flows}, ofp.FlowGroups{Items: groups})
+ if err != nil {
+ return err
+ }
+ logger.Debugw("rules", log.Fields{"rules": deviceRules.String()})
+
+ if groupsChanged {
+ if err := agent.updateLogicalDeviceFlowGroupsWithoutLock(ctx, &ofp.FlowGroups{Items: groups}); err != nil {
+ logger.Errorw("cannot-update-group", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ }
+ if flowsChanged {
+ if err := agent.updateLogicalDeviceFlowsWithoutLock(ctx, &ofp.Flows{Items: flows}); err != nil {
+ logger.Errorw("cannot-update-flow", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+ }
+
+ // Update the devices
+ respChnls := agent.updateFlowsAndGroupsOfDevice(ctx, deviceRules, nil)
+
+ // Wait for completion
+ go func() {
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+ logger.Warnw("failure-updating-device-flows-groups", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "errors": res})
+ //TODO: Revert flow changes
+ }
+ }()
+ }
+ return nil
+}
+
+func (agent *LogicalAgent) groupModify(ctx context.Context, groupMod *ofp.OfpGroupMod) error {
+ logger.Debug("groupModify")
+ if groupMod == nil {
+ return nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ lDevice := agent.getLogicalDeviceWithoutLock()
+ groups := lDevice.FlowGroups.Items
+ var groupsChanged bool
+ groupID := groupMod.GroupId
+ idx := fu.FindGroup(groups, groupID)
+ if idx == -1 {
+ return fmt.Errorf("group-absent:%d", groupID)
+ }
+ //replace existing group entry with new group definition
+ groupEntry := fu.GroupEntryFromGroupMod(groupMod)
+ groups[idx] = groupEntry
+ groupsChanged = true
+ if groupsChanged {
+ deviceRules := fu.NewDeviceRules()
+ deviceRules.CreateEntryIfNotExist(agent.rootDeviceID)
+ fg := fu.NewFlowsAndGroups()
+ fg.AddGroup(fu.GroupEntryFromGroupMod(groupMod))
+ deviceRules.AddFlowsAndGroup(agent.rootDeviceID, fg)
+
+ logger.Debugw("rules", log.Fields{"rules for group-modify": deviceRules.String()})
+
+ if err := agent.updateLogicalDeviceFlowGroupsWithoutLock(ctx, &ofp.FlowGroups{Items: groups}); err != nil {
+ logger.Errorw("Cannot-update-logical-group", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+
+ // Update the devices
+ respChnls := agent.updateFlowsAndGroupsOfDevice(ctx, deviceRules, &voltha.FlowMetadata{})
+
+ // Wait for completion
+ go func() {
+ if res := coreutils.WaitForNilOrErrorResponses(agent.defaultTimeout, respChnls...); res != nil {
+ logger.Warnw("failure-updating-device-flows-groups", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "errors": res})
+ //TODO: Revert flow changes
+ }
+ }()
+ }
+ return nil
+}
+
+// deleteLogicalPort removes the logical port
+func (agent *LogicalAgent) deleteLogicalPort(ctx context.Context, lPort *voltha.LogicalPort) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logicalDevice := agent.getLogicalDeviceWithoutLock()
+
+ index := -1
+ for i, logicalPort := range logicalDevice.Ports {
+ if logicalPort.Id == lPort.Id {
+ index = i
+ break
+ }
+ }
+ if index >= 0 {
+ clonedPorts := clonePorts(logicalDevice.Ports)
+ if index < len(clonedPorts)-1 {
+ copy(clonedPorts[index:], clonedPorts[index+1:])
+ }
+ clonedPorts[len(clonedPorts)-1] = nil
+ clonedPorts = clonedPorts[:len(clonedPorts)-1]
+ logger.Debugw("logical-port-deleted", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ if err := agent.updateLogicalDevicePortsWithoutLock(ctx, logicalDevice, clonedPorts); err != nil {
+ logger.Errorw("logical-device-update-failed", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ return err
+ }
+
+ // Remove the logical port from cache
+ agent.deleteLogicalPortsFromMap([]uint32{lPort.DevicePortNo})
+
+ // Reset the logical device routes
+ go func() {
+ if err := agent.buildRoutes(context.Background()); err != nil {
+ logger.Warnw("device-routes-not-ready", log.Fields{"logicalDeviceId": agent.logicalDeviceID, "error": err})
+ }
+ }()
+ }
+ return nil
+}
+
+// deleteLogicalPorts removes the logical ports associated with that deviceId
+func (agent *LogicalAgent) deleteLogicalPorts(ctx context.Context, deviceID string) error {
+ logger.Debugw("deleting-logical-ports", log.Fields{"device-id": deviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logicalDevice := agent.getLogicalDeviceWithoutLock()
+ lPortstoKeep := []*voltha.LogicalPort{}
+ lPortsNoToDelete := []uint32{}
+ for _, logicalPort := range logicalDevice.Ports {
+ if logicalPort.DeviceId != deviceID {
+ lPortstoKeep = append(lPortstoKeep, logicalPort)
+ } else {
+ lPortsNoToDelete = append(lPortsNoToDelete, logicalPort.DevicePortNo)
+ }
+ }
+ logger.Debugw("deleted-logical-ports", log.Fields{"ports": lPortstoKeep})
+ if err := agent.updateLogicalDevicePortsWithoutLock(ctx, logicalDevice, lPortstoKeep); err != nil {
+ logger.Errorw("logical-device-update-failed", log.Fields{"logical-device-id": agent.logicalDeviceID})
+ return err
+ }
+ // Remove the port from the cached logical ports set
+ agent.deleteLogicalPortsFromMap(lPortsNoToDelete)
+
+ // Reset the logical device routes
+ go func() {
+ if err := agent.buildRoutes(context.Background()); err != nil {
+ logger.Warnw("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+ }
+ }()
+
+ return nil
+}
+
+// enableLogicalPort enables the logical port
+func (agent *LogicalAgent) enableLogicalPort(ctx context.Context, lPortID string) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ logicalDevice := agent.getLogicalDeviceWithoutLock()
+
+ index := -1
+ for i, logicalPort := range logicalDevice.Ports {
+ if logicalPort.Id == lPortID {
+ index = i
+ break
+ }
+ }
+ if index >= 0 {
+ clonedPorts := clonePorts(logicalDevice.Ports)
+ clonedPorts[index].OfpPort.Config = clonedPorts[index].OfpPort.Config & ^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ return agent.updateLogicalDevicePortsWithoutLock(ctx, logicalDevice, clonedPorts)
+ }
+ return status.Errorf(codes.NotFound, "Port %s on Logical Device %s", lPortID, agent.logicalDeviceID)
+}
+
+// disableLogicalPort disabled the logical port
+func (agent *LogicalAgent) disableLogicalPort(ctx context.Context, lPortID string) error {
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ // Get the most up to date logical device
+ logicalDevice := agent.getLogicalDeviceWithoutLock()
+ index := -1
+ for i, logicalPort := range logicalDevice.Ports {
+ if logicalPort.Id == lPortID {
+ index = i
+ break
+ }
+ }
+ if index >= 0 {
+ clonedPorts := clonePorts(logicalDevice.Ports)
+ clonedPorts[index].OfpPort.Config = (clonedPorts[index].OfpPort.Config & ^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)) | uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ return agent.updateLogicalDevicePortsWithoutLock(ctx, logicalDevice, clonedPorts)
+ }
+ return status.Errorf(codes.NotFound, "Port %s on Logical Device %s", lPortID, agent.logicalDeviceID)
+}
+
+func (agent *LogicalAgent) getPreCalculatedRoute(ingress, egress uint32) ([]route.Hop, error) {
+ logger.Debugw("ROUTE", log.Fields{"len": len(agent.deviceRoutes.Routes)})
+ for routeLink, route := range agent.deviceRoutes.Routes {
+ logger.Debugw("ROUTELINKS", log.Fields{"ingress": ingress, "egress": egress, "routelink": routeLink})
+ if ingress == routeLink.Ingress && egress == routeLink.Egress {
+ return route, nil
+ }
+ }
+ return nil, status.Errorf(codes.FailedPrecondition, "no route from:%d to:%d", ingress, egress)
+}
+
+// GetRoute returns route
+func (agent *LogicalAgent) GetRoute(ctx context.Context, ingressPortNo uint32, egressPortNo uint32) ([]route.Hop, error) {
+ logger.Debugw("getting-route", log.Fields{"ingress-port": ingressPortNo, "egress-port": egressPortNo})
+ routes := make([]route.Hop, 0)
+
+ // Note: A port value of 0 is equivalent to a nil port
+
+ // Consider different possibilities
+ if egressPortNo != 0 && ((egressPortNo & 0x7fffffff) == uint32(ofp.OfpPortNo_OFPP_CONTROLLER)) {
+ logger.Debugw("controller-flow", log.Fields{"ingressPortNo": ingressPortNo, "egressPortNo": egressPortNo, "logicalPortsNo": agent.logicalPortsNo})
+ if agent.isNNIPort(ingressPortNo) {
+ //This is a trap on the NNI Port
+ if len(agent.deviceRoutes.Routes) == 0 {
+ // If there are no routes set (usually when the logical device has only NNI port(s), then just return an
+ // route with same IngressHop and EgressHop
+ hop := route.Hop{DeviceID: agent.rootDeviceID, Ingress: ingressPortNo, Egress: ingressPortNo}
+ routes = append(routes, hop)
+ routes = append(routes, hop)
+ return routes, nil
+ }
+ //Return a 'half' route to make the flow decomposer logic happy
+ for routeLink, path := range agent.deviceRoutes.Routes {
+ if agent.isNNIPort(routeLink.Egress) {
+ routes = append(routes, route.Hop{}) // first hop is set to empty
+ routes = append(routes, path[1])
+ return routes, nil
+ }
+ }
+ return nil, status.Errorf(codes.FailedPrecondition, "no upstream route from:%d to:%d", ingressPortNo, egressPortNo)
+ }
+ //treat it as if the output port is the first NNI of the OLT
+ var err error
+ if egressPortNo, err = agent.getFirstNNIPort(); err != nil {
+ logger.Warnw("no-nni-port", log.Fields{"error": err})
+ return nil, err
+ }
+ }
+ //If ingress port is not specified (nil), it may be a wildcarded
+ //route if egress port is OFPP_CONTROLLER or a nni logical port,
+ //in which case we need to create a half-route where only the egress
+ //hop is filled, the first hop is nil
+ if ingressPortNo == 0 && agent.isNNIPort(egressPortNo) {
+ // We can use the 2nd hop of any upstream route, so just find the first upstream:
+ for routeLink, path := range agent.deviceRoutes.Routes {
+ if agent.isNNIPort(routeLink.Egress) {
+ routes = append(routes, route.Hop{}) // first hop is set to empty
+ routes = append(routes, path[1])
+ return routes, nil
+ }
+ }
+ return nil, status.Errorf(codes.FailedPrecondition, "no upstream route from:%d to:%d", ingressPortNo, egressPortNo)
+ }
+ //If egress port is not specified (nil), we can also can return a "half" route
+ if egressPortNo == 0 {
+ for routeLink, path := range agent.deviceRoutes.Routes {
+ if routeLink.Ingress == ingressPortNo {
+ routes = append(routes, path[0])
+ routes = append(routes, route.Hop{})
+ return routes, nil
+ }
+ }
+ return nil, status.Errorf(codes.FailedPrecondition, "no downstream route from:%d to:%d", ingressPortNo, egressPortNo)
+ }
+ // Return the pre-calculated route
+ return agent.getPreCalculatedRoute(ingressPortNo, egressPortNo)
+}
+
+//GetWildcardInputPorts filters out the logical port number from the set of logical ports on the device and
+//returns their port numbers. This function is invoked only during flow decomposition where the lock on the logical
+//device is already held. Therefore it is safe to retrieve the logical device without lock.
+func (agent *LogicalAgent) GetWildcardInputPorts(excludePort ...uint32) []uint32 {
+ lPorts := make([]uint32, 0)
+ var exclPort uint32
+ if len(excludePort) == 1 {
+ exclPort = excludePort[0]
+ }
+ lDevice := agent.getLogicalDeviceWithoutLock()
+ for _, port := range lDevice.Ports {
+ if port.OfpPort.PortNo != exclPort {
+ lPorts = append(lPorts, port.OfpPort.PortNo)
+ }
+ }
+ return lPorts
+}
+
+// GetDeviceRoutes returns device graph
+func (agent *LogicalAgent) GetDeviceRoutes() *route.DeviceRoutes {
+ return agent.deviceRoutes
+}
+
+//rebuildRoutes rebuilds the device routes
+func (agent *LogicalAgent) buildRoutes(ctx context.Context) error {
+ logger.Debugf("building-routes", log.Fields{"logical-device-id": agent.logicalDeviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ if agent.deviceRoutes == nil {
+ agent.deviceRoutes = route.NewDeviceRoutes(agent.logicalDeviceID, agent.deviceMgr.GetDevice)
+ }
+ // Get all the logical ports on that logical device
+ lDevice := agent.getLogicalDeviceWithoutLock()
+
+ if err := agent.deviceRoutes.ComputeRoutes(ctx, lDevice.Ports); err != nil {
+ return err
+ }
+ if err := agent.deviceRoutes.Print(); err != nil {
+ return err
+ }
+
+ return nil
+}
+
+//updateRoutes updates the device routes
+func (agent *LogicalAgent) updateRoutes(ctx context.Context, lp *voltha.LogicalPort) error {
+ logger.Debugw("updateRoutes", log.Fields{"logicalDeviceId": agent.logicalDeviceID})
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return err
+ }
+ defer agent.requestQueue.RequestComplete()
+
+ if agent.deviceRoutes == nil {
+ agent.deviceRoutes = route.NewDeviceRoutes(agent.logicalDeviceID, agent.deviceMgr.GetDevice)
+ }
+ if err := agent.deviceRoutes.AddPort(ctx, lp, agent.logicalDevice.Ports); err != nil {
+ return err
+ }
+ if err := agent.deviceRoutes.Print(); err != nil {
+ return err
+ }
+ return nil
+}
+
+// diff go over two lists of logical ports and return what's new, what's changed and what's removed.
+func diff(oldList, newList []*voltha.LogicalPort) (newPorts, changedPorts, deletedPorts []*voltha.LogicalPort) {
+ newPorts = make([]*voltha.LogicalPort, 0)
+ changedPorts = make([]*voltha.LogicalPort, 0)
+ deletedPorts = make([]*voltha.LogicalPort, 0)
+ for _, o := range oldList {
+ found := false
+ for _, n := range newList {
+ if o.Id == n.Id {
+ found = true
+ break
+ }
+ }
+ if !found {
+ deletedPorts = append(deletedPorts, o)
+ }
+ }
+ for _, n := range newList {
+ found := false
+ changed := false
+ for _, o := range oldList {
+ if o.Id == n.Id {
+ changed = !reflect.DeepEqual(o, n)
+ found = true
+ break
+ }
+ }
+ if !found {
+ newPorts = append(newPorts, n)
+ }
+ if changed {
+ changedPorts = append(changedPorts, n)
+ }
+ }
+ return
+}
+
+// portUpdated is invoked when a port is updated on the logical device. Until
+// the POST_ADD notification is fixed, we will use the logical device to
+// update that data.
+func (agent *LogicalAgent) portUpdated(oldPorts, newPorts []*voltha.LogicalPort) interface{} {
+ if reflect.DeepEqual(oldPorts, newPorts) {
+ logger.Debug("ports-have-not-changed")
+ return nil
+ }
+
+ // Get the difference between the two list
+ newPorts, changedPorts, deletedPorts := diff(oldPorts, newPorts)
+
+ // Send the port change events to the OF controller
+ for _, newP := range newPorts {
+ go agent.ldeviceMgr.eventCallbacks.SendChangeEvent(agent.logicalDeviceID,
+ &ofp.OfpPortStatus{Reason: ofp.OfpPortReason_OFPPR_ADD, Desc: newP.OfpPort})
+ }
+ for _, change := range changedPorts {
+ go agent.ldeviceMgr.eventCallbacks.SendChangeEvent(agent.logicalDeviceID,
+ &ofp.OfpPortStatus{Reason: ofp.OfpPortReason_OFPPR_MODIFY, Desc: change.OfpPort})
+ }
+ for _, del := range deletedPorts {
+ go agent.ldeviceMgr.eventCallbacks.SendChangeEvent(agent.logicalDeviceID,
+ &ofp.OfpPortStatus{Reason: ofp.OfpPortReason_OFPPR_DELETE, Desc: del.OfpPort})
+ }
+
+ return nil
+}
+
+// addNNILogicalPort adds an NNI port to the logical device. It returns a bool representing whether a port has been
+// added and an eror in case a valid error is encountered. If the port was successfully added it will return
+// (true, nil). If the device is not in the correct state it will return (false, nil) as this is a valid
+// scenario. This also applies to the case where the port was already added.
+func (agent *LogicalAgent) addNNILogicalPort(ctx context.Context, device *voltha.Device, port *voltha.Port) (bool, error) {
+ logger.Debugw("addNNILogicalPort", log.Fields{"NNI": port})
+
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return false, err
+ }
+ if agent.portExist(device, port) {
+ logger.Debugw("port-already-exist", log.Fields{"port": port})
+ agent.requestQueue.RequestComplete()
+ return false, nil
+ }
+ agent.requestQueue.RequestComplete()
+
+ var portCap *ic.PortCapability
+ var err error
+ // First get the port capability
+ if portCap, err = agent.deviceMgr.getPortCapability(ctx, device.Id, port.PortNo); err != nil {
+ logger.Errorw("error-retrieving-port-capabilities", log.Fields{"error": err})
+ return false, err
+ }
+
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return false, err
+ }
+
+ defer agent.requestQueue.RequestComplete()
+ // Double check again if this port has been already added since the getPortCapability could have taken a long time
+ if agent.portExist(device, port) {
+ logger.Debugw("port-already-exist", log.Fields{"port": port})
+ return false, nil
+ }
+
+ portCap.Port.RootPort = true
+ lp := (proto.Clone(portCap.Port)).(*voltha.LogicalPort)
+ lp.DeviceId = device.Id
+ lp.Id = fmt.Sprintf("nni-%d", port.PortNo)
+ lp.OfpPort.PortNo = port.PortNo
+ lp.OfpPort.Name = lp.Id
+ lp.DevicePortNo = port.PortNo
+
+ ld := agent.getLogicalDeviceWithoutLock()
+
+ clonedPorts := clonePorts(ld.Ports)
+ if clonedPorts == nil {
+ clonedPorts = make([]*voltha.LogicalPort, 0)
+ }
+ clonedPorts = append(clonedPorts, lp)
+
+ if err = agent.updateLogicalDevicePortsWithoutLock(ctx, ld, clonedPorts); err != nil {
+ logger.Errorw("error-updating-logical-device", log.Fields{"error": err})
+ return false, err
+ }
+
+ // Update the device routes with this new logical port
+ clonedLP := (proto.Clone(lp)).(*voltha.LogicalPort)
+ go func() {
+ if err := agent.updateRoutes(context.Background(), clonedLP); err != nil {
+ logger.Warnw("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "logical-port": lp.OfpPort.PortNo, "error": err})
+ }
+ }()
+
+ return true, nil
+}
+
+func (agent *LogicalAgent) portExist(device *voltha.Device, port *voltha.Port) bool {
+ ldevice := agent.getLogicalDeviceWithoutLock()
+ for _, lPort := range ldevice.Ports {
+ if lPort.DeviceId == device.Id && lPort.DevicePortNo == port.PortNo && lPort.Id == port.Label {
+ return true
+ }
+ }
+ return false
+}
+
+// addUNILogicalPort adds an UNI port to the logical device. It returns a bool representing whether a port has been
+// added and an eror in case a valid error is encountered. If the port was successfully added it will return
+// (true, nil). If the device is not in the correct state it will return (false, nil) as this is a valid
+// scenario. This also applies to the case where the port was already added.
+func (agent *LogicalAgent) addUNILogicalPort(ctx context.Context, childDevice *voltha.Device, port *voltha.Port) (bool, error) {
+ logger.Debugw("addUNILogicalPort", log.Fields{"port": port})
+ if childDevice.AdminState != voltha.AdminState_ENABLED || childDevice.OperStatus != voltha.OperStatus_ACTIVE {
+ logger.Infow("device-not-ready", log.Fields{"deviceId": childDevice.Id, "admin": childDevice.AdminState, "oper": childDevice.OperStatus})
+ return false, nil
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return false, err
+ }
+
+ if agent.portExist(childDevice, port) {
+ logger.Debugw("port-already-exist", log.Fields{"port": port})
+ agent.requestQueue.RequestComplete()
+ return false, nil
+ }
+ agent.requestQueue.RequestComplete()
+ var portCap *ic.PortCapability
+ var err error
+ // First get the port capability
+ if portCap, err = agent.deviceMgr.getPortCapability(ctx, childDevice.Id, port.PortNo); err != nil {
+ logger.Errorw("error-retrieving-port-capabilities", log.Fields{"error": err})
+ return false, err
+ }
+ if err := agent.requestQueue.WaitForGreenLight(ctx); err != nil {
+ return false, err
+ }
+ defer agent.requestQueue.RequestComplete()
+ // Double check again if this port has been already added since the getPortCapability could have taken a long time
+ if agent.portExist(childDevice, port) {
+ logger.Debugw("port-already-exist", log.Fields{"port": port})
+ return false, nil
+ }
+ // Get stored logical device
+ ldevice := agent.getLogicalDeviceWithoutLock()
+
+ logger.Debugw("adding-uni", log.Fields{"deviceId": childDevice.Id})
+ portCap.Port.RootPort = false
+ portCap.Port.Id = port.Label
+ portCap.Port.OfpPort.PortNo = port.PortNo
+ portCap.Port.DeviceId = childDevice.Id
+ portCap.Port.DevicePortNo = port.PortNo
+ clonedPorts := clonePorts(ldevice.Ports)
+ if clonedPorts == nil {
+ clonedPorts = make([]*voltha.LogicalPort, 0)
+ }
+ clonedPorts = append(clonedPorts, portCap.Port)
+ if err := agent.updateLogicalDevicePortsWithoutLock(ctx, ldevice, clonedPorts); err != nil {
+ return false, err
+ }
+ // Update the device graph with this new logical port
+ clonedLP := (proto.Clone(portCap.Port)).(*voltha.LogicalPort)
+
+ go func() {
+ if err := agent.updateRoutes(context.Background(), clonedLP); err != nil {
+ logger.Warn("routes-not-ready", log.Fields{"logical-device-id": agent.logicalDeviceID, "error": err})
+ }
+ }()
+
+ return true, nil
+}
+
+func (agent *LogicalAgent) packetOut(ctx context.Context, packet *ofp.OfpPacketOut) {
+ logger.Debugw("packet-out", log.Fields{
+ "packet": hex.EncodeToString(packet.Data),
+ "inPort": packet.GetInPort(),
+ })
+ outPort := fu.GetPacketOutPort(packet)
+ //frame := packet.GetData()
+ //TODO: Use a channel between the logical agent and the device agent
+ if err := agent.deviceMgr.packetOut(ctx, agent.rootDeviceID, outPort, packet); err != nil {
+ logger.Error("packetout-failed", log.Fields{"logicalDeviceID": agent.rootDeviceID})
+ }
+}
+
+func (agent *LogicalAgent) packetIn(port uint32, transactionID string, packet []byte) {
+ logger.Debugw("packet-in", log.Fields{
+ "port": port,
+ "packet": hex.EncodeToString(packet),
+ "transactionId": transactionID,
+ })
+ packetIn := fu.MkPacketIn(port, packet)
+ agent.ldeviceMgr.eventCallbacks.SendPacketIn(agent.logicalDeviceID, transactionID, packetIn)
+ logger.Debugw("sending-packet-in", log.Fields{"packet": hex.EncodeToString(packetIn.Data)})
+}
+
+func (agent *LogicalAgent) addLogicalPortToMap(portNo uint32, nniPort bool) {
+ agent.lockLogicalPortsNo.Lock()
+ defer agent.lockLogicalPortsNo.Unlock()
+ if exist := agent.logicalPortsNo[portNo]; !exist {
+ agent.logicalPortsNo[portNo] = nniPort
+ }
+}
+
+func (agent *LogicalAgent) deleteLogicalPortsFromMap(portsNo []uint32) {
+ agent.lockLogicalPortsNo.Lock()
+ defer agent.lockLogicalPortsNo.Unlock()
+ for _, pNo := range portsNo {
+ delete(agent.logicalPortsNo, pNo)
+ }
+}
+
+func (agent *LogicalAgent) addLogicalPortsToMap(lps []*voltha.LogicalPort) {
+ agent.lockLogicalPortsNo.Lock()
+ defer agent.lockLogicalPortsNo.Unlock()
+ for _, lp := range lps {
+ if exist := agent.logicalPortsNo[lp.DevicePortNo]; !exist {
+ agent.logicalPortsNo[lp.DevicePortNo] = lp.RootPort
+ }
+ }
+}
+
+func (agent *LogicalAgent) isNNIPort(portNo uint32) bool {
+ agent.lockLogicalPortsNo.RLock()
+ defer agent.lockLogicalPortsNo.RUnlock()
+ if exist := agent.logicalPortsNo[portNo]; exist {
+ return agent.logicalPortsNo[portNo]
+ }
+ return false
+}
+
+func (agent *LogicalAgent) getFirstNNIPort() (uint32, error) {
+ agent.lockLogicalPortsNo.RLock()
+ defer agent.lockLogicalPortsNo.RUnlock()
+ for portNo, nni := range agent.logicalPortsNo {
+ if nni {
+ return portNo, nil
+ }
+ }
+ return 0, status.Error(codes.NotFound, "No NNI port found")
+}
+
+//GetNNIPorts returns NNI ports.
+func (agent *LogicalAgent) GetNNIPorts() []uint32 {
+ agent.lockLogicalPortsNo.RLock()
+ defer agent.lockLogicalPortsNo.RUnlock()
+ nniPorts := make([]uint32, 0)
+ for portNo, nni := range agent.logicalPortsNo {
+ if nni {
+ nniPorts = append(nniPorts, portNo)
+ }
+ }
+ return nniPorts
+}
diff --git a/rw_core/core/device/logical_agent_test.go b/rw_core/core/device/logical_agent_test.go
new file mode 100644
index 0000000..3c3b2b0
--- /dev/null
+++ b/rw_core/core/device/logical_agent_test.go
@@ -0,0 +1,616 @@
+/*
+ * 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"
+ "github.com/opencord/voltha-go/db/model"
+ "github.com/opencord/voltha-go/rw_core/core/adapter"
+ "github.com/opencord/voltha-lib-go/v3/pkg/db"
+ "math/rand"
+ "sync"
+ "testing"
+ "time"
+
+ "github.com/gogo/protobuf/proto"
+ "github.com/opencord/voltha-go/rw_core/config"
+ com "github.com/opencord/voltha-lib-go/v3/pkg/adapters/common"
+ fu "github.com/opencord/voltha-lib-go/v3/pkg/flows"
+ "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+ mock_etcd "github.com/opencord/voltha-lib-go/v3/pkg/mocks/etcd"
+ mock_kafka "github.com/opencord/voltha-lib-go/v3/pkg/mocks/kafka"
+ ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "github.com/phayes/freeport"
+ "github.com/stretchr/testify/assert"
+)
+
+func TestLogicalDeviceAgent_diff_nochange_1(t *testing.T) {
+ currentLogicalPorts := []*voltha.LogicalPort{}
+ updatedLogicalPorts := []*voltha.LogicalPort{}
+ newPorts, changedPorts, deletedPorts := diff(currentLogicalPorts, updatedLogicalPorts)
+ assert.Equal(t, 0, len(newPorts))
+ assert.Equal(t, 0, len(changedPorts))
+ assert.Equal(t, 0, len(deletedPorts))
+}
+
+func TestLogicalDeviceAgent_diff_nochange_2(t *testing.T) {
+ currentLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1232",
+ DeviceId: "d1234",
+ DevicePortNo: 2,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1233",
+ DeviceId: "d1234",
+ DevicePortNo: 3,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 3,
+ Name: "port3",
+ Config: 1,
+ State: 1,
+ },
+ },
+ }
+ updatedLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1232",
+ DeviceId: "d1234",
+ DevicePortNo: 2,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1233",
+ DeviceId: "d1234",
+ DevicePortNo: 3,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 3,
+ Name: "port3",
+ Config: 1,
+ State: 1,
+ },
+ },
+ }
+ newPorts, changedPorts, deletedPorts := diff(currentLogicalPorts, updatedLogicalPorts)
+ assert.Equal(t, 0, len(newPorts))
+ assert.Equal(t, 0, len(changedPorts))
+ assert.Equal(t, 0, len(deletedPorts))
+}
+
+func TestLogicalDeviceAgent_diff_add(t *testing.T) {
+ currentLogicalPorts := []*voltha.LogicalPort{}
+ updatedLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1232",
+ DeviceId: "d1234",
+ DevicePortNo: 2,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 1,
+ State: 1,
+ },
+ },
+ }
+ newPorts, changedPorts, deletedPorts := diff(currentLogicalPorts, updatedLogicalPorts)
+ assert.Equal(t, 2, len(newPorts))
+ assert.Equal(t, 0, len(changedPorts))
+ assert.Equal(t, 0, len(deletedPorts))
+ assert.Equal(t, updatedLogicalPorts[0], newPorts[0])
+ assert.Equal(t, updatedLogicalPorts[1], newPorts[1])
+}
+
+func TestLogicalDeviceAgent_diff_delete(t *testing.T) {
+ currentLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 1,
+ State: 1,
+ },
+ },
+ }
+ updatedLogicalPorts := []*voltha.LogicalPort{}
+ newPorts, changedPorts, deletedPorts := diff(currentLogicalPorts, updatedLogicalPorts)
+ assert.Equal(t, 0, len(newPorts))
+ assert.Equal(t, 0, len(changedPorts))
+ assert.Equal(t, 1, len(deletedPorts))
+ assert.Equal(t, currentLogicalPorts[0], deletedPorts[0])
+}
+
+func TestLogicalDeviceAgent_diff_changed(t *testing.T) {
+ currentLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1232",
+ DeviceId: "d1234",
+ DevicePortNo: 2,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1233",
+ DeviceId: "d1234",
+ DevicePortNo: 3,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 3,
+ Name: "port3",
+ Config: 1,
+ State: 1,
+ },
+ },
+ }
+ updatedLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 4,
+ State: 4,
+ },
+ },
+ {
+ Id: "1232",
+ DeviceId: "d1234",
+ DevicePortNo: 2,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 4,
+ State: 4,
+ },
+ },
+ {
+ Id: "1233",
+ DeviceId: "d1234",
+ DevicePortNo: 3,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 3,
+ Name: "port3",
+ Config: 1,
+ State: 1,
+ },
+ },
+ }
+ newPorts, changedPorts, deletedPorts := diff(currentLogicalPorts, updatedLogicalPorts)
+ assert.Equal(t, 0, len(newPorts))
+ assert.Equal(t, 2, len(changedPorts))
+ assert.Equal(t, 0, len(deletedPorts))
+ assert.Equal(t, updatedLogicalPorts[0], changedPorts[0])
+ assert.Equal(t, updatedLogicalPorts[1], changedPorts[1])
+}
+
+func TestLogicalDeviceAgent_diff_mix(t *testing.T) {
+ currentLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1232",
+ DeviceId: "d1234",
+ DevicePortNo: 2,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 1,
+ State: 1,
+ },
+ },
+ {
+ Id: "1233",
+ DeviceId: "d1234",
+ DevicePortNo: 3,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 3,
+ Name: "port3",
+ Config: 1,
+ State: 1,
+ },
+ },
+ }
+ updatedLogicalPorts := []*voltha.LogicalPort{
+ {
+ Id: "1231",
+ DeviceId: "d1234",
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 4,
+ State: 4,
+ },
+ },
+ {
+ Id: "1232",
+ DeviceId: "d1234",
+ DevicePortNo: 2,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 4,
+ State: 4,
+ },
+ },
+ {
+ Id: "1234",
+ DeviceId: "d1234",
+ DevicePortNo: 4,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 4,
+ Name: "port4",
+ Config: 4,
+ State: 4,
+ },
+ },
+ }
+ newPorts, changedPorts, deletedPorts := diff(currentLogicalPorts, updatedLogicalPorts)
+ assert.Equal(t, 1, len(newPorts))
+ assert.Equal(t, 2, len(changedPorts))
+ assert.Equal(t, 1, len(deletedPorts))
+ assert.Equal(t, updatedLogicalPorts[0], changedPorts[0])
+ assert.Equal(t, updatedLogicalPorts[1], changedPorts[1])
+ assert.Equal(t, currentLogicalPorts[2], deletedPorts[0])
+}
+
+type LDATest struct {
+ etcdServer *mock_etcd.EtcdServer
+ deviceMgr *Manager
+ kmp kafka.InterContainerProxy
+ logicalDeviceMgr *LogicalManager
+ kClient kafka.Client
+ kvClientPort int
+ oltAdapterName string
+ onuAdapterName string
+ coreInstanceID string
+ defaultTimeout time.Duration
+ maxTimeout time.Duration
+ logicalDevice *voltha.LogicalDevice
+ deviceIds []string
+ done chan int
+}
+
+func newLDATest() *LDATest {
+ test := &LDATest{}
+ // Start the embedded etcd server
+ var err error
+ test.etcdServer, test.kvClientPort, err = startEmbeddedEtcdServer("voltha.rwcore.lda.test", "voltha.rwcore.lda.etcd", "error")
+ if err != nil {
+ logger.Fatal(err)
+ }
+ // Create the kafka client
+ test.kClient = mock_kafka.NewKafkaClient()
+ test.oltAdapterName = "olt_adapter_mock"
+ test.onuAdapterName = "onu_adapter_mock"
+ test.coreInstanceID = "rw-da-test"
+ test.defaultTimeout = 5 * time.Second
+ test.maxTimeout = 20 * time.Second
+ test.done = make(chan int)
+ test.deviceIds = []string{com.GetRandomString(10), com.GetRandomString(10), com.GetRandomString(10)}
+ test.logicalDevice = &voltha.LogicalDevice{
+ Desc: &ofp.OfpDesc{
+ HwDesc: "olt_adapter_mock",
+ SwDesc: "olt_adapter_mock",
+ SerialNum: com.GetRandomSerialNumber(),
+ },
+ SwitchFeatures: &ofp.OfpSwitchFeatures{
+ NBuffers: 256,
+ NTables: 2,
+ Capabilities: uint32(ofp.OfpCapabilities_OFPC_FLOW_STATS |
+ ofp.OfpCapabilities_OFPC_TABLE_STATS |
+ ofp.OfpCapabilities_OFPC_PORT_STATS |
+ ofp.OfpCapabilities_OFPC_GROUP_STATS),
+ },
+ RootDeviceId: test.deviceIds[0],
+ Ports: []*voltha.LogicalPort{
+ {
+ Id: "1001",
+ DeviceId: test.deviceIds[0],
+ DevicePortNo: 1,
+ RootPort: true,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 1,
+ Name: "port1",
+ Config: 4,
+ State: 4,
+ },
+ },
+ {
+ Id: "1002",
+ DeviceId: test.deviceIds[1],
+ DevicePortNo: 2,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 2,
+ Name: "port2",
+ Config: 4,
+ State: 4,
+ },
+ },
+ {
+ Id: "1003",
+ DeviceId: test.deviceIds[2],
+ DevicePortNo: 3,
+ RootPort: false,
+ OfpPort: &ofp.OfpPort{
+ PortNo: 4,
+ Name: "port3",
+ Config: 4,
+ State: 4,
+ },
+ },
+ },
+ }
+ return test
+}
+
+func (lda *LDATest) startCore(inCompeteMode bool) {
+ cfg := config.NewRWCoreFlags()
+ cfg.CorePairTopic = "rw_core"
+ cfg.DefaultRequestTimeout = lda.defaultTimeout
+ cfg.KVStorePort = lda.kvClientPort
+ cfg.InCompetingMode = inCompeteMode
+ grpcPort, err := freeport.GetFreePort()
+ if err != nil {
+ logger.Fatal("Cannot get a freeport for grpc")
+ }
+ cfg.GrpcPort = grpcPort
+ cfg.GrpcHost = "127.0.0.1"
+ client := setupKVClient(cfg, lda.coreInstanceID)
+ backend := &db.Backend{
+ Client: client,
+ StoreType: cfg.KVStoreType,
+ Host: cfg.KVStoreHost,
+ Port: cfg.KVStorePort,
+ Timeout: cfg.KVStoreTimeout,
+ LivenessChannelInterval: cfg.LiveProbeInterval / 2,
+ PathPrefix: cfg.KVStoreDataPrefix}
+ lda.kmp = kafka.NewInterContainerProxy(
+ kafka.InterContainerHost(cfg.KafkaAdapterHost),
+ kafka.InterContainerPort(cfg.KafkaAdapterPort),
+ kafka.MsgClient(lda.kClient),
+ kafka.DefaultTopic(&kafka.Topic{Name: cfg.CoreTopic}),
+ kafka.DeviceDiscoveryTopic(&kafka.Topic{Name: cfg.AffinityRouterTopic}))
+
+ endpointMgr := kafka.NewEndpointManager(backend)
+ proxy := model.NewProxy(backend, "/")
+ adapterMgr := adapter.NewAdapterManager(proxy, lda.coreInstanceID, lda.kClient)
+
+ lda.deviceMgr, lda.logicalDeviceMgr = NewDeviceManagers(proxy, adapterMgr, lda.kmp, endpointMgr, cfg.CorePairTopic, lda.coreInstanceID, cfg.DefaultCoreTimeout)
+ lda.logicalDeviceMgr.SetEventCallbacks(fakeEventCallbacks{})
+ if err = lda.kmp.Start(); err != nil {
+ logger.Fatal("Cannot start InterContainerProxy")
+ }
+ if err = adapterMgr.Start(context.Background()); err != nil {
+ logger.Fatal("Cannot start adapterMgr")
+ }
+ lda.deviceMgr.Start(context.Background())
+ lda.logicalDeviceMgr.Start(context.Background())
+}
+
+func (lda *LDATest) stopAll() {
+ if lda.kClient != nil {
+ lda.kClient.Stop()
+ }
+ if lda.logicalDeviceMgr != nil {
+ lda.logicalDeviceMgr.Stop(context.Background())
+ }
+ if lda.deviceMgr != nil {
+ lda.deviceMgr.Stop(context.Background())
+ }
+ if lda.kmp != nil {
+ lda.kmp.Stop()
+ }
+ if lda.etcdServer != nil {
+ stopEmbeddedEtcdServer(lda.etcdServer)
+ }
+}
+
+func (lda *LDATest) createLogicalDeviceAgent(t *testing.T) *LogicalAgent {
+ lDeviceMgr := lda.logicalDeviceMgr
+ deviceMgr := lda.deviceMgr
+ clonedLD := proto.Clone(lda.logicalDevice).(*voltha.LogicalDevice)
+ clonedLD.Id = com.GetRandomString(10)
+ clonedLD.DatapathId = rand.Uint64()
+ lDeviceAgent := newLogicalDeviceAgent(clonedLD.Id, clonedLD.Id, clonedLD.RootDeviceId, lDeviceMgr, deviceMgr, lDeviceMgr.clusterDataProxy, lDeviceMgr.defaultTimeout)
+ lDeviceAgent.logicalDevice = clonedLD
+ err := lDeviceAgent.clusterDataProxy.AddWithID(context.Background(), "logical_devices", clonedLD.Id, clonedLD)
+ assert.Nil(t, err)
+ lDeviceMgr.addLogicalDeviceAgentToMap(lDeviceAgent)
+ return lDeviceAgent
+}
+
+func (lda *LDATest) updateLogicalDeviceConcurrently(t *testing.T, ldAgent *LogicalAgent, globalWG *sync.WaitGroup) {
+ originalLogicalDevice, _ := ldAgent.GetLogicalDevice(context.Background())
+ assert.NotNil(t, originalLogicalDevice)
+ var localWG sync.WaitGroup
+
+ // Change the state of the first port to FAILED
+ localWG.Add(1)
+ go func() {
+ err := ldAgent.updatePortState(context.Background(), lda.logicalDevice.Ports[0].DeviceId, lda.logicalDevice.Ports[0].DevicePortNo, voltha.OperStatus_FAILED)
+ assert.Nil(t, err)
+ localWG.Done()
+ }()
+
+ // Change the state of the second port to TESTING
+ localWG.Add(1)
+ go func() {
+ err := ldAgent.updatePortState(context.Background(), lda.logicalDevice.Ports[1].DeviceId, lda.logicalDevice.Ports[1].DevicePortNo, voltha.OperStatus_TESTING)
+ assert.Nil(t, err)
+ localWG.Done()
+ }()
+
+ // Change the state of the third port to UNKNOWN and then back to ACTIVE
+ localWG.Add(1)
+ go func() {
+ err := ldAgent.updatePortState(context.Background(), lda.logicalDevice.Ports[2].DeviceId, lda.logicalDevice.Ports[2].DevicePortNo, voltha.OperStatus_UNKNOWN)
+ assert.Nil(t, err)
+ err = ldAgent.updatePortState(context.Background(), lda.logicalDevice.Ports[2].DeviceId, lda.logicalDevice.Ports[2].DevicePortNo, voltha.OperStatus_ACTIVE)
+ assert.Nil(t, err)
+ localWG.Done()
+ }()
+
+ // Add a meter to the logical device
+ meterMod := &ofp.OfpMeterMod{
+ Command: ofp.OfpMeterModCommand_OFPMC_ADD,
+ Flags: rand.Uint32(),
+ MeterId: rand.Uint32(),
+ Bands: []*ofp.OfpMeterBandHeader{
+ {Type: ofp.OfpMeterBandType_OFPMBT_EXPERIMENTER,
+ Rate: rand.Uint32(),
+ BurstSize: rand.Uint32(),
+ Data: nil,
+ },
+ },
+ }
+ localWG.Add(1)
+ go func() {
+ err := ldAgent.meterAdd(context.Background(), meterMod)
+ assert.Nil(t, err)
+ localWG.Done()
+ }()
+
+ // wait for go routines to be done
+ localWG.Wait()
+
+ expectedChange := proto.Clone(originalLogicalDevice).(*voltha.LogicalDevice)
+ expectedChange.Ports[0].OfpPort.Config = originalLogicalDevice.Ports[0].OfpPort.Config | uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ expectedChange.Ports[0].OfpPort.State = uint32(ofp.OfpPortState_OFPPS_LINK_DOWN)
+ expectedChange.Ports[1].OfpPort.Config = originalLogicalDevice.Ports[0].OfpPort.Config | uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ expectedChange.Ports[1].OfpPort.State = uint32(ofp.OfpPortState_OFPPS_LINK_DOWN)
+ expectedChange.Ports[2].OfpPort.Config = originalLogicalDevice.Ports[0].OfpPort.Config & ^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN)
+ expectedChange.Ports[2].OfpPort.State = uint32(ofp.OfpPortState_OFPPS_LIVE)
+ expectedChange.Meters = &voltha.Meters{Items: nil}
+ expectedChange.Meters.Items = append(expectedChange.Meters.Items, fu.MeterEntryFromMeterMod(meterMod))
+ updatedLogicalDevice, _ := ldAgent.GetLogicalDevice(context.Background())
+ assert.NotNil(t, updatedLogicalDevice)
+ assert.True(t, proto.Equal(expectedChange, updatedLogicalDevice))
+ globalWG.Done()
+}
+
+func TestConcurrentLogicalDeviceUpdate(t *testing.T) {
+ lda := newLDATest()
+ assert.NotNil(t, lda)
+ defer lda.stopAll()
+
+ // Start the Core
+ lda.startCore(false)
+
+ var wg sync.WaitGroup
+ numConCurrentLogicalDeviceAgents := 3
+ for i := 0; i < numConCurrentLogicalDeviceAgents; i++ {
+ wg.Add(1)
+ a := lda.createLogicalDeviceAgent(t)
+ go lda.updateLogicalDeviceConcurrently(t, a, &wg)
+ }
+
+ wg.Wait()
+}
diff --git a/rw_core/core/device/logical_manager.go b/rw_core/core/device/logical_manager.go
new file mode 100644
index 0000000..5005e0c
--- /dev/null
+++ b/rw_core/core/device/logical_manager.go
@@ -0,0 +1,612 @@
+/*
+ * 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 device
+
+import (
+ "context"
+ "errors"
+ "github.com/opencord/voltha-go/rw_core/utils"
+ "strings"
+ "sync"
+ "time"
+
+ "github.com/opencord/voltha-go/db/model"
+ "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+ "github.com/opencord/voltha-lib-go/v3/pkg/probe"
+ "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+)
+
+// LogicalManager represent logical device manager attributes
+type LogicalManager struct {
+ logicalDeviceAgents sync.Map
+ deviceMgr *Manager
+ eventCallbacks EventCallbacks
+ kafkaICProxy kafka.InterContainerProxy
+ clusterDataProxy *model.Proxy
+ exitChannel chan int
+ defaultTimeout time.Duration
+ logicalDevicesLoadingLock sync.RWMutex
+ logicalDeviceLoadingInProgress map[string][]chan int
+}
+
+type EventCallbacks interface {
+ SendChangeEvent(deviceID string, portStatus *openflow_13.OfpPortStatus)
+ SendPacketIn(deviceID string, transactionID string, packet *openflow_13.OfpPacketIn)
+}
+
+func (ldMgr *LogicalManager) SetEventCallbacks(callbacks EventCallbacks) {
+ ldMgr.eventCallbacks = callbacks
+}
+
+func (ldMgr *LogicalManager) Start(ctx context.Context) {
+ logger.Info("starting-logical-device-manager")
+ probe.UpdateStatusFromContext(ctx, "logical-device-manager", probe.ServiceStatusRunning)
+ logger.Info("logical-device-manager-started")
+}
+
+func (ldMgr *LogicalManager) Stop(ctx context.Context) {
+ logger.Info("stopping-logical-device-manager")
+ ldMgr.exitChannel <- 1
+ probe.UpdateStatusFromContext(ctx, "logical-device-manager", probe.ServiceStatusStopped)
+ logger.Info("logical-device-manager-stopped")
+}
+
+func sendAPIResponse(ctx context.Context, ch chan interface{}, result interface{}) {
+ if ctx.Err() == nil {
+ // Returned response only of the ctx has not been cancelled/timeout/etc
+ // Channel is automatically closed when a context is Done
+ ch <- result
+ logger.Debugw("sendResponse", log.Fields{"result": result})
+ } else {
+ // Should the transaction be reverted back?
+ logger.Debugw("sendResponse-context-error", log.Fields{"context-error": ctx.Err()})
+ }
+}
+
+func (ldMgr *LogicalManager) addLogicalDeviceAgentToMap(agent *LogicalAgent) {
+ if _, exist := ldMgr.logicalDeviceAgents.Load(agent.logicalDeviceID); !exist {
+ ldMgr.logicalDeviceAgents.Store(agent.logicalDeviceID, agent)
+ }
+}
+
+// getLogicalDeviceAgent returns the logical device agent. If the device is not in memory then the device will
+// be loaded from dB and a logical device agent created to managed it.
+func (ldMgr *LogicalManager) getLogicalDeviceAgent(ctx context.Context, logicalDeviceID string) *LogicalAgent {
+ logger.Debugw("get-logical-device-agent", log.Fields{"logical-device-id": logicalDeviceID})
+ agent, ok := ldMgr.logicalDeviceAgents.Load(logicalDeviceID)
+ if ok {
+ lda := agent.(*LogicalAgent)
+ if lda.logicalDevice == nil {
+ // This can happen when an agent for the logical device has been created but the logical device
+ // itself is not ready for action as it is waiting for switch and port capabilities from the
+ // relevant adapter. In such a case prevent any request aimed at that logical device.
+ logger.Debugf("Logical device %s is not ready to serve requests", logicalDeviceID)
+ return nil
+ }
+ return lda
+ }
+ // Try to load into memory - loading will also create the logical device agent
+ if err := ldMgr.load(ctx, logicalDeviceID); err == nil {
+ if agent, ok = ldMgr.logicalDeviceAgents.Load(logicalDeviceID); ok {
+ return agent.(*LogicalAgent)
+ }
+ }
+ return nil
+}
+
+func (ldMgr *LogicalManager) deleteLogicalDeviceAgent(logicalDeviceID string) {
+ ldMgr.logicalDeviceAgents.Delete(logicalDeviceID)
+}
+
+// GetLogicalDevice provides a cloned most up to date logical device. If device is not in memory
+// it will be fetched from the dB
+func (ldMgr *LogicalManager) GetLogicalDevice(ctx context.Context, id string) (*voltha.LogicalDevice, error) {
+ logger.Debugw("getlogicalDevice", log.Fields{"logicaldeviceid": id})
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ return agent.GetLogicalDevice(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", id)
+}
+
+//ListLogicalDevices returns the list of all logical devices
+func (ldMgr *LogicalManager) ListLogicalDevices(ctx context.Context) (*voltha.LogicalDevices, error) {
+ logger.Debug("ListAllLogicalDevices")
+
+ var logicalDevices []*voltha.LogicalDevice
+ if err := ldMgr.clusterDataProxy.List(ctx, "logical_devices", &logicalDevices); err != nil {
+ logger.Errorw("failed-to-list-logical-devices-from-cluster-proxy", log.Fields{"error": err})
+ return nil, err
+ }
+ return &voltha.LogicalDevices{Items: logicalDevices}, nil
+}
+
+func (ldMgr *LogicalManager) createLogicalDevice(ctx context.Context, device *voltha.Device) (*string, error) {
+ logger.Debugw("creating-logical-device", log.Fields{"deviceId": device.Id})
+ // Sanity check
+ if !device.Root {
+ return nil, errors.New("device-not-root")
+ }
+
+ // Create a logical device agent - the logical device Id is based on the mac address of the device
+ // For now use the serial number - it may contain any combination of alphabetic characters and numbers,
+ // with length varying from eight characters to a maximum of 14 characters. Mac Address is part of oneof
+ // in the Device model. May need to be moved out.
+ id := utils.CreateLogicalDeviceID()
+ sn := strings.Replace(device.MacAddress, ":", "", -1)
+ if id == "" {
+ logger.Errorw("mac-address-not-set", log.Fields{"deviceId": device.Id, "serial-number": sn})
+ return nil, errors.New("mac-address-not-set")
+ }
+
+ logger.Debugw("logical-device-id", log.Fields{"logicaldeviceId": id})
+
+ agent := newLogicalDeviceAgent(id, sn, device.Id, ldMgr, ldMgr.deviceMgr, ldMgr.clusterDataProxy, ldMgr.defaultTimeout)
+ ldMgr.addLogicalDeviceAgentToMap(agent)
+
+ // Update the root device with the logical device Id reference
+ if err := ldMgr.deviceMgr.setParentID(ctx, device, id); err != nil {
+ logger.Errorw("failed-setting-parent-id", log.Fields{"logicalDeviceId": id, "deviceId": device.Id})
+ return nil, err
+ }
+
+ go func() {
+ //agent := newLogicalDeviceAgent(id, device.Id, ldMgr, ldMgr.deviceMgr, ldMgr.clusterDataProxy, ldMgr.defaultTimeout)
+ err := agent.start(context.Background(), false)
+ if err != nil {
+ logger.Errorw("unable-to-create-the-logical-device", log.Fields{"error": err})
+ ldMgr.deleteLogicalDeviceAgent(id)
+ }
+ }()
+
+ logger.Debug("creating-logical-device-ends")
+ return &id, nil
+}
+
+// stopManagingLogicalDeviceWithDeviceId stops the management of the logical device. This implies removal of any
+// reference of this logical device in cache. The device Id is passed as param because the logical device may already
+// have been removed from the model. This function returns the logical device Id if found
+func (ldMgr *LogicalManager) stopManagingLogicalDeviceWithDeviceID(ctx context.Context, id string) string {
+ logger.Infow("stop-managing-logical-device", log.Fields{"deviceId": id})
+ // Go over the list of logical device agents to find the one which has rootDeviceId as id
+ var ldID = ""
+ ldMgr.logicalDeviceAgents.Range(func(key, value interface{}) bool {
+ ldAgent := value.(*LogicalAgent)
+ if ldAgent.rootDeviceID == id {
+ logger.Infow("stopping-logical-device-agent", log.Fields{"lDeviceId": key})
+ if err := ldAgent.stop(ctx); err != nil {
+ logger.Errorw("failed-to-stop-LDAgent", log.Fields{"error": err})
+ return false
+ }
+ ldID = key.(string)
+ ldMgr.logicalDeviceAgents.Delete(ldID)
+ }
+ return true
+ })
+ return ldID
+}
+
+//getLogicalDeviceFromModel retrieves the logical device data from the model.
+func (ldMgr *LogicalManager) getLogicalDeviceFromModel(ctx context.Context, lDeviceID string) (*voltha.LogicalDevice, error) {
+ logicalDevice := &voltha.LogicalDevice{}
+ if have, err := ldMgr.clusterDataProxy.Get(ctx, "logical_devices/"+lDeviceID, logicalDevice); err != nil {
+ logger.Errorw("failed-to-get-logical-devices-from-cluster-proxy", log.Fields{"error": err})
+ return nil, err
+ } else if !have {
+ return nil, status.Error(codes.NotFound, lDeviceID)
+ }
+
+ return logicalDevice, nil
+}
+
+// load loads a logical device manager in memory
+func (ldMgr *LogicalManager) load(ctx context.Context, lDeviceID string) error {
+ if lDeviceID == "" {
+ return nil
+ }
+ // Add a lock to prevent two concurrent calls from loading the same device twice
+ ldMgr.logicalDevicesLoadingLock.Lock()
+ if _, exist := ldMgr.logicalDeviceLoadingInProgress[lDeviceID]; !exist {
+ if ldAgent, _ := ldMgr.logicalDeviceAgents.Load(lDeviceID); ldAgent == nil {
+ ldMgr.logicalDeviceLoadingInProgress[lDeviceID] = []chan int{make(chan int, 1)}
+ ldMgr.logicalDevicesLoadingLock.Unlock()
+ if _, err := ldMgr.getLogicalDeviceFromModel(ctx, lDeviceID); err == nil {
+ logger.Debugw("loading-logical-device", log.Fields{"lDeviceId": lDeviceID})
+ agent := newLogicalDeviceAgent(lDeviceID, "", "", ldMgr, ldMgr.deviceMgr, ldMgr.clusterDataProxy, ldMgr.defaultTimeout)
+ if err := agent.start(ctx, true); err != nil {
+ return err
+ }
+ ldMgr.logicalDeviceAgents.Store(agent.logicalDeviceID, agent)
+ } else {
+ logger.Debugw("logicalDevice not in model", log.Fields{"lDeviceId": lDeviceID})
+ }
+ // announce completion of task to any number of waiting channels
+ ldMgr.logicalDevicesLoadingLock.Lock()
+ if v, ok := ldMgr.logicalDeviceLoadingInProgress[lDeviceID]; ok {
+ for _, ch := range v {
+ close(ch)
+ }
+ delete(ldMgr.logicalDeviceLoadingInProgress, lDeviceID)
+ }
+ ldMgr.logicalDevicesLoadingLock.Unlock()
+ } else {
+ ldMgr.logicalDevicesLoadingLock.Unlock()
+ }
+ } else {
+ ch := make(chan int, 1)
+ ldMgr.logicalDeviceLoadingInProgress[lDeviceID] = append(ldMgr.logicalDeviceLoadingInProgress[lDeviceID], ch)
+ ldMgr.logicalDevicesLoadingLock.Unlock()
+ // Wait for the channel to be closed, implying the process loading this device is done.
+ <-ch
+ }
+ if _, exist := ldMgr.logicalDeviceAgents.Load(lDeviceID); exist {
+ return nil
+ }
+ return status.Errorf(codes.Aborted, "Error loading logical device %s", lDeviceID)
+}
+
+func (ldMgr *LogicalManager) deleteLogicalDevice(ctx context.Context, device *voltha.Device) error {
+ logger.Debugw("deleting-logical-device", log.Fields{"deviceId": device.Id})
+ // Sanity check
+ if !device.Root {
+ return errors.New("device-not-root")
+ }
+ logDeviceID := device.ParentId
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, logDeviceID); agent != nil {
+ // Stop the logical device agent
+ if err := agent.stop(ctx); err != nil {
+ logger.Errorw("failed-to-stop-agent", log.Fields{"error": err})
+ return err
+ }
+ //Remove the logical device agent from the Map
+ ldMgr.deleteLogicalDeviceAgent(logDeviceID)
+ }
+
+ logger.Debug("deleting-logical-device-ends")
+ return nil
+}
+
+func (ldMgr *LogicalManager) getLogicalDeviceID(ctx context.Context, device *voltha.Device) (*string, error) {
+ // Device can either be a parent or a child device
+ if device.Root {
+ // Parent device. The ID of a parent device is the logical device ID
+ return &device.ParentId, nil
+ }
+ // Device is child device
+ // retrieve parent device using child device ID
+ if parentDevice := ldMgr.deviceMgr.getParentDevice(ctx, device); parentDevice != nil {
+ return &parentDevice.ParentId, nil
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", device.Id)
+}
+
+func (ldMgr *LogicalManager) getLogicalDeviceIDFromDeviceID(ctx context.Context, deviceID string) (*string, error) {
+ // Get the device
+ var device *voltha.Device
+ var err error
+ if device, err = ldMgr.deviceMgr.GetDevice(ctx, deviceID); err != nil {
+ return nil, err
+ }
+ return ldMgr.getLogicalDeviceID(ctx, device)
+}
+
+func (ldMgr *LogicalManager) getLogicalPortID(ctx context.Context, device *voltha.Device) (*voltha.LogicalPortId, error) {
+ // Get the logical device where this device is attached
+ var lDeviceID *string
+ var err error
+ if lDeviceID, err = ldMgr.getLogicalDeviceID(ctx, device); err != nil {
+ return nil, err
+ }
+ var lDevice *voltha.LogicalDevice
+ if lDevice, err = ldMgr.GetLogicalDevice(ctx, *lDeviceID); err != nil {
+ return nil, err
+ }
+ // Go over list of ports
+ for _, port := range lDevice.Ports {
+ if port.DeviceId == device.Id {
+ return &voltha.LogicalPortId{Id: *lDeviceID, PortId: port.Id}, nil
+ }
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", device.Id)
+}
+
+// ListLogicalDeviceFlows returns the flows of logical device
+func (ldMgr *LogicalManager) ListLogicalDeviceFlows(ctx context.Context, id string) (*openflow_13.Flows, error) {
+ logger.Debugw("ListLogicalDeviceFlows", log.Fields{"logicaldeviceid": id})
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ return agent.ListLogicalDeviceFlows(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", id)
+}
+
+// ListLogicalDeviceFlowGroups returns logical device flow groups
+func (ldMgr *LogicalManager) ListLogicalDeviceFlowGroups(ctx context.Context, id string) (*openflow_13.FlowGroups, error) {
+ logger.Debugw("ListLogicalDeviceFlowGroups", log.Fields{"logicaldeviceid": id})
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ return agent.ListLogicalDeviceFlowGroups(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", id)
+}
+
+// ListLogicalDevicePorts returns logical device ports
+func (ldMgr *LogicalManager) ListLogicalDevicePorts(ctx context.Context, id string) (*voltha.LogicalPorts, error) {
+ logger.Debugw("ListLogicalDevicePorts", log.Fields{"logicaldeviceid": id})
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ return agent.ListLogicalDevicePorts(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", id)
+}
+
+func (ldMgr *LogicalManager) GetLogicalPort(ctx context.Context, lPortID *voltha.LogicalPortId) (*voltha.LogicalPort, error) {
+ // Get the logical device where this device is attached
+ var err error
+ var lDevice *voltha.LogicalDevice
+ if lDevice, err = ldMgr.GetLogicalDevice(ctx, lPortID.Id); err != nil {
+ return nil, err
+ }
+ // Go over list of ports
+ for _, port := range lDevice.Ports {
+ if port.Id == lPortID.PortId {
+ return port, nil
+ }
+ }
+ return nil, status.Errorf(codes.NotFound, "%s-%s", lPortID.Id, lPortID.PortId)
+}
+
+// updateLogicalPort sets up a logical port on the logical device based on the device port
+// information, if needed
+func (ldMgr *LogicalManager) updateLogicalPort(ctx context.Context, device *voltha.Device, port *voltha.Port) error {
+ ldID, err := ldMgr.getLogicalDeviceID(ctx, device)
+ if err != nil || *ldID == "" {
+ // This is not an error as the logical device may not have been created at this time. In such a case,
+ // the ports will be created when the logical device is ready.
+ return nil
+ }
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, *ldID); agent != nil {
+ if err := agent.updateLogicalPort(ctx, device, port); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+// deleteLogicalPort removes the logical port associated with a device
+func (ldMgr *LogicalManager) deleteLogicalPort(ctx context.Context, lPortID *voltha.LogicalPortId) error {
+ logger.Debugw("deleting-logical-port", log.Fields{"LDeviceId": lPortID.Id})
+ // Get logical port
+ var logicalPort *voltha.LogicalPort
+ var err error
+ if logicalPort, err = ldMgr.GetLogicalPort(ctx, lPortID); err != nil {
+ logger.Debugw("no-logical-device-port-present", log.Fields{"logicalPortId": lPortID.PortId})
+ return err
+ }
+ // Sanity check
+ if logicalPort.RootPort {
+ return errors.New("device-root")
+ }
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, lPortID.Id); agent != nil {
+ if err := agent.deleteLogicalPort(ctx, logicalPort); err != nil {
+ logger.Warnw("deleting-logicalport-failed", log.Fields{"LDeviceId": lPortID.Id, "error": err})
+ }
+ }
+
+ logger.Debug("deleting-logical-port-ends")
+ return nil
+}
+
+// deleteLogicalPort removes the logical port associated with a child device
+func (ldMgr *LogicalManager) deleteLogicalPorts(ctx context.Context, deviceID string) error {
+ logger.Debugw("deleting-logical-ports", log.Fields{"device-id": deviceID})
+ // Get logical port
+ ldID, err := ldMgr.getLogicalDeviceIDFromDeviceID(ctx, deviceID)
+ if err != nil {
+ return err
+ }
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, *ldID); agent != nil {
+ if err = agent.deleteLogicalPorts(ctx, deviceID); err != nil {
+ logger.Warnw("delete-logical-ports-failed", log.Fields{"logical-device-id": *ldID})
+ return err
+ }
+ }
+ logger.Debug("deleting-logical-ports-ends")
+ return nil
+}
+
+func (ldMgr *LogicalManager) setupUNILogicalPorts(ctx context.Context, childDevice *voltha.Device) error {
+ logger.Debugw("setupUNILogicalPorts", log.Fields{"childDeviceId": childDevice.Id, "parentDeviceId": childDevice.ParentId, "current-data": childDevice})
+ // Sanity check
+ if childDevice.Root {
+ return errors.New("Device-root")
+ }
+
+ // Get the logical device id parent device
+ parentID := childDevice.ParentId
+ logDeviceID := ldMgr.deviceMgr.GetParentDeviceID(ctx, parentID)
+
+ logger.Debugw("setupUNILogicalPorts", log.Fields{"logDeviceId": logDeviceID, "parentId": parentID})
+
+ if parentID == "" || logDeviceID == "" {
+ return errors.New("device-in-invalid-state")
+ }
+
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, logDeviceID); agent != nil {
+ if err := agent.setupUNILogicalPorts(ctx, childDevice); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+func (ldMgr *LogicalManager) deleteAllLogicalPorts(ctx context.Context, device *voltha.Device) error {
+ logger.Debugw("deleteAllLogicalPorts", log.Fields{"deviceId": device.Id})
+
+ var ldID *string
+ var err error
+ //Get the logical device Id for this device
+ if ldID, err = ldMgr.getLogicalDeviceID(ctx, device); err != nil {
+ logger.Warnw("no-logical-device-found", log.Fields{"deviceId": device.Id, "error": err})
+ return err
+ }
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, *ldID); agent != nil {
+ if err := agent.deleteAllLogicalPorts(ctx); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+func (ldMgr *LogicalManager) deleteAllUNILogicalPorts(ctx context.Context, parentDevice *voltha.Device) error {
+ logger.Debugw("delete-all-uni-logical-ports", log.Fields{"parent-device-id": parentDevice.Id})
+
+ var ldID *string
+ var err error
+ //Get the logical device Id for this device
+ if ldID, err = ldMgr.getLogicalDeviceID(ctx, parentDevice); err != nil {
+ return err
+ }
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, *ldID); agent != nil {
+ if err := agent.deleteAllUNILogicalPorts(ctx, parentDevice); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+func (ldMgr *LogicalManager) updatePortState(ctx context.Context, deviceID string, portNo uint32, state voltha.OperStatus_Types) error {
+ logger.Debugw("updatePortState", log.Fields{"deviceId": deviceID, "state": state, "portNo": portNo})
+
+ var ldID *string
+ var err error
+ //Get the logical device Id for this device
+ if ldID, err = ldMgr.getLogicalDeviceIDFromDeviceID(ctx, deviceID); err != nil {
+ logger.Warnw("no-logical-device-found", log.Fields{"deviceId": deviceID, "error": err})
+ return err
+ }
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, *ldID); agent != nil {
+ if err := agent.updatePortState(ctx, deviceID, portNo, state); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+func (ldMgr *LogicalManager) updatePortsState(ctx context.Context, device *voltha.Device, state voltha.OperStatus_Types) error {
+ logger.Debugw("updatePortsState", log.Fields{"deviceId": device.Id, "state": state, "current-data": device})
+
+ var ldID *string
+ var err error
+ //Get the logical device Id for this device
+ if ldID, err = ldMgr.getLogicalDeviceID(ctx, device); err != nil {
+ logger.Warnw("no-logical-device-found", log.Fields{"deviceId": device.Id, "error": err})
+ return err
+ }
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, *ldID); agent != nil {
+ if err := agent.updatePortsState(ctx, device, state); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+func (ldMgr *LogicalManager) UpdateFlowTable(ctx context.Context, id string, flow *openflow_13.OfpFlowMod, ch chan interface{}) {
+ logger.Debugw("UpdateFlowTable", log.Fields{"logicalDeviceId": id})
+ var res interface{}
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ res = agent.updateFlowTable(ctx, flow)
+ logger.Debugw("UpdateFlowTable-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id)
+ }
+ sendAPIResponse(ctx, ch, res)
+}
+
+func (ldMgr *LogicalManager) UpdateMeterTable(ctx context.Context, id string, meter *openflow_13.OfpMeterMod, ch chan interface{}) {
+ logger.Debugw("UpdateMeterTable", log.Fields{"logicalDeviceId": id})
+ var res interface{}
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ res = agent.updateMeterTable(ctx, meter)
+ logger.Debugw("UpdateMeterTable-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id)
+ }
+ sendAPIResponse(ctx, ch, res)
+}
+
+// ListLogicalDeviceMeters returns logical device meters
+func (ldMgr *LogicalManager) ListLogicalDeviceMeters(ctx context.Context, id string) (*openflow_13.Meters, error) {
+ logger.Debugw("ListLogicalDeviceMeters", log.Fields{"logicalDeviceId": id})
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ return agent.ListLogicalDeviceMeters(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", id)
+}
+func (ldMgr *LogicalManager) UpdateGroupTable(ctx context.Context, id string, groupMod *openflow_13.OfpGroupMod, ch chan interface{}) {
+ logger.Debugw("UpdateGroupTable", log.Fields{"logicalDeviceId": id})
+ var res interface{}
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id); agent != nil {
+ res = agent.updateGroupTable(ctx, groupMod)
+ logger.Debugw("UpdateGroupTable-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id)
+ }
+ sendAPIResponse(ctx, ch, res)
+}
+
+func (ldMgr *LogicalManager) EnableLogicalPort(ctx context.Context, id *voltha.LogicalPortId, ch chan interface{}) {
+ logger.Debugw("EnableLogicalPort", log.Fields{"logicalDeviceId": id})
+ var res interface{}
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id.Id); agent != nil {
+ res = agent.enableLogicalPort(ctx, id.PortId)
+ logger.Debugw("EnableLogicalPort-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id.Id)
+ }
+ sendAPIResponse(ctx, ch, res)
+}
+
+func (ldMgr *LogicalManager) DisableLogicalPort(ctx context.Context, id *voltha.LogicalPortId, ch chan interface{}) {
+ logger.Debugw("DisableLogicalPort", log.Fields{"logicalDeviceId": id})
+ var res interface{}
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, id.Id); agent != nil {
+ res = agent.disableLogicalPort(ctx, id.PortId)
+ logger.Debugw("DisableLogicalPort-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id.Id)
+ }
+ sendAPIResponse(ctx, ch, res)
+}
+
+func (ldMgr *LogicalManager) packetIn(ctx context.Context, logicalDeviceID string, port uint32, transactionID string, packet []byte) error {
+ logger.Debugw("packetIn", log.Fields{"logicalDeviceId": logicalDeviceID, "port": port})
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, logicalDeviceID); agent != nil {
+ agent.packetIn(port, transactionID, packet)
+ } else {
+ logger.Error("logical-device-not-exist", log.Fields{"logicalDeviceId": logicalDeviceID})
+ }
+ return nil
+}
+
+func (ldMgr *LogicalManager) PacketOut(ctx context.Context, packet *openflow_13.PacketOut) {
+ if agent := ldMgr.getLogicalDeviceAgent(ctx, packet.Id); agent != nil {
+ agent.packetOut(ctx, packet.PacketOut)
+ } else {
+ logger.Errorf("No logical device agent present", log.Fields{"logicalDeviceID": packet.Id})
+ }
+}
diff --git a/rw_core/core/device/manager.go b/rw_core/core/device/manager.go
new file mode 100755
index 0000000..48edc5b
--- /dev/null
+++ b/rw_core/core/device/manager.go
@@ -0,0 +1,1553 @@
+/*
+ * 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 device
+
+import (
+ "context"
+ "errors"
+ "reflect"
+ "runtime"
+ "sync"
+ "time"
+
+ "github.com/opencord/voltha-go/db/model"
+ "github.com/opencord/voltha-go/rw_core/core/adapter"
+ "github.com/opencord/voltha-go/rw_core/core/device/remote"
+ "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"
+ "github.com/opencord/voltha-lib-go/v3/pkg/probe"
+ ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+ ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+)
+
+// Manager represent device manager attributes
+type Manager struct {
+ deviceAgents sync.Map
+ rootDevices map[string]bool
+ lockRootDeviceMap sync.RWMutex
+ adapterProxy *remote.AdapterProxy
+ adapterMgr *adapter.Manager
+ logicalDeviceMgr *LogicalManager
+ kafkaICProxy kafka.InterContainerProxy
+ stateTransitions *TransitionMap
+ clusterDataProxy *model.Proxy
+ coreInstanceID string
+ exitChannel chan int
+ defaultTimeout time.Duration
+ devicesLoadingLock sync.RWMutex
+ deviceLoadingInProgress map[string][]chan int
+}
+
+func NewDeviceManagers(proxy *model.Proxy, adapterMgr *adapter.Manager, kmp kafka.InterContainerProxy, endpointMgr kafka.EndpointManager, corePairTopic, coreInstanceID string, defaultCoreTimeout time.Duration) (*Manager, *LogicalManager) {
+ deviceMgr := &Manager{
+ exitChannel: make(chan int, 1),
+ rootDevices: make(map[string]bool),
+ kafkaICProxy: kmp,
+ adapterProxy: remote.NewAdapterProxy(kmp, corePairTopic, endpointMgr),
+ coreInstanceID: coreInstanceID,
+ clusterDataProxy: proxy,
+ adapterMgr: adapterMgr,
+ defaultTimeout: defaultCoreTimeout * time.Millisecond,
+ deviceLoadingInProgress: make(map[string][]chan int),
+ }
+ logicalDeviceMgr := &LogicalManager{
+ exitChannel: make(chan int, 1),
+ deviceMgr: deviceMgr,
+ kafkaICProxy: kmp,
+ clusterDataProxy: proxy,
+ defaultTimeout: defaultCoreTimeout,
+ logicalDeviceLoadingInProgress: make(map[string][]chan int),
+ }
+ deviceMgr.logicalDeviceMgr = logicalDeviceMgr
+
+ adapterMgr.SetAdapterRestartedCallback(deviceMgr.adapterRestarted)
+
+ return deviceMgr, logicalDeviceMgr
+}
+
+func (dMgr *Manager) Start(ctx context.Context) {
+ logger.Info("starting-device-manager")
+ dMgr.stateTransitions = NewTransitionMap(dMgr)
+ probe.UpdateStatusFromContext(ctx, "device-manager", probe.ServiceStatusRunning)
+ logger.Info("device-manager-started")
+}
+
+func (dMgr *Manager) Stop(ctx context.Context) {
+ logger.Info("stopping-device-manager")
+ dMgr.exitChannel <- 1
+ probe.UpdateStatusFromContext(ctx, "device-manager", probe.ServiceStatusStopped)
+ logger.Info("device-manager-stopped")
+}
+
+func sendResponse(ctx context.Context, ch chan interface{}, result interface{}) {
+ if ctx.Err() == nil {
+ // Returned response only of the ctx has not been cancelled/timeout/etc
+ // Channel is automatically closed when a context is Done
+ ch <- result
+ logger.Debugw("sendResponse", log.Fields{"result": result})
+ } else {
+ // Should the transaction be reverted back?
+ logger.Debugw("sendResponse-context-error", log.Fields{"context-error": ctx.Err()})
+ }
+}
+
+func (dMgr *Manager) addDeviceAgentToMap(agent *Agent) {
+ if _, exist := dMgr.deviceAgents.Load(agent.deviceID); !exist {
+ dMgr.deviceAgents.Store(agent.deviceID, agent)
+ }
+ dMgr.lockRootDeviceMap.Lock()
+ defer dMgr.lockRootDeviceMap.Unlock()
+ dMgr.rootDevices[agent.deviceID] = agent.isRootdevice
+
+}
+
+func (dMgr *Manager) deleteDeviceAgentFromMap(agent *Agent) {
+ dMgr.deviceAgents.Delete(agent.deviceID)
+ dMgr.lockRootDeviceMap.Lock()
+ defer dMgr.lockRootDeviceMap.Unlock()
+ delete(dMgr.rootDevices, agent.deviceID)
+}
+
+// getDeviceAgent returns the agent managing the device. If the device is not in memory, it will loads it, if it exists
+func (dMgr *Manager) getDeviceAgent(ctx context.Context, deviceID string) *Agent {
+ agent, ok := dMgr.deviceAgents.Load(deviceID)
+ if ok {
+ return agent.(*Agent)
+ }
+ // Try to load into memory - loading will also create the device agent and set the device ownership
+ err := dMgr.load(ctx, deviceID)
+ if err == nil {
+ agent, ok = dMgr.deviceAgents.Load(deviceID)
+ if !ok {
+ return nil
+ }
+ return agent.(*Agent)
+ }
+ //TODO: Change the return params to return an error as well
+ logger.Errorw("loading-device-failed", log.Fields{"deviceId": deviceID, "error": err})
+ return nil
+}
+
+// listDeviceIdsFromMap returns the list of device IDs that are in memory
+func (dMgr *Manager) listDeviceIdsFromMap() *voltha.IDs {
+ result := &voltha.IDs{Items: make([]*voltha.ID, 0)}
+
+ dMgr.deviceAgents.Range(func(key, value interface{}) bool {
+ result.Items = append(result.Items, &voltha.ID{Id: key.(string)})
+ return true
+ })
+
+ return result
+}
+
+func (dMgr *Manager) CreateDevice(ctx context.Context, device *voltha.Device, ch chan interface{}) {
+ deviceExist, err := dMgr.isParentDeviceExist(ctx, device)
+ if err != nil {
+ logger.Errorf("Failed to fetch parent device info")
+ sendResponse(ctx, ch, err)
+ return
+ }
+ if deviceExist {
+ logger.Errorf("Device is Pre-provisioned already with same IP-Port or MAC Address")
+ sendResponse(ctx, ch, errors.New("Device is already pre-provisioned"))
+ return
+ }
+ logger.Debugw("CreateDevice", log.Fields{"device": device, "aproxy": dMgr.adapterProxy})
+
+ // Ensure this device is set as root
+ device.Root = true
+ // Create and start a device agent for that device
+ agent := newAgent(dMgr.adapterProxy, device, dMgr, dMgr.clusterDataProxy, dMgr.defaultTimeout)
+ device, err = agent.start(ctx, device)
+ if err != nil {
+ logger.Errorw("Fail-to-start-device", log.Fields{"device-id": agent.deviceID, "error": err})
+ sendResponse(ctx, ch, err)
+ return
+ }
+ dMgr.addDeviceAgentToMap(agent)
+
+ sendResponse(ctx, ch, device)
+}
+
+func (dMgr *Manager) EnableDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+ logger.Debugw("EnableDevice", log.Fields{"deviceid": id})
+ var res interface{}
+ if agent := dMgr.getDeviceAgent(ctx, id.Id); agent != nil {
+ res = agent.enableDevice(ctx)
+ logger.Debugw("EnableDevice-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) DisableDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+ logger.Debugw("DisableDevice", log.Fields{"deviceid": id})
+ var res interface{}
+ if agent := dMgr.getDeviceAgent(ctx, id.Id); agent != nil {
+ res = agent.disableDevice(ctx)
+ logger.Debugw("DisableDevice-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id.Id)
+ }
+
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) RebootDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+ logger.Debugw("RebootDevice", log.Fields{"deviceid": id})
+ var res interface{}
+ if agent := dMgr.getDeviceAgent(ctx, id.Id); agent != nil {
+ res = agent.rebootDevice(ctx)
+ logger.Debugw("RebootDevice-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) DeleteDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+ logger.Debugw("DeleteDevice", log.Fields{"deviceid": id})
+ var res interface{}
+ if agent := dMgr.getDeviceAgent(ctx, id.Id); agent != nil {
+ res = agent.deleteDevice(ctx)
+ logger.Debugw("DeleteDevice-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", id.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+// stopManagingDevice stops the management of the device as well as any of its reference device and logical device.
+// This function is called only in the Core that does not own this device. In the Core that owns this device then a
+// deletion deletion also includes removal of any reference of this device.
+func (dMgr *Manager) stopManagingDevice(ctx context.Context, id string) {
+ logger.Infow("stopManagingDevice", log.Fields{"deviceId": id})
+ if dMgr.IsDeviceInCache(id) { // Proceed only if an agent is present for this device
+ if root, _ := dMgr.IsRootDevice(id); root {
+ // stop managing the logical device
+ _ = dMgr.logicalDeviceMgr.stopManagingLogicalDeviceWithDeviceID(ctx, id)
+ }
+ if agent := dMgr.getDeviceAgent(ctx, id); agent != nil {
+ if err := agent.stop(ctx); err != nil {
+ logger.Warnw("unable-to-stop-device-agent", log.Fields{"device-id": agent.deviceID, "error": err})
+ }
+ dMgr.deleteDeviceAgentFromMap(agent)
+ }
+ }
+}
+
+// RunPostDeviceDelete removes any reference of this device
+func (dMgr *Manager) RunPostDeviceDelete(ctx context.Context, cDevice *voltha.Device) error {
+ logger.Infow("RunPostDeviceDelete", log.Fields{"deviceId": cDevice.Id})
+ dMgr.stopManagingDevice(ctx, cDevice.Id)
+ return nil
+}
+
+// GetDevice will returns a device, either from memory or from the dB, if present
+func (dMgr *Manager) GetDevice(ctx context.Context, id string) (*voltha.Device, error) {
+ logger.Debugw("GetDevice", log.Fields{"deviceid": id})
+ if agent := dMgr.getDeviceAgent(ctx, id); agent != nil {
+ return agent.getDevice(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", id)
+}
+
+// GetChildDevice will return a device, either from memory or from the dB, if present
+func (dMgr *Manager) GetChildDevice(ctx context.Context, parentDeviceID string, serialNumber string, onuID int64, parentPortNo int64) (*voltha.Device, error) {
+ logger.Debugw("GetChildDevice", log.Fields{"parentDeviceid": parentDeviceID, "serialNumber": serialNumber,
+ "parentPortNo": parentPortNo, "onuId": onuID})
+
+ var parentDevice *voltha.Device
+ var err error
+ if parentDevice, err = dMgr.GetDevice(ctx, parentDeviceID); err != nil {
+ return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+ }
+ var childDeviceIds []string
+ if childDeviceIds, err = dMgr.getAllChildDeviceIds(parentDevice); err != nil {
+ return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+ }
+ if len(childDeviceIds) == 0 {
+ logger.Debugw("no-child-devices", log.Fields{"parentDeviceId": parentDevice.Id, "serialNumber": serialNumber, "onuId": onuID})
+ return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
+ }
+
+ var foundChildDevice *voltha.Device
+ for _, childDeviceID := range childDeviceIds {
+ var found bool
+ if searchDevice, err := dMgr.GetDevice(ctx, childDeviceID); err == nil {
+
+ foundOnuID := false
+ if searchDevice.ProxyAddress.OnuId == uint32(onuID) {
+ if searchDevice.ParentPortNo == uint32(parentPortNo) {
+ logger.Debugw("found-child-by-onuid", log.Fields{"parentDeviceId": parentDevice.Id, "onuId": onuID})
+ foundOnuID = true
+ }
+ }
+
+ foundSerialNumber := false
+ if searchDevice.SerialNumber == serialNumber {
+ logger.Debugw("found-child-by-serialnumber", log.Fields{"parentDeviceId": parentDevice.Id, "serialNumber": serialNumber})
+ foundSerialNumber = true
+ }
+
+ // if both onuId and serialNumber are provided both must be true for the device to be found
+ // otherwise whichever one found a match is good enough
+ if onuID > 0 && serialNumber != "" {
+ found = foundOnuID && foundSerialNumber
+ } else {
+ found = foundOnuID || foundSerialNumber
+ }
+
+ if found {
+ foundChildDevice = searchDevice
+ break
+ }
+ }
+ }
+
+ if foundChildDevice != nil {
+ logger.Debugw("child-device-found", log.Fields{"parentDeviceId": parentDevice.Id, "foundChildDevice": foundChildDevice})
+ return foundChildDevice, nil
+ }
+
+ logger.Warnw("child-device-not-found", log.Fields{"parentDeviceId": parentDevice.Id,
+ "serialNumber": serialNumber, "onuId": onuID, "parentPortNo": parentPortNo})
+ return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
+}
+
+// GetChildDeviceWithProxyAddress will return a device based on proxy address
+func (dMgr *Manager) GetChildDeviceWithProxyAddress(ctx context.Context, proxyAddress *voltha.Device_ProxyAddress) (*voltha.Device, error) {
+ logger.Debugw("GetChildDeviceWithProxyAddress", log.Fields{"proxyAddress": proxyAddress})
+
+ var parentDevice *voltha.Device
+ var err error
+ if parentDevice, err = dMgr.GetDevice(ctx, proxyAddress.DeviceId); err != nil {
+ return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+ }
+ var childDeviceIds []string
+ if childDeviceIds, err = dMgr.getAllChildDeviceIds(parentDevice); err != nil {
+ return nil, status.Errorf(codes.Aborted, "%s", err.Error())
+ }
+ if len(childDeviceIds) == 0 {
+ logger.Debugw("no-child-devices", log.Fields{"parentDeviceId": parentDevice.Id})
+ return nil, status.Errorf(codes.NotFound, "%s", proxyAddress)
+ }
+
+ var foundChildDevice *voltha.Device
+ for _, childDeviceID := range childDeviceIds {
+ if searchDevice, err := dMgr.GetDevice(ctx, childDeviceID); err == nil {
+ if searchDevice.ProxyAddress == proxyAddress {
+ foundChildDevice = searchDevice
+ break
+ }
+ }
+ }
+
+ if foundChildDevice != nil {
+ logger.Debugw("child-device-found", log.Fields{"proxyAddress": proxyAddress})
+ return foundChildDevice, nil
+ }
+
+ logger.Warnw("child-device-not-found", log.Fields{"proxyAddress": proxyAddress})
+ return nil, status.Errorf(codes.NotFound, "%s", proxyAddress)
+}
+
+// IsDeviceInCache returns true if device is found in the map
+func (dMgr *Manager) IsDeviceInCache(id string) bool {
+ _, exist := dMgr.deviceAgents.Load(id)
+ 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) (*voltha.Devices, error) {
+ logger.Debug("ListDevices")
+ result := &voltha.Devices{}
+
+ var devices []*voltha.Device
+ if err := dMgr.clusterDataProxy.List(ctx, "devices", &devices); err != nil {
+ logger.Errorw("failed-to-list-devices-from-cluster-proxy", log.Fields{"error": err})
+ return nil, err
+ }
+
+ for _, device := range devices {
+ // If device is not in memory then set it up
+ if !dMgr.IsDeviceInCache(device.Id) {
+ logger.Debugw("loading-device-from-Model", log.Fields{"id": device.Id})
+ agent := newAgent(dMgr.adapterProxy, device, dMgr, dMgr.clusterDataProxy, dMgr.defaultTimeout)
+ if _, err := agent.start(ctx, nil); err != nil {
+ logger.Warnw("failure-starting-agent", log.Fields{"deviceId": device.Id})
+ } else {
+ dMgr.addDeviceAgentToMap(agent)
+ }
+ }
+ result.Items = append(result.Items, device)
+ }
+ logger.Debugw("ListDevices-end", log.Fields{"len": len(result.Items)})
+ return result, nil
+}
+
+//isParentDeviceExist checks whether device is already preprovisioned.
+func (dMgr *Manager) isParentDeviceExist(ctx context.Context, newDevice *voltha.Device) (bool, error) {
+ hostPort := newDevice.GetHostAndPort()
+ var devices []*voltha.Device
+ if err := dMgr.clusterDataProxy.List(ctx, "devices", &devices); err != nil {
+ logger.Errorw("Failed to list devices from cluster data proxy", log.Fields{"error": err})
+ return false, err
+ }
+ for _, device := range devices {
+ if !device.Root {
+ continue
+ }
+ if hostPort != "" && hostPort == device.GetHostAndPort() && device.AdminState != voltha.AdminState_DELETED {
+ return true, nil
+ }
+ if newDevice.MacAddress != "" && newDevice.MacAddress == device.MacAddress && device.AdminState != voltha.AdminState_DELETED {
+ return true, nil
+ }
+ }
+ return false, nil
+}
+
+//getDeviceFromModelretrieves the device data from the model.
+func (dMgr *Manager) getDeviceFromModel(ctx context.Context, deviceID string) (*voltha.Device, error) {
+ device := &voltha.Device{}
+ if have, err := dMgr.clusterDataProxy.Get(ctx, "devices/"+deviceID, device); err != nil {
+ logger.Errorw("failed-to-get-device-info-from-cluster-proxy", log.Fields{"error": err})
+ return nil, err
+ } else if !have {
+ return nil, status.Error(codes.NotFound, deviceID)
+ }
+
+ return device, nil
+}
+
+// loadDevice loads the deviceID in memory, if not present
+func (dMgr *Manager) loadDevice(ctx context.Context, deviceID string) (*Agent, error) {
+ if deviceID == "" {
+ return nil, status.Error(codes.InvalidArgument, "deviceId empty")
+ }
+ var err error
+ var device *voltha.Device
+ dMgr.devicesLoadingLock.Lock()
+ if _, exist := dMgr.deviceLoadingInProgress[deviceID]; !exist {
+ if !dMgr.IsDeviceInCache(deviceID) {
+ dMgr.deviceLoadingInProgress[deviceID] = []chan int{make(chan int, 1)}
+ dMgr.devicesLoadingLock.Unlock()
+ // Proceed with the loading only if the device exist in the Model (could have been deleted)
+ if device, err = dMgr.getDeviceFromModel(ctx, deviceID); err == nil {
+ logger.Debugw("loading-device", log.Fields{"deviceId": deviceID})
+ agent := newAgent(dMgr.adapterProxy, device, dMgr, dMgr.clusterDataProxy, dMgr.defaultTimeout)
+ if _, err = agent.start(ctx, nil); err != nil {
+ logger.Warnw("Failure loading device", log.Fields{"deviceId": deviceID, "error": err})
+ } else {
+ dMgr.addDeviceAgentToMap(agent)
+ }
+ } else {
+ logger.Debugw("Device not in model", log.Fields{"deviceId": deviceID})
+ }
+ // announce completion of task to any number of waiting channels
+ dMgr.devicesLoadingLock.Lock()
+ if v, ok := dMgr.deviceLoadingInProgress[deviceID]; ok {
+ for _, ch := range v {
+ close(ch)
+ }
+ delete(dMgr.deviceLoadingInProgress, deviceID)
+ }
+ dMgr.devicesLoadingLock.Unlock()
+ } else {
+ dMgr.devicesLoadingLock.Unlock()
+ }
+ } else {
+ ch := make(chan int, 1)
+ dMgr.deviceLoadingInProgress[deviceID] = append(dMgr.deviceLoadingInProgress[deviceID], ch)
+ dMgr.devicesLoadingLock.Unlock()
+ // Wait for the channel to be closed, implying the process loading this device is done.
+ <-ch
+ }
+ if agent, ok := dMgr.deviceAgents.Load(deviceID); ok {
+ return agent.(*Agent), nil
+ }
+ return nil, status.Errorf(codes.Aborted, "Error loading device %s", deviceID)
+}
+
+// loadRootDeviceParentAndChildren loads the children and parents of a root device in memory
+func (dMgr *Manager) loadRootDeviceParentAndChildren(ctx context.Context, device *voltha.Device) error {
+ logger.Debugw("loading-parent-and-children", log.Fields{"deviceId": device.Id})
+ if device.Root {
+ // Scenario A
+ if device.ParentId != "" {
+ // Load logical device if needed.
+ if err := dMgr.logicalDeviceMgr.load(ctx, device.ParentId); err != nil {
+ logger.Warnw("failure-loading-logical-device", log.Fields{"lDeviceId": device.ParentId})
+ }
+ } else {
+ logger.Debugw("no-parent-to-load", log.Fields{"deviceId": device.Id})
+ }
+ // Load all child devices, if needed
+ if childDeviceIds, err := dMgr.getAllChildDeviceIds(device); err == nil {
+ for _, childDeviceID := range childDeviceIds {
+ if _, err := dMgr.loadDevice(ctx, childDeviceID); err != nil {
+ logger.Warnw("failure-loading-device", log.Fields{"deviceId": childDeviceID, "error": err})
+ return err
+ }
+ }
+ logger.Debugw("loaded-children", log.Fields{"deviceId": device.Id, "numChildren": len(childDeviceIds)})
+ } else {
+ logger.Debugw("no-child-to-load", log.Fields{"deviceId": device.Id})
+ }
+ }
+ return nil
+}
+
+// load loads the deviceId in memory, if not present, and also loads its accompanying parents and children. Loading
+// in memory is for improved performance. It is not imperative that a device needs to be in memory when a request
+// acting on the device is received by the core. In such a scenario, the Core will load the device in memory first
+// and the proceed with the request.
+func (dMgr *Manager) load(ctx context.Context, deviceID string) error {
+ logger.Debug("load...")
+ // First load the device - this may fail in case the device was deleted intentionally by the other core
+ var dAgent *Agent
+ var err error
+ if dAgent, err = dMgr.loadDevice(ctx, deviceID); err != nil {
+ return err
+ }
+ // Get the loaded device details
+ device, err := dAgent.getDevice(ctx)
+ if err != nil {
+ return err
+ }
+
+ // If the device is in Pre-provisioning or deleted state stop here
+ if device.AdminState == voltha.AdminState_PREPROVISIONED || device.AdminState == voltha.AdminState_DELETED {
+ return nil
+ }
+
+ // Now we face two scenarios
+ if device.Root {
+ // Load all children as well as the parent of this device (logical_device)
+ if err := dMgr.loadRootDeviceParentAndChildren(ctx, device); err != nil {
+ logger.Warnw("failure-loading-device-parent-and-children", log.Fields{"deviceId": deviceID})
+ return err
+ }
+ logger.Debugw("successfully-loaded-parent-and-children", log.Fields{"deviceId": deviceID})
+ } else {
+ // Scenario B - use the parentId of that device (root device) to trigger the loading
+ if device.ParentId != "" {
+ return dMgr.load(ctx, device.ParentId)
+ }
+ }
+ return nil
+}
+
+// ListDeviceIds retrieves the latest device IDs information from the data model (memory data only)
+func (dMgr *Manager) ListDeviceIds() (*voltha.IDs, error) {
+ logger.Debug("ListDeviceIDs")
+ // Report only device IDs that are in the device agent map
+ return dMgr.listDeviceIdsFromMap(), nil
+}
+
+//ReconcileDevices is a request to a voltha core to update its list of managed devices. This will
+//trigger loading the devices along with their children and parent in memory
+func (dMgr *Manager) ReconcileDevices(ctx context.Context, ids *voltha.IDs, ch chan interface{}) {
+ logger.Debugw("ReconcileDevices", log.Fields{"numDevices": len(ids.Items)})
+ var res interface{}
+ if ids != nil && len(ids.Items) != 0 {
+ toReconcile := len(ids.Items)
+ reconciled := 0
+ var err error
+ for _, id := range ids.Items {
+ if err = dMgr.load(ctx, id.Id); err != nil {
+ logger.Warnw("failure-reconciling-device", log.Fields{"deviceId": id.Id, "error": err})
+ } else {
+ reconciled++
+ }
+ }
+ if toReconcile != reconciled {
+ res = status.Errorf(codes.DataLoss, "less-device-reconciled-than-requested:%d/%d", reconciled, toReconcile)
+ }
+ } else {
+ res = status.Errorf(codes.InvalidArgument, "empty-list-of-ids")
+ }
+ sendResponse(ctx, ch, res)
+}
+
+// isOkToReconcile validates whether a device is in the correct status to be reconciled
+func isOkToReconcile(device *voltha.Device) bool {
+ if device == nil {
+ return false
+ }
+ return device.AdminState != voltha.AdminState_PREPROVISIONED && device.AdminState != voltha.AdminState_DELETED
+}
+
+// adapterRestarted is invoked whenever an adapter is restarted
+func (dMgr *Manager) adapterRestarted(ctx context.Context, adapter *voltha.Adapter) error {
+ logger.Debugw("adapter-restarted", log.Fields{"adapterId": adapter.Id, "vendor": adapter.Vendor,
+ "currentReplica": adapter.CurrentReplica, "totalReplicas": adapter.TotalReplicas, "endpoint": adapter.Endpoint})
+
+ // Let's reconcile the device managed by this Core only
+ if len(dMgr.rootDevices) == 0 {
+ logger.Debugw("nothing-to-reconcile", log.Fields{"adapterId": adapter.Id})
+ return nil
+ }
+
+ responses := make([]utils.Response, 0)
+ for rootDeviceID := range dMgr.rootDevices {
+ if rootDevice, _ := dMgr.getDeviceFromModel(ctx, rootDeviceID); rootDevice != nil {
+ isDeviceOwnedByService, err := dMgr.adapterProxy.IsDeviceOwnedByService(rootDeviceID, adapter.Type, adapter.CurrentReplica)
+ if err != nil {
+ logger.Warnw("is-device-owned-by-service", log.Fields{"error": err, "root-device-id": rootDeviceID, "adapterType": adapter.Type, "replica-number": adapter.CurrentReplica})
+ continue
+ }
+ if isDeviceOwnedByService {
+ if isOkToReconcile(rootDevice) {
+ logger.Debugw("reconciling-root-device", log.Fields{"rootId": rootDevice.Id})
+ responses = append(responses, dMgr.sendReconcileDeviceRequest(ctx, rootDevice))
+ } else {
+ logger.Debugw("not-reconciling-root-device", log.Fields{"rootId": rootDevice.Id, "state": rootDevice.AdminState})
+ }
+ } else { // Should we be reconciling the root's children instead?
+ childManagedByAdapter:
+ for _, port := range rootDevice.Ports {
+ for _, peer := range port.Peers {
+ if childDevice, _ := dMgr.getDeviceFromModel(ctx, peer.DeviceId); childDevice != nil {
+ isDeviceOwnedByService, err := dMgr.adapterProxy.IsDeviceOwnedByService(childDevice.Id, adapter.Type, adapter.CurrentReplica)
+ if err != nil {
+ logger.Warnw("is-device-owned-by-service", log.Fields{"error": err, "child-device-id": childDevice.Id, "adapterType": adapter.Type, "replica-number": adapter.CurrentReplica})
+ }
+ if isDeviceOwnedByService {
+ if isOkToReconcile(childDevice) {
+ logger.Debugw("reconciling-child-device", log.Fields{"child-device-id": childDevice.Id})
+ responses = append(responses, dMgr.sendReconcileDeviceRequest(ctx, childDevice))
+ } else {
+ logger.Debugw("not-reconciling-child-device", log.Fields{"child-device-id": childDevice.Id, "state": childDevice.AdminState})
+ }
+ } else {
+ // All child devices under a parent device are typically managed by the same adapter type.
+ // Therefore we only need to check whether the first device we retrieved is managed by that adapter
+ break childManagedByAdapter
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ if len(responses) > 0 {
+ // Wait for completion
+ if res := utils.WaitForNilOrErrorResponses(dMgr.defaultTimeout, responses...); res != nil {
+ return status.Errorf(codes.Aborted, "errors-%s", res)
+ }
+ } else {
+ logger.Debugw("no-managed-device-to-reconcile", log.Fields{"adapterId": adapter.Id})
+ }
+ return nil
+}
+
+func (dMgr *Manager) sendReconcileDeviceRequest(ctx context.Context, device *voltha.Device) utils.Response {
+ // Send a reconcile request to the adapter. Since this Core may not be managing this device then there is no
+ // point of creating a device agent (if the device is not being managed by this Core) before sending the request
+ // to the adapter. We will therefore bypass the adapter adapter and send the request directly to the adapter via
+ // the adapter proxy.
+ response := utils.NewResponse()
+ ch, err := dMgr.adapterProxy.ReconcileDevice(ctx, device)
+ if err != nil {
+ response.Error(err)
+ }
+ // Wait for adapter response in its own routine
+ go func() {
+ resp, ok := <-ch
+ if !ok {
+ response.Error(status.Errorf(codes.Aborted, "channel-closed-device: %s", device.Id))
+ } else if resp.Err != nil {
+ response.Error(resp.Err)
+ }
+ response.Done()
+ }()
+ return response
+}
+
+func (dMgr *Manager) ReconcileChildDevices(ctx context.Context, parentDeviceID string) error {
+ if parentDevice, _ := dMgr.getDeviceFromModel(ctx, parentDeviceID); parentDevice != nil {
+ responses := make([]utils.Response, 0)
+ for _, port := range parentDevice.Ports {
+ for _, peer := range port.Peers {
+ if childDevice, _ := dMgr.getDeviceFromModel(ctx, peer.DeviceId); childDevice != nil {
+ responses = append(responses, dMgr.sendReconcileDeviceRequest(ctx, childDevice))
+ }
+ }
+ }
+ // Wait for completion
+ if res := utils.WaitForNilOrErrorResponses(dMgr.defaultTimeout, responses...); res != nil {
+ return status.Errorf(codes.Aborted, "errors-%s", res)
+ }
+ }
+ return nil
+}
+
+func (dMgr *Manager) UpdateDeviceUsingAdapterData(ctx context.Context, device *voltha.Device) error {
+ logger.Debugw("UpdateDeviceUsingAdapterData", log.Fields{"deviceid": device.Id, "device": device})
+ if agent := dMgr.getDeviceAgent(ctx, device.Id); agent != nil {
+ return agent.updateDeviceUsingAdapterData(ctx, device)
+ }
+ return status.Errorf(codes.NotFound, "%s", device.Id)
+}
+
+func (dMgr *Manager) AddPort(ctx context.Context, deviceID string, port *voltha.Port) error {
+ agent := dMgr.getDeviceAgent(ctx, deviceID)
+ if agent != nil {
+ if err := agent.addPort(ctx, port); err != nil {
+ return err
+ }
+ // Setup peer ports
+ meAsPeer := &voltha.Port_PeerPort{DeviceId: deviceID, PortNo: port.PortNo}
+ for _, peerPort := range port.Peers {
+ if agent := dMgr.getDeviceAgent(ctx, peerPort.DeviceId); agent != nil {
+ if err := agent.addPeerPort(ctx, meAsPeer); err != nil {
+ logger.Errorw("failed-to-add-peer", log.Fields{"peer-device-id": peerPort.DeviceId})
+ return err
+ }
+ }
+ }
+ // Notify the logical device manager to setup a logical port, if needed. If the added port is an NNI or UNI
+ // then a logical port will be added to the logical device and the device graph generated. If the port is a
+ // PON port then only the device graph will be generated.
+ if device, err := dMgr.GetDevice(ctx, deviceID); err == nil {
+ go func() {
+ err = dMgr.logicalDeviceMgr.updateLogicalPort(context.Background(), device, port)
+ if err != nil {
+ logger.Errorw("unable-to-update-logical-port", log.Fields{"error": err})
+ }
+ }()
+ } else {
+ logger.Errorw("failed-to-retrieve-device", log.Fields{"deviceId": deviceID})
+ return err
+ }
+ return nil
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) addFlowsAndGroups(ctx context.Context, deviceID string, flows []*ofp.OfpFlowStats, groups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) error {
+ logger.Debugw("addFlowsAndGroups", log.Fields{"deviceid": deviceID, "groups:": groups, "flowMetadata": flowMetadata})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.addFlowsAndGroups(ctx, flows, groups, flowMetadata)
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) deleteFlowsAndGroups(ctx context.Context, deviceID string, flows []*ofp.OfpFlowStats, groups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) error {
+ logger.Debugw("deleteFlowsAndGroups", log.Fields{"deviceid": deviceID})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.deleteFlowsAndGroups(ctx, flows, groups, flowMetadata)
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) updateFlowsAndGroups(ctx context.Context, deviceID string, flows []*ofp.OfpFlowStats, groups []*ofp.OfpGroupEntry, flowMetadata *voltha.FlowMetadata) error {
+ logger.Debugw("updateFlowsAndGroups", log.Fields{"deviceid": deviceID})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.updateFlowsAndGroups(ctx, flows, groups, flowMetadata)
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+// UpdatePmConfigs updates the PM configs. This is executed when the northbound gRPC API is invoked, typically
+// following a user action
+func (dMgr *Manager) UpdatePmConfigs(ctx context.Context, pmConfigs *voltha.PmConfigs, ch chan interface{}) {
+ var res interface{}
+ if pmConfigs.Id == "" {
+ res = status.Errorf(codes.FailedPrecondition, "invalid-device-Id")
+ } else if agent := dMgr.getDeviceAgent(ctx, pmConfigs.Id); agent != nil {
+ res = agent.updatePmConfigs(ctx, pmConfigs)
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", pmConfigs.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+// InitPmConfigs initialize the pm configs as defined by the adapter.
+func (dMgr *Manager) InitPmConfigs(ctx context.Context, deviceID string, pmConfigs *voltha.PmConfigs) error {
+ if pmConfigs.Id == "" {
+ return status.Errorf(codes.FailedPrecondition, "invalid-device-Id")
+ }
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.initPmConfigs(ctx, pmConfigs)
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) ListPmConfigs(ctx context.Context, deviceID string) (*voltha.PmConfigs, error) {
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.listPmConfigs(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) getSwitchCapability(ctx context.Context, deviceID string) (*ic.SwitchCapability, error) {
+ logger.Debugw("getSwitchCapability", log.Fields{"deviceid": deviceID})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.getSwitchCapability(ctx)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) GetPorts(ctx context.Context, deviceID string, portType voltha.Port_PortType) (*voltha.Ports, error) {
+ logger.Debugw("GetPorts", log.Fields{"deviceid": deviceID, "portType": portType})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.getPorts(ctx, portType), nil
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) getPortCapability(ctx context.Context, deviceID string, portNo uint32) (*ic.PortCapability, error) {
+ logger.Debugw("getPortCapability", log.Fields{"deviceid": deviceID})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.getPortCapability(ctx, portNo)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) UpdateDeviceStatus(ctx context.Context, deviceID string, operStatus voltha.OperStatus_Types, connStatus voltha.ConnectStatus_Types) error {
+ logger.Debugw("UpdateDeviceStatus", log.Fields{"deviceid": deviceID, "operStatus": operStatus, "connStatus": connStatus})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.updateDeviceStatus(ctx, operStatus, connStatus)
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) UpdateChildrenStatus(ctx context.Context, deviceID string, operStatus voltha.OperStatus_Types, connStatus voltha.ConnectStatus_Types) error {
+ logger.Debugw("UpdateChildrenStatus", log.Fields{"parentDeviceid": deviceID, "operStatus": operStatus, "connStatus": connStatus})
+ var parentDevice *voltha.Device
+ var err error
+ if parentDevice, err = dMgr.GetDevice(ctx, deviceID); err != nil {
+ return status.Errorf(codes.Aborted, "%s", err.Error())
+ }
+ var childDeviceIds []string
+ if childDeviceIds, err = dMgr.getAllChildDeviceIds(parentDevice); err != nil {
+ return status.Errorf(codes.Aborted, "%s", err.Error())
+ }
+ if len(childDeviceIds) == 0 {
+ logger.Debugw("no-child-device", log.Fields{"parentDeviceId": parentDevice.Id})
+ }
+ for _, childDeviceID := range childDeviceIds {
+ if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
+ if err = agent.updateDeviceStatus(ctx, operStatus, connStatus); err != nil {
+ return status.Errorf(codes.Aborted, "childDevice:%s, error:%s", childDeviceID, err.Error())
+ }
+ }
+ }
+ return nil
+}
+
+func (dMgr *Manager) UpdatePortState(ctx context.Context, deviceID string, portType voltha.Port_PortType, portNo uint32, operStatus voltha.OperStatus_Types) error {
+ logger.Debugw("UpdatePortState", log.Fields{"deviceid": deviceID, "portType": portType, "portNo": portNo, "operStatus": operStatus})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ if err := agent.updatePortState(ctx, portType, portNo, operStatus); err != nil {
+ logger.Errorw("updating-port-state-failed", log.Fields{"deviceid": deviceID, "portNo": portNo, "error": err})
+ return err
+ }
+ // Notify the logical device manager to change the port state
+ // Do this for NNI and UNIs only. PON ports are not known by logical device
+ if portType == voltha.Port_ETHERNET_NNI || portType == voltha.Port_ETHERNET_UNI {
+ go func() {
+ err := dMgr.logicalDeviceMgr.updatePortState(context.Background(), deviceID, portNo, operStatus)
+ if err != nil {
+ // While we want to handle (catch) and log when
+ // an update to a port was not able to be
+ // propagated to the logical port, we can report
+ // it as a warning and not an error because it
+ // doesn't stop or modify processing.
+ // TODO: VOL-2707
+ logger.Warnw("unable-to-update-logical-port-state", log.Fields{"error": err})
+ }
+ }()
+ }
+ return nil
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) DeleteAllPorts(ctx context.Context, deviceID string) error {
+ logger.Debugw("DeleteAllPorts", log.Fields{"deviceid": deviceID})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ if err := agent.deleteAllPorts(ctx); err != nil {
+ return err
+ }
+ // Notify the logical device manager to remove all logical ports, if needed.
+ // At this stage the device itself may gave been deleted already at a DeleteAllPorts
+ // typically is part of a device deletion phase.
+ if device, err := dMgr.GetDevice(ctx, deviceID); err == nil {
+ go func() {
+ err = dMgr.logicalDeviceMgr.deleteAllLogicalPorts(ctx, device)
+ if err != nil {
+ logger.Errorw("unable-to-delete-logical-ports", log.Fields{"error": err})
+ }
+ }()
+ } else {
+ logger.Warnw("failed-to-retrieve-device", log.Fields{"deviceId": deviceID})
+ return err
+ }
+ return nil
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+//UpdatePortsState updates all ports on the device
+func (dMgr *Manager) UpdatePortsState(ctx context.Context, deviceID string, state voltha.OperStatus_Types) error {
+ logger.Debugw("UpdatePortsState", log.Fields{"deviceid": deviceID})
+
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ switch state {
+ case voltha.OperStatus_ACTIVE:
+ if err := agent.updatePortsOperState(ctx, state); err != nil {
+ logger.Warnw("updatePortsOperState-failed", log.Fields{"deviceId": deviceID, "error": err})
+ return err
+ }
+ case voltha.OperStatus_UNKNOWN:
+ if err := agent.updatePortsOperState(ctx, state); err != nil {
+ logger.Warnw("updatePortsOperState-failed", log.Fields{"deviceId": deviceID, "error": err})
+ return err
+ }
+ default:
+ return status.Error(codes.Unimplemented, "state-change-not-implemented")
+ }
+ // Notify the logical device about the state change
+ device, err := dMgr.GetDevice(ctx, deviceID)
+ if err != nil {
+ logger.Warnw("non-existent-device", log.Fields{"deviceId": deviceID, "error": err})
+ return err
+ }
+ if err := dMgr.logicalDeviceMgr.updatePortsState(ctx, device, state); err != nil {
+ logger.Warnw("failed-updating-ports-state", log.Fields{"deviceId": deviceID, "error": err})
+ return err
+ }
+ return nil
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) ChildDeviceDetected(ctx context.Context, parentDeviceID string, parentPortNo int64, deviceType string,
+ channelID int64, vendorID string, serialNumber string, onuID int64) (*voltha.Device, error) {
+ logger.Debugw("ChildDeviceDetected", log.Fields{"parentDeviceId": parentDeviceID, "parentPortNo": parentPortNo, "deviceType": deviceType, "channelId": channelID, "vendorId": vendorID, "serialNumber": serialNumber, "onuId": onuID})
+
+ if deviceType == "" && vendorID != "" {
+ logger.Debug("device-type-is-nil-fetching-device-type")
+ OLoop:
+ for _, dType := range dMgr.adapterMgr.ListDeviceTypes() {
+ for _, v := range dType.VendorIds {
+ if v == vendorID {
+ deviceType = dType.Adapter
+ break OLoop
+ }
+ }
+ }
+ }
+ //if no match found for the vendorid,report adapter with the custom error message
+ if deviceType == "" {
+ logger.Errorw("failed-to-fetch-adapter-name ", log.Fields{"vendorId": vendorID})
+ return nil, status.Errorf(codes.NotFound, "%s", vendorID)
+ }
+
+ // Create the ONU device
+ childDevice := &voltha.Device{}
+ childDevice.Type = deviceType
+ childDevice.ParentId = parentDeviceID
+ childDevice.ParentPortNo = uint32(parentPortNo)
+ childDevice.VendorId = vendorID
+ childDevice.SerialNumber = serialNumber
+ childDevice.Root = false
+
+ // Get parent device type
+ pAgent := dMgr.getDeviceAgent(ctx, parentDeviceID)
+ if pAgent == nil {
+ return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
+ }
+ if pAgent.deviceType == "" {
+ return nil, status.Errorf(codes.FailedPrecondition, "device Type not set %s", parentDeviceID)
+ }
+
+ if device, err := dMgr.GetChildDevice(ctx, parentDeviceID, serialNumber, onuID, parentPortNo); err == nil {
+ logger.Warnw("child-device-exists", log.Fields{"parentId": parentDeviceID, "serialNumber": serialNumber})
+ return device, status.Errorf(codes.AlreadyExists, "%s", serialNumber)
+ }
+
+ childDevice.ProxyAddress = &voltha.Device_ProxyAddress{DeviceId: parentDeviceID, DeviceType: pAgent.deviceType, ChannelId: uint32(channelID), OnuId: uint32(onuID)}
+
+ // Create and start a device agent for that device
+ agent := newAgent(dMgr.adapterProxy, childDevice, dMgr, dMgr.clusterDataProxy, dMgr.defaultTimeout)
+ childDevice, err := agent.start(ctx, childDevice)
+ if err != nil {
+ logger.Errorw("error-starting-child-device", log.Fields{"parent-device-id": childDevice.ParentId, "child-device-id": agent.deviceID, "error": err})
+ return nil, err
+ }
+ dMgr.addDeviceAgentToMap(agent)
+
+ // Activate the child device
+ if agent = dMgr.getDeviceAgent(ctx, agent.deviceID); agent != nil {
+ go func() {
+ err := agent.enableDevice(context.Background())
+ if err != nil {
+ logger.Errorw("unable-to-enable-device", log.Fields{"error": err})
+ }
+ }()
+ }
+
+ // Publish on the messaging bus that we have discovered new devices
+ go func() {
+ err := dMgr.kafkaICProxy.DeviceDiscovered(agent.deviceID, deviceType, parentDeviceID, dMgr.coreInstanceID)
+ if err != nil {
+ logger.Errorw("unable-to-discover-the-device", log.Fields{"error": err})
+ }
+ }()
+
+ 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("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(device, previousState)
+ if handlers == nil {
+ logger.Debugw("no-op-transition", log.Fields{"deviceId": device.Id})
+ return nil
+ }
+ logger.Debugw("handler-found", log.Fields{"num-expectedHandlers": len(handlers), "isParent": device.Root, "current-data": device, "previous-state": previousState})
+ for _, handler := range handlers {
+ logger.Debugw("running-handler", log.Fields{"handler": funcName(handler)})
+ if err := handler(ctx, device); err != nil {
+ logger.Warnw("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("packetOut", log.Fields{"deviceId": deviceID, "outPort": outPort})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.packetOut(ctx, outPort, packet)
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+// PacketIn receives packet from adapter
+func (dMgr *Manager) PacketIn(ctx context.Context, deviceID string, port uint32, transactionID string, packet []byte) error {
+ logger.Debugw("PacketIn", log.Fields{"deviceId": deviceID, "port": port})
+ // Get the logical device Id based on the deviceId
+ var device *voltha.Device
+ var err error
+ if device, err = dMgr.GetDevice(ctx, deviceID); err != nil {
+ logger.Errorw("device-not-found", log.Fields{"deviceId": deviceID})
+ return err
+ }
+ if !device.Root {
+ logger.Errorw("device-not-root", log.Fields{"deviceId": deviceID})
+ return status.Errorf(codes.FailedPrecondition, "%s", deviceID)
+ }
+
+ if err := dMgr.logicalDeviceMgr.packetIn(ctx, device.ParentId, port, transactionID, packet); err != nil {
+ return err
+ }
+ return nil
+}
+
+func (dMgr *Manager) setParentID(ctx context.Context, device *voltha.Device, parentID string) error {
+ logger.Debugw("setParentId", log.Fields{"deviceId": device.Id, "parentId": parentID})
+ if agent := dMgr.getDeviceAgent(ctx, device.Id); agent != nil {
+ return agent.setParentID(ctx, device, parentID)
+ }
+ return status.Errorf(codes.NotFound, "%s", device.Id)
+}
+
+// CreateLogicalDevice creates logical device in core
+func (dMgr *Manager) CreateLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
+ logger.Info("CreateLogicalDevice")
+ // Verify whether the logical device has already been created
+ if cDevice.ParentId != "" {
+ logger.Debugw("Parent device already exist.", log.Fields{"deviceId": cDevice.Id, "logicalDeviceId": cDevice.Id})
+ return nil
+ }
+ var err error
+ if _, err = dMgr.logicalDeviceMgr.createLogicalDevice(ctx, cDevice); err != nil {
+ logger.Warnw("createlogical-device-error", log.Fields{"device": cDevice})
+ return err
+ }
+ return nil
+}
+
+// DeleteLogicalDevice deletes logical device from core
+func (dMgr *Manager) DeleteLogicalDevice(ctx context.Context, cDevice *voltha.Device) error {
+ logger.Info("DeleteLogicalDevice")
+ var err error
+ if err = dMgr.logicalDeviceMgr.deleteLogicalDevice(ctx, cDevice); err != nil {
+ logger.Warnw("deleteLogical-device-error", log.Fields{"deviceId": cDevice.Id})
+ return err
+ }
+ // Remove the logical device Id from the parent device
+ logicalID := ""
+ dMgr.UpdateDeviceAttribute(ctx, cDevice.Id, "ParentId", logicalID)
+ return nil
+}
+
+// DeleteLogicalPort removes the logical port associated with a device
+func (dMgr *Manager) DeleteLogicalPort(ctx context.Context, device *voltha.Device) error {
+ logger.Info("deleteLogicalPort")
+ var err error
+ // Get the logical port associated with this device
+ var lPortID *voltha.LogicalPortId
+ if lPortID, err = dMgr.logicalDeviceMgr.getLogicalPortID(ctx, device); err != nil {
+ logger.Warnw("getLogical-port-error", log.Fields{"deviceId": device.Id, "error": err})
+ return err
+ }
+ if err = dMgr.logicalDeviceMgr.deleteLogicalPort(ctx, lPortID); err != nil {
+ logger.Warnw("deleteLogical-port-error", log.Fields{"deviceId": device.Id})
+ return err
+ }
+ return nil
+}
+
+// DeleteLogicalPorts removes the logical ports associated with that deviceId
+func (dMgr *Manager) DeleteLogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
+ logger.Debugw("delete-all-logical-ports", log.Fields{"device-id": cDevice.Id})
+ if err := dMgr.logicalDeviceMgr.deleteLogicalPorts(ctx, cDevice.Id); err != nil {
+ // Just log the error. The logical device or port may already have been deleted before this callback is invoked.
+ logger.Warnw("deleteLogical-ports-error", log.Fields{"device-id": cDevice.Id, "error": err})
+ }
+ return nil
+}
+
+func (dMgr *Manager) getParentDevice(ctx context.Context, childDevice *voltha.Device) *voltha.Device {
+ // Sanity check
+ if childDevice.Root {
+ // childDevice is the parent device
+ return childDevice
+ }
+ parentDevice, _ := dMgr.GetDevice(ctx, childDevice.ParentId)
+ return parentDevice
+}
+
+//ChildDevicesLost is invoked by an adapter to indicate that a parent device is in a state (Disabled) where it
+//cannot manage the child devices. This will trigger the Core to disable all the child devices.
+func (dMgr *Manager) ChildDevicesLost(ctx context.Context, parentDeviceID string) error {
+ logger.Debug("ChildDevicesLost")
+ var err error
+ var parentDevice *voltha.Device
+ if parentDevice, err = dMgr.GetDevice(ctx, parentDeviceID); err != nil {
+ logger.Warnw("failed-getting-device", log.Fields{"deviceId": parentDeviceID, "error": err})
+ return err
+ }
+ return dMgr.DisableAllChildDevices(ctx, parentDevice)
+}
+
+//ChildDevicesDetected is invoked by an adapter when child devices are found, typically after after a
+// disable/enable sequence. This will trigger the Core to Enable all the child devices of that parent.
+func (dMgr *Manager) ChildDevicesDetected(ctx context.Context, parentDeviceID string) error {
+ logger.Debug("ChildDevicesDetected")
+ var err error
+ var parentDevice *voltha.Device
+ var childDeviceIds []string
+
+ if parentDevice, err = dMgr.GetDevice(ctx, parentDeviceID); err != nil {
+ logger.Warnw("failed-getting-device", log.Fields{"deviceId": parentDeviceID, "error": err})
+ return err
+ }
+
+ if childDeviceIds, err = dMgr.getAllChildDeviceIds(parentDevice); err != nil {
+ return status.Errorf(codes.NotFound, "%s", parentDevice.Id)
+ }
+ if len(childDeviceIds) == 0 {
+ logger.Debugw("no-child-device", log.Fields{"parentDeviceId": parentDevice.Id})
+ }
+ allChildEnableRequestSent := true
+ for _, childDeviceID := range childDeviceIds {
+ if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
+ // Run the children re-registration in its own routine
+ go func() {
+ err = agent.enableDevice(ctx)
+ if err != nil {
+ logger.Errorw("unable-to-enable-device", log.Fields{"error": err})
+ }
+ }()
+ } else {
+ err = status.Errorf(codes.Unavailable, "no agent for child device %s", childDeviceID)
+ logger.Errorw("no-child-device-agent", log.Fields{"parentDeviceId": parentDevice.Id, "childId": childDeviceID})
+ allChildEnableRequestSent = false
+ }
+ }
+ if !allChildEnableRequestSent {
+ return err
+ }
+ return nil
+}
+
+/*
+All the functions below are callback functions where they are invoked with the latest and previous data. We can
+therefore use the data as is without trying to get the latest from the model.
+*/
+
+//DisableAllChildDevices is invoked as a callback when the parent device is disabled
+func (dMgr *Manager) DisableAllChildDevices(ctx context.Context, parentCurrDevice *voltha.Device) error {
+ logger.Debug("DisableAllChildDevices")
+ var childDeviceIds []string
+ var err error
+ if childDeviceIds, err = dMgr.getAllChildDeviceIds(parentCurrDevice); err != nil {
+ return status.Errorf(codes.NotFound, "%s", parentCurrDevice.Id)
+ }
+ if len(childDeviceIds) == 0 {
+ logger.Debugw("no-child-device", log.Fields{"parentDeviceId": parentCurrDevice.Id})
+ }
+ for _, childDeviceID := range childDeviceIds {
+ if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
+ if err = agent.disableDevice(ctx); err != nil {
+ // Just log the error - this error happens only if the child device was already in deleted state.
+ logger.Errorw("failure-disable-device", log.Fields{"deviceId": childDeviceID, "error": err.Error()})
+ }
+ }
+ }
+ return nil
+}
+
+//DeleteAllChildDevices is invoked as a callback when the parent device is deleted
+func (dMgr *Manager) DeleteAllChildDevices(ctx context.Context, parentCurrDevice *voltha.Device) error {
+ logger.Debug("DeleteAllChildDevices")
+ var childDeviceIds []string
+ var err error
+ if childDeviceIds, err = dMgr.getAllChildDeviceIds(parentCurrDevice); err != nil {
+ return status.Errorf(codes.NotFound, "%s", parentCurrDevice.Id)
+ }
+ if len(childDeviceIds) == 0 {
+ logger.Debugw("no-child-device", log.Fields{"parentDeviceId": parentCurrDevice.Id})
+ }
+ for _, childDeviceID := range childDeviceIds {
+ if agent := dMgr.getDeviceAgent(ctx, childDeviceID); agent != nil {
+ if err = agent.deleteDevice(ctx); err != nil {
+ logger.Warnw("failure-delete-device", log.Fields{"deviceId": childDeviceID, "error": err.Error()})
+ }
+ // No further action is required here. The deleteDevice will change the device state where the resulting
+ // callback will take care of cleaning the child device agent.
+ }
+ }
+ return nil
+}
+
+//DeleteAllUNILogicalPorts is invoked as a callback when the parent device is deleted
+func (dMgr *Manager) DeleteAllUNILogicalPorts(ctx context.Context, curr *voltha.Device) error {
+ logger.Debugw("delete-all-uni-logical-ports", log.Fields{"parent-device-id": curr.Id})
+ if err := dMgr.logicalDeviceMgr.deleteAllUNILogicalPorts(ctx, curr); err != nil {
+ // Just log the error and let the remaining pipeline proceed - ports may already have been deleted
+ logger.Warnw("delete-all-uni-logical-ports-failed", log.Fields{"parent-device-id": curr.Id, "error": err})
+ }
+ return nil
+}
+
+//DeleteAllLogicalPorts is invoked as a callback when the parent device's connection status moves to UNREACHABLE
+func (dMgr *Manager) DeleteAllLogicalPorts(ctx context.Context, parentDevice *voltha.Device) error {
+ logger.Debugw("delete-all-logical-ports", log.Fields{"parent-device-id": parentDevice.Id})
+ if err := dMgr.logicalDeviceMgr.deleteAllLogicalPorts(ctx, parentDevice); err != nil {
+ // Just log error as logical device may already have been deleted
+ logger.Warnw("delete-all-logical-ports-fail", log.Fields{"parent-device-id": parentDevice.Id, "error": err})
+ }
+ return nil
+}
+
+//DeleteAllDeviceFlows is invoked as a callback when the parent device's connection status moves to UNREACHABLE
+func (dMgr *Manager) DeleteAllDeviceFlows(ctx context.Context, parentDevice *voltha.Device) error {
+ logger.Debugw("delete-all-device-flows", log.Fields{"parent-device-id": parentDevice.Id})
+ if agent := dMgr.getDeviceAgent(ctx, parentDevice.Id); agent != nil {
+ if err := agent.deleteAllFlows(ctx); err != nil {
+ logger.Errorw("error-deleting-all-device-flows", log.Fields{"parent-device-id": parentDevice.Id})
+ return err
+ }
+ return nil
+ }
+ return status.Errorf(codes.NotFound, "%s", parentDevice.Id)
+}
+
+//getAllChildDeviceIds is a helper method to get all the child device IDs from the device passed as parameter
+func (dMgr *Manager) getAllChildDeviceIds(parentDevice *voltha.Device) ([]string, error) {
+ logger.Debugw("getAllChildDeviceIds", log.Fields{"parentDeviceId": parentDevice.Id})
+ childDeviceIds := make([]string, 0)
+ if parentDevice != nil {
+ for _, port := range parentDevice.Ports {
+ for _, peer := range port.Peers {
+ childDeviceIds = append(childDeviceIds, peer.DeviceId)
+ }
+ }
+ logger.Debugw("returning-getAllChildDeviceIds", log.Fields{"parentDeviceId": parentDevice.Id, "childDeviceIds": childDeviceIds})
+ }
+ return childDeviceIds, nil
+}
+
+//GetAllChildDevices is a helper method to get all the child device IDs from the device passed as parameter
+func (dMgr *Manager) GetAllChildDevices(ctx context.Context, parentDeviceID string) (*voltha.Devices, error) {
+ logger.Debugw("GetAllChildDevices", log.Fields{"parentDeviceId": parentDeviceID})
+ if parentDevice, err := dMgr.GetDevice(ctx, parentDeviceID); err == nil {
+ childDevices := make([]*voltha.Device, 0)
+ if childDeviceIds, er := dMgr.getAllChildDeviceIds(parentDevice); er == nil {
+ for _, deviceID := range childDeviceIds {
+ if d, e := dMgr.GetDevice(ctx, deviceID); e == nil && d != nil {
+ childDevices = append(childDevices, d)
+ }
+ }
+ }
+ return &voltha.Devices{Items: childDevices}, nil
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", parentDeviceID)
+}
+
+// SetupUNILogicalPorts creates UNI ports on the logical device that represents a child UNI interface
+func (dMgr *Manager) SetupUNILogicalPorts(ctx context.Context, cDevice *voltha.Device) error {
+ logger.Info("addUNILogicalPort")
+ if err := dMgr.logicalDeviceMgr.setupUNILogicalPorts(ctx, cDevice); err != nil {
+ logger.Warnw("addUNILogicalPort-error", log.Fields{"device": cDevice, "err": err})
+ return err
+ }
+ return nil
+}
+
+func (dMgr *Manager) DownloadImage(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+ logger.Debugw("DownloadImage", log.Fields{"deviceid": img.Id, "imageName": img.Name})
+ var res interface{}
+ var err error
+ if agent := dMgr.getDeviceAgent(ctx, img.Id); agent != nil {
+ if res, err = agent.downloadImage(ctx, img); err != nil {
+ logger.Debugw("DownloadImage-failed", log.Fields{"err": err, "imageName": img.Name})
+ res = err
+ }
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", img.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) CancelImageDownload(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+ logger.Debugw("CancelImageDownload", log.Fields{"deviceid": img.Id, "imageName": img.Name})
+ var res interface{}
+ var err error
+ if agent := dMgr.getDeviceAgent(ctx, img.Id); agent != nil {
+ if res, err = agent.cancelImageDownload(ctx, img); err != nil {
+ logger.Debugw("CancelImageDownload-failed", log.Fields{"err": err, "imageName": img.Name})
+ res = err
+ }
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", img.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) ActivateImage(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+ logger.Debugw("ActivateImage", log.Fields{"deviceid": img.Id, "imageName": img.Name})
+ var res interface{}
+ var err error
+ if agent := dMgr.getDeviceAgent(ctx, img.Id); agent != nil {
+ if res, err = agent.activateImage(ctx, img); err != nil {
+ logger.Debugw("ActivateImage-failed", log.Fields{"err": err, "imageName": img.Name})
+ res = err
+ }
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", img.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) RevertImage(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+ logger.Debugw("RevertImage", log.Fields{"deviceid": img.Id, "imageName": img.Name})
+ var res interface{}
+ var err error
+ if agent := dMgr.getDeviceAgent(ctx, img.Id); agent != nil {
+ if res, err = agent.revertImage(ctx, img); err != nil {
+ logger.Debugw("RevertImage-failed", log.Fields{"err": err, "imageName": img.Name})
+ res = err
+ }
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", img.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) GetImageDownloadStatus(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+ logger.Debugw("GetImageDownloadStatus", log.Fields{"deviceid": img.Id, "imageName": img.Name})
+ var res interface{}
+ var err error
+ if agent := dMgr.getDeviceAgent(ctx, img.Id); agent != nil {
+ if res, err = agent.getImageDownloadStatus(ctx, img); err != nil {
+ logger.Debugw("GetImageDownloadStatus-failed", log.Fields{"err": err, "imageName": img.Name})
+ res = err
+ }
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", img.Id)
+ }
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) UpdateImageDownload(ctx context.Context, deviceID string, img *voltha.ImageDownload) error {
+ logger.Debugw("UpdateImageDownload", log.Fields{"deviceid": img.Id, "imageName": img.Name})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ if err := agent.updateImageDownload(ctx, img); err != nil {
+ logger.Debugw("UpdateImageDownload-failed", log.Fields{"err": err, "imageName": img.Name})
+ return err
+ }
+ } else {
+ return status.Errorf(codes.NotFound, "%s", img.Id)
+ }
+ return nil
+}
+
+func (dMgr *Manager) GetImageDownload(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+ logger.Debugw("GetImageDownload", log.Fields{"deviceid": img.Id, "imageName": img.Name})
+ if agent := dMgr.getDeviceAgent(ctx, img.Id); agent != nil {
+ return agent.getImageDownload(ctx, img)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", img.Id)
+}
+
+func (dMgr *Manager) ListImageDownloads(ctx context.Context, deviceID string) (*voltha.ImageDownloads, error) {
+ logger.Debugw("ListImageDownloads", log.Fields{"deviceID": deviceID})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.listImageDownloads(ctx, deviceID)
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) NotifyInvalidTransition(_ context.Context, device *voltha.Device) error {
+ logger.Errorw("NotifyInvalidTransition", log.Fields{
+ "device": device.Id,
+ "curr-admin-state": device.AdminState,
+ "curr-oper-state": device.OperStatus,
+ "curr-conn-state": device.ConnectStatus,
+ })
+ //TODO: notify over kafka?
+ 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 {
+ agent.(*Agent).updateDeviceAttribute(ctx, attribute, value)
+ }
+}
+
+// GetParentDeviceID returns parent device id, either from memory or from the dB, if present
+func (dMgr *Manager) GetParentDeviceID(ctx context.Context, deviceID string) string {
+ if device, _ := dMgr.GetDevice(ctx, deviceID); device != nil {
+ logger.Infow("GetParentDeviceId", log.Fields{"deviceId": device.Id, "parentId": device.ParentId})
+ return device.ParentId
+ }
+ return ""
+}
+
+func (dMgr *Manager) SimulateAlarm(ctx context.Context, simulatereq *voltha.SimulateAlarmRequest, ch chan interface{}) {
+ logger.Debugw("SimulateAlarm", log.Fields{"id": simulatereq.Id, "Indicator": simulatereq.Indicator, "IntfId": simulatereq.IntfId,
+ "PortTypeName": simulatereq.PortTypeName, "OnuDeviceId": simulatereq.OnuDeviceId, "InverseBitErrorRate": simulatereq.InverseBitErrorRate,
+ "Drift": simulatereq.Drift, "NewEqd": simulatereq.NewEqd, "OnuSerialNumber": simulatereq.OnuSerialNumber, "Operation": simulatereq.Operation})
+ var res interface{}
+ if agent := dMgr.getDeviceAgent(ctx, simulatereq.Id); agent != nil {
+ res = agent.simulateAlarm(ctx, simulatereq)
+ logger.Debugw("SimulateAlarm-result", log.Fields{"result": res})
+ }
+ //TODO CLI always get successful response
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) UpdateDeviceReason(ctx context.Context, deviceID string, reason string) error {
+ logger.Debugw("UpdateDeviceReason", log.Fields{"deviceid": deviceID, "reason": reason})
+ if agent := dMgr.getDeviceAgent(ctx, deviceID); agent != nil {
+ return agent.updateDeviceReason(ctx, reason)
+ }
+ return status.Errorf(codes.NotFound, "%s", deviceID)
+}
+
+func (dMgr *Manager) EnablePort(ctx context.Context, port *voltha.Port, ch chan interface{}) {
+ logger.Debugw("EnablePort", log.Fields{"device-id": port.DeviceId, "port-no": port.PortNo})
+ var res interface{}
+ if agent := dMgr.getDeviceAgent(ctx, port.DeviceId); agent != nil {
+ res = agent.enablePort(ctx, port)
+ logger.Debugw("EnablePort-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", port.DeviceId)
+ }
+
+ sendResponse(ctx, ch, res)
+}
+
+func (dMgr *Manager) DisablePort(ctx context.Context, port *voltha.Port, ch chan interface{}) {
+ logger.Debugw("DisablePort", log.Fields{"device-id": port.DeviceId, "port-no": port.PortNo})
+ var res interface{}
+ if agent := dMgr.getDeviceAgent(ctx, port.DeviceId); agent != nil {
+ res = agent.disablePort(ctx, port)
+ logger.Debugw("DisablePort-result", log.Fields{"result": res})
+ } else {
+ res = status.Errorf(codes.NotFound, "%s", port.DeviceId)
+ }
+
+ sendResponse(ctx, ch, res)
+}
+
+// ChildDeviceLost calls parent adapter to delete child device and all its references
+func (dMgr *Manager) ChildDeviceLost(ctx context.Context, curr *voltha.Device) error {
+ logger.Debugw("childDeviceLost", log.Fields{"child-device-id": curr.Id, "parent-device-id": curr.ParentId})
+ if parentAgent := dMgr.getDeviceAgent(ctx, curr.ParentId); parentAgent != nil {
+ if err := parentAgent.ChildDeviceLost(ctx, curr); err != nil {
+ // Just log the message and let the remaining pipeline proceed.
+ logger.Warnw("childDeviceLost", log.Fields{"child-device-id": curr.Id, "parent-device-id": curr.ParentId, "error": err})
+ }
+ }
+ // Do not return an error as parent device may also have been deleted. Let the remaining pipeline proceed.
+ return nil
+}
+
+func (dMgr *Manager) StartOmciTest(ctx context.Context, omcitestrequest *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
+ logger.Debugw("Omci_test_Request", log.Fields{"device-id": omcitestrequest.Id, "uuid": omcitestrequest.Uuid})
+ if agent := dMgr.getDeviceAgent(ctx, omcitestrequest.Id); agent != nil {
+ res, err := agent.startOmciTest(ctx, omcitestrequest)
+ if err != nil {
+ return nil, err
+ }
+ logger.Debugw("Omci_test_Response_result-device-magnager", log.Fields{"result": res})
+ return res, nil
+ }
+ return nil, status.Errorf(codes.NotFound, "%s", omcitestrequest.Id)
+}
diff --git a/rw_core/core/device/remote/adapter_proxy.go b/rw_core/core/device/remote/adapter_proxy.go
new file mode 100755
index 0000000..4b04ee5
--- /dev/null
+++ b/rw_core/core/device/remote/adapter_proxy.go
@@ -0,0 +1,432 @@
+/*
+ * 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 remote
+
+import (
+ "context"
+ "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+ ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+ "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+)
+
+// AdapterProxy represents adapter proxy attributes
+type AdapterProxy struct {
+ kafka.EndpointManager
+ deviceTopicRegistered bool
+ corePairTopic string
+ kafkaICProxy kafka.InterContainerProxy
+}
+
+// NewAdapterProxy will return adapter proxy instance
+func NewAdapterProxy(kafkaProxy kafka.InterContainerProxy, corePairTopic string, endpointManager kafka.EndpointManager) *AdapterProxy {
+ return &AdapterProxy{
+ EndpointManager: endpointManager,
+ kafkaICProxy: kafkaProxy,
+ corePairTopic: corePairTopic,
+ deviceTopicRegistered: false,
+ }
+}
+
+func (ap *AdapterProxy) getCoreTopic() kafka.Topic {
+ return kafka.Topic{Name: ap.corePairTopic}
+}
+
+func (ap *AdapterProxy) getAdapterTopic(deviceID string, adapterType string) (*kafka.Topic, error) {
+
+ endpoint, err := ap.GetEndpoint(deviceID, adapterType)
+ if err != nil {
+ return nil, err
+ }
+
+ return &kafka.Topic{Name: string(endpoint)}, nil
+}
+
+func (ap *AdapterProxy) sendRPC(ctx context.Context, rpc string, toTopic *kafka.Topic, replyToTopic *kafka.Topic,
+ waitForResponse bool, deviceID string, kvArgs ...*kafka.KVArg) (chan *kafka.RpcResponse, error) {
+
+ // Sent the request to kafka
+ respChnl := ap.kafkaICProxy.InvokeAsyncRPC(ctx, rpc, toTopic, replyToTopic, waitForResponse, deviceID, kvArgs...)
+
+ // Wait for first response which would indicate whether the request was successfully sent to kafka.
+ firstResponse, ok := <-respChnl
+ if !ok || firstResponse.MType != kafka.RpcSent {
+ logger.Errorw("failure to request to kafka", log.Fields{"rpc": rpc, "device-id": deviceID, "error": firstResponse.Err})
+ return nil, firstResponse.Err
+ }
+ // return the kafka channel for the caller to wait for the response of the RPC call
+ return respChnl, nil
+}
+
+// AdoptDevice invokes adopt device rpc
+func (ap *AdapterProxy) AdoptDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("AdoptDevice", log.Fields{"device-id": device.Id})
+ rpc := "adopt_device"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ }
+ replyToTopic := ap.getCoreTopic()
+ ap.deviceTopicRegistered = true
+ logger.Debugw("adoptDevice-send-request", log.Fields{"device-id": device.Id, "deviceType": device.Type, "serialNumber": device.SerialNumber})
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// DisableDevice invokes disable device rpc
+func (ap *AdapterProxy) DisableDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("DisableDevice", log.Fields{"device-id": device.Id})
+ rpc := "disable_device"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// ReEnableDevice invokes reenable device rpc
+func (ap *AdapterProxy) ReEnableDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("ReEnableDevice", log.Fields{"device-id": device.Id})
+ rpc := "reenable_device"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// RebootDevice invokes reboot device rpc
+func (ap *AdapterProxy) RebootDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("RebootDevice", log.Fields{"device-id": device.Id})
+ rpc := "reboot_device"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// DeleteDevice invokes delete device rpc
+func (ap *AdapterProxy) DeleteDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("DeleteDevice", log.Fields{"device-id": device.Id})
+ rpc := "delete_device"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// GetOfpDeviceInfo invokes get ofp device info rpc
+func (ap *AdapterProxy) GetOfpDeviceInfo(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("GetOfpDeviceInfo", log.Fields{"device-id": device.Id})
+ rpc := "get_ofp_device_info"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// GetOfpPortInfo invokes get ofp port info rpc
+func (ap *AdapterProxy) GetOfpPortInfo(ctx context.Context, device *voltha.Device, portNo uint32) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("GetOfpPortInfo", log.Fields{"device-id": device.Id, "port-no": portNo})
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "port_no", Value: &ic.IntType{Val: int64(portNo)}},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, "get_ofp_port_info", toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// ReconcileDevice invokes reconcile device rpc
+func (ap *AdapterProxy) ReconcileDevice(ctx context.Context, device *voltha.Device) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("ReconcileDevice", log.Fields{"device-id": device.Id})
+ rpc := "reconcile_device"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// DownloadImage invokes download image rpc
+func (ap *AdapterProxy) DownloadImage(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("DownloadImage", log.Fields{"device-id": device.Id, "image": download.Name})
+ rpc := "download_image"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "request", Value: download},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// GetImageDownloadStatus invokes get image download status rpc
+func (ap *AdapterProxy) GetImageDownloadStatus(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("GetImageDownloadStatus", log.Fields{"device-id": device.Id, "image": download.Name})
+ rpc := "get_image_download_status"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "request", Value: download},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// CancelImageDownload invokes cancel image download rpc
+func (ap *AdapterProxy) CancelImageDownload(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("CancelImageDownload", log.Fields{"device-id": device.Id, "image": download.Name})
+ rpc := "cancel_image_download"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "request", Value: download},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// ActivateImageUpdate invokes activate image update rpc
+func (ap *AdapterProxy) ActivateImageUpdate(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("ActivateImageUpdate", log.Fields{"device-id": device.Id, "image": download.Name})
+ rpc := "activate_image_update"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "request", Value: download},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// RevertImageUpdate invokes revert image update rpc
+func (ap *AdapterProxy) RevertImageUpdate(ctx context.Context, device *voltha.Device, download *voltha.ImageDownload) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("RevertImageUpdate", log.Fields{"device-id": device.Id, "image": download.Name})
+ rpc := "revert_image_update"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "request", Value: download},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+func (ap *AdapterProxy) PacketOut(ctx context.Context, deviceType string, deviceID string, outPort uint32, packet *openflow_13.OfpPacketOut) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("PacketOut", log.Fields{"device-id": deviceID, "device-type": deviceType, "out-port": outPort})
+ toTopic, err := ap.getAdapterTopic(deviceID, deviceType)
+ if err != nil {
+ return nil, err
+ }
+ rpc := "receive_packet_out"
+ args := []*kafka.KVArg{
+ {Key: "deviceId", Value: &ic.StrType{Val: deviceID}},
+ {Key: "outPort", Value: &ic.IntType{Val: int64(outPort)}},
+ {Key: "packet", Value: packet},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, deviceID, args...)
+}
+
+// UpdateFlowsBulk invokes update flows bulk rpc
+func (ap *AdapterProxy) UpdateFlowsBulk(ctx context.Context, device *voltha.Device, flows *voltha.Flows, groups *voltha.FlowGroups, flowMetadata *voltha.FlowMetadata) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("UpdateFlowsBulk", log.Fields{"device-id": device.Id, "flow-count": len(flows.Items), "group-count": len(groups.Items), "flow-metadata": flowMetadata})
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ rpc := "update_flows_bulk"
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "flows", Value: flows},
+ {Key: "groups", Value: groups},
+ {Key: "flow_metadata", Value: flowMetadata},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(context.TODO(), rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// UpdateFlowsIncremental invokes update flows incremental rpc
+func (ap *AdapterProxy) UpdateFlowsIncremental(ctx context.Context, device *voltha.Device, flowChanges *openflow_13.FlowChanges, groupChanges *openflow_13.FlowGroupChanges, flowMetadata *voltha.FlowMetadata) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("UpdateFlowsIncremental",
+ log.Fields{
+ "device-id": device.Id,
+ "flow-to-add-count": len(flowChanges.ToAdd.Items),
+ "flow-to-delete-count": len(flowChanges.ToRemove.Items),
+ "group-to-add-count": len(groupChanges.ToAdd.Items),
+ "group-to-delete-count": len(groupChanges.ToRemove.Items),
+ "group-to-update-count": len(groupChanges.ToUpdate.Items),
+ })
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ rpc := "update_flows_incrementally"
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "flow_changes", Value: flowChanges},
+ {Key: "group_changes", Value: groupChanges},
+ {Key: "flow_metadata", Value: flowMetadata},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(context.TODO(), rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// UpdatePmConfigs invokes update pm configs rpc
+func (ap *AdapterProxy) UpdatePmConfigs(ctx context.Context, device *voltha.Device, pmConfigs *voltha.PmConfigs) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("UpdatePmConfigs", log.Fields{"device-id": device.Id, "pm-configs-id": pmConfigs.Id})
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ rpc := "Update_pm_config"
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "pm_configs", Value: pmConfigs},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// SimulateAlarm invokes simulate alarm rpc
+func (ap *AdapterProxy) SimulateAlarm(ctx context.Context, device *voltha.Device, simulateReq *voltha.SimulateAlarmRequest) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("SimulateAlarm", log.Fields{"device-id": device.Id, "simulate-req-id": simulateReq.Id})
+ rpc := "simulate_alarm"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "device", Value: device},
+ {Key: "request", Value: simulateReq},
+ }
+ replyToTopic := ap.getCoreTopic()
+ ap.deviceTopicRegistered = true
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+func (ap *AdapterProxy) DisablePort(ctx context.Context, device *voltha.Device, port *voltha.Port) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("DisablePort", log.Fields{"device-id": device.Id, "port-no": port.PortNo})
+ rpc := "disable_port"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "deviceId", Value: &ic.StrType{Val: device.Id}},
+ {Key: "port", Value: port},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+func (ap *AdapterProxy) EnablePort(ctx context.Context, device *voltha.Device, port *voltha.Port) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("EnablePort", log.Fields{"device-id": device.Id, "port-no": port.PortNo})
+ rpc := "enable_port"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "deviceId", Value: &ic.StrType{Val: device.Id}},
+ {Key: "port", Value: port},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id, args...)
+}
+
+// ChildDeviceLost invokes child device_lost rpc
+func (ap *AdapterProxy) ChildDeviceLost(ctx context.Context, deviceType string, deviceID string, pPortNo uint32, onuID uint32) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("ChildDeviceLost", log.Fields{"device-id": deviceID, "parent-port-no": pPortNo, "onu-id": onuID})
+ rpc := "child_device_lost"
+ toTopic, err := ap.getAdapterTopic(deviceID, deviceType)
+ if err != nil {
+ return nil, err
+ }
+ args := []*kafka.KVArg{
+ {Key: "pDeviceId", Value: &ic.StrType{Val: deviceID}},
+ {Key: "pPortNo", Value: &ic.IntType{Val: int64(pPortNo)}},
+ {Key: "onuID", Value: &ic.IntType{Val: int64(onuID)}},
+ }
+ replyToTopic := ap.getCoreTopic()
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, deviceID, args...)
+}
+
+func (ap *AdapterProxy) StartOmciTest(ctx context.Context, device *voltha.Device, omcitestrequest *voltha.OmciTestRequest) (chan *kafka.RpcResponse, error) {
+ logger.Debugw("Omci_test_Request_adapter_proxy", log.Fields{"device": device, "omciTestRequest": omcitestrequest})
+ rpc := "start_omci_test"
+ toTopic, err := ap.getAdapterTopic(device.Id, device.Adapter)
+ if err != nil {
+ return nil, err
+ }
+ // Use a device specific topic as we are the only core handling requests for this device
+ replyToTopic := ap.getCoreTopic()
+ // TODO: Perhaps this should have used omcitestrequest.uuid as the second argument rather
+ // than including the whole request, which is (deviceid, uuid)
+ return ap.sendRPC(ctx, rpc, toTopic, &replyToTopic, true, device.Id,
+ &kafka.KVArg{Key: "device", Value: device},
+ &kafka.KVArg{Key: "omcitestrequest", Value: omcitestrequest})
+}
diff --git a/rw_core/core/device/remote/adapter_proxy_test.go b/rw_core/core/device/remote/adapter_proxy_test.go
new file mode 100755
index 0000000..17627dc
--- /dev/null
+++ b/rw_core/core/device/remote/adapter_proxy_test.go
@@ -0,0 +1,257 @@
+/*
+ * 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 remote
+
+import (
+ "context"
+ "crypto/rand"
+ "github.com/golang/protobuf/ptypes"
+ any2 "github.com/golang/protobuf/ptypes/any"
+ cm "github.com/opencord/voltha-go/rw_core/mocks"
+ com "github.com/opencord/voltha-lib-go/v3/pkg/adapters/common"
+ "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+ mock_kafka "github.com/opencord/voltha-lib-go/v3/pkg/mocks/kafka"
+ ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+ of "github.com/opencord/voltha-protos/v3/go/openflow_13"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+ "github.com/stretchr/testify/assert"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+ "strings"
+ "testing"
+ "time"
+)
+
+const (
+ coreName = "rw_core"
+ adapterName = "adapter_mock"
+ coreInstanceID = "1000"
+)
+
+var (
+ coreKafkaICProxy kafka.InterContainerProxy
+ adapterKafkaICProxy kafka.InterContainerProxy
+ kc kafka.Client
+ adapterReqHandler *com.RequestHandlerProxy
+ adapter *cm.Adapter
+)
+
+func init() {
+ if _, err := log.SetDefaultLogger(log.JSON, 0, log.Fields{"instanceId": coreInstanceID}); err != nil {
+ log.With(log.Fields{"error": err}).Fatal("Cannot setup logging")
+ }
+ // Set the log level to Warning
+ log.SetAllLogLevel(log.WarnLevel)
+
+ var err error
+
+ // Create the KV client
+ kc = mock_kafka.NewKafkaClient()
+
+ // Setup core inter-container proxy and core request handler
+ coreKafkaICProxy = kafka.NewInterContainerProxy(
+ kafka.MsgClient(kc),
+ kafka.DefaultTopic(&kafka.Topic{Name: coreName}))
+
+ if err = coreKafkaICProxy.Start(); err != nil {
+ logger.Fatalw("Failure-starting-core-kafka-intercontainerProxy", log.Fields{"error": err})
+ }
+ if err = coreKafkaICProxy.SubscribeWithDefaultRequestHandler(kafka.Topic{Name: coreName}, 0); err != nil {
+ logger.Fatalw("Failure-subscribing-core-request-handler", log.Fields{"error": err})
+ }
+
+ // Setup adapter inter-container proxy and adapter request handler
+ adapterCoreProxy := com.NewCoreProxy(nil, adapterName, coreName)
+ adapter = cm.NewAdapter(adapterCoreProxy)
+ adapterReqHandler = com.NewRequestHandlerProxy(coreInstanceID, adapter, adapterCoreProxy)
+ adapterKafkaICProxy = kafka.NewInterContainerProxy(
+ kafka.MsgClient(kc),
+ kafka.DefaultTopic(&kafka.Topic{Name: adapterName}),
+ kafka.RequestHandlerInterface(adapterReqHandler))
+
+ if err = adapterKafkaICProxy.Start(); err != nil {
+ logger.Fatalw("Failure-starting-adapter-kafka-intercontainerProxy", log.Fields{"error": err})
+ }
+ if err = adapterKafkaICProxy.SubscribeWithDefaultRequestHandler(kafka.Topic{Name: adapterName}, 0); err != nil {
+ logger.Fatalw("Failure-subscribing-adapter-request-handler", log.Fields{"error": err})
+ }
+}
+
+func getRandomBytes(size int) (bytes []byte, err error) {
+ bytes = make([]byte, size)
+ _, err = rand.Read(bytes)
+ return
+}
+
+func TestCreateAdapterProxy(t *testing.T) {
+ ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
+ assert.NotNil(t, ap)
+}
+
+func waitForResponse(ctx context.Context, ch chan *kafka.RpcResponse) (*any2.Any, error) {
+ select {
+ case rpcResponse, ok := <-ch:
+ if !ok {
+ return nil, status.Error(codes.Aborted, "channel-closed")
+ } else if rpcResponse.Err != nil {
+ return nil, rpcResponse.Err
+ } else {
+ return rpcResponse.Reply, nil
+ }
+ case <-ctx.Done():
+ return nil, ctx.Err()
+ }
+}
+
+func testSimpleRequests(t *testing.T) {
+ type simpleRequest func(context.Context, *voltha.Device) (chan *kafka.RpcResponse, error)
+ ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
+ simpleRequests := []simpleRequest{
+ ap.AdoptDevice,
+ ap.DisableDevice,
+ ap.RebootDevice,
+ ap.DeleteDevice,
+ ap.ReconcileDevice,
+ ap.ReEnableDevice,
+ }
+ for _, f := range simpleRequests {
+ // Success
+ d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+ ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
+ rpcResponse, err := f(ctx, d)
+ assert.Nil(t, err)
+ _, err = waitForResponse(ctx, rpcResponse)
+ assert.Nil(t, err)
+ cancel()
+
+ // Failure - invalid adapter
+ expectedError := "context deadline exceeded"
+ d = &voltha.Device{Id: "deviceId", Adapter: "adapter_mock_1"}
+ ctx, cancel = context.WithTimeout(context.Background(), 20*time.Millisecond)
+ rpcResponse, err = f(ctx, d)
+ assert.Nil(t, err)
+ _, err = waitForResponse(ctx, rpcResponse)
+ cancel()
+ assert.NotNil(t, err)
+ assert.True(t, strings.Contains(err.Error(), expectedError))
+
+ // Failure - timeout
+ d = &voltha.Device{Id: "deviceId", Adapter: adapterName}
+ ctx, cancel = context.WithTimeout(context.Background(), 100*time.Nanosecond)
+ rpcResponse, err = f(ctx, d)
+ assert.Nil(t, err)
+ _, err = waitForResponse(ctx, rpcResponse)
+ cancel()
+ assert.NotNil(t, err)
+ assert.True(t, strings.Contains(err.Error(), expectedError))
+ }
+}
+
+func testGetSwitchCapabilityFromAdapter(t *testing.T) {
+ ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
+ d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+ ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+ defer cancel()
+ rpcResponse, err := ap.GetOfpDeviceInfo(ctx, d)
+ assert.Nil(t, err)
+ response, err := waitForResponse(ctx, rpcResponse)
+ assert.Nil(t, err)
+ switchCap := &ic.SwitchCapability{}
+ err = ptypes.UnmarshalAny(response, switchCap)
+ assert.Nil(t, err)
+ assert.NotNil(t, switchCap)
+ expectedCap, _ := adapter.Get_ofp_device_info(d)
+ assert.Equal(t, switchCap.String(), expectedCap.String())
+}
+
+func testGetPortInfoFromAdapter(t *testing.T) {
+ ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
+ d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+ ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+ defer cancel()
+ portNo := uint32(1)
+ rpcResponse, err := ap.GetOfpPortInfo(ctx, d, portNo)
+ assert.Nil(t, err)
+ response, err := waitForResponse(ctx, rpcResponse)
+ assert.Nil(t, err)
+ portCap := &ic.PortCapability{}
+ err = ptypes.UnmarshalAny(response, portCap)
+ assert.Nil(t, err)
+ assert.NotNil(t, portCap)
+ expectedPortInfo, _ := adapter.Get_ofp_port_info(d, int64(portNo))
+ assert.Equal(t, portCap.String(), expectedPortInfo.String())
+}
+
+func testPacketOut(t *testing.T) {
+ ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
+ d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+ outPort := uint32(1)
+ packet, err := getRandomBytes(50)
+ assert.Nil(t, err)
+ ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
+ defer cancel()
+ rpcResponse, err := ap.PacketOut(ctx, adapterName, d.Id, outPort, &of.OfpPacketOut{Data: packet})
+ assert.Nil(t, err)
+ _, err = waitForResponse(ctx, rpcResponse)
+ assert.Nil(t, err)
+}
+
+func testFlowUpdates(t *testing.T) {
+ ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
+ d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+ _, err := ap.UpdateFlowsBulk(context.Background(), d, &voltha.Flows{}, &voltha.FlowGroups{}, &voltha.FlowMetadata{})
+ assert.Nil(t, err)
+ flowChanges := &voltha.FlowChanges{ToAdd: &voltha.Flows{Items: nil}, ToRemove: &voltha.Flows{Items: nil}}
+ groupChanges := &voltha.FlowGroupChanges{ToAdd: &voltha.FlowGroups{Items: nil}, ToRemove: &voltha.FlowGroups{Items: nil}, ToUpdate: &voltha.FlowGroups{Items: nil}}
+ ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
+ defer cancel()
+ rpcResponse, err := ap.UpdateFlowsIncremental(ctx, d, flowChanges, groupChanges, &voltha.FlowMetadata{})
+ assert.Nil(t, err)
+ _, err = waitForResponse(ctx, rpcResponse)
+ assert.Nil(t, err)
+}
+
+func testPmUpdates(t *testing.T) {
+ ap := NewAdapterProxy(coreKafkaICProxy, coreName, mock_kafka.NewEndpointManager())
+ d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+ ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+ defer cancel()
+ rpcResponse, err := ap.UpdatePmConfigs(ctx, d, &voltha.PmConfigs{})
+ assert.Nil(t, err)
+ _, err = waitForResponse(ctx, rpcResponse)
+ assert.Nil(t, err)
+}
+
+func TestSuiteAdapterProxy(t *testing.T) {
+ //1. Test the simple requests first
+ testSimpleRequests(t)
+
+ //2. Test get switch capability
+ testGetSwitchCapabilityFromAdapter(t)
+
+ //3. Test get port info
+ testGetPortInfoFromAdapter(t)
+
+ //4. Test PacketOut
+ testPacketOut(t)
+
+ // 5. Test flow updates
+ testFlowUpdates(t)
+
+ //6. Pm configs
+ testPmUpdates(t)
+}
diff --git a/rw_core/core/device/remote/common.go b/rw_core/core/device/remote/common.go
new file mode 100644
index 0000000..7383bd9
--- /dev/null
+++ b/rw_core/core/device/remote/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 remote
+
+import (
+ "github.com/opencord/voltha-lib-go/v3/pkg/log"
+)
+
+var logger log.Logger
+
+func init() {
+ // Setup this package so that it's log level can be modified at run time
+ var err error
+ logger, err = log.AddPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "remote"})
+ if err != nil {
+ panic(err)
+ }
+}
diff --git a/rw_core/core/device/state_transitions.go b/rw_core/core/device/state_transitions.go
new file mode 100644
index 0000000..f7be154
--- /dev/null
+++ b/rw_core/core/device/state_transitions.go
@@ -0,0 +1,338 @@
+/*
+ * 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 device
+
+import (
+ "context"
+ "github.com/opencord/voltha-go/rw_core/coreif"
+ "github.com/opencord/voltha-protos/v3/go/voltha"
+)
+
+// deviceType mentions type of device like parent, child
+type deviceType int32
+
+const (
+ parent deviceType = 0
+ child deviceType = 1
+ any deviceType = 2
+)
+
+type matchResult uint8
+
+const (
+ noMatch matchResult = iota // current state has not match in the transition table
+ currWildcardMatch // current state matches the wildcard *_UNKNOWN state in the transition table
+ currStateOnlyMatch // current state matches the current state and previous state matches the wildcard in the transition table
+ currPrevStateMatch // both current and previous states match in the transition table
+)
+
+// match is used to keep the current match states
+type match struct {
+ admin, oper, conn matchResult
+}
+
+// toInt returns an integer representing the matching level of the match (the larger the number the better)
+func (m *match) toInt() int {
+ return int(m.admin<<4 | m.oper<<2 | m.conn)
+}
+
+// isExactMatch returns true if match is an exact match
+func (m *match) isExactMatch() bool {
+ return m.admin == currPrevStateMatch && m.oper == currPrevStateMatch && m.conn == currPrevStateMatch
+}
+
+// isBetterMatch returns true if newMatch is a worse match
+func (m *match) isBetterMatch(newMatch *match) bool {
+ return m.toInt() > newMatch.toInt()
+}
+
+// deviceState has admin, operational and connection status of device
+type deviceState struct {
+ Admin voltha.AdminState_Types
+ Connection voltha.ConnectStatus_Types
+ 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
+
+// Transition represent transition related attributes
+type Transition struct {
+ deviceType deviceType
+ previousState deviceState
+ currentState deviceState
+ handlers []TransitionHandler
+}
+
+// TransitionMap represent map of transitions and device manager
+type TransitionMap struct {
+ transitions []Transition
+ dMgr coreif.DeviceManager
+}
+
+// NewTransitionMap creates transition map
+func NewTransitionMap(dMgr coreif.DeviceManager) *TransitionMap {
+ var transitionMap TransitionMap
+ transitionMap.dMgr = dMgr
+ transitionMap.transitions = make([]Transition, 0)
+ transitionMap.transitions = append(
+ transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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{}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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{}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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.DisableAllChildDevices, dMgr.DeleteAllUNILogicalPorts, dMgr.DeleteAllChildDevices, dMgr.DeleteAllLogicalPorts, dMgr.DeleteLogicalDevice, dMgr.RunPostDeviceDelete}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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.DeleteLogicalDevice, dMgr.DeleteAllChildDevices, dMgr.DeleteAllDeviceFlows}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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.DeleteLogicalDevice, dMgr.DeleteAllChildDevices, dMgr.DeleteAllDeviceFlows}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+ transitionMap.transitions = append(transitionMap.transitions,
+ 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}})
+
+ return &transitionMap
+}
+
+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) {
+ m := &match{}
+ // Do we have an exact match?
+ if *previous == transition.previousState && *current == transition.currentState {
+ return transition.handlers, &match{admin: currPrevStateMatch, oper: currPrevStateMatch, conn: currPrevStateMatch}
+ }
+
+ // Do we have Admin state match?
+ if current.Admin == transition.currentState.Admin && transition.currentState.Admin != voltha.AdminState_UNKNOWN {
+ if previous.Admin == transition.previousState.Admin {
+ m.admin = currPrevStateMatch
+ } else if transition.previousState.Admin == voltha.AdminState_UNKNOWN {
+ m.admin = currStateOnlyMatch
+ }
+ } else if current.Admin == transition.currentState.Admin && transition.currentState.Admin == voltha.AdminState_UNKNOWN {
+ if previous.Admin == transition.previousState.Admin || transition.previousState.Admin == voltha.AdminState_UNKNOWN {
+ m.admin = currWildcardMatch
+ }
+ }
+ if m.admin == noMatch {
+ // invalid transition - need to match on current admin state
+ return nil, m
+ }
+
+ // Do we have an operational state match?
+ if current.Operational == transition.currentState.Operational && transition.previousState.Operational != voltha.OperStatus_UNKNOWN {
+ if previous.Operational == transition.previousState.Operational || transition.previousState.Operational == voltha.OperStatus_UNKNOWN {
+ m.oper = currPrevStateMatch
+ } else {
+ m.oper = currStateOnlyMatch
+ }
+ } else if current.Operational == transition.currentState.Operational && transition.previousState.Operational == voltha.OperStatus_UNKNOWN {
+ if previous.Operational == transition.previousState.Operational || transition.previousState.Operational == voltha.OperStatus_UNKNOWN {
+ m.oper = currWildcardMatch
+ }
+ }
+
+ // Do we have an connection state match?
+ if current.Connection == transition.currentState.Connection && transition.previousState.Connection != voltha.ConnectStatus_UNKNOWN {
+ if previous.Connection == transition.previousState.Connection || transition.previousState.Connection == voltha.ConnectStatus_UNKNOWN {
+ m.conn = currPrevStateMatch
+ } else {
+ m.conn = currStateOnlyMatch
+ }
+ } else if current.Connection == transition.currentState.Connection && transition.previousState.Connection == voltha.ConnectStatus_UNKNOWN {
+ if previous.Connection == transition.previousState.Connection || transition.previousState.Connection == voltha.ConnectStatus_UNKNOWN {
+ m.conn = currWildcardMatch
+ }
+ }
+
+ return transition.handlers, m
+}
+
+// GetTransitionHandler returns transition handler & a flag that's set if the transition is invalid
+func (tMap *TransitionMap) GetTransitionHandler(device *voltha.Device, pState *deviceState) []TransitionHandler {
+ //1. Get the previous and current set of states
+ cState := getDeviceStates(device)
+
+ // Do nothing is there are no states change
+ if *pState == *cState {
+ return nil
+ }
+
+ //logger.Infow("deviceType", log.Fields{"device": pDevice})
+ deviceType := parent
+ if !device.Root {
+ logger.Info("device is child")
+ deviceType = child
+ }
+ logger.Infof("deviceType:%d-deviceId:%s-previous:%v-current:%v", deviceType, device.Id, pState, cState)
+
+ //2. Go over transition array to get the right transition
+ var currentMatch []TransitionHandler
+ var tempHandler []TransitionHandler
+ var m *match
+ bestMatch := &match{}
+ for _, aTransition := range tMap.transitions {
+ // consider transition only if it matches deviceType or is a wild card - any
+ if aTransition.deviceType != deviceType && aTransition.deviceType != any {
+ continue
+ }
+ tempHandler, m = getHandler(pState, cState, &aTransition)
+ if tempHandler != nil {
+ if m.isExactMatch() && aTransition.deviceType == deviceType {
+ return tempHandler
+ } else if m.isExactMatch() || m.isBetterMatch(bestMatch) {
+ currentMatch = tempHandler
+ bestMatch = m
+ }
+ }
+ }
+ return currentMatch
+}
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..0a1b43b
--- /dev/null
+++ b/rw_core/core/device/state_transitions_test.go
@@ -0,0 +1,339 @@
+/*
+ * 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 (
+ "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(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(device, previousState)
+ assert.Equal(t, 0, len(handlers))
+}
+
+func TestValidTransitions(t *testing.T) {
+ 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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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.DeleteLogicalDevice).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).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(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(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.DeleteLogicalDevice).Pointer() == reflect.ValueOf(handlers[1]).Pointer())
+ assert.True(t, reflect.ValueOf(tdm.DeleteAllChildDevices).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(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.DisableAllChildDevices,
+ tdm.DeleteAllUNILogicalPorts,
+ tdm.DeleteAllChildDevices,
+ tdm.DeleteAllLogicalPorts,
+ 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(device, previousState)
+ assert.Equal(t, 6, 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(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())
+}