VOL-2337 There is only one Multicast GEM port id value system-wise and it is
predefined in the technology profile. ResourceManager should reserve this GEM
port value and not give it to the GEM port requesters to avoid GEM port id
collisions.

Change-Id: Iaf878e695d3cd427844ec0464fe108ca0c25cca2
diff --git a/VERSION b/VERSION
index 4d92398..d19cf84 100644
--- a/VERSION
+++ b/VERSION
@@ -1 +1 @@
-3.0.16-dev
+3.0.17
diff --git a/pkg/ponresourcemanager/ponresourcemanager.go b/pkg/ponresourcemanager/ponresourcemanager.go
index 6ed18e9..79fefc5 100755
--- a/pkg/ponresourcemanager/ponresourcemanager.go
+++ b/pkg/ponresourcemanager/ponresourcemanager.go
@@ -42,6 +42,16 @@
 	//Constants for passing command line arugments
 	OLT_MODEL_ARG = "--olt_model"
 	PATH_PREFIX   = "service/voltha/resource_manager/{%s}"
+
+	/*The path under which configuration data is stored is defined as technology/device agnostic.
+	  That means the path does not include any specific technology/device variable. Using technology/device
+	  agnostic path also makes northbound applications, that need to write to this path,
+	  technology/device agnostic.
+
+	  Default kv client of PonResourceManager reads from/writes to PATH_PREFIX defined above.
+	  That is why, an additional kv client (named KVStoreForConfig) is defined to read from the config path.
+	*/
+	PATH_PREFIX_FOR_CONFIG = "service/voltha/resource_manager/config"
 	/*The resource ranges for a given device model should be placed
 	  at 'resource_manager/<technology>/resource_ranges/<olt_model_type>'
 	  path on the KV store.
@@ -115,6 +125,9 @@
 	NUM_OF_PON_INTF = 16
 
 	KVSTORE_RETRY_TIMEOUT = 5
+	//Path on the KV store for storing reserved gem ports
+	//Format: reserved_gemport_ids
+	RESERVED_GEMPORT_IDS_PATH = "reserved_gemport_ids"
 )
 
 //type ResourceTypeIndex string
@@ -122,15 +135,16 @@
 
 type PONResourceManager struct {
 	//Implements APIs to initialize/allocate/release alloc/gemport/onu IDs.
-	Technology     string
-	DeviceType     string
-	DeviceID       string
-	Backend        string // ETCD, or consul
-	Host           string // host ip of the KV store
-	Port           int    // port number for the KV store
-	OLTModel       string
-	KVStore        *db.Backend
-	TechProfileMgr tp.TechProfileIf // create object of *tp.TechProfileMgr
+	Technology       string
+	DeviceType       string
+	DeviceID         string
+	Backend          string // ETCD, or consul
+	Host             string // host ip of the KV store
+	Port             int    // port number for the KV store
+	OLTModel         string
+	KVStore          *db.Backend
+	KVStoreForConfig *db.Backend
+	TechProfileMgr   tp.TechProfileIf // create object of *tp.TechProfileMgr
 
 	// Below attribute, pon_resource_ranges, should be initialized
 	// by reading from KV store.
@@ -152,7 +166,7 @@
 	return nil, errors.New("unsupported-kv-store")
 }
 
-func SetKVClient(Technology string, Backend string, Host string, Port int) *db.Backend {
+func SetKVClient(Technology string, Backend string, Host string, Port int, configClient bool) *db.Backend {
 	addr := Host + ":" + strconv.Itoa(Port)
 	// TODO : Make sure direct call to NewBackend is working fine with backend , currently there is some
 	// issue between kv store and backend , core is not calling NewBackend directly
@@ -161,13 +175,21 @@
 		log.Fatalw("Failed to init KV client\n", log.Fields{"err": err})
 		return nil
 	}
+
+	var pathPrefix string
+	if configClient {
+		pathPrefix = PATH_PREFIX_FOR_CONFIG
+	} else {
+		pathPrefix = fmt.Sprintf(PATH_PREFIX, Technology)
+	}
+
 	kvbackend := &db.Backend{
 		Client:     kvClient,
 		StoreType:  Backend,
 		Host:       Host,
 		Port:       Port,
 		Timeout:    KVSTORE_RETRY_TIMEOUT,
-		PathPrefix: fmt.Sprintf(PATH_PREFIX, Technology)}
+		PathPrefix: pathPrefix}
 
 	return kvbackend
 }
@@ -181,11 +203,17 @@
 	PONMgr.Backend = Backend
 	PONMgr.Host = Host
 	PONMgr.Port = Port
-	PONMgr.KVStore = SetKVClient(Technology, Backend, Host, Port)
+	PONMgr.KVStore = SetKVClient(Technology, Backend, Host, Port, false)
 	if PONMgr.KVStore == nil {
 		log.Error("KV Client initilization failed")
 		return nil, errors.New("Failed to init KV client")
 	}
+	// init kv client to read from the config path
+	PONMgr.KVStoreForConfig = SetKVClient(Technology, Backend, Host, Port, true)
+	if PONMgr.KVStoreForConfig == nil {
+		log.Error("KV Config Client initilization failed")
+		return nil, errors.New("Failed to init KV Config client")
+	}
 	// Initialize techprofile for this technology
 	if PONMgr.TechProfileMgr, _ = tp.NewTechProfile(&PONMgr, Backend, Host, Port); PONMgr.TechProfileMgr == nil {
 		log.Error("Techprofile initialization failed")
@@ -493,7 +521,15 @@
 		log.Debugf("Resource %s already present in store ", Path)
 		return nil
 	} else {
-		FormatResult, err := PONRMgr.FormatResource(Intf, StartID, EndID)
+		var excluded []uint32
+		if ResourceType == GEMPORT_ID {
+			//get gem port ids defined in the KV store, if any, and exclude them from the gem port id pool
+			if reservedGemPortIds, defined := PONRMgr.getReservedGemPortIdsFromKVStore(ctx); defined {
+				excluded = reservedGemPortIds
+				log.Debugw("Excluding some ports from GEM port id pool", log.Fields{"excluded gem ports": excluded})
+			}
+		}
+		FormatResult, err := PONRMgr.FormatResource(Intf, StartID, EndID, excluded)
 		if err != nil {
 			log.Errorf("Failed to format resource")
 			return err
@@ -511,12 +547,38 @@
 	return err
 }
 
-func (PONRMgr *PONResourceManager) FormatResource(IntfID uint32, StartIDx uint32, EndIDx uint32) ([]byte, error) {
+func (PONRMgr *PONResourceManager) getReservedGemPortIdsFromKVStore(ctx context.Context) ([]uint32, bool) {
+	var reservedGemPortIds []uint32
+	// read reserved gem ports from the config path
+	KvPair, err := PONRMgr.KVStoreForConfig.Get(ctx, RESERVED_GEMPORT_IDS_PATH)
+	if err != nil {
+		log.Errorw("Unable to get reserved GEM port ids from the kv store", log.Fields{"err": err})
+		return reservedGemPortIds, false
+	}
+	if KvPair == nil || KvPair.Value == nil {
+		//no reserved gem port defined in the store
+		return reservedGemPortIds, false
+	}
+	Val, err := kvstore.ToByte(KvPair.Value)
+	if err != nil {
+		log.Errorw("Failed to convert reserved gem port ids into byte array", log.Fields{"err": err})
+		return reservedGemPortIds, false
+	}
+	if err = json.Unmarshal(Val, &reservedGemPortIds); err != nil {
+		log.Errorw("Failed to unmarshal reservedGemPortIds", log.Fields{"err": err})
+		return reservedGemPortIds, false
+	}
+	return reservedGemPortIds, true
+}
+
+func (PONRMgr *PONResourceManager) FormatResource(IntfID uint32, StartIDx uint32, EndIDx uint32,
+	Excluded []uint32) ([]byte, error) {
 	/*
 	   Format resource as json.
 	   :param pon_intf_id: OLT PON interface id
 	   :param start_idx: start index for id pool
 	   :param end_idx: end index for id pool
+	   :Id values to be Excluded from the pool
 	   :return dictionary: resource formatted as map
 	*/
 	// Format resource as json to be stored in backend store
