You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by mi...@apache.org on 2020/08/06 16:59:36 UTC

[trafficcontrol] branch master updated: Add Topology ATS Config Generation (#4790)

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

mitchell852 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 dababd8  Add Topology ATS Config Generation (#4790)
dababd8 is described below

commit dababd866a47777dedecd4c0f05cb5a112d57763
Author: Robert O Butts <ro...@users.noreply.github.com>
AuthorDate: Thu Aug 6 10:59:23 2020 -0600

    Add Topology ATS Config Generation (#4790)
    
    * Add ORT inferring location params
    
    Adds ORT location param inference. If location Parameters do not exist
    the files are created and added anyway with the directory determined
    from the local ATS install, for:
    - cache.config
    - hosting.config
    - ip_allow.config
    - parent.config
    - plugin.config
    - records.config
    - remap.config
    - storage.config
    - volume.config
    - Delivery Services with
      - Edge Header Rewrite
      - Mid Header Rewrite,
      - Cache URL
      - URL Sig
      - URI Signing
    
    Note this is not an exhaustive of required files, and many files
    must be dynamic and cannot be inferred.
    
    But this does remove the manual configuration for this list.
    More files may be added in the future.
    
    * Refactor ConfigGen parent.config to reduce code
    
    Refactors parent.config generation to have a single DS for-loop,
    instead of different loops for top-level vs non-top-level caches.
    
    This is to prepare for Topologies, which will need their own
    behavior, irrespective of "top level" Edge vs Mid, for every DS.
    
    * Add ORT atstccfg Topology support
    
    * Fix code broken from changed TO Client functions
    
    * Fix log typo
    
    Also adds a TODO to propogate, once config gen returns errors.
    
    * Removed commented code
    
    * Remove unused code, add topology name to comment
    
    * Add ORT Topology Header Rewrites
    
    * Add ORT atstccfg error log for parentless topology
    
    * Add ORT atstccfg new req for update_status
    
    Necessary because the latest API considers Topologies, previous API
    doesn't.
    
    * Fix tests for atscfg topology/capability change
    
    * Change ORT/atstccfg to break outer loop
    
    Optimization.
    
    * Change ORT atscfg variable name
    
    Per code review.
    
    * Change ORT atscfg to use topology map
    
    More performant for large numbers of topologies. Per PR review.
---
 CHANGELOG.md                                       |   2 +
 lib/go-atscfg/atscfg.go                            | 160 ++++
 lib/go-atscfg/hostingdotconfig.go                  |  27 +-
 lib/go-atscfg/hostingdotconfig_test.go             |  72 +-
 lib/go-atscfg/meta.go                              |  88 ++-
 lib/go-atscfg/meta_test.go                         |   7 +-
 lib/go-atscfg/parentdotconfig.go                   | 413 ++++++++--
 lib/go-atscfg/parentdotconfig_test.go              | 575 +++++++++++++-
 lib/go-atscfg/remapdotconfig.go                    | 118 ++-
 lib/go-atscfg/remapdotconfig_test.go               | 834 ++++++++++++++++++---
 lib/go-atscfg/topologyheaderrewritedotconfig.go    | 135 ++++
 .../ats/atsserver/hostingdotconfig.go              |  49 +-
 .../traffic_ops_golang/ats/atsserver/meta.go       |  15 +-
 traffic_ops/traffic_ops_golang/ats/db.go           | 113 +++
 traffic_ops_ort/atstccfg/cfgfile/cachedotconfig.go |   2 +-
 .../atstccfg/cfgfile/cacheurldotconfig.go          |   2 +-
 traffic_ops_ort/atstccfg/cfgfile/cfgfile.go        |  93 ++-
 .../atstccfg/cfgfile/headerrewritedotconfig.go     |   2 +-
 .../atstccfg/cfgfile/headerrewritemiddotconfig.go  |  13 -
 .../atstccfg/cfgfile/hostingdotconfig.go           |  67 +-
 traffic_ops_ort/atstccfg/cfgfile/meta.go           |   5 +-
 .../atstccfg/cfgfile/parentdotconfig.go            |  17 +-
 traffic_ops_ort/atstccfg/cfgfile/remapdotconfig.go |  28 +-
 traffic_ops_ort/atstccfg/cfgfile/routing.go        |   8 +-
 .../cfgfile/topologyheaderrewritedotconfig.go      |  61 ++
 traffic_ops_ort/atstccfg/config/config.go          |   4 +
 traffic_ops_ort/atstccfg/getdata/getdata.go        |   6 +-
 traffic_ops_ort/atstccfg/toreqnew/toreqnew.go      |  62 +-
 28 files changed, 2595 insertions(+), 383 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 74f240b..2f54d81 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -15,6 +15,7 @@ The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/).
     - Traffic Ops: Added new topology-based delivery service fields for header rewrites: `firstHeaderRewrite`, `innerHeaderRewrite`, `lastHeaderRewrite`
     - Traffic Ops: Added validation to prohibit assigning caches to topology-based delivery services
     - Traffic Ops: Consider Topologies parentage when queueing or checking server updates
+    - ORT: Added Topologies to Config Generation.
     - Traffic Portal: Added the ability to create, read, update and delete flexible topologies.
     - Traffic Portal: Added the ability to assign topologies to delivery services.
     - Traffic Portal: Added the ability to view all delivery services and cache groups associated with a topology.
@@ -54,6 +55,7 @@ The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/).
 - Changed some Traffic Ops Go Client methods to use `DeliveryServiceNullable` inputs and outputs.
 - Changed Traffic Portal to use Traffic Ops API v3
 - Changed ORT Config Generation to be deterministic, which will prevent spurious diffs when nothing actually changed.
+- Changed ORT to find the local ATS config directory and use it when location Parameters don't exist for many required configs, including all Delivery Service files (Header Rewrites, Regex Remap, URL Sig, URI Signing).
 - Changed the access logs in Traffic Ops to now show the route ID with every API endpoint call. The Route ID is appended to the end of the access log line.
 - [Multiple Interface Servers](https://github.com/apache/trafficcontrol/blob/master/blueprints/multi-interface-servers.md)
     - Interface data is constructed from IP Address/Gateway/Netmask (and their IPv6 counterparts) and Interface Name and Interface MTU fields on services. These **MUST** have proper, valid data before attempting to upgrade or the upgrade **WILL** fail. In particular IP fields need to be valid IP addresses/netmasks, and MTU must only be positive integers of at least 1280.
diff --git a/lib/go-atscfg/atscfg.go b/lib/go-atscfg/atscfg.go
index 8798ed3..33a572a 100644
--- a/lib/go-atscfg/atscfg.go
+++ b/lib/go-atscfg/atscfg.go
@@ -20,12 +20,14 @@ package atscfg
  */
 
 import (
+	"encoding/json"
 	"errors"
 	"sort"
 	"strconv"
 	"strings"
 	"time"
 
+	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
@@ -39,10 +41,13 @@ const ContentTypeTextASCII = `text/plain; charset=us-ascii`
 
 const LineCommentHash = "#"
 
+type TopologyName string
+
 type ServerCapability string
 
 type ServerInfo struct {
 	CacheGroupID                  int
+	CacheGroupName                string
 	CDN                           tc.CDNName
 	CDNID                         int
 	DomainName                    string
@@ -134,3 +139,158 @@ const ConfigSuffix = ".config"
 func GetConfigFile(prefix string, xmlId string) string {
 	return prefix + xmlId + ConfigSuffix
 }
+
+// topologyIncludesServer returns whether the given topology includes the given server
+func topologyIncludesServer(topology tc.Topology, server tc.Server) bool {
+	for _, node := range topology.Nodes {
+		if node.Cachegroup == server.Cachegroup {
+			return true
+		}
+	}
+	return false
+}
+
+// TopologyCacheTier is the position of a cache in the topology.
+// Note this is the cache tier itself, notwithstanding MSO. So for an MSO service,
+// Caches immediately before the origin are the TopologyCacheTierLast, even for MSO.
+type TopologyCacheTier string
+
+const (
+	TopologyCacheTierFirst   = TopologyCacheTier("first")
+	TopologyCacheTierInner   = TopologyCacheTier("inner")
+	TopologyCacheTierLast    = TopologyCacheTier("last")
+	TopologyCacheTierInvalid = TopologyCacheTier("")
+)
+
+// TopologyPlacement contains data about the placement of a server in a topology.
+type TopologyPlacement struct {
+	// InTopology is whether the server is in the topology at all.
+	InTopology bool
+	// IsLastTier is whether the server is the last tier in the topology.
+	// Note this is different for MSO vs non-MSO. For MSO, the last tier is the Origin. For non-MSO, the last tier is the last cache tier.
+	IsLastTier bool
+	// CacheTier is the position of the cache in the topology.
+	// Note this is whether the cache is the last cache, even if it has parents in the topology who are origins (MSO).
+	// Thus, it's possible for a server to be CacheTierLast and not IsLastTier.
+	CacheTier TopologyCacheTier
+}
+
+// getTopologyPlacement returns information about the cachegroup's placement in the topology.
+// - Whether the cachegroup is the last tier in the topology.
+// - Whether the cachegroup is in the topology at all.
+// - Whether it's the first, inner, or last cache tier before the Origin.
+func getTopologyPlacement(cacheGroup tc.CacheGroupName, topology tc.Topology, cacheGroups map[tc.CacheGroupName]tc.CacheGroupNullable) TopologyPlacement {
+	serverNode := tc.TopologyNode{}
+	serverNodeIndex := -1
+	for nodeI, node := range topology.Nodes {
+		if node.Cachegroup == string(cacheGroup) {
+			serverNode = node
+			serverNodeIndex = nodeI
+			break
+		}
+	}
+	if serverNode.Cachegroup == "" {
+		return TopologyPlacement{InTopology: false}
+	}
+
+	topologyNodeHasChildren := false
+nodeFor:
+	for _, node := range topology.Nodes {
+		for _, parent := range node.Parents {
+			if parent == serverNodeIndex {
+				topologyNodeHasChildren = true
+				break nodeFor
+			}
+		}
+	}
+
+	cacheTier := TopologyCacheTierFirst
+	if topologyNodeHasChildren {
+		cacheTier = TopologyCacheTierInner
+	}
+
+	isLastTier := len(serverNode.Parents) == 0
+
+	if isLastTier {
+		cacheTier = TopologyCacheTierLast
+	}
+	// Check if the parent is an Origin, and if so, set to Last
+	if cacheTier == TopologyCacheTierInner {
+		// TODO extra safety: check other parents, and warn if parents have different types?
+		parentI := serverNode.Parents[0]
+		if parentI >= len(topology.Nodes) {
+			log.Errorln("ATS config generation: topology '" + topology.Name + "' has node with parent larger than nodes size! Config Generation will be malformed!")
+		} else {
+			parentNode := topology.Nodes[parentI]
+			cg, ok := cacheGroups[tc.CacheGroupName(parentNode.Cachegroup)]
+			if !ok {
+				log.Errorln("ATS config generation: topology '" + topology.Name + "' has node with cachegroup '" + parentNode.Cachegroup + "' that wasn't found in cachegroups! Config Generation will be malformed!")
+			} else if cg.Type == nil {
+				log.Errorln("ATS config generation: cachegroup '" + parentNode.Cachegroup + "' with nil type! Config Generation will be malformed!")
+			} else if *cg.Type == tc.CacheGroupOriginTypeName {
+				// this server's parent in the topology is an Origin, so this server is the last cache tier.
+				cacheTier = TopologyCacheTierLast
+			}
+		}
+	}
+	return TopologyPlacement{InTopology: true, IsLastTier: isLastTier, CacheTier: cacheTier}
+}
+
+func MakeTopologyNameMap(topologies []tc.Topology) map[TopologyName]tc.Topology {
+	topoNames := map[TopologyName]tc.Topology{}
+	for _, to := range topologies {
+		topoNames[TopologyName(to.Name)] = to
+	}
+	return topoNames
+}
+
+func MakeCGMap(cgs []tc.CacheGroupNullable) map[tc.CacheGroupName]tc.CacheGroupNullable {
+	cgMap := map[tc.CacheGroupName]tc.CacheGroupNullable{}
+	for _, cg := range cgs {
+		if cg.Name == nil {
+			log.Errorln("ATS config generation: got cachegroup with nil name, skipping!")
+			continue
+		}
+		cgMap[tc.CacheGroupName(*cg.Name)] = cg
+	}
+	return cgMap
+}
+
+type ParameterWithProfiles struct {
+	tc.Parameter
+	ProfileNames []string
+}
+
+type ParameterWithProfilesMap struct {
+	tc.Parameter
+	ProfileNames map[string]struct{}
+}
+
+// TCParamsToParamsWithProfiles unmarshals the Profiles that the tc struct doesn't.
+func TCParamsToParamsWithProfiles(tcParams []tc.Parameter) ([]ParameterWithProfiles, error) {
+	params := make([]ParameterWithProfiles, 0, len(tcParams))
+	for _, tcParam := range tcParams {
+		param := ParameterWithProfiles{Parameter: tcParam}
+
+		profiles := []string{}
+		if err := json.Unmarshal(tcParam.Profiles, &profiles); err != nil {
+			return nil, errors.New("unmarshalling JSON from parameter '" + strconv.Itoa(param.ID) + "': " + err.Error())
+		}
+		param.ProfileNames = profiles
+		param.Profiles = nil
+		params = append(params, param)
+	}
+	return params, nil
+}
+
+func ParameterWithProfilesToMap(tcParams []ParameterWithProfiles) []ParameterWithProfilesMap {
+	params := []ParameterWithProfilesMap{}
+	for _, tcParam := range tcParams {
+		param := ParameterWithProfilesMap{Parameter: tcParam.Parameter, ProfileNames: map[string]struct{}{}}
+		for _, profile := range tcParam.ProfileNames {
+			param.ProfileNames[profile] = struct{}{}
+		}
+		params = append(params, param)
+	}
+	return params
+}
diff --git a/lib/go-atscfg/hostingdotconfig.go b/lib/go-atscfg/hostingdotconfig.go
index 5df86e9..f2d57ef 100644
--- a/lib/go-atscfg/hostingdotconfig.go
+++ b/lib/go-atscfg/hostingdotconfig.go
@@ -35,14 +35,20 @@ const LineCommentHostingDotConfig = LineCommentHash
 const ParamDrivePrefix = "Drive_Prefix"
 const ParamRAMDrivePrefix = "RAM_Drive_Prefix"
 
+const ServerHostingDotConfigMidIncludeInactive = false
+const ServerHostingDotConfigEdgeIncludeInactive = true
+
 func MakeHostingDotConfig(
-	serverName tc.CacheName,
+	server tc.Server,
 	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
 	toURL string, // tm.url global parameter (TODO: cache itself?)
 	params map[string]string, // map[name]value - config file should always be storage.config
-	origins []string, // origins of delivery services assigned to this server. Should only include LIVE and LIVE_NATNL for Edges, and only LIVE_NATNL for Mids.
+	dses []tc.DeliveryServiceNullable,
+	topologies []tc.Topology,
 ) string {
-	text := GenericHeaderComment(string(serverName), toToolName, toURL)
+	text := GenericHeaderComment(server.HostName, toToolName, toURL)
+
+	nameTopologies := MakeTopologyNameMap(topologies)
 
 	lines := []string{}
 	if _, ok := params[ParamRAMDrivePrefix]; ok {
@@ -56,10 +62,23 @@ func MakeHostingDotConfig(
 		text += `# TRAFFIC OPS NOTE: volume ` + strconv.Itoa(ramVolume) + ` is the RAM volume` + "\n"
 
 		seenOrigins := map[string]struct{}{}
-		for _, origin := range origins {
+		for _, ds := range dses {
+			if ds.OrgServerFQDN == nil || ds.XMLID == nil || ds.Active == nil {
+				continue // TODO warn?
+			}
+
+			origin := *ds.OrgServerFQDN
 			if _, ok := seenOrigins[origin]; ok {
 				continue
 			}
+
+			if ds.Topology != nil && *ds.Topology != "" {
+				topology, hasTopology := nameTopologies[TopologyName(*ds.Topology)]
+				if hasTopology && !topologyIncludesServer(topology, server) {
+					continue
+				}
+			}
+
 			seenOrigins[origin] = struct{}{}
 			origin = strings.TrimPrefix(origin, `http://`)
 			origin = strings.TrimPrefix(origin, `https://`)
diff --git a/lib/go-atscfg/hostingdotconfig_test.go b/lib/go-atscfg/hostingdotconfig_test.go
index d8b2a48..e1b309e 100644
--- a/lib/go-atscfg/hostingdotconfig_test.go
+++ b/lib/go-atscfg/hostingdotconfig_test.go
@@ -24,10 +24,11 @@ import (
 	"testing"
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-util"
 )
 
 func TestMakeHostingDotConfig(t *testing.T) {
-	serverName := tc.CacheName("server0")
+	server := tc.Server{HostName: "server0"}
 	toToolName := "to0"
 	toURL := "trafficops.example.net"
 	params := map[string]string{
@@ -43,8 +44,14 @@ func TestMakeHostingDotConfig(t *testing.T) {
 		"https://origin4.example.net/",
 		"http://origin5.example.net/",
 	}
+	dses := []tc.DeliveryServiceNullable{}
+	for _, origin := range origins {
+		ds := tc.DeliveryServiceNullable{}
+		ds.OrgServerFQDN = util.StrPtr(origin)
+		dses = append(dses, ds)
+	}
 
-	txt := MakeHostingDotConfig(serverName, toToolName, toURL, params, origins)
+	txt := MakeHostingDotConfig(server, toToolName, toURL, params, dses, nil)
 
 	lines := strings.Split(txt, "\n")
 
@@ -84,6 +91,67 @@ func TestMakeHostingDotConfig(t *testing.T) {
 	}
 }
 
+func TestMakeHostingDotConfigTopologiesIgnoreDSS(t *testing.T) {
+	server := tc.Server{HostName: "server0", Cachegroup: "edgeCG"}
+	toToolName := "to0"
+	toURL := "trafficops.example.net"
+	params := map[string]string{
+		ParamRAMDrivePrefix: "ParamRAMDrivePrefix-shouldnotappearinconfig",
+		ParamDrivePrefix:    "ParamDrivePrefix-shouldnotappearinconfig",
+		"somethingelse":     "somethingelse-shouldnotappearinconfig",
+	}
+
+	dsTopology := tc.DeliveryServiceNullable{}
+	dsTopology.OrgServerFQDN = util.StrPtr("https://origin0.example.net")
+	dsTopology.XMLID = util.StrPtr("ds-topology")
+	dsTopology.Topology = util.StrPtr("t0")
+	dsTopology.Active = util.BoolPtr(true)
+
+	dsTopologyWithoutServer := tc.DeliveryServiceNullable{}
+	dsTopologyWithoutServer.OrgServerFQDN = util.StrPtr("https://origin1.example.net")
+	dsTopologyWithoutServer.XMLID = util.StrPtr("ds-topology-without-server")
+	dsTopologyWithoutServer.Topology = util.StrPtr("t1")
+	dsTopologyWithoutServer.Active = util.BoolPtr(true)
+
+	dses := []tc.DeliveryServiceNullable{dsTopology, dsTopologyWithoutServer}
+
+	topologies := []tc.Topology{
+		tc.Topology{
+			Name: "t0",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "edgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+		tc.Topology{
+			Name: "t1",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "otherEdgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+	}
+
+	txt := MakeHostingDotConfig(server, toToolName, toURL, params, dses, topologies)
+
+	if !strings.Contains(txt, "origin0") {
+		t.Errorf("expected origin0 in topology, actual %v\n", txt)
+	}
+	if strings.Contains(txt, "origin1") {
+		t.Errorf("expected no origin1 not in topology, actual %v\n", txt)
+	}
+}
+
 func strArrContainsSubstr(arr []string, substr string) bool {
 	for _, as := range arr {
 		if strings.Contains(as, substr) {
diff --git a/lib/go-atscfg/meta.go b/lib/go-atscfg/meta.go
index fd80993..cd23d76 100644
--- a/lib/go-atscfg/meta.go
+++ b/lib/go-atscfg/meta.go
@@ -66,15 +66,15 @@ func MakeMetaConfig(
 	locationParams map[string]ConfigProfileParams, // map[configFile]params; 'location' and 'URL' Parameters on serverHostName's Profile
 	uriSignedDSes []tc.DeliveryServiceName,
 	scopeParams map[string]string, // map[configFileName]scopeParam
-	dsNames map[tc.DeliveryServiceName]struct{},
+	dses map[tc.DeliveryServiceName]tc.DeliveryServiceNullable,
+	cacheGroupArr []tc.CacheGroupNullable,
+	topologies []tc.Topology,
 ) string {
-	// dses are only used when configDir is not empty
-	dses := map[tc.DeliveryServiceName]tc.DeliveryServiceNullable{}
-	for dsName, _ := range dsNames {
-		dses[dsName] = tc.DeliveryServiceNullable{}
-	}
-	configDir := ""
-	atsData, err := MakeMetaObj(serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, configDir)
+	configDir := "" // this should only be used for Traffic Ops, which doesn't have a local ATS install config directory (and thus will fail if any location Parameters are missing or relative).
+	return MetaObjToMetaConfig(MakeMetaObj(serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cacheGroupArr, topologies, configDir))
+}
+
+func MetaObjToMetaConfig(atsData tc.ATSConfigMetaData, err error) string {
 	if err != nil {
 		return "error creating meta config: " + err.Error()
 	}
@@ -102,7 +102,10 @@ func AddMetaObjConfigDir(
 	uriSignedDSes []tc.DeliveryServiceName,
 	scopeParams map[string]string, // map[configFileName]scopeParam
 	dses map[tc.DeliveryServiceName]tc.DeliveryServiceNullable,
+	cacheGroupArr []tc.CacheGroupNullable,
+	topologies []tc.Topology,
 ) (tc.ATSConfigMetaData, error) {
+	cacheGroups := MakeCGMap(cacheGroupArr)
 
 	// Note there may be multiple files with the same name in different directories.
 	configFilesM := map[string][]tc.ATSConfigMetaDataConfigFile{} // map[fileShortName]tc.ATSConfigMetaDataConfigFile
@@ -143,29 +146,65 @@ func AddMetaObjConfigDir(
 		configFilesM[fileName] = newFis
 	}
 
+	nameTopologies := MakeTopologyNameMap(topologies)
+
 	for _, ds := range dses {
 		if ds.XMLID == nil {
 			log.Errorln("meta config generation got Delivery Service with nil XMLID - not considering!")
 			continue
 		}
+
 		err := error(nil)
 		// Note we log errors, but don't return them.
 		// If an individual DS has an error, we don't want to break the rest of the CDN.
-		if (ds.EdgeHeaderRewrite != nil || ds.MaxOriginConnections != nil) &&
-			strings.HasPrefix(server.Type, tc.EdgeTypePrefix) {
-			fileName := "hdr_rw_" + *ds.XMLID + ".config"
-			scope := tc.ATSConfigMetaDataConfigFileScopeCDNs
-			if configFilesM, err = ensureConfigFile(configFilesM, fileName, configDir, scope); err != nil {
-				log.Errorln("meta config generation: " + err.Error())
+		if ds.Topology != nil && *ds.Topology != "" {
+			topology := nameTopologies[TopologyName(*ds.Topology)]
+
+			placement := getTopologyPlacement(tc.CacheGroupName(server.CacheGroupName), topology, cacheGroups)
+			switch placement.CacheTier {
+			case TopologyCacheTierFirst:
+				if ds.FirstHeaderRewrite != nil && *ds.FirstHeaderRewrite != "" || ds.MaxOriginConnections != nil {
+					fileName := FirstHeaderRewriteConfigFileName(*ds.XMLID)
+					scope := tc.ATSConfigMetaDataConfigFileScopeServers
+					if configFilesM, err = ensureConfigFile(configFilesM, fileName, configDir, scope); err != nil {
+						log.Errorln("meta config generation: " + err.Error())
+					}
+				}
+			case TopologyCacheTierInner:
+				if ds.InnerHeaderRewrite != nil && *ds.InnerHeaderRewrite != "" || ds.MaxOriginConnections != nil {
+					fileName := InnerHeaderRewriteConfigFileName(*ds.XMLID)
+					scope := tc.ATSConfigMetaDataConfigFileScopeServers
+					if configFilesM, err = ensureConfigFile(configFilesM, fileName, configDir, scope); err != nil {
+						log.Errorln("meta config generation: " + err.Error())
+					}
+				}
+			case TopologyCacheTierLast:
+				if ds.LastHeaderRewrite != nil && *ds.LastHeaderRewrite != "" || ds.MaxOriginConnections != nil {
+					fileName := LastHeaderRewriteConfigFileName(*ds.XMLID)
+					scope := tc.ATSConfigMetaDataConfigFileScopeServers
+					if configFilesM, err = ensureConfigFile(configFilesM, fileName, configDir, scope); err != nil {
+						log.Errorln("meta config generation: " + err.Error())
+					}
+				}
 			}
-		}
-		if (ds.MidHeaderRewrite != nil || ds.MaxOriginConnections != nil) &&
-			ds.Type != nil && ds.Type.UsesMidCache() &&
-			strings.HasPrefix(server.Type, tc.MidTypePrefix) {
-			fileName := "hdr_rw_mid_" + *ds.XMLID + ".config"
-			scope := tc.ATSConfigMetaDataConfigFileScopeCDNs
-			if configFilesM, err = ensureConfigFile(configFilesM, fileName, configDir, scope); err != nil {
-				log.Errorln("meta config generation: " + err.Error())
+		} else if strings.HasPrefix(server.Type, tc.EdgeTypePrefix) {
+			if (ds.EdgeHeaderRewrite != nil || ds.MaxOriginConnections != nil) &&
+				strings.HasPrefix(server.Type, tc.EdgeTypePrefix) {
+				fileName := "hdr_rw_" + *ds.XMLID + ".config"
+				scope := tc.ATSConfigMetaDataConfigFileScopeCDNs
+				if configFilesM, err = ensureConfigFile(configFilesM, fileName, configDir, scope); err != nil {
+					log.Errorln("meta config generation: " + err.Error())
+				}
+			}
+		} else if strings.HasPrefix(server.Type, tc.MidTypePrefix) {
+			if (ds.MidHeaderRewrite != nil || ds.MaxOriginConnections != nil) &&
+				ds.Type != nil && ds.Type.UsesMidCache() &&
+				strings.HasPrefix(server.Type, tc.MidTypePrefix) {
+				fileName := "hdr_rw_mid_" + *ds.XMLID + ".config"
+				scope := tc.ATSConfigMetaDataConfigFileScopeCDNs
+				if configFilesM, err = ensureConfigFile(configFilesM, fileName, configDir, scope); err != nil {
+					log.Errorln("meta config generation: " + err.Error())
+				}
 			}
 		}
 		if ds.RegexRemap != nil {
@@ -197,7 +236,6 @@ func AddMetaObjConfigDir(
 			}
 		}
 	}
-	// TODO add location params for ds ensure garbage here
 
 	newFiles := []tc.ATSConfigMetaDataConfigFile{}
 	for _, fis := range configFilesM {
@@ -236,6 +274,8 @@ func MakeMetaObj(
 	uriSignedDSes []tc.DeliveryServiceName,
 	scopeParams map[string]string, // map[configFileName]scopeParam
 	dses map[tc.DeliveryServiceName]tc.DeliveryServiceNullable,
+	cacheGroupArr []tc.CacheGroupNullable,
+	topologies []tc.Topology,
 	configDir string,
 ) (tc.ATSConfigMetaData, error) {
 	if tmURL == "" {
@@ -313,7 +353,7 @@ locationParamsFor:
 		atsData.ConfigFiles = append(atsData.ConfigFiles, atsCfg)
 	}
 
-	return AddMetaObjConfigDir(atsData, configDir, serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses)
+	return AddMetaObjConfigDir(atsData, configDir, serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cacheGroupArr, topologies)
 }
 
 func getServerScope(cfgFile string, serverType string, scopeParams map[string]string) tc.ATSConfigMetaDataConfigFileScope {
diff --git a/lib/go-atscfg/meta_test.go b/lib/go-atscfg/meta_test.go
index e735c75..fd5ed97 100644
--- a/lib/go-atscfg/meta_test.go
+++ b/lib/go-atscfg/meta_test.go
@@ -114,8 +114,11 @@ func TestMakeMetaConfig(t *testing.T) {
 
 	scopeParams := map[string]string{"custom.config": string(tc.ATSConfigMetaDataConfigFileScopeProfiles)}
 
+	cgs := []tc.CacheGroupNullable{}
+	topologies := []tc.Topology{}
+
 	cfgPath := "/etc/foo/trafficserver"
-	cfg, err := MakeMetaObj(serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cfgPath)
+	cfg, err := MakeMetaObj(serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cgs, topologies, cfgPath)
 	if err != nil {
 		t.Fatalf("MakeMetaObj: " + err.Error())
 	}
@@ -291,7 +294,7 @@ func TestMakeMetaConfig(t *testing.T) {
 	}
 
 	server.Type = "MID"
-	cfg, err = MakeMetaObj(serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cfgPath)
+	cfg, err = MakeMetaObj(serverHostName, server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cgs, topologies, cfgPath)
 	if err != nil {
 		t.Fatalf("MakeMetaObj: " + err.Error())
 	}
diff --git a/lib/go-atscfg/parentdotconfig.go b/lib/go-atscfg/parentdotconfig.go
index 621cf3e..e631030 100644
--- a/lib/go-atscfg/parentdotconfig.go
+++ b/lib/go-atscfg/parentdotconfig.go
@@ -20,6 +20,7 @@ package atscfg
  */
 
 import (
+	"errors"
 	"net/url"
 	"regexp"
 	"sort"
@@ -59,15 +60,15 @@ const ParentConfigCacheParamNotAParent = "not_a_parent"
 const DeliveryServicesAllParentsKey = "all_parents"
 
 type ParentConfigDS struct {
-	Name            tc.DeliveryServiceName
-	QStringIgnore   tc.QStringIgnore
-	OriginFQDN      string
-	MultiSiteOrigin bool
-	OriginShield    string
-	Type            tc.DSType
-	QStringHandling string
-
+	Name                 tc.DeliveryServiceName
+	QStringIgnore        tc.QStringIgnore
+	OriginFQDN           string
+	MultiSiteOrigin      bool
+	OriginShield         string
+	Type                 tc.DSType
+	QStringHandling      string
 	RequiredCapabilities map[ServerCapability]struct{}
+	Topology             string
 }
 
 type ParentConfigDSTopLevel struct {
@@ -142,18 +143,20 @@ func DefaultProfileCache() ProfileCache {
 
 // CGServer is the server table data needed when selecting the servers assigned to a cachegroup.
 type CGServer struct {
-	ServerID     ServerID
-	ServerHost   string
-	ServerIP     string
-	ServerPort   int
-	CacheGroupID int
-	Status       int
-	Type         int
-	ProfileID    ProfileID
-	CDN          int
-	TypeName     string
-	Domain       string
-	Capabilities map[ServerCapability]struct{}
+	ServerID       ServerID
+	ServerHost     string
+	ServerIP       string
+	ServerPort     int
+	CacheGroupID   int
+	CacheGroupName string
+	Status         int
+	Type           int
+	ProfileID      ProfileID
+	ProfileName    string
+	CDN            int
+	TypeName       string
+	Domain         string
+	Capabilities   map[ServerCapability]struct{}
 }
 
 type OriginURI struct {
@@ -170,45 +173,63 @@ func MakeParentDotConfig(
 	parentConfigDSes []ParentConfigDSTopLevel, // getParentConfigDSTopLevel(cdn) OR getParentConfigDS(server) (TODO determine how to handle non-top missing MSO?)
 	serverParams map[string]string, // getParentConfigServerProfileParams(serverID)
 	parentInfos map[OriginHost][]ParentInfo, // getParentInfo(profileID, parentCachegroupID, secondaryParentCachegroupID)
+	server tc.Server,
+	servers []tc.Server,
+	topologies []tc.Topology,
+	tcParentConfigParams []tc.Parameter,
+	serverCapabilities map[int]map[ServerCapability]struct{},
+	cacheGroupArr []tc.CacheGroupNullable,
 ) string {
+	cacheGroups := MakeCGMap(cacheGroupArr)
+
 	sort.Sort(ParentConfigDSTopLevelSortByName(parentConfigDSes))
 
 	nameVersionStr := GetNameVersionStringFromToolNameAndURL(toToolName, toURL)
 	hdr := HeaderCommentWithTOVersionStr(serverInfo.HostName, nameVersionStr)
 
 	textArr := []string{}
-	text := ""
-	// TODO put these in separate functions. No if-statement should be this long.
-	if serverInfo.IsTopLevelCache() {
-		uniqueOrigins := map[string]struct{}{}
+	processedOriginsToDSNames := map[string]tc.DeliveryServiceName{}
+
+	parentConfigParamsWithProfiles, err := TCParamsToParamsWithProfiles(tcParentConfigParams)
+	if err != nil {
+		log.Errorln("parent.config generation: error getting profiles from Traffic Ops Parameters, Parameters will not be considered for generation! : " + err.Error())
+		parentConfigParamsWithProfiles = []ParameterWithProfiles{}
+	}
+	parentConfigParams := ParameterWithProfilesToMap(parentConfigParamsWithProfiles)
 
-		for _, ds := range parentConfigDSes {
+	nameTopologies := MakeTopologyNameMap(topologies)
+
+	for _, ds := range parentConfigDSes {
+		log.Infoln("parent.config processing ds '" + ds.Name + "'")
+
+		if existingDS, ok := processedOriginsToDSNames[ds.OriginFQDN]; ok {
+			log.Errorln("parent.config generation: duplicate origin! services '" + string(ds.Name) + "' and '" + string(existingDS) + "' share origin '" + ds.OriginFQDN + "': skipping '" + string(ds.Name) + "'!")
+			continue
+		}
+
+		// TODO put these in separate functions. No if-statement should be this long.
+		if ds.Topology != "" {
+			log.Infoln("parent.config generating Topology line for ds '" + ds.Name + "'")
+			txt, err := GetTopologyParentConfigLine(server, servers, ds, serverParams, parentConfigParams, nameTopologies, serverCapabilities, cacheGroups)
+			if err != nil {
+				log.Errorln(err)
+				continue
+			}
+			if txt != "" { // will be empty with no error if this server isn't in the Topology, or if it doesn't have the Required Capabilities
+				textArr = append(textArr, txt)
+			}
+		} else if serverInfo.IsTopLevelCache() {
+			log.Infoln("parent.config generating top level line for ds '" + ds.Name + "'")
 			parentQStr := "ignore"
 			if ds.QStringHandling == "" && ds.MSOAlgorithm == tc.AlgorithmConsistentHash && ds.QStringIgnore == tc.QStringIgnoreUseInCacheKeyAndPassUp {
 				parentQStr = "consider"
 			}
 
-			orgURIStr := ds.OriginFQDN
-			orgURI, err := url.Parse(orgURIStr) // TODO verify origin is always a host:port
+			orgURI, err := GetOriginURI(ds.OriginFQDN)
 			if err != nil {
-				log.Errorln("Malformed ds '" + string(ds.Name) + "' origin  URI: '" + orgURIStr + "', skipping! : " + err.Error())
+				log.Errorln("Malformed ds '" + string(ds.Name) + "' origin  URI: '" + ds.OriginFQDN + "': skipping!" + err.Error())
 				continue
 			}
-			// TODO put in function, to remove duplication
-			if orgURI.Port() == "" {
-				if orgURI.Scheme == "http" {
-					orgURI.Host += ":80"
-				} else if orgURI.Scheme == "https" {
-					orgURI.Host += ":443"
-				} else {
-					log.Errorln("parent.config generation: delivery service '" + string(ds.Name) + "' origin  URI: '" + orgURIStr + "' is unknown scheme '" + orgURI.Scheme + "', but has no port! Using as-is! ")
-				}
-			}
-
-			if _, ok := uniqueOrigins[ds.OriginFQDN]; ok {
-				continue // TODO warn?
-			}
-			uniqueOrigins[ds.OriginFQDN] = struct{}{}
 
 			textLine := ""
 
@@ -246,48 +267,25 @@ func MakeParentDotConfig(
 				textLine += "\n" // TODO remove, and join later on "\n" instead of ""?
 				textArr = append(textArr, textLine)
 			}
-		}
-		sort.Sort(sort.StringSlice(textArr))
-		text = hdr + strings.Join(textArr, "")
-	} else {
-		processedOriginsToDSNames := map[string]tc.DeliveryServiceName{}
-
-		queryStringHandling := serverParams[ParentConfigParamQStringHandling] // "qsh" in Perl
+		} else {
+			log.Infoln("parent.config generating non-top level line for ds '" + ds.Name + "'")
+			queryStringHandling := serverParams[ParentConfigParamQStringHandling] // "qsh" in Perl
 
-		roundRobin := `round_robin=consistent_hash`
-		goDirect := `go_direct=false`
+			roundRobin := `round_robin=consistent_hash`
+			goDirect := `go_direct=false`
 
-		for _, ds := range parentConfigDSes {
 			parents, secondaryParents := getParentStrs(ds, parentInfos[DeliveryServicesAllParentsKey], atsMajorVer)
 
 			text := ""
-			originFQDN := ds.OriginFQDN
-			if originFQDN == "" {
+			if ds.OriginFQDN == "" {
 				continue // TODO warn? (Perl doesn't)
 			}
-
-			orgURI, err := url.Parse(originFQDN) // TODO verify
+			orgURI, err := GetOriginURI(ds.OriginFQDN)
 			if err != nil {
-				log.Errorln("Malformed ds '" + string(ds.Name) + "' origin  URI: '" + originFQDN + "': skipping!" + err.Error())
+				log.Errorln("Malformed ds '" + string(ds.Name) + "' origin  URI: '" + ds.OriginFQDN + "': skipping!" + err.Error())
 				continue
 			}
 
-			if existingDS, ok := processedOriginsToDSNames[originFQDN]; ok {
-				log.Errorln("parent.config generation: duplicate origin! services '" + string(ds.Name) + "' and '" + string(existingDS) + "' share origin '" + orgURI.Host + "': skipping '" + string(ds.Name) + "'!")
-				continue
-			}
-
-			// TODO put in function, to remove duplication
-			if orgURI.Port() == "" {
-				if orgURI.Scheme == "http" {
-					orgURI.Host += ":80"
-				} else if orgURI.Scheme == "https" {
-					orgURI.Host += ":443"
-				} else {
-					log.Errorln("parent.config generation non-top-level: ds '" + string(ds.Name) + "' origin  URI: '" + originFQDN + "' is unknown scheme '" + orgURI.Scheme + "', but has no port! Using as-is! ")
-				}
-			}
-
 			// TODO encode this in a DSType func, IsGoDirect() ?
 			if dsType := tc.DSType(ds.Type); dsType == tc.DSTypeHTTPNoCache || dsType == tc.DSTypeHTTPLive || dsType == tc.DSTypeDNSLive {
 				text += `dest_domain=` + orgURI.Hostname() + ` port=` + orgURI.Port() + ` go_direct=true` + "\n"
@@ -316,12 +314,15 @@ func MakeParentDotConfig(
 				text += `dest_domain=` + orgURI.Hostname() + ` port=` + orgURI.Port() + ` ` + parents + ` ` + secondaryParents + ` ` + roundRobin + ` ` + goDirect + ` qstring=` + parentQStr + "\n"
 			}
 			textArr = append(textArr, text)
-			processedOriginsToDSNames[originFQDN] = ds.Name
 		}
+		processedOriginsToDSNames[ds.OriginFQDN] = ds.Name
+	}
 
+	// TODO determine if this is necessary. It's super-dangerous, and moreover ignores Server Capabilitites.
+	defaultDestText := ""
+	if !serverInfo.IsTopLevelCache() {
 		parents, secondaryParents := getParentStrs(ParentConfigDSTopLevel{}, parentInfos[DeliveryServicesAllParentsKey], atsMajorVer)
-		// TODO determine if this is necessary. It's super-dangerous, and moreover ignores Server Capabilitites.
-		defaultDestText := `dest_domain=. ` + parents
+		defaultDestText = `dest_domain=. ` + parents
 		if serverParams[ParentConfigParamAlgorithm] == tc.AlgorithmConsistentHash {
 			defaultDestText += secondaryParents
 		}
@@ -331,13 +332,265 @@ func MakeParentDotConfig(
 			defaultDestText += ` qstring=` + qStr
 		}
 		defaultDestText += "\n"
-
-		sort.Sort(sort.StringSlice(textArr))
-		text = hdr + strings.Join(textArr, "") + defaultDestText
 	}
+
+	sort.Sort(sort.StringSlice(textArr))
+	text := hdr + strings.Join(textArr, "") + defaultDestText
 	return text
 }
 
+func GetTopologyParentConfigLine(
+	server tc.Server,
+	servers []tc.Server,
+	ds ParentConfigDSTopLevel,
+	serverParams map[string]string,
+	parentConfigParams []ParameterWithProfilesMap, // all params with configFile parent.config
+	nameTopologies map[TopologyName]tc.Topology,
+	serverCapabilities map[int]map[ServerCapability]struct{},
+	cacheGroups map[tc.CacheGroupName]tc.CacheGroupNullable,
+) (string, error) {
+	txt := ""
+
+	if !HasRequiredCapabilities(serverCapabilities[server.ID], ds.RequiredCapabilities) {
+		return "", nil
+	}
+
+	orgURI, err := GetOriginURI(ds.OriginFQDN)
+	if err != nil {
+		return "", errors.New("Malformed ds '" + string(ds.Name) + "' origin  URI: '" + ds.OriginFQDN + "': skipping!" + err.Error())
+	}
+
+	topology := nameTopologies[TopologyName(ds.Topology)]
+	if topology.Name == "" {
+		return "", errors.New("DS " + string(ds.Name) + " topology '" + ds.Topology + "' not found in Topologies!")
+	}
+
+	txt += "dest_domain=" + orgURI.Hostname() + " port=" + orgURI.Port()
+
+	log.Errorf("DEBUG topo GetTopologyParentConfigLine calling getTopologyPlacement cg '" + server.Cachegroup + "'\n")
+	serverPlacement := getTopologyPlacement(tc.CacheGroupName(server.Cachegroup), topology, cacheGroups)
+	if !serverPlacement.InTopology {
+		return "", nil // server isn't in topology, no error
+	}
+	// TODO add Topology/Capabilities to remap.config
+
+	parents, secondaryParents, err := GetTopologyParents(server, ds, servers, parentConfigParams, topology, serverPlacement.IsLastTier, serverCapabilities)
+	if err != nil {
+		return "", errors.New("getting topology parents for '" + string(ds.Name) + "': skipping! " + err.Error())
+	}
+	if len(parents) == 0 {
+		return "", errors.New("getting topology parents for '" + string(ds.Name) + "': no parents found! skipping! (Does your Topology have a CacheGroup with no servers in it?)")
+	}
+
+	txt += ` parent="` + strings.Join(parents, `;`) + `"`
+	if len(secondaryParents) > 0 {
+		txt += ` secondary_parent="` + strings.Join(secondaryParents, `;`) + `"`
+	}
+	txt += ` round_robin=` + getTopologyRoundRobin(ds, serverParams, serverPlacement.IsLastTier)
+	txt += ` go_direct=` + getTopologyGoDirect(ds, serverPlacement.IsLastTier)
+	txt += ` qstring=` + getTopologyQueryString(ds, serverParams, serverPlacement.IsLastTier)
+	txt += getTopologyParentIsProxyStr(serverPlacement.IsLastTier)
+	txt += " # topology '" + ds.Topology + "'"
+	txt += "\n"
+	return txt, nil
+}
+
+func getTopologyParentIsProxyStr(serverIsLastTier bool) string {
+	if serverIsLastTier {
+		return ` parent_is_proxy=false`
+	}
+	return ""
+}
+
+func getTopologyRoundRobin(ds ParentConfigDSTopLevel, serverParams map[string]string, serverIsLastTier bool) string {
+	roundRobinConsistentHash := "consistent_hash"
+	if !serverIsLastTier {
+		return roundRobinConsistentHash
+	}
+	if parentSelectAlg := serverParams[ParentConfigParamAlgorithm]; ds.OriginShield != "" && strings.TrimSpace(parentSelectAlg) != "" {
+		return parentSelectAlg
+	}
+	if ds.MultiSiteOrigin {
+		return ds.MSOAlgorithm
+	}
+	return roundRobinConsistentHash
+}
+
+func getTopologyGoDirect(ds ParentConfigDSTopLevel, serverIsLastTier bool) string {
+	if !serverIsLastTier {
+		return "false"
+	}
+	if ds.OriginShield != "" {
+		return "true"
+	}
+	if ds.MultiSiteOrigin {
+		return "false"
+	}
+	return "true"
+}
+
+func getTopologyQueryString(ds ParentConfigDSTopLevel, serverParams map[string]string, serverIsLastTier bool) string {
+	if serverIsLastTier {
+		if ds.MultiSiteOrigin && ds.QStringHandling == "" && ds.MSOAlgorithm == tc.AlgorithmConsistentHash && ds.QStringIgnore == tc.QStringIgnoreUseInCacheKeyAndPassUp {
+			return "consider"
+		}
+		return "ignore"
+	}
+
+	if param := serverParams[ParentConfigParamQStringHandling]; param != "" {
+		return param
+	}
+	if ds.QStringHandling != "" {
+		return ds.QStringHandling
+	}
+	if ds.QStringIgnore == tc.QStringIgnoreUseInCacheKeyAndPassUp {
+		return "consider"
+	}
+	return "ignore"
+}
+
+// serverParentageParams gets the Parameters used for parent= line, or defaults if they don't exist
+// Returns the Parameters used for parent= lines, for the given server.
+func serverParentageParams(sv tc.Server, params []ParameterWithProfilesMap) ProfileCache {
+	// TODO deduplicate with atstccfg/parentdotconfig.go
+	profileCache := DefaultProfileCache()
+	profileCache.Port = sv.TCPPort
+	for _, param := range params {
+		if _, ok := param.ProfileNames[sv.Profile]; !ok {
+			continue
+		}
+		switch param.Name {
+		case ParentConfigCacheParamWeight:
+			profileCache.Weight = param.Value
+		case ParentConfigCacheParamPort:
+			i, err := strconv.ParseInt(param.Value, 10, 64)
+			if err != nil {
+				log.Errorln("parent.config generation: port param is not an integer, skipping! : " + err.Error())
+			} else {
+				profileCache.Port = int(i)
+			}
+		case ParentConfigCacheParamUseIP:
+			profileCache.UseIP = param.Value == "1"
+		case ParentConfigCacheParamRank:
+			i, err := strconv.ParseInt(param.Value, 10, 64)
+			if err != nil {
+				log.Errorln("parent.config generation: rank param is not an integer, skipping! : " + err.Error())
+			} else {
+				profileCache.Rank = int(i)
+			}
+		case ParentConfigCacheParamNotAParent:
+			profileCache.NotAParent = param.Value != "false"
+		}
+	}
+	return profileCache
+}
+
+func serverParentStr(sv tc.Server, params []ParameterWithProfilesMap) string {
+	svParams := serverParentageParams(sv, params)
+	if svParams.NotAParent {
+		return ""
+	}
+	host := ""
+	if svParams.UseIP {
+		host = sv.IPAddress
+	} else {
+		host = sv.HostName + "." + sv.DomainName
+	}
+	return host + ":" + strconv.Itoa(svParams.Port) + "|" + svParams.Weight
+}
+
+func GetTopologyParents(
+	server tc.Server,
+	ds ParentConfigDSTopLevel,
+	servers []tc.Server,
+	parentConfigParams []ParameterWithProfilesMap, // all params with configFile parent.confign
+	topology tc.Topology,
+	serverIsLastTier bool,
+	serverCapabilities map[int]map[ServerCapability]struct{},
+) ([]string, []string, error) {
+	// 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, err := GetOriginURI(ds.OriginFQDN) // TODO pass, instead of calling again
+		if err != nil {
+			return nil, nil, err
+		}
+		return []string{orgURI.Host}, nil, nil
+	}
+
+	svNode := tc.TopologyNode{}
+	for _, node := range topology.Nodes {
+		if node.Cachegroup == server.Cachegroup {
+			svNode = node
+			break
+		}
+	}
+	if svNode.Cachegroup == "" {
+		return nil, nil, errors.New("This server '" + server.HostName + "' not in DS " + string(ds.Name) + " topology, skipping")
+	}
+
+	if len(svNode.Parents) == 0 {
+		return nil, nil, errors.New("DS " + string(ds.Name) + " topology '" + ds.Topology + "' is last tier, but NonLastTier called! Should never happen")
+	}
+	if numParents := len(svNode.Parents); numParents > 2 {
+		log.Errorln("DS " + string(ds.Name) + " topology '" + ds.Topology + "' has " + strconv.Itoa(numParents) + " parent nodes, but Apache Traffic Server only supports Primary and Secondary (2) lists of parents. CacheGroup nodes after the first 2 will be ignored!")
+	}
+	if len(topology.Nodes) <= svNode.Parents[0] {
+		return nil, nil, errors.New("DS " + string(ds.Name) + " topology '" + ds.Topology + "' node parent " + strconv.Itoa(svNode.Parents[0]) + " greater than number of topology nodes " + strconv.Itoa(len(topology.Nodes)) + ". Cannot create parents!")
+	}
+	if len(svNode.Parents) > 1 && len(topology.Nodes) <= svNode.Parents[1] {
+		log.Errorln("DS " + string(ds.Name) + " topology '" + ds.Topology + "' node secondary parent " + strconv.Itoa(svNode.Parents[1]) + " greater than number of topology nodes " + strconv.Itoa(len(topology.Nodes)) + ". Secondary parent will be ignored!")
+	}
+
+	parentCG := topology.Nodes[svNode.Parents[0]].Cachegroup
+	secondaryParentCG := ""
+	if len(svNode.Parents) > 1 && len(topology.Nodes) > svNode.Parents[1] {
+		secondaryParentCG = topology.Nodes[svNode.Parents[1]].Cachegroup
+	}
+
+	if parentCG == "" {
+		return nil, nil, errors.New("Server '" + server.HostName + "' DS " + string(ds.Name) + " topology '" + ds.Topology + "' cachegroup '" + server.Cachegroup + "' topology node parent " + strconv.Itoa(svNode.Parents[0]) + " is not in the topology!")
+	}
+
+	parentStrs := []string{}
+	secondaryParentStrs := []string{}
+	for _, sv := range servers {
+		if tc.CacheType(sv.Type) != tc.CacheTypeEdge && tc.CacheType(sv.Type) != tc.CacheTypeMid && sv.Type != tc.OriginTypeName {
+			continue // only consider edges, mids, and origins in the CacheGroup.
+		}
+		if !HasRequiredCapabilities(serverCapabilities[sv.ID], ds.RequiredCapabilities) {
+			continue
+		}
+		if sv.Cachegroup == parentCG {
+			parentStr := serverParentStr(sv, parentConfigParams)
+			if parentStr != "" { // will be empty if server is not_a_parent (possibly other reasons)
+				parentStrs = append(parentStrs, parentStr)
+			}
+		}
+		if sv.Cachegroup == secondaryParentCG {
+			secondaryParentStrs = append(secondaryParentStrs, serverParentStr(sv, parentConfigParams))
+		}
+	}
+	return parentStrs, secondaryParentStrs, nil
+}
+
+func GetOriginURI(fqdn string) (*url.URL, error) {
+	orgURI, err := url.Parse(fqdn) // TODO verify origin is always a host:port
+	if err != nil {
+		return nil, errors.New("parsing: " + err.Error())
+	}
+	if orgURI.Port() == "" {
+		if orgURI.Scheme == "http" {
+			orgURI.Host += ":80"
+		} else if orgURI.Scheme == "https" {
+			orgURI.Host += ":443"
+		} else {
+			log.Errorln("parent.config generation non-top-level: origin '" + fqdn + "' is unknown scheme '" + orgURI.Scheme + "', but has no port! Using as-is! ")
+		}
+	}
+	return orgURI, nil
+}
+
 // getParentStrs returns the parents= and secondary_parents= strings for ATS parent.config lines.
 func getParentStrs(ds ParentConfigDSTopLevel, parentInfos []ParentInfo, atsMajorVer int) (string, string) {
 	parentInfo := []string{}
diff --git a/lib/go-atscfg/parentdotconfig_test.go b/lib/go-atscfg/parentdotconfig_test.go
index cdda47c..8ed3f0c 100644
--- a/lib/go-atscfg/parentdotconfig_test.go
+++ b/lib/go-atscfg/parentdotconfig_test.go
@@ -95,7 +95,55 @@ func TestMakeParentDotConfig(t *testing.T) {
 		},
 	}
 
-	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos)
+	server := tc.Server{
+		CachegroupID: 42,
+		Cachegroup:   "myCG",
+		CDNName:      "myCDN",
+		CDNID:        43,
+		DomainName:   "serverdomain.example.net",
+		HostName:     "myserver",
+		ID:           44,
+		IPAddress:    "192.168.2.1",
+		ProfileID:    46,
+		Profile:      "MyProfileName",
+		TCPPort:      80,
+		Type:         "EDGE",
+	}
+
+	mid0 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid0",
+		ID:         45,
+		IPAddress:  "192.168.2.2",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	mid1 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid1",
+		ID:         45,
+		IPAddress:  "192.168.2.3",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	servers := []tc.Server{server, *mid0, *mid1}
+
+	topologies := []tc.Topology{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	parentConfigParams := []tc.Parameter{}
+	cgs := []tc.CacheGroupNullable{}
+
+	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos, server, servers, topologies, parentConfigParams, serverCapabilities, cgs)
 
 	testComment(t, txt, serverName, toolName, toURL)
 
@@ -202,14 +250,60 @@ func TestMakeParentDotConfigCapabilities(t *testing.T) {
 		},
 	}
 
-	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos)
+	server := tc.Server{
+		CachegroupID: 42,
+		Cachegroup:   "myCG",
+		CDNName:      "myCDN",
+		CDNID:        43,
+		DomainName:   "serverdomain.example.net",
+		HostName:     "myserver",
+		ID:           44,
+		IPAddress:    "192.168.2.1",
+		ProfileID:    46,
+		Profile:      "MyProfileName",
+		TCPPort:      80,
+		Type:         "EDGE",
+	}
+	mid0 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid0",
+		ID:         45,
+		IPAddress:  "192.168.2.2",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	mid1 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid1",
+		ID:         45,
+		IPAddress:  "192.168.2.3",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	servers := []tc.Server{server, *mid0, *mid1}
+
+	topologies := []tc.Topology{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	parentConfigParams := []tc.Parameter{}
+	cgs := []tc.CacheGroupNullable{}
+	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos, server, servers, topologies, parentConfigParams, serverCapabilities, cgs)
 
 	testComment(t, txt, serverName, toolName, toURL)
 
 	lines := strings.Split(txt, "\n")
 
 	if len(lines) != 4 {
-		t.Fatalf("expected 4 lines (comment, ds, dot remap, and empty newline), actual: '%+v'", len(lines))
+		t.Fatalf("expected 4 lines (comment, ds, dot remap, and empty newline), actual: '%+v' text %v", len(lines), txt)
 	}
 
 	for _, line := range lines {
@@ -314,7 +408,54 @@ func TestMakeParentDotConfigMSOSecondaryParent(t *testing.T) {
 		t.Fatal("server should have been top level, was not; cannot test MSO Secondary Parent")
 	}
 
-	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos)
+	server := tc.Server{
+		CachegroupID: 42,
+		Cachegroup:   "myCG",
+		CDNName:      "myCDN",
+		CDNID:        43,
+		DomainName:   "serverdomain.example.net",
+		HostName:     "myserver",
+		ID:           44,
+		IPAddress:    "192.168.2.1",
+		ProfileID:    46,
+		Profile:      "MyProfileName",
+		TCPPort:      80,
+		Type:         "EDGE",
+	}
+
+	mid0 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid0",
+		ID:         45,
+		IPAddress:  "192.168.2.2",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	mid1 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid1",
+		ID:         45,
+		IPAddress:  "192.168.2.3",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	servers := []tc.Server{server, *mid0, *mid1}
+
+	topologies := []tc.Topology{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	parentConfigParams := []tc.Parameter{}
+	cgs := []tc.CacheGroupNullable{}
+	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos, server, servers, topologies, parentConfigParams, serverCapabilities, cgs)
 
 	testComment(t, txt, serverName, toolName, toURL)
 
@@ -324,3 +465,429 @@ func TestMakeParentDotConfigMSOSecondaryParent(t *testing.T) {
 		t.Errorf("expected secondary parent 'my-parent-1.my-parent-1-domain', actual: '%v'", txt)
 	}
 }
+
+func TestMakeParentDotConfigTopologies(t *testing.T) {
+	atsMajorVer := 7
+	serverName := "myserver"
+	toolName := "myToolName"
+	toURL := "https://myto.example.net"
+
+	parentConfigDSes := []ParentConfigDSTopLevel{
+		ParentConfigDSTopLevel{
+			ParentConfigDS: ParentConfigDS{
+				Name:            "ds0",
+				QStringIgnore:   tc.QStringIgnoreUseInCacheKeyAndPassUp,
+				OriginFQDN:      "http://ds0.example.net",
+				MultiSiteOrigin: false,
+				Type:            tc.DSTypeHTTP,
+				QStringHandling: "ds0qstringhandling",
+				// no Topology - test that generation works right with a DS with and one without
+			},
+		},
+		ParentConfigDSTopLevel{
+			ParentConfigDS: ParentConfigDS{
+				Name:            "ds1",
+				QStringIgnore:   tc.QStringIgnoreDrop,
+				OriginFQDN:      "http://ds1.example.net",
+				MultiSiteOrigin: false,
+				Type:            tc.DSTypeDNS,
+				QStringHandling: "ds1qstringhandling",
+				Topology:        "t0",
+			},
+		},
+	}
+
+	serverInfo := &ServerInfo{
+		CacheGroupID:                  42,
+		CacheGroupName:                "edgeCG",
+		CDN:                           "myCDN",
+		CDNID:                         43,
+		DomainName:                    "serverdomain.example.net",
+		HostName:                      "myserver",
+		ID:                            44,
+		IP:                            "192.168.2.1",
+		ParentCacheGroupID:            45,
+		ParentCacheGroupType:          "myParentCGType",
+		ProfileID:                     46,
+		ProfileName:                   "MyProfileName",
+		Port:                          80,
+		SecondaryParentCacheGroupID:   47,
+		SecondaryParentCacheGroupType: "MySecondaryParentCGType",
+		Type:                          "EDGE",
+	}
+
+	serverParams := map[string]string{
+		ParentConfigParamQStringHandling: "myQStringHandlingParam",
+		ParentConfigParamAlgorithm:       tc.AlgorithmConsistentHash,
+		ParentConfigParamQString:         "myQstringParam",
+	}
+
+	parentInfos := map[OriginHost][]ParentInfo{
+		"ds1.example.net": []ParentInfo{
+			ParentInfo{
+				Host:            "my-parent-0",
+				Port:            80,
+				Domain:          "my-parent-0-domain",
+				Weight:          "1",
+				UseIP:           false,
+				Rank:            1,
+				IP:              "192.168.2.2",
+				PrimaryParent:   true,
+				SecondaryParent: true,
+			},
+		},
+	}
+
+	server := serverInfoToServer(serverInfo)
+	server.Cachegroup = "edgeCG"
+
+	mid0 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid0",
+		ID:         45,
+		IPAddress:  "192.168.2.2",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	mid1 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid1",
+		ID:         45,
+		IPAddress:  "192.168.2.3",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	servers := []tc.Server{server, *mid0, *mid1}
+
+	topologies := []tc.Topology{
+		tc.Topology{
+			Name: "t0",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "edgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+	}
+
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	parentConfigParams := []tc.Parameter{}
+	cgs := []tc.CacheGroupNullable{}
+	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos, server, servers, topologies, parentConfigParams, serverCapabilities, cgs)
+
+	testComment(t, txt, serverName, toolName, toURL)
+
+	if !strings.Contains(txt, "dest_domain=ds0.example.net") {
+		t.Errorf("expected parent 'dest_domain=ds0.example.net', actual: '%v'", txt)
+	}
+	if !strings.Contains(txt, "dest_domain=ds1.example.net") {
+		t.Errorf("expected parent 'dest_domain=ds1.example.net', actual: '%v'", txt)
+	}
+	if !strings.Contains(txt, "qstring=myQStringHandlingParam") {
+		t.Errorf("expected qstring from param 'qstring=myQStringHandlingParam', actual: '%v'", txt)
+	}
+}
+
+// TestMakeParentDotConfigNotInTopologies tests when a given edge is NOT in a Topology, that it doesn't add a remap line.
+func TestMakeParentDotConfigNotInTopologies(t *testing.T) {
+	atsMajorVer := 7
+	serverName := "myserver"
+	toolName := "myToolName"
+	toURL := "https://myto.example.net"
+
+	parentConfigDSes := []ParentConfigDSTopLevel{
+		ParentConfigDSTopLevel{
+			ParentConfigDS: ParentConfigDS{
+				Name:            "ds0",
+				QStringIgnore:   tc.QStringIgnoreUseInCacheKeyAndPassUp,
+				OriginFQDN:      "http://ds0.example.net",
+				MultiSiteOrigin: false,
+				Type:            tc.DSTypeHTTP,
+				QStringHandling: "ds0qstringhandling",
+				Topology:        "t0",
+			},
+		},
+		ParentConfigDSTopLevel{
+			ParentConfigDS: ParentConfigDS{
+				Name:            "ds1",
+				QStringIgnore:   tc.QStringIgnoreDrop,
+				OriginFQDN:      "http://ds1.example.net",
+				MultiSiteOrigin: false,
+				Type:            tc.DSTypeDNS,
+				QStringHandling: "ds1qstringhandling",
+				// no Topology - test that generation works right with a DS with and one without
+			},
+		},
+	}
+
+	serverInfo := &ServerInfo{
+		CacheGroupID:                  42,
+		CDN:                           "myCDN",
+		CDNID:                         43,
+		DomainName:                    "serverdomain.example.net",
+		HostName:                      "myserver",
+		ID:                            44,
+		IP:                            "192.168.2.1",
+		ParentCacheGroupID:            45,
+		ParentCacheGroupType:          "myParentCGType",
+		ProfileID:                     46,
+		ProfileName:                   "MyProfileName",
+		Port:                          80,
+		SecondaryParentCacheGroupID:   47,
+		SecondaryParentCacheGroupType: "MySecondaryParentCGType",
+		Type:                          "EDGE",
+	}
+
+	serverParams := map[string]string{
+		ParentConfigParamQStringHandling: "myQStringHandlingParam",
+		ParentConfigParamAlgorithm:       tc.AlgorithmConsistentHash,
+		ParentConfigParamQString:         "myQstringParam",
+	}
+
+	parentInfos := map[OriginHost][]ParentInfo{
+		"ds1.example.net": []ParentInfo{
+			ParentInfo{
+				Host:            "my-parent-0",
+				Port:            80,
+				Domain:          "my-parent-0-domain",
+				Weight:          "1",
+				UseIP:           false,
+				Rank:            1,
+				IP:              "192.168.2.2",
+				PrimaryParent:   true,
+				SecondaryParent: true,
+			},
+		},
+	}
+
+	server := serverInfoToServer(serverInfo)
+	server.Cachegroup = "edgeCG"
+
+	mid0 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid0",
+		ID:         45,
+		IPAddress:  "192.168.2.2",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	mid1 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid1",
+		ID:         45,
+		IPAddress:  "192.168.2.3",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	servers := []tc.Server{server, *mid0, *mid1}
+
+	topologies := []tc.Topology{
+		tc.Topology{
+			Name: "t0",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "otherEdgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+	}
+
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	parentConfigParams := []tc.Parameter{}
+	cgs := []tc.CacheGroupNullable{}
+	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos, server, servers, topologies, parentConfigParams, serverCapabilities, cgs)
+
+	testComment(t, txt, serverName, toolName, toURL)
+
+	if strings.Contains(txt, "dest_domain=ds0.example.net") {
+		t.Errorf("expected parent 'dest_domain=ds0.example.net' to NOT contain Topology DS without this edge: '%v'", txt)
+	}
+	if !strings.Contains(txt, "dest_domain=ds1.example.net") {
+		t.Errorf("expected parent 'dest_domain=ds0.example.net', actual: '%v'", txt)
+	}
+}
+
+func TestMakeParentDotConfigTopologiesCapabilities(t *testing.T) {
+	atsMajorVer := 7
+	serverName := "myserver"
+	toolName := "myToolName"
+	toURL := "https://myto.example.net"
+
+	parentConfigDSes := []ParentConfigDSTopLevel{
+		ParentConfigDSTopLevel{
+			ParentConfigDS: ParentConfigDS{
+				Name:            "ds0",
+				QStringIgnore:   tc.QStringIgnoreUseInCacheKeyAndPassUp,
+				OriginFQDN:      "http://ds0.example.net",
+				MultiSiteOrigin: false,
+				Type:            tc.DSTypeHTTP,
+				QStringHandling: "ds0qstringhandling",
+				Topology:        "t0",
+			},
+		},
+		ParentConfigDSTopLevel{
+			ParentConfigDS: ParentConfigDS{
+				Name:            "ds1",
+				QStringIgnore:   tc.QStringIgnoreDrop,
+				OriginFQDN:      "http://ds1.example.net",
+				MultiSiteOrigin: false,
+				Type:            tc.DSTypeDNS,
+				QStringHandling: "ds1qstringhandling",
+				Topology:        "t0",
+				RequiredCapabilities: map[ServerCapability]struct{}{
+					"FOO": {},
+				},
+			},
+		},
+		ParentConfigDSTopLevel{
+			ParentConfigDS: ParentConfigDS{
+				Name:            "ds2",
+				QStringIgnore:   tc.QStringIgnoreDrop,
+				OriginFQDN:      "http://ds2.example.net",
+				MultiSiteOrigin: false,
+				Type:            tc.DSTypeDNS,
+				QStringHandling: "ds2qstringhandling",
+				Topology:        "t0",
+				RequiredCapabilities: map[ServerCapability]struct{}{
+					"BAR": {},
+				},
+			},
+		},
+	}
+
+	serverInfo := &ServerInfo{
+		CacheGroupID:                  42,
+		CDN:                           "myCDN",
+		CDNID:                         43,
+		DomainName:                    "serverdomain.example.net",
+		HostName:                      "myserver",
+		ID:                            44,
+		IP:                            "192.168.2.1",
+		ParentCacheGroupID:            45,
+		ParentCacheGroupType:          "myParentCGType",
+		ProfileID:                     46,
+		ProfileName:                   "MyProfileName",
+		Port:                          80,
+		SecondaryParentCacheGroupID:   47,
+		SecondaryParentCacheGroupType: "MySecondaryParentCGType",
+		Type:                          "EDGE",
+	}
+
+	serverParams := map[string]string{
+		ParentConfigParamQStringHandling: "myQStringHandlingParam",
+		ParentConfigParamAlgorithm:       tc.AlgorithmConsistentHash,
+		ParentConfigParamQString:         "myQstringParam",
+	}
+
+	parentInfos := map[OriginHost][]ParentInfo{
+		"ds1.example.net": []ParentInfo{
+			ParentInfo{
+				Host:            "my-parent-0",
+				Port:            80,
+				Domain:          "my-parent-0-domain",
+				Weight:          "1",
+				UseIP:           false,
+				Rank:            1,
+				IP:              "192.168.2.2",
+				PrimaryParent:   true,
+				SecondaryParent: true,
+			},
+		},
+	}
+
+	server := serverInfoToServer(serverInfo)
+	server.Cachegroup = "edgeCG"
+
+	mid0 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid0",
+		ID:         45,
+		IPAddress:  "192.168.2.2",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	mid1 := &tc.Server{
+		Cachegroup: "midCG",
+		CDNName:    "myCDN",
+		CDNID:      43,
+		DomainName: "serverdomain.example.net",
+		HostName:   "mymid1",
+		ID:         46,
+		IPAddress:  "192.168.2.3",
+		ProfileID:  46,
+		Profile:    "MyProfileName",
+		TCPPort:    80,
+		Type:       "EDGE",
+	}
+	servers := []tc.Server{server, *mid0, *mid1}
+
+	topologies := []tc.Topology{
+		tc.Topology{
+			Name: "t0",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "edgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+	}
+
+	serverCapabilities := map[int]map[ServerCapability]struct{}{
+		44: map[ServerCapability]struct{}{"FOO": {}},
+		45: map[ServerCapability]struct{}{"FOO": {}},
+		46: map[ServerCapability]struct{}{"FOO": {}},
+	}
+	parentConfigParams := []tc.Parameter{}
+	cgs := []tc.CacheGroupNullable{}
+
+	txt := MakeParentDotConfig(serverInfo, atsMajorVer, toolName, toURL, parentConfigDSes, serverParams, parentInfos, server, servers, topologies, parentConfigParams, serverCapabilities, cgs)
+
+	testComment(t, txt, serverName, toolName, toURL)
+
+	if !strings.Contains(txt, "dest_domain=ds0.example.net") {
+		t.Errorf("expected parent 'dest_domain=ds0.example.net' without required capabilities: '%v'", txt)
+	}
+	if !strings.Contains(txt, "dest_domain=ds1.example.net") {
+		t.Errorf("expected parent 'dest_domain=ds1.example.net' with necessary required capabilities, actual: '%v'", txt)
+	}
+	if strings.Contains(txt, "dest_domain=ds2.example.net") {
+		t.Errorf("expected no parent 'dest_domain=ds2.example.net' without necessary required capabilities, actual: '%v'", txt)
+	}
+}
diff --git a/lib/go-atscfg/remapdotconfig.go b/lib/go-atscfg/remapdotconfig.go
index cd6d857..64e7221 100644
--- a/lib/go-atscfg/remapdotconfig.go
+++ b/lib/go-atscfg/remapdotconfig.go
@@ -62,10 +62,14 @@ type RemapConfigDSData struct {
 	AnonymousBlockingEnabled *bool
 	RangeSliceBlockSize      *int
 	Active                   bool
+	Topology                 string
+	FirstHeaderRewrite       string
+	InnerHeaderRewrite       string
+	LastHeaderRewrite        string
 }
 
 func MakeRemapDotConfig(
-	serverName tc.CacheName,
+	server tc.Server,
 	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
 	toURL string, // tm.url global parameter (TODO: cache itself?)
 	atsMajorVersion int,
@@ -74,28 +78,43 @@ func MakeRemapDotConfig(
 	serverPackageParamData map[string]string, // map[paramName]paramVal for this server, config file 'package'
 	serverInfo *ServerInfo, // ServerInfo for this server
 	remapDSData []RemapConfigDSData,
+	topologies []tc.Topology,
+	cacheGroupArr []tc.CacheGroupNullable,
+	serverCapabilities map[int]map[ServerCapability]struct{},
+	dsRequiredCapabilities map[int]map[ServerCapability]struct{},
 ) string {
-	hdr := GenericHeaderComment(string(serverName), toToolName, toURL)
-	text := ""
+	cacheGroups := MakeCGMap(cacheGroupArr)
+	nameTopologies := MakeTopologyNameMap(topologies)
+	hdr := GenericHeaderComment(server.HostName, toToolName, toURL)
 	if tc.CacheTypeFromString(serverInfo.Type) == tc.CacheTypeMid {
-		text = GetServerConfigRemapDotConfigForMid(atsMajorVersion, dsProfilesCacheKeyConfigParams, serverInfo, remapDSData, hdr)
-	} else {
-		text = GetServerConfigRemapDotConfigForEdge(cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, atsMajorVersion, hdr)
+		return GetServerConfigRemapDotConfigForMid(atsMajorVersion, dsProfilesCacheKeyConfigParams, remapDSData, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities)
 	}
-	return text
+	return GetServerConfigRemapDotConfigForEdge(cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, atsMajorVersion, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities)
 }
 
 func GetServerConfigRemapDotConfigForMid(
 	atsMajorVersion int,
 	profilesCacheKeyConfigParams map[int]map[string]string,
-	server *ServerInfo,
 	dses []RemapConfigDSData,
 	header string,
+	server tc.Server,
+	nameTopologies map[TopologyName]tc.Topology,
+	cacheGroups map[tc.CacheGroupName]tc.CacheGroupNullable,
+	serverCapabilities map[int]map[ServerCapability]struct{},
+	dsRequiredCapabilities map[int]map[ServerCapability]struct{},
 ) string {
 	midRemaps := map[string]string{}
 	for _, ds := range dses {
-		if ds.Type.IsLive() && !ds.Type.IsNational() {
-			continue // Live local delivery services skip mids
+		if !HasRequiredCapabilities(serverCapabilities[server.ID], dsRequiredCapabilities[ds.ID]) {
+			continue
+		}
+
+		topology, hasTopology := nameTopologies[TopologyName(ds.Topology)]
+		if ds.Topology != "" && hasTopology && !topologyIncludesServer(topology, server) {
+			continue
+		}
+		if ds.Type.IsLive() && !ds.Type.IsNational() && !hasTopology {
+			continue // Live local delivery services skip mids (except Topologies ignore DS types)
 		}
 
 		if ds.OriginFQDN == nil || *ds.OriginFQDN == "" {
@@ -113,9 +132,13 @@ func GetServerConfigRemapDotConfigForMid(
 		hasCacheKey := false
 
 		midRemap := ""
-		if ds.MidHeaderRewrite != nil && *ds.MidHeaderRewrite != "" {
+
+		if ds.Topology != "" {
+			midRemap += MakeDSTopologyHeaderRewriteTxt(ds, tc.CacheGroupName(server.Cachegroup), topology, cacheGroups)
+		} else if ds.MidHeaderRewrite != nil && *ds.MidHeaderRewrite != "" {
 			midRemap += ` @plugin=header_rewrite.so @pparam=` + MidHeaderRewriteConfigFileName(ds.Name)
 		}
+
 		if ds.QStringIgnore != nil && *ds.QStringIgnore == tc.QueryStringIgnoreIgnoreInCacheKeyAndPassUp {
 			qstr, addedCacheURL, addedCacheKey := GetQStringIgnoreRemap(atsMajorVersion)
 			if addedCacheURL {
@@ -166,14 +189,27 @@ func GetServerConfigRemapDotConfigForEdge(
 	cacheURLConfigParams map[string]string,
 	profilesCacheKeyConfigParams map[int]map[string]string,
 	serverPackageParamData map[string]string, // map[paramName]paramVal for this server, config file 'package'
-	server *ServerInfo,
+	serverInfo *ServerInfo,
 	dses []RemapConfigDSData,
 	atsMajorVersion int,
 	header string,
+	server tc.Server,
+	nameTopologies map[TopologyName]tc.Topology,
+	cacheGroups map[tc.CacheGroupName]tc.CacheGroupNullable,
+	serverCapabilities map[int]map[ServerCapability]struct{},
+	dsRequiredCapabilities map[int]map[ServerCapability]struct{},
 ) string {
 	textLines := []string{}
 
 	for _, ds := range dses {
+		if !HasRequiredCapabilities(serverCapabilities[server.ID], dsRequiredCapabilities[ds.ID]) {
+			continue
+		}
+
+		topology, hasTopology := nameTopologies[TopologyName(ds.Topology)]
+		if ds.Topology != "" && hasTopology && !topologyIncludesServer(topology, server) {
+			continue
+		}
 		remapText := ""
 		if ds.Type == tc.DSTypeAnyMap {
 			if ds.RemapText == nil {
@@ -185,7 +221,7 @@ func GetServerConfigRemapDotConfigForEdge(
 			continue
 		}
 
-		remapLines, err := MakeEdgeDSDataRemapLines(ds, server)
+		remapLines, err := MakeEdgeDSDataRemapLines(ds, serverInfo)
 		if err != nil {
 			log.Errorln("making remap lines for DS '" + ds.Name + "' - skipping! : " + err.Error())
 			continue
@@ -196,7 +232,11 @@ func GetServerConfigRemapDotConfigForEdge(
 			if ds.ProfileID != nil {
 				profilecacheKeyConfigParams = profilesCacheKeyConfigParams[*ds.ProfileID]
 			}
-			remapText = BuildRemapLine(cacheURLConfigParams, atsMajorVersion, server, serverPackageParamData, remapText, ds, line.From, line.To, profilecacheKeyConfigParams)
+			remapText = BuildEdgeRemapLine(cacheURLConfigParams, atsMajorVersion, serverInfo, serverPackageParamData, remapText, ds, line.From, line.To, profilecacheKeyConfigParams, cacheGroups, nameTopologies)
+			if hasTopology {
+				remapText += " # topology '" + topology.Name + "'"
+			}
+			remapText += "\n"
 		}
 		textLines = append(textLines, remapText)
 	}
@@ -209,9 +249,21 @@ func GetServerConfigRemapDotConfigForEdge(
 
 const RemapConfigRangeDirective = `__RANGE_DIRECTIVE__`
 
-// BuildRemapLine builds the remap line for the given server and delivery service.
+// BuildEdgeRemapLine builds the remap line for the given server and delivery service.
 // The cacheKeyConfigParams map may be nil, if this ds profile had no cache key config params.
-func BuildRemapLine(cacheURLConfigParams map[string]string, atsMajorVersion int, server *ServerInfo, pData map[string]string, text string, ds RemapConfigDSData, mapFrom string, mapTo string, cacheKeyConfigParams map[string]string) string {
+func BuildEdgeRemapLine(
+	cacheURLConfigParams map[string]string,
+	atsMajorVersion int,
+	server *ServerInfo,
+	pData map[string]string,
+	text string,
+	ds RemapConfigDSData,
+	mapFrom string,
+	mapTo string,
+	cacheKeyConfigParams map[string]string,
+	cacheGroups map[tc.CacheGroupName]tc.CacheGroupNullable,
+	nameTopologies map[TopologyName]tc.Topology,
+) string {
 	// ds = 'remap' in perl
 	mapFrom = strings.Replace(mapFrom, `__http__`, server.HostName, -1)
 
@@ -221,7 +273,9 @@ func BuildRemapLine(cacheURLConfigParams map[string]string, atsMajorVersion int,
 		text += "map	" + mapFrom + "     " + mapTo + ` @plugin=header_rewrite.so @pparam=dscp/set_dscp_` + strconv.Itoa(ds.DSCP) + ".config"
 	}
 
-	if ds.EdgeHeaderRewrite != nil && *ds.EdgeHeaderRewrite != "" {
+	if ds.Topology != "" {
+		text += MakeDSTopologyHeaderRewriteTxt(ds, tc.CacheGroupName(server.CacheGroupName), nameTopologies[TopologyName(ds.Topology)], cacheGroups)
+	} else if ds.EdgeHeaderRewrite != nil && *ds.EdgeHeaderRewrite != "" {
 		text += ` @plugin=header_rewrite.so @pparam=` + EdgeHeaderRewriteConfigFileName(ds.Name)
 	}
 
@@ -316,10 +370,38 @@ func BuildRemapLine(cacheURLConfigParams map[string]string, atsMajorVersion int,
 	if ds.FQPacingRate != nil && *ds.FQPacingRate > 0 {
 		text += ` @plugin=fq_pacing.so @pparam=--rate=` + strconv.Itoa(*ds.FQPacingRate)
 	}
-	text += "\n"
 	return text
 }
 
+// MakeDSTopologyHeaderRewriteTxt returns the appropriate header rewrite remap line text for the given DS on the given server.
+// May be empty, if the DS has no header rewrite for the server's position in the topology.
+func MakeDSTopologyHeaderRewriteTxt(ds RemapConfigDSData, cg tc.CacheGroupName, topology tc.Topology, cacheGroups map[tc.CacheGroupName]tc.CacheGroupNullable) string {
+	placement := getTopologyPlacement(cg, topology, cacheGroups)
+	log.Errorf("DEBUG topo MakeDSTopologyHeaderRewriteTxt calling getTopologyPlacement cg '"+string(cg)+"' placement %+v\n", placement)
+	const pluginTxt = ` @plugin=header_rewrite.so @pparam=`
+	switch placement.CacheTier {
+	case TopologyCacheTierFirst:
+		if ds.FirstHeaderRewrite == "" {
+			return ""
+		}
+		log.Errorf("DEBUG topo MakeDSTopologyHeaderRewriteTxt first returning '" + pluginTxt + FirstHeaderRewriteConfigFileName(ds.Name) + "'")
+		return pluginTxt + FirstHeaderRewriteConfigFileName(ds.Name)
+	case TopologyCacheTierInner:
+		if ds.InnerHeaderRewrite == "" {
+			return ""
+		}
+		return pluginTxt + InnerHeaderRewriteConfigFileName(ds.Name)
+	case TopologyCacheTierLast:
+		if ds.LastHeaderRewrite == "" {
+			return ""
+		}
+		return pluginTxt + LastHeaderRewriteConfigFileName(ds.Name)
+	default:
+		log.Errorln("Making topology header rewrite text: got unknown cache tier '" + placement.CacheTier + "' - not setting!")
+		return ""
+	}
+}
+
 func DSProfileIDs(dses []RemapConfigDSData) []int {
 	dsProfileIDs := []int{}
 	for _, ds := range dses {
diff --git a/lib/go-atscfg/remapdotconfig_test.go b/lib/go-atscfg/remapdotconfig_test.go
index 36b922f..c77f1e8 100644
--- a/lib/go-atscfg/remapdotconfig_test.go
+++ b/lib/go-atscfg/remapdotconfig_test.go
@@ -52,7 +52,7 @@ func TestMakeRemapDotConfig(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -97,7 +97,12 @@ func TestMakeRemapDotConfig(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -153,7 +158,7 @@ func TestMakeRemapDotConfigMidLiveLocalExcluded(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -198,7 +203,12 @@ func TestMakeRemapDotConfigMidLiveLocalExcluded(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -236,7 +246,7 @@ func TestMakeRemapDotConfigMid(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -281,7 +291,12 @@ func TestMakeRemapDotConfigMid(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -333,7 +348,7 @@ func TestMakeRemapDotConfigNilOrigin(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -378,7 +393,12 @@ func TestMakeRemapDotConfigNilOrigin(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -416,7 +436,7 @@ func TestMakeRemapDotConfigEmptyOrigin(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -461,7 +481,12 @@ func TestMakeRemapDotConfigEmptyOrigin(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -499,7 +524,7 @@ func TestMakeRemapDotConfigDuplicateOrigins(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -572,7 +597,12 @@ func TestMakeRemapDotConfigDuplicateOrigins(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -610,7 +640,7 @@ func TestMakeRemapDotConfigNilMidRewrite(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -655,7 +685,12 @@ func TestMakeRemapDotConfigNilMidRewrite(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -716,7 +751,7 @@ func TestMakeRemapDotConfigMidHasNoEdgeRewrite(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -761,7 +796,12 @@ func TestMakeRemapDotConfigMidHasNoEdgeRewrite(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -813,7 +853,7 @@ func TestMakeRemapDotConfigMidQStringPassUpATS7CacheKey(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -858,7 +898,12 @@ func TestMakeRemapDotConfigMidQStringPassUpATS7CacheKey(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -917,7 +962,7 @@ func TestMakeRemapDotConfigMidQStringPassUpATS5CacheURL(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -962,7 +1007,12 @@ func TestMakeRemapDotConfigMidQStringPassUpATS5CacheURL(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -1024,7 +1074,7 @@ func TestMakeRemapDotConfigMidProfileCacheKey(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -1069,7 +1119,12 @@ func TestMakeRemapDotConfigMidProfileCacheKey(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -1132,7 +1187,7 @@ func TestMakeRemapDotConfigMidRangeRequestHandling(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -1177,7 +1232,12 @@ func TestMakeRemapDotConfigMidRangeRequestHandling(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -1232,7 +1292,7 @@ func TestMakeRemapDotConfigMidSlicePluginRangeRequestHandling(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -1278,7 +1338,12 @@ func TestMakeRemapDotConfigMidSlicePluginRangeRequestHandling(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -1334,7 +1399,7 @@ func TestMakeRemapDotConfigFirstExcludedSecondIncluded(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -1407,7 +1472,12 @@ func TestMakeRemapDotConfigFirstExcludedSecondIncluded(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -1445,7 +1515,7 @@ func TestMakeRemapDotConfigAnyMap(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -1518,7 +1588,12 @@ func TestMakeRemapDotConfigAnyMap(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -1567,7 +1642,7 @@ func TestMakeRemapDotConfigEdgeMissingRemapData(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -1892,7 +1967,12 @@ func TestMakeRemapDotConfigEdgeMissingRemapData(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -1931,7 +2011,7 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacement(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -1976,7 +2056,12 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacement(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2032,7 +2117,7 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacementHTTP(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2077,7 +2162,12 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacementHTTP(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2133,7 +2223,7 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacementHTTPS(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2178,7 +2268,12 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacementHTTPS(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2234,7 +2329,7 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacementHTTPToHTTPS(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2279,7 +2374,12 @@ func TestMakeRemapDotConfigEdgeHostRegexReplacementHTTPToHTTPS(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2335,7 +2435,7 @@ func TestMakeRemapDotConfigEdgeRemapUnderscoreHTTPReplace(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2380,7 +2480,12 @@ func TestMakeRemapDotConfigEdgeRemapUnderscoreHTTPReplace(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2432,7 +2537,7 @@ func TestMakeRemapDotConfigEdgeDSCPRemap(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2477,7 +2582,12 @@ func TestMakeRemapDotConfigEdgeDSCPRemap(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2529,7 +2639,7 @@ func TestMakeRemapDotConfigEdgeNoDSCPRemap(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2574,7 +2684,12 @@ func TestMakeRemapDotConfigEdgeNoDSCPRemap(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2626,7 +2741,7 @@ func TestMakeRemapDotConfigEdgeHeaderRewrite(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2671,7 +2786,12 @@ func TestMakeRemapDotConfigEdgeHeaderRewrite(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2727,7 +2847,7 @@ func TestMakeRemapDotConfigEdgeHeaderRewriteEmpty(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2772,7 +2892,12 @@ func TestMakeRemapDotConfigEdgeHeaderRewriteEmpty(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2828,7 +2953,7 @@ func TestMakeRemapDotConfigEdgeHeaderRewriteNil(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2873,7 +2998,12 @@ func TestMakeRemapDotConfigEdgeHeaderRewriteNil(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -2929,7 +3059,7 @@ func TestMakeRemapDotConfigEdgeSigningURLSig(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -2974,7 +3104,12 @@ func TestMakeRemapDotConfigEdgeSigningURLSig(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3025,7 +3160,7 @@ func TestMakeRemapDotConfigEdgeSigningURISigning(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3070,7 +3205,12 @@ func TestMakeRemapDotConfigEdgeSigningURISigning(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3121,7 +3261,7 @@ func TestMakeRemapDotConfigEdgeSigningNone(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3166,7 +3306,12 @@ func TestMakeRemapDotConfigEdgeSigningNone(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3217,7 +3362,7 @@ func TestMakeRemapDotConfigEdgeSigningEmpty(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3262,7 +3407,12 @@ func TestMakeRemapDotConfigEdgeSigningEmpty(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3313,7 +3463,7 @@ func TestMakeRemapDotConfigEdgeSigningWrong(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3358,7 +3508,12 @@ func TestMakeRemapDotConfigEdgeSigningWrong(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3409,7 +3564,7 @@ func TestMakeRemapDotConfigEdgeQStringDropAtEdge(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3454,7 +3609,12 @@ func TestMakeRemapDotConfigEdgeQStringDropAtEdge(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3503,7 +3663,7 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUp(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3548,7 +3708,12 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUp(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3601,7 +3766,7 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpWithCacheKeyParameter(t *testi
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3646,7 +3811,12 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpWithCacheKeyParameter(t *testi
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3698,7 +3868,7 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpCacheURLParam(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3743,7 +3913,12 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpCacheURLParam(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3791,7 +3966,7 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpCacheURLParamCacheURL(t *testi
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3836,7 +4011,12 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpCacheURLParamCacheURL(t *testi
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3891,7 +4071,7 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpCacheURLParamCacheURLAndDSCach
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -3936,7 +4116,12 @@ func TestMakeRemapDotConfigEdgeQStringIgnorePassUpCacheURLParamCacheURLAndDSCach
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -3995,7 +4180,7 @@ func TestMakeRemapDotConfigMidQStringIgnorePassUpCacheURLParamCacheURLAndDSCache
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4040,7 +4225,12 @@ func TestMakeRemapDotConfigMidQStringIgnorePassUpCacheURLParamCacheURLAndDSCache
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4096,7 +4286,7 @@ func TestMakeRemapDotConfigEdgeCacheURL(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4141,7 +4331,12 @@ func TestMakeRemapDotConfigEdgeCacheURL(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4192,7 +4387,7 @@ func TestMakeRemapDotConfigEdgeCacheKeyParams(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4237,7 +4432,12 @@ func TestMakeRemapDotConfigEdgeCacheKeyParams(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4300,7 +4500,7 @@ func TestMakeRemapDotConfigEdgeRegexRemap(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4345,7 +4545,12 @@ func TestMakeRemapDotConfigEdgeRegexRemap(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4400,7 +4605,7 @@ func TestMakeRemapDotConfigEdgeRegexRemapEmpty(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4445,7 +4650,12 @@ func TestMakeRemapDotConfigEdgeRegexRemapEmpty(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4496,7 +4706,7 @@ func TestMakeRemapDotConfigEdgeRangeRequestNil(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4541,7 +4751,12 @@ func TestMakeRemapDotConfigEdgeRangeRequestNil(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4596,7 +4811,7 @@ func TestMakeRemapDotConfigEdgeRangeRequestDontCache(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4641,7 +4856,12 @@ func TestMakeRemapDotConfigEdgeRangeRequestDontCache(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4696,7 +4916,7 @@ func TestMakeRemapDotConfigEdgeRangeRequestBGFetch(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4741,7 +4961,12 @@ func TestMakeRemapDotConfigEdgeRangeRequestBGFetch(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4796,7 +5021,7 @@ func TestMakeRemapDotConfigEdgeRangeRequestSlice(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4842,7 +5067,12 @@ func TestMakeRemapDotConfigEdgeRangeRequestSlice(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -4901,7 +5131,7 @@ func TestMakeRemapDotConfigRawRemapRangeDirective(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -4947,7 +5177,12 @@ func TestMakeRemapDotConfigRawRemapRangeDirective(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5019,7 +5254,7 @@ func TestMakeRemapDotConfigRawRemapWithoutRangeDirective(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5065,7 +5300,12 @@ func TestMakeRemapDotConfigRawRemapWithoutRangeDirective(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5131,7 +5371,7 @@ func TestMakeRemapDotConfigEdgeRangeRequestCache(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5176,7 +5416,12 @@ func TestMakeRemapDotConfigEdgeRangeRequestCache(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5231,7 +5476,7 @@ func TestMakeRemapDotConfigEdgeFQPacingNil(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5276,7 +5521,12 @@ func TestMakeRemapDotConfigEdgeFQPacingNil(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5327,7 +5577,7 @@ func TestMakeRemapDotConfigEdgeFQPacingNegative(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5372,7 +5622,12 @@ func TestMakeRemapDotConfigEdgeFQPacingNegative(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5423,7 +5678,7 @@ func TestMakeRemapDotConfigEdgeFQPacingZero(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5468,7 +5723,12 @@ func TestMakeRemapDotConfigEdgeFQPacingZero(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5519,7 +5779,7 @@ func TestMakeRemapDotConfigEdgeFQPacingPositive(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5564,7 +5824,12 @@ func TestMakeRemapDotConfigEdgeFQPacingPositive(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5619,7 +5884,7 @@ func TestMakeRemapDotConfigEdgeDNS(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5664,7 +5929,12 @@ func TestMakeRemapDotConfigEdgeDNS(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5715,7 +5985,7 @@ func TestMakeRemapDotConfigEdgeDNSNoRoutingName(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5760,7 +6030,12 @@ func TestMakeRemapDotConfigEdgeDNSNoRoutingName(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5801,7 +6076,7 @@ func TestMakeRemapDotConfigEdgeRegexTypeNil(t *testing.T) {
 		CDN:                           "mycdn",
 		CDNID:                         43,
 		DomainName:                    "mydomain",
-		HostName:                      "myhost",
+		HostName:                      string(serverName),
 		HTTPSPort:                     12443,
 		ID:                            44,
 		IP:                            "192.168.2.4",
@@ -5846,7 +6121,12 @@ func TestMakeRemapDotConfigEdgeRegexTypeNil(t *testing.T) {
 		},
 	}
 
-	txt := MakeRemapDotConfig(serverName, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData)
+	server := serverInfoToServer(serverInfo)
+	topologies := []tc.Topology{}
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
 
 	txt = strings.TrimSpace(txt)
 
@@ -5859,3 +6139,333 @@ func TestMakeRemapDotConfigEdgeRegexTypeNil(t *testing.T) {
 	}
 
 }
+
+func TestMakeRemapDotConfigTopologies(t *testing.T) {
+	serverName := tc.CacheName("server0")
+	toToolName := "to0"
+	toURL := "trafficops.example.net"
+	atsMajorVersion := 7
+
+	cacheURLConfigParams := map[string]string{
+		"not_location": "notinconfig",
+	}
+
+	dsProfilesCacheKeyConfigParams := map[int]map[string]string{
+		46: map[string]string{
+			"cachekeykey": "cachekeyval",
+		},
+	}
+
+	serverPackageParamData := map[string]string{
+		"serverpkgval": "serverpkgval __HOSTNAME__ foo",
+	}
+
+	serverInfo := &ServerInfo{
+		CacheGroupID:                  42,
+		CDN:                           "mycdn",
+		CDNID:                         43,
+		DomainName:                    "mydomain",
+		HostName:                      string(serverName),
+		HTTPSPort:                     12443,
+		ID:                            44,
+		IP:                            "192.168.2.4",
+		ParentCacheGroupID:            45,
+		ParentCacheGroupType:          "CGType4",
+		ProfileID:                     46,
+		ProfileName:                   "MyProfile",
+		Port:                          12080,
+		SecondaryParentCacheGroupID:   47,
+		SecondaryParentCacheGroupType: "MySecondaryParentCG",
+		Type:                          "EDGE",
+	}
+
+	remapDSData := []RemapConfigDSData{
+		RemapConfigDSData{
+			ID:                       48,
+			Type:                     "HTTP_LIVE",
+			OriginFQDN:               util.StrPtr("origin0.example.test"),
+			MidHeaderRewrite:         util.StrPtr("mymidrewrite"),
+			CacheURL:                 util.StrPtr("mycacheurl"),
+			RangeRequestHandling:     util.IntPtr(0),
+			CacheKeyConfigParams:     map[string]string{"cachekeyparamname": "cachekeyparamval"},
+			RemapText:                util.StrPtr("myremaptext"),
+			EdgeHeaderRewrite:        util.StrPtr("myedgeheaderrewrite"),
+			SigningAlgorithm:         util.StrPtr("url_sig"),
+			Name:                     "mydsname0",
+			QStringIgnore:            util.IntPtr(0),
+			RegexRemap:               util.StrPtr("myregexremap"),
+			FQPacingRate:             util.IntPtr(0),
+			DSCP:                     0,
+			RoutingName:              util.StrPtr("myroutingname"),
+			MultiSiteOrigin:          util.StrPtr("mymso"),
+			Pattern:                  util.StrPtr("myregexpattern"),
+			RegexType:                util.StrPtr(string(tc.DSMatchTypeHostRegex)),
+			Domain:                   util.StrPtr("mydomain"),
+			RegexSetNumber:           util.StrPtr("myregexsetnum"),
+			OriginShield:             util.StrPtr("myoriginshield"),
+			ProfileID:                util.IntPtr(49),
+			Protocol:                 util.IntPtr(0),
+			AnonymousBlockingEnabled: util.BoolPtr(false),
+			Active:                   true,
+			Topology:                 "t0",
+		},
+		RemapConfigDSData{
+			ID:                       48,
+			Type:                     "HTTP_LIVE",
+			OriginFQDN:               util.StrPtr("origin1.example.test"),
+			MidHeaderRewrite:         util.StrPtr("mymidrewrite"),
+			CacheURL:                 util.StrPtr("mycacheurl"),
+			RangeRequestHandling:     util.IntPtr(0),
+			CacheKeyConfigParams:     map[string]string{"cachekeyparamname": "cachekeyparamval"},
+			RemapText:                util.StrPtr("myremaptext"),
+			EdgeHeaderRewrite:        util.StrPtr("myedgeheaderrewrite"),
+			SigningAlgorithm:         util.StrPtr("url_sig"),
+			Name:                     "mydsname1",
+			QStringIgnore:            util.IntPtr(0),
+			RegexRemap:               util.StrPtr("myregexremap"),
+			FQPacingRate:             util.IntPtr(0),
+			DSCP:                     0,
+			RoutingName:              util.StrPtr("myroutingname"),
+			MultiSiteOrigin:          util.StrPtr("mymso"),
+			Pattern:                  util.StrPtr("myregexpattern"),
+			RegexType:                util.StrPtr(string(tc.DSMatchTypeHostRegex)),
+			Domain:                   util.StrPtr("mydomain"),
+			RegexSetNumber:           util.StrPtr("myregexsetnum"),
+			OriginShield:             util.StrPtr("myoriginshield"),
+			ProfileID:                util.IntPtr(49),
+			Protocol:                 util.IntPtr(0),
+			AnonymousBlockingEnabled: util.BoolPtr(false),
+			Active:                   true,
+			Topology:                 "t1",
+		},
+	}
+
+	server := serverInfoToServer(serverInfo)
+	server.Cachegroup = "edgeCG"
+
+	topologies := []tc.Topology{
+		tc.Topology{
+			Name: "t0",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "edgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+		tc.Topology{
+			Name: "t1",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "otherEdgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+	}
+
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{}
+
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
+
+	txt = strings.TrimSpace(txt)
+
+	testComment(t, txt, string(serverName), toToolName, toURL)
+
+	if !strings.Contains(txt, "origin0") {
+		t.Errorf("expected to contain DS with edge in topology, actual '%v'", txt)
+	}
+
+	if strings.Contains(txt, "origin1") {
+		t.Errorf("expected to not contain DS with edge not in topology, actual '%v'", txt)
+	}
+}
+
+func TestMakeRemapDotConfigTopologyCapabilities(t *testing.T) {
+	serverName := tc.CacheName("server0")
+	toToolName := "to0"
+	toURL := "trafficops.example.net"
+	atsMajorVersion := 7
+
+	cacheURLConfigParams := map[string]string{
+		"not_location": "notinconfig",
+	}
+
+	dsProfilesCacheKeyConfigParams := map[int]map[string]string{
+		46: map[string]string{
+			"cachekeykey": "cachekeyval",
+		},
+	}
+
+	serverPackageParamData := map[string]string{
+		"serverpkgval": "serverpkgval __HOSTNAME__ foo",
+	}
+
+	serverInfo := &ServerInfo{
+		CacheGroupID:                  42,
+		CDN:                           "mycdn",
+		CDNID:                         43,
+		DomainName:                    "mydomain",
+		HostName:                      string(serverName),
+		HTTPSPort:                     12443,
+		ID:                            44,
+		IP:                            "192.168.2.4",
+		ParentCacheGroupID:            45,
+		ParentCacheGroupType:          "CGType4",
+		ProfileID:                     46,
+		ProfileName:                   "MyProfile",
+		Port:                          12080,
+		SecondaryParentCacheGroupID:   47,
+		SecondaryParentCacheGroupType: "MySecondaryParentCG",
+		Type:                          "EDGE",
+	}
+
+	remapDSData := []RemapConfigDSData{
+		RemapConfigDSData{
+			ID:                       48,
+			Type:                     "HTTP_LIVE",
+			OriginFQDN:               util.StrPtr("origin0.example.test"),
+			MidHeaderRewrite:         util.StrPtr("mymidrewrite"),
+			CacheURL:                 util.StrPtr("mycacheurl"),
+			RangeRequestHandling:     util.IntPtr(0),
+			CacheKeyConfigParams:     map[string]string{"cachekeyparamname": "cachekeyparamval"},
+			RemapText:                util.StrPtr("myremaptext"),
+			EdgeHeaderRewrite:        util.StrPtr("myedgeheaderrewrite"),
+			SigningAlgorithm:         util.StrPtr("url_sig"),
+			Name:                     "mydsname0",
+			QStringIgnore:            util.IntPtr(0),
+			RegexRemap:               util.StrPtr("myregexremap"),
+			FQPacingRate:             util.IntPtr(0),
+			DSCP:                     0,
+			RoutingName:              util.StrPtr("myroutingname"),
+			MultiSiteOrigin:          util.StrPtr("mymso"),
+			Pattern:                  util.StrPtr("myregexpattern"),
+			RegexType:                util.StrPtr(string(tc.DSMatchTypeHostRegex)),
+			Domain:                   util.StrPtr("mydomain"),
+			RegexSetNumber:           util.StrPtr("myregexsetnum"),
+			OriginShield:             util.StrPtr("myoriginshield"),
+			ProfileID:                util.IntPtr(49),
+			Protocol:                 util.IntPtr(0),
+			AnonymousBlockingEnabled: util.BoolPtr(false),
+			Active:                   true,
+			Topology:                 "t0",
+		},
+		RemapConfigDSData{
+			ID:                       48,
+			Type:                     "HTTP_LIVE",
+			OriginFQDN:               util.StrPtr("origin1.example.test"),
+			MidHeaderRewrite:         util.StrPtr("mymidrewrite"),
+			CacheURL:                 util.StrPtr("mycacheurl"),
+			RangeRequestHandling:     util.IntPtr(0),
+			CacheKeyConfigParams:     map[string]string{"cachekeyparamname": "cachekeyparamval"},
+			RemapText:                util.StrPtr("myremaptext"),
+			EdgeHeaderRewrite:        util.StrPtr("myedgeheaderrewrite"),
+			SigningAlgorithm:         util.StrPtr("url_sig"),
+			Name:                     "mydsname1",
+			QStringIgnore:            util.IntPtr(0),
+			RegexRemap:               util.StrPtr("myregexremap"),
+			FQPacingRate:             util.IntPtr(0),
+			DSCP:                     0,
+			RoutingName:              util.StrPtr("myroutingname"),
+			MultiSiteOrigin:          util.StrPtr("mymso"),
+			Pattern:                  util.StrPtr("myregexpattern"),
+			RegexType:                util.StrPtr(string(tc.DSMatchTypeHostRegex)),
+			Domain:                   util.StrPtr("mydomain"),
+			RegexSetNumber:           util.StrPtr("myregexsetnum"),
+			OriginShield:             util.StrPtr("myoriginshield"),
+			ProfileID:                util.IntPtr(49),
+			Protocol:                 util.IntPtr(0),
+			AnonymousBlockingEnabled: util.BoolPtr(false),
+			Active:                   true,
+			Topology:                 "t1",
+		},
+	}
+
+	server := serverInfoToServer(serverInfo)
+	server.Cachegroup = "edgeCG"
+
+	topologies := []tc.Topology{
+		tc.Topology{
+			Name: "t0",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "edgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+		tc.Topology{
+			Name: "t1",
+			Nodes: []tc.TopologyNode{
+				tc.TopologyNode{
+					Cachegroup: "otherEdgeCG",
+					Parents:    []int{1},
+				},
+				tc.TopologyNode{
+					Cachegroup: "midCG",
+				},
+			},
+		},
+	}
+
+	cgs := []tc.CacheGroupNullable{}
+	serverCapabilities := map[int]map[ServerCapability]struct{}{
+		serverInfo.ID: map[ServerCapability]struct{}{
+			"capx": {},
+		},
+	}
+	dsRequiredCapabilities := map[int]map[ServerCapability]struct{}{
+		remapDSData[0].ID: map[ServerCapability]struct{}{
+			"capx": {},
+		},
+	}
+
+	txt := MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
+
+	txt = strings.TrimSpace(txt)
+
+	testComment(t, txt, string(serverName), toToolName, toURL)
+
+	if !strings.Contains(txt, "origin0") {
+		t.Errorf("expected to contain DS with edge in topology, actual '%v'", txt)
+	}
+
+	if strings.Contains(txt, "origin1") {
+		t.Errorf("expected to not contain DS with edge not in topology, actual '%v'", txt)
+	}
+
+	serverCapabilities[serverInfo.ID] = map[ServerCapability]struct{}{}
+	txt = MakeRemapDotConfig(server, toToolName, toURL, atsMajorVersion, cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapDSData, topologies, cgs, serverCapabilities, dsRequiredCapabilities)
+	if strings.Contains(txt, "origin1") {
+		t.Errorf("expected to not contain DS with edge missing required capabilities, actual '%v'", txt)
+	}
+}
+
+// serverInfoToServer is for tests, and does NOT copy all data, do NOT use for real code.
+func serverInfoToServer(si *ServerInfo) tc.Server {
+	return tc.Server{
+		CachegroupID: si.CacheGroupID,
+		CDNName:      string(si.CDN),
+		CDNID:        si.CDNID,
+		DomainName:   si.DomainName,
+		HostName:     si.HostName,
+		HTTPSPort:    si.HTTPSPort,
+		ID:           si.ID,
+		IPAddress:    si.IP,
+		ProfileID:    int(si.ProfileID),
+		Profile:      si.ProfileName,
+		TCPPort:      si.Port,
+		Type:         si.Type,
+	}
+}
diff --git a/lib/go-atscfg/topologyheaderrewritedotconfig.go b/lib/go-atscfg/topologyheaderrewritedotconfig.go
new file mode 100644
index 0000000..094bcfc
--- /dev/null
+++ b/lib/go-atscfg/topologyheaderrewritedotconfig.go
@@ -0,0 +1,135 @@
+package atscfg
+
+/*
+ * 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 (
+	"math"
+	"strconv"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+const HeaderRewriteFirstPrefix = HeaderRewritePrefix + "first_"
+const HeaderRewriteInnerPrefix = HeaderRewritePrefix + "inner_"
+const HeaderRewriteLastPrefix = HeaderRewritePrefix + "last_"
+
+func FirstHeaderRewriteConfigFileName(dsName string) string {
+	return HeaderRewriteFirstPrefix + dsName + ConfigSuffix
+}
+
+func InnerHeaderRewriteConfigFileName(dsName string) string {
+	return HeaderRewriteInnerPrefix + dsName + ConfigSuffix
+}
+
+func LastHeaderRewriteConfigFileName(dsName string) string {
+	return HeaderRewriteLastPrefix + dsName + ConfigSuffix
+}
+
+func MakeTopologyHeaderRewriteDotConfig(
+	server tc.Server,
+	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
+	toURL string, // tm.url global parameter (TODO: cache itself?)
+	ds tc.DeliveryServiceNullable,
+	topologies []tc.Topology,
+	cacheGroupArr []tc.CacheGroupNullable,
+	servers []tc.Server,
+	serverCapabilities map[int]map[ServerCapability]struct{},
+	dsRequiredCapabilities map[ServerCapability]struct{},
+) string {
+	text := GenericHeaderComment(server.HostName, toToolName, toURL)
+
+	cacheGroups := MakeCGMap(cacheGroupArr)
+
+	if ds.Topology == nil || *ds.Topology == "" {
+		log.Errorln("Config generation: Topology Header Rewrite called for DS '" + *ds.XMLID + "' with no Topology! This should never be called, a DS with no topology should never have a First, Inner, or Last Header Rewrite config in the list of config files! Returning blank config!")
+		return text
+	}
+
+	nameTopologies := MakeTopologyNameMap(topologies)
+	topology := nameTopologies[TopologyName(*ds.Topology)]
+	if topology.Name == "" {
+		log.Errorln("Config generation: Topology Header Rewrite called for DS '" + *ds.XMLID + "' but its Topology '" + *ds.Topology + "' not found in list of topologies! Returning blank config!")
+		return text
+	}
+
+	log.Errorf("DEBUG topo MakeTopologyHeaderRewriteDotConfig calling getTopologyPlacement cg '" + server.Cachegroup + "'\n")
+	serverPlacement := getTopologyPlacement(tc.CacheGroupName(server.Cachegroup), topology, cacheGroups)
+	if !serverPlacement.InTopology {
+		log.Errorln("Config generation: Topology Header Rewrite called for DS '" + *ds.XMLID + "' on server '" + server.HostName + "' is not in the DS's topology! Returning blank config!")
+		return text
+	}
+
+	headerRewrite := (*string)(nil)
+	switch serverPlacement.CacheTier {
+	case TopologyCacheTierFirst:
+		headerRewrite = ds.FirstHeaderRewrite
+	case TopologyCacheTierInner:
+		headerRewrite = ds.InnerHeaderRewrite
+	case TopologyCacheTierLast:
+		headerRewrite = ds.LastHeaderRewrite
+	default:
+		log.Errorln("Config generation: Topology Header Rewrite called for DS '" + *ds.XMLID + "' on server '" + server.HostName + "' got unknown topology cache tier '" + string(serverPlacement.CacheTier) + "'! Returning blank config!")
+		return text
+	}
+
+	if serverPlacement.CacheTier == TopologyCacheTierLast && ds.MaxOriginConnections != nil && *ds.MaxOriginConnections > 0 {
+		lastTierCacheCount := GetTopologyDSServerCount(dsRequiredCapabilities, tc.CacheGroupName(server.Cachegroup), servers, serverCapabilities)
+
+		maxOriginConnectionsPerServer := int(math.Round(float64(*ds.MaxOriginConnections) / float64(lastTierCacheCount)))
+		if maxOriginConnectionsPerServer < 1 {
+			maxOriginConnectionsPerServer = 1
+		}
+
+		text += "cond %{REMAP_PSEUDO_HOOK}\nset-config proxy.config.http.origin_max_connections " + strconv.Itoa(maxOriginConnectionsPerServer)
+		if headerRewrite == nil || *headerRewrite == "" {
+			text += " [L]"
+		} else {
+			text += "\n"
+		}
+	}
+
+	if headerRewrite != nil && *headerRewrite != "" {
+		text += *headerRewrite
+	}
+	text += "\n"
+	return text
+}
+
+// GetTopologyDSServerCount returns the number of servers in cg which will be used to serve ds.
+// This should only be used for DSes with Topologies.
+// It returns all servers in CG with the Capabilities of ds in cg.
+// It will not be the number of servers for Delivery Services not using Topologies, which use DeliveryService-Server assignments instead.
+func GetTopologyDSServerCount(dsRequiredCapabilities map[ServerCapability]struct{}, cg tc.CacheGroupName, servers []tc.Server, serverCapabilities map[int]map[ServerCapability]struct{}) int {
+	count := 0
+	for _, sv := range servers {
+		if sv.Cachegroup != string(cg) {
+			continue
+		}
+		if sv.Status != string(tc.CacheStatusReported) && sv.Status != string(tc.CacheStatusOnline) {
+			continue
+		}
+		if !HasRequiredCapabilities(serverCapabilities[sv.ID], dsRequiredCapabilities) {
+			continue
+		}
+		count++
+	}
+	return count
+}
diff --git a/traffic_ops/traffic_ops_golang/ats/atsserver/hostingdotconfig.go b/traffic_ops/traffic_ops_golang/ats/atsserver/hostingdotconfig.go
index c441687..424e303 100644
--- a/traffic_ops/traffic_ops_golang/ats/atsserver/hostingdotconfig.go
+++ b/traffic_ops/traffic_ops_golang/ats/atsserver/hostingdotconfig.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-atscfg"
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/ats"
 )
@@ -73,13 +74,41 @@ func GetHostingDotConfig(w http.ResponseWriter, r *http.Request) {
 		params[name] = vals[0]
 	}
 
-	origins, err := GetServerHostingOrigins(inf.Tx.Tx, serverName, serverType)
+	origins, dsNames, err := GetServerHostingOrigins(inf.Tx.Tx, serverName, serverType)
 	if err != nil {
 		api.HandleErr(w, r, inf.Tx.Tx, http.StatusInternalServerError, nil, errors.New("getting server '"+string(serverName)+"' hosting origins: "+err.Error()))
 		return
+	} else if len(origins) != len(dsNames) {
+		api.HandleErr(w, r, inf.Tx.Tx, http.StatusInternalServerError, nil, errors.New("getting server '"+string(serverName)+"' hosting origins: mismatched origins and ds names!")) // should never happen
+		return
+	}
+
+	topologies, err := ats.GetTopologies(inf.Tx.Tx)
+	if err != nil {
+		api.HandleErr(w, r, inf.Tx.Tx, http.StatusInternalServerError, nil, errors.New("getting topologies: "+err.Error()))
+		return
+	}
+
+	cg, err := ats.GetServerCacheGroup(inf.Tx.Tx, serverName)
+	if err != nil {
+		api.HandleErr(w, r, inf.Tx.Tx, http.StatusInternalServerError, nil, errors.New("getting server cachegroup: "+err.Error()))
+		return
+	}
+
+	// TODO this is risky, because it will cause generation errors if atscfg is changed to use more data from Server or DS.
+	// But it would require a lot of abstraction to add the generic server and DS loading from the rest of TO, so new fields aren't missed.
+	// But, this endpoint is legacy and should never be used.
+	// If TO config gen sticks around, we should add that abstraction, to prevent future bugs.
+	server := tc.Server{HostName: string(serverName), Cachegroup: string(cg)}
+	dses := []tc.DeliveryServiceNullable{}
+	for i := 0; i < len(origins); i++ {
+		ds := tc.DeliveryServiceNullable{}
+		ds.OrgServerFQDN = util.StrPtr(origins[i])
+		ds.XMLID = util.StrPtr(string(dsNames[i]))
+		dses = append(dses, ds)
 	}
 
-	txt := atscfg.MakeHostingDotConfig(serverName, toToolName, toURL, params, origins)
+	txt := atscfg.MakeHostingDotConfig(server, toToolName, toURL, params, dses, topologies)
 
 	w.Header().Set("Content-Type", "text/plain")
 	w.Write([]byte(txt))
@@ -87,10 +116,11 @@ func GetHostingDotConfig(w http.ResponseWriter, r *http.Request) {
 
 // GetServerHostingOrigins returns the list of origins on delivery services assigned to the given server, to be used in the ATS config file.
 // It returns only LIVE_NATNL delivery services, for mids; and only LIVE and LIVE_NATNL services for edges.
-func GetServerHostingOrigins(tx *sql.Tx, serverName tc.CacheName, serverType tc.CacheType) ([]string, error) {
+func GetServerHostingOrigins(tx *sql.Tx, serverName tc.CacheName, serverType tc.CacheType) ([]string, []tc.DeliveryServiceName, error) {
 	qry := `
 SELECT
-  DISTINCT(SELECT o.protocol::text || '://' || o.fqdn || rtrim(concat(':', o.port::text), ':')) as org_server_fqdn
+  DISTINCT(SELECT o.protocol::text || '://' || o.fqdn || rtrim(concat(':', o.port::text), ':')) as org_server_fqdn,
+  ds.xml_id as ds_name
 FROM
   deliveryservice ds
   JOIN deliveryservice_server dss on dss.deliveryservice = ds.id
@@ -118,17 +148,20 @@ WHERE
 
 	rows, err := tx.Query(qry, serverName)
 	if err != nil {
-		return nil, errors.New("querying: " + err.Error())
+		return nil, nil, errors.New("querying: " + err.Error())
 	}
 	defer rows.Close()
 
 	origins := []string{}
+	dses := []tc.DeliveryServiceName{}
 	for rows.Next() {
 		origin := ""
-		if err := rows.Scan(&origin); err != nil {
-			return nil, errors.New("scanning: " + err.Error())
+		ds := tc.DeliveryServiceName("")
+		if err := rows.Scan(&origin, &ds); err != nil {
+			return nil, nil, errors.New("scanning: " + err.Error())
 		}
 		origins = append(origins, origin)
+		dses = append(dses, ds)
 	}
-	return origins, nil
+	return origins, dses, nil
 }
diff --git a/traffic_ops/traffic_ops_golang/ats/atsserver/meta.go b/traffic_ops/traffic_ops_golang/ats/atsserver/meta.go
index d6d94c3..68b9bf5 100644
--- a/traffic_ops/traffic_ops_golang/ats/atsserver/meta.go
+++ b/traffic_ops/traffic_ops_golang/ats/atsserver/meta.go
@@ -84,7 +84,20 @@ func GetConfigMetaData(w http.ResponseWriter, r *http.Request) {
 		return
 	}
 
-	txt := atscfg.MakeMetaConfig(tc.CacheName(serverName), server, tmParams.URL, tmParams.ReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dsNames)
+	// DSes are only used by the Meta generation for inferring location Parameters, which can't be done anyway since TO doesn't have a local ATS install config directory.
+	// This would be fragile, and break if the Meta were changed to use them for anything else,
+	// Except this endpoint is going away shortly anyway, and is not used by ORT.
+	dses := map[tc.DeliveryServiceName]tc.DeliveryServiceNullable{}
+	for name, _ := range dsNames {
+		dses[name] = tc.DeliveryServiceNullable{}
+	}
+
+	// This endpoint should not be used for new features with Topologies.
+	// If a new TO install is using Topologies, it must be upgraded to the latest ORT (which doesn't use this endpoint).
+	cgs := []tc.CacheGroupNullable{}
+	topologies := []tc.Topology{}
+
+	txt := atscfg.MakeMetaConfig(tc.CacheName(serverName), server, tmParams.URL, tmParams.ReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cgs, topologies)
 	w.Header().Set(rfc.ContentType, rfc.ApplicationJSON)
 	w.Write([]byte(txt))
 }
diff --git a/traffic_ops/traffic_ops_golang/ats/db.go b/traffic_ops/traffic_ops_golang/ats/db.go
index bce3488..bd80e66 100644
--- a/traffic_ops/traffic_ops_golang/ats/db.go
+++ b/traffic_ops/traffic_ops_golang/ats/db.go
@@ -1091,3 +1091,116 @@ WHERE
 	}
 	return dsCaps, nil
 }
+
+func GetTopologies(tx *sql.Tx) ([]tc.Topology, error) {
+	// TODO if this sticks around, abstract topology/topologies.go to de-duplicate.
+	// TO-side config files should be going away soon.
+	qry := `
+SELECT t.name, t.description, t.last_updated,
+tc.id, tc.cachegroup,
+	(SELECT COALESCE (ARRAY_AGG (CAST (tcp.parent as INT) ORDER BY tcp.rank ASC)) AS parents
+	FROM topology_cachegroup tc2
+	JOIN topology_cachegroup_parents tcp ON tc2.id = tcp.child
+	WHERE tc2.topology = tc.topology
+	AND tc2.cachegroup = tc.cachegroup
+	)
+FROM topology t
+JOIN topology_cachegroup tc on t.name = tc.topology
+`
+	rows, err := tx.Query(qry)
+	if err != nil {
+		return nil, errors.New("querying: " + err.Error())
+	}
+	defer log.Close(rows, "unable to close DB connection")
+
+	topologies := map[string]*tc.Topology{}
+	for rows.Next() {
+		tp := tc.Topology{}
+		tn := tc.TopologyNode{}
+		parents := pq.Int64Array{}
+		if err = rows.Scan(&tp.Name, &tp.Description, &tp.LastUpdated, &tn.Id, &tn.Cachegroup, &parents); err != nil {
+			return nil, errors.New("scanning: " + err.Error())
+		}
+		for _, id := range parents {
+			tn.Parents = append(tn.Parents, int(id))
+		}
+		if _, exists := topologies[tp.Name]; !exists {
+			topologies[tp.Name] = &tp
+		} else {
+			tp = *topologies[tp.Name]
+		}
+		tp.Nodes = append(tp.Nodes, tn)
+		topologies[tp.Name] = &tp
+	}
+
+	tops := []tc.Topology{}
+	for _, topology := range topologies {
+		nodeMap := map[int]int{}
+		for index, node := range topology.Nodes {
+			nodeMap[node.Id] = index
+		}
+		for _, node := range topology.Nodes {
+			for parentIndex := 0; parentIndex < len(node.Parents); parentIndex++ {
+				node.Parents[parentIndex] = nodeMap[node.Parents[parentIndex]]
+			}
+		}
+		tops = append(tops, *topology)
+	}
+	return tops, nil
+}
+
+func GetServerCacheGroup(tx *sql.Tx, server tc.CacheName) (tc.CacheGroupName, error) {
+	qry := `
+SELECT
+  cg.name
+FROM
+  cachegroup cg
+  JOIN server s on s.cachegroup = cg.id
+WHERE
+  s.host_name = $1
+`
+	cg := tc.CacheGroupName("")
+	if err := tx.QueryRow(qry, server).Scan(&cg); err != nil {
+		if err == sql.ErrNoRows {
+			return "", errors.New("not found")
+		}
+		return "", errors.New("querying: " + err.Error())
+	}
+	return cg, nil
+}
+
+func GetConfigFileParams(tx *sql.Tx, configFile string) ([]tc.Parameter, error) {
+	qry := `
+SELECT
+  p.config_file,
+  p.id,
+  p.last_updated,
+  p.name,
+  p.value,
+  p.secure,
+  COALESCE(array_to_json(array_agg(pr.name) FILTER (WHERE pr.name IS NOT NULL)), '[]') AS profiles
+FROM
+  parameter p
+  LEFT JOIN profile_parameter pp ON p.id = pp.parameter
+  LEFT JOIN profile pr ON pp.profile = pr.id
+WHERE
+  p.config_file = $1
+GROUP BY p.config_file, p.id, p.last_updated, p.name, p.value, p.secure
+`
+	rows, err := tx.Query(qry, configFile)
+	if err != nil {
+		return nil, errors.New("querying: " + err.Error())
+	}
+	defer rows.Close()
+
+	params := []tc.Parameter{}
+	for rows.Next() {
+		p := tc.Parameter{}
+		if err = rows.Scan(&p.ConfigFile, &p.ID, &p.LastUpdated, &p.Name, &p.Value, &p.Secure, &p.Profiles); err != nil {
+			return nil, errors.New("scanning: " + err.Error())
+		}
+		params = append(params, p)
+	}
+
+	return params, nil
+}
diff --git a/traffic_ops_ort/atstccfg/cfgfile/cachedotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/cachedotconfig.go
index a9a5c4c..b4018b4 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/cachedotconfig.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/cachedotconfig.go
@@ -58,7 +58,7 @@ func GetConfigFileProfileCacheDotConfig(toData *config.TOData) (string, string,
 		if *ds.OrgServerFQDN == "" {
 			continue // TODO warn? err?
 		}
-		if _, ok := dsIDs[*ds.ID]; !ok {
+		if _, ok := dsIDs[*ds.ID]; !ok && ds.Topology == nil {
 			continue
 		}
 		origin := *ds.OrgServerFQDN
diff --git a/traffic_ops_ort/atstccfg/cfgfile/cacheurldotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/cacheurldotconfig.go
index 9e036ef..37c3c33 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/cacheurldotconfig.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/cacheurldotconfig.go
@@ -52,7 +52,7 @@ func GetConfigFileCDNCacheURL(toData *config.TOData, fileName string) (string, s
 		if ds.Type != nil && (*ds.Type == tc.DSTypeAnyMap || *ds.Type == tc.DSTypeSteering) {
 			continue
 		}
-		if len(dssMap[*ds.ID]) == 0 {
+		if len(dssMap[*ds.ID]) == 0 && ds.Topology == nil {
 			continue
 		}
 		dsesWithServers = append(dsesWithServers, ds)
diff --git a/traffic_ops_ort/atstccfg/cfgfile/cfgfile.go b/traffic_ops_ort/atstccfg/cfgfile/cfgfile.go
index 13c6d8c..a29a691 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/cfgfile.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/cfgfile.go
@@ -20,9 +20,7 @@ package cfgfile
  */
 
 import (
-	"encoding/json"
 	"errors"
-	"strconv"
 	"strings"
 	"time"
 
@@ -121,6 +119,27 @@ func GetTOData(cfg config.TCCfg) (*config.TOData, error) {
 			}
 			toData.DeliveryServices = dses
 
+			allDSesHaveTopologies := true
+			for _, ds := range toData.DeliveryServices {
+				if ds.CDNID == nil || *ds.CDNID != server.CDNID {
+					continue
+				}
+				if ds.Topology == nil {
+					allDSesHaveTopologies = false
+					break
+				}
+			}
+
+			dssF := func() error {
+				defer func(start time.Time) { log.Infof("dssF took %v\n", time.Since(start)) }(time.Now())
+				dss, err := cfg.TOClient.GetDeliveryServiceServers(nil, nil)
+				if err != nil {
+					return errors.New("getting delivery service servers: " + err.Error())
+				}
+				toData.DeliveryServiceServers = dss
+				return nil
+			}
+
 			uriSignKeysF := func() error {
 				defer func(start time.Time) { log.Infof("uriF took %v\n", time.Since(start)) }(time.Now())
 				uriSigningKeys := map[tc.DeliveryServiceName][]byte{}
@@ -177,6 +196,12 @@ func GetTOData(cfg config.TCCfg) (*config.TOData, error) {
 			if !cfg.RevalOnly {
 				fs = append([]func() error{uriSignKeysF, urlSigKeysF}, fs...) // skip keys for reval-only, which doesn't need them
 			}
+			if !cfg.RevalOnly && !allDSesHaveTopologies {
+				// skip DSS if reval-only (which doesn't need DSS)
+				// or if all DSes have Topologies (which don't use DSS)
+				fs = append([]func() error{dssF}, fs...)
+			}
+
 			return util.JoinErrs(runParallel(fs))
 		}
 		serverParamsF := func() error {
@@ -233,15 +258,6 @@ func GetTOData(cfg config.TCCfg) (*config.TOData, error) {
 		toData.ScopeParams = scopeParams
 		return nil
 	}
-	dssF := func() error {
-		defer func(start time.Time) { log.Infof("dssF took %v\n", time.Since(start)) }(time.Now())
-		dss, err := cfg.TOClient.GetDeliveryServiceServers(nil, nil)
-		if err != nil {
-			return errors.New("getting delivery service servers: " + err.Error())
-		}
-		toData.DeliveryServiceServers = dss
-		return nil
-	}
 	jobsF := func() error {
 		defer func(start time.Time) { log.Infof("jobsF took %v\n", time.Since(start)) }(time.Now())
 		jobs, err := cfg.TOClient.GetJobs() // TODO add cdn query param to jobs endpoint
@@ -301,10 +317,24 @@ func GetTOData(cfg config.TCCfg) (*config.TOData, error) {
 		return nil
 	}
 
+	topologiesF := func() error {
+		defer func(start time.Time) { log.Infof("topologiesF took %v\n", time.Since(start)) }(time.Now())
+		topologies, unsupported, err := cfg.TOClientNew.GetTopologies()
+		if err != nil {
+			return errors.New("getting topologies: " + err.Error())
+		}
+		if unsupported {
+			log.Warnln("Traffic Ops didn't support Topologies, topologies will be not be used for config generation!")
+			return nil
+		}
+		toData.Topologies = topologies
+		return nil
+	}
+
 	fs := []func() error{serversF, cgF, scopeParamsF, jobsF}
 	if !cfg.RevalOnly {
 		// skip data not needed for reval, if we're reval-only
-		fs = append([]func() error{dssF, dsrF, cacheKeyParamsF, parentConfigParamsF, capsF, dsCapsF}, fs...)
+		fs = append([]func() error{dsrF, cacheKeyParamsF, parentConfigParamsF, capsF, dsCapsF, topologiesF}, fs...)
 	}
 	errs := runParallel(fs)
 	return toData, util.JoinErrs(errs)
@@ -349,45 +379,6 @@ func FilterDSS(dsses []tc.DeliveryServiceServer, dsIDs map[int]struct{}, serverI
 	return filtered
 }
 
-// TCParamsToParamsWithProfiles unmarshals the Profiles that the tc struct doesn't.
-func TCParamsToParamsWithProfiles(tcParams []tc.Parameter) ([]ParameterWithProfiles, error) {
-	params := make([]ParameterWithProfiles, 0, len(tcParams))
-	for _, tcParam := range tcParams {
-		param := ParameterWithProfiles{Parameter: tcParam}
-
-		profiles := []string{}
-		if err := json.Unmarshal(tcParam.Profiles, &profiles); err != nil {
-			return nil, errors.New("unmarshalling JSON from parameter '" + strconv.Itoa(param.ID) + "': " + err.Error())
-		}
-		param.ProfileNames = profiles
-		param.Profiles = nil
-		params = append(params, param)
-	}
-	return params, nil
-}
-
-type ParameterWithProfiles struct {
-	tc.Parameter
-	ProfileNames []string
-}
-
-type ParameterWithProfilesMap struct {
-	tc.Parameter
-	ProfileNames map[string]struct{}
-}
-
-func ParameterWithProfilesToMap(tcParams []ParameterWithProfiles) []ParameterWithProfilesMap {
-	params := []ParameterWithProfilesMap{}
-	for _, tcParam := range tcParams {
-		param := ParameterWithProfilesMap{Parameter: tcParam.Parameter, ProfileNames: map[string]struct{}{}}
-		for _, profile := range tcParam.ProfileNames {
-			param.ProfileNames[profile] = struct{}{}
-		}
-		params = append(params, param)
-	}
-	return params
-}
-
 // FilterParams filters params and returns only the parameters which match configFile, name, and value.
 // If configFile, name, or value is the empty string, it is not filtered.
 // Returns a slice of parameters.
diff --git a/traffic_ops_ort/atstccfg/cfgfile/headerrewritedotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/headerrewritedotconfig.go
index 5896727..c691538 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/headerrewritedotconfig.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/headerrewritedotconfig.go
@@ -70,7 +70,7 @@ func GetConfigFileCDNHeaderRewrite(toData *config.TOData, fileName string) (stri
 		if server.CDNName != *tcDS.CDNName {
 			continue
 		}
-		if _, ok := dsServerIDs[server.ID]; !ok {
+		if _, ok := dsServerIDs[server.ID]; !ok && tcDS.Topology == nil {
 			continue
 		}
 		cfgServer, err := atscfg.HeaderRewriteServerFromServerNotNullable(server)
diff --git a/traffic_ops_ort/atstccfg/cfgfile/headerrewritemiddotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/headerrewritemiddotconfig.go
index 3df4e96..e3bfec9 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/headerrewritemiddotconfig.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/headerrewritemiddotconfig.go
@@ -52,19 +52,6 @@ func GetConfigFileCDNHeaderRewriteMid(toData *config.TOData, fileName string) (s
 		return "", "", "", errors.New("converting ds to config ds: " + err.Error())
 	}
 
-	dsServers := FilterDSS(toData.DeliveryServiceServers, map[int]struct{}{cfgDS.ID: {}}, nil)
-
-	dsServerIDs := map[int]struct{}{}
-	for _, dss := range dsServers {
-		if dss.Server == nil || dss.DeliveryService == nil {
-			continue // TODO warn?
-		}
-		if *dss.DeliveryService != *tcDS.ID {
-			continue
-		}
-		dsServerIDs[*dss.Server] = struct{}{}
-	}
-
 	serverCGs := map[tc.CacheGroupName]struct{}{}
 	for _, sv := range toData.Servers {
 		if sv.CDNName != toData.Server.CDNName {
diff --git a/traffic_ops_ort/atstccfg/cfgfile/hostingdotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/hostingdotconfig.go
index a88cb4f..f80ef4c 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/hostingdotconfig.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/hostingdotconfig.go
@@ -28,9 +28,6 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops_ort/atstccfg/config"
 )
 
-const ServerHostingDotConfigMidIncludeInactive = false
-const ServerHostingDotConfigEdgeIncludeInactive = true
-
 func GetConfigFileServerHostingDotConfig(toData *config.TOData) (string, string, string, error) {
 	fileParams := ParamsToMap(FilterParams(toData.ServerParams, atscfg.HostingConfigParamConfigFile, "", "", ""))
 
@@ -67,59 +64,49 @@ func GetConfigFileServerHostingDotConfig(toData *config.TOData) (string, string,
 		dsServerMap[*dss.DeliveryService][*dss.Server] = struct{}{}
 	}
 
-	hostingDSes := map[tc.DeliveryServiceName]tc.DeliveryServiceNullable{}
-
 	isMid := strings.HasPrefix(toData.Server.Type, tc.MidTypePrefix)
 
+	filteredDSes := []tc.DeliveryServiceNullable{}
 	for _, ds := range toData.DeliveryServices {
 		if ds.Active == nil || ds.Type == nil || ds.XMLID == nil || ds.CDNID == nil || ds.ID == nil || ds.OrgServerFQDN == nil {
 			// some DSes have nil origins. I think MSO? TODO: verify
 			continue
 		}
-
-		if !*ds.Active && ((!isMid && !ServerHostingDotConfigEdgeIncludeInactive) || (isMid && !ServerHostingDotConfigMidIncludeInactive)) {
-			continue
-		}
-
 		if *ds.CDNID != toData.Server.CDNID {
 			continue
 		}
-
-		if isMid {
-			if !strings.HasSuffix(string(*ds.Type), tc.DSTypeLiveNationalSuffix) {
+		if ds.Topology == nil {
+			if !*ds.Active && ((!isMid && !atscfg.ServerHostingDotConfigEdgeIncludeInactive) || (isMid && !atscfg.ServerHostingDotConfigMidIncludeInactive)) {
 				continue
 			}
 
-			// mids: include all DSes with at least one server assigned
-			if len(dsServerMap[*ds.ID]) == 0 {
-				continue
-			}
-		} else {
-			if !strings.HasSuffix(string(*ds.Type), tc.DSTypeLiveNationalSuffix) && !strings.HasSuffix(string(*ds.Type), tc.DSTypeLiveSuffix) {
-				continue
-			}
-
-			// edges: only include DSes assigned to this edge
-			if dsServerMap[*ds.ID] == nil {
-				continue
-			}
-
-			if _, ok := dsServerMap[*ds.ID][toData.Server.ID]; !ok {
-				continue
+			if isMid {
+				if !strings.HasSuffix(string(*ds.Type), tc.DSTypeLiveNationalSuffix) {
+					continue
+				}
+
+				// mids: include all DSes with at least one server assigned
+				if len(dsServerMap[*ds.ID]) == 0 {
+					continue
+				}
+			} else {
+				if !strings.HasSuffix(string(*ds.Type), tc.DSTypeLiveNationalSuffix) && !strings.HasSuffix(string(*ds.Type), tc.DSTypeLiveSuffix) {
+					continue
+				}
+
+				// edges: only include DSes assigned to this edge
+				if dsServerMap[*ds.ID] == nil {
+					continue
+				}
+
+				if _, ok := dsServerMap[*ds.ID][toData.Server.ID]; !ok {
+					continue
+				}
 			}
 		}
 
-		hostingDSes[tc.DeliveryServiceName(*ds.XMLID)] = ds
-	}
-
-	originSet := map[string]struct{}{}
-	for _, ds := range hostingDSes {
-		originSet[*ds.OrgServerFQDN] = struct{}{}
-	}
-	origins := []string{}
-	for origin, _ := range originSet {
-		origins = append(origins, origin)
+		filteredDSes = append(filteredDSes, ds)
 	}
 
-	return atscfg.MakeHostingDotConfig(tc.CacheName(toData.Server.HostName), toData.TOToolName, toData.TOURL, fileParams, origins), atscfg.ContentTypeHostingDotConfig, atscfg.LineCommentHostingDotConfig, nil
+	return atscfg.MakeHostingDotConfig(toData.Server, toData.TOToolName, toData.TOURL, fileParams, filteredDSes, toData.Topologies), atscfg.ContentTypeHostingDotConfig, atscfg.LineCommentHostingDotConfig, nil
 }
diff --git a/traffic_ops_ort/atstccfg/cfgfile/meta.go b/traffic_ops_ort/atstccfg/cfgfile/meta.go
index b4b4490..065962a 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/meta.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/meta.go
@@ -80,6 +80,7 @@ func GetMeta(toData *config.TOData, dir string) (*tc.ATSConfigMetaData, error) {
 
 	serverInfo := atscfg.ServerInfo{
 		CacheGroupID:                  toData.Server.CachegroupID,
+		CacheGroupName:                toData.Server.Cachegroup,
 		CDN:                           tc.CDNName(toData.Server.CDNName),
 		CDNID:                         toData.Server.CDNID,
 		DomainName:                    toData.Server.DomainName,
@@ -160,7 +161,7 @@ func GetMeta(toData *config.TOData, dir string) (*tc.ATSConfigMetaData, error) {
 			if ds.XMLID == nil {
 				continue // TODO log?
 			}
-			if _, ok := dssMap[*ds.ID]; !ok {
+			if _, ok := dssMap[*ds.ID]; !ok && ds.Topology == nil {
 				continue
 			}
 			dses[tc.DeliveryServiceName(*ds.XMLID)] = ds
@@ -197,7 +198,7 @@ func GetMeta(toData *config.TOData, dir string) (*tc.ATSConfigMetaData, error) {
 		uriSignedDSes = append(uriSignedDSes, tc.DeliveryServiceName(*ds.XMLID))
 	}
 
-	metaObj, err := atscfg.MakeMetaObj(tc.CacheName(toData.Server.HostName), &serverInfo, toURL, toReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, dir)
+	metaObj, err := atscfg.MakeMetaObj(tc.CacheName(toData.Server.HostName), &serverInfo, toURL, toReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, toData.CacheGroups, toData.Topologies, dir)
 	if err != nil {
 		return nil, errors.New("generating: " + err.Error())
 	}
diff --git a/traffic_ops_ort/atstccfg/cfgfile/parentdotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/parentdotconfig.go
index 52d8346..9afd3d6 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/parentdotconfig.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/parentdotconfig.go
@@ -86,6 +86,7 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 
 	serverInfo := atscfg.ServerInfo{
 		CacheGroupID:                  toData.Server.CachegroupID,
+		CacheGroupName:                toData.Server.Cachegroup,
 		CDN:                           tc.CDNName(toData.Server.CDNName),
 		CDNID:                         toData.Server.CDNID,
 		DomainName:                    toData.Server.DomainName,
@@ -197,7 +198,7 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 		return "", "", "", errors.New("getting ATS major version from version parameter (profile '" + toData.Server.Profile + "' configFile 'package' name 'trafficserver'): " + err.Error())
 	}
 
-	parentConfigParamsWithProfiles, err := TCParamsToParamsWithProfiles(toData.ParentConfigParams)
+	parentConfigParamsWithProfiles, err := atscfg.TCParamsToParamsWithProfiles(toData.ParentConfigParams)
 	if err != nil {
 		return "", "", "", errors.New("unmarshalling parent.config parameters profiles: " + err.Error())
 	}
@@ -301,11 +302,6 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 		}
 	}
 
-	allDSes := []int{}
-	for ds, _ := range allDSMap {
-		allDSes = append(allDSes, int(ds))
-	}
-
 	parentConfigDSes := []atscfg.ParentConfigDSTopLevel{}
 	for _, tcDS := range toData.DeliveryServices {
 		if tcDS.ID == nil {
@@ -313,7 +309,7 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 		}
 
 		if !serverInfo.IsTopLevelCache() {
-			if _, ok := parentServerDSes[toData.Server.ID][*tcDS.ID]; !ok {
+			if _, ok := parentServerDSes[toData.Server.ID][*tcDS.ID]; !ok && tcDS.Topology == nil {
 				continue // skip DSes not assigned to this server.
 			}
 		}
@@ -336,6 +332,7 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 		multiSiteOrigin := false
 		originShield := ""
 		dsType := tc.DSTypeFromString("")
+		dsTopology := ""
 		if tcDS.QStringIgnore != nil {
 			qStringIgnore = *tcDS.QStringIgnore
 		}
@@ -348,6 +345,9 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 		if tcDS.Type != nil {
 			dsType = *tcDS.Type
 		}
+		if tcDS.Topology != nil {
+			dsTopology = *tcDS.Topology
+		}
 
 		ds := atscfg.ParentConfigDSTopLevel{
 			ParentConfigDS: atscfg.ParentConfigDS{
@@ -357,6 +357,7 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 				MultiSiteOrigin: multiSiteOrigin,
 				OriginShield:    originShield,
 				Type:            dsType,
+				Topology:        dsTopology,
 			},
 		}
 
@@ -455,7 +456,7 @@ func GetConfigFileServerParentDotConfig(toData *config.TOData) (string, string,
 
 	parentInfos := atscfg.MakeParentInfo(&serverInfo, serverCDNDomain, profileCaches, originServers)
 
-	return atscfg.MakeParentDotConfig(&serverInfo, atsMajorVer, toData.TOToolName, toData.TOURL, parentConfigDSes, serverParams, parentInfos), atscfg.ContentTypeParentDotConfig, atscfg.LineCommentParentDotConfig, nil
+	return atscfg.MakeParentDotConfig(&serverInfo, atsMajorVer, toData.TOToolName, toData.TOURL, parentConfigDSes, serverParams, parentInfos, toData.Server, toData.Servers, toData.Topologies, toData.ParentConfigParams, toData.ServerCapabilities, toData.CacheGroups), atscfg.ContentTypeParentDotConfig, atscfg.LineCommentParentDotConfig, nil
 }
 
 // GetDSOrigins takes a map[deliveryServiceID]DeliveryService, and returns a map[DeliveryServiceID]OriginURI.
diff --git a/traffic_ops_ort/atstccfg/cfgfile/remapdotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/remapdotconfig.go
index 6991524..07c24bd 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/remapdotconfig.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/remapdotconfig.go
@@ -96,7 +96,7 @@ func GetConfigFileServerRemapDotConfig(toData *config.TOData) (string, string, s
 		if ds.Active == nil {
 			continue // TODO log?
 		}
-		if _, ok := dssMap[*ds.ID]; !ok {
+		if _, ok := dssMap[*ds.ID]; !ok && ds.Topology == nil {
 			continue
 		}
 		if !useInactive && !*ds.Active {
@@ -116,10 +116,14 @@ func GetConfigFileServerRemapDotConfig(toData *config.TOData) (string, string, s
 		if ds.ID == nil || ds.Type == nil || ds.XMLID == nil || ds.DSCP == nil || ds.Active == nil {
 			continue // TODO log error?
 		}
+		dsTopology := ""
+		if ds.Topology != nil {
+			dsTopology = *ds.Topology
+		}
 		// TODO sort by DS ID? the old Perl query does, but it shouldn't be necessary, except for determinism.
 		// TODO warn if no regexes?
 		for _, dsRegex := range dsRegexMap[tc.DeliveryServiceName(*ds.XMLID)] {
-			remapConfigDSData = append(remapConfigDSData, atscfg.RemapConfigDSData{
+			rds := atscfg.RemapConfigDSData{
 				ID:                       *ds.ID,
 				Type:                     *ds.Type,
 				OriginFQDN:               ds.OrgServerFQDN,
@@ -144,7 +148,18 @@ func GetConfigFileServerRemapDotConfig(toData *config.TOData) (string, string, s
 				AnonymousBlockingEnabled: ds.AnonymousBlockingEnabled,
 				Active:                   *ds.Active,
 				RangeSliceBlockSize:      ds.RangeSliceBlockSize,
-			})
+				Topology:                 dsTopology,
+			}
+			if ds.FirstHeaderRewrite != nil {
+				rds.FirstHeaderRewrite = *ds.FirstHeaderRewrite
+			}
+			if ds.InnerHeaderRewrite != nil {
+				rds.InnerHeaderRewrite = *ds.InnerHeaderRewrite
+			}
+			if ds.LastHeaderRewrite != nil {
+				rds.LastHeaderRewrite = *ds.LastHeaderRewrite
+			}
+			remapConfigDSData = append(remapConfigDSData, rds)
 		}
 	}
 
@@ -181,12 +196,12 @@ func GetConfigFileServerRemapDotConfig(toData *config.TOData) (string, string, s
 
 	cacheURLParams := ParamsToMap(FilterParams(toData.ServerParams, atscfg.CacheURLParameterConfigFile, "", "", ""))
 
-	cacheKeyParamsWithProfiles, err := TCParamsToParamsWithProfiles(toData.CacheKeyParams)
+	cacheKeyParamsWithProfiles, err := atscfg.TCParamsToParamsWithProfiles(toData.CacheKeyParams)
 	if err != nil {
 		return "", "", "", errors.New("decoding cache key parameter profiles: " + err.Error())
 	}
 
-	cacheKeyParamsWithProfilesMap := ParameterWithProfilesToMap(cacheKeyParamsWithProfiles)
+	cacheKeyParamsWithProfilesMap := atscfg.ParameterWithProfilesToMap(cacheKeyParamsWithProfiles)
 
 	dsProfileNamesToIDs := map[string]int{}
 	for _, ds := range filteredDSes {
@@ -272,6 +287,7 @@ func GetConfigFileServerRemapDotConfig(toData *config.TOData) (string, string, s
 
 	serverInfo := &atscfg.ServerInfo{
 		CacheGroupID:                  toData.Server.CachegroupID,
+		CacheGroupName:                toData.Server.Cachegroup,
 		CDN:                           tc.CDNName(toData.Server.CDNName),
 		CDNID:                         toData.Server.CDNID,
 		DomainName:                    toData.CDN.DomainName, // note this is intentionally the CDN domain, not the server domain. It's what's remapped to.
@@ -288,7 +304,7 @@ func GetConfigFileServerRemapDotConfig(toData *config.TOData) (string, string, s
 		SecondaryParentCacheGroupType: secondaryParentCGType,
 		Type:                          toData.Server.Type,
 	}
-	return atscfg.MakeRemapDotConfig(tc.CacheName(toData.Server.HostName), toData.TOToolName, toData.TOURL, atsMajorVer, cacheURLParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapConfigDSData), atscfg.ContentTypeRemapDotConfig, atscfg.LineCommentRemapDotConfig, nil
+	return atscfg.MakeRemapDotConfig(toData.Server, toData.TOToolName, toData.TOURL, atsMajorVer, cacheURLParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, serverInfo, remapConfigDSData, toData.Topologies, toData.CacheGroups, toData.ServerCapabilities, toData.DSRequiredCapabilities), atscfg.ContentTypeRemapDotConfig, atscfg.LineCommentRemapDotConfig, nil
 }
 
 type DeliveryServiceRegexesSortByTypeThenSetNum []tc.DeliveryServiceRegex
diff --git a/traffic_ops_ort/atstccfg/cfgfile/routing.go b/traffic_ops_ort/atstccfg/cfgfile/routing.go
index 01fd4e9..a00e137 100644
--- a/traffic_ops_ort/atstccfg/cfgfile/routing.go
+++ b/traffic_ops_ort/atstccfg/cfgfile/routing.go
@@ -24,6 +24,7 @@ import (
 	"strings"
 	"time"
 
+	"github.com/apache/trafficcontrol/lib/go-atscfg"
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/traffic_ops_ort/atstccfg/config"
@@ -167,7 +168,12 @@ func GetConfigFileServer(toData *config.TOData, fileName string) (string, string
 	contentType := ""
 	lineComment := ""
 	err := error(nil)
-	if getCfgFunc, ok := ServerConfigFileFuncs()[fileName]; ok {
+
+	if strings.HasPrefix(fileName, atscfg.HeaderRewriteFirstPrefix) ||
+		strings.HasPrefix(fileName, atscfg.HeaderRewriteInnerPrefix) ||
+		strings.HasPrefix(fileName, atscfg.HeaderRewriteLastPrefix) {
+		txt, contentType, lineComment, err = GetConfigFileServerTopologyHeaderRewrite(toData, fileName)
+	} else if getCfgFunc, ok := ServerConfigFileFuncs()[fileName]; ok {
 		txt, contentType, lineComment, err = getCfgFunc(toData)
 	} else {
 		txt, contentType, lineComment, err = GetConfigFileServerUnknownConfig(toData, fileName)
diff --git a/traffic_ops_ort/atstccfg/cfgfile/topologyheaderrewritedotconfig.go b/traffic_ops_ort/atstccfg/cfgfile/topologyheaderrewritedotconfig.go
new file mode 100644
index 0000000..c1b1204
--- /dev/null
+++ b/traffic_ops_ort/atstccfg/cfgfile/topologyheaderrewritedotconfig.go
@@ -0,0 +1,61 @@
+package cfgfile
+
+/*
+ * 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"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-atscfg"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/traffic_ops_ort/atstccfg/config"
+)
+
+func GetConfigFileServerTopologyHeaderRewrite(toData *config.TOData, fileName string) (string, string, string, error) {
+	dsName := fileName
+	dsName = strings.TrimSuffix(dsName, atscfg.ConfigSuffix)
+	dsName = strings.TrimPrefix(dsName, atscfg.HeaderRewriteFirstPrefix)
+	dsName = strings.TrimPrefix(dsName, atscfg.HeaderRewriteInnerPrefix)
+	dsName = strings.TrimPrefix(dsName, atscfg.HeaderRewriteLastPrefix)
+
+	tcDS := tc.DeliveryServiceNullable{}
+	for _, ds := range toData.DeliveryServices {
+		if ds.XMLID == nil || *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return "", "", "", errors.New("topology ds '" + dsName + "' not found")
+	}
+
+	return atscfg.MakeTopologyHeaderRewriteDotConfig(
+		toData.Server,
+		toData.TOToolName,
+		toData.TOURL,
+		tcDS,
+		toData.Topologies,
+		toData.CacheGroups,
+		toData.Servers,
+		toData.ServerCapabilities,
+		toData.DSRequiredCapabilities[*tcDS.ID],
+	), atscfg.ContentTypeHeaderRewriteDotConfig, atscfg.LineCommentHeaderRewriteDotConfig, nil
+}
diff --git a/traffic_ops_ort/atstccfg/config/config.go b/traffic_ops_ort/atstccfg/config/config.go
index 5197dee..fd63a49 100644
--- a/traffic_ops_ort/atstccfg/config/config.go
+++ b/traffic_ops_ort/atstccfg/config/config.go
@@ -297,4 +297,8 @@ type TOData struct {
 
 	// SSLKeys must be all the ssl keys for the server's cdn.
 	SSLKeys []tc.CDNSSLKeys
+
+	// Topologies must be all the topologies for the server's cdn.
+	// May incude topologies of other cdns.
+	Topologies []tc.Topology
 }
diff --git a/traffic_ops_ort/atstccfg/getdata/getdata.go b/traffic_ops_ort/atstccfg/getdata/getdata.go
index 23a9ceb..8146f05 100644
--- a/traffic_ops_ort/atstccfg/getdata/getdata.go
+++ b/traffic_ops_ort/atstccfg/getdata/getdata.go
@@ -112,7 +112,11 @@ func WriteStatuses(cfg config.TCCfg, output io.Writer) error {
 // WriteUpdateStatus writes the Traffic Ops server update status to output.
 // Note this is identical to /api/1.x/servers/name/update_status except it omits the '[]' wrapper.
 func WriteServerUpdateStatus(cfg config.TCCfg, output io.Writer) error {
-	status, err := cfg.TOClient.GetServerUpdateStatus(tc.CacheName(cfg.CacheHostName))
+	status, unsupported, err := cfg.TOClientNew.GetServerUpdateStatus(tc.CacheName(cfg.CacheHostName))
+	if err == nil && unsupported {
+		log.Warnln("ORT newer than Traffic Ops, falling back to previous API Delivery Services!")
+		status, err = cfg.TOClient.GetServerUpdateStatus(tc.CacheName(cfg.CacheHostName))
+	}
 	if err != nil {
 		return errors.New("getting server update status: " + err.Error())
 	}
diff --git a/traffic_ops_ort/atstccfg/toreqnew/toreqnew.go b/traffic_ops_ort/atstccfg/toreqnew/toreqnew.go
index 81db18f..07e7470 100644
--- a/traffic_ops_ort/atstccfg/toreqnew/toreqnew.go
+++ b/traffic_ops_ort/atstccfg/toreqnew/toreqnew.go
@@ -39,8 +39,9 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	toclient "github.com/apache/trafficcontrol/traffic_ops/client"
 	"github.com/apache/trafficcontrol/traffic_ops_ort/atstccfg/torequtil"
-	toclient "github.com/apache/trafficcontrol/traffic_ops/v2-client"
 )
 
 type TOClient struct {
@@ -75,9 +76,9 @@ func (cl *TOClient) GetCDNDeliveryServices(cdnID int) ([]tc.DeliveryServiceNulla
 	deliveryServices := []tc.DeliveryServiceNullable{}
 	unsupported := false
 	err := torequtil.GetRetry(cl.NumRetries, "cdn_"+strconv.Itoa(cdnID)+"_deliveryservices", &deliveryServices, func(obj interface{}) error {
-		toDSes, reqInf, err := cl.C.GetDeliveryServicesByCDNID(cdnID)
+		toDSes, reqInf, err := cl.C.GetDeliveryServicesByCDNID(cdnID, nil)
 		if err != nil {
-			if errStr := strings.ToLower(err.Error()); strings.Contains(errStr, "not found") || strings.Contains(errStr, "not impl") {
+			if IsUnsupportedErr(err) {
 				unsupported = true
 				return nil
 			}
@@ -95,3 +96,58 @@ func (cl *TOClient) GetCDNDeliveryServices(cdnID int) ([]tc.DeliveryServiceNulla
 	}
 	return deliveryServices, false, nil
 }
+
+func (cl *TOClient) GetTopologies() ([]tc.Topology, bool, error) {
+	topologies := []tc.Topology{}
+	unsupported := false
+	err := torequtil.GetRetry(cl.NumRetries, "topologies", &topologies, func(obj interface{}) error {
+		toTopologies, reqInf, err := cl.C.GetTopologies(nil)
+		if err != nil {
+			if IsUnsupportedErr(err) {
+				unsupported = true
+				return nil
+			}
+			return errors.New("getting topologies from Traffic Ops '" + torequtil.MaybeIPStr(reqInf.RemoteAddr) + "': " + err.Error())
+		}
+		topologies := obj.(*[]tc.Topology)
+		*topologies = toTopologies
+		return nil
+	})
+	if unsupported {
+		return nil, true, nil
+	}
+	if err != nil {
+		return nil, false, errors.New("getting topologies: " + err.Error())
+	}
+	return topologies, false, nil
+}
+
+func (cl *TOClient) GetServerUpdateStatus(cacheHostName tc.CacheName) (tc.ServerUpdateStatus, bool, error) {
+	status := tc.ServerUpdateStatus{}
+	unsupported := false
+	err := torequtil.GetRetry(cl.NumRetries, "server_update_status_"+string(cacheHostName), &status, func(obj interface{}) error {
+		toStatus, reqInf, err := cl.C.GetServerUpdateStatus(string(cacheHostName), nil)
+		if err != nil {
+			if IsUnsupportedErr(err) {
+				unsupported = true
+				return nil
+			}
+			return errors.New("getting server update status from Traffic Ops '" + torequtil.MaybeIPStr(reqInf.RemoteAddr) + "': " + err.Error())
+		}
+		status := obj.(*tc.ServerUpdateStatus)
+		*status = toStatus
+		return nil
+	})
+	if unsupported {
+		return tc.ServerUpdateStatus{}, true, nil
+	}
+	if err != nil {
+		return tc.ServerUpdateStatus{}, false, errors.New("getting server update status: " + err.Error())
+	}
+	return status, false, nil
+}
+
+func IsUnsupportedErr(err error) bool {
+	errStr := strings.ToLower(err.Error())
+	return strings.Contains(errStr, "not found") || strings.Contains(errStr, "not impl")
+}