You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by ra...@apache.org on 2020/11/19 21:22:51 UTC

[trafficcontrol] branch master updated: Validate Topology Cache Group non-emptiness on Delivery Services's CDN (#5304)

This is an automated email from the ASF dual-hosted git repository.

rawlin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/trafficcontrol.git


The following commit(s) were added to refs/heads/master by this push:
     new f70f16b  Validate Topology Cache Group non-emptiness on Delivery Services's CDN (#5304)
f70f16b is described below

commit f70f16bb8b0cb23e529dacd044557b030ca689e2
Author: Zach Hoffman <zr...@apache.org>
AuthorDate: Thu Nov 19 14:22:38 2020 -0700

    Validate Topology Cache Group non-emptiness on Delivery Services's CDN (#5304)
    
    * Make it so Delivery Services cannot be created or updated if a Cache
    Group in its Topology has no Servers in the Delivery Service's CDN
    
    * Move import
    
    * Make GetTopologyCachegroups description accurate
    
    * Check the lengths of the right variables
    
    * Group ATC imports
---
 CHANGELOG.md                                       |   1 +
 .../testing/api/v3/deliveryservices_test.go        | 104 +++++++++++++
 .../traffic_ops_golang/dbhelpers/db_helpers.go     |  52 ++++++-
 .../deliveryservice/deliveryservices.go            |  16 +-
 .../deliveryservice/servers/servers.go             |   2 +-
 traffic_ops/traffic_ops_golang/server/servers.go   |   6 +-
 .../traffic_ops_golang/topology/topologies.go      | 143 +----------------
 .../topology_validation/topology_validation.go     | 172 +++++++++++++++++++++
 8 files changed, 329 insertions(+), 167 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index b4a82eb..514fc97 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,6 +7,7 @@ The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/).
 ### Added
 - Traffic Ops: added validation for assigning ORG servers to topology-based delivery services
 - Traffic Ops: added validation for topology updates and server updates/deletions to ensure that topologies have at least one server per cachegroup in each CDN of any assigned delivery services
+- Traffic Ops: added validation for delivery service updates to ensure that topologies have at least one server per cachegroup in each CDN of any assigned delivery services
 - Added locationByDeepCoverageZone to the `crs/stats/ip/{ip}` endpoint in the Traffic Router API
 
 ### Fixed
