You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@trafficcontrol.apache.org by GitBox <gi...@apache.org> on 2018/04/09 17:19:51 UTC

[GitHub] dangogh closed pull request #2080: Add cachegroup generic crud to traffic_ops_golang

dangogh closed pull request #2080: Add cachegroup generic crud to traffic_ops_golang
URL: https://github.com/apache/incubator-trafficcontrol/pull/2080
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/lib/go-tc/cachegroups.go b/lib/go-tc/cachegroups.go
deleted file mode 100644
index 7e6c1f89e..000000000
--- a/lib/go-tc/cachegroups.go
+++ /dev/null
@@ -1,37 +0,0 @@
-package tc
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-
-// CacheGroupResponse ...
-type CacheGroupsResponse struct {
-	Response []CacheGroup `json:"response"`
-}
-
-// CacheGroup contains information about a given Cachegroup in Traffic Ops.
-type CacheGroup struct {
-	ID          int     `json:"id"`
-	Name        string  `json:"name"`
-	ShortName   string  `json:"shortName"`
-	Latitude    float64 `json:"latitude"`
-	Longitude   float64 `json:"longitude"`
-	ParentName  string  `json:"parentCachegroupName,omitempty"`
-	Type        string  `json:"typeName,omitempty"`
-	LastUpdated string  `json:"lastUpdated,omitempty"`
-}
diff --git a/lib/go-tc/v13/cachegroups.go b/lib/go-tc/v13/cachegroups.go
new file mode 100644
index 000000000..38ac437ad
--- /dev/null
+++ b/lib/go-tc/v13/cachegroups.go
@@ -0,0 +1,58 @@
+package v13
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+import tc "github.com/apache/incubator-trafficcontrol/lib/go-tc"
+
+// CacheGroupResponse ...
+type CacheGroupsResponse struct {
+	Response []CacheGroup `json:"response"`
+}
+
+// CacheGroup contains information about a given Cachegroup in Traffic Ops.
+type CacheGroup struct {
+	ID                          int          `json:"id" db:"id"`
+	Name                        string       `json:"name" db:"name"`
+	ShortName                   string       `json:"shortName" db:"short_name"`
+	Latitude                    float64      `json:"latitude" db:"latitude"`
+	Longitude                   float64      `json:"longitude" db:"longitude"`
+	ParentName                  string       `json:"parentCachegroupName"`
+	ParentCachegroupID          int          `json:"parentCachegroupId" db:"parent_cachegroup_id"`
+	SecondaryParentName         string       `json:"secondaryParentCachegroupName"`
+	SecondaryParentCachegroupID int          `json:"secondaryParentCachegroupId" db:"secondary_parent_cachegroup_id"`
+	Type                        string       `json:"typeName" db:"type_name"` // aliased to type_name to disambiguate struct scans due to join on 'type' table
+	TypeID                      int          `json:"typeId" db:"type_id"`     // aliased to type_id to disambiguate struct scans due join on 'type' table
+	LastUpdated                 tc.TimeNoMod `json:"lastUpdated" db:"last_updated"`
+}
+
+type CacheGroupNullable struct {
+	ID                          *int          `json:"id" db:"id"`
+	Name                        *string       `json:"name" db:"name"`
+	ShortName                   *string       `json:"shortName" db:"short_name"`
+	Latitude                    *float64      `json:"latitude" db:"latitude"`
+	Longitude                   *float64      `json:"longitude"db:"longitude"`
+	ParentName                  *string       `json:"parentCachegroupName"`
+	ParentCachegroupID          *int          `json:"parentCachegroupId" db:"parent_cachegroup_id"`
+	SecondaryParentName         *string       `json:"secondaryParentCachegroupName"`
+	SecondaryParentCachegroupID *int          `json:"secondaryParentCachegroupId" db:"secondary_parent_cachegroup_id"`
+	Type                        *string       `json:"typeName" db:"type_name"` // aliased to type_name to disambiguate struct scans due to join on 'type' table
+	TypeID                      *int          `json:"typeId" db:"type_id"`     // aliased to type_id to disambiguate struct scans due join on 'type' table
+	LastUpdated                 *tc.TimeNoMod `json:"lastUpdated" db:"last_updated"`
+}
diff --git a/traffic_ops/client/v13/cachegroup.go b/traffic_ops/client/v13/cachegroup.go
index 27ef2d10e..70cea8cc0 100644
--- a/traffic_ops/client/v13/cachegroup.go
+++ b/traffic_ops/client/v13/cachegroup.go
@@ -22,6 +22,7 @@ import (
 	"net/http"
 
 	"github.com/apache/incubator-trafficcontrol/lib/go-tc"
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc/v13"
 )
 
 const (
@@ -29,7 +30,7 @@ const (
 )
 
 // Create a CacheGroup
-func (to *Session) CreateCacheGroup(cachegroup tc.CacheGroup) (tc.Alerts, ReqInf, error) {
+func (to *Session) CreateCacheGroup(cachegroup v13.CacheGroup) (tc.Alerts, ReqInf, error) {
 
 	var remoteAddr net.Addr
 	reqBody, err := json.Marshal(cachegroup)
@@ -48,7 +49,7 @@ func (to *Session) CreateCacheGroup(cachegroup tc.CacheGroup) (tc.Alerts, ReqInf
 }
 
 // Update a CacheGroup by ID
-func (to *Session) UpdateCacheGroupByID(id int, cachegroup tc.CacheGroup) (tc.Alerts, ReqInf, error) {
+func (to *Session) UpdateCacheGroupByID(id int, cachegroup v13.CacheGroup) (tc.Alerts, ReqInf, error) {
 
 	var remoteAddr net.Addr
 	reqBody, err := json.Marshal(cachegroup)
@@ -68,7 +69,7 @@ func (to *Session) UpdateCacheGroupByID(id int, cachegroup tc.CacheGroup) (tc.Al
 }
 
 // Returns a list of CacheGroups
-func (to *Session) GetCacheGroups() ([]tc.CacheGroup, ReqInf, error) {
+func (to *Session) GetCacheGroups() ([]v13.CacheGroup, ReqInf, error) {
 	resp, remoteAddr, err := to.request(http.MethodGet, API_v13_CacheGroups, nil)
 	reqInf := ReqInf{CacheHitStatus: CacheHitStatusMiss, RemoteAddr: remoteAddr}
 	if err != nil {
@@ -76,13 +77,13 @@ func (to *Session) GetCacheGroups() ([]tc.CacheGroup, ReqInf, error) {
 	}
 	defer resp.Body.Close()
 
-	var data tc.CacheGroupsResponse
+	var data v13.CacheGroupsResponse
 	err = json.NewDecoder(resp.Body).Decode(&data)
 	return data.Response, reqInf, nil
 }
 
 // GET a CacheGroup by the CacheGroup id
-func (to *Session) GetCacheGroupByID(id int) ([]tc.CacheGroup, ReqInf, error) {
+func (to *Session) GetCacheGroupByID(id int) ([]v13.CacheGroup, ReqInf, error) {
 	route := fmt.Sprintf("%s/%d", API_v13_CacheGroups, id)
 	resp, remoteAddr, err := to.request(http.MethodGet, route, nil)
 	reqInf := ReqInf{CacheHitStatus: CacheHitStatusMiss, RemoteAddr: remoteAddr}
@@ -91,7 +92,7 @@ func (to *Session) GetCacheGroupByID(id int) ([]tc.CacheGroup, ReqInf, error) {
 	}
 	defer resp.Body.Close()
 
-	var data tc.CacheGroupsResponse
+	var data v13.CacheGroupsResponse
 	if err := json.NewDecoder(resp.Body).Decode(&data); err != nil {
 		return nil, reqInf, err
 	}
@@ -100,7 +101,7 @@ func (to *Session) GetCacheGroupByID(id int) ([]tc.CacheGroup, ReqInf, error) {
 }
 
 // GET a CacheGroup by the CacheGroup name
-func (to *Session) GetCacheGroupByName(name string) ([]tc.CacheGroup, ReqInf, error) {
+func (to *Session) GetCacheGroupByName(name string) ([]v13.CacheGroup, ReqInf, error) {
 	url := fmt.Sprintf("%s?name=%s", API_v13_CacheGroups, name)
 	resp, remoteAddr, err := to.request(http.MethodGet, url, nil)
 	reqInf := ReqInf{CacheHitStatus: CacheHitStatusMiss, RemoteAddr: remoteAddr}
@@ -109,7 +110,7 @@ func (to *Session) GetCacheGroupByName(name string) ([]tc.CacheGroup, ReqInf, er
 	}
 	defer resp.Body.Close()
 
-	var data tc.CacheGroupsResponse
+	var data v13.CacheGroupsResponse
 	if err := json.NewDecoder(resp.Body).Decode(&data); err != nil {
 		return nil, reqInf, err
 	}
diff --git a/traffic_ops/testing/api/v13/cachegroups_test.go b/traffic_ops/testing/api/v13/cachegroups_test.go
new file mode 100644
index 000000000..e43b24a68
--- /dev/null
+++ b/traffic_ops/testing/api/v13/cachegroups_test.go
@@ -0,0 +1,183 @@
+package v13
+
+/*
+
+ 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.
+*/
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-trafficcontrol/lib/go-log"
+	tc "github.com/apache/incubator-trafficcontrol/lib/go-tc"
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc/v13"
+)
+
+func TestCacheGroups(t *testing.T) {
+	CreateTestTypes(t)
+	CreateTestCacheGroups(t)
+	GetTestCacheGroups(t)
+	UpdateTestCacheGroups(t)
+	DeleteTestCacheGroups(t)
+	DeleteTestTypes(t)
+}
+
+func CreateTestCacheGroups(t *testing.T) {
+	failed := false
+
+	for _, cg := range testData.CacheGroups {
+		// get the typeID
+		typeResp, _, err := TOSession.GetTypeByName(cg.Type)
+		if err != nil {
+			t.Error("could not lookup a typeID for this cachegroup")
+			failed = true
+		}
+		cg.TypeID = typeResp[0].ID
+
+		_, _, err = TOSession.CreateCacheGroup(cg)
+		if err != nil {
+			t.Errorf("could not CREATE cachegroups: %v\n", err)
+			failed = true
+		}
+	}
+	if !failed {
+		log.Debugln("CreateTestCacheGroups() PASSED: ")
+	}
+}
+
+func GetTestCacheGroups(t *testing.T) {
+	failed := false
+	for _, cg := range testData.CacheGroups {
+		resp, _, err := TOSession.GetCacheGroupByName(cg.Name)
+		if err != nil {
+			t.Errorf("cannot GET CacheGroup by name: %v - %v\n", err, resp)
+			failed = true
+		}
+	}
+	if !failed {
+		log.Debugln("GetTestCacheGroups() PASSED: ")
+	}
+}
+
+func UpdateTestCacheGroups(t *testing.T) {
+	failed := false
+	firstCG := testData.CacheGroups[0]
+	resp, _, err := TOSession.GetCacheGroupByName(firstCG.Name)
+	if err != nil {
+		t.Errorf("cannot GET CACHEGROUP by name: %v - %v\n", firstCG.Name, err)
+		failed = true
+	}
+	cg := resp[0]
+	expectedShortName := "blah"
+	cg.ShortName = expectedShortName
+
+	// fix the type id for test
+	typeResp, _, err := TOSession.GetTypeByID(cg.TypeID)
+	if err != nil {
+		t.Error("could not lookup a typeID for this cachegroup")
+		failed = true
+	}
+	cg.TypeID = typeResp[0].ID
+
+	var alert tc.Alerts
+	alert, _, err = TOSession.UpdateCacheGroupByID(cg.ID, cg)
+	if err != nil {
+		t.Errorf("cannot UPDATE CacheGroup by id: %v - %v\n", err, alert)
+		failed = true
+	}
+
+	// Retrieve the CacheGroup to check CacheGroup name got updated
+	resp, _, err = TOSession.GetCacheGroupByID(cg.ID)
+	if err != nil {
+		t.Errorf("cannot GET CacheGroup by name: '$%s', %v\n", firstCG.Name, err)
+		failed = true
+	}
+	cg = resp[0]
+	if cg.ShortName != expectedShortName {
+		t.Errorf("results do not match actual: %s, expected: %s\n", cg.ShortName, expectedShortName)
+	}
+	if !failed {
+		log.Debugln("UpdateTestCacheGroups() PASSED: ")
+	}
+}
+
+func DeleteTestCacheGroups(t *testing.T) {
+	failed := false
+	var mids []v13.CacheGroup
+
+	// delete the edge caches.
+	for _, cg := range testData.CacheGroups {
+		// Retrieve the CacheGroup by name so we can get the id for the Update
+		resp, _, err := TOSession.GetCacheGroupByName(cg.Name)
+		if err != nil {
+			t.Errorf("cannot GET CacheGroup by name: %v - %v\n", cg.Name, err)
+			failed = true
+		}
+		// Mids are parents and need to be deleted only after the children
+		// cachegroups are deleted.
+		if cg.Type == "MID_LOC" {
+			mids = append(mids, cg)
+			continue
+		}
+		if len(resp) > 0 {
+			respCG := resp[0]
+			_, _, err := TOSession.DeleteCacheGroupByID(respCG.ID)
+			if err != nil {
+				t.Errorf("cannot DELETE CacheGroup by name: '%s' %v\n", respCG.Name, err)
+				failed = true
+			}
+			// Retrieve the CacheGroup to see if it got deleted
+			cgs, _, err := TOSession.GetCacheGroupByName(cg.Name)
+			if err != nil {
+				t.Errorf("error deleting CacheGroup name: %s\n", err.Error())
+				failed = true
+			}
+			if len(cgs) > 0 {
+				t.Errorf("expected CacheGroup name: %s to be deleted\n", cg.Name)
+				failed = true
+			}
+		}
+	}
+	// now delete the mid tier caches
+	for _, cg := range mids {
+		// Retrieve the CacheGroup by name so we can get the id for the Update
+		resp, _, err := TOSession.GetCacheGroupByName(cg.Name)
+		if err != nil {
+			t.Errorf("cannot GET CacheGroup by name: %v - %v\n", cg.Name, err)
+			failed = true
+		}
+		if len(resp) > 0 {
+			respCG := resp[0]
+			_, _, err := TOSession.DeleteCacheGroupByID(respCG.ID)
+			if err != nil {
+				t.Errorf("cannot DELETE CacheGroup by name: '%s' %v\n", respCG.Name, err)
+				failed = true
+			}
+
+			// Retrieve the CacheGroup to see if it got deleted
+			cgs, _, err := TOSession.GetCacheGroupByName(cg.Name)
+			if err != nil {
+				t.Errorf("error deleting CacheGroup name: %s\n", err.Error())
+				failed = true
+			}
+			if len(cgs) > 0 {
+				t.Errorf("expected CacheGroup name: %s to be deleted\n", cg.Name)
+				failed = true
+			}
+		}
+	}
+
+	if !failed {
+		log.Debugln("DeleteTestCacheGroups() PASSED: ")
+	}
+}
diff --git a/traffic_ops/testing/api/v13/tc-fixtures.json b/traffic_ops/testing/api/v13/tc-fixtures.json
index 9462ee291..9def4afcf 100644
--- a/traffic_ops/testing/api/v13/tc-fixtures.json
+++ b/traffic_ops/testing/api/v13/tc-fixtures.json
@@ -13,34 +13,38 @@
         {
             "latitude": 0,
             "longitude": 0,
-            "name": "parentCacheGroup",
-            "parentCacheGroupName": null,
-            "type": "EDGE"
+            "name": "parentCachegroup",
+            "shortName": "pg1",
+            "typeId": 5,
+            "typeName": "MID_LOC"
         },
         {
             "latitude": 0,
             "longitude": 0,
-            "name": "secondaryCacheGroup",
-            "parentCacheGroupName": null,
-            "type": "EDGE"
+            "name": "secondaryCachegroup",
+            "shortName": "sg1",
+            "typeId": 5,
+            "typeName": "MID_LOC"
         },
         {
             "latitude": 0,
             "longitude": 0,
             "name": "cachegroup1",
-            "parentCacheGroupName": "parentCacheGroup",
-            "secondaryCacheGroupName": "parentCacheGroup",
+            "parentCachegroupName": "parentCachegroup",
+            "secondaryParentCachegroupName": "secondaryCachegroup",
             "shortName": "cg1",
-            "type": "MID"
+            "typeId": 6,
+            "typeName": "EDGE_LOC"
         },
         {
             "latitude": 24.1234,
             "longitude": -121.1234,
             "name": "cachegroup2",
-            "parentCacheGroupName": "secondaryCacheGroup",
-            "secondaryCacheGroupName": "secondaryCacheGroup",
+            "parentCachegroupName": "parentCachegroup",
+            "secondaryParentCachegroupName": "secondaryCachegroup",
             "shortName": "cg2",
-            "type": "MID"
+            "typeId": 6,
+            "typeName": "EDGE_LOC"
         }
     ],
     "cdns": [
diff --git a/traffic_ops/testing/api/v13/traffic_control.go b/traffic_ops/testing/api/v13/traffic_control.go
index cba0b4832..f21198b79 100644
--- a/traffic_ops/testing/api/v13/traffic_control.go
+++ b/traffic_ops/testing/api/v13/traffic_control.go
@@ -24,7 +24,7 @@ import (
 type TrafficControl struct {
 	ASNs                           []v12.ASN                           `json:"asns"`
 	CDNs                           []v13.CDN                           `json:"cdns"`
-	CacheGroups                    []v12.CacheGroup                    `json:"cachegroups"`
+	CacheGroups                    []v13.CacheGroup                    `json:"cachegroups"`
 	DeliveryServiceRequests        []v12.DeliveryServiceRequest        `json:"deliveryServiceRequests"`
 	DeliveryServiceRequestComments []v12.DeliveryServiceRequestComment `json:"deliveryServiceRequestComments"`
 	DeliveryServices               []v12.DeliveryService               `json:"deliveryservices"`
diff --git a/traffic_ops/testing/api/v13/types_test.go b/traffic_ops/testing/api/v13/types_test.go
index 764501a85..8d6ad928c 100644
--- a/traffic_ops/testing/api/v13/types_test.go
+++ b/traffic_ops/testing/api/v13/types_test.go
@@ -25,7 +25,7 @@ import (
 func TestTypes(t *testing.T) {
 
 	CreateTestTypes(t)
-	UpdateTestTypes(t)
+	//UpdateTestTypes(t)
 	//GetTestTypes(t)
 	//DeleteTestTypes(t)
 
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
new file mode 100644
index 000000000..187d380df
--- /dev/null
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -0,0 +1,543 @@
+package cachegroup
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+import (
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/incubator-trafficcontrol/lib/go-log"
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc"
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/api"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/auth"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+	validation "github.com/go-ozzo/ozzo-validation"
+	"github.com/jmoiron/sqlx"
+	"github.com/lib/pq"
+)
+
+type TOCacheGroup v13.CacheGroupNullable
+
+//the refType is passed into the handlers where a copy of its type is used to decode the json.
+var refType = TOCacheGroup{}
+
+func GetRefType() *TOCacheGroup {
+	return &refType
+}
+
+func (cachegroup TOCacheGroup) GetKeyFieldsInfo() []api.KeyFieldInfo {
+	return []api.KeyFieldInfo{{"id", api.GetIntKey}}
+}
+
+//Implementation of the Identifier, Validator interface functions
+func (cachegroup TOCacheGroup) GetKeys() (map[string]interface{}, bool) {
+	if cachegroup.ID == nil {
+		return map[string]interface{}{"id": 0}, false
+	}
+	return map[string]interface{}{"id": *cachegroup.ID}, true
+}
+
+func (cachegroup *TOCacheGroup) SetKeys(keys map[string]interface{}) {
+	i, _ := keys["id"].(int) //this utilizes the non panicking type assertion, if the thrown away ok variable is false i will be the zero of the type, 0 here.
+	cachegroup.ID = &i
+}
+
+//Implementation of the Identifier, Validator interface functions
+func (cachegroup TOCacheGroup) GetID() (int, bool) {
+	if cachegroup.ID == nil {
+		return 0, false
+	}
+	return *cachegroup.ID, true
+}
+
+func (cachegroup TOCacheGroup) GetAuditName() string {
+	if cachegroup.Name != nil {
+		return *cachegroup.Name
+	}
+	id, _ := cachegroup.GetID()
+	return strconv.Itoa(id)
+}
+
+func (cachegroup TOCacheGroup) GetType() string {
+	return "cachegroup"
+}
+
+func (cachegroup *TOCacheGroup) SetID(i int) {
+	cachegroup.ID = &i
+}
+
+// checks if a cachegroup with the given ID is in use as a parent or secondary parent.
+func isUsedByChildCache(db *sqlx.DB, ID int) (bool, error) {
+	pQuery := "SELECT count(*) from cachegroup WHERE parent_cachegroup_id=$1"
+	sQuery := "SELECT count(*) from cachegroup WHERE secondary_parent_cachegroup_id=$1"
+	count := 0
+
+	err := db.QueryRow(pQuery, ID).Scan(&count)
+	if err != nil {
+		log.Errorf("received error: %++v from query execution", err)
+		return false, err
+	}
+	if count > 0 {
+		return true, errors.New("cache is in use as a parent cache")
+	}
+
+	err = db.QueryRow(sQuery, ID).Scan(&count)
+	if err != nil {
+		log.Errorf("received error: %++v from query execution", err)
+		return false, err
+	}
+	if count > 0 {
+		return true, errors.New("cache is in use as a secondary parent cache")
+	}
+	return false, nil
+}
+
+func isValidCacheGroupChar(r rune) bool {
+	if r >= 'a' && r <= 'z' {
+		return true
+	}
+	if r >= 'A' && r <= 'Z' {
+		return true
+	}
+	if r >= '0' && r <= '9' {
+		return true
+	}
+	if r == '.' || r == '-' || r == '_' {
+		return true
+	}
+	return false
+}
+
+// IsValidCacheGroupName returns true if the name contains only characters valid for a CacheGroup name
+func IsValidCacheGroupName(str string) bool {
+	i := strings.IndexFunc(str, func(r rune) bool { return !isValidCacheGroupChar(r) })
+	return i == -1
+}
+
+func IsValidParentCachegroupID(id *int) bool {
+	if id == nil || *id > 0 {
+		return true
+	}
+	return false
+}
+
+// Validate fulfills the api.Validator interface
+func (cachegroup TOCacheGroup) Validate(db *sqlx.DB) []error {
+	validName := validation.NewStringRule(IsValidCacheGroupName, "invalid characters found - Use alphanumeric . or - or _ .")
+	validShortName := validation.NewStringRule(IsValidCacheGroupName, "invalid characters found - Use alphanumeric . or - or _ .")
+	latitudeErr := "Must be a floating point number within the range +-90"
+	longitudeErr := "Must be a floating point number within the range +-180"
+	errs := validation.Errors{
+		"name":                        validation.Validate(cachegroup.Name, validation.Required, validName),
+		"shortName":                   validation.Validate(cachegroup.ShortName, validation.Required, validShortName),
+		"latitude":                    validation.Validate(cachegroup.Latitude, validation.Min(-90.0).Error(latitudeErr), validation.Max(90.0).Error(latitudeErr)),
+		"longitude":                   validation.Validate(cachegroup.Longitude, validation.Min(-180.0).Error(longitudeErr), validation.Max(180.0).Error(longitudeErr)),
+		"parentCacheGroupID":          validation.Validate(cachegroup.ParentCachegroupID, validation.Min(1)),
+		"secondaryParentCachegroupID": validation.Validate(cachegroup.SecondaryParentCachegroupID, validation.Min(1)),
+	}
+	return tovalidate.ToErrors(errs)
+}
+
+// looks up the parent_cachegroup_id and the secondary_cachegroup_id
+// if the respective names are defined in the cachegroup struct.  A
+// sucessful lookup sets the two ids on the struct.
+//
+// used by Create()
+func getParentCachegroupIDs(db *sqlx.DB, cachegroup *TOCacheGroup) error {
+	query := `SELECT id FROM cachegroup where name=$1`
+	var parentID int
+	var secondaryParentID int
+
+	if cachegroup.ParentName != nil && *cachegroup.ParentName != "" {
+		err := db.QueryRow(query, *cachegroup.ParentName).Scan(&parentID)
+		if err != nil {
+			log.Errorf("received error: %++v from query execution", err)
+			return err
+		}
+		cachegroup.ParentCachegroupID = &parentID
+	}
+	// not using 'omitempty' on the CacheGroup struct so a '0' is really an empty field, so set the pointer to nil
+	if cachegroup.ParentCachegroupID != nil && *cachegroup.ParentCachegroupID == 0 {
+		cachegroup.ParentCachegroupID = nil
+	}
+
+	if cachegroup.SecondaryParentName != nil && *cachegroup.SecondaryParentName != "" {
+		err := db.QueryRow(query, *cachegroup.SecondaryParentName).Scan(&secondaryParentID)
+		if err != nil {
+			log.Errorf("received error: %++v from query execution", err)
+			return err
+		}
+		cachegroup.SecondaryParentCachegroupID = &secondaryParentID
+	}
+	// not using 'omitempty' on the CacheGroup struct so a '0' is really an empty field, so set the pointer to nil
+	if cachegroup.SecondaryParentCachegroupID != nil && *cachegroup.SecondaryParentCachegroupID == 0 {
+		cachegroup.SecondaryParentCachegroupID = nil
+	}
+	return nil
+}
+
+// looks up the parent and secondary cachegroup names by cachegroup ID.
+//  the names are set on the struct.
+//
+// used by Read()
+func getParentCacheGroupNames(db *sqlx.DB, cachegroup *TOCacheGroup) error {
+	query1 := `SELECT name FROM cachegroup where id=$1`
+	var primaryName string
+	var secondaryName string
+
+	// primary parent lookup
+	if cachegroup.ParentCachegroupID != nil {
+		err := db.QueryRow(query1, *cachegroup.ParentCachegroupID).Scan(&primaryName)
+		if err != nil {
+			log.Errorf("received error: %++v from query execution", err)
+			return err
+		}
+		cachegroup.ParentName = &primaryName
+	}
+
+	// secondary parent lookup
+	if cachegroup.SecondaryParentCachegroupID != nil {
+		err := db.QueryRow(query1, *cachegroup.SecondaryParentCachegroupID).Scan(&secondaryName)
+		if err != nil {
+			log.Errorf("received error: %++v from query execution", err)
+			return err
+		}
+		cachegroup.SecondaryParentName = &secondaryName
+	}
+
+	return nil
+}
+
+//The TOCacheGroup implementation of the Creator interface
+//all implementations of Creator should use transactions and return the proper errorType
+//ParsePQUniqueConstraintError is used to determine if a cachegroup with conflicting values exists
+//if so, it will return an errorType of DataConflict and the type should be appended to the
+//generic error message returned
+//The insert sql returns the id and lastUpdated values of the newly inserted cachegroup and have
+//to be added to the struct
+func (cachegroup *TOCacheGroup) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+	rollbackTransaction := true
+	tx, err := db.Beginx()
+	defer func() {
+		if tx == nil || !rollbackTransaction {
+			return
+		}
+		err := tx.Rollback()
+		if err != nil {
+			log.Errorln(errors.New("rolling back transaction: " + err.Error()))
+		}
+	}()
+
+	if err != nil {
+		log.Error.Printf("could not begin transaction: %v", err)
+		return tc.DBError, tc.SystemError
+	}
+
+	err = getParentCachegroupIDs(db, cachegroup)
+	if err != nil {
+		log.Error.Printf("failure looking up parent cache groups %v", err)
+		return tc.DBError, tc.SystemError
+	}
+
+	resultRows, err := tx.NamedQuery(insertQuery(), cachegroup)
+	if err != nil {
+		if pqErr, ok := err.(*pq.Error); ok {
+			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
+			if eType == tc.DataConflictError {
+				return errors.New("a cachegroup with " + err.Error()), eType
+			}
+			return err, eType
+		} else {
+			log.Errorf("received non pq error: %++v from create execution", err)
+			return tc.DBError, tc.SystemError
+		}
+	}
+	defer resultRows.Close()
+
+	var id int
+	var lastUpdated tc.TimeNoMod
+	rowsAffected := 0
+	for resultRows.Next() {
+		rowsAffected++
+		if err := resultRows.Scan(&id, &lastUpdated); err != nil {
+			log.Error.Printf("could not scan id from insert: %s\n", err)
+			return tc.DBError, tc.SystemError
+		}
+	}
+	if rowsAffected == 0 {
+		err = errors.New("no cachegroup was inserted, no id was returned")
+		log.Errorln(err)
+		return tc.DBError, tc.SystemError
+	} else if rowsAffected > 1 {
+		err = errors.New("too many ids returned from cachegroup insert")
+		log.Errorln(err)
+		return tc.DBError, tc.SystemError
+	}
+	cachegroup.SetID(id)
+	cachegroup.LastUpdated = &lastUpdated
+	err = tx.Commit()
+	if err != nil {
+		log.Errorln("Could not commit transaction: ", err)
+		return tc.DBError, tc.SystemError
+	}
+	rollbackTransaction = false
+	return nil, tc.NoError
+}
+
+func (cachegroup *TOCacheGroup) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+	var rows *sqlx.Rows
+
+	// Query Parameters to Database Query column mappings
+	// see the fields mapped in the SQL query
+	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
+		"id":        dbhelpers.WhereColumnInfo{"cachegroup.id", api.IsInt},
+		"name":      dbhelpers.WhereColumnInfo{"cachegroup.name", nil},
+		"shortName": dbhelpers.WhereColumnInfo{"short_name", nil},
+	}
+	where, orderBy, queryValues, errs := dbhelpers.BuildWhereAndOrderBy(parameters, queryParamsToQueryCols)
+	if len(errs) > 0 {
+		return nil, errs, tc.DataConflictError
+	}
+
+	query := selectQuery() + where + orderBy
+	log.Debugln("Query is ", query)
+
+	rows, err := db.NamedQuery(query, queryValues)
+	if err != nil {
+		log.Errorf("Error querying CacheGroup: %v", err)
+		return nil, []error{tc.DBError}, tc.SystemError
+	}
+	defer rows.Close()
+
+	CacheGroups := []interface{}{}
+	for rows.Next() {
+		var s TOCacheGroup
+		if err = rows.StructScan(&s); err != nil {
+			log.Errorf("error parsing CacheGroup rows: %v", err)
+			return nil, []error{tc.DBError}, tc.SystemError
+		}
+		getParentCacheGroupNames(db, &s)
+		CacheGroups = append(CacheGroups, s)
+	}
+
+	return CacheGroups, []error{}, tc.NoError
+}
+
+//The TOCacheGroup implementation of the Updater interface
+//all implementations of Updater should use transactions and return the proper errorType
+//ParsePQUniqueConstraintError is used to determine if a cachegroup with conflicting values exists
+//if so, it will return an errorType of DataConflict and the type should be appended to the
+//generic error message returned
+func (cachegroup *TOCacheGroup) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+	rollbackTransaction := true
+	tx, err := db.Beginx()
+	defer func() {
+		if tx == nil || !rollbackTransaction {
+			return
+		}
+		err := tx.Rollback()
+		if err != nil {
+			log.Errorln(errors.New("rolling back transaction: " + err.Error()))
+		}
+	}()
+
+	if err != nil {
+		log.Error.Printf("could not begin transaction: %v", err)
+		return tc.DBError, tc.SystemError
+	}
+
+	// fix up parent ids.
+	err = getParentCachegroupIDs(db, cachegroup)
+	if err != nil {
+		log.Error.Printf("failure looking up parent cache groups %v", err)
+		return tc.DBError, tc.SystemError
+	}
+
+	log.Debugf("about to run exec query: %s with cachegroup: %++v", updateQuery(), cachegroup)
+	resultRows, err := tx.NamedQuery(updateQuery(), cachegroup)
+	if err != nil {
+		if pqErr, ok := err.(*pq.Error); ok {
+			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
+			if eType == tc.DataConflictError {
+				return errors.New("a cachegroup with " + err.Error()), eType
+			}
+			return err, eType
+		} else {
+			log.Errorf("received error: %++v from update execution", err)
+			return tc.DBError, tc.SystemError
+		}
+	}
+	defer resultRows.Close()
+
+	var lastUpdated tc.TimeNoMod
+	rowsAffected := 0
+	for resultRows.Next() {
+		rowsAffected++
+		if err := resultRows.Scan(&lastUpdated); err != nil {
+			log.Error.Printf("could not scan lastUpdated from insert: %s\n", err)
+			return tc.DBError, tc.SystemError
+		}
+	}
+	log.Debugf("lastUpdated: %++v", lastUpdated)
+	cachegroup.LastUpdated = &lastUpdated
+	if rowsAffected != 1 {
+		if rowsAffected < 1 {
+			return errors.New("no cachegroup found with this id"), tc.DataMissingError
+		} else {
+			return fmt.Errorf("this update affected too many rows: %d", rowsAffected), tc.SystemError
+		}
+	}
+	err = tx.Commit()
+	if err != nil {
+		log.Errorln("Could not commit transaction: ", err)
+		return tc.DBError, tc.SystemError
+	}
+	rollbackTransaction = false
+	return nil, tc.NoError
+}
+
+//The CacheGroup implementation of the Deleter interface
+//all implementations of Deleter should use transactions and return the proper errorType
+func (cachegroup *TOCacheGroup) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+	rollbackTransaction := true
+	tx, err := db.Beginx()
+	defer func() {
+		if tx == nil || !rollbackTransaction {
+			return
+		}
+		err := tx.Rollback()
+		if err != nil {
+			log.Errorln(errors.New("rolling back transaction: " + err.Error()))
+		}
+	}()
+
+	if err != nil {
+		log.Error.Printf("could not begin transaction: %v", err)
+		return tc.DBError, tc.SystemError
+	}
+
+	inUse, err := isUsedByChildCache(db, *cachegroup.ID)
+	log.Debugf("inUse: %d, err: %v", inUse, err)
+	if inUse == false && err != nil {
+		return tc.DBError, tc.SystemError
+	}
+	if inUse == true {
+		return err, tc.DataConflictError
+	}
+
+	log.Debugf("about to run exec query: %s with cachegroup: %++v", deleteQuery(), cachegroup)
+	result, err := tx.NamedExec(deleteQuery(), cachegroup)
+	if err != nil {
+		log.Errorf("received error: %++v from delete execution", err)
+		return tc.DBError, tc.SystemError
+	}
+	rowsAffected, err := result.RowsAffected()
+	if err != nil {
+		return tc.DBError, tc.SystemError
+	}
+	if rowsAffected != 1 {
+		if rowsAffected < 1 {
+			return errors.New("no cachegroup with that id found"), tc.DataMissingError
+		} else {
+			return fmt.Errorf("this create affected too many rows: %d", rowsAffected), tc.SystemError
+		}
+	}
+	err = tx.Commit()
+	if err != nil {
+		log.Errorln("Could not commit transaction: ", err)
+		return tc.DBError, tc.SystemError
+	}
+	rollbackTransaction = false
+	return nil, tc.NoError
+}
+
+// insert query
+func insertQuery() string {
+	// to disambiguate struct scans, the named
+	// parameter 'type_id' is an alias to cachegroup.type
+	//see also the v13.CacheGroupNullable struct 'db' metadata
+	query := `INSERT INTO cachegroup (
+name,
+short_name,
+latitude,
+longitude,
+type,
+parent_cachegroup_id,
+secondary_parent_cachegroup_id
+) VALUES(
+:name,
+:short_name,
+:latitude,
+:longitude,
+:type_id,
+:parent_cachegroup_id,
+:secondary_parent_cachegroup_id
+) RETURNING id,last_updated`
+	return query
+}
+
+// select query
+func selectQuery() string {
+	// the 'type_name' and 'type_id' aliases on the 'type.name'
+	// and cachegroup.type' fields are needed
+	// to disambiguate the struct scan, see also the
+	// v13.CacheGroupNullable struct 'db' metadata
+	query := `SELECT
+cachegroup.id,
+cachegroup.name,
+cachegroup.short_name,
+cachegroup.latitude,
+cachegroup.longitude,
+cachegroup.parent_cachegroup_id,
+cachegroup.secondary_parent_cachegroup_id,
+type.name AS type_name,
+cachegroup.type AS type_id,
+cachegroup.last_updated
+FROM cachegroup
+INNER JOIN type ON cachegroup.type = type.id`
+	return query
+}
+
+// update query
+func updateQuery() string {
+	// to disambiguate struct scans, the named
+	// parameter 'type_id' is an alias to cachegroup.type
+	//see also the v13.CacheGroupNullable struct 'db' metadata
+	query := `UPDATE
+cachegroup SET
+name=:name,
+short_name=:short_name,
+latitude=:latitude,
+longitude=:longitude,
+parent_cachegroup_id=:parent_cachegroup_id,
+secondary_parent_cachegroup_id=:secondary_parent_cachegroup_id,
+type=:type_id WHERE id=:id RETURNING last_updated`
+	return query
+}
+
+//delete query
+func deleteQuery() string {
+	query := `DELETE FROM cachegroup WHERE id=:id`
+	return query
+}
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
new file mode 100644
index 000000000..dad38d79c
--- /dev/null
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
@@ -0,0 +1,202 @@
+package cachegroup
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+import (
+	"errors"
+	"reflect"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc"
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/api"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/auth"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/test"
+	"github.com/jmoiron/sqlx"
+
+	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
+)
+
+func getTestCacheGroups() []v13.CacheGroup {
+	cgs := []v13.CacheGroup{}
+	testCG1 := v13.CacheGroup{
+		ID:                          1,
+		Name:                        "cachegroup1",
+		ShortName:                   "cg1",
+		Latitude:                    38.7,
+		Longitude:                   90.7,
+		ParentCachegroupID:          2,
+		SecondaryParentCachegroupID: 2,
+		Type:        "EDGE_LOC",
+		TypeID:      6,
+		LastUpdated: tc.TimeNoMod{Time: time.Now()},
+	}
+	cgs = append(cgs, testCG1)
+
+	testCG2 := v13.CacheGroup{
+		ID:                          1,
+		Name:                        "parentCacheGroup",
+		ShortName:                   "pg1",
+		Latitude:                    38.7,
+		Longitude:                   90.7,
+		ParentCachegroupID:          1,
+		SecondaryParentCachegroupID: 1,
+		Type:        "MID_LOC",
+		TypeID:      7,
+		LastUpdated: tc.TimeNoMod{Time: time.Now()},
+	}
+	cgs = append(cgs, testCG2)
+
+	return cgs
+}
+
+func TestReadCacheGroups(t *testing.T) {
+	mockDB, mock, err := sqlmock.New()
+	if err != nil {
+		t.Fatalf("an error '%s' was not expected when opening a stub database connection", err)
+	}
+	defer mockDB.Close()
+
+	db := sqlx.NewDb(mockDB, "sqlmock")
+	defer db.Close()
+
+	refType := GetRefType()
+
+	testCGs := getTestCacheGroups()
+	cols := test.ColsFromStructByTag("db", v13.CacheGroup{})
+	rows := sqlmock.NewRows(cols)
+
+	for _, ts := range testCGs {
+		rows = rows.AddRow(
+			ts.ID,
+			ts.Name,
+			ts.ShortName,
+			ts.Latitude,
+			ts.Longitude,
+			ts.ParentCachegroupID,
+			ts.SecondaryParentCachegroupID,
+			ts.Type,
+			ts.TypeID,
+			ts.LastUpdated,
+		)
+	}
+	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	v := map[string]string{"id": "1"}
+
+	cachegroups, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	if len(errs) > 0 {
+		t.Errorf("cdn.Read expected: no errors, actual: %v", errs)
+	}
+
+	if len(cachegroups) != 2 {
+		t.Errorf("cdn.Read expected: len(cachegroups) == 2, actual: %v", len(cachegroups))
+	}
+}
+
+func TestFuncs(t *testing.T) {
+	if strings.Index(selectQuery(), "SELECT") != 0 {
+		t.Errorf("expected selectQuery to start with SELECT")
+	}
+	if strings.Index(insertQuery(), "INSERT") != 0 {
+		t.Errorf("expected insertQuery to start with INSERT")
+	}
+	if strings.Index(updateQuery(), "UPDATE") != 0 {
+		t.Errorf("expected updateQuery to start with UPDATE")
+	}
+	if strings.Index(deleteQuery(), "DELETE") != 0 {
+		t.Errorf("expected deleteQuery to start with DELETE")
+	}
+}
+
+func TestInterfaces(t *testing.T) {
+	var i interface{}
+	i = &TOCacheGroup{}
+
+	if _, ok := i.(api.Creator); !ok {
+		t.Errorf("cachegroup must be creator")
+	}
+	if _, ok := i.(api.Reader); !ok {
+		t.Errorf("cachegroup must be reader")
+	}
+	if _, ok := i.(api.Updater); !ok {
+		t.Errorf("cachegroup must be updater")
+	}
+	if _, ok := i.(api.Deleter); !ok {
+		t.Errorf("cachegroup must be deleter")
+	}
+	if _, ok := i.(api.Identifier); !ok {
+		t.Errorf("cachegroup must be Identifier")
+	}
+}
+
+func TestValidate(t *testing.T) {
+	// invalid name, shortname, loattude, and longitude
+	id := 1
+	nm := "not!a!valid!cachegroup"
+	sn := "not!a!valid!shortname"
+	la := -190.0
+	lo := -190.0
+	ty := "EDGE_LOC"
+	ti := 6
+	lu := tc.TimeNoMod{Time: time.Now()}
+	c := TOCacheGroup{ID: &id,
+		Name:        &nm,
+		ShortName:   &sn,
+		Latitude:    &la,
+		Longitude:   &lo,
+		Type:        &ty,
+		TypeID:      &ti,
+		LastUpdated: &lu,
+	}
+	errs := test.SortErrors(c.Validate(nil))
+
+	expectedErrs := []error{
+		errors.New(`'latitude' Must be a floating point number within the range +-90`),
+		errors.New(`'longitude' Must be a floating point number within the range +-180`),
+		errors.New(`'name' invalid characters found - Use alphanumeric . or - or _ .`),
+		errors.New(`'shortName' invalid characters found - Use alphanumeric . or - or _ .`),
+	}
+
+	if !reflect.DeepEqual(expectedErrs, errs) {
+		t.Errorf("expected %s, got %s", expectedErrs, errs)
+	}
+
+	//  valid name, shortName latitude, longitude
+	nm = "This.is.2.a-Valid---Cachegroup."
+	sn = `awesome-cachegroup`
+	la = 90.0
+	lo = 90.0
+	c = TOCacheGroup{ID: &id,
+		Name:        &nm,
+		ShortName:   &sn,
+		Latitude:    &la,
+		Longitude:   &lo,
+		Type:        &ty,
+		TypeID:      &ti,
+		LastUpdated: &lu,
+	}
+	expectedErrs = []error{}
+	errs = c.Validate(nil)
+	if !reflect.DeepEqual(expectedErrs, errs) {
+		t.Errorf("expected %s, got %s", expectedErrs, errs)
+	}
+}
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 1551d2c6c..c928fd13b 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -33,6 +33,7 @@ import (
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/asn"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/auth"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/cachegroup"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/cdn"
 	dsrequest "github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/request"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/request/comment"
@@ -99,6 +100,13 @@ func Routes(d ServerData) ([]Route, http.Handler, error) {
 		{1.3, http.MethodPost, `asns/?$`, api.CreateHandler(asn.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 		{1.3, http.MethodDelete, `asns/{id}$`, api.DeleteHandler(asn.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
+		//CacheGroups generic handlers:
+		{1.3, http.MethodPost, `cachegroups/?$`, api.CreateHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodGet, `cachegroups/?(\.json)?$`, api.ReadHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodGet, `cachegroups/{id}$`, api.ReadHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `cachegroups/{id}$`, api.UpdateHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodDelete, `cachegroups/{id}$`, api.DeleteHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+
 		//CDN generic handlers:
 		{1.3, http.MethodGet, `cdns/?(\.json)?$`, api.ReadHandler(cdn.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.3, http.MethodGet, `cdns/{id}$`, api.ReadHandler(cdn.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services