[VOL-4290] Voltha go library updates for gRPC migration

Change-Id: I1aa2774beb6b7ed7419bc45aeb53fcae8a8ecda0
diff --git a/pkg/grpc/client.go b/pkg/grpc/client.go
new file mode 100644
index 0000000..de649d6
--- /dev/null
+++ b/pkg/grpc/client.go
@@ -0,0 +1,541 @@
+/*
+ * 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 grpc
+
+import (
+	"context"
+	"fmt"
+	"reflect"
+	"strings"
+	"sync"
+	"time"
+
+	grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
+	grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/keepalive"
+)
+
+type event byte
+type state byte
+type SetAndTestServiceHandler func(context.Context, *grpc.ClientConn) interface{}
+type RestartedHandler func(ctx context.Context, endPoint string) error
+
+type contextKey string
+
+func (c contextKey) String() string {
+	return string(c)
+}
+
+var (
+	grpcMonitorContextKey = contextKey("grpc-monitor")
+)
+
+const (
+	grpcBackoffInitialInterval = "GRPC_BACKOFF_INITIAL_INTERVAL"
+	grpcBackoffMaxInterval     = "GRPC_BACKOFF_MAX_INTERVAL"
+	grpcBackoffMaxElapsedTime  = "GRPC_BACKOFF_MAX_ELAPSED_TIME"
+	grpcMonitorInterval        = "GRPC_MONITOR_INTERVAL"
+)
+
+const (
+	DefaultBackoffInitialInterval = 100 * time.Millisecond
+	DefaultBackoffMaxInterval     = 5 * time.Second
+	DefaultBackoffMaxElapsedTime  = 0 * time.Second // No time limit
+	DefaultGRPCMonitorInterval    = 5 * time.Second
+)
+
+const (
+	connectionErrorSubString  = "SubConns are in TransientFailure"
+	connectionClosedSubstring = "client connection is closing"
+	connectionError           = "connection error"
+	connectionSystemNotReady  = "system is not ready"
+)
+
+const (
+	eventConnecting = event(iota)
+	eventConnected
+	eventDisconnected
+	eventStopped
+	eventError
+
+	stateConnected = state(iota)
+	stateConnecting
+	stateDisconnected
+)
+
+type Client struct {
+	apiEndPoint            string
+	connection             *grpc.ClientConn
+	connectionLock         sync.RWMutex
+	stateLock              sync.RWMutex
+	state                  state
+	service                interface{}
+	events                 chan event
+	onRestart              RestartedHandler
+	backoffInitialInterval time.Duration
+	backoffMaxInterval     time.Duration
+	backoffMaxElapsedTime  time.Duration
+	activityCheck          bool
+	monitorInterval        time.Duration
+	activeCh               chan struct{}
+	activeChMutex          sync.RWMutex
+	done                   bool
+	livenessCallback       func(timestamp time.Time)
+}
+
+type ClientOption func(*Client)
+
+func ActivityCheck(enable bool) ClientOption {
+	return func(args *Client) {
+		args.activityCheck = enable
+	}
+}
+
+func NewClient(endpoint string, onRestart RestartedHandler, opts ...ClientOption) (*Client, error) {
+	c := &Client{
+		apiEndPoint:            endpoint,
+		onRestart:              onRestart,
+		events:                 make(chan event, 1),
+		state:                  stateDisconnected,
+		backoffInitialInterval: DefaultBackoffInitialInterval,
+		backoffMaxInterval:     DefaultBackoffMaxInterval,
+		backoffMaxElapsedTime:  DefaultBackoffMaxElapsedTime,
+		monitorInterval:        DefaultGRPCMonitorInterval,
+	}
+	for _, option := range opts {
+		option(c)
+	}
+
+	// Check for environment variables
+	if err := SetFromEnvVariable(grpcBackoffInitialInterval, &c.backoffInitialInterval); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcBackoffInitialInterval})
+	}
+
+	if err := SetFromEnvVariable(grpcBackoffMaxInterval, &c.backoffMaxInterval); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcBackoffMaxInterval})
+	}
+
+	if err := SetFromEnvVariable(grpcBackoffMaxElapsedTime, &c.backoffMaxElapsedTime); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcBackoffMaxElapsedTime})
+	}
+
+	if err := SetFromEnvVariable(grpcMonitorInterval, &c.monitorInterval); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcMonitorInterval})
+	}
+
+	logger.Infow(context.Background(), "initialized-client", log.Fields{"client": c})
+
+	// Sanity check
+	if c.backoffInitialInterval > c.backoffMaxInterval {
+		return nil, fmt.Errorf("initial retry delay %v is greater than maximum retry delay %v", c.backoffInitialInterval, c.backoffMaxInterval)
+	}
+
+	return c, nil
+}
+
+func (c *Client) GetClient() (interface{}, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no connection to %s", c.apiEndPoint)
+	}
+	return c.service, nil
+}
+
+// GetCoreServiceClient is a helper function that returns a concrete service instead of the GetClient() API
+// which returns an interface
+func (c *Client) GetCoreServiceClient() (core.CoreServiceClient, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no core connection to %s", c.apiEndPoint)
+	}
+	client, ok := c.service.(core.CoreServiceClient)
+	if ok {
+		return client, nil
+	}
+	return nil, fmt.Errorf("invalid-service-%s", reflect.TypeOf(c.service))
+}
+
+// GetOnuAdapterServiceClient is a helper function that returns a concrete service instead of the GetClient() API
+// which returns an interface
+func (c *Client) GetOnuInterAdapterServiceClient() (adapter_services.OnuInterAdapterServiceClient, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no child adapter connection to %s", c.apiEndPoint)
+	}
+	client, ok := c.service.(adapter_services.OnuInterAdapterServiceClient)
+	if ok {
+		return client, nil
+	}
+	return nil, fmt.Errorf("invalid-service-%s", reflect.TypeOf(c.service))
+}
+
+// GetOltAdapterServiceClient is a helper function that returns a concrete service instead of the GetClient() API
+// which returns an interface
+func (c *Client) GetOltInterAdapterServiceClient() (adapter_services.OltInterAdapterServiceClient, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no parent adapter connection to %s", c.apiEndPoint)
+	}
+	client, ok := c.service.(adapter_services.OltInterAdapterServiceClient)
+	if ok {
+		return client, nil
+	}
+	return nil, fmt.Errorf("invalid-service-%s", reflect.TypeOf(c.service))
+}
+
+func (c *Client) Reset(ctx context.Context) {
+	logger.Debugw(ctx, "resetting-client-connection", log.Fields{"endpoint": c.apiEndPoint})
+	c.stateLock.Lock()
+	defer c.stateLock.Unlock()
+	if c.state == stateConnected {
+		c.state = stateDisconnected
+		c.events <- eventDisconnected
+	}
+}
+
+func (c *Client) clientInterceptor(ctx context.Context, method string, req interface{}, reply interface{},
+	cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+	// Nothing to do before intercepting the call
+	err := invoker(ctx, method, req, reply, cc, opts...)
+	// On connection failure, start the reconnect process depending on the error response
+	if err != nil {
+		logger.Errorw(ctx, "received-error", log.Fields{"error": err, "context": ctx, "endpoint": c.apiEndPoint})
+		if strings.Contains(err.Error(), connectionErrorSubString) ||
+			strings.Contains(err.Error(), connectionError) ||
+			strings.Contains(err.Error(), connectionSystemNotReady) ||
+			isGrpcMonitorKeyPresentInContext(ctx) {
+			c.stateLock.Lock()
+			if c.state == stateConnected {
+				logger.Warnw(context.Background(), "sending-disconnect-event", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+				c.state = stateDisconnected
+				c.events <- eventDisconnected
+			}
+			c.stateLock.Unlock()
+		} else if strings.Contains(err.Error(), connectionClosedSubstring) {
+			logger.Errorw(context.Background(), "invalid-client-connection-closed", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+		}
+		return err
+	}
+	// Update activity on success only
+	c.updateActivity(ctx)
+	return nil
+}
+
+// updateActivity pushes an activity indication on the channel so that the monitoring routine does not validate
+// the gRPC connection when the connection is being used. Note that this update is done both when the connection
+// is alive or a connection error is returned. A separate routine takes care of doing the re-connect.
+func (c *Client) updateActivity(ctx context.Context) {
+	c.activeChMutex.RLock()
+	defer c.activeChMutex.RUnlock()
+	if c.activeCh != nil {
+		logger.Debugw(ctx, "update-activity", log.Fields{"api-endpoint": c.apiEndPoint})
+		c.activeCh <- struct{}{}
+
+		// Update liveness only in connected state
+		if c.livenessCallback != nil {
+			c.stateLock.RLock()
+			if c.state == stateConnected {
+				c.livenessCallback(time.Now())
+			}
+			c.stateLock.RUnlock()
+		}
+	}
+}
+
+func WithGrpcMonitorContext(ctx context.Context, name string) context.Context {
+	ctx = context.WithValue(ctx, grpcMonitorContextKey, name)
+	return ctx
+}
+
+func isGrpcMonitorKeyPresentInContext(ctx context.Context) bool {
+	if ctx != nil {
+		_, present := ctx.Value(grpcMonitorContextKey).(string)
+		return present
+	}
+	return false
+}
+
+// monitorActivity monitors the activity on the gRPC connection.   If there are no activity after a specified
+// timeout, it will send a default API request on that connection.   If the connection is good then nothing
+// happens.  If it's bad this will trigger reconnection attempts.
+func (c *Client) monitorActivity(ctx context.Context, handler SetAndTestServiceHandler) {
+	logger.Infow(ctx, "start-activity-monitor", log.Fields{"endpoint": c.apiEndPoint})
+
+	// Create an activity monitor channel.  Unbuffered channel works well.  However, we use a buffered
+	// channel here as a safeguard of having the grpc interceptor publishing too many events that can be
+	// consumed by this monitoring thread
+	c.activeChMutex.Lock()
+	c.activeCh = make(chan struct{}, 10)
+	c.activeChMutex.Unlock()
+
+	// Interval to wait for no activity before probing the connection
+	timeout := c.monitorInterval
+loop:
+	for {
+		timeoutTimer := time.NewTimer(timeout)
+		select {
+
+		case <-c.activeCh:
+			logger.Debugw(ctx, "received-active-notification", log.Fields{"endpoint": c.apiEndPoint})
+
+			// Reset timer
+			if !timeoutTimer.Stop() {
+				<-timeoutTimer.C
+			}
+
+		case <-ctx.Done():
+			break loop
+
+		case <-timeoutTimer.C:
+			// Trigger an activity check if the state is connected.  If the state is not connected then there is already
+			// a backoff retry mechanism in place to retry establishing connection.
+			c.stateLock.RLock()
+			runCheck := c.state == stateConnected
+			c.stateLock.RUnlock()
+			if runCheck {
+				go func() {
+					logger.Debugw(ctx, "connection-check-start", log.Fields{"api-endpoint": c.apiEndPoint})
+					subCtx, cancel := context.WithTimeout(ctx, c.backoffMaxInterval)
+					defer cancel()
+					subCtx = WithGrpcMonitorContext(subCtx, "grpc-monitor")
+					c.connectionLock.RLock()
+					defer c.connectionLock.RUnlock()
+					if c.connection != nil {
+						response := handler(subCtx, c.connection)
+						logger.Debugw(ctx, "connection-check-response", log.Fields{"api-endpoint": c.apiEndPoint, "up": response != nil})
+					}
+				}()
+			}
+		}
+	}
+	logger.Infow(ctx, "activity-monitor-stopping", log.Fields{"endpoint": c.apiEndPoint})
+}
+
+// Start kicks off the adapter agent by trying to connect to the adapter
+func (c *Client) Start(ctx context.Context, handler SetAndTestServiceHandler) {
+	logger.Debugw(ctx, "Starting GRPC - Client", log.Fields{"api-endpoint": c.apiEndPoint})
+
+	// If the context contains a k8s probe then register services
+	p := probe.GetProbeFromContext(ctx)
+	if p != nil {
+		p.RegisterService(ctx, c.apiEndPoint)
+	}
+
+	// Enable activity check, if required
+	if c.activityCheck {
+		go c.monitorActivity(ctx, handler)
+	}
+
+	initialConnection := true
+	c.events <- eventConnecting
+	backoff := NewBackoff(c.backoffInitialInterval, c.backoffMaxInterval, c.backoffMaxElapsedTime)
+	attempt := 1
+loop:
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Debugw(ctx, "context-closing", log.Fields{"endpoint": c.apiEndPoint})
+			return
+		case event := <-c.events:
+			logger.Debugw(ctx, "received-event", log.Fields{"event": event, "endpoint": c.apiEndPoint})
+			switch event {
+			case eventConnecting:
+				logger.Debugw(ctx, "connection-start", log.Fields{"endpoint": c.apiEndPoint, "attempts": attempt})
+
+				c.stateLock.Lock()
+				if c.state == stateConnected {
+					c.state = stateDisconnected
+				}
+				if c.state != stateConnecting {
+					c.state = stateConnecting
+					go func() {
+						if err := c.connectToEndpoint(ctx, handler, p); err != nil {
+							c.stateLock.Lock()
+							c.state = stateDisconnected
+							c.stateLock.Unlock()
+							logger.Errorw(ctx, "connection-failed", log.Fields{"endpoint": c.apiEndPoint, "attempt": attempt, "error": err})
+
+							// Retry connection after a delay
+							if err = backoff.Backoff(ctx); err != nil {
+								// Context has closed or reached maximum elapsed time, if set
+								logger.Errorw(ctx, "retry-aborted", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+								return
+							}
+							attempt += 1
+							c.events <- eventConnecting
+						} else {
+							backoff.Reset()
+						}
+					}()
+				}
+				c.stateLock.Unlock()
+
+			case eventConnected:
+				logger.Debugw(ctx, "endpoint-connected", log.Fields{"endpoint": c.apiEndPoint})
+				attempt = 1
+				c.stateLock.Lock()
+				if c.state != stateConnected {
+					c.state = stateConnected
+					if initialConnection {
+						logger.Debugw(ctx, "initial-endpoint-connection", log.Fields{"endpoint": c.apiEndPoint})
+						initialConnection = false
+					} else {
+						logger.Debugw(ctx, "endpoint-reconnection", log.Fields{"endpoint": c.apiEndPoint})
+						// Trigger any callback on a restart
+						go func() {
+							err := c.onRestart(log.WithSpanFromContext(context.Background(), ctx), c.apiEndPoint)
+							if err != nil {
+								logger.Errorw(ctx, "unable-to-restart-endpoint", log.Fields{"error": err, "endpoint": c.apiEndPoint})
+							}
+						}()
+					}
+				}
+				c.stateLock.Unlock()
+
+			case eventDisconnected:
+				if p != nil {
+					p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusNotReady)
+				}
+				logger.Debugw(ctx, "endpoint-disconnected", log.Fields{"endpoint": c.apiEndPoint, "status": c.state})
+
+				// Try to connect again
+				c.events <- eventConnecting
+
+			case eventStopped:
+				logger.Debugw(ctx, "endPoint-stopped", log.Fields{"adapter": c.apiEndPoint})
+				go func() {
+					if err := c.closeConnection(ctx, p); err != nil {
+						logger.Errorw(ctx, "endpoint-closing-connection-failed", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+					}
+				}()
+				break loop
+			case eventError:
+				logger.Errorw(ctx, "endpoint-error-event", log.Fields{"endpoint": c.apiEndPoint})
+			default:
+				logger.Errorw(ctx, "endpoint-unknown-event", log.Fields{"endpoint": c.apiEndPoint, "error": event})
+			}
+		}
+	}
+	logger.Infow(ctx, "endpoint-stopped", log.Fields{"endpoint": c.apiEndPoint})
+}
+
+func (c *Client) connectToEndpoint(ctx context.Context, handler SetAndTestServiceHandler, p *probe.Probe) error {
+	if p != nil {
+		p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusPreparing)
+	}
+
+	c.connectionLock.Lock()
+	defer c.connectionLock.Unlock()
+
+	if c.connection != nil {
+		_ = c.connection.Close()
+		c.connection = nil
+	}
+
+	c.service = nil
+
+	// Use Interceptors to:
+	// 1. automatically inject
+	// 2. publish Open Tracing Spans by this GRPC Client
+	// 3. detect connection failure on client calls such that the reconnection process can begin
+	conn, err := grpc.Dial(c.apiEndPoint,
+		grpc.WithInsecure(),
+		grpc.WithStreamInterceptor(grpc_middleware.ChainStreamClient(
+			grpc_opentracing.StreamClientInterceptor(grpc_opentracing.WithTracer(log.ActiveTracerProxy{})),
+		)),
+		grpc.WithUnaryInterceptor(grpc_middleware.ChainUnaryClient(
+			grpc_opentracing.UnaryClientInterceptor(grpc_opentracing.WithTracer(log.ActiveTracerProxy{})),
+		)),
+		grpc.WithUnaryInterceptor(c.clientInterceptor),
+		// Set keealive parameter - use default grpc values
+		grpc.WithKeepaliveParams(keepalive.ClientParameters{
+			Time:                c.monitorInterval,
+			Timeout:             c.backoffMaxInterval,
+			PermitWithoutStream: true,
+		}),
+	)
+
+	if err == nil {
+		subCtx, cancel := context.WithTimeout(ctx, c.backoffMaxInterval)
+		defer cancel()
+		svc := handler(subCtx, conn)
+		if svc != nil {
+			c.connection = conn
+			c.service = svc
+			if p != nil {
+				p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusRunning)
+			}
+			logger.Infow(ctx, "connected-to-endpoint", log.Fields{"endpoint": c.apiEndPoint})
+			c.events <- eventConnected
+			return nil
+		}
+	}
+	logger.Warnw(ctx, "Failed to connect to endpoint",
+		log.Fields{
+			"endpoint": c.apiEndPoint,
+			"error":    err,
+		})
+
+	if p != nil {
+		p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusFailed)
+	}
+	return fmt.Errorf("no connection to endpoint %s", c.apiEndPoint)
+}
+
+func (c *Client) closeConnection(ctx context.Context, p *probe.Probe) error {
+	if p != nil {
+		p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusStopped)
+	}
+
+	c.connectionLock.Lock()
+	defer c.connectionLock.Unlock()
+
+	if c.connection != nil {
+		err := c.connection.Close()
+		c.connection = nil
+		return err
+	}
+
+	return nil
+}
+
+func (c *Client) Stop(ctx context.Context) {
+	if !c.done {
+		c.events <- eventStopped
+		close(c.events)
+		c.done = true
+	}
+}
+
+// SetService is used for testing only
+func (c *Client) SetService(srv interface{}) {
+	c.connectionLock.Lock()
+	defer c.connectionLock.Unlock()
+	c.service = srv
+}
+
+func (c *Client) SubscribeForLiveness(callback func(timestamp time.Time)) {
+	c.livenessCallback = callback
+}
diff --git a/pkg/grpc/client_test.go b/pkg/grpc/client_test.go
new file mode 100644
index 0000000..d8ea31d
--- /dev/null
+++ b/pkg/grpc/client_test.go
@@ -0,0 +1,530 @@
+/*
+ * 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 grpc
+
+import (
+	"context"
+	"fmt"
+	"os"
+	"strconv"
+	"sync"
+	"testing"
+	"time"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"github.com/phayes/freeport"
+	"github.com/stretchr/testify/assert"
+	"google.golang.org/grpc"
+)
+
+const (
+	testGrpcServer  = "test-grpc-server"
+	initialInterval = 100 * time.Millisecond
+	maxInterval     = 5000 * time.Millisecond
+	maxElapsedTime  = 0 * time.Millisecond
+	monitorInterval = 2 * time.Second
+	timeout         = 10 * time.Second
+)
+
+var testForNoActivityCh = make(chan time.Time, 10)
+
+type testCoreServer struct {
+	apiEndPoint string
+	server      *GrpcServer
+	probe       *probe.Probe
+}
+
+func newTestCoreServer(apiEndpoint string) *testCoreServer {
+	return &testCoreServer{
+		apiEndPoint: apiEndpoint,
+		probe:       &probe.Probe{},
+	}
+}
+
+func (s *testCoreServer) registerService(ctx context.Context, t *testing.T) {
+	assert.NotEqual(t, "", s.apiEndPoint)
+
+	probePort, err := freeport.GetFreePort()
+	assert.Nil(t, err)
+	probeEndpoint := "127.0.0.1:" + strconv.Itoa(probePort)
+	go s.probe.ListenAndServe(ctx, probeEndpoint)
+	s.probe.RegisterService(ctx, testGrpcServer)
+
+	s.server = NewGrpcServer(s.apiEndPoint, nil, false, s.probe)
+
+	s.server.AddService(func(server *grpc.Server) {
+		core.RegisterCoreServiceServer(server, &MockCoreServiceHandler{})
+	})
+}
+
+func (s *testCoreServer) start(ctx context.Context, t *testing.T) {
+	assert.NotNil(t, s.server)
+	assert.NotEqual(t, "", s.apiEndPoint)
+
+	s.probe.UpdateStatus(ctx, testGrpcServer, probe.ServiceStatusRunning)
+	s.server.Start(ctx)
+	s.probe.UpdateStatus(ctx, testGrpcServer, probe.ServiceStatusStopped)
+}
+
+func (s *testCoreServer) stop() {
+	if s.server != nil {
+		s.server.Stop()
+	}
+}
+
+type testClient struct {
+	apiEndPoint string
+	probe       *probe.Probe
+	client      *Client
+}
+
+func serverRestarted(ctx context.Context, endPoint string) error {
+	logger.Infow(ctx, "remote-restarted", log.Fields{"endpoint": endPoint})
+	return nil
+}
+
+func newTestClient(apiEndpoint string, handler RestartedHandler) *testClient {
+	tc := &testClient{
+		apiEndPoint: apiEndpoint,
+		probe:       &probe.Probe{},
+	}
+	// Set the environment variables that this client will use
+	var err error
+	err = os.Setenv(grpcBackoffInitialInterval, initialInterval.String())
+	if err != nil {
+		logger.Warnw(context.Background(), "setting-env-variable-failed", log.Fields{"error": err, "variable": grpcBackoffInitialInterval})
+		return nil
+	}
+	err = os.Setenv(grpcBackoffInitialInterval, maxInterval.String())
+	if err != nil {
+		logger.Warnw(context.Background(), "setting-env-variable-failed", log.Fields{"error": err, "variable": grpcBackoffInitialInterval})
+		return nil
+	}
+	err = os.Setenv(grpcBackoffMaxElapsedTime, maxElapsedTime.String())
+	if err != nil {
+		logger.Warnw(context.Background(), "setting-env-variable-failed", log.Fields{"error": err, "variable": grpcBackoffMaxElapsedTime})
+		return nil
+	}
+
+	err = os.Setenv(grpcMonitorInterval, monitorInterval.String())
+	if err != nil {
+		logger.Warnw(context.Background(), "setting-env-variable-failed", log.Fields{"error": err, "variable": grpcMonitorInterval})
+		return nil
+	}
+
+	tc.client, err = NewClient(apiEndpoint,
+		handler,
+		ActivityCheck(true))
+	if err != nil {
+		return nil
+	}
+	return tc
+}
+
+func setAndTestCoreServiceHandler(ctx context.Context, conn *grpc.ClientConn) interface{} {
+	if conn == nil {
+		return nil
+	}
+	svc := core.NewCoreServiceClient(conn)
+	if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+		return nil
+	}
+	return svc
+}
+
+func idleConnectionTest(ctx context.Context, conn *grpc.ClientConn) interface{} {
+	if conn == nil {
+		return nil
+	}
+	svc := core.NewCoreServiceClient(conn)
+	if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+		return nil
+	}
+	testForNoActivityCh <- time.Now()
+	return svc
+}
+
+func (c *testClient) start(ctx context.Context, t *testing.T, handler SetAndTestServiceHandler) {
+	assert.NotNil(t, c.client)
+
+	probePort, err := freeport.GetFreePort()
+	assert.Nil(t, err)
+	probeEndpoint := "127.0.0.1:" + strconv.Itoa(probePort)
+	go c.probe.ListenAndServe(ctx, probeEndpoint)
+
+	probeCtx := context.WithValue(ctx, probe.ProbeContextKey, c.probe)
+	c.client.Start(probeCtx, handler)
+}
+
+func (c *testClient) getClient(t *testing.T) core.CoreServiceClient {
+	gc, err := c.client.GetClient()
+	assert.Nil(t, err)
+	coreClient, ok := gc.(core.CoreServiceClient)
+	assert.True(t, ok)
+	return coreClient
+}
+
+func serverStartsFirstTest(t *testing.T) {
+	// Setup
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+
+	// Create and start the test server
+	grpcPort, err := freeport.GetFreePort()
+	assert.Nil(t, err)
+	apiEndpoint := "127.0.0.1:" + strconv.Itoa(grpcPort)
+	ts := newTestCoreServer(apiEndpoint)
+	ts.registerService(ctx, t)
+	go ts.start(ctx, t)
+
+	// Create the test client and start it
+	tc := newTestClient(apiEndpoint, serverRestarted)
+	assert.NotNil(t, tc)
+	go tc.start(ctx, t, setAndTestCoreServiceHandler)
+
+	// Test 1: Verify that probe status shows ready eventually
+	var servicesReady isConditionSatisfied = func() bool {
+		return ts.probe.IsReady() && tc.probe.IsReady()
+	}
+	err = waitUntilCondition(timeout, servicesReady)
+	assert.Nil(t, err)
+
+	// Test 2: Verify we get a valid client and can make grpc requests with it
+	coreClient := tc.getClient(t)
+	assert.NotNil(t, coreClient)
+
+	device, err := coreClient.GetDevice(context.Background(), &common.ID{Id: "1234"})
+	assert.Nil(t, err)
+	assert.NotNil(t, device)
+	assert.Equal(t, "test-1234", device.Type)
+}
+
+func clientStartsFirstTest(t *testing.T) {
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+
+	// Create a grpc endpoint for the server
+	grpcPort, err := freeport.GetFreePort()
+	assert.Nil(t, err)
+	apiEndpoint := "127.0.0.1:" + strconv.Itoa(grpcPort)
+
+	// Create the test client and start it
+	tc := newTestClient(apiEndpoint, serverRestarted)
+	assert.NotNil(t, tc)
+	go tc.start(ctx, t, setAndTestCoreServiceHandler)
+
+	// Verify client is not ready
+	var clientNotReady isConditionSatisfied = func() bool {
+		serviceStatus := tc.probe.GetStatus(apiEndpoint)
+		return serviceStatus == probe.ServiceStatusNotReady ||
+			serviceStatus == probe.ServiceStatusPreparing ||
+			serviceStatus == probe.ServiceStatusFailed
+	}
+	err = waitUntilCondition(timeout, clientNotReady)
+	assert.Nil(t, err)
+
+	// Create and start the test server
+	ts := newTestCoreServer(apiEndpoint)
+	ts.registerService(ctx, t)
+	go ts.start(ctx, t)
+
+	// Test 1: Verify that probe status shows ready eventually
+	var servicesReady isConditionSatisfied = func() bool {
+		return ts.probe.IsReady() && tc.probe.IsReady()
+	}
+	err = waitUntilCondition(timeout, servicesReady)
+	assert.Nil(t, err)
+
+	// Test 2: Verify we get a valid client and can make grpc requests with it
+	coreClient := tc.getClient(t)
+	assert.NotNil(t, coreClient)
+
+	device, err := coreClient.GetDevice(context.Background(), &common.ID{Id: "1234"})
+	assert.Nil(t, err)
+	assert.NotNil(t, device)
+	assert.Equal(t, "test-1234", device.Type)
+}
+
+// Liveness function
+func livessness(timestamp time.Time) {
+	logger.Debugw(context.Background(), "received-liveness", log.Fields{"timestamp": timestamp})
+}
+
+func serverRestarts(t *testing.T, numRestartRuns int) {
+	// Setup
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+
+	// Create and start the test server
+	grpcPort, err := freeport.GetFreePort()
+	assert.Nil(t, err)
+	apiEndpoint := "127.0.0.1:" + strconv.Itoa(grpcPort)
+	ts := newTestCoreServer(apiEndpoint)
+	ts.registerService(ctx, t)
+	go ts.start(ctx, t)
+
+	// Create the test client and start it
+	tc := newTestClient(apiEndpoint, serverRestarted)
+	assert.NotNil(t, tc)
+
+	// Subscribe for liveness
+	tc.client.SubscribeForLiveness(livessness)
+	go tc.start(ctx, t, setAndTestCoreServiceHandler)
+
+	// Test 1: Verify that probe status shows ready eventually
+	var servicesReady isConditionSatisfied = func() bool {
+		return ts.probe.IsReady() && tc.probe.IsReady()
+	}
+	err = waitUntilCondition(timeout, servicesReady)
+	assert.Nil(t, err)
+
+	// Test 2: Verify we get a valid client and can make grpc requests with it
+	coreClient := tc.getClient(t)
+	assert.NotNil(t, coreClient)
+
+	device, err := coreClient.GetDevice(context.Background(), &common.ID{Id: "1234"})
+	assert.Nil(t, err)
+	assert.NotNil(t, device)
+	assert.Equal(t, "test-1234", device.Type)
+
+	for i := 1; i <= numRestartRuns; i++ {
+		//Test 3: Stop server and verify server status
+		ts.stop()
+		var serverDown isConditionSatisfied = func() bool {
+			return ts.probe.GetStatus(testGrpcServer) == probe.ServiceStatusStopped
+		}
+		err = waitUntilCondition(timeout, serverDown)
+		assert.Nil(t, err)
+
+		// Make a grpc request - this will detect the server being down and automatically trigger the grpc client
+		// to reconnect
+		_, err = coreClient.GetDevice(context.Background(), &common.ID{Id: "1234"})
+		assert.NotNil(t, err)
+
+		// Wait until the client service shows as not ready. A wait is not needed.  It's just to verify that the
+		// client changes connection state.
+		var clientNotReady isConditionSatisfied = func() bool {
+			serviceStatus := tc.probe.GetStatus(apiEndpoint)
+			return serviceStatus == probe.ServiceStatusNotReady ||
+				serviceStatus == probe.ServiceStatusPreparing ||
+				serviceStatus == probe.ServiceStatusFailed
+		}
+		err = waitUntilCondition(timeout, clientNotReady)
+
+		assert.Nil(t, err)
+
+		// Keep the server down for 1/2 second
+		time.Sleep(500 * time.Millisecond)
+
+		// Test 4: Restart the server and verify the server is back online
+		go ts.start(ctx, t)
+		err = waitUntilCondition(timeout, servicesReady)
+		assert.Nil(t, err)
+
+		// Test 5: verify we can pull new device with a new client instance
+		coreClient = tc.getClient(t)
+		assert.NotNil(t, coreClient)
+		device, err := coreClient.GetDevice(context.Background(), &common.ID{Id: "1234"})
+		assert.Nil(t, err)
+		assert.Equal(t, "test-1234", device.Type)
+	}
+	// Stop the server
+	ts.stop()
+}
+
+func testNoActivity(t *testing.T) {
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+
+	// Create a grpc endpoint for the server
+	grpcPort, err := freeport.GetFreePort()
+	assert.Nil(t, err)
+	apiEndpoint := "127.0.0.1:" + strconv.Itoa(grpcPort)
+
+	// Create the test client and start it
+	tc := newTestClient(apiEndpoint, serverRestarted)
+	assert.NotNil(t, tc)
+	go tc.start(ctx, t, idleConnectionTest)
+
+	// Create and start the test server
+	ts := newTestCoreServer(apiEndpoint)
+	ts.registerService(ctx, t)
+	go ts.start(ctx, t)
+
+	// Test 1: Verify that probe status shows ready eventually
+	var servicesReady isConditionSatisfied = func() bool {
+		return ts.probe.IsReady() && tc.probe.IsReady()
+	}
+	err = waitUntilCondition(timeout, servicesReady)
+	assert.Nil(t, err)
+
+	// Test 2: Verify we get a valid client and can make grpc requests with it
+	coreClient := tc.getClient(t)
+	assert.NotNil(t, coreClient)
+
+	device, err := coreClient.GetDevice(context.Background(), &common.ID{Id: "1234"})
+	assert.Nil(t, err)
+	assert.NotNil(t, device)
+	assert.Equal(t, "test-1234", device.Type)
+
+	start := time.Now()
+	numChecks := 3 // Test for 3 checks
+	// Wait on the the idle channel - on no activity a connection probe will be attempted by the client
+	timer := time.NewTimer((monitorInterval + 1*time.Second) * time.Duration(numChecks))
+	defer timer.Stop()
+	count := 0
+loop:
+	for {
+		select {
+		case timestamp := <-testForNoActivityCh:
+			if timestamp.After(start) {
+				count += 1
+				if count > numChecks {
+					break loop
+				}
+			}
+		case <-timer.C:
+			t.Fatal("no activity on the idle channel")
+		}
+	}
+}
+
+func testServerLimit(t *testing.T) {
+	t.Skip() // Not needed for regular unit tests
+
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+
+	// Create a grpc endpoint for the server
+	grpcPort, err := freeport.GetFreePort()
+	assert.Nil(t, err)
+	apiEndpoint := "127.0.0.1:" + strconv.Itoa(grpcPort)
+
+	// Create the test client and start it
+	tc := newTestClient(apiEndpoint, serverRestarted)
+	assert.NotNil(t, tc)
+	go tc.start(ctx, t, idleConnectionTest)
+
+	// Create and start the test server
+	ts := newTestCoreServer(apiEndpoint)
+	ts.registerService(ctx, t)
+	go ts.start(ctx, t)
+
+	// Test 1: Verify that probe status shows ready eventually
+	var servicesReady isConditionSatisfied = func() bool {
+		return ts.probe.IsReady() && tc.probe.IsReady()
+	}
+	err = waitUntilCondition(timeout, servicesReady)
+	assert.Nil(t, err)
+
+	// Test 2: Verify we get a valid client and can make grpc requests with it
+	coreClient := tc.getClient(t)
+	assert.NotNil(t, coreClient)
+
+	var lock sync.RWMutex
+	bad := []time.Duration{}
+	bad_err := []string{}
+	good := []time.Duration{}
+	var wg sync.WaitGroup
+	numRPCs := 10
+	total_good := time.Duration(0)
+	max_good := time.Duration(0)
+	total_bad := time.Duration(0)
+	max_bad := time.Duration(0)
+	order_received := []uint32{}
+	for i := 1; i <= numRPCs; i++ {
+		wg.Add(1)
+		go func(seq int) {
+			local := time.Now()
+			ctx, cancel := context.WithTimeout(context.Background(), 10000*time.Millisecond)
+			defer cancel()
+			var err error
+			var d *voltha.Device
+			d, err = coreClient.GetDevice(ctx, &common.ID{Id: strconv.Itoa(seq)})
+			if err != nil {
+				lock.Lock()
+				bad = append(bad, time.Since(local))
+				bad_err = append(bad_err, err.Error())
+				total_bad += time.Since(local)
+				if time.Since(local) > max_bad {
+					max_bad = time.Since(local)
+				}
+				logger.Errorw(ctx, "error produced", log.Fields{"error": err})
+				lock.Unlock()
+			} else {
+				lock.Lock()
+				good = append(good, time.Since(local))
+				total_good += time.Since(local)
+				if time.Since(local) > max_good {
+					max_good = time.Since(local)
+				}
+				if d != nil {
+					order_received = append(order_received, d.Vlan)
+				}
+				lock.Unlock()
+			}
+			wg.Done()
+		}(i)
+	}
+	wg.Wait()
+	assert.Equal(t, 0, len(bad))
+	assert.Equal(t, numRPCs, len(good))
+	//fmt.Println("Bad:", bad[:10])
+	if len(bad_err) > 0 {
+		fmt.Println("Bad Err Last:", bad_err[len(bad_err)-1:])
+		fmt.Println("Bad Err First:", bad_err[:1])
+	}
+	fmt.Println("Good:", good[len(good)-10:])
+	fmt.Println("Good average time:", total_good.Milliseconds()/int64(numRPCs))
+	fmt.Println("Bad average time:", total_bad.Milliseconds()/int64(numRPCs))
+	fmt.Println("Bad Max:", max_bad)
+	fmt.Println("Good Max:", max_good)
+	//fmt.Println("Order received:", order_received)
+
+	prev := order_received[0]
+
+	for i := 1; i < len(order_received); i++ {
+		if order_received[i] < prev {
+			fmt.Println("Prev:", prev, " curr:", order_received[i])
+		}
+		prev = order_received[i]
+	}
+}
+
+func TestSuiteClient3(t *testing.T) {
+	// Setup
+	log.SetAllLogLevel(volthaTestLogLevel)
+
+	// Test starting server before client
+	serverStartsFirstTest(t)
+
+	// Test starting client before server
+	clientStartsFirstTest(t)
+
+	// Test server restarts
+	serverRestarts(t, 1)
+
+	//Test that the client test the grpc connection on no activity
+	testNoActivity(t)
+
+	// Test client queueing with server limit
+	testServerLimit(t)
+}
diff --git a/pkg/grpc/common.go b/pkg/grpc/common.go
index 7ef7c17..77aad4f 100644
--- a/pkg/grpc/common.go
+++ b/pkg/grpc/common.go
@@ -16,7 +16,7 @@
 package grpc
 
 import (
-	"github.com/opencord/voltha-lib-go/v6/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/pkg/grpc/common_test.go b/pkg/grpc/common_test.go
index 7760c33..7a5f394 100644
--- a/pkg/grpc/common_test.go
+++ b/pkg/grpc/common_test.go
@@ -16,7 +16,10 @@
 package grpc
 
 import (
-	"github.com/opencord/voltha-lib-go/v6/pkg/log"
+	"fmt"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 const (
@@ -30,15 +33,44 @@
 	 * useful.
 	 */
 