diff --git a/traffic_ops/testing/api/v3/deliveryservices_test.go b/traffic_ops/testing/api/v3/deliveryservices_test.go
index f21c50a..2b9099f 100644
--- a/traffic_ops/testing/api/v3/deliveryservices_test.go
+++ b/traffic_ops/testing/api/v3/deliveryservices_test.go
@@ -603,6 +603,110 @@ func UpdateDeliveryServiceWithInvalidTopology(t *testing.T) {
 	if err != nil {
 		t.Errorf("updating DS topology - expected: no error, actual: %v", err)
 	}
+
+	const xmlId = "top-ds-in-cdn2"
+	dses, _, err = TOSession.GetDeliveryServicesV30WithHdr(nil, url.Values{"xmlId": {xmlId}})
+	if err != nil {
+		t.Fatalf("getting Delivery Service %s: %s", xmlId, err.Error())
+	}
+	const expectedSize = 1
+	if len(dses) != expectedSize {
+		t.Fatalf("expected %d Delivery Service with xmlId %s but instead received %d Delivery Services", expectedSize, xmlId, len(dses))
+	}
+	ds = dses[0]
+	dsTopology := ds.Topology
+	ds.Topology = nil
+	ds, _, err = TOSession.UpdateDeliveryServiceV30WithHdr(*ds.ID, ds, nil)
+	if err != nil {
+		t.Fatalf("updating Delivery Service %s: %s", xmlId, err.Error())
+	}
+	const cdn1Name = "cdn1"
+	cdns, _, err := TOSession.GetCDNByNameWithHdr(cdn1Name, nil)
+	if err != nil {
+		t.Fatalf("getting CDN %s: %s", cdn1Name, err.Error())
+	}
+	if len(cdns) != expectedSize {
+		t.Fatalf("expected %d CDN with name %s but instead received %d CDNs", expectedSize, cdn1Name, len(cdns))
+	}
+	cdn1 := cdns[0]
+	const cacheGroupName = "dtrc1"
+	cachegroups, _, err := TOSession.GetCacheGroupsByQueryParamsWithHdr(url.Values{"name": {cacheGroupName}}, nil)
+	if err != nil {
+		t.Fatalf("getting Cache Group %s: %s", cacheGroupName, err.Error())
+	}
+	if len(cachegroups) != expectedSize {
+		t.Fatalf("expected %d Cache Group with name %s but instead received %d Cache Groups", expectedSize, cacheGroupName, len(cachegroups))
+	}
+	cachegroup := cachegroups[0]
+	params = url.Values{"cdn": {strconv.Itoa(*ds.CDNID)}, "cachegroup": {strconv.Itoa(*cachegroup.ID)}}
+	servers, _, err := TOSession.GetServersWithHdr(&params, nil)
+	if err != nil {
+		t.Fatalf("getting Server with params %v: %s", params, err.Error())
+	}
+	if len(servers.Response) != expectedSize {
+		t.Fatalf("expected %d Server returned for query params %v but instead received %d Servers", expectedSize, params, len(servers.Response))
+	}
+	server := servers.Response[0]
+	*server.CDNID = cdn1.ID
+
+	// A profile specific to CDN 1 is required
+	profileCopy := tc.ProfileCopy{
+		Name:         *server.Profile + "_BUT_IN_CDN1",
+		ExistingID:   *server.ProfileID,
+		ExistingName: *server.Profile,
+		Description:  *server.ProfileDesc,
+	}
+	_, _, err = TOSession.CopyProfile(profileCopy)
+	if err != nil {
+		t.Fatalf("copying Profile %s: %s", *server.Profile, err.Error())
+	}
+
+	profiles, _, err := TOSession.GetProfileByNameWithHdr(profileCopy.Name, nil)
+	if err != nil {
+		t.Fatalf("getting Profile %s: %s", profileCopy.Name, err.Error())
+	}
+	if len(profiles) != expectedSize {
+		t.Fatalf("expected %d Profile with name %s but instead received %d Profiles", expectedSize, profileCopy.Name, len(profiles))
+	}
+	profile := profiles[0]
+	profile.CDNID = cdn1.ID
+	_, _, err = TOSession.UpdateProfileByIDWithHdr(profile.ID, profile, nil)
+	if err != nil {
+		t.Fatalf("updating Profile %s: %s", profile.Name, err.Error())
+	}
+	*server.ProfileID = profile.ID
+
+	// Empty Cache Group dtrc1 with respect to CDN 2
+	_, _, err = TOSession.UpdateServerByIDWithHdr(*server.ID, server, nil)
+	if err != nil {
+		t.Fatalf("updating Server %s: %s", *server.HostName, err.Error())
+	}
+	ds.Topology = dsTopology
+	_, reqInf, err = TOSession.UpdateDeliveryServiceV30WithHdr(*ds.ID, ds, nil)
+	if err == nil {
+		t.Fatalf("expected 400-level error assigning Topology %s to Delivery Service %s because Cache Group %s has no Servers in it in CDN %d, no error received", *dsTopology, xmlId, cacheGroupName, *ds.CDNID)
+	}
+	if reqInf.StatusCode < http.StatusBadRequest || reqInf.StatusCode >= http.StatusInternalServerError {
+		t.Fatalf("expected %d-level status code but received status code %d", http.StatusBadRequest, reqInf.StatusCode)
+	}
+	*server.CDNID = *ds.CDNID
+	*server.ProfileID = profileCopy.ExistingID
+
+	// Put things back the way they were
+	_, _, err = TOSession.UpdateServerByIDWithHdr(*server.ID, server, nil)
+	if err != nil {
+		t.Fatalf("updating Server %s: %s", *server.HostName, err.Error())
+	}
+
+	_, _, err = TOSession.DeleteProfileByID(profile.ID)
+	if err != nil {
+		t.Fatalf("deleting Profile %s: %s", profile.Name, err.Error())
+	}
+
+	ds, reqInf, err = TOSession.UpdateDeliveryServiceV30WithHdr(*ds.ID, ds, nil)
+	if err != nil {
+		t.Fatalf("updating Delivery Service %s: %s", xmlId, err.Error())
+	}
 }
 
 // UpdateDeliveryServiceTopologyHeaderRewriteFields ensures that a delivery service can only use firstHeaderRewrite,