@@ -534,6 +596,14 @@
 		log.Error("Failed to create a bitmap")
 		return nil, errors.New("Failed to create bitmap")
 	}
+	for _, excludedID := range Excluded {
+		if excludedID < StartIDx || excludedID > EndIDx {
+			log.Warnf("Cannot reserve %d. It must be in the range of [%d, %d]", excludedID,
+				StartIDx, EndIDx)
+			continue
+		}
+		PONRMgr.reserveID(TSData, StartIDx, excludedID)
+	}
 	Resource[POOL] = TSData.Data(false) //we pass false so as the TSData lib api does not do a copy of the data and return
 
 	Value, err := json.Marshal(Resource)
@@ -1147,6 +1217,21 @@
 	return true
 }
 
+/* Reserves a unique id in the specified resource pool.
+:param Resource: resource used to reserve ID
+:param Id: ID to be reserved
+*/
+func (PONRMgr *PONResourceManager) reserveID(TSData *bitmap.Threadsafe, StartIndex uint32, Id uint32) bool {
+	Data := bitmap.TSFromData(TSData.Data(false), false)
+	if Data == nil {
+		log.Error("Failed to get resource pool")
+		return false
+	}
+	Idx := Id - StartIndex
+	Data.Set(int(Idx), true)
+	return true
+}
+
 func (PONRMgr *PONResourceManager) GetTechnology() string {
 	return PONRMgr.Technology
 }
