[VOL-4194] Change voltha-lib-go version to use Etcd pool

Change-Id: I1734765a379dbcfca8af355d70f097c0ee65b8e8
diff --git a/go.mod b/go.mod
index 5a1da5c..32be0bc 100644
--- a/go.mod
+++ b/go.mod
@@ -8,7 +8,7 @@
 	github.com/golang/mock v1.5.0
 	github.com/golang/protobuf v1.3.2
 	github.com/google/uuid v1.1.1
-	github.com/opencord/voltha-lib-go/v5 v5.0.1
+	github.com/opencord/voltha-lib-go/v5 v5.0.2
 	github.com/opencord/voltha-protos/v4 v4.2.0
 	github.com/opentracing/opentracing-go v1.1.0
 	github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2
diff --git a/go.sum b/go.sum
index 3b0fa2a..ae4b4e4 100644
--- a/go.sum
+++ b/go.sum
@@ -143,8 +143,8 @@
 github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
 github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I=
 github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
-github.com/opencord/voltha-lib-go/v5 v5.0.1 h1:cWgOFon1AAUMlMCXgKO4sf8/s38LOzzBvUpq/IJP4Zc=
-github.com/opencord/voltha-lib-go/v5 v5.0.1/go.mod h1:i1fwPMicFccG38L200+IQAlfHSbszWg//jF1pDQxTPQ=
+github.com/opencord/voltha-lib-go/v5 v5.0.2 h1:nLs42QM75BhKt4eXLdHhQwRPLrI2V2BjWJJlzGMUixg=
+github.com/opencord/voltha-lib-go/v5 v5.0.2/go.mod h1:i1fwPMicFccG38L200+IQAlfHSbszWg//jF1pDQxTPQ=
 github.com/opencord/voltha-protos/v4 v4.2.0 h1:QJZqHPRKa1E1xh40F3UA4xSjBI+6EmW7OfIcJqPNc4A=
 github.com/opencord/voltha-protos/v4 v4.2.0/go.mod h1:wNzWqmTwe7+DbYbpmOX6eMlglREtMkNxIDv3lyI2bco=
 github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU=
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/client.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/client.go
index b35f1f3..e4b1fff 100644
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/client.go
+++ b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/client.go
@@ -79,14 +79,17 @@
 	Put(ctx context.Context, key string, value interface{}) error
 	Delete(ctx context.Context, key string) error
 	DeleteWithPrefix(ctx context.Context, prefixKey string) error
+	Watch(ctx context.Context, key string, withPrefix bool) chan *Event
+	IsConnectionUp(ctx context.Context) bool // timeout in second
+	CloseWatch(ctx context.Context, key string, ch chan *Event)
+	Close(ctx context.Context)
+
+	// These APIs are not used.  They will be cleaned up in release Voltha 2.9.
+	// It's not cleaned now to limit changes in all components
 	Reserve(ctx context.Context, key string, value interface{}, ttl time.Duration) (interface{}, error)
 	ReleaseReservation(ctx context.Context, key string) error
 	ReleaseAllReservations(ctx context.Context) error
 	RenewReservation(ctx context.Context, key string) error
-	Watch(ctx context.Context, key string, withPrefix bool) chan *Event
 	AcquireLock(ctx context.Context, lockName string, timeout time.Duration) error
 	ReleaseLock(lockName string) error
-	IsConnectionUp(ctx context.Context) bool // timeout in second
-	CloseWatch(ctx context.Context, key string, ch chan *Event)
-	Close(ctx context.Context)
 }
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdclient.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdclient.go
index c2a38c6..96ffc2f 100644
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdclient.go
+++ b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdclient.go
@@ -19,54 +19,75 @@
 	"context"
 	"errors"
 	"fmt"
-	"sync"
-	"time"
-
 	"github.com/opencord/voltha-lib-go/v5/pkg/log"
 	v3Client "go.etcd.io/etcd/clientv3"
-
-	v3Concurrency "go.etcd.io/etcd/clientv3/concurrency"
 	v3rpcTypes "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	"os"
+	"strconv"
+	"sync"
+	"time"
+)
+
+const (
+	poolCapacityEnvName = "VOLTHA_ETCD_CLIENT_POOL_CAPACITY"
+	maxUsageEnvName     = "VOLTHA_ETCD_CLIENT_MAX_USAGE"
+)
+
+const (
+	defaultMaxPoolCapacity = 1000 // Default size of an Etcd Client pool
+	defaultMaxPoolUsage    = 100  // Maximum concurrent request an Etcd Client is allowed to process
 )
 
 // EtcdClient represents the Etcd KV store client
 type EtcdClient struct {
-	ectdAPI             *v3Client.Client
-	keyReservations     map[string]*v3Client.LeaseID
-	watchedChannels     sync.Map
-	keyReservationsLock sync.RWMutex
-	lockToMutexMap      map[string]*v3Concurrency.Mutex
-	lockToSessionMap    map[string]*v3Concurrency.Session
-	lockToMutexLock     sync.Mutex
+	pool               EtcdClientAllocator
+	watchedChannels    sync.Map
+	watchedClients     map[string]*v3Client.Client
+	watchedClientsLock sync.RWMutex
 }
 
 // NewEtcdCustomClient returns a new client for the Etcd KV store allowing
 // the called to specify etcd client configuration
