You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by ro...@apache.org on 2022/11/02 14:36:34 UTC

[trafficcontrol] branch master updated: T3C parent.config simulate topology for non topo delivery services (#7137)

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

rob 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 56c636a6d4 T3C parent.config simulate topology for non topo delivery services (#7137)
56c636a6d4 is described below

commit 56c636a6d483a4ea3c6e98d80aed436b0b9b6243
Author: Brian Olsen <br...@comcast.com>
AuthorDate: Wed Nov 2 08:36:27 2022 -0600

    T3C parent.config simulate topology for non topo delivery services (#7137)
    
    * T3C parent.config simulate topology for non topo delivery services
    
    * add changelog entry
    
    * change topology check and remove debug
    
    * t3c parentdotconfig: factor out CreateTopologies
---
 CHANGELOG.md                          |   1 +
 lib/go-atscfg/atscfg.go               |   5 +
 lib/go-atscfg/parentdotconfig.go      | 385 +++++++++++++---------------------
 lib/go-atscfg/parentdotconfig_test.go |  25 +--
 4 files changed, 160 insertions(+), 256 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 8b5ac921bb..59dc09c414 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -41,6 +41,7 @@ The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/).
 - Updated Apache Tomcat from 9.0.43 to 9.0.67
 - [#7125](https://github.com/apache/trafficcontrol/issues/7125) *Docs* Reflect implementation and deprecation notice for `letsencrypt/autorenew` endpoint.
 - [#7158](https://github.com/apache/trafficcontrol/issues/7158) *Traffic Vault* Fix the `reencrypt` utility to uniquely reencrypt each version of the SSL Certificates.
+- [#7137](https://github.com/apache/trafficcontrol/pull/7137) *Cache Config* parent.config simulate topology for non topo delivery services.
 
 ## [7.0.0] - 2022-07-19
 ### Added
diff --git a/lib/go-atscfg/atscfg.go b/lib/go-atscfg/atscfg.go
index 55467a93c2..91dbdde4cf 100644
--- a/lib/go-atscfg/atscfg.go
+++ b/lib/go-atscfg/atscfg.go
@@ -886,3 +886,8 @@ func ProfilesMatch(pa []string, pb []string) bool {
 	}
 	return true
 }
+
+// IsGoDirect checks if this ds type is edge only.
+func IsGoDirect(ds DeliveryService) bool {
+	return *ds.Type == tc.DSTypeHTTPNoCache || *ds.Type == tc.DSTypeHTTPLive || *ds.Type == tc.DSTypeDNSLive
+}
diff --git a/lib/go-atscfg/parentdotconfig.go b/lib/go-atscfg/parentdotconfig.go
index 5bdb475748..9047ef2497 100644
--- a/lib/go-atscfg/parentdotconfig.go
+++ b/lib/go-atscfg/parentdotconfig.go
@@ -165,6 +165,80 @@ func MakeParentDotConfig(
 	}, nil
 }
 
+// CreateTopology creates an on the fly topology for this server and non topology delivery service.
+func CreateTopology(server *Server, ds DeliveryService, nameTopologies map[TopologyName]tc.Topology, ocgmap map[OriginHost][]string) (string, tc.Topology, []string) {
+
+	topoName := ""
+	topo := tc.Topology{}
+	warns := []string{}
+
+	orgFQDNStr := *ds.OrgServerFQDN
+	orgURI, orgWarns, err := getOriginURI(orgFQDNStr)
+	warns = append(warns, orgWarns...)
+	if err != nil {
+		warns = append(warns, "DS '"+*ds.XMLID+"' has malformed origin URI: '"+orgFQDNStr+"': skipping!"+err.Error())
+		return topoName, topo, warns
+	}
+
+	// use the topology name for the fqdn
+	cgnames, ok := ocgmap[OriginHost(orgURI.Hostname())]
+	if !ok {
+		cgnames, ok = ocgmap[OriginHost(deliveryServicesAllParentsKey)]
+		if !ok {
+			warns = append(warns, "DS '"+*ds.XMLID+"' has no parent cache groups! Skipping!")
+			return topoName, topo, warns
+		}
+	}
+
+	// Manufactured topology
+	topoName = "otf_" + *ds.XMLID
+
+	// ensure name is unique
+	if _, ok := nameTopologies[TopologyName(topoName)]; ok {
+		warns = append(warns, "Found collision for topo name '"+topoName+"' for ds: '", *ds.XMLID+"'")
+		topoName = topoName + "_"
+	}
+
+	topo = tc.Topology{Name: topoName}
+
+	if IsGoDirect(ds) {
+		node := tc.TopologyNode{
+			Cachegroup: *server.Cachegroup,
+		}
+		topo.Nodes = append(topo.Nodes, node)
+	} else {
+		// If mid cache group, insert fake edge cache group.
+		// This is incorrect if there are multiple MID tiers.
+		pind := 1
+		if strings.HasPrefix(server.Type, tc.MidTypePrefix) {
+			parents := []int{pind}
+			pind++
+			edgeNode := tc.TopologyNode{
+				Cachegroup: "fake_edgecg",
+				Parents:    parents,
+			}
+			topo.Nodes = append(topo.Nodes, edgeNode)
+		}
+
+		parents := []int{}
+		for ind := 0; ind < len(cgnames); ind++ {
+			parents = append(parents, pind)
+			pind++
+		}
+
+		node := tc.TopologyNode{
+			Cachegroup: *server.Cachegroup,
+			Parents:    parents,
+		}
+		topo.Nodes = append(topo.Nodes, node)
+
+		for _, cg := range cgnames {
+			topo.Nodes = append(topo.Nodes, tc.TopologyNode{Cachegroup: cg})
+		}
+	}
+	return topoName, topo, warns
+}
+
 func makeParentDotConfigData(
 	dses []DeliveryService,
 	server *Server,
@@ -355,6 +429,8 @@ func makeParentDotConfigData(
 	dsOrigins, dsOriginWarns := makeDSOrigins(dss, dses, servers)
 	warnings = append(warnings, dsOriginWarns...)
 
+	ocgmap := map[OriginHost][]string{}
+
 	for _, ds := range dses {
 
 		if ds.XMLID == nil || *ds.XMLID == "" {
@@ -383,258 +459,56 @@ func makeParentDotConfigData(
 			continue
 		}
 
-		isMSO := ds.MultiSiteOrigin != nil && *ds.MultiSiteOrigin
-
-		// TODO put these in separate functions. No if-statement should be this long.
-		if ds.Topology != nil && *ds.Topology != "" {
-			pasvc, topoWarnings, err := getTopologyParentConfigLine(
-				server,
-				serversWithParams,
-				&ds,
-				serverParams,
-				parentConfigParams,
-				nameTopologies,
-				serverCapabilities,
-				dsRequiredCapabilities,
-				cacheGroups,
-				profileParentConfigParams,
-				isMSO,
-				atsMajorVersion,
-				dsOrigins[DeliveryServiceID(*ds.ID)],
-				opt.AddComments,
-			)
-			warnings = append(warnings, topoWarnings...)
-			if err != nil {
-				// we don't want to fail generation with an error if one ds is malformed
-				warnings = append(warnings, err.Error()) // getTopologyParentConfigLine includes error context
-				continue
-			}
-
-			if pasvc != nil { // will be nil with no error if this server isn't in the Topology, or if it doesn't have the Required Capabilities
-				parentAbstraction.Services = append(parentAbstraction.Services, pasvc)
-			}
-		} else {
-			isLastCacheTier := noTopologyServerIsLastCacheForDS(server, &ds, cacheGroups)
-			serverPlacement := TopologyPlacement{
-				IsLastCacheTier:  isLastCacheTier,
-				IsFirstCacheTier: !isLastCacheTier || !ds.Type.UsesMidCache(),
-			}
+		// manufacture a topology for this DS.
+		if ds.Topology == nil || *ds.Topology == "" {
 
-			dsParams, dswarns := getParentDSParams(ds, profileParentConfigParams, serverPlacement, isMSO)
-			warnings = append(warnings, dswarns...)
-
-			if cacheIsTopLevel {
-				parentQStr := false
-				if dsParams.QueryStringHandling == "" && dsParams.Algorithm == tc.AlgorithmConsistentHash && ds.QStringIgnore != nil && tc.QStringIgnore(*ds.QStringIgnore) == tc.QStringIgnoreUseInCacheKeyAndPassUp {
-					parentQStr = true
-				}
-
-				orgFQDNStr := *ds.OrgServerFQDN
-				// if this cache isn't the last tier, i.e. we're not going to the origin, use http not https
-				if !isLastCacheTier {
-					orgFQDNStr = strings.Replace(orgFQDNStr, `https://`, `http://`, -1)
-				}
-				orgURI, orgWarns, err := getOriginURI(orgFQDNStr)
-				warnings = append(warnings, orgWarns...)
-				if err != nil {
-					warnings = append(warnings, "DS '"+*ds.XMLID+"' has malformed origin URI: '"+orgFQDNStr+"': skipping!"+err.Error())
-					continue
+			// only populate if there are non topology ds's
+			if len(ocgmap) == 0 {
+				ocgmap = makeOCGMap(parentInfos)
+				if len(ocgmap) == 0 {
+					ocgmap[""] = []string{}
 				}
+			}
 
-				pasvc := &ParentAbstractionService{}
-				pasvc.Name = *ds.XMLID
-
-				if ds.OriginShield != nil && *ds.OriginShield != "" {
-
-					policy := ParentAbstractionServiceRetryPolicyConsistentHash
-
-					if parentSelectAlg := serverParams[ParentConfigRetryKeysDefault.Algorithm]; strings.TrimSpace(parentSelectAlg) != "" {
-						paramPolicy := ParentSelectAlgorithmToParentAbstractionServiceRetryPolicy(parentSelectAlg)
-						if paramPolicy != ParentAbstractionServiceRetryPolicyInvalid {
-							policy = paramPolicy
-						} else {
-							warnings = append(warnings, "DS '"+*ds.XMLID+"' had malformed "+ParentConfigRetryKeysDefault.Algorithm+" parameter '"+parentSelectAlg+"', not using!")
-						}
-					}
-					pasvc.Comment = makeParentComment(opt.AddComments, *ds.XMLID, "")
-					pasvc.DestDomain = orgURI.Hostname()
-					pasvc.Port, err = strconv.Atoi(orgURI.Port())
-					if err != nil {
-						if strings.ToLower(orgURI.Scheme) == "https" {
-							pasvc.Port = 443
-						} else {
-							pasvc.Port = 80
-						}
-						warnings = append(warnings, "DS '"+*ds.XMLID+"' had malformed origin  port: '"+orgURI.Port()+"': using "+strconv.Itoa(pasvc.Port)+"! : "+err.Error())
-					}
-
-					fqdnPort := strings.Split(*ds.OriginShield, ":")
-					parent := &ParentAbstractionServiceParent{}
-					parent.FQDN = fqdnPort[0]
-					if len(fqdnPort) > 1 {
-						parent.Port, err = strconv.Atoi(fqdnPort[1])
-						if err != nil {
-							parent.Port = 80
-							warnings = append(warnings, "DS '"+*ds.XMLID+"' had malformed origin  port: '"+*ds.OriginShield+"': using "+strconv.Itoa(parent.Port)+"! : "+err.Error())
-						}
-					} else {
-						parent.Port = 80
-						warnings = append(warnings, "DS '"+*ds.XMLID+"' had no origin port: '"+*ds.OriginShield+"': using "+strconv.Itoa(parent.Port)+"!")
-					}
-					pasvc.Parents = append(pasvc.Parents, parent)
-					pasvc.RetryPolicy = policy
-					pasvc.GoDirect = true
-
-					// textLine += "dest_domain=" + orgURI.Hostname() + " port=" + orgURI.Port() + " parent=" + *ds.OriginShield + " " + algorithm + " go_direct=true\n"
-
-				} else if ds.MultiSiteOrigin != nil && *ds.MultiSiteOrigin {
-					pasvc.Comment = makeParentComment(opt.AddComments, *ds.XMLID, "")
-					pasvc.DestDomain = orgURI.Hostname()
-					pasvc.Port, err = strconv.Atoi(orgURI.Port())
-					if err != nil {
-						pasvc.Port = 80
-						warnings = append(warnings, "DS '"+*ds.XMLID+"' had malformed origin  port: '"+orgURI.Port()+"': using "+strconv.Itoa(pasvc.Port)+"! : "+err.Error())
-					}
-
-					// textLine += "dest_domain=" + orgURI.Hostname() + " port=" + orgURI.Port() + " "
-					if len(parentInfos) == 0 {
-					}
-
-					if len(parentInfos[OriginHost(orgURI.Hostname())]) == 0 {
-						// TODO error? emulates Perl
-						warnings = append(warnings, "DS "+*ds.XMLID+" has no parent servers")
-					}
-
-					parents, secondaryParents, secondaryMode, parentWarns := getMSOParentStrs(&ds, parentInfos[OriginHost(orgURI.Hostname())], atsMajorVersion, dsParams.Algorithm, dsParams.TryAllPrimariesBeforeSecondary)
-					warnings = append(warnings, parentWarns...)
-					pasvc.Parents = parents
-					pasvc.SecondaryParents = secondaryParents
-					pasvc.SecondaryMode = secondaryMode
-					pasvc.RetryPolicy = dsParams.Algorithm // TODO convert
-					pasvc.IgnoreQueryStringInParentSelection = !parentQStr
-					pasvc.GoDirect = true
-
-					// textLine += parents + secondaryParents + ` round_robin=` + dsParams.Algorithm + ` qstring=` + parentQStr + ` go_direct=false parent_is_proxy=false`
-
-					prWarns := dsParams.FillParentSvcRetries(cacheIsTopLevel, atsMajorVersion, pasvc)
-					warnings = append(warnings, prWarns...)
-
-					parentAbstraction.Services = append(parentAbstraction.Services, pasvc)
-				}
-			} else {
-				queryStringHandling := ParentSelectParamQStringHandlingToBool(serverParams[ParentConfigParamQStringHandling]) // "qsh" in Perl
-				if queryStringHandling == nil && serverParams[ParentConfigParamQStringHandling] != "" {
-					warnings = append(warnings, "Server Parameter '"+ParentConfigParamQStringHandling+"' value '"+serverParams[ParentConfigParamQStringHandling]+"' malformed, not using!")
-				}
-
-				roundRobin := ParentAbstractionServiceRetryPolicyConsistentHash
-				// roundRobin := `round_robin=consistent_hash`
-				goDirect := false
-				// goDirect := `go_direct=false`
-
-				parents, secondaryParents, secondaryMode, parentWarns := getParentStrs(&ds, dsRequiredCapabilities, parentInfos[deliveryServicesAllParentsKey], atsMajorVersion, dsParams.TryAllPrimariesBeforeSecondary)
-				warnings = append(warnings, parentWarns...)
-
-				pasvc := &ParentAbstractionService{}
-				pasvc.Name = *ds.XMLID
+			topoName, topo, warns := CreateTopology(server, ds, nameTopologies, ocgmap)
 
-				// peering ring check
-				if dsParams.UsePeering {
-					secondaryMode = ParentAbstractionServiceParentSecondaryModePeering
-				}
+			warnings = append(warnings, warns...)
+			if topoName == "" {
+				continue
+			}
 
-				orgFQDNStr := *ds.OrgServerFQDN
-				// if this cache isn't the last tier, i.e. we're not going to the origin, use http not https
-				if !isLastCacheTier {
-					orgFQDNStr = strings.Replace(orgFQDNStr, `https://`, `http://`, -1)
-				}
-				orgURI, orgWarns, err := getOriginURI(orgFQDNStr)
-				warnings = append(warnings, orgWarns...)
-				if err != nil {
-					warnings = append(warnings, "DS '"+*ds.XMLID+"' had malformed origin  URI: '"+*ds.OrgServerFQDN+"': skipping!"+err.Error())
-					continue
-				}
+			// check if topology already exists
+			nameTopologies[TopologyName(topoName)] = topo
+			ds.Topology = util.StrPtr(topoName)
+		}
 
-				pasvc.Comment = makeParentComment(opt.AddComments, *ds.XMLID, "")
-
-				// TODO encode this in a DSType func, IsGoDirect() ?
-				if *ds.Type == tc.DSTypeHTTPNoCache || *ds.Type == tc.DSTypeHTTPLive || *ds.Type == tc.DSTypeDNSLive {
-					pasvc.DestDomain = orgURI.Hostname()
-					pasvc.Port, err = strconv.Atoi(orgURI.Port())
-					if err != nil {
-						if strings.ToLower(orgURI.Scheme) == "https" {
-							pasvc.Port = 443
-						} else {
-							pasvc.Port = 80
-						}
-						warnings = append(warnings, "DS '"+*ds.XMLID+"' had malformed origin  port: '"+orgURI.Port()+"': using "+strconv.Itoa(pasvc.Port)+"! : "+err.Error())
-					}
-
-					pasvc.GoDirect = true
-
-					pasvc.Parents = []*ParentAbstractionServiceParent{&ParentAbstractionServiceParent{
-						FQDN:   pasvc.DestDomain,
-						Port:   pasvc.Port,
-						Weight: 1.0,
-					}}
-
-					// text += `dest_domain=` + orgURI.Hostname() + ` port=` + orgURI.Port() + ` go_direct=true` + "\n"
-				} else {
-
-					// check for profile psel.qstring_handling.  If this parameter is assigned to the server profile,
-					// then edges will use the qstring handling value specified in the parameter for all profiles.
-
-					// If there is no defined parameter in the profile, then check the delivery service profile.
-					// If psel.qstring_handling exists in the DS profile, then we use that value for the specified DS only.
-					// This is used only if not overridden by a server profile qstring handling parameter.
-
-					// TODO refactor this logic, hard to understand (transliterated from Perl)
-					dsQSH := queryStringHandling
-					if dsQSH == nil {
-						dsQSH = ParentSelectParamQStringHandlingToBool(dsParams.QueryStringHandling)
-						if dsQSH == nil && dsParams.QueryStringHandling != "" {
-							warnings = append(warnings, "Delivery Service parameter '"+ParentConfigParamQStringHandling+"' value '"+dsParams.QueryStringHandling+"' malformed, not using!")
-						}
-
-					}
-					parentQStr := dsQSH
-					if parentQStr == nil {
-						v := false
-						parentQStr = &v
-					}
-					if ds.QStringIgnore != nil && tc.QStringIgnore(*ds.QStringIgnore) == tc.QStringIgnoreUseInCacheKeyAndPassUp && dsQSH == nil {
-						v := true
-						parentQStr = &v
-					}
-					if parentQStr == nil {
-						b := !DefaultIgnoreQueryStringInParentSelection
-						parentQStr = &b
-					}
-
-					pasvc.DestDomain = orgURI.Hostname()
-					pasvc.Port, err = strconv.Atoi(orgURI.Port())
-					if err != nil {
-						if strings.ToLower(orgURI.Scheme) == "https" {
-							pasvc.Port = 443
-						} else {
-							pasvc.Port = 80
-						}
-						warnings = append(warnings, "DS '"+*ds.XMLID+"' had malformed origin  port: '"+orgURI.Port()+"': using "+strconv.Itoa(pasvc.Port)+"! : "+err.Error())
-					}
-					pasvc.Parents = parents
-					pasvc.SecondaryParents = secondaryParents
-					pasvc.SecondaryMode = secondaryMode
-					pasvc.RetryPolicy = roundRobin
-					pasvc.GoDirect = goDirect
-					pasvc.IgnoreQueryStringInParentSelection = !*parentQStr
-					// text += `dest_domain=` + orgURI.Hostname() + ` port=` + orgURI.Port() + ` ` + parents + ` ` + secondaryParents + ` ` + roundRobin + ` ` + goDirect + ` qstring=` + parentQStr + "\n"
-				}
+		isMSO := ds.MultiSiteOrigin != nil && *ds.MultiSiteOrigin
 
-				prWarns := dsParams.FillParentSvcRetries(cacheIsTopLevel, atsMajorVersion, pasvc)
-				warnings = append(warnings, prWarns...)
+		pasvc, topoWarnings, err := getTopologyParentConfigLine(
+			server,
+			serversWithParams,
+			&ds,
+			serverParams,
+			parentConfigParams,
+			nameTopologies,
+			serverCapabilities,
+			dsRequiredCapabilities,
+			cacheGroups,
+			profileParentConfigParams,
+			isMSO,
+			atsMajorVersion,
+			dsOrigins[DeliveryServiceID(*ds.ID)],
+			opt.AddComments,
+		)
+		warnings = append(warnings, topoWarnings...)
+		if err != nil {
+			// we don't want to fail generation with an error if one ds is malformed
+			warnings = append(warnings, err.Error()) // getTopologyParentConfigLine includes error context
+			continue
+		}
 
-				parentAbstraction.Services = append(parentAbstraction.Services, pasvc)
-			}
+		if pasvc != nil { // will be nil with no error if this server isn't in the Topology, or if it doesn't have the Required Capabilities
+			parentAbstraction.Services = append(parentAbstraction.Services, pasvc)
 		}
 	}
 
@@ -701,6 +575,7 @@ type parentInfo struct {
 	UseIP           bool
 	Rank            int
 	IP              string
+	Cachegroup      string
 	PrimaryParent   bool
 	SecondaryParent bool
 	Capabilities    map[ServerCapability]struct{}
@@ -732,6 +607,24 @@ func (p parentInfo) ToAbstract() *ParentAbstractionServiceParent {
 
 type parentInfos map[OriginHost]parentInfo
 
+// Returns a map of parent cache groups names per origin host.
+func makeOCGMap(opis map[OriginHost][]parentInfo) map[OriginHost][]string {
+	ocgnames := map[OriginHost][]string{}
+
+	for host, pis := range opis {
+		cgnames := make(map[string]struct{})
+		for _, pi := range pis {
+			cgnames[string(pi.Cachegroup)] = struct{}{}
+		}
+
+		for cg, _ := range cgnames {
+			ocgnames[host] = append(ocgnames[host], cg)
+		}
+	}
+
+	return ocgnames
+}
+
 type parentInfoSortByRank []parentInfo
 
 func (s parentInfoSortByRank) Len() int      { return len(s) }
@@ -1012,6 +905,7 @@ func getTopologyParentConfigLine(
 	addComments bool,
 ) (*ParentAbstractionService, []string, error) {
 	warnings := []string{}
+
 	if !hasRequiredCapabilities(serverCapabilities[*server.ID], dsRequiredCapabilities[*ds.ID]) {
 		return nil, warnings, nil
 	}
@@ -1391,6 +1285,7 @@ func getTopologyParents(
 	warnings := []string{}
 	// If it's the last tier, then the parent is the origin.
 	// Note this doesn't include MSO, whose final tier cachegroup points to the origin cachegroup.
+
 	if serverIsLastTier {
 		orgURI, orgWarns, err := getOriginURI(*ds.OrgServerFQDN) // TODO pass, instead of calling again
 		warnings = append(warnings, orgWarns...)
@@ -1699,6 +1594,7 @@ func makeParentInfo(
 				UseIP:           sv.Params.UseIP,
 				Rank:            sv.Params.Rank,
 				IP:              ipAddr.String(),
+				Cachegroup:      *sv.Cachegroup,
 				PrimaryParent:   serverParentCGData.ParentID == *sv.CachegroupID,
 				SecondaryParent: serverParentCGData.SecondaryParentID == *sv.CachegroupID,
 				Capabilities:    serverCapabilities[*sv.ID],
@@ -1706,6 +1602,7 @@ func makeParentInfo(
 			if parentInf.Port < 1 {
 				parentInf.Port = *sv.TCPPort
 			}
+
 			parentInfos[originHost] = append(parentInfos[originHost], parentInf)
 		}
 	}
diff --git a/lib/go-atscfg/parentdotconfig_test.go b/lib/go-atscfg/parentdotconfig_test.go
index 6ed2122816..aa14565fe8 100644
--- a/lib/go-atscfg/parentdotconfig_test.go
+++ b/lib/go-atscfg/parentdotconfig_test.go
@@ -214,8 +214,9 @@ func TestMakeParentDotConfigCapabilities(t *testing.T) {
 
 	topologies := []tc.Topology{}
 	serverCapabilities := map[int]map[ServerCapability]struct{}{
-		*mid1.ID: map[ServerCapability]struct{}{"FOO": {}},
-		*mid2.ID: map[ServerCapability]struct{}{"FOO": {}, "BAR": {}},
+		*server.ID: map[ServerCapability]struct{}{"FOO": {}},
+		*mid1.ID:   map[ServerCapability]struct{}{"FOO": {}},
+		*mid2.ID:   map[ServerCapability]struct{}{"FOO": {}, "BAR": {}},
 	}
 	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{
 		*ds0.ID: map[ServerCapability]struct{}{"FOO": {}},
@@ -1746,11 +1747,11 @@ func TestMakeParentDotConfigMSOWithCapabilities(t *testing.T) {
 		},
 	}
 
-	server := makeTestParentServer()
-	server.Cachegroup = util.StrPtr("midCG")
-	server.Type = "MID"
-	server.CachegroupID = util.IntPtr(400)
-	server.ID = util.IntPtr(44)
+	mid := makeTestParentServer()
+	mid.Cachegroup = util.StrPtr("midCG")
+	mid.Type = "MID"
+	mid.CachegroupID = util.IntPtr(400)
+	mid.ID = util.IntPtr(44)
 
 	origin0 := makeTestParentServer()
 	origin0.Cachegroup = util.StrPtr("originCG")
@@ -1770,10 +1771,10 @@ func TestMakeParentDotConfigMSOWithCapabilities(t *testing.T) {
 	origin1.Type = tc.OriginTypeName
 	origin1.TypeID = util.IntPtr(991)
 
-	servers := []Server{*server, *origin0, *origin1}
+	servers := []Server{*mid, *origin0, *origin1}
 
 	serverCapabilities := map[int]map[ServerCapability]struct{}{
-		*server.ID: {
+		*mid.ID: {
 			ServerCapability("FOO"): struct{}{},
 		},
 	}
@@ -1784,8 +1785,8 @@ func TestMakeParentDotConfigMSOWithCapabilities(t *testing.T) {
 	}
 
 	midCG := &tc.CacheGroupNullable{}
-	midCG.Name = server.Cachegroup
-	midCG.ID = server.CachegroupID
+	midCG.Name = mid.Cachegroup
+	midCG.ID = mid.CachegroupID
 	midCG.ParentName = origin0.Cachegroup
 	midCG.ParentCachegroupID = origin0.CachegroupID
 	midCGType := tc.CacheGroupMidTypeName
@@ -1811,7 +1812,7 @@ func TestMakeParentDotConfigMSOWithCapabilities(t *testing.T) {
 	}
 	topologies := []tc.Topology{}
 
-	cfg, err := MakeParentDotConfig(dses, server, servers, topologies, serverParams, parentConfigParams, serverCapabilities, dsRequiredCapabilities, cgs, dss, cdn, hdr)
+	cfg, err := MakeParentDotConfig(dses, mid, servers, topologies, serverParams, parentConfigParams, serverCapabilities, dsRequiredCapabilities, cgs, dss, cdn, hdr)
 	if err != nil {
 		t.Fatal(err)
 	}