diff --git a/pkg/ponresourcemanager/ponresourcemanager_test.go b/pkg/ponresourcemanager/ponresourcemanager_test.go
new file mode 100644
index 0000000..e4d8021
--- /dev/null
+++ b/pkg/ponresourcemanager/ponresourcemanager_test.go
@@ -0,0 +1,206 @@
+/*
+ * Copyright 2020-present Open Networking Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package ponresourcemanager
+
+import (
+	"context"
+	"encoding/json"
+	"errors"
+	"github.com/opencord/voltha-lib-go/v3/pkg/db"
+	"github.com/opencord/voltha-lib-go/v3/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+	"github.com/stretchr/testify/assert"
+	"strings"
+	"testing"
+)
+
+const (
+	GEM_POOL_PATH        = "gemport_id_pool"
+	RESERVED_GEM_PORT_ID = uint32(5)
+)
+
+func init() {
+	_, err := log.SetDefaultLogger(log.JSON, log.DebugLevel, nil)
+	if err != nil {
+		panic(err)
+	}
+}
+
+// MockKVClient mocks the AdapterProxy interface.
+type MockResKVClient struct {
+	resourceMap map[string]interface{}
+}
+
+func newMockKvClient() *MockResKVClient {
+	var mockResKVClient MockResKVClient
+	mockResKVClient.resourceMap = make(map[string]interface{})
+	return &mockResKVClient
+}
+
+// List function implemented for KVClient.
+func (kvclient *MockResKVClient) List(ctx context.Context, key string) (map[string]*kvstore.KVPair, error) {
+	return nil, errors.New("key didn't find")
+}
+
+// Get mock function implementation for KVClient
+func (kvclient *MockResKVClient) Get(ctx context.Context, key string) (*kvstore.KVPair, error) {
+	log.Debugw("Get of MockKVClient called", log.Fields{"key": key})
+	if key != "" {
+		if strings.Contains(key, RESERVED_GEMPORT_IDS_PATH) {
+			log.Debug("Getting Key:", RESERVED_GEMPORT_IDS_PATH)
+			reservedGemPorts := []uint32{RESERVED_GEM_PORT_ID}
+			str, _ := json.Marshal(reservedGemPorts)
+			return kvstore.NewKVPair(key, str, "mock", 3000, 1), nil
+		}
+		if strings.Contains(key, GEM_POOL_PATH) {
+			log.Debug("Getting Key:", GEM_POOL_PATH)
+			resource := kvclient.resourceMap[key]
+			return kvstore.NewKVPair(key, resource, "mock", 3000, 1), nil
+		}
+		maps := make(map[string]*kvstore.KVPair)
+		maps[key] = &kvstore.KVPair{Key: key}
+		return maps[key], nil
+	}
+	return nil, errors.New("key didn't find")
+}
+
+// Put mock function implementation for KVClient
+func (kvclient *MockResKVClient) Put(ctx context.Context, key string, value interface{}) error {
+	if key != "" {
+		if strings.Contains(key, GEMPORT_ID_POOL_PATH) && value != nil {
+			kvclient.resourceMap[key] = value
+		}
+		return nil
+	}
+	return errors.New("key didn't find")
+}
+
+// Delete mock function implementation for KVClient
+func (kvclient *MockResKVClient) Delete(ctx context.Context, key string) error {
+	return nil
+}
+
+// Reserve mock function implementation for KVClient
+func (kvclient *MockResKVClient) Reserve(ctx context.Context, key string, value interface{}, ttl int64) (interface{}, error) {
+	return nil, errors.New("key didn't find")
+}
+
+// ReleaseReservation mock function implementation for KVClient
+func (kvclient *MockResKVClient) ReleaseReservation(ctx context.Context, key string) error {
+	return nil
+}
+
+// ReleaseAllReservations mock function implementation for KVClient
+func (kvclient *MockResKVClient) ReleaseAllReservations(ctx context.Context) error {
+	return nil
+}
+
+// RenewReservation mock function implementation for KVClient
+func (kvclient *MockResKVClient) RenewReservation(ctx context.Context, key string) error {
+	return nil
+}
+
+// Watch mock function implementation for KVClient
+func (kvclient *MockResKVClient) Watch(ctx context.Context, key string, withPrefix bool) chan *kvstore.Event {
+	return nil
+}
+
+// AcquireLock mock function implementation for KVClient
+func (kvclient *MockResKVClient) AcquireLock(ctx context.Context, lockName string, timeout int) error {
+	return nil
+}
+
+// ReleaseLock mock function implementation for KVClient
+func (kvclient *MockResKVClient) ReleaseLock(lockName string) error {
+	return nil
+}
+
+// IsConnectionUp mock function implementation for KVClient
+func (kvclient *MockResKVClient) IsConnectionUp(ctx context.Context) bool { // timeout in second
+	return true
+}
+
+// CloseWatch mock function implementation for KVClient
+func (kvclient *MockResKVClient) CloseWatch(key string, ch chan *kvstore.Event) {
+}
+
+// Close mock function implementation for KVClient
+func (kvclient *MockResKVClient) Close() {
+}
+
+func TestExcludeReservedGemPortIdFromThePool(t *testing.T) {
+	PONRMgr, err := NewPONResourceManager("gpon", "onu", "olt1",
+		"etcd", "1", 1)
+	if err != nil {
+		return
+	}
+	PONRMgr.KVStore = &db.Backend{
+		Client: newMockKvClient(),
+	}
+
+	PONRMgr.KVStoreForConfig = &db.Backend{
+		Client: newMockKvClient(),
+	}
+	// create a pool in the range of [1,16]
+	// and exclude id 5 from this pool
+	StartIndex := uint32(1)
+	EndIndex := uint32(16)
+
+	ctx := context.Background()
+	reservedGemPortIds, defined := PONRMgr.getReservedGemPortIdsFromKVStore(ctx)
+	if !defined {
+		return
+	}
+
+	FormatResult, err := PONRMgr.FormatResource(1, StartIndex, EndIndex, reservedGemPortIds)
+	if err != nil {
+		t.Error("Failed to format resource", err)
+		return
+	}
+
+	// Add resource as json in kv store.
+	err = PONRMgr.KVStore.Put(ctx, GEMPORT_ID_POOL_PATH, FormatResult)
+	if err != nil {
+		t.Error("Error in posting data to kv store", GEMPORT_ID_POOL_PATH)
+		return
+	}
+
+	for i := StartIndex; i <= (EndIndex - uint32(len(reservedGemPortIds))); i++ {
+		// get gem port id pool from the kv store
+		resource, err := PONRMgr.GetResource(context.Background(), GEMPORT_ID_POOL_PATH)
+		if err != nil {
+			t.Error("Failed to get resource from gem port id pool", err)
+			return
+		}
+		// get a gem port id from the pool
+		nextID, err := PONRMgr.GenerateNextID(resource)
+		if err != nil {
+			t.Error("Failed to get gem port id from the pool", err)
+			return
+		}
+
+		//given gem port id should not equal to the reserved gem port id
+		assert.NotEqual(t, nextID, RESERVED_GEM_PORT_ID)
+		// put updated gem port id pool into the kv store
+		err = PONRMgr.UpdateResource(context.Background(), GEMPORT_ID_POOL_PATH, resource)
+		if err != nil {
+			t.Error("Failed to put updated gem port id pool into the kv store", err)
+			return
+		}
+	}
+
+}