diff --git a/traffic_ops/traffic_ops_golang/dbhelpers/db_helpers.go b/traffic_ops/traffic_ops_golang/dbhelpers/db_helpers.go
index 53ce367..53edd6d 100644
--- a/traffic_ops/traffic_ops_golang/dbhelpers/db_helpers.go
+++ b/traffic_ops/traffic_ops_golang/dbhelpers/db_helpers.go
@@ -23,11 +23,14 @@ import (
 	"database/sql"
 	"errors"
 	"fmt"
+	"github.com/jmoiron/sqlx"
+	"net/http"
 	"strconv"
 	"strings"
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/topology/topology_validation"
 
 	"github.com/lib/pq"
 )
@@ -839,18 +842,51 @@ func TopologyExists(tx *sql.Tx, name string) (bool, error) {
 	return count > 0, err
 }
 
-// GetTopologyCachegroups returns the set of cachegroup names for the given topology.
-func GetTopologyCachegroups(tx *sql.Tx, name string) ([]string, error) {
+// CheckTopology returns an error if the given Topology does not exist or if one of the Topology's Cache Groups is
+// empty with respect to the Delivery Service's CDN.
+func CheckTopology(tx *sqlx.Tx, ds tc.DeliveryServiceNullableV30) (int, error, error) {
+	statusCode, userErr, sysErr := http.StatusOK, error(nil), error(nil)
+
+	if ds.Topology == nil {
+		return statusCode, userErr, sysErr
+	}
+
+	cacheGroupIDs, _, err := GetTopologyCachegroups(tx.Tx, *ds.Topology)
+	if err != nil {
+		return http.StatusInternalServerError, nil, fmt.Errorf("getting topology cachegroups: %v", err)
+	}
+	if len(cacheGroupIDs) == 0 {
+		return http.StatusBadRequest, fmt.Errorf("no such Topology '%s'", *ds.Topology), nil
+	}
+
+	if err = topology_validation.CheckForEmptyCacheGroups(tx, cacheGroupIDs, []int{*ds.CDNID}, true, []int{}); err != nil {
+		return http.StatusBadRequest, fmt.Errorf("empty cachegroups in Topology %s found for CDN %d: %s", *ds.Topology, ds.CDNID, err.Error()), nil
+	}
+
+	return statusCode, userErr, sysErr
+}
+
+// GetTopologyCachegroups returns an array of cachegroup IDs and an array of cachegroup
+// names for the given topology, or any error.
+func GetTopologyCachegroups(tx *sql.Tx, name string) ([]int, []string, error) {
 	q := `
-	SELECT ARRAY_AGG(tc.cachegroup)
+	SELECT ARRAY_AGG(c.id), ARRAY_AGG(tc.cachegroup)
 	FROM topology_cachegroup tc
+	JOIN cachegroup c ON tc.cachegroup = c."name"
 	WHERE tc.topology = $1
-	`
-	cachegroups := []string{}
-	if err := tx.QueryRow(q, name).Scan(pq.Array(&cachegroups)); err != nil {
-		return nil, fmt.Errorf("querying topology '%s' cachegroups: %s", name, err)
+`
+	int64Ids := []int64{}
+	names := []string{}
+	if err := tx.QueryRow(q, name).Scan(pq.Array(&int64Ids), pq.Array(&names)); err != nil {
+		return nil, nil, fmt.Errorf("querying topology '%s' cachegroups: %s", name, err)
+	}
+
+	ids := make([]int, len(int64Ids))
+	for index, int64Id := range int64Ids {
+		ids[index] = int(int64Id)
 	}
-	return cachegroups, nil
+
+	return ids, names, nil
 }
 
 // GetDeliveryServicesWithTopologies returns a list containing the delivery services in the given dsIDs
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservices.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservices.go
index 5bd05fd..d8adb48 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservices.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservices.go
@@ -238,18 +238,6 @@ func createV15(w http.ResponseWriter, r *http.Request, inf *api.APIInfo, reqDS t
 	return nil, status, userErr, sysErr
 }
 
-func checkTopology(tx *sql.Tx, ds tc.DeliveryServiceNullableV30) (int, error, error) {
-	if ds.Topology != nil {
-		if ok, err := dbhelpers.TopologyExists(tx, *ds.Topology); err != nil {
-			return http.StatusInternalServerError, nil, fmt.Errorf("checking topology existence: %v", err)
-		} else if !ok {
-			return http.StatusBadRequest, fmt.Errorf("no such Topology '%s'", *ds.Topology), nil
-		}
-	}
-
-	return http.StatusOK, nil, nil
-}
-
 // create creates the given ds in the database, and returns the DS with its id and other fields created on insert set. On error, the HTTP status code, user error, and system error are returned. The status code SHOULD NOT be used, if both errors are nil.
 func createV30(w http.ResponseWriter, r *http.Request, inf *api.APIInfo, ds tc.DeliveryServiceNullableV30) (*tc.DeliveryServiceNullableV30, int, error, error) {
 	user := inf.User
@@ -272,7 +260,7 @@ func createV30(w http.ResponseWriter, r *http.Request, inf *api.APIInfo, ds tc.D
 		deepCachingType = ds.DeepCachingType.String() // necessary, because DeepCachingType's default needs to insert the string, not "", and Query doesn't call .String().
 	}
 
-	if errCode, userErr, sysErr := checkTopology(tx, ds); userErr != nil || sysErr != nil {
+	if errCode, userErr, sysErr := dbhelpers.CheckTopology(inf.Tx, ds); userErr != nil || sysErr != nil {
 		return nil, errCode, userErr, sysErr
 	}
 
@@ -794,7 +782,7 @@ func updateV30(w http.ResponseWriter, r *http.Request, inf *api.APIInfo, ds *tc.
 		return nil, errCode, userErr, sysErr
 	}
 
-	if errCode, userErr, sysErr = checkTopology(tx, *ds); userErr != nil || sysErr != nil {
+	if errCode, userErr, sysErr = dbhelpers.CheckTopology(inf.Tx, *ds); userErr != nil || sysErr != nil {
 		return nil, errCode, userErr, sysErr
 	}
 
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
index d099a3f..1c5126b 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
@@ -521,7 +521,7 @@ func validateDSS(tx *sql.Tx, ds DSInfo, servers []tc.ServerInfo) (error, error,
 		}
 	}
 
-	cachegroups, sysErr := dbhelpers.GetTopologyCachegroups(tx, *ds.Topology)
+	_, cachegroups, sysErr := dbhelpers.GetTopologyCachegroups(tx, *ds.Topology)
 	if sysErr != nil {
 		return nil, fmt.Errorf("validating %s servers in topology %s: %v", tc.OriginTypeName, *ds.Topology, sysErr), http.StatusInternalServerError
 	}
diff --git a/traffic_ops/traffic_ops_golang/server/servers.go b/traffic_ops/traffic_ops_golang/server/servers.go
index 369dee9..bb91fd5 100644
--- a/traffic_ops/traffic_ops_golang/server/servers.go
+++ b/traffic_ops/traffic_ops_golang/server/servers.go
@@ -42,7 +42,7 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/routing/middleware"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/topology"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/topology/topology_validation"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/util/ims"
 
 	validation "github.com/go-ozzo/ozzo-validation"
@@ -1295,7 +1295,7 @@ func Update(w http.ResponseWriter, r *http.Request) {
 		}
 		cacheGroupIds := []int{*origServer.CachegroupID}
 		serverIds := []int{*origServer.ID}
-		if err = topology.CheckForEmptyCacheGroups(inf.Tx, cacheGroupIds, CDNIDs, true, serverIds); err != nil {
+		if err = topology_validation.CheckForEmptyCacheGroups(inf.Tx, cacheGroupIds, CDNIDs, true, serverIds); err != nil {
 			api.HandleErr(w, r, tx, http.StatusBadRequest, errors.New("server is the last one in its cachegroup, which is used by a topology, so it cannot be moved to another cachegroup: "+err.Error()), nil)
 			return
 		}
@@ -1646,7 +1646,7 @@ func Delete(w http.ResponseWriter, r *http.Request) {
 	if hasDSOnCDN {
 		CDNIDs = append(CDNIDs, *server.CDNID)
 	}
-	if err := topology.CheckForEmptyCacheGroups(inf.Tx, cacheGroupIds, CDNIDs, true, serverIds); err != nil {
+	if err := topology_validation.CheckForEmptyCacheGroups(inf.Tx, cacheGroupIds, CDNIDs, true, serverIds); err != nil {
 		api.HandleErr(w, r, tx, http.StatusBadRequest, errors.New("server is the last one in its cachegroup, which is used by a topology: "+err.Error()), nil)
 		return
 	}
diff --git a/traffic_ops/traffic_ops_golang/topology/topologies.go b/traffic_ops/traffic_ops_golang/topology/topologies.go
index 57a9745..ee7e0df 100644
--- a/traffic_ops/traffic_ops_golang/topology/topologies.go
+++ b/traffic_ops/traffic_ops_golang/topology/topologies.go
@@ -35,10 +35,10 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/cachegroup"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/topology/topology_validation"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/util/ims"
 
 	validation "github.com/go-ozzo/ozzo-validation"
-	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
 
@@ -147,7 +147,7 @@ func (topology *TOTopology) Validate() error {
 		log.Errorf("validating topology: %v", err)
 		return errors.New("unable to validate topology")
 	}
-	rules["empty cachegroups"] = CheckForEmptyCacheGroups(topology.ReqInfo.Tx, cacheGroupIds, dsCDNs, false, nil)
+	rules["empty cachegroups"] = topology_validation.CheckForEmptyCacheGroups(topology.ReqInfo.Tx, cacheGroupIds, dsCDNs, false, nil)
 	rules["required capabilities"] = topology.validateDSRequiredCapabilities()
 
 	/* Only perform further checks if everything so far is valid */
@@ -164,114 +164,6 @@ func (topology *TOTopology) Validate() error {
 	return util.JoinErrs(tovalidate.ToErrors(rules))
 }
 
-// CheckForEmptyCacheGroups checks if the cachegroups are empty (altogether) or empty in any of the given CDN IDs.
-// If cachegroupsInTopology is true, it will only check cachegroups that are used in a topology. Any server IDs in
-// excludeServerIds will not be counted.
-func CheckForEmptyCacheGroups(tx *sqlx.Tx, cacheGroupIds []int, CDNIDs []int, cachegroupsInTopology bool, excludeServerIds []int) error {
-	if excludeServerIds == nil {
-		excludeServerIds = []int{}
-	}
-	var (
-		baseError   = errors.New("unable to check for cachegroups with no servers")
-		systemError = "checking for cachegroups with no servers: %s"
-		query       = selectEmptyCacheGroupsQuery(cachegroupsInTopology)
-		parameters  = map[string]interface{}{
-			"cachegroup_ids":     pq.Array(cacheGroupIds),
-			"exclude_server_ids": pq.Array(excludeServerIds),
-		}
-	)
-
-	rows, err := tx.NamedQuery(query, parameters)
-	if err != nil {
-		log.Errorf(systemError, err.Error())
-		return baseError
-	}
-
-	var (
-		serverCountByCDN int
-		cacheGroup       string
-		cdnID            *int
-	)
-	cgServerCountsByCDN := make(map[int]map[string]int)
-	cgServerCounts := make(map[string]int)
-	topologySetByCachegroup := make(map[string]map[string]struct{})
-	defer log.Close(rows, "unable to close DB connection when checking for cachegroups with no servers")
-	for rows.Next() {
-		var scanTo = []interface{}{&cacheGroup, &cdnID, &serverCountByCDN}
-		var topologiesForRow []string
-		if cachegroupsInTopology {
-			scanTo = append(scanTo, pq.Array(&topologiesForRow))
-		}
-		if err := rows.Scan(scanTo...); err != nil {
-			log.Errorf(systemError, err.Error())
-			return baseError
-		}
-		if cdnID != nil {
-			if _, ok := cgServerCountsByCDN[*cdnID]; !ok {
-				cgServerCountsByCDN[*cdnID] = make(map[string]int)
-			}
-			cgServerCountsByCDN[*cdnID][cacheGroup] = serverCountByCDN
-		}
-		cgServerCounts[cacheGroup] += serverCountByCDN
-
-		if cachegroupsInTopology {
-			if _, ok := topologySetByCachegroup[cacheGroup]; !ok {
-				topologySetByCachegroup[cacheGroup] = make(map[string]struct{})
-			}
-			for _, topology := range topologiesForRow {
-				topologySetByCachegroup[cacheGroup][topology] = struct{}{}
-			}
-		}
-	}
-	topologiesByCachegroup := make(map[string][]string, len(topologySetByCachegroup))
-	for cg, topologySet := range topologySetByCachegroup {
-		for topology := range topologySet {
-			topologiesByCachegroup[cg] = append(topologiesByCachegroup[cg], topology)
-		}
-	}
-	emptyCachegroups := []string{}
-	for cg, count := range cgServerCounts {
-		if count == 0 {
-			messageEntry := cg
-			if cachegroupsInTopology {
-				messageEntry += " (in topologies: " + strings.Join(topologiesByCachegroup[cg], ", ") + ")"
-			}
-			emptyCachegroups = append(emptyCachegroups, messageEntry)
-		}
-	}
-
-	if len(emptyCachegroups) > 0 {
-		errMessage := "cachegroups with no servers in them: " + strings.Join(emptyCachegroups, ", ")
-		return errors.New(errMessage)
-	}
-
-	errMessage := []string{}
-	for _, cdnID := range CDNIDs {
-		if _, ok := cgServerCountsByCDN[cdnID]; !ok {
-			return fmt.Errorf("topology is assigned to delivery service on CDN %d, but that CDN has no servers", cdnID)
-		}
-		emptyCachegroupsByCDN := []string{}
-		for cg, serverCount := range cgServerCountsByCDN[cdnID] {
-			if serverCount == 0 {
-				emptyCachegroupsByCDN = append(emptyCachegroupsByCDN, cg)
-			}
-		}
-		// check that this CDN has a count for all given cachegroups
-		for cg := range cgServerCounts {
-			if _, ok := cgServerCountsByCDN[cdnID][cg]; !ok {
-				emptyCachegroupsByCDN = append(emptyCachegroupsByCDN, cg)
-			}
-		}
-		if len(emptyCachegroupsByCDN) > 0 {
-			errMessage = append(errMessage, fmt.Sprintf("topology is assigned to delivery service(s) on CDN %d, but the following cachegroups have no servers in CDN %d: %s", cdnID, cdnID, strings.Join(emptyCachegroupsByCDN, ", ")))
-		}
-	}
-	if len(errMessage) > 0 {
-		return errors.New(strings.Join(errMessage, "; "))
-	}
-	return nil
-}
-
 func (topology *TOTopology) nodesInOtherTopologies() ([]tc.TopologyNode, map[string][]string, error) {
 	baseError := errors.New("unable to verify that there are no cycles across all topologies")
 	where := `WHERE name != :topology_name`
@@ -809,37 +701,6 @@ JOIN topology_cachegroup tc on t.name = tc.topology
 	return query
 }
 
-func selectEmptyCacheGroupsQuery(cachegroupsInTopology bool) string {
-	var joinTopologyCachegroups string
-	var topologyNames string
-	if cachegroupsInTopology {
-		// language=SQL
-		topologyNames = `
-		, ARRAY_AGG(tc.topology)
-`
-		// language=SQL
-		joinTopologyCachegroups = `
-		JOIN topology_cachegroup tc ON c."name" = tc.cachegroup
-`
-	}
-	// language=SQL
-	query := fmt.Sprintf(`
-		SELECT
-			c."name",
-			s.cdn_id,
-			COUNT(*) FILTER (
-			    WHERE s.id IS NOT NULL
-			    AND NOT(s."id" = ANY(CAST(:exclude_server_ids AS INT[])))
-			) AS server_count %s
-		FROM cachegroup c
-		%s
-		LEFT JOIN "server" s ON c.id = s.cachegroup
-		WHERE c."id" = ANY(CAST(:cachegroup_ids AS BIGINT[]))
-		GROUP BY c."name", s.cdn_id
-	`, topologyNames, joinTopologyCachegroups)
-	return query
-}
-
 func selectNonTopologyCacheGroupsQuery() string {
 	query := `
 SELECT 'non-topology cachegroups' AS name, c."name" AS cachegroup,
diff --git a/traffic_ops/traffic_ops_golang/topology/topology_validation/topology_validation.go b/traffic_ops/traffic_ops_golang/topology/topology_validation/topology_validation.go
new file mode 100644
index 0000000..e91f9e7
--- /dev/null
+++ b/traffic_ops/traffic_ops_golang/topology/topology_validation/topology_validation.go
@@ -0,0 +1,172 @@
+// The topology_validation package is for topology validation functions that are used outside of the topology
+// package, in order to prevent import cycles.
+
+package topology_validation
+
+/*
+ * 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"
+	"github.com/jmoiron/sqlx"
+	"github.com/lib/pq"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+)
+
+// CheckForEmptyCacheGroups checks if the cachegroups are empty (altogether) or empty in any of the given CDN IDs.
+// If cachegroupsInTopology is true, it will only check cachegroups that are used in a topology. Any server IDs in
+// excludeServerIds will not be counted.
+func CheckForEmptyCacheGroups(tx *sqlx.Tx, cacheGroupIds []int, CDNIDs []int, cachegroupsInTopology bool, excludeServerIds []int) error {
+	if excludeServerIds == nil {
+		excludeServerIds = []int{}
+	}
+	var (
+		baseError   = errors.New("unable to check for cachegroups with no servers")
+		systemError = "checking for cachegroups with no servers: %s"
+		query       = selectEmptyCacheGroupsQuery(cachegroupsInTopology)
+		parameters  = map[string]interface{}{
+			"cachegroup_ids":     pq.Array(cacheGroupIds),
+			"exclude_server_ids": pq.Array(excludeServerIds),
+		}
+	)
+
+	rows, err := tx.NamedQuery(query, parameters)
+	if err != nil {
+		log.Errorf(systemError, err.Error())
+		return baseError
+	}
+
+	var (
+		serverCountByCDN int
+		cacheGroup       string
+		cdnID            *int
+	)
+	cgServerCountsByCDN := make(map[int]map[string]int)
+	cgServerCounts := make(map[string]int)
+	topologySetByCachegroup := make(map[string]map[string]struct{})
+	defer log.Close(rows, "unable to close DB connection when checking for cachegroups with no servers")
+	for rows.Next() {
+		var scanTo = []interface{}{&cacheGroup, &cdnID, &serverCountByCDN}
+		var topologiesForRow []string
+		if cachegroupsInTopology {
+			scanTo = append(scanTo, pq.Array(&topologiesForRow))
+		}
+		if err := rows.Scan(scanTo...); err != nil {
+			log.Errorf(systemError, err.Error())
+			return baseError
+		}
+		if cdnID != nil {
+			if _, ok := cgServerCountsByCDN[*cdnID]; !ok {
+				cgServerCountsByCDN[*cdnID] = make(map[string]int)
+			}
+			cgServerCountsByCDN[*cdnID][cacheGroup] = serverCountByCDN
+		}
+		cgServerCounts[cacheGroup] += serverCountByCDN
+
+		if cachegroupsInTopology {
+			if _, ok := topologySetByCachegroup[cacheGroup]; !ok {
+				topologySetByCachegroup[cacheGroup] = make(map[string]struct{})
+			}
+			for _, topology := range topologiesForRow {
+				topologySetByCachegroup[cacheGroup][topology] = struct{}{}
+			}
+		}
+	}
+	topologiesByCachegroup := make(map[string][]string, len(topologySetByCachegroup))
+	for cg, topologySet := range topologySetByCachegroup {
+		for topology := range topologySet {
+			topologiesByCachegroup[cg] = append(topologiesByCachegroup[cg], topology)
+		}
+	}
+	emptyCachegroups := []string{}
+	for cg, count := range cgServerCounts {
+		if count == 0 {
+			messageEntry := cg
+			if cachegroupsInTopology {
+				messageEntry += " (in topologies: " + strings.Join(topologiesByCachegroup[cg], ", ") + ")"
+			}
+			emptyCachegroups = append(emptyCachegroups, messageEntry)
+		}
+	}
+
+	if len(emptyCachegroups) > 0 {
+		errMessage := "cachegroups with no servers in them: " + strings.Join(emptyCachegroups, ", ")
+		return errors.New(errMessage)
+	}
+
+	errMessage := []string{}
+	for _, cdnID := range CDNIDs {
+		if _, ok := cgServerCountsByCDN[cdnID]; !ok {
+			return fmt.Errorf("topology is assigned to delivery service on CDN %d, but that CDN has no servers", cdnID)
+		}
+		emptyCachegroupsByCDN := []string{}
+		for cg, serverCount := range cgServerCountsByCDN[cdnID] {
+			if serverCount == 0 {
+				emptyCachegroupsByCDN = append(emptyCachegroupsByCDN, cg)
+			}
+		}
+		// check that this CDN has a count for all given cachegroups
+		for cg := range cgServerCounts {
+			if _, ok := cgServerCountsByCDN[cdnID][cg]; !ok {
+				emptyCachegroupsByCDN = append(emptyCachegroupsByCDN, cg)
+			}
+		}
+		if len(emptyCachegroupsByCDN) > 0 {
+			errMessage = append(errMessage, fmt.Sprintf("topology is assigned to delivery service(s) on CDN %d, but the following cachegroups have no servers in CDN %d: %s", cdnID, cdnID, strings.Join(emptyCachegroupsByCDN, ", ")))
+		}
+	}
+	if len(errMessage) > 0 {
+		return errors.New(strings.Join(errMessage, "; "))
+	}
+	return nil
+}
+
+func selectEmptyCacheGroupsQuery(cachegroupsInTopology bool) string {
+	var joinTopologyCachegroups string
+	var topologyNames string
+	if cachegroupsInTopology {
+		// language=SQL
+		topologyNames = `
+		, ARRAY_AGG(tc.topology)
+`
+		// language=SQL
+		joinTopologyCachegroups = `
+		JOIN topology_cachegroup tc ON c."name" = tc.cachegroup
+`
+	}
+	// language=SQL
+	query := fmt.Sprintf(`
+		SELECT
+			c."name",
+			s.cdn_id,
+			COUNT(*) FILTER (
+			    WHERE s.id IS NOT NULL
+			    AND NOT(s."id" = ANY(CAST(:exclude_server_ids AS INT[])))
+			) AS server_count %s
+		FROM cachegroup c
+		%s
+		LEFT JOIN "server" s ON c.id = s.cachegroup
+		WHERE c."id" = ANY(CAST(:cachegroup_ids AS BIGINT[]))
+		GROUP BY c."name", s.cdn_id
+	`, topologyNames, joinTopologyCachegroups)
+	return query
+}