blob: 543a19d5402f91660de80952e2bdf15cabb215be [file] [log] [blame]
khenaidoob64fc8a2019-11-27 15:08:19 -05001/*
Kent Hagerman45a13e42020-04-13 12:23:50 -04002 * Copyright 2019-present Open Networking Foundation
3 *
4 * Licensed under the Apache License, Version 2.0 (the "License");
5 * you may not use this file except in compliance with the License.
6 * You may obtain a copy of the License at
7 *
8 * http://www.apache.org/licenses/LICENSE-2.0
9 *
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
khenaidoob64fc8a2019-11-27 15:08:19 -050015 */
Kent Hagerman45a13e42020-04-13 12:23:50 -040016
Kent Hagerman2b216042020-04-03 18:28:56 -040017package api
khenaidoob64fc8a2019-11-27 15:08:19 -050018
19import (
20 "context"
21 "errors"
22 "fmt"
khenaidoo67b22152020-03-02 16:01:25 -050023 "math/rand"
24 "os"
25 "runtime"
26 "runtime/pprof"
serkant.uluderya2ae470f2020-01-21 11:13:09 -080027 "strings"
khenaidoo67b22152020-03-02 16:01:25 -050028 "sync"
serkant.uluderya2ae470f2020-01-21 11:13:09 -080029 "testing"
30 "time"
31
khenaidoob64fc8a2019-11-27 15:08:19 -050032 "github.com/golang/protobuf/ptypes/empty"
Kent Hagerman2b216042020-04-03 18:28:56 -040033 "github.com/opencord/voltha-go/db/model"
khenaidoob64fc8a2019-11-27 15:08:19 -050034 "github.com/opencord/voltha-go/rw_core/config"
Kent Hagerman2b216042020-04-03 18:28:56 -040035 "github.com/opencord/voltha-go/rw_core/core/adapter"
36 "github.com/opencord/voltha-go/rw_core/core/device"
khenaidoob64fc8a2019-11-27 15:08:19 -050037 cm "github.com/opencord/voltha-go/rw_core/mocks"
Mahir Gunyel03de0d32020-06-03 01:36:59 -070038 tst "github.com/opencord/voltha-go/rw_core/test"
Kent Hagerman2b216042020-04-03 18:28:56 -040039 "github.com/opencord/voltha-lib-go/v3/pkg/db"
40 "github.com/opencord/voltha-lib-go/v3/pkg/flows"
serkant.uluderya2ae470f2020-01-21 11:13:09 -080041 "github.com/opencord/voltha-lib-go/v3/pkg/kafka"
Matteo Scandolod525ae32020-04-02 17:27:29 -070042 mock_etcd "github.com/opencord/voltha-lib-go/v3/pkg/mocks/etcd"
43 mock_kafka "github.com/opencord/voltha-lib-go/v3/pkg/mocks/kafka"
serkant.uluderya2ae470f2020-01-21 11:13:09 -080044 ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
45 "github.com/opencord/voltha-protos/v3/go/voltha"
khenaidoob64fc8a2019-11-27 15:08:19 -050046 "github.com/phayes/freeport"
47 "github.com/stretchr/testify/assert"
48 "google.golang.org/grpc/codes"
49 "google.golang.org/grpc/status"
khenaidoob64fc8a2019-11-27 15:08:19 -050050)
51
52type NBTest struct {
Matteo Scandolod525ae32020-04-02 17:27:29 -070053 etcdServer *mock_etcd.EtcdServer
Kent Hagerman2b216042020-04-03 18:28:56 -040054 deviceMgr *device.Manager
55 logicalDeviceMgr *device.LogicalManager
56 adapterMgr *adapter.Manager
57 kmp kafka.InterContainerProxy
khenaidoo67b22152020-03-02 16:01:25 -050058 kClient kafka.Client
59 kvClientPort int
60 numONUPerOLT int
61 startingUNIPortNo int
62 oltAdapter *cm.OLTAdapter
63 onuAdapter *cm.ONUAdapter
64 oltAdapterName string
65 onuAdapterName string
66 coreInstanceID string
67 defaultTimeout time.Duration
68 maxTimeout time.Duration
khenaidoob64fc8a2019-11-27 15:08:19 -050069}
70
71func newNBTest() *NBTest {
72 test := &NBTest{}
73 // Start the embedded etcd server
74 var err error
Mahir Gunyel03de0d32020-06-03 01:36:59 -070075 test.etcdServer, test.kvClientPort, err = tst.StartEmbeddedEtcdServer("voltha.rwcore.nb.test", "voltha.rwcore.nb.etcd", "error")
khenaidoob64fc8a2019-11-27 15:08:19 -050076 if err != nil {
Girish Kumarf56a4682020-03-20 20:07:46 +000077 logger.Fatal(err)
khenaidoob64fc8a2019-11-27 15:08:19 -050078 }
79 // Create the kafka client
Matteo Scandolod525ae32020-04-02 17:27:29 -070080 test.kClient = mock_kafka.NewKafkaClient()
khenaidoob64fc8a2019-11-27 15:08:19 -050081 test.oltAdapterName = "olt_adapter_mock"
82 test.onuAdapterName = "onu_adapter_mock"
83 test.coreInstanceID = "rw-nbi-test"
khenaidoo32836732020-03-05 16:10:44 -050084 test.defaultTimeout = 10 * time.Second
85 test.maxTimeout = 20 * time.Second
khenaidoob64fc8a2019-11-27 15:08:19 -050086 return test
87}
88
89func (nb *NBTest) startCore(inCompeteMode bool) {
Thomas Lee Se5a44012019-11-07 20:32:24 +053090 ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
91 defer cancel()
khenaidoob64fc8a2019-11-27 15:08:19 -050092 cfg := config.NewRWCoreFlags()
93 cfg.CorePairTopic = "rw_core"
khenaidoo442e7c72020-03-10 16:13:48 -040094 cfg.DefaultRequestTimeout = nb.defaultTimeout
95 cfg.DefaultCoreTimeout = nb.defaultTimeout
khenaidoob64fc8a2019-11-27 15:08:19 -050096 cfg.KVStorePort = nb.kvClientPort
97 cfg.InCompetingMode = inCompeteMode
98 grpcPort, err := freeport.GetFreePort()
99 if err != nil {
Girish Kumarf56a4682020-03-20 20:07:46 +0000100 logger.Fatal("Cannot get a freeport for grpc")
khenaidoob64fc8a2019-11-27 15:08:19 -0500101 }
102 cfg.GrpcPort = grpcPort
103 cfg.GrpcHost = "127.0.0.1"
104 setCoreCompeteMode(inCompeteMode)
Mahir Gunyel03de0d32020-06-03 01:36:59 -0700105 client := tst.SetupKVClient(cfg, nb.coreInstanceID)
Kent Hagerman2b216042020-04-03 18:28:56 -0400106 backend := &db.Backend{
107 Client: client,
108 StoreType: cfg.KVStoreType,
109 Host: cfg.KVStoreHost,
110 Port: cfg.KVStorePort,
111 Timeout: cfg.KVStoreTimeout,
112 LivenessChannelInterval: cfg.LiveProbeInterval / 2,
113 PathPrefix: cfg.KVStoreDataPrefix}
114 nb.kmp = kafka.NewInterContainerProxy(
115 kafka.InterContainerHost(cfg.KafkaAdapterHost),
116 kafka.InterContainerPort(cfg.KafkaAdapterPort),
117 kafka.MsgClient(nb.kClient),
118 kafka.DefaultTopic(&kafka.Topic{Name: cfg.CoreTopic}),
119 kafka.DeviceDiscoveryTopic(&kafka.Topic{Name: cfg.AffinityRouterTopic}))
120
121 endpointMgr := kafka.NewEndpointManager(backend)
Kent Hagermanf5a67352020-04-30 15:15:26 -0400122 proxy := model.NewDBPath(backend)
Kent Hagerman2b216042020-04-03 18:28:56 -0400123 nb.adapterMgr = adapter.NewAdapterManager(proxy, nb.coreInstanceID, nb.kClient)
Kent Hagerman45a13e42020-04-13 12:23:50 -0400124 nb.deviceMgr, nb.logicalDeviceMgr = device.NewManagers(proxy, nb.adapterMgr, nb.kmp, endpointMgr, cfg.CorePairTopic, nb.coreInstanceID, cfg.DefaultCoreTimeout)
Kent Hagerman2f0d0552020-04-23 17:28:52 -0400125 nb.adapterMgr.Start(ctx)
Kent Hagerman2b216042020-04-03 18:28:56 -0400126
Kent Hagerman2f0d0552020-04-23 17:28:52 -0400127 if err := nb.kmp.Start(); err != nil {
Kent Hagerman2b216042020-04-03 18:28:56 -0400128 logger.Fatalf("Cannot start InterContainerProxy: %s", err)
129 }
Kent Hagerman2f0d0552020-04-23 17:28:52 -0400130 requestProxy := NewAdapterRequestHandlerProxy(nb.deviceMgr, nb.adapterMgr)
Kent Hagerman2b216042020-04-03 18:28:56 -0400131 if err := nb.kmp.SubscribeWithRequestHandlerInterface(kafka.Topic{Name: cfg.CoreTopic}, requestProxy); err != nil {
132 logger.Fatalf("Cannot add request handler: %s", err)
133 }
134 if err := nb.kmp.SubscribeWithDefaultRequestHandler(kafka.Topic{Name: cfg.CorePairTopic}, kafka.OffsetNewest); err != nil {
135 logger.Fatalf("Cannot add default request handler: %s", err)
Thomas Lee Se5a44012019-11-07 20:32:24 +0530136 }
khenaidoob64fc8a2019-11-27 15:08:19 -0500137}
138
khenaidoob64fc8a2019-11-27 15:08:19 -0500139func (nb *NBTest) stopAll() {
140 if nb.kClient != nil {
141 nb.kClient.Stop()
142 }
Kent Hagerman2b216042020-04-03 18:28:56 -0400143 if nb.kmp != nil {
144 nb.kmp.Stop()
khenaidoob64fc8a2019-11-27 15:08:19 -0500145 }
146 if nb.etcdServer != nil {
Mahir Gunyel03de0d32020-06-03 01:36:59 -0700147 tst.StopEmbeddedEtcdServer(nb.etcdServer)
khenaidoob64fc8a2019-11-27 15:08:19 -0500148 }
149}
150
Kent Hagerman2b216042020-04-03 18:28:56 -0400151func (nb *NBTest) verifyLogicalDevices(t *testing.T, oltDevice *voltha.Device, nbi *NBIHandler) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500152 // Get the latest set of logical devices
153 logicalDevices, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
154 assert.Nil(t, err)
155 assert.NotNil(t, logicalDevices)
156 assert.Equal(t, 1, len(logicalDevices.Items))
157
158 ld := logicalDevices.Items[0]
159 assert.NotEqual(t, "", ld.Id)
160 assert.NotEqual(t, uint64(0), ld.DatapathId)
161 assert.Equal(t, "olt_adapter_mock", ld.Desc.HwDesc)
162 assert.Equal(t, "olt_adapter_mock", ld.Desc.SwDesc)
163 assert.NotEqual(t, "", ld.RootDeviceId)
164 assert.NotEqual(t, "", ld.Desc.SerialNum)
165 assert.Equal(t, uint32(256), ld.SwitchFeatures.NBuffers)
166 assert.Equal(t, uint32(2), ld.SwitchFeatures.NTables)
167 assert.Equal(t, uint32(15), ld.SwitchFeatures.Capabilities)
168 assert.Equal(t, 1+nb.numONUPerOLT, len(ld.Ports))
169 assert.Equal(t, oltDevice.ParentId, ld.Id)
170 //Expected port no
171 expectedPortNo := make(map[uint32]bool)
172 expectedPortNo[uint32(2)] = false
173 for i := 0; i < nb.numONUPerOLT; i++ {
174 expectedPortNo[uint32(i+100)] = false
175 }
176 for _, p := range ld.Ports {
177 assert.Equal(t, p.OfpPort.PortNo, p.DevicePortNo)
178 assert.Equal(t, uint32(4), p.OfpPort.State)
179 expectedPortNo[p.OfpPort.PortNo] = true
180 if strings.HasPrefix(p.Id, "nni") {
181 assert.Equal(t, true, p.RootPort)
182 //assert.Equal(t, uint32(2), p.OfpPort.PortNo)
183 assert.Equal(t, p.Id, fmt.Sprintf("nni-%d", p.DevicePortNo))
184 } else {
185 assert.Equal(t, p.Id, fmt.Sprintf("uni-%d", p.DevicePortNo))
186 assert.Equal(t, false, p.RootPort)
187 }
188 }
189}
190
Kent Hagerman2b216042020-04-03 18:28:56 -0400191func (nb *NBTest) verifyDevices(t *testing.T, nbi *NBIHandler) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500192 // Get the latest set of devices
193 devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
194 assert.Nil(t, err)
195 assert.NotNil(t, devices)
196
khenaidoo67b22152020-03-02 16:01:25 -0500197 // A device is ready to be examined when its ADMIN state is ENABLED and OPERATIONAL state is ACTIVE
khenaidoob64fc8a2019-11-27 15:08:19 -0500198 var vFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
199 return device.AdminState == voltha.AdminState_ENABLED && device.OperStatus == voltha.OperStatus_ACTIVE
200 }
khenaidoob64fc8a2019-11-27 15:08:19 -0500201
khenaidoo67b22152020-03-02 16:01:25 -0500202 var wg sync.WaitGroup
203 for _, device := range devices.Items {
204 wg.Add(1)
205 go func(wg *sync.WaitGroup, device *voltha.Device) {
206 // Wait until the device is in the right state
207 err := waitUntilDeviceReadiness(device.Id, nb.maxTimeout, vFunction, nbi)
208 assert.Nil(t, err)
209
210 // Now, verify the details of the device. First get the latest update
211 d, err := nbi.GetDevice(getContext(), &voltha.ID{Id: device.Id})
212 assert.Nil(t, err)
213 assert.Equal(t, voltha.AdminState_ENABLED, d.AdminState)
214 assert.Equal(t, voltha.ConnectStatus_REACHABLE, d.ConnectStatus)
215 assert.Equal(t, voltha.OperStatus_ACTIVE, d.OperStatus)
216 assert.Equal(t, d.Type, d.Adapter)
217 assert.NotEqual(t, "", d.MacAddress)
218 assert.NotEqual(t, "", d.SerialNumber)
219
220 if d.Type == "olt_adapter_mock" {
221 assert.Equal(t, true, d.Root)
222 assert.NotEqual(t, "", d.Id)
223 assert.NotEqual(t, "", d.ParentId)
224 assert.Nil(t, d.ProxyAddress)
225 } else if d.Type == "onu_adapter_mock" {
226 assert.Equal(t, false, d.Root)
227 assert.NotEqual(t, uint32(0), d.Vlan)
228 assert.NotEqual(t, "", d.Id)
229 assert.NotEqual(t, "", d.ParentId)
230 assert.NotEqual(t, "", d.ProxyAddress.DeviceId)
231 assert.Equal(t, "olt_adapter_mock", d.ProxyAddress.DeviceType)
khenaidoob64fc8a2019-11-27 15:08:19 -0500232 } else {
khenaidoo67b22152020-03-02 16:01:25 -0500233 assert.Error(t, errors.New("invalid-device-type"))
khenaidoob64fc8a2019-11-27 15:08:19 -0500234 }
khenaidoo67b22152020-03-02 16:01:25 -0500235 assert.Equal(t, 2, len(d.Ports))
236 for _, p := range d.Ports {
237 assert.Equal(t, voltha.AdminState_ENABLED, p.AdminState)
238 assert.Equal(t, voltha.OperStatus_ACTIVE, p.OperStatus)
239 if p.Type == voltha.Port_ETHERNET_NNI || p.Type == voltha.Port_ETHERNET_UNI {
240 assert.Equal(t, 0, len(p.Peers))
241 } else if p.Type == voltha.Port_PON_OLT {
242 assert.Equal(t, nb.numONUPerOLT, len(p.Peers))
243 assert.Equal(t, uint32(1), p.PortNo)
244 } else if p.Type == voltha.Port_PON_ONU {
245 assert.Equal(t, 1, len(p.Peers))
246 assert.Equal(t, uint32(1), p.PortNo)
247 } else {
248 assert.Error(t, errors.New("invalid-port"))
249 }
250 }
251 wg.Done()
252 }(&wg, device)
khenaidoob64fc8a2019-11-27 15:08:19 -0500253 }
khenaidoo67b22152020-03-02 16:01:25 -0500254 wg.Wait()
khenaidoob64fc8a2019-11-27 15:08:19 -0500255}
256
Kent Hagerman2b216042020-04-03 18:28:56 -0400257func (nb *NBTest) getADevice(rootDevice bool, nbi *NBIHandler) (*voltha.Device, error) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500258 devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
259 if err != nil {
260 return nil, err
261 }
262 for _, d := range devices.Items {
263 if d.Root == rootDevice {
264 return d, nil
265 }
266 }
267 return nil, status.Errorf(codes.NotFound, "%v device not found", rootDevice)
268}
269
Kent Hagerman2b216042020-04-03 18:28:56 -0400270func (nb *NBTest) testCoreWithoutData(t *testing.T, nbi *NBIHandler) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500271 lds, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
272 assert.Nil(t, err)
273 assert.NotNil(t, lds)
274 assert.Equal(t, 0, len(lds.Items))
275 devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
276 assert.Nil(t, err)
277 assert.NotNil(t, devices)
278 assert.Equal(t, 0, len(devices.Items))
279 adapters, err := nbi.ListAdapters(getContext(), &empty.Empty{})
khenaidoo442e7c72020-03-10 16:13:48 -0400280 assert.Equal(t, 0, len(adapters.Items))
khenaidoob64fc8a2019-11-27 15:08:19 -0500281 assert.Nil(t, err)
282 assert.NotNil(t, adapters)
khenaidoob64fc8a2019-11-27 15:08:19 -0500283}
284
Kent Hagerman2b216042020-04-03 18:28:56 -0400285func (nb *NBTest) testAdapterRegistration(t *testing.T, nbi *NBIHandler) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500286 adapters, err := nbi.ListAdapters(getContext(), &empty.Empty{})
287 assert.Nil(t, err)
288 assert.NotNil(t, adapters)
289 assert.Equal(t, 2, len(adapters.Items))
290 for _, a := range adapters.Items {
291 switch a.Id {
292 case nb.oltAdapterName:
293 assert.Equal(t, "Voltha-olt", a.Vendor)
294 case nb.onuAdapterName:
295 assert.Equal(t, "Voltha-onu", a.Vendor)
296 default:
Girish Kumarf56a4682020-03-20 20:07:46 +0000297 logger.Fatal("unregistered-adapter", a.Id)
khenaidoob64fc8a2019-11-27 15:08:19 -0500298 }
299 }
300 deviceTypes, err := nbi.ListDeviceTypes(getContext(), &empty.Empty{})
301 assert.Nil(t, err)
302 assert.NotNil(t, deviceTypes)
303 assert.Equal(t, 2, len(deviceTypes.Items))
304 for _, dt := range deviceTypes.Items {
305 switch dt.Id {
306 case nb.oltAdapterName:
307 assert.Equal(t, nb.oltAdapterName, dt.Adapter)
308 assert.Equal(t, false, dt.AcceptsBulkFlowUpdate)
309 assert.Equal(t, true, dt.AcceptsAddRemoveFlowUpdates)
310 case nb.onuAdapterName:
311 assert.Equal(t, nb.onuAdapterName, dt.Adapter)
312 assert.Equal(t, false, dt.AcceptsBulkFlowUpdate)
313 assert.Equal(t, true, dt.AcceptsAddRemoveFlowUpdates)
314 default:
Girish Kumarf56a4682020-03-20 20:07:46 +0000315 logger.Fatal("invalid-device-type", dt.Id)
khenaidoob64fc8a2019-11-27 15:08:19 -0500316 }
317 }
318}
319
Kent Hagerman2b216042020-04-03 18:28:56 -0400320func (nb *NBTest) testCreateDevice(t *testing.T, nbi *NBIHandler) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500321 // Create a valid device
322 oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
323 assert.Nil(t, err)
324 assert.NotNil(t, oltDevice)
325 device, err := nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
326 assert.Nil(t, err)
327 assert.NotNil(t, device)
328 assert.Equal(t, oltDevice.String(), device.String())
329
330 // Try to create the same device
331 _, err = nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
332 assert.NotNil(t, err)
Kent Hagerman45a13e42020-04-13 12:23:50 -0400333 assert.Equal(t, "device is already pre-provisioned", err.Error())
khenaidoob64fc8a2019-11-27 15:08:19 -0500334
335 // Try to create a device with invalid data
336 _, err = nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName})
337 assert.NotNil(t, err)
Thomas Lee Se5a44012019-11-07 20:32:24 +0530338 assert.Equal(t, "no-device-info-present; MAC or HOSTIP&PORT", err.Error())
khenaidoob64fc8a2019-11-27 15:08:19 -0500339
340 // Ensure we only have 1 device in the Core
341 devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
342 assert.Nil(t, err)
343 assert.NotNil(t, devices)
344 assert.Equal(t, 1, len(devices.Items))
345 assert.Equal(t, oltDevice.String(), devices.Items[0].String())
346
347 //Remove the device
348 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
349 assert.Nil(t, err)
350
351 //Ensure there are no devices in the Core now - wait until condition satisfied or timeout
352 var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
353 return devices != nil && len(devices.Items) == 0
354 }
khenaidoo442e7c72020-03-10 16:13:48 -0400355 err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
khenaidoob64fc8a2019-11-27 15:08:19 -0500356 assert.Nil(t, err)
357}
358
Kent Hagerman2b216042020-04-03 18:28:56 -0400359func (nb *NBTest) testEnableDevice(t *testing.T, nbi *NBIHandler) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500360 // Create a device that has no adapter registered
361 oltDeviceNoAdapter, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: "noAdapterRegistered", MacAddress: "aa:bb:cc:cc:ee:ff"})
362 assert.Nil(t, err)
363 assert.NotNil(t, oltDeviceNoAdapter)
364
365 // Try to enable the oltDevice and check the error message
366 _, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDeviceNoAdapter.Id})
367 assert.NotNil(t, err)
Kent Hagerman45a13e42020-04-13 12:23:50 -0400368 assert.Equal(t, "adapter-not-registered-for-device-type noAdapterRegistered", err.Error())
khenaidoob64fc8a2019-11-27 15:08:19 -0500369
370 //Remove the device
371 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDeviceNoAdapter.Id})
372 assert.Nil(t, err)
373
374 //Ensure there are no devices in the Core now - wait until condition satisfied or timeout
375 var vdFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
376 return devices != nil && len(devices.Items) == 0
377 }
khenaidoo442e7c72020-03-10 16:13:48 -0400378 err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vdFunction)
khenaidoob64fc8a2019-11-27 15:08:19 -0500379 assert.Nil(t, err)
380
khenaidoo67b22152020-03-02 16:01:25 -0500381 // Create a logical device monitor will automatically send trap and eapol flows to the devices being enables
382 var wg sync.WaitGroup
383 wg.Add(1)
khenaidoo8b4abbf2020-04-24 17:04:30 -0400384 go nb.monitorLogicalDevice(t, nbi, 1, nb.numONUPerOLT, &wg, false, false)
khenaidoo67b22152020-03-02 16:01:25 -0500385
khenaidoob64fc8a2019-11-27 15:08:19 -0500386 // Create the device with valid data
387 oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
388 assert.Nil(t, err)
389 assert.NotNil(t, oltDevice)
390
391 // Verify oltDevice exist in the core
392 devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
393 assert.Nil(t, err)
394 assert.Equal(t, 1, len(devices.Items))
395 assert.Equal(t, oltDevice.Id, devices.Items[0].Id)
396
397 // Enable the oltDevice
398 _, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
399 assert.Nil(t, err)
400
401 // Wait for the logical device to be in the ready state
402 var vldFunction isLogicalDeviceConditionSatisfied = func(ld *voltha.LogicalDevice) bool {
403 return ld != nil && len(ld.Ports) == nb.numONUPerOLT+1
404 }
405 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
406 assert.Nil(t, err)
407
408 // Verify that the devices have been setup correctly
409 nb.verifyDevices(t, nbi)
410
411 // Get latest oltDevice data
412 oltDevice, err = nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
413 assert.Nil(t, err)
414
415 // Verify that the logical device has been setup correctly
416 nb.verifyLogicalDevices(t, oltDevice, nbi)
khenaidoo67b22152020-03-02 16:01:25 -0500417
418 // Wait until all flows has been sent to the devices successfully
419 wg.Wait()
khenaidoob64fc8a2019-11-27 15:08:19 -0500420}
421
Kent Hagerman2b216042020-04-03 18:28:56 -0400422func (nb *NBTest) testDisableAndReEnableRootDevice(t *testing.T, nbi *NBIHandler) {
khenaidoob64fc8a2019-11-27 15:08:19 -0500423 //Get an OLT device
424 oltDevice, err := nb.getADevice(true, nbi)
425 assert.Nil(t, err)
426 assert.NotNil(t, oltDevice)
427
428 // Disable the oltDevice
429 _, err = nbi.DisableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
430 assert.Nil(t, err)
431
432 // Wait for the old device to be disabled
433 var vdFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
434 return device.AdminState == voltha.AdminState_DISABLED && device.OperStatus == voltha.OperStatus_UNKNOWN
435 }
436 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
437 assert.Nil(t, err)
438
439 // Verify that all onu devices are disabled as well
Kent Hagerman2b216042020-04-03 18:28:56 -0400440 onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
khenaidoob64fc8a2019-11-27 15:08:19 -0500441 assert.Nil(t, err)
442 for _, onu := range onuDevices.Items {
443 err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
444 assert.Nil(t, err)
445 }
446
447 // Wait for the logical device to satisfy the expected condition
448 var vlFunction isLogicalDeviceConditionSatisfied = func(ld *voltha.LogicalDevice) bool {
khenaidoo67b22152020-03-02 16:01:25 -0500449 if ld == nil {
450 return false
451 }
khenaidoob64fc8a2019-11-27 15:08:19 -0500452 for _, lp := range ld.Ports {
453 if (lp.OfpPort.Config&uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
454 lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LINK_DOWN) {
455 return false
456 }
457 }
458 return true
459 }
460 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
461 assert.Nil(t, err)
462
463 // Reenable the oltDevice
464 _, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
465 assert.Nil(t, err)
466
467 // Wait for the old device to be enabled
468 vdFunction = func(device *voltha.Device) bool {
469 return device.AdminState == voltha.AdminState_ENABLED && device.OperStatus == voltha.OperStatus_ACTIVE
470 }
471 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
472 assert.Nil(t, err)
473
474 // Verify that all onu devices are enabled as well
Kent Hagerman2b216042020-04-03 18:28:56 -0400475 onuDevices, err = nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
khenaidoob64fc8a2019-11-27 15:08:19 -0500476 assert.Nil(t, err)
477 for _, onu := range onuDevices.Items {
478 err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
479 assert.Nil(t, err)
480 }
481
482 // Wait for the logical device to satisfy the expected condition
483 vlFunction = func(ld *voltha.LogicalDevice) bool {
khenaidoo67b22152020-03-02 16:01:25 -0500484 if ld == nil {
485 return false
486 }
khenaidoob64fc8a2019-11-27 15:08:19 -0500487 for _, lp := range ld.Ports {
488 if (lp.OfpPort.Config&^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
489 lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LIVE) {
490 return false
491 }
492 }
493 return true
494 }
495 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
496 assert.Nil(t, err)
497}
498
Kent Hagerman2b216042020-04-03 18:28:56 -0400499func (nb *NBTest) testDisableAndDeleteAllDevice(t *testing.T, nbi *NBIHandler) {
khenaidoo93d5a3d2020-01-15 12:37:05 -0500500 //Get an OLT device
501 oltDevice, err := nb.getADevice(true, nbi)
502 assert.Nil(t, err)
503 assert.NotNil(t, oltDevice)
504
505 // Disable the oltDevice
506 _, err = nbi.DisableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
507 assert.Nil(t, err)
508
509 // Wait for the olt device to be disabled
510 var vdFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
511 return device.AdminState == voltha.AdminState_DISABLED && device.OperStatus == voltha.OperStatus_UNKNOWN
512 }
513 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
514 assert.Nil(t, err)
515
516 // Verify that all onu devices are disabled as well
Kent Hagerman2b216042020-04-03 18:28:56 -0400517 onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
khenaidoo93d5a3d2020-01-15 12:37:05 -0500518 assert.Nil(t, err)
519 for _, onu := range onuDevices.Items {
520 err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
521 assert.Nil(t, err)
522 }
523
524 // Delete the oltDevice
525 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
526 assert.Nil(t, err)
527
528 var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
529 return devices != nil && len(devices.Items) == 0
530 }
531 err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
532 assert.Nil(t, err)
533
534 // Wait for absence of logical device
535 var vlFunction isLogicalDevicesConditionSatisfied = func(lds *voltha.LogicalDevices) bool {
536 return lds != nil && len(lds.Items) == 0
537 }
538
539 err = waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
540 assert.Nil(t, err)
541}
khenaidoo8b4abbf2020-04-24 17:04:30 -0400542
543func (nb *NBTest) deleteAllDevices(t *testing.T, nbi *NBIHandler) {
khenaidoo0db4c812020-05-27 15:27:30 -0400544 devices, _ := nbi.ListDevices(getContext(), &empty.Empty{})
545 if len(devices.Items) == 0 {
546 // Nothing to do
547 return
548 }
khenaidoo8b4abbf2020-04-24 17:04:30 -0400549 //Get an OLT device
550 oltDevice, err := nb.getADevice(true, nbi)
551 assert.Nil(t, err)
552 assert.NotNil(t, oltDevice)
553
554 // Delete the oltDevice
555 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
556 assert.Nil(t, err)
557
558 // Wait for all devices to be deleted
559 vFunction := func(devices *voltha.Devices) bool {
560 return devices != nil && len(devices.Items) == 0
561 }
562 err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
563 assert.Nil(t, err)
564
565 // Wait for absence of logical device
566 vlFunction := func(lds *voltha.LogicalDevices) bool {
567 return lds != nil && len(lds.Items) == 0
568 }
569
570 err = waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
571 assert.Nil(t, err)
572}
573
Kent Hagerman2b216042020-04-03 18:28:56 -0400574func (nb *NBTest) testEnableAndDeleteAllDevice(t *testing.T, nbi *NBIHandler) {
Chaitrashree G S543df3e2020-02-24 22:36:54 -0500575 //Create the device with valid data
576 oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
577 assert.Nil(t, err)
578 assert.NotNil(t, oltDevice)
579
580 //Get an OLT device
581 oltDevice, err = nb.getADevice(true, nbi)
582 assert.Nil(t, err)
583 assert.NotNil(t, oltDevice)
584
585 // Enable the oltDevice
586 _, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
587 assert.Nil(t, err)
588
589 // Wait for the logical device to be in the ready state
590 var vldFunction isLogicalDeviceConditionSatisfied = func(ld *voltha.LogicalDevice) bool {
591 return ld != nil && len(ld.Ports) == nb.numONUPerOLT+1
592 }
593 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
594 assert.Nil(t, err)
595
596 //Get all child devices
Kent Hagerman2b216042020-04-03 18:28:56 -0400597 onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
Chaitrashree G S543df3e2020-02-24 22:36:54 -0500598 assert.Nil(t, err)
599
600 // Wait for the all onu devices to be enabled
601 var vdFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
602 return device.AdminState == voltha.AdminState_ENABLED
603 }
604 for _, onu := range onuDevices.Items {
605 err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunction, nbi)
606 assert.Nil(t, err)
607 }
Chaitrashree G Se8ad0202020-02-27 18:48:00 -0500608 // Wait for each onu device to get deleted
609 var vdFunc isDeviceConditionSatisfied = func(device *voltha.Device) bool {
610 return device == nil
611 }
612
Chaitrashree G S543df3e2020-02-24 22:36:54 -0500613 // Delete the onuDevice
614 for _, onu := range onuDevices.Items {
615 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: onu.Id})
616 assert.Nil(t, err)
Chaitrashree G Se8ad0202020-02-27 18:48:00 -0500617 err = waitUntilDeviceReadiness(onu.Id, nb.maxTimeout, vdFunc, nbi)
618 assert.Nil(t, err)
Chaitrashree G S543df3e2020-02-24 22:36:54 -0500619 }
Chaitrashree G Se8ad0202020-02-27 18:48:00 -0500620
Chaitrashree G S543df3e2020-02-24 22:36:54 -0500621 // Disable the oltDevice
622 _, err = nbi.DisableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
623 assert.Nil(t, err)
624
625 // Wait for the olt device to be disabled
626 var vFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
627 return device.AdminState == voltha.AdminState_DISABLED && device.OperStatus == voltha.OperStatus_UNKNOWN
628 }
629 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vFunction, nbi)
630 assert.Nil(t, err)
631
632 // Delete the oltDevice
633 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
634 assert.Nil(t, err)
635
636 var vFunc isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
637 return devices != nil && len(devices.Items) == 0
638 }
639 err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunc)
640 assert.Nil(t, err)
641}
Kent Hagerman2b216042020-04-03 18:28:56 -0400642func (nb *NBTest) testDisableAndEnablePort(t *testing.T, nbi *NBIHandler) {
kesavandbc2d1622020-01-21 00:42:01 -0500643 //Get an OLT device
644 var cp *voltha.Port
645 oltDevice, err := nb.getADevice(true, nbi)
646 assert.Nil(t, err)
647 assert.NotNil(t, oltDevice)
648
649 for _, cp = range oltDevice.Ports {
650 if cp.Type == voltha.Port_PON_OLT {
651 break
652 }
653
654 }
655 assert.NotNil(t, cp)
656 cp.DeviceId = oltDevice.Id
657
658 // Disable the NW Port of oltDevice
659 _, err = nbi.DisablePort(getContext(), cp)
660 assert.Nil(t, err)
661 // Wait for the olt device Port to be disabled
662 var vdFunction isDeviceConditionSatisfied = func(device *voltha.Device) bool {
663 for _, port := range device.Ports {
664 if port.PortNo == cp.PortNo {
665 return port.AdminState == voltha.AdminState_DISABLED
666 }
667 }
668 return false
669 }
670 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
671 assert.Nil(t, err)
672 // Wait for the logical device to satisfy the expected condition
673 var vlFunction = func(ld *voltha.LogicalDevice) bool {
khenaidoo67b22152020-03-02 16:01:25 -0500674 if ld == nil {
675 return false
676 }
kesavandbc2d1622020-01-21 00:42:01 -0500677 for _, lp := range ld.Ports {
678 if (lp.OfpPort.Config&^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
679 lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LIVE) {
680 return false
681 }
682 }
683 return true
684 }
685 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
686 assert.Nil(t, err)
687
688 // Enable the NW Port of oltDevice
689 _, err = nbi.EnablePort(getContext(), cp)
690 assert.Nil(t, err)
691
692 // Wait for the olt device Port to be enabled
693 vdFunction = func(device *voltha.Device) bool {
694 for _, port := range device.Ports {
695 if port.PortNo == cp.PortNo {
696 return port.AdminState == voltha.AdminState_ENABLED
697 }
698 }
699 return false
700 }
701 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
702 assert.Nil(t, err)
703 // Wait for the logical device to satisfy the expected condition
704 vlFunction = func(ld *voltha.LogicalDevice) bool {
khenaidoo67b22152020-03-02 16:01:25 -0500705 if ld == nil {
706 return false
707 }
kesavandbc2d1622020-01-21 00:42:01 -0500708 for _, lp := range ld.Ports {
709 if (lp.OfpPort.Config&^uint32(ofp.OfpPortConfig_OFPPC_PORT_DOWN) != lp.OfpPort.Config) ||
710 lp.OfpPort.State != uint32(ofp.OfpPortState_OFPPS_LIVE) {
711 return false
712 }
713 }
714 return true
715 }
716 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction)
717 assert.Nil(t, err)
718
719 // Disable a non-PON port
720 for _, cp = range oltDevice.Ports {
721 if cp.Type != voltha.Port_PON_OLT {
722 break
723 }
724
725 }
726 assert.NotNil(t, cp)
727 cp.DeviceId = oltDevice.Id
728
729 // Disable the NW Port of oltDevice
730 _, err = nbi.DisablePort(getContext(), cp)
731 assert.NotNil(t, err)
732
733}
khenaidoo93d5a3d2020-01-15 12:37:05 -0500734
Kent Hagerman2b216042020-04-03 18:28:56 -0400735func (nb *NBTest) testDeviceRebootWhenOltIsEnabled(t *testing.T, nbi *NBIHandler) {
Girish Gowdra408cd962020-03-11 14:31:31 -0700736 //Get an OLT device
737 oltDevice, err := nb.getADevice(true, nbi)
738 assert.Nil(t, err)
739 assert.NotNil(t, oltDevice)
740 assert.Equal(t, oltDevice.ConnectStatus, voltha.ConnectStatus_REACHABLE)
741 assert.Equal(t, oltDevice.AdminState, voltha.AdminState_ENABLED)
742
743 // Verify that we have one or more ONUs to start with
Kent Hagerman2b216042020-04-03 18:28:56 -0400744 onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
Girish Gowdra408cd962020-03-11 14:31:31 -0700745 assert.Nil(t, err)
746 assert.NotNil(t, onuDevices)
747 assert.Greater(t, len(onuDevices.Items), 0)
748
749 // Reboot the OLT and very that Connection Status goes to UNREACHABLE and operation status to UNKNOWN
750 _, err = nbi.RebootDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
751 assert.Nil(t, err)
752
753 var vlFunction0 = func(d *voltha.Device) bool {
754 return d.ConnectStatus == voltha.ConnectStatus_UNREACHABLE && d.OperStatus == voltha.OperStatus_UNKNOWN
755 }
756
757 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vlFunction0, nbi)
758 assert.Nil(t, err)
759
760 // Wait for the logical device to satisfy the expected condition
761 var vlFunction1 = func(ld *voltha.LogicalDevice) bool {
762 return ld == nil
763 }
764
765 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction1)
766 assert.Nil(t, err)
767
768 // Wait for the device to satisfy the expected condition (device does not have flows)
769 var vlFunction2 = func(d *voltha.Device) bool {
770 var deviceFlows *ofp.Flows
771 var err error
772 if deviceFlows, err = nbi.ListDeviceFlows(getContext(), &voltha.ID{Id: d.Id}); err != nil {
773 return false
774 }
775 return len(deviceFlows.Items) == 0
776 }
777
778 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vlFunction2, nbi)
779 assert.Nil(t, err)
780
781 // Wait for the device to satisfy the expected condition (there are no child devices)
782 var vlFunction3 = func(d *voltha.Device) bool {
783 var devices *voltha.Devices
784 var err error
785 if devices, err = nbi.ListDevices(getContext(), nil); err != nil {
786 return false
787 }
788 for _, device := range devices.Items {
789 if device.ParentId == d.Id {
790 // We have a child device still left
791 return false
792 }
793 }
794 return true
795 }
796
797 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vlFunction3, nbi)
798 assert.Nil(t, err)
799
800 // Update the OLT Connection Status to REACHABLE and operation status to ACTIVE
801 // Normally, in a real adapter this happens after connection regain via a heartbeat mechanism with real hardware
Kent Hagerman45a13e42020-04-13 12:23:50 -0400802 err = nbi.UpdateDeviceStatus(getContext(), oltDevice.Id, voltha.OperStatus_ACTIVE, voltha.ConnectStatus_REACHABLE)
Girish Gowdra408cd962020-03-11 14:31:31 -0700803 assert.Nil(t, err)
804
805 // Verify the device connection and operation states
806 oltDevice, err = nb.getADevice(true, nbi)
807 assert.Nil(t, err)
808 assert.NotNil(t, oltDevice)
809 assert.Equal(t, oltDevice.ConnectStatus, voltha.ConnectStatus_REACHABLE)
810 assert.Equal(t, oltDevice.AdminState, voltha.AdminState_ENABLED)
811
812 // Wait for the logical device to satisfy the expected condition
813 var vlFunction4 = func(ld *voltha.LogicalDevice) bool {
814 return ld != nil
815 }
816 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vlFunction4)
817 assert.Nil(t, err)
818
819 // Verify that logical device is created again
820 logicalDevices, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
821 assert.Nil(t, err)
822 assert.NotNil(t, logicalDevices)
823 assert.Equal(t, 1, len(logicalDevices.Items))
824
825 // Verify that we have no ONUs left
Kent Hagerman2b216042020-04-03 18:28:56 -0400826 onuDevices, err = nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
Girish Gowdra408cd962020-03-11 14:31:31 -0700827 assert.Nil(t, err)
828 assert.NotNil(t, onuDevices)
829 assert.Equal(t, 0, len(onuDevices.Items))
830}
831
Kent Hagerman2b216042020-04-03 18:28:56 -0400832func (nb *NBTest) testStartOmciTestAction(t *testing.T, nbi *NBIHandler) {
Scott Baker432f9be2020-03-26 11:56:30 -0700833 // -----------------------------------------------------------------------
834 // SubTest 1: Omci test action should fail due to nonexistent device id
835
836 request := &voltha.OmciTestRequest{Id: "123", Uuid: "456"}
837 _, err := nbi.StartOmciTestAction(getContext(), request)
838 assert.NotNil(t, err)
839 assert.Equal(t, "rpc error: code = NotFound desc = 123", err.Error())
840
841 // -----------------------------------------------------------------------
842 // SubTest 2: Error should be returned for device with no adapter registered
843
844 // Create a device that has no adapter registered
845 deviceNoAdapter, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: "noAdapterRegisteredOmciTest", MacAddress: "aa:bb:cc:cc:ee:01"})
846 assert.Nil(t, err)
847 assert.NotNil(t, deviceNoAdapter)
848
849 // Omci test action should fail due to nonexistent adapter
850 request = &voltha.OmciTestRequest{Id: deviceNoAdapter.Id, Uuid: "456"}
851 _, err = nbi.StartOmciTestAction(getContext(), request)
852 assert.NotNil(t, err)
Kent Hagerman45a13e42020-04-13 12:23:50 -0400853 assert.Equal(t, "adapter-not-registered-for-device-type noAdapterRegisteredOmciTest", err.Error())
Scott Baker432f9be2020-03-26 11:56:30 -0700854
855 //Remove the device
856 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: deviceNoAdapter.Id})
857 assert.Nil(t, err)
858
859 //Ensure there are no devices in the Core now - wait until condition satisfied or timeout
860 var vFunction isDevicesConditionSatisfied = func(devices *voltha.Devices) bool {
861 return devices != nil && len(devices.Items) == 0
862 }
863 err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
864 assert.Nil(t, err)
865
866 // -----------------------------------------------------------------------
867 // SubTest 3: Omci test action should succeed on valid ONU
868
869 // Create the device with valid data
870 oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
871 assert.Nil(t, err)
872 assert.NotNil(t, oltDevice)
873
874 // Verify oltDevice exist in the core
875 devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
876 assert.Nil(t, err)
877 assert.Equal(t, 1, len(devices.Items))
878 assert.Equal(t, oltDevice.Id, devices.Items[0].Id)
879
880 // Enable the oltDevice
881 _, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
882 assert.Nil(t, err)
883
884 // Wait for the logical device to be in the ready state
885 var vldFunction isLogicalDeviceConditionSatisfied = func(ld *voltha.LogicalDevice) bool {
886 return ld != nil && len(ld.Ports) == nb.numONUPerOLT+1
887 }
888 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
889 assert.Nil(t, err)
890
891 // Wait for the olt device to be enabled
892 vdFunction := func(device *voltha.Device) bool {
893 return device.AdminState == voltha.AdminState_ENABLED && device.OperStatus == voltha.OperStatus_ACTIVE
894 }
895 err = waitUntilDeviceReadiness(oltDevice.Id, nb.maxTimeout, vdFunction, nbi)
896 assert.Nil(t, err)
897
Kent Hagerman2b216042020-04-03 18:28:56 -0400898 onuDevices, err := nb.deviceMgr.GetAllChildDevices(getContext(), oltDevice.Id)
Scott Baker432f9be2020-03-26 11:56:30 -0700899 assert.Nil(t, err)
900 assert.Greater(t, len(onuDevices.Items), 0)
901
902 onuDevice := onuDevices.Items[0]
903
904 // Omci test action should succeed
905 request = &voltha.OmciTestRequest{Id: onuDevice.Id, Uuid: "456"}
906 resp, err := nbi.StartOmciTestAction(getContext(), request)
907 assert.Nil(t, err)
908 assert.Equal(t, resp.Result, voltha.TestResponse_SUCCESS)
909
910 //Remove the device
911 _, err = nbi.DeleteDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
912 assert.Nil(t, err)
913 //Ensure there are no devices in the Core now - wait until condition satisfied or timeout
914 err = waitUntilConditionForDevices(nb.maxTimeout, nbi, vFunction)
915 assert.Nil(t, err)
916}
917
khenaidoo67b22152020-03-02 16:01:25 -0500918func makeSimpleFlowMod(fa *flows.FlowArgs) *ofp.OfpFlowMod {
919 matchFields := make([]*ofp.OfpOxmField, 0)
920 for _, val := range fa.MatchFields {
921 matchFields = append(matchFields, &ofp.OfpOxmField{Field: &ofp.OfpOxmField_OfbField{OfbField: val}})
922 }
923 return flows.MkSimpleFlowMod(matchFields, fa.Actions, fa.Command, fa.KV)
924}
925
926func createMetadata(cTag int, techProfile int, port int) uint64 {
927 md := 0
928 md = (md | (cTag & 0xFFFF)) << 16
929 md = (md | (techProfile & 0xFFFF)) << 32
930 return uint64(md | (port & 0xFFFFFFFF))
931}
932
khenaidoo8b4abbf2020-04-24 17:04:30 -0400933func (nb *NBTest) verifyLogicalDeviceFlowCount(t *testing.T, nbi *NBIHandler, numNNIPorts int, numUNIPorts int, flowAddFail bool) {
khenaidoo67b22152020-03-02 16:01:25 -0500934 expectedNumFlows := numNNIPorts*3 + numNNIPorts*numUNIPorts
khenaidoo8b4abbf2020-04-24 17:04:30 -0400935 if flowAddFail {
936 expectedNumFlows = 0
937 }
938 // Wait for logical device to have the flows (or none
khenaidoo67b22152020-03-02 16:01:25 -0500939 var vlFunction isLogicalDevicesConditionSatisfied = func(lds *voltha.LogicalDevices) bool {
Mahir Gunyeladdb66a2020-04-29 18:08:50 -0700940 flows, _ := nbi.ListLogicalDeviceFlows(getContext(), &voltha.ID{Id: lds.Items[0].Id})
941 return lds != nil && len(lds.Items) == 1 && len(flows.Items) == expectedNumFlows
khenaidoo67b22152020-03-02 16:01:25 -0500942 }
943 // No timeout implies a success
944 err := waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
945 assert.Nil(t, err)
946}
947
Kent Hagerman2b216042020-04-03 18:28:56 -0400948func (nb *NBTest) sendTrapFlows(t *testing.T, nbi *NBIHandler, logicalDevice *voltha.LogicalDevice, meterID uint64, startingVlan int) (numNNIPorts, numUNIPorts int) {
khenaidoo67b22152020-03-02 16:01:25 -0500949 // Send flows for the parent device
950 var nniPorts []*voltha.LogicalPort
951 var uniPorts []*voltha.LogicalPort
952 for _, p := range logicalDevice.Ports {
953 if p.RootPort {
954 nniPorts = append(nniPorts, p)
955 } else {
956 uniPorts = append(uniPorts, p)
957 }
958 }
959 assert.Equal(t, 1, len(nniPorts))
960 //assert.Greater(t, len(uniPorts), 1 )
961 nniPort := nniPorts[0].OfpPort.PortNo
962 maxInt32 := uint64(0xFFFFFFFF)
963 controllerPortMask := uint32(4294967293) // will result in 4294967293&0x7fffffff => 2147483645 which is the actual controller port
964 var fa *flows.FlowArgs
965 fa = &flows.FlowArgs{
966 KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1},
967 MatchFields: []*ofp.OfpOxmOfbField{
968 flows.InPort(nniPort),
969 flows.EthType(35020),
970 },
971 Actions: []*ofp.OfpAction{
972 flows.Output(controllerPortMask),
973 },
974 }
975 flowLLDP := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDevice.Id}
976 _, err := nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowLLDP)
977 assert.Nil(t, err)
978
979 fa = &flows.FlowArgs{
980 KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1},
981 MatchFields: []*ofp.OfpOxmOfbField{
982 flows.InPort(nniPort),
983 flows.EthType(2048),
984 flows.IpProto(17),
985 flows.UdpSrc(67),
986 flows.UdpDst(68),
987 },
988 Actions: []*ofp.OfpAction{
989 flows.Output(controllerPortMask),
990 },
991 }
992 flowIPV4 := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDevice.Id}
993 _, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowIPV4)
994 assert.Nil(t, err)
995
996 fa = &flows.FlowArgs{
997 KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1},
998 MatchFields: []*ofp.OfpOxmOfbField{
999 flows.InPort(nniPort),
1000 flows.EthType(34525),
1001 flows.IpProto(17),
1002 flows.UdpSrc(546),
1003 flows.UdpDst(547),
1004 },
1005 Actions: []*ofp.OfpAction{
1006 flows.Output(controllerPortMask),
1007 },
1008 }
1009 flowIPV6 := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDevice.Id}
1010 _, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowIPV6)
1011 assert.Nil(t, err)
1012
1013 return len(nniPorts), len(uniPorts)
1014}
1015
Kent Hagerman2b216042020-04-03 18:28:56 -04001016func (nb *NBTest) sendEAPFlows(t *testing.T, nbi *NBIHandler, logicalDeviceID string, port *ofp.OfpPort, vlan int, meterID uint64) {
khenaidoo67b22152020-03-02 16:01:25 -05001017 maxInt32 := uint64(0xFFFFFFFF)
1018 controllerPortMask := uint32(4294967293) // will result in 4294967293&0x7fffffff => 2147483645 which is the actual controller port
1019 fa := &flows.FlowArgs{
1020 KV: flows.OfpFlowModArgs{"priority": 10000, "buffer_id": maxInt32, "out_port": maxInt32, "out_group": maxInt32, "flags": 1, "write_metadata": createMetadata(vlan, 64, 0), "meter_id": meterID},
1021 MatchFields: []*ofp.OfpOxmOfbField{
1022 flows.InPort(port.PortNo),
1023 flows.EthType(34958),
1024 flows.VlanVid(8187),
1025 },
1026 Actions: []*ofp.OfpAction{
1027 flows.Output(controllerPortMask),
1028 },
1029 }
1030 flowEAP := ofp.FlowTableUpdate{FlowMod: makeSimpleFlowMod(fa), Id: logicalDeviceID}
khenaidoo0db4c812020-05-27 15:27:30 -04001031 maxTries := 3
1032 var err error
1033 for {
1034 if _, err = nbi.UpdateLogicalDeviceFlowTable(getContext(), &flowEAP); err == nil {
1035 if maxTries < 3 {
1036 t.Log("Re-sending EAPOL flow succeeded for port:", port)
1037 }
1038 break
1039 }
1040 t.Log("Sending EAPOL flows fail:", err)
1041 time.Sleep(50 * time.Millisecond)
1042 maxTries--
1043 if maxTries == 0 {
1044 break
1045 }
1046 }
khenaidoo67b22152020-03-02 16:01:25 -05001047 assert.Nil(t, err)
1048}
1049
khenaidoo0db4c812020-05-27 15:27:30 -04001050func (nb *NBTest) monitorLogicalDevice(t *testing.T, nbi *NBIHandler, numNNIPorts int, numUNIPorts int, wg *sync.WaitGroup, flowAddFail bool, flowDeleteFail bool) {
khenaidoo67b22152020-03-02 16:01:25 -05001051 defer wg.Done()
khenaidoo67b22152020-03-02 16:01:25 -05001052
1053 // Clear any existing flows on the adapters
1054 nb.oltAdapter.ClearFlows()
1055 nb.onuAdapter.ClearFlows()
1056
khenaidoo8b4abbf2020-04-24 17:04:30 -04001057 // Set the adapter actions on flow addition/deletion
khenaidoo0db4c812020-05-27 15:27:30 -04001058 nb.oltAdapter.SetFlowAction(flowAddFail, flowDeleteFail)
1059 nb.onuAdapter.SetFlowAction(flowAddFail, flowDeleteFail)
khenaidoo8b4abbf2020-04-24 17:04:30 -04001060
khenaidoo67b22152020-03-02 16:01:25 -05001061 // Wait until a logical device is ready
1062 var vlFunction isLogicalDevicesConditionSatisfied = func(lds *voltha.LogicalDevices) bool {
1063 if lds == nil || len(lds.Items) != 1 {
1064 return false
1065 }
1066 // Ensure there are both NNI ports and at least one UNI port on the logical device
1067 ld := lds.Items[0]
1068 nniPort := false
1069 uniPort := false
1070 for _, p := range ld.Ports {
1071 nniPort = nniPort || p.RootPort == true
1072 uniPort = uniPort || p.RootPort == false
1073 if nniPort && uniPort {
1074 return true
1075 }
1076 }
1077 return false
1078 }
1079 err := waitUntilConditionForLogicalDevices(nb.maxTimeout, nbi, vlFunction)
1080 assert.Nil(t, err)
1081
1082 logicalDevices, err := nbi.ListLogicalDevices(getContext(), &empty.Empty{})
1083 assert.Nil(t, err)
1084 assert.NotNil(t, logicalDevices)
1085 assert.Equal(t, 1, len(logicalDevices.Items))
1086
1087 logicalDevice := logicalDevices.Items[0]
1088 meterID := rand.Uint32()
1089
1090 // Add a meter to the logical device
1091 meterMod := &ofp.OfpMeterMod{
1092 Command: ofp.OfpMeterModCommand_OFPMC_ADD,
1093 Flags: rand.Uint32(),
1094 MeterId: meterID,
1095 Bands: []*ofp.OfpMeterBandHeader{
1096 {Type: ofp.OfpMeterBandType_OFPMBT_EXPERIMENTER,
1097 Rate: rand.Uint32(),
1098 BurstSize: rand.Uint32(),
1099 Data: nil,
1100 },
1101 },
1102 }
1103 _, err = nbi.UpdateLogicalDeviceMeterTable(getContext(), &ofp.MeterModUpdate{Id: logicalDevice.Id, MeterMod: meterMod})
1104 assert.Nil(t, err)
1105
1106 // Send initial set of Trap flows
1107 startingVlan := 4091
1108 nb.sendTrapFlows(t, nbi, logicalDevice, uint64(meterID), startingVlan)
1109
1110 // Listen for port events
khenaidoo442e7c72020-03-10 16:13:48 -04001111 start := time.Now()
Girish Gowdra408cd962020-03-11 14:31:31 -07001112 processedNniLogicalPorts := 0
1113 processedUniLogicalPorts := 0
1114
Kent Hagerman45a13e42020-04-13 12:23:50 -04001115 for event := range nbi.GetChangeEventsQueueForTest() {
khenaidoo67b22152020-03-02 16:01:25 -05001116 startingVlan++
1117 if portStatus, ok := (event.Event).(*ofp.ChangeEvent_PortStatus); ok {
1118 ps := portStatus.PortStatus
1119 if ps.Reason == ofp.OfpPortReason_OFPPR_ADD {
khenaidoo67b22152020-03-02 16:01:25 -05001120 if ps.Desc.PortNo >= uint32(nb.startingUNIPortNo) {
Girish Gowdra408cd962020-03-11 14:31:31 -07001121 processedUniLogicalPorts++
khenaidoo67b22152020-03-02 16:01:25 -05001122 nb.sendEAPFlows(t, nbi, logicalDevice.Id, ps.Desc, startingVlan, uint64(meterID))
Girish Gowdra408cd962020-03-11 14:31:31 -07001123 } else {
1124 processedNniLogicalPorts++
khenaidoo67b22152020-03-02 16:01:25 -05001125 }
1126 }
1127 }
Girish Gowdra408cd962020-03-11 14:31:31 -07001128
1129 if processedNniLogicalPorts >= numNNIPorts && processedUniLogicalPorts >= numUNIPorts {
khenaidoo442e7c72020-03-10 16:13:48 -04001130 fmt.Println("Total time to send all flows:", time.Since(start))
khenaidoo67b22152020-03-02 16:01:25 -05001131 break
1132 }
1133 }
1134 //Verify the flow count on the logical device
khenaidoo8b4abbf2020-04-24 17:04:30 -04001135 nb.verifyLogicalDeviceFlowCount(t, nbi, numNNIPorts, numUNIPorts, flowAddFail)
khenaidoo67b22152020-03-02 16:01:25 -05001136
khenaidoo8b4abbf2020-04-24 17:04:30 -04001137 // Wait until all flows have been sent to the OLT adapters (or all failed)
1138 expectedFlowCount := (numNNIPorts * 3) + numNNIPorts*numUNIPorts
1139 if flowAddFail {
1140 expectedFlowCount = 0
1141 }
khenaidoo67b22152020-03-02 16:01:25 -05001142 var oltVFunc isConditionSatisfied = func() bool {
khenaidoo8b4abbf2020-04-24 17:04:30 -04001143 return nb.oltAdapter.GetFlowCount() >= expectedFlowCount
khenaidoo67b22152020-03-02 16:01:25 -05001144 }
1145 err = waitUntilCondition(nb.maxTimeout, nbi, oltVFunc)
1146 assert.Nil(t, err)
1147
khenaidoo8b4abbf2020-04-24 17:04:30 -04001148 // Wait until all flows have been sent to the ONU adapters (or all failed)
1149 expectedFlowCount = numUNIPorts
1150 if flowAddFail {
1151 expectedFlowCount = 0
1152 }
khenaidoo67b22152020-03-02 16:01:25 -05001153 var onuVFunc isConditionSatisfied = func() bool {
khenaidoo8b4abbf2020-04-24 17:04:30 -04001154 return nb.onuAdapter.GetFlowCount() == expectedFlowCount
khenaidoo67b22152020-03-02 16:01:25 -05001155 }
1156 err = waitUntilCondition(nb.maxTimeout, nbi, onuVFunc)
1157 assert.Nil(t, err)
1158}
1159
khenaidoo8b4abbf2020-04-24 17:04:30 -04001160func (nb *NBTest) testFlowAddFailure(t *testing.T, nbi *NBIHandler) {
1161
1162 // Create a logical device monitor will automatically send trap and eapol flows to the devices being enables
1163 var wg sync.WaitGroup
1164 wg.Add(1)
khenaidoo0db4c812020-05-27 15:27:30 -04001165 go nb.monitorLogicalDevice(t, nbi, 1, nb.numONUPerOLT, &wg, true, false)
khenaidoo8b4abbf2020-04-24 17:04:30 -04001166
1167 // Create the device with valid data
1168 oltDevice, err := nbi.CreateDevice(getContext(), &voltha.Device{Type: nb.oltAdapterName, MacAddress: "aa:bb:cc:cc:ee:ee"})
1169 assert.Nil(t, err)
1170 assert.NotNil(t, oltDevice)
1171
1172 // Verify oltDevice exist in the core
1173 devices, err := nbi.ListDevices(getContext(), &empty.Empty{})
1174 assert.Nil(t, err)
1175 assert.Equal(t, 1, len(devices.Items))
1176 assert.Equal(t, oltDevice.Id, devices.Items[0].Id)
1177
1178 // Enable the oltDevice
1179 _, err = nbi.EnableDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
1180 assert.Nil(t, err)
1181
1182 // Wait for the logical device to be in the ready state
1183 var vldFunction isLogicalDeviceConditionSatisfied = func(ld *voltha.LogicalDevice) bool {
1184 return ld != nil && len(ld.Ports) == nb.numONUPerOLT+1
1185 }
1186 err = waitUntilLogicalDeviceReadiness(oltDevice.Id, nb.maxTimeout, nbi, vldFunction)
1187 assert.Nil(t, err)
1188
1189 // Verify that the devices have been setup correctly
1190 nb.verifyDevices(t, nbi)
1191
1192 // Get latest oltDevice data
1193 oltDevice, err = nbi.GetDevice(getContext(), &voltha.ID{Id: oltDevice.Id})
1194 assert.Nil(t, err)
1195
1196 // Verify that the logical device has been setup correctly
1197 nb.verifyLogicalDevices(t, oltDevice, nbi)
1198
1199 // Wait until all flows has been sent to the devices successfully
1200 wg.Wait()
1201}
1202
Matteo Scandolod525ae32020-04-02 17:27:29 -07001203func TestSuiteNbiApiHandler(t *testing.T) {
Kent Hagerman2b216042020-04-03 18:28:56 -04001204 f, err := os.Create("../../../tests/results/profile.cpu")
khenaidoo67b22152020-03-02 16:01:25 -05001205 if err != nil {
Girish Kumarf56a4682020-03-20 20:07:46 +00001206 logger.Fatalf("could not create CPU profile: %v\n ", err)
khenaidoo67b22152020-03-02 16:01:25 -05001207 }
1208 defer f.Close()
1209 runtime.SetBlockProfileRate(1)
1210 runtime.SetMutexProfileFraction(-1)
1211 if err := pprof.StartCPUProfile(f); err != nil {
Girish Kumarf56a4682020-03-20 20:07:46 +00001212 logger.Fatalf("could not start CPU profile: %v\n", err)
khenaidoo67b22152020-03-02 16:01:25 -05001213 }
1214 defer pprof.StopCPUProfile()
1215
khenaidoo442e7c72020-03-10 16:13:48 -04001216 //log.SetPackageLogLevel("github.com/opencord/voltha-go/rw_core/core", log.DebugLevel)
1217
khenaidoob64fc8a2019-11-27 15:08:19 -05001218 nb := newNBTest()
1219 assert.NotNil(t, nb)
1220
1221 defer nb.stopAll()
1222
1223 // Start the Core
1224 nb.startCore(false)
1225
1226 // Set the grpc API interface - no grpc server is running in unit test
Kent Hagerman45a13e42020-04-13 12:23:50 -04001227 nbi := NewNBIHandler(nb.deviceMgr, nb.logicalDeviceMgr, nb.adapterMgr)
khenaidoob64fc8a2019-11-27 15:08:19 -05001228
1229 // 1. Basic test with no data in Core
1230 nb.testCoreWithoutData(t, nbi)
1231
1232 // Create/register the adapters
Mahir Gunyel03de0d32020-06-03 01:36:59 -07001233 nb.oltAdapter, nb.onuAdapter = tst.CreateAndregisterAdapters(t, nb.kClient, nb.coreInstanceID, nb.oltAdapterName, nb.onuAdapterName, nb.adapterMgr)
1234 nb.numONUPerOLT = nb.oltAdapter.GetNumONUPerOLT()
1235 nb.startingUNIPortNo = nb.oltAdapter.GetStartingUNIPortNo()
khenaidoob64fc8a2019-11-27 15:08:19 -05001236
1237 // 2. Test adapter registration
1238 nb.testAdapterRegistration(t, nbi)
1239
khenaidoo8b4abbf2020-04-24 17:04:30 -04001240 numberOfTestRuns := 2
1241 for i := 1; i <= numberOfTestRuns; i++ {
khenaidoo67b22152020-03-02 16:01:25 -05001242 //3. Test create device
khenaidoo93d5a3d2020-01-15 12:37:05 -05001243 nb.testCreateDevice(t, nbi)
khenaidoob64fc8a2019-11-27 15:08:19 -05001244
khenaidoo93d5a3d2020-01-15 12:37:05 -05001245 // 4. Test Enable a device
1246 nb.testEnableDevice(t, nbi)
khenaidoob64fc8a2019-11-27 15:08:19 -05001247
khenaidoo0db4c812020-05-27 15:27:30 -04001248 //// 5. Test disable and ReEnable a root device
khenaidoo93d5a3d2020-01-15 12:37:05 -05001249 nb.testDisableAndReEnableRootDevice(t, nbi)
khenaidoo67b22152020-03-02 16:01:25 -05001250
kesavandbc2d1622020-01-21 00:42:01 -05001251 // 6. Test disable and Enable pon port of OLT device
1252 nb.testDisableAndEnablePort(t, nbi)
khenaidoo93d5a3d2020-01-15 12:37:05 -05001253
Girish Gowdra408cd962020-03-11 14:31:31 -07001254 // 7.Test Device unreachable when OLT is enabled
1255 nb.testDeviceRebootWhenOltIsEnabled(t, nbi)
1256
1257 // 8. Test disable and delete all devices
khenaidoo93d5a3d2020-01-15 12:37:05 -05001258 nb.testDisableAndDeleteAllDevice(t, nbi)
Chaitrashree G S543df3e2020-02-24 22:36:54 -05001259
Girish Gowdra408cd962020-03-11 14:31:31 -07001260 // 9. Test enable and delete all devices
Chaitrashree G S543df3e2020-02-24 22:36:54 -05001261 nb.testEnableAndDeleteAllDevice(t, nbi)
Scott Baker432f9be2020-03-26 11:56:30 -07001262
1263 // 10. Test omci test
1264 nb.testStartOmciTestAction(t, nbi)
khenaidoob64fc8a2019-11-27 15:08:19 -05001265
khenaidoo0db4c812020-05-27 15:27:30 -04001266 // 11. Remove all devices from tests above
1267 nb.deleteAllDevices(t, nbi)
1268
khenaidoo8b4abbf2020-04-24 17:04:30 -04001269 // 11. Test flow add failure
1270 nb.testFlowAddFailure(t, nbi)
1271
1272 // 12. Clean up
1273 nb.deleteAllDevices(t, nbi)
1274 }
khenaidoob64fc8a2019-11-27 15:08:19 -05001275}