-	VOLTHA_LOGLEVEL = log.FatalLevel
+	volthaTestLogLevel = log.FatalLevel
+	retryInterval      = 50 * time.Millisecond
 )
 
+type isConditionSatisfied func() bool
+
 // Unit test initialization. This init() function handles all unit tests in
 // the current directory.
 func init() {
 	// Logger must be configured or bad things happen
-	_, err := log.SetDefaultLogger(log.JSON, VOLTHA_LOGLEVEL, log.Fields{"instanceId": 1})
+	_, err := log.SetDefaultLogger(log.JSON, volthaTestLogLevel, log.Fields{"instanceId": 1})
 	if err != nil {
 		panic(err)
 	}
 }
+
+func waitUntilCondition(timeout time.Duration, verificationFunction isConditionSatisfied) error {
+	ch := make(chan int, 1)
+	done := false
+	go func() {
+		for {
+			if verificationFunction() {
+				ch <- 1
+				break
+			}
+			if done {
+				break
+			}
+			time.Sleep(retryInterval)
+		}
+	}()
+	timer := time.NewTimer(timeout)
+	defer timer.Stop()
+	select {
+	case <-ch:
+		return nil
+	case <-timer.C:
+		done = true
+		return fmt.Errorf("timeout-waiting-for-condition")
+	}
+}
diff --git a/pkg/grpc/mock_core_service.go b/pkg/grpc/mock_core_service.go
new file mode 100644
index 0000000..745753c
--- /dev/null
+++ b/pkg/grpc/mock_core_service.go
@@ -0,0 +1,135 @@
+/*
+ * 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 grpc
+
+import (
+	"context"
+	"strconv"
+	"time"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+//MockCoreServiceHandler implements the methods in the core service
+type MockCoreServiceHandler struct{}
+
+func (handler *MockCoreServiceHandler) RegisterAdapter(ctx context.Context, reg *ic.AdapterRegistration) (*empty.Empty, error) {
+	//logger.Debugw(ctx, "registration-received", log.Fields{"input": reg})
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeviceUpdate(context.Context, *voltha.Device) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) PortCreated(context.Context, *voltha.Port) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) PortsStateUpdate(context.Context, *ic.PortStateFilter) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeleteAllPorts(context.Context, *common.ID) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetDevicePort(context.Context, *ic.PortFilter) (*voltha.Port, error) {
+	return &voltha.Port{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ListDevicePorts(context.Context, *common.ID) (*voltha.Ports, error) {
+	return &voltha.Ports{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeviceStateUpdate(context.Context, *ic.DeviceStateFilter) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DevicePMConfigUpdate(context.Context, *voltha.PmConfigs) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildDeviceDetected(context.Context, *ic.DeviceDiscovery) (*voltha.Device, error) {
+	return &voltha.Device{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildDevicesLost(context.Context, *common.ID) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildDevicesDetected(context.Context, *common.ID) (*empty.Empty, error) {
+	time.Sleep(50 * time.Millisecond)
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetDevice(ctx context.Context, id *common.ID) (*voltha.Device, error) {
+	time.Sleep(50 * time.Millisecond)
+	vlan, _ := strconv.Atoi(id.Id)
+	return &voltha.Device{
+		Id:   id.Id,
+		Type: "test-1234",
+		Vlan: uint32(vlan),
+	}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetChildDevice(context.Context, *ic.ChildDeviceFilter) (*voltha.Device, error) {
+	return nil, nil
+}
+
+func (handler *MockCoreServiceHandler) GetChildDevices(context.Context, *common.ID) (*voltha.Devices, error) {
+	return &voltha.Devices{}, nil
+}
+
+func (handler *MockCoreServiceHandler) SendPacketIn(context.Context, *ic.PacketIn) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeviceReasonUpdate(context.Context, *ic.DeviceReason) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) PortStateUpdate(context.Context, *ic.PortState) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+// Additional API found in the Core - unused?
+func (handler *MockCoreServiceHandler) ReconcileChildDevices(context.Context, *common.ID) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetChildDeviceWithProxyAddress(context.Context, *voltha.Device_ProxyAddress) (*voltha.Device, error) {
+	return &voltha.Device{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetPorts(context.Context, *ic.PortFilter) (*voltha.Ports, error) {
+	return &voltha.Ports{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildrenStateUpdate(context.Context, *ic.DeviceStateFilter) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) UpdateImageDownload(context.Context, *voltha.ImageDownload) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetHealthStatus(ctx context.Context, empty *empty.Empty) (*voltha.HealthStatus, error) {
+	return &voltha.HealthStatus{State: voltha.HealthStatus_HEALTHY}, nil
+}
diff --git a/pkg/grpc/server.go b/pkg/grpc/server.go
index 3674c91..bee418d 100644
--- a/pkg/grpc/server.go
+++ b/pkg/grpc/server.go
@@ -21,7 +21,7 @@
 
 	grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
 	grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
-	"github.com/opencord/voltha-lib-go/v6/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/credentials"
diff --git a/pkg/grpc/utils.go b/pkg/grpc/utils.go
new file mode 100644
index 0000000..85686de
--- /dev/null
+++ b/pkg/grpc/utils.go
@@ -0,0 +1,126 @@
+/*
+ * 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 grpc
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"math"
+	"os"
+	"reflect"
+	"sync"
+	"time"
+)
+
+const (
+	incrementalFactor = 1.5
+	minBackOff        = 10 * time.Millisecond
+)
+
+type Backoff struct {
+	attempt          int
+	initialInterval  time.Duration
+	maxElapsedTime   time.Duration
+	maxInterval      time.Duration
+	totalElapsedTime time.Duration
+	mutex            sync.RWMutex
+}
+
+func NewBackoff(initialInterval, maxInterval, maxElapsedTime time.Duration) *Backoff {
+	bo := &Backoff{}
+	bo.initialInterval = initialInterval
+	bo.maxInterval = maxInterval
+	bo.maxElapsedTime = maxElapsedTime
+	return bo
+}
+
+func (bo *Backoff) Backoff(ctx context.Context) error {
+	duration, err := bo.getBackOffDuration()
+	if err != nil {
+		return err
+	}
+
+	ticker := time.NewTicker(duration)
+	defer ticker.Stop()
+	select {
+	case <-ctx.Done():
+		return ctx.Err()
+	case <-ticker.C:
+	}
+	return nil
+}
+
+func (bo *Backoff) getBackOffDuration() (duration time.Duration, err error) {
+	err = nil
+	defer func() {
+		bo.mutex.Lock()
+		defer bo.mutex.Unlock()
+		bo.attempt += 1
+		bo.totalElapsedTime += duration
+		if bo.maxElapsedTime > 0 && bo.totalElapsedTime > bo.maxElapsedTime {
+			err = errors.New("max elapsed backoff time reached")
+		}
+	}()
+
+	if bo.initialInterval <= minBackOff {
+		bo.initialInterval = minBackOff
+	}
+	if bo.initialInterval > bo.maxInterval {
+		duration = bo.initialInterval
+		return
+	}
+
+	// Calculate incremental duration
+	minf := float64(bo.initialInterval)
+	durf := minf * math.Pow(incrementalFactor, float64(bo.attempt))
+
+	if durf > math.MaxInt64 {
+		duration = bo.maxInterval
+		return
+	}
+	duration = time.Duration(durf)
+
+	//Keep within bounds
+	if duration < bo.initialInterval {
+		duration = bo.initialInterval
+	}
+	if duration > bo.maxInterval {
+		duration = bo.maxInterval
+	}
+	return
+}
+
+func (bo *Backoff) Reset() {
+	bo.mutex.Lock()
+	defer bo.mutex.Unlock()
+	bo.attempt = 0
+	bo.totalElapsedTime = 0
+}
+
+func SetFromEnvVariable(key string, variableToSet interface{}) error {
+	if _, ok := variableToSet.(*time.Duration); !ok {
+		return fmt.Errorf("unsupported type %T", variableToSet)
+	}
+	if valStr, present := os.LookupEnv(key); present {
+		val, err := time.ParseDuration(valStr)
+		if err != nil {
+			return err
+		}
+		reflect.ValueOf(variableToSet).Elem().Set(reflect.ValueOf(val))
+	}
+	return nil
+}
diff --git a/pkg/grpc/utils_test.go b/pkg/grpc/utils_test.go
new file mode 100644
index 0000000..55ffdf8
--- /dev/null
+++ b/pkg/grpc/utils_test.go
@@ -0,0 +1,103 @@
+/*
+ * 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 grpc
+
+import (
+	"context"
+	"os"
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestBackoffNoWait(t *testing.T) {
+	initTime := 1 * time.Millisecond
+	maxTime := 500 * time.Millisecond
+	maxElapsedTime := 0 * time.Millisecond
+	backoff := NewBackoff(initTime, maxTime, maxElapsedTime)
+	ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond)
+	defer cancel()
+	err := backoff.Backoff(ctx)
+	assert.Nil(t, err)
+}
+
+func TestBackoffElapsedTime(t *testing.T) {
+	initTime := 10 * time.Millisecond
+	maxTime := 50 * time.Millisecond
+	maxElapsedTime := 500 * time.Millisecond
+	b := NewBackoff(initTime, maxTime, maxElapsedTime)
+	start := time.Now()
+loop:
+	for {
+		err := b.Backoff(context.Background())
+		if err != nil {
+			break loop
+		}
+	}
+	assert.GreaterOrEqual(t, time.Since(start).Milliseconds(), maxTime.Milliseconds())
+}
+
+func TestBackoffSuccess(t *testing.T) {
+	initTime := 100 * time.Millisecond
+	maxTime := 500 * time.Millisecond
+	maxElapsedTime := 0 * time.Millisecond
+	backoff := NewBackoff(initTime, maxTime, maxElapsedTime)
+	ctx, cancel := context.WithTimeout(context.Background(), 2000*time.Millisecond)
+	defer cancel()
+	previous := time.Duration(0)
+	for i := 1; i < 5; i++ {
+		start := time.Now()
+		err := backoff.Backoff(ctx)
+		assert.Nil(t, err)
+		current := time.Since(start)
+		if current < maxTime {
+			assert.GreaterOrEqual(t, current.Milliseconds(), previous.Milliseconds())
+		}
+		previous = current
+	}
+}
+
+func TestBackoffContextTimeout(t *testing.T) {
+	initTime := 1000 * time.Millisecond
+	maxTime := 1100 * time.Millisecond
+	maxElapsedTime := 0 * time.Millisecond
+	backoff := NewBackoff(initTime, maxTime, maxElapsedTime)
+	ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond)
+	defer cancel()
+	err := backoff.Backoff(ctx)
+	assert.NotNil(t, err)
+	assert.Equal(t, context.DeadlineExceeded, err)
+}
+
+func TestSetFromEnvVariable(t *testing.T) {
+	// 1. Test with unsupported type
+	var valInt int
+	err := SetFromEnvVariable("MY-KEY", valInt)
+	assert.NotNil(t, err)
+
+	//2. Test with supported type but no env variable present
+	var valDuration time.Duration
+	err = SetFromEnvVariable("MY-KEY", &valDuration)
+	assert.Nil(t, err)
+
+	//3. Test with supported type and env variable present
+	err = os.Setenv("MY-KEY", "10s")
+	assert.Nil(t, err)
+	err = SetFromEnvVariable("MY-KEY", &valDuration)
+	assert.Nil(t, err)
+	assert.Equal(t, 10*time.Second, valDuration)
+}