-func NewEtcdCustomClient(ctx context.Context, config *v3Client.Config) (*EtcdClient, error) {
-	c, err := v3Client.New(*config)
-	if err != nil {
-		logger.Error(ctx, err)
-		return nil, err
+func NewEtcdCustomClient(ctx context.Context, addr string, timeout time.Duration, level log.LogLevel) (*EtcdClient, error) {
+	// Get the capacity and max usage from the environment
+	capacity := defaultMaxPoolCapacity
+	maxUsage := defaultMaxPoolUsage
+	if capacityStr, present := os.LookupEnv(poolCapacityEnvName); present {
+		if val, err := strconv.Atoi(capacityStr); err == nil {
+			capacity = val
+			logger.Infow(ctx, "env-variable-set", log.Fields{"pool-capacity": capacity})
+		} else {
+			logger.Warnw(ctx, "invalid-capacity-value", log.Fields{"error": err, "capacity": capacityStr})
+		}
+	}
+	if maxUsageStr, present := os.LookupEnv(maxUsageEnvName); present {
+		if val, err := strconv.Atoi(maxUsageStr); err == nil {
+			maxUsage = val
+			logger.Infow(ctx, "env-variable-set", log.Fields{"max-usage": maxUsage})
+		} else {
+			logger.Warnw(ctx, "invalid-max-usage-value", log.Fields{"error": err, "max-usage": maxUsageStr})
+		}
 	}
 
-	reservations := make(map[string]*v3Client.LeaseID)
-	lockMutexMap := make(map[string]*v3Concurrency.Mutex)
-	lockSessionMap := make(map[string]*v3Concurrency.Session)
+	var err error
 
-	return &EtcdClient{ectdAPI: c, keyReservations: reservations, lockToMutexMap: lockMutexMap,
-		lockToSessionMap: lockSessionMap}, nil
+	pool, err := NewRoundRobinEtcdClientAllocator([]string{addr}, timeout, capacity, maxUsage, level)
+	if err != nil {
+		logger.Errorw(ctx, "failed-to-create-rr-client", log.Fields{
+			"error": err,
+		})
+	}
+
+	logger.Infow(ctx, "etcd-pool-created", log.Fields{"capacity": capacity, "max-usage": maxUsage})
+
+	return &EtcdClient{pool: pool,
+		watchedClients: make(map[string]*v3Client.Client),
+	}, nil
 }
 
 // NewEtcdClient returns a new client for the Etcd KV store
 func NewEtcdClient(ctx context.Context, addr string, timeout time.Duration, level log.LogLevel) (*EtcdClient, error) {
-	logconfig := log.ConstructZapConfig(log.JSON, level, log.Fields{})
-
-	return NewEtcdCustomClient(
-		ctx,
-		&v3Client.Config{
-			Endpoints:   []string{addr},
-			DialTimeout: timeout,
-			LogConfig:   &logconfig})
+	return NewEtcdCustomClient(ctx, addr, timeout, level)
 }
 
 // IsConnectionUp returns whether the connection to the Etcd KV store is up.  If a timeout occurs then
@@ -76,14 +97,19 @@
 	if _, err := c.Get(ctx, "non-existent-key"); err != nil {
 		return false
 	}
-	//cancel()
 	return true
 }
 
 // List returns an array of key-value pairs with key as a prefix.  Timeout defines how long the function will
 // wait for a response
 func (c *EtcdClient) List(ctx context.Context, key string) (map[string]*KVPair, error) {
-	resp, err := c.ectdAPI.Get(ctx, key, v3Client.WithPrefix())
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return nil, err
+	}
+	defer c.pool.Put(client)
+	resp, err := client.Get(ctx, key, v3Client.WithPrefix())
+
 	if err != nil {
 		logger.Error(ctx, err)
 		return nil, err
@@ -98,11 +124,17 @@
 // Get returns a key-value pair for a given key. Timeout defines how long the function will
 // wait for a response
 func (c *EtcdClient) Get(ctx context.Context, key string) (*KVPair, error) {
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return nil, err
+	}
+	defer c.pool.Put(client)
 
 	attempt := 0
+
 startLoop:
 	for {
-		resp, err := c.ectdAPI.Get(ctx, key)
+		resp, err := client.Get(ctx, key)
 		if err != nil {
 			switch err {
 			case context.Canceled:
@@ -145,25 +177,21 @@
 
 	// Validate that we can convert value to a string as etcd API expects a string
 	var val string
-	var er error
-	if val, er = ToString(value); er != nil {
+	var err error
+	if val, err = ToString(value); err != nil {
 		return fmt.Errorf("unexpected-type-%T", value)
 	}
 
-	// Check if there is already a lease for this key - if there is then use it, otherwise a PUT will make
-	// that KV key permanent instead of automatically removing it after a lease expiration
-	c.keyReservationsLock.RLock()
-	leaseID, ok := c.keyReservations[key]
-	c.keyReservationsLock.RUnlock()
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return err
+	}
+	defer c.pool.Put(client)
+
 	attempt := 0
 startLoop:
 	for {
-		var err error
-		if ok {
-			_, err = c.ectdAPI.Put(ctx, key, val, v3Client.WithLease(*leaseID))
-		} else {
-			_, err = c.ectdAPI.Put(ctx, key, val)
-		}
+		_, err = client.Put(ctx, key, val)
 		if err != nil {
 			switch err {
 			case context.Canceled:
@@ -197,11 +225,16 @@
 // Delete removes a key from the KV store. Timeout defines how long the function will
 // wait for a response
 func (c *EtcdClient) Delete(ctx context.Context, key string) error {
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return err
+	}
+	defer c.pool.Put(client)
 
 	attempt := 0
 startLoop:
 	for {
-		_, err := c.ectdAPI.Delete(ctx, key)
+		_, err = client.Delete(ctx, key)
 		if err != nil {
 			switch err {
 			case context.Canceled:
@@ -235,8 +268,14 @@
 
 func (c *EtcdClient) DeleteWithPrefix(ctx context.Context, prefixKey string) error {
 
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return err
+	}
+	defer c.pool.Put(client)
+
 	//delete the prefix
-	if _, err := c.ectdAPI.Delete(ctx, prefixKey, v3Client.WithPrefix()); err != nil {
+	if _, err := client.Delete(ctx, prefixKey, v3Client.WithPrefix()); err != nil {
 		logger.Errorw(ctx, "failed-to-delete-prefix-key", log.Fields{"key": prefixKey, "error": err})
 		return err
 	}
@@ -244,150 +283,25 @@
 	return nil
 }
 
-// Reserve is invoked to acquire a key and set it to a given value. Value can only be a string or []byte since
-// the etcd API accepts only a string.  Timeout defines how long the function will wait for a response.  TTL
-// defines how long that reservation is valid.  When TTL expires the key is unreserved by the KV store itself.
-// If the key is acquired then the value returned will be the value passed in.  If the key is already acquired
-// then the value assigned to that key will be returned.
-func (c *EtcdClient) Reserve(ctx context.Context, key string, value interface{}, ttl time.Duration) (interface{}, error) {
-	// Validate that we can convert value to a string as etcd API expects a string
-	var val string
-	var er error
-	if val, er = ToString(value); er != nil {
-		return nil, fmt.Errorf("unexpected-type%T", value)
-	}
-
-	resp, err := c.ectdAPI.Grant(ctx, int64(ttl.Seconds()))
-	if err != nil {
-		logger.Error(ctx, err)
-		return nil, err
-	}
-	// Register the lease id
-	c.keyReservationsLock.Lock()
-	c.keyReservations[key] = &resp.ID
-	c.keyReservationsLock.Unlock()
-
-	// Revoke lease if reservation is not successful
-	reservationSuccessful := false
-	defer func() {
-		if !reservationSuccessful {
-			if err = c.ReleaseReservation(context.Background(), key); err != nil {
-				logger.Error(ctx, "cannot-release-lease")
-			}
-		}
-	}()
-
-	// Try to grap the Key with the above lease
-	c.ectdAPI.Txn(context.Background())
-	txn := c.ectdAPI.Txn(context.Background())
-	txn = txn.If(v3Client.Compare(v3Client.Version(key), "=", 0))
-	txn = txn.Then(v3Client.OpPut(key, val, v3Client.WithLease(resp.ID)))
-	txn = txn.Else(v3Client.OpGet(key))
-	result, er := txn.Commit()
-	if er != nil {
-		return nil, er
-	}
-
-	if !result.Succeeded {
-		// Verify whether we are already the owner of that Key
-		if len(result.Responses) > 0 &&
-			len(result.Responses[0].GetResponseRange().Kvs) > 0 {
-			kv := result.Responses[0].GetResponseRange().Kvs[0]
-			if string(kv.Value) == val {
-				reservationSuccessful = true
-				return value, nil
-			}
-			return kv.Value, nil
-		}
-	} else {
-		// Read the Key to ensure this is our Key
-		m, err := c.Get(ctx, key)
-		if err != nil {
-			return nil, err
-		}
-		if m != nil {
-			if m.Key == key && isEqual(m.Value, value) {
-				// My reservation is successful - register it.  For now, support is only for 1 reservation per key
-				// per session.
-				reservationSuccessful = true
-				return value, nil
-			}
-			// My reservation has failed.  Return the owner of that key
-			return m.Value, nil
-		}
-	}
-	return nil, nil
-}
-
-// ReleaseAllReservations releases all key reservations previously made (using Reserve API)
-func (c *EtcdClient) ReleaseAllReservations(ctx context.Context) error {
-	c.keyReservationsLock.Lock()
-	defer c.keyReservationsLock.Unlock()
-
-	for key, leaseID := range c.keyReservations {
-		_, err := c.ectdAPI.Revoke(ctx, *leaseID)
-		if err != nil {
-			logger.Errorw(ctx, "cannot-release-reservation", log.Fields{"key": key, "error": err})
-			return err
-		}
-		delete(c.keyReservations, key)
-	}
-	return nil
-}
-
-// ReleaseReservation releases reservation for a specific key.
-func (c *EtcdClient) ReleaseReservation(ctx context.Context, key string) error {
-	// Get the leaseid using the key
-	logger.Debugw(ctx, "Release-reservation", log.Fields{"key": key})
-	var ok bool
-	var leaseID *v3Client.LeaseID
-	c.keyReservationsLock.Lock()
-	defer c.keyReservationsLock.Unlock()
-	if leaseID, ok = c.keyReservations[key]; !ok {
-		return nil
-	}
-
-	if leaseID != nil {
-		_, err := c.ectdAPI.Revoke(ctx, *leaseID)
-		if err != nil {
-			logger.Error(ctx, err)
-			return err
-		}
-		delete(c.keyReservations, key)
-	}
-	return nil
-}
-
-// RenewReservation renews a reservation.  A reservation will go stale after the specified TTL (Time To Live)
-// period specified when reserving the key
-func (c *EtcdClient) RenewReservation(ctx context.Context, key string) error {
-	// Get the leaseid using the key
-	var ok bool
-	var leaseID *v3Client.LeaseID
-	c.keyReservationsLock.RLock()
-	leaseID, ok = c.keyReservations[key]
-	c.keyReservationsLock.RUnlock()
-
-	if !ok {
-		return errors.New("key-not-reserved")
-	}
-
-	if leaseID != nil {
-		_, err := c.ectdAPI.KeepAliveOnce(ctx, *leaseID)
-		if err != nil {
-			logger.Errorw(ctx, "lease-may-have-expired", log.Fields{"error": err})
-			return err
-		}
-	} else {
-		return errors.New("lease-expired")
-	}
-	return nil
-}
-
 // Watch provides the watch capability on a given key.  It returns a channel onto which the callee needs to
 // listen to receive Events.
 func (c *EtcdClient) Watch(ctx context.Context, key string, withPrefix bool) chan *Event {
-	w := v3Client.NewWatcher(c.ectdAPI)
+	var err error
+	// Reuse the Etcd client when multiple callees are watching the same key.
+	c.watchedClientsLock.Lock()
+	client, exist := c.watchedClients[key]
+	if !exist {
+		client, err = c.pool.Get(ctx)
+		if err != nil {
+			logger.Errorw(ctx, "failed-to-an-etcd-client", log.Fields{"key": key, "error": err})
+			c.watchedClientsLock.Unlock()
+			return nil
+		}
+		c.watchedClients[key] = client
+	}
+	c.watchedClientsLock.Unlock()
+
+	w := v3Client.NewWatcher(client)
 	ctx, cancel := context.WithCancel(ctx)
 	var channel v3Client.WatchChan
 	if withPrefix {
@@ -402,7 +316,6 @@
 	// Keep track of the created channels so they can be closed when required
 	channelMap := make(map[chan *Event]v3Client.Watcher)
 	channelMap[ch] = w
-
 	channelMaps := c.addChannelMap(key, channelMap)
 
 	// Changing the log field (from channelMaps) as the underlying logger cannot format the map of channels into a
@@ -484,6 +397,17 @@
 	if pos >= 0 {
 		channelMaps = c.removeChannelMap(key, pos)
 	}
+
+	// If we don't have any keys being watched then return the Etcd client to the pool
+	if len(channelMaps) == 0 {
+		c.watchedClientsLock.Lock()
+		// Sanity
+		if client, ok := c.watchedClients[key]; ok {
+			c.pool.Put(client)
+			delete(c.watchedClients, key)
+		}
+		c.watchedClientsLock.Unlock()
+	}
 	logger.Infow(ctx, "watcher-channel-exiting", log.Fields{"key": key, "channel": channelMaps})
 }
 
@@ -509,66 +433,41 @@
 	return UNKNOWN
 }
 
-// Close closes the KV store client
+// Close closes all the connection in the pool store client
 func (c *EtcdClient) Close(ctx context.Context) {
-	if err := c.ectdAPI.Close(); err != nil {
-		logger.Errorw(ctx, "error-closing-client", log.Fields{"error": err})
-	}
+	logger.Debug(ctx, "closing-etcd-pool")
+	c.pool.Close(ctx)
 }
 
-func (c *EtcdClient) addLockName(lockName string, lock *v3Concurrency.Mutex, session *v3Concurrency.Session) {
-	c.lockToMutexLock.Lock()
-	defer c.lockToMutexLock.Unlock()
-	c.lockToMutexMap[lockName] = lock
-	c.lockToSessionMap[lockName] = session
+// The APIs below are not used
+var errUnimplemented = errors.New("deprecated")
+
+// Reserve is deprecated
+func (c *EtcdClient) Reserve(ctx context.Context, key string, value interface{}, ttl time.Duration) (interface{}, error) {
+	return nil, errUnimplemented
 }
 
-func (c *EtcdClient) deleteLockName(lockName string) {
-	c.lockToMutexLock.Lock()
-	defer c.lockToMutexLock.Unlock()
-	delete(c.lockToMutexMap, lockName)
-	delete(c.lockToSessionMap, lockName)
+// ReleaseAllReservations is deprecated
+func (c *EtcdClient) ReleaseAllReservations(ctx context.Context) error {
+	return errUnimplemented
 }
 
-func (c *EtcdClient) getLock(lockName string) (*v3Concurrency.Mutex, *v3Concurrency.Session) {
-	c.lockToMutexLock.Lock()
-	defer c.lockToMutexLock.Unlock()
-	var lock *v3Concurrency.Mutex
-	var session *v3Concurrency.Session
-	if l, exist := c.lockToMutexMap[lockName]; exist {
-		lock = l
-	}
-	if s, exist := c.lockToSessionMap[lockName]; exist {
-		session = s
-	}
-	return lock, session
+// ReleaseReservation is deprecated
+func (c *EtcdClient) ReleaseReservation(ctx context.Context, key string) error {
+	return errUnimplemented
 }
 
+// RenewReservation is deprecated
+func (c *EtcdClient) RenewReservation(ctx context.Context, key string) error {
+	return errUnimplemented
+}
+
+// AcquireLock is deprecated
 func (c *EtcdClient) AcquireLock(ctx context.Context, lockName string, timeout time.Duration) error {
-	session, _ := v3Concurrency.NewSession(c.ectdAPI, v3Concurrency.WithContext(ctx))
-	mu := v3Concurrency.NewMutex(session, "/devicelock_"+lockName)
-	if err := mu.Lock(context.Background()); err != nil {
-		//cancel()
-		return err
-	}
-	c.addLockName(lockName, mu, session)
-	return nil
+	return errUnimplemented
 }
 
+// ReleaseLock is deprecated
 func (c *EtcdClient) ReleaseLock(lockName string) error {
-	lock, session := c.getLock(lockName)
-	var err error
-	if lock != nil {
-		if e := lock.Unlock(context.Background()); e != nil {
-			err = e
-		}
-	}
-	if session != nil {
-		if e := session.Close(); e != nil {
-			err = e
-		}
-	}
-	c.deleteLockName(lockName)
-
-	return err
+	return errUnimplemented
 }
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdpool.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdpool.go
new file mode 100644
index 0000000..6af7d3d
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdpool.go
@@ -0,0 +1,239 @@
+/*
+ * Copyright 2021-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 kvstore
+
+import (
+	"container/list"
+	"context"
+	"errors"
+	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"go.etcd.io/etcd/clientv3"
+	"sync"
+	"time"
+)
+
+// EtcdClientAllocator represents a generic interface to allocate an Etcd Client
+type EtcdClientAllocator interface {
+	Get(context.Context) (*clientv3.Client, error)
+	Put(*clientv3.Client)
+	Close(ctx context.Context)
+}
+
+// NewRoundRobinEtcdClientAllocator creates a new ETCD Client Allocator using a Round Robin scheme
+func NewRoundRobinEtcdClientAllocator(endpoints []string, timeout time.Duration, capacity, maxUsage int, level log.LogLevel) (EtcdClientAllocator, error) {
+	return &roundRobin{
+		all:       make(map[*clientv3.Client]*rrEntry),
+		full:      make(map[*clientv3.Client]*rrEntry),
+		waitList:  list.New(),
+		max:       maxUsage,
+		capacity:  capacity,
+		timeout:   timeout,
+		endpoints: endpoints,
+		logLevel:  level,
+		closingCh: make(chan struct{}, capacity*maxUsage),
+		stopCh:    make(chan struct{}),
+	}, nil
+}
+
+type rrEntry struct {
+	client *clientv3.Client
+	count  int
+	age    time.Time
+}
+
+type roundRobin struct {
+	//block chan struct{}
+	sync.Mutex
+	available []*rrEntry
+	all       map[*clientv3.Client]*rrEntry
+	full      map[*clientv3.Client]*rrEntry
+	waitList  *list.List
+	max       int
+	capacity  int
+	timeout   time.Duration
+	//ageOut    time.Duration
+	endpoints []string
+	size      int
+	logLevel  log.LogLevel
+	closing   bool
+	closingCh chan struct{}
+	stopCh    chan struct{}
+}
+
+// Get returns an Etcd client. If not is available, it will create one
+// until the maximum allowed capacity.  If maximum capacity has been
+// reached then it will wait until s used one is freed.
+func (r *roundRobin) Get(ctx context.Context) (*clientv3.Client, error) {
+	r.Lock()
+
+	if r.closing {
+		r.Unlock()
+		return nil, errors.New("pool-is-closing")
+	}
+
+	// first determine if we need to block, which would mean the
+	// available queue is empty and we are at capacity
+	if len(r.available) == 0 && r.size >= r.capacity {
+
+		// create a channel on which to wait and
+		// add it to the list
+		ch := make(chan struct{})
+		element := r.waitList.PushBack(ch)
+		r.Unlock()
+
+		// block until it is our turn or context
+		// expires or is canceled
+		select {
+		case <-r.stopCh:
+			logger.Info(ctx, "stop-waiting-pool-is-closing")
+			r.waitList.Remove(element)
+			return nil, errors.New("stop-waiting-pool-is-closing")
+		case <-ch:
+			r.waitList.Remove(element)
+		case <-ctx.Done():
+			r.waitList.Remove(element)
+			return nil, ctx.Err()
+		}
+		r.Lock()
+	}
+
+	defer r.Unlock()
+	if len(r.available) > 0 {
+		// pull off back end as it is operationally quicker
+		last := len(r.available) - 1
+		entry := r.available[last]
+		entry.count++
+		if entry.count >= r.max {
+			r.available = r.available[:last]
+			r.full[entry.client] = entry
+		}
+		entry.age = time.Now()
+		return entry.client, nil
+	}
+
+	logConfig := log.ConstructZapConfig(log.JSON, r.logLevel, log.Fields{})
+	// increase capacity
+	client, err := clientv3.New(clientv3.Config{
+		Endpoints:   r.endpoints,
+		DialTimeout: r.timeout,
+		LogConfig:   &logConfig,
+	})
+	if err != nil {
+		return nil, err
+	}
+	entry := &rrEntry{
+		client: client,
+		count:  1,
+	}
+	r.all[entry.client] = entry
+
+	if r.max > 1 {
+		r.available = append(r.available, entry)
+	} else {
+		r.full[entry.client] = entry
+	}
+	r.size++
+	return client, nil
+}
+
+// Put returns the Etcd Client back to the pool
+func (r *roundRobin) Put(client *clientv3.Client) {
+	r.Lock()
+
+	entry := r.all[client]
+	entry.count--
+
+	if r.closing {
+		// Close client if count is 0
+		if entry.count == 0 {
+			if err := entry.client.Close(); err != nil {
+				logger.Warnw(context.Background(), "error-closing-client", log.Fields{"error": err})
+			}
+			delete(r.all, entry.client)
+		}
+		// Notify Close function that a client was returned to the pool
+		r.closingCh <- struct{}{}
+		r.Unlock()
+		return
+	}
+
+	// This entry is now available for use, so
+	// if in full map add it to available and
+	// remove from full
+	if _, ok := r.full[client]; ok {
+		r.available = append(r.available, entry)
+		delete(r.full, client)
+	}
+
+	front := r.waitList.Front()
+	if front != nil {
+		ch := r.waitList.Remove(front)
+		r.Unlock()
+		// need to unblock if someone is waiting
+		ch.(chan struct{}) <- struct{}{}
+		return
+	}
+	r.Unlock()
+}
+
+func (r *roundRobin) Close(ctx context.Context) {
+	r.Lock()
+	r.closing = true
+
+	// Notify anyone waiting for a client to stop waiting
+	close(r.stopCh)
+
+	// Clean-up unused clients
+	for i := 0; i < len(r.available); i++ {
+		// Count 0 means no one is using that client
+		if r.available[i].count == 0 {
+			if err := r.available[i].client.Close(); err != nil {
+				logger.Warnw(ctx, "failure-closing-client", log.Fields{"client": r.available[i].client, "error": err})
+			}
+			// Remove client for all list
+			delete(r.all, r.available[i].client)
+		}
+	}
+
+	// Figure out how many clients are in use
+	numberInUse := 0
+	for _, rrEntry := range r.all {
+		numberInUse += rrEntry.count
+	}
+	r.Unlock()
+
+	if numberInUse == 0 {
+		logger.Info(ctx, "no-connection-in-use")
+		return
+	}
+
+	logger.Infow(ctx, "waiting-for-clients-return", log.Fields{"count": numberInUse})
+
+	// Wait for notifications when a client is returned to the pool
+	for {
+		select {
+		case <-r.closingCh:
+			numberInUse--
+			if numberInUse == 0 {
+				logger.Info(ctx, "all-connections-closed")
+				return
+			}
+		case <-ctx.Done():
+			logger.Warnw(ctx, "context-done", log.Fields{"error": ctx.Err()})
+			return
+		}
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/kvutils.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/kvutils.go
index 946dbf2..ca57542 100644
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/kvutils.go
+++ b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/kvutils.go
@@ -16,7 +16,6 @@
 package kvstore
 
 import (
-	"bytes"
 	"context"
 	"fmt"
 	"math"
@@ -57,17 +56,6 @@
 	}
 }
 
-// Helper function to verify mostly whether the content of two interface types are the same.  Focus is []byte and
-// string types
-func isEqual(val1 interface{}, val2 interface{}) bool {
-	b1, err := ToByte(val1)
-	b2, er := ToByte(val2)
-	if err == nil && er == nil {
-		return bytes.Equal(b1, b2)
-	}
-	return val1 == val2
-}
-
 // backoff waits an amount of time that is proportional to the attempt value.  The wait time in a range of
 // minRetryInterval and maxRetryInterval.
 func backoff(ctx context.Context, attempt int) error {
diff --git a/vendor/modules.txt b/vendor/modules.txt
index d19c3b4..2087424 100644
--- a/vendor/modules.txt
+++ b/vendor/modules.txt
@@ -93,7 +93,7 @@
 github.com/modern-go/concurrent
 # github.com/modern-go/reflect2 v1.0.1
 github.com/modern-go/reflect2
-# github.com/opencord/voltha-lib-go/v5 v5.0.1
+# github.com/opencord/voltha-lib-go/v5 v5.0.2
 ## explicit
 github.com/opencord/voltha-lib-go/v5/pkg/adapters
 github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif