You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@trafficcontrol.apache.org by GitBox <gi...@apache.org> on 2020/11/05 00:03:47 UTC

[GitHub] [trafficcontrol] rob05c opened a new pull request #5247: Change ORT/atstccfg to use standard TC objects

rob05c opened a new pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247


   Changes `lib/go-atscfg` to take standard `lib/go-tc` objects, instead of custom stuff. This makes it much easier to use as a library. Users can simply get objects from the TO API, and pass them to generate config, instead of having to build custom objects, with a lot of logic in their app.
   
   As a reminder, it didn't do this at first, because the custom objects were what we had in Traffic Ops from the DB; TO config files have now been removed.
   
   This also
   - changes the comment header to be the real TO URL that ORT has, use the same generation timestamp on all files, and include the TO IP(s). Should improve debugging.
   - changes `lib/go-atscfg` to track non-fatal warnings, and return them in a list, so the library itself doesn't log (libraries should never log)
   - changes `lib/go-atscfg` generation funcs to return real errors, instead of an "ERROR: " config file string (again, it did that because TO did it, which has now been removed).
   - Fixes a bug with ORT not fetching the latest servers, even though they were needed and added to `atstccfg/toreqnew`.
   
   Sorry for the size. If I tried to make all those things separate, they would've had terrible merge conflicts. For what it's worth, this should be a big improvement in production debugability, safety, better testing, not generating broken files, and better error and warning reporting.
   
   `lib/go-atscfg` has extensive unit tests, and they pass. Even better, with the custom-object logic now in the library, even more is being unit tested.
   I also manually tested generating both edges and mids on our two largest CDNs, they all diff identical, except the comment header and a ip_allow change/fix that's caused by the bug fix.
   
   This is all boilerplate changes, no logic. The actual config logic should be unchanged, and any logic change is a bug.
   
   Also for what it's worth, I'm hoping `lib/go-atscfg` is getting very close to "stable" and we can start supporting Go Modules and a "version promise" for the library. This is a huge step in that direction.
   
   Includes extensive tests.
   No docs, no interface change (outside the non-stable library).
   No changelog, no interface change.
   
   - [x] This PR is not related to any other Issue
   
   ## Which Traffic Control components are affected by this PR?
   - Traffic Ops ORT
   
   ## What is the best way to verify this PR?
   Run unit tests.
   Generate config with the `master` ORT, generate config with this PR, verify no changes (except the aforementioned comment header and corrected ip_allow CIDR).
   
   ## If this is a bug fix, what versions of Traffic Control are affected?
   Not a bug fix. The bug mentioned is in master, not in a release.
   
   ## The following criteria are ALL met by this PR
   - [x] This PR includes tests OR I have explained why tests are unnecessary
   - [x] This PR includes documentation OR I have explained why documentation is unnecessary
   - [x] This PR includes an update to CHANGELOG.md OR such an update is not necessary
   - [x] This PR includes any and all required license headers
   - [x] This PR **DOES NOT FIX A SERIOUS SECURITY VULNERABILITY** (see [the Apache Software Foundation's security guidelines](https://www.apache.org/security/) for details)
   
   ## Additional Information


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525517643



##########
File path: lib/go-atscfg/remapdotconfig.go
##########
@@ -408,22 +448,22 @@ func MakeDSTopologyHeaderRewriteTxt(ds tc.DeliveryServiceNullableV30, cg tc.Cach
 	if placement.IsLastCacheTier && ds.LastHeaderRewrite != nil && *ds.LastHeaderRewrite != "" {
 		txt += pluginTxt + LastHeaderRewriteConfigFileName(*ds.XMLID) + ` `
 	}
-	return txt
+	return txt, nil
 }
 
-type RemapLine struct {
+type remapLine struct {
 	From string
 	To   string
 }
 
 // MakeEdgeDSDataRemapLines returns the remap lines for the given server and delivery service.

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r524816128



##########
File path: lib/go-atscfg/headerrewritemiddotconfig.go
##########
@@ -20,24 +20,134 @@ package atscfg
  */
 
 import (
+	"fmt"
 	"math"
 	"regexp"
 	"strconv"
+	"strings"
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const HeaderRewriteMidPrefix = "hdr_rw_mid_"
 
 func MakeHeaderRewriteMidDotConfig(
-	cdnName tc.CDNName,
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-	ds HeaderRewriteDS,
-	assignedMids []HeaderRewriteServer, // the mids assigned to ds (mids whose cachegroup is the parent of the cachegroup of any edge assigned to this ds)
-) string {
-	text := GenericHeaderComment(string(cdnName), toToolName, toURL)
+	fileName string,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	cacheGroups []tc.CacheGroupNullable,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+	if server.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "this server missing CDNName")
+	}
+
+	dsName := strings.TrimSuffix(strings.TrimPrefix(fileName, HeaderRewriteMidPrefix), ConfigSuffix) // TODO verify prefix and suffix? Perl doesn't
+
+	tcDS := tc.DeliveryServiceNullableV30{}
+	for _, ds := range deliveryServices {
+		if ds.XMLID == nil || *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' not found")
+	}
+
+	if tcDS.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' missing cdn")
+	}
+
+	ds, err := headerRewriteDSFromDS(&tcDS)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, "converting ds to config ds: "+err.Error())
+	}
+
+	assignedServers := map[int]struct{}{}
+	for _, dss := range deliveryServiceServers {
+		if dss.Server == nil || dss.DeliveryService == nil {
+			continue
+		}
+		if *dss.DeliveryService != *tcDS.ID {
+			continue
+		}
+		assignedServers[*dss.Server] = struct{}{}
+	}
 
+	serverCGs := map[tc.CacheGroupName]struct{}{}
+	for _, sv := range servers {
+		if sv.CDNName == nil {
+			warnings = append(warnings, "TO returned Servers server with missing CDNName, skipping!")
+			continue
+		} else if sv.ID == nil {
+			warnings = append(warnings, "TO returned Servers server with missing ID, skipping!")
+			continue
+		} else if sv.Status == nil {
+			warnings = append(warnings, "TO returned Servers server with missing Status, skipping!")
+			continue
+		} else if sv.Cachegroup == nil {
+			warnings = append(warnings, "TO returned Servers server with missing Cachegroup, skipping!")
+			continue
+		}
+
+		if sv.CDNName != server.CDNName {
+			continue
+		}
+		if _, ok := assignedServers[*sv.ID]; !ok && (tcDS.Topology == nil || *tcDS.Topology == "") {
+			continue
+		}
+		if tc.CacheStatus(*sv.Status) != tc.CacheStatusReported && tc.CacheStatus(*sv.Status) != tc.CacheStatusOnline {
+			continue
+		}
+		serverCGs[tc.CacheGroupName(*sv.Cachegroup)] = struct{}{}
+	}
+
+	parentCGs := map[string]struct{}{} // names of cachegroups which are parent cachegroups of the cachegroup of any edge assigned to the given DS
+	for _, cg := range cacheGroups {
+		if cg.Name == nil {
+			continue // TODO warn?
+		}
+		if cg.ParentName == nil {
+			continue // TODO warn?

Review comment:
       Same as above, fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525506341



##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,56 +22,90 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const InvalidID = -1
-
 const DefaultATSVersion = "5" // TODO Emulates Perl; change to 6? ATC no longer officially supports ATS 5.
-
 const HeaderCommentDateFormat = "Mon Jan 2 15:04:05 MST 2006"
-
 const ContentTypeTextASCII = `text/plain; charset=us-ascii`
-
 const LineCommentHash = "#"
+const ConfigSuffix = ".config"
+
+type DeliveryServiceID int
+type ProfileID int
+type ServerID int
 
 type TopologyName string
 type CacheGroupType string
 type ServerCapability string
 
-type ServerInfo struct {
-	CacheGroupID                  int
-	CacheGroupName                string
-	CDN                           tc.CDNName
-	CDNID                         int
-	DomainName                    string
-	HostName                      string
-	HTTPSPort                     int
-	ID                            int
-	IP                            string
-	ParentCacheGroupID            int
-	ParentCacheGroupType          string
-	ProfileID                     ProfileID
-	ProfileName                   string
-	Port                          int
-	SecondaryParentCacheGroupID   int
-	SecondaryParentCacheGroupType string
-	Type                          string
-}
-
-func (s *ServerInfo) IsTopLevelCache() bool {
-	return (s.ParentCacheGroupType == tc.CacheGroupOriginTypeName || s.ParentCacheGroupID == InvalidID) &&
-		(s.SecondaryParentCacheGroupType == tc.CacheGroupOriginTypeName || s.SecondaryParentCacheGroupID == InvalidID)
-}
-
-func MakeCGMap(cgs []tc.CacheGroupNullable) (map[tc.CacheGroupName]tc.CacheGroupNullable, error) {
+// Server is a tc.Server for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type Server tc.ServerV30
+
+// DeliveryService is a tc.DeliveryService for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type DeliveryService tc.DeliveryServiceNullableV30
+
+// ToDeliveryServices converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToDeliveryServices(dses []tc.DeliveryServiceNullableV30) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		ad = append(ad, DeliveryService(ds))
+	}
+	return ad
+}
+
+// OldToDeliveryServices converts a slice of the old traffic_ops/client type to the local alias.
+func OldToDeliveryServices(dses []tc.DeliveryServiceNullable) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		upgradedDS := tc.DeliveryServiceNullableV30{DeliveryServiceNullableV15: tc.DeliveryServiceNullableV15(ds)}
+		ad = append(ad, DeliveryService(upgradedDS))
+	}
+	return ad
+}
+
+// ToServers converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToServers(servers []tc.ServerV30) []Server {
+	as := []Server{}
+	for _, sv := range servers {
+		as = append(as, Server(sv))
+	}
+	return as
+}

Review comment:
       ```
   func ToServers(servers []tc.ServerV30) []Server {
   	as := []Server{}
   	for _, sv := range servers {
   		as = append(as, Server(sv))
   	}
   	return as
   }
   func ToServersMake(servers []tc.ServerV30) []Server {
   	as := make([]Server, 0, len(servers))
   	for _, sv := range servers {
   		as = append(as, Server(sv))
   	}
   	return as
   }
   func BenchmarkToServers1mil(b *testing.B) {
   	num := 1000000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServers(servers)
   	}
   }
   func BenchmarkToServersMake1mil(b *testing.B) {
   	num := 1000000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServersMake(servers)
   	}
   }
   func BenchmarkToServers2k(b *testing.B) {
   	num := 2000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServers(servers)
   	}
   }
   func BenchmarkToServersMake2k(b *testing.B) {
   	num := 2000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServersMake(servers)
   	}
   }
   ```
   ```
   $ go test -bench=.
   DEBUG moc 42 usesmid true assignedmids 3
   DEBUG moc 0 usesmid true assignedmids 0
   goos: darwin
   goarch: amd64
   pkg: github.com/apache/trafficcontrol/lib/go-atscfg
   BenchmarkToServers1mil-16                      1        1577990978 ns/op
   BenchmarkToServersMake1mil-16                  3         387240011 ns/op
   BenchmarkToServers2k-16                     1321            821199 ns/op
   BenchmarkToServersMake2k-16                 2414            417031 ns/op
   PASS
   ok      github.com/apache/trafficcontrol/lib/go-atscfg  8.505s
   ```
   So on my laptop, it takes 0.0016 milliseconds without `make` vs 0.00039 milliseconds with `make`, for a million servers,
   and 0.00041 milliseconds without vs 0.00021 milliseconds with, for 2,000 servers.
   
   I think we can spare the performance to improve the readability.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525517027



##########
File path: lib/go-atscfg/remapdotconfig.go
##########
@@ -35,81 +34,104 @@ const CacheKeyParameterConfigFile = "cachekey.config"
 const ContentTypeRemapDotConfig = ContentTypeTextASCII
 const LineCommentRemapDotConfig = LineCommentHash
 
+const RemapConfigRangeDirective = `__RANGE_DIRECTIVE__`
+
 func MakeRemapDotConfig(
-	server *tc.ServerNullable,
-	unfilteredDSes []tc.DeliveryServiceNullableV30,
+	server *Server,
+	unfilteredDSes []DeliveryService,
 	dss []tc.DeliveryServiceServer,
 	dsRegexArr []tc.DeliveryServiceRegexes,
 	serverParams []tc.Parameter,
 	cdn *tc.CDN,
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
 	cacheKeyParams []tc.Parameter,
 	topologies []tc.Topology,
 	cacheGroupArr []tc.CacheGroupNullable,
 	serverCapabilities map[int]map[ServerCapability]struct{},
 	dsRequiredCapabilities map[int]map[ServerCapability]struct{},
-) string {
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
 	if server.HostName == nil {
-		return "ERROR: server HostName missing"
+		return Cfg{}, makeErr(warnings, "server HostName missing")
 	} else if server.ID == nil {
-		return "ERROR: server ID missing"
+		return Cfg{}, makeErr(warnings, "server ID missing")
 	} else if server.Cachegroup == nil {
-		return "ERROR: server Cachegroup missing"
+		return Cfg{}, makeErr(warnings, "server Cachegroup missing")
 	} else if server.DomainName == nil {
-		return "ERROR: server DomainName missing"
+		return Cfg{}, makeErr(warnings, "server DomainName missing")
 	}
 
 	cdnDomain := cdn.DomainName
 	dsRegexes := makeDSRegexMap(dsRegexArr)
 	// Returned DSes are guaranteed to have a non-nil XMLID, Type, DSCP, ID, and Active.
-	dses := remapFilterDSes(server, dss, unfilteredDSes, cacheKeyParams)
+	dses, dsWarns := remapFilterDSes(server, dss, unfilteredDSes, cacheKeyParams)
+	warnings = append(warnings, dsWarns...)
 
-	dsProfilesCacheKeyConfigParams, err := makeDSProfilesCacheKeyConfigParams(server, dses, cacheKeyParams)
+	dsProfilesCacheKeyConfigParams, paramWarns, err := makeDSProfilesCacheKeyConfigParams(server, dses, cacheKeyParams)
+	warnings = append(warnings, paramWarns...)
 	if err != nil {
-		log.Errorln("Error making Delivery Service Cache Key Params, cache key will be missing! : " + err.Error())
+		warnings = append(warnings, "making Delivery Service Cache Key Params, cache key will be missing! : "+err.Error())
 	}
 
-	atsMajorVersion := getATSMajorVersion(serverParams)
-	serverPackageParamData := makeServerPackageParamData(server, serverParams)
-	cacheURLConfigParams := ParamsToMap(FilterParams(serverParams, CacheURLParameterConfigFile, "", "", ""))
-	cacheGroups, err := MakeCGMap(cacheGroupArr)
+	atsMajorVersion, verWarns := getATSMajorVersion(serverParams)
+	warnings = append(warnings, verWarns...)
+	serverPackageParamData, paramWarns := makeServerPackageParamData(server, serverParams)
+	warnings = append(warnings, paramWarns...)
+	cacheURLConfigParams, paramWarns := paramsToMap(filterParams(serverParams, CacheURLParameterConfigFile, "", "", ""))
+	warnings = append(warnings, paramWarns...)
+	cacheGroups, err := makeCGMap(cacheGroupArr)
 	if err != nil {
-		log.Errorln("making remap.config, config will be malformed! : " + err.Error())
+		return Cfg{}, makeErr(warnings, "making remap.config, config will be malformed! : "+err.Error())
 	}
 
-	nameTopologies := MakeTopologyNameMap(topologies)
+	nameTopologies := makeTopologyNameMap(topologies)
 
-	hdr := GenericHeaderComment(*server.HostName, toToolName, toURL)
+	hdr := makeHdrComment(hdrComment)
+	txt := ""
+	typeWarns := []string{}
 	if tc.CacheTypeFromString(server.Type) == tc.CacheTypeMid {
-		return GetServerConfigRemapDotConfigForMid(atsMajorVersion, dsProfilesCacheKeyConfigParams, dses, dsRegexes, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities)
+		txt, typeWarns, err = getServerConfigRemapDotConfigForMid(atsMajorVersion, dsProfilesCacheKeyConfigParams, dses, dsRegexes, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities)
+	} else {
+		txt, typeWarns, err = getServerConfigRemapDotConfigForEdge(cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, dses, dsRegexes, atsMajorVersion, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities, cdnDomain)
 	}
-	return GetServerConfigRemapDotConfigForEdge(cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, dses, dsRegexes, atsMajorVersion, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities, cdnDomain)
+	warnings = append(warnings, typeWarns...)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, err.Error()) // the GetFor funcs include error context
+	}
+
+	return Cfg{
+		Text:        txt,
+		ContentType: ContentTypeRemapDotConfig,
+		LineComment: LineCommentRemapDotConfig,
+		Warnings:    warnings,
+	}, nil
 }
 
-func GetServerConfigRemapDotConfigForMid(
+// GetServerConfigRemapDotConfigForMid returns the remap lines, any warnings, and any error.

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525516725



##########
File path: lib/go-atscfg/sslmulticertdotconfig.go
##########
@@ -95,11 +112,11 @@ func GetSSLMultiCertDotConfigCertAndKeyName(dsName tc.DeliveryServiceName, ds SS
 	return cerName, keyName
 }
 
-// GetSSLMultiCertDotConfigDeliveryServices takes a list of delivery services, and returns the delivery services which will be inserted into the config by MakeSSLMultiCertDotConfig.
+// getSSLMultiCertDotConfigDeliveryServices takes a list of delivery services, and returns the delivery services which will be inserted into the config by MakeSSLMultiCertDotConfig.

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] alficles commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
alficles commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r523973339



##########
File path: lib/go-atscfg/cachedotconfig.go
##########
@@ -23,33 +23,95 @@ import (
 	"sort"
 	"strings"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const ContentTypeCacheDotConfig = ContentTypeTextASCII
 const LineCommentCacheDotConfig = LineCommentHash
 
-type ProfileDS struct {
-	Type       tc.DSType
-	OriginFQDN *string
+func MakeCacheDotConfig(
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	hdrComment string,
+) (Cfg, error) {
+	if tc.CacheTypeFromString(server.Type) == tc.CacheTypeMid {
+		return makeCacheDotConfigMid(server, deliveryServices, hdrComment)
+	} else {
+		return makeCacheDotConfigEdge(server, servers, deliveryServices, deliveryServiceServers, hdrComment)
+	}
 }
 
 // MakeCacheDotConfig makes the ATS cache.config config file.
 // profileDSes must be the list of delivery services, which are assigned to severs, for which this profile is assigned. It MUST NOT contain any other delivery services. Note DSesToProfileDSes may be helpful if you have a []tc.DeliveryServiceNullable, for example from traffic_ops/client.
-func MakeCacheDotConfig(
-	profileName string,
-	profileDSes []ProfileDS,
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-) string {
+func makeCacheDotConfigEdge(
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+
+	if server.Profile == nil {
+		return Cfg{}, makeErr(warnings, "server missing profile")
+	}
+
+	profileServerIDsMap := map[int]struct{}{}
+	for _, sv := range servers {
+		if sv.Profile == nil {
+			warnings = append(warnings, "servers had server with nil profile, skipping!")
+			continue
+		}
+		if sv.ID == nil {
+			warnings = append(warnings, "servers had server with nil id, skipping!")
+			continue
+		}
+		if *sv.Profile != *server.Profile {
+			continue
+		}
+		profileServerIDsMap[*sv.ID] = struct{}{}
+	}
+
+	dsServers := filterDSS(deliveryServiceServers, nil, profileServerIDsMap)
+
+	dsIDs := map[int]struct{}{}
+	for _, dss := range dsServers {
+		if dss.Server == nil || dss.DeliveryService == nil {
+			continue // TODO warn? err?
+		}
+		if _, ok := profileServerIDsMap[*dss.Server]; !ok {
+			continue
+		}
+		dsIDs[*dss.DeliveryService] = struct{}{}
+	}
+
+	profileDSes := []profileDS{}
+	for _, ds := range deliveryServices {
+		if ds.ID == nil || ds.Type == nil || ds.OrgServerFQDN == nil {
+			continue // TODO warn? err?
+		}
+		if *ds.Type == tc.DSTypeInvalid {
+			continue // TODO warn? err?
+		}
+		if *ds.OrgServerFQDN == "" {
+			continue // TODO warn? err?
+		}

Review comment:
       If I understand this correctly, these only occur if TC passes ORT data it thinks is bad. This suggests either a defect in TC or a misunderstanding on ort's part. This warrants a warning, I think. Not necessarily an error, though one hopes that operators have alarms on warnings here.
   
   I think this is a result of moved code, though, instead of new code. So this isn't a major issue.

##########
File path: lib/go-atscfg/cacheurldotconfig.go
##########
@@ -86,7 +110,8 @@ func MakeCacheURLDotConfig(
 			}
 
 			if !strings.HasPrefix(org, scheme) {
-				log.Errorln("MakeCacheURLDotConfig got ds '" + string(dsName) + "' origin '" + org + "' with no scheme! cacheurl.config will likely be malformed!")
+				// TODO determine if we should return an empty config here. A bad DS should not break config gen, and MUST NOT for self-service
+				warnings = append(warnings, "ds '"+string(dsName)+"' origin '"+org+"' with no scheme! cacheurl.config will likely be malformed!")

Review comment:
       This should probably be opened as a separate ticket for resolution. That TODO is spot on and we shouldn't lose track of it. This change is fine, though.

##########
File path: lib/go-atscfg/headerrewritedotconfig.go
##########
@@ -39,7 +39,127 @@ const ServiceCategoryHeader = "CDN-SVC"
 
 const MaxOriginConnectionsNoMax = 0 // 0 indicates no limit on origin connections
 
-type HeaderRewriteDS struct {
+func MakeHeaderRewriteDotConfig(
+	fileName string,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+
+	dsName := strings.TrimSuffix(strings.TrimPrefix(fileName, HeaderRewritePrefix), ConfigSuffix) // TODO verify prefix and suffix? Perl doesn't
+
+	tcDS := tc.DeliveryServiceNullableV30{}
+	for _, ds := range deliveryServices {
+		if ds.XMLID == nil {
+			warnings = append(warnings, "deliveryServices had DS with nil xmlId (name)")
+			continue
+		}
+		if *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' not found")
+	}
+
+	if tcDS.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' missing cdn")
+	}
+
+	ds, err := headerRewriteDSFromDS(&tcDS)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, "converting ds to config ds: "+err.Error())
+	}
+
+	dsServers := filterDSS(deliveryServiceServers, map[int]struct{}{ds.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{}{}
+	}
+
+	assignedEdges := []headerRewriteServer{}
+	for _, server := range servers {
+		if server.CDNName == nil {
+			warnings = append(warnings, "servers had server with missing cdnName, skipping!")
+			continue
+		}
+		if server.ID == nil {
+			warnings = append(warnings, "servers had server with missing kid, skipping!")

Review comment:
       Typo in name here? s/kid/id/?

##########
File path: lib/go-atscfg/headerrewritedotconfig.go
##########
@@ -39,7 +39,127 @@ const ServiceCategoryHeader = "CDN-SVC"
 
 const MaxOriginConnectionsNoMax = 0 // 0 indicates no limit on origin connections
 
-type HeaderRewriteDS struct {
+func MakeHeaderRewriteDotConfig(
+	fileName string,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+
+	dsName := strings.TrimSuffix(strings.TrimPrefix(fileName, HeaderRewritePrefix), ConfigSuffix) // TODO verify prefix and suffix? Perl doesn't
+
+	tcDS := tc.DeliveryServiceNullableV30{}
+	for _, ds := range deliveryServices {
+		if ds.XMLID == nil {
+			warnings = append(warnings, "deliveryServices had DS with nil xmlId (name)")
+			continue
+		}
+		if *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' not found")
+	}
+
+	if tcDS.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' missing cdn")
+	}
+
+	ds, err := headerRewriteDSFromDS(&tcDS)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, "converting ds to config ds: "+err.Error())
+	}
+
+	dsServers := filterDSS(deliveryServiceServers, map[int]struct{}{ds.ID: {}}, nil)
+
+	dsServerIDs := map[int]struct{}{}
+	for _, dss := range dsServers {
+		if dss.Server == nil || dss.DeliveryService == nil {
+			continue // TODO warn?

Review comment:
       Same story as above. This should be a warning.

##########
File path: lib/go-atscfg/headerrewritemiddotconfig.go
##########
@@ -20,24 +20,134 @@ package atscfg
  */
 
 import (
+	"fmt"
 	"math"
 	"regexp"
 	"strconv"
+	"strings"
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const HeaderRewriteMidPrefix = "hdr_rw_mid_"
 
 func MakeHeaderRewriteMidDotConfig(
-	cdnName tc.CDNName,
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-	ds HeaderRewriteDS,
-	assignedMids []HeaderRewriteServer, // the mids assigned to ds (mids whose cachegroup is the parent of the cachegroup of any edge assigned to this ds)
-) string {
-	text := GenericHeaderComment(string(cdnName), toToolName, toURL)
+	fileName string,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	cacheGroups []tc.CacheGroupNullable,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+	if server.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "this server missing CDNName")
+	}
+
+	dsName := strings.TrimSuffix(strings.TrimPrefix(fileName, HeaderRewriteMidPrefix), ConfigSuffix) // TODO verify prefix and suffix? Perl doesn't
+
+	tcDS := tc.DeliveryServiceNullableV30{}
+	for _, ds := range deliveryServices {
+		if ds.XMLID == nil || *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' not found")
+	}
+
+	if tcDS.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' missing cdn")
+	}
+
+	ds, err := headerRewriteDSFromDS(&tcDS)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, "converting ds to config ds: "+err.Error())
+	}
+
+	assignedServers := map[int]struct{}{}
+	for _, dss := range deliveryServiceServers {
+		if dss.Server == nil || dss.DeliveryService == nil {
+			continue
+		}
+		if *dss.DeliveryService != *tcDS.ID {
+			continue
+		}
+		assignedServers[*dss.Server] = struct{}{}
+	}
 
+	serverCGs := map[tc.CacheGroupName]struct{}{}
+	for _, sv := range servers {
+		if sv.CDNName == nil {
+			warnings = append(warnings, "TO returned Servers server with missing CDNName, skipping!")
+			continue
+		} else if sv.ID == nil {
+			warnings = append(warnings, "TO returned Servers server with missing ID, skipping!")
+			continue
+		} else if sv.Status == nil {
+			warnings = append(warnings, "TO returned Servers server with missing Status, skipping!")
+			continue
+		} else if sv.Cachegroup == nil {
+			warnings = append(warnings, "TO returned Servers server with missing Cachegroup, skipping!")
+			continue
+		}
+
+		if sv.CDNName != server.CDNName {
+			continue
+		}
+		if _, ok := assignedServers[*sv.ID]; !ok && (tcDS.Topology == nil || *tcDS.Topology == "") {
+			continue
+		}
+		if tc.CacheStatus(*sv.Status) != tc.CacheStatusReported && tc.CacheStatus(*sv.Status) != tc.CacheStatusOnline {
+			continue
+		}
+		serverCGs[tc.CacheGroupName(*sv.Cachegroup)] = struct{}{}
+	}
+
+	parentCGs := map[string]struct{}{} // names of cachegroups which are parent cachegroups of the cachegroup of any edge assigned to the given DS
+	for _, cg := range cacheGroups {
+		if cg.Name == nil {
+			continue // TODO warn?
+		}
+		if cg.ParentName == nil {
+			continue // TODO warn?

Review comment:
       Same as above. Worth a warning, imo.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r518199960



##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,16 +22,36 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
+// ATSConfigFile is all the information necessary to create an ATS config file, including the file name, path, data, and metadata.

Review comment:
       Fixed

##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,16 +22,36 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
+// ATSConfigFile is all the information necessary to create an ATS config file, including the file name, path, data, and metadata.
+// This is provided as a convenience and unified structure for users. The lib/go-atscfg library doesn't actually use or return this. See ATSConfigFileData.
+type CfgFile struct {
+	Name string
+	Path string
+	Cfg
+}
+
+// ATSConfigFileData is the data and metadata for an ATS Config File.

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525516324



##########
File path: lib/go-atscfg/servercachedotconfig.go
##########
@@ -27,18 +27,36 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
-type ServerCacheConfigDS struct {
-	OrgServerFQDN string
-	Type          tc.DSType
-}
+const ServerCacheDotConfigIncludeInactiveDSes = false // TODO move to lib/go-atscfg

Review comment:
       Removed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525511660



##########
File path: lib/go-atscfg/cachedotconfig.go
##########
@@ -23,33 +23,106 @@ import (
 	"sort"
 	"strings"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const ContentTypeCacheDotConfig = ContentTypeTextASCII
 const LineCommentCacheDotConfig = LineCommentHash
 
-type ProfileDS struct {
-	Type       tc.DSType
-	OriginFQDN *string
+func MakeCacheDotConfig(
+	server *Server,
+	servers []Server,
+	deliveryServices []DeliveryService,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	hdrComment string,
+) (Cfg, error) {
+	if tc.CacheTypeFromString(server.Type) == tc.CacheTypeMid {
+		return makeCacheDotConfigMid(server, deliveryServices, hdrComment)
+	} else {
+		return makeCacheDotConfigEdge(server, servers, deliveryServices, deliveryServiceServers, hdrComment)
+	}
 }
 
 // MakeCacheDotConfig makes the ATS cache.config config file.
 // profileDSes must be the list of delivery services, which are assigned to severs, for which this profile is assigned. It MUST NOT contain any other delivery services. Note DSesToProfileDSes may be helpful if you have a []tc.DeliveryServiceNullable, for example from traffic_ops/client.

Review comment:
       Yep. Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r524814204



##########
File path: lib/go-atscfg/headerrewritedotconfig.go
##########
@@ -39,7 +39,127 @@ const ServiceCategoryHeader = "CDN-SVC"
 
 const MaxOriginConnectionsNoMax = 0 // 0 indicates no limit on origin connections
 
-type HeaderRewriteDS struct {
+func MakeHeaderRewriteDotConfig(
+	fileName string,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+
+	dsName := strings.TrimSuffix(strings.TrimPrefix(fileName, HeaderRewritePrefix), ConfigSuffix) // TODO verify prefix and suffix? Perl doesn't
+
+	tcDS := tc.DeliveryServiceNullableV30{}
+	for _, ds := range deliveryServices {
+		if ds.XMLID == nil {
+			warnings = append(warnings, "deliveryServices had DS with nil xmlId (name)")
+			continue
+		}
+		if *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' not found")
+	}
+
+	if tcDS.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' missing cdn")
+	}
+
+	ds, err := headerRewriteDSFromDS(&tcDS)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, "converting ds to config ds: "+err.Error())
+	}
+
+	dsServers := filterDSS(deliveryServiceServers, map[int]struct{}{ds.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{}{}
+	}
+
+	assignedEdges := []headerRewriteServer{}
+	for _, server := range servers {
+		if server.CDNName == nil {
+			warnings = append(warnings, "servers had server with missing cdnName, skipping!")
+			continue
+		}
+		if server.ID == nil {
+			warnings = append(warnings, "servers had server with missing kid, skipping!")

Review comment:
       Or our Servers are doing some strange DNSSEC things.
   But yes, typo, fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525517301



##########
File path: lib/go-atscfg/remapdotconfig.go
##########
@@ -192,37 +218,39 @@ func GetServerConfigRemapDotConfigForMid(
 
 	text := header
 	text += strings.Join(textLines, "")
-	return text
+	return text, warnings, nil
 }
 
-func GetServerConfigRemapDotConfigForEdge(
+// GetServerConfigRemapDotConfigForEdge returns the remap lines, any warnings, and any error

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525512984



##########
File path: lib/go-atscfg/meta.go
##########
@@ -20,102 +20,148 @@ package atscfg
  */
 
 import (
-	"encoding/json"
 	"errors"
 	"path/filepath"
 	"strings"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
-type ConfigProfileParams struct {
-	FileNameOnDisk string
-	Location       string
-	URL            string
+type CfgMeta struct {
+	Name string
+	Path string
 }
 
-// APIVersion is the Traffic Ops API version for config fiels.
-// This is used to generate the meta config, which has API paths.
-// Note the version in the meta config is not used by the atstccfg generator, which isn't actually an API.
-// TODO change the config system to not use old API paths, and remove this.
-const APIVersion = "2.0"
+// MakeMetaObj returns the list of config files, any warnings, and any errors.
+func MakeConfigFilesList(
+	configDir string,
+	server *Server,
+	serverParams []tc.Parameter,
+	deliveryServices []DeliveryService,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	globalParams []tc.Parameter,
+	cacheGroupArr []tc.CacheGroupNullable,
+	topologies []tc.Topology,
+) ([]CfgMeta, []string, error) {
+	warnings := []string{}
 
-// requiredFiles is a constant (because Go doesn't allow const slices).
-// Note these are not exhaustive. This is only used to error if these are missing.
-// The presence of these is no guarantee the location Parameters are complete and correct.
-func requiredFiles() []string {
-	return []string{
-		"cache.config",
-		"hosting.config",
-		"ip_allow.config",
-		"parent.config",
-		"plugin.config",
-		"records.config",
-		"remap.config",
-		"storage.config",
-		"volume.config",
+	if server.Cachegroup == nil {
+		return nil, warnings, errors.New("this server missing Cachegroup")
+	} else if server.CachegroupID == nil {
+		return nil, warnings, errors.New("this server missing CachegroupID")
+	} else if server.ProfileID == nil {
+		return nil, warnings, errors.New("server missing ProfileID")
+	} else if server.TCPPort == nil {
+		return nil, warnings, errors.New("server missing TCPPort")
+	} else if server.HostName == nil {
+		return nil, warnings, errors.New("server missing HostName")
+	} else if server.CDNID == nil {
+		return nil, warnings, errors.New("server missing CDNID")
+	} else if server.CDNName == nil {
+		return nil, warnings, errors.New("server missing CDNName")
+	} else if server.ID == nil {
+		return nil, warnings, errors.New("server missing ID")
+	} else if server.Profile == nil {
+		return nil, warnings, errors.New("server missing Profile")
 	}
-}
 
-func MakeMetaConfig(
-	server *tc.ServerNullable,
-	tmURL string, // global tm.url Parameter
-	tmReverseProxyURL string, // global tm.rev_proxy.url Parameter
-	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
-	dses map[tc.DeliveryServiceName]tc.DeliveryServiceNullableV30,
-	cacheGroupArr []tc.CacheGroupNullable,
-	topologies []tc.Topology,
-) string {
-	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(server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cacheGroupArr, topologies, configDir))
-}
+	tmURL, tmReverseProxyURL := getTOURLAndReverseProxy(globalParams)
+	if tmURL == "" {
+		warnings = append(warnings, "global tm.url parameter missing or empty! Setting empty in meta config!")
+	}
 
-func MetaObjToMetaConfig(atsData tc.ATSConfigMetaData, err error) string {
-	if err != nil {
-		return "error creating meta config: " + err.Error()
+	dses, dsWarns := filterConfigFileDSes(server, deliveryServices, deliveryServiceServers)
+	warnings = append(warnings, dsWarns...)
+
+	locationParams := getLocationParams(serverParams)
+
+	uriSignedDSes, signDSWarns := getURISignedDSes(dses)
+	warnings = append(warnings, signDSWarns...)
+
+	configFiles := []CfgMeta{}
+
+	if locationParams["remap.config"].Path != "" {
+		configLocation := locationParams["remap.config"].Path
+		for _, ds := range uriSignedDSes {
+			cfgName := "uri_signing_" + string(ds) + ".config"
+			// If there's already a parameter for it, don't clobber it. The user may wish to override the location.
+			if _, ok := locationParams[cfgName]; !ok {
+				p := locationParams[cfgName]
+				p.Name = cfgName
+				p.Path = configLocation
+				locationParams[cfgName] = p
+			}
+		}
 	}
-	bts, err := json.Marshal(atsData)
-	if err != nil {
-		// should never happen
-		log.Errorln("marshalling meta config: " + err.Error())
-		bts = []byte("error encoding to json, see log for details")
+
+locationParamsFor:

Review comment:
       To `continue` and `break` out of the outer loop, from the inner loop:
   https://github.com/apache/trafficcontrol/blob/c28a6dc95577796bdefa0da935ab520dcf08f63f/lib/go-atscfg/meta.go#L113
   https://github.com/apache/trafficcontrol/blob/c28a6dc95577796bdefa0da935ab520dcf08f63f/lib/go-atscfg/meta.go#L115
   The `prefixFor` label isn't technically necessary, but I think it makes it clearer, since there are multiple loops we're continuing and breaking from.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r524813764



##########
File path: lib/go-atscfg/headerrewritedotconfig.go
##########
@@ -39,7 +39,127 @@ const ServiceCategoryHeader = "CDN-SVC"
 
 const MaxOriginConnectionsNoMax = 0 // 0 indicates no limit on origin connections
 
-type HeaderRewriteDS struct {
+func MakeHeaderRewriteDotConfig(
+	fileName string,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+
+	dsName := strings.TrimSuffix(strings.TrimPrefix(fileName, HeaderRewritePrefix), ConfigSuffix) // TODO verify prefix and suffix? Perl doesn't
+
+	tcDS := tc.DeliveryServiceNullableV30{}
+	for _, ds := range deliveryServices {
+		if ds.XMLID == nil {
+			warnings = append(warnings, "deliveryServices had DS with nil xmlId (name)")
+			continue
+		}
+		if *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' not found")
+	}
+
+	if tcDS.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' missing cdn")
+	}
+
+	ds, err := headerRewriteDSFromDS(&tcDS)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, "converting ds to config ds: "+err.Error())
+	}
+
+	dsServers := filterDSS(deliveryServiceServers, map[int]struct{}{ds.ID: {}}, nil)
+
+	dsServerIDs := map[int]struct{}{}
+	for _, dss := range dsServers {
+		if dss.Server == nil || dss.DeliveryService == nil {
+			continue // TODO warn?

Review comment:
       Yep, same as above, I just missed it. Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r518225146



##########
File path: lib/go-atscfg/astatsdotconfig.go
##########
@@ -19,23 +19,41 @@ package atscfg
  * under the License.
  */
 
+import (
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
 const AstatsSeparator = "="
 const AstatsFileName = "astats.config"
 
 const ContentTypeAstatsDotConfig = ContentTypeTextASCII
 const LineCommentAstatsDotConfig = LineCommentHash
 
 func MakeAStatsDotConfig(
-	profileName string,
-	paramData map[string]string, // GetProfileParamData(tx, profile.ID, AstatsFileName)
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-) string {
-	hdr := GenericHeaderComment(profileName, toToolName, toURL)
+	server *tc.ServerNullable,

Review comment:
       I opened a PR to fix it: #5248 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525513995



##########
File path: lib/go-atscfg/packages.go
##########
@@ -32,39 +32,48 @@ const PackagesParamConfigFile = `package`
 const ContentTypePackages = ContentTypeTextASCII

Review comment:
       Yes, it's doing that to emulate old Perl behavior, which I've been hesitant to change for fear of breaking something.
   We could probably change this safely, but I'd prefer to do it in a separate PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525524113



##########
File path: lib/go-atscfg/meta.go
##########
@@ -20,102 +20,148 @@ package atscfg
  */
 
 import (
-	"encoding/json"
 	"errors"
 	"path/filepath"
 	"strings"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
-type ConfigProfileParams struct {
-	FileNameOnDisk string
-	Location       string
-	URL            string
+type CfgMeta struct {
+	Name string
+	Path string
 }
 
-// APIVersion is the Traffic Ops API version for config fiels.
-// This is used to generate the meta config, which has API paths.
-// Note the version in the meta config is not used by the atstccfg generator, which isn't actually an API.
-// TODO change the config system to not use old API paths, and remove this.
-const APIVersion = "2.0"
+// MakeMetaObj returns the list of config files, any warnings, and any errors.
+func MakeConfigFilesList(
+	configDir string,
+	server *Server,
+	serverParams []tc.Parameter,
+	deliveryServices []DeliveryService,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	globalParams []tc.Parameter,
+	cacheGroupArr []tc.CacheGroupNullable,
+	topologies []tc.Topology,
+) ([]CfgMeta, []string, error) {
+	warnings := []string{}
 
-// requiredFiles is a constant (because Go doesn't allow const slices).
-// Note these are not exhaustive. This is only used to error if these are missing.
-// The presence of these is no guarantee the location Parameters are complete and correct.
-func requiredFiles() []string {
-	return []string{
-		"cache.config",
-		"hosting.config",
-		"ip_allow.config",
-		"parent.config",
-		"plugin.config",
-		"records.config",
-		"remap.config",
-		"storage.config",
-		"volume.config",
+	if server.Cachegroup == nil {
+		return nil, warnings, errors.New("this server missing Cachegroup")
+	} else if server.CachegroupID == nil {
+		return nil, warnings, errors.New("this server missing CachegroupID")
+	} else if server.ProfileID == nil {
+		return nil, warnings, errors.New("server missing ProfileID")
+	} else if server.TCPPort == nil {
+		return nil, warnings, errors.New("server missing TCPPort")
+	} else if server.HostName == nil {
+		return nil, warnings, errors.New("server missing HostName")
+	} else if server.CDNID == nil {
+		return nil, warnings, errors.New("server missing CDNID")
+	} else if server.CDNName == nil {
+		return nil, warnings, errors.New("server missing CDNName")
+	} else if server.ID == nil {
+		return nil, warnings, errors.New("server missing ID")
+	} else if server.Profile == nil {
+		return nil, warnings, errors.New("server missing Profile")
 	}
-}
 
-func MakeMetaConfig(
-	server *tc.ServerNullable,
-	tmURL string, // global tm.url Parameter
-	tmReverseProxyURL string, // global tm.rev_proxy.url Parameter
-	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
-	dses map[tc.DeliveryServiceName]tc.DeliveryServiceNullableV30,
-	cacheGroupArr []tc.CacheGroupNullable,
-	topologies []tc.Topology,
-) string {
-	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(server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cacheGroupArr, topologies, configDir))
-}
+	tmURL, tmReverseProxyURL := getTOURLAndReverseProxy(globalParams)
+	if tmURL == "" {
+		warnings = append(warnings, "global tm.url parameter missing or empty! Setting empty in meta config!")
+	}
 
-func MetaObjToMetaConfig(atsData tc.ATSConfigMetaData, err error) string {
-	if err != nil {
-		return "error creating meta config: " + err.Error()
+	dses, dsWarns := filterConfigFileDSes(server, deliveryServices, deliveryServiceServers)
+	warnings = append(warnings, dsWarns...)
+
+	locationParams := getLocationParams(serverParams)
+
+	uriSignedDSes, signDSWarns := getURISignedDSes(dses)
+	warnings = append(warnings, signDSWarns...)
+
+	configFiles := []CfgMeta{}
+
+	if locationParams["remap.config"].Path != "" {
+		configLocation := locationParams["remap.config"].Path
+		for _, ds := range uriSignedDSes {
+			cfgName := "uri_signing_" + string(ds) + ".config"
+			// If there's already a parameter for it, don't clobber it. The user may wish to override the location.
+			if _, ok := locationParams[cfgName]; !ok {
+				p := locationParams[cfgName]
+				p.Name = cfgName
+				p.Path = configLocation
+				locationParams[cfgName] = p
+			}
+		}
 	}
-	bts, err := json.Marshal(atsData)
-	if err != nil {
-		// should never happen
-		log.Errorln("marshalling meta config: " + err.Error())
-		bts = []byte("error encoding to json, see log for details")
+
+locationParamsFor:

Review comment:
       oh, I see.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525524649



##########
File path: lib/go-atscfg/packages.go
##########
@@ -32,39 +32,48 @@ const PackagesParamConfigFile = `package`
 const ContentTypePackages = ContentTypeTextASCII

Review comment:
       I think the temporary Go rewrite at one point had that fixed.
   
   > I'd prefer to do it in a separate PR
   
   Fair.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525504320



##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,56 +22,90 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const InvalidID = -1
-
 const DefaultATSVersion = "5" // TODO Emulates Perl; change to 6? ATC no longer officially supports ATS 5.
-
 const HeaderCommentDateFormat = "Mon Jan 2 15:04:05 MST 2006"
-
 const ContentTypeTextASCII = `text/plain; charset=us-ascii`
-
 const LineCommentHash = "#"
+const ConfigSuffix = ".config"
+
+type DeliveryServiceID int
+type ProfileID int
+type ServerID int
 
 type TopologyName string
 type CacheGroupType string
 type ServerCapability string
 
-type ServerInfo struct {
-	CacheGroupID                  int
-	CacheGroupName                string
-	CDN                           tc.CDNName
-	CDNID                         int
-	DomainName                    string
-	HostName                      string
-	HTTPSPort                     int
-	ID                            int
-	IP                            string
-	ParentCacheGroupID            int
-	ParentCacheGroupType          string
-	ProfileID                     ProfileID
-	ProfileName                   string
-	Port                          int
-	SecondaryParentCacheGroupID   int
-	SecondaryParentCacheGroupType string
-	Type                          string
-}
-
-func (s *ServerInfo) IsTopLevelCache() bool {
-	return (s.ParentCacheGroupType == tc.CacheGroupOriginTypeName || s.ParentCacheGroupID == InvalidID) &&
-		(s.SecondaryParentCacheGroupType == tc.CacheGroupOriginTypeName || s.SecondaryParentCacheGroupID == InvalidID)
-}
-
-func MakeCGMap(cgs []tc.CacheGroupNullable) (map[tc.CacheGroupName]tc.CacheGroupNullable, error) {
+// Server is a tc.Server for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type Server tc.ServerV30
+
+// DeliveryService is a tc.DeliveryService for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type DeliveryService tc.DeliveryServiceNullableV30
+
+// ToDeliveryServices converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToDeliveryServices(dses []tc.DeliveryServiceNullableV30) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		ad = append(ad, DeliveryService(ds))
+	}
+	return ad
+}
+
+// OldToDeliveryServices converts a slice of the old traffic_ops/client type to the local alias.
+func OldToDeliveryServices(dses []tc.DeliveryServiceNullable) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		upgradedDS := tc.DeliveryServiceNullableV30{DeliveryServiceNullableV15: tc.DeliveryServiceNullableV15(ds)}
+		ad = append(ad, DeliveryService(upgradedDS))
+	}
+	return ad
+}
+
+// ToServers converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToServers(servers []tc.ServerV30) []Server {
+	as := []Server{}
+	for _, sv := range servers {
+		as = append(as, Server(sv))
+	}
+	return as
+}

Review comment:
       Up to you, just a suggestion.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r518162230



##########
File path: lib/go-atscfg/astatsdotconfig.go
##########
@@ -19,23 +19,41 @@ package atscfg
  * under the License.
  */
 
+import (
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
 const AstatsSeparator = "="
 const AstatsFileName = "astats.config"
 
 const ContentTypeAstatsDotConfig = ContentTypeTextASCII
 const LineCommentAstatsDotConfig = LineCommentHash
 
 func MakeAStatsDotConfig(
-	profileName string,
-	paramData map[string]string, // GetProfileParamData(tx, profile.ID, AstatsFileName)
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-) string {
-	hdr := GenericHeaderComment(profileName, toToolName, toURL)
+	server *tc.ServerNullable,

Review comment:
       Yep, looks like `github.com/apache/trafficcontrol/traffic_ops/client.GetServers` changed call signatures between 4.1.1 and 5.0; used to return `[]tc.Server`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#issuecomment-724166801


   Fixed conflicts


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r524744540



##########
File path: lib/go-atscfg/headerrewritedotconfig.go
##########
@@ -39,7 +39,127 @@ const ServiceCategoryHeader = "CDN-SVC"
 
 const MaxOriginConnectionsNoMax = 0 // 0 indicates no limit on origin connections
 
-type HeaderRewriteDS struct {
+func MakeHeaderRewriteDotConfig(
+	fileName string,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+
+	dsName := strings.TrimSuffix(strings.TrimPrefix(fileName, HeaderRewritePrefix), ConfigSuffix) // TODO verify prefix and suffix? Perl doesn't
+
+	tcDS := tc.DeliveryServiceNullableV30{}
+	for _, ds := range deliveryServices {
+		if ds.XMLID == nil {
+			warnings = append(warnings, "deliveryServices had DS with nil xmlId (name)")
+			continue
+		}
+		if *ds.XMLID != dsName {
+			continue
+		}
+		tcDS = ds
+		break
+	}
+	if tcDS.ID == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' not found")
+	}
+
+	if tcDS.CDNName == nil {
+		return Cfg{}, makeErr(warnings, "ds '"+dsName+"' missing cdn")
+	}
+
+	ds, err := headerRewriteDSFromDS(&tcDS)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, "converting ds to config ds: "+err.Error())
+	}
+
+	dsServers := filterDSS(deliveryServiceServers, map[int]struct{}{ds.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{}{}
+	}
+
+	assignedEdges := []headerRewriteServer{}
+	for _, server := range servers {
+		if server.CDNName == nil {
+			warnings = append(warnings, "servers had server with missing cdnName, skipping!")
+			continue
+		}
+		if server.ID == nil {
+			warnings = append(warnings, "servers had server with missing kid, skipping!")

Review comment:
       either that or we need to add some kind of `printToMilkCarton` handler for things like this




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525449058



##########
File path: lib/go-atscfg/meta.go
##########
@@ -20,102 +20,148 @@ package atscfg
  */
 
 import (
-	"encoding/json"
 	"errors"
 	"path/filepath"
 	"strings"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
-type ConfigProfileParams struct {
-	FileNameOnDisk string
-	Location       string
-	URL            string
+type CfgMeta struct {
+	Name string
+	Path string
 }
 
-// APIVersion is the Traffic Ops API version for config fiels.
-// This is used to generate the meta config, which has API paths.
-// Note the version in the meta config is not used by the atstccfg generator, which isn't actually an API.
-// TODO change the config system to not use old API paths, and remove this.
-const APIVersion = "2.0"
+// MakeMetaObj returns the list of config files, any warnings, and any errors.
+func MakeConfigFilesList(
+	configDir string,
+	server *Server,
+	serverParams []tc.Parameter,
+	deliveryServices []DeliveryService,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	globalParams []tc.Parameter,
+	cacheGroupArr []tc.CacheGroupNullable,
+	topologies []tc.Topology,
+) ([]CfgMeta, []string, error) {
+	warnings := []string{}
 
-// requiredFiles is a constant (because Go doesn't allow const slices).
-// Note these are not exhaustive. This is only used to error if these are missing.
-// The presence of these is no guarantee the location Parameters are complete and correct.
-func requiredFiles() []string {
-	return []string{
-		"cache.config",
-		"hosting.config",
-		"ip_allow.config",
-		"parent.config",
-		"plugin.config",
-		"records.config",
-		"remap.config",
-		"storage.config",
-		"volume.config",
+	if server.Cachegroup == nil {
+		return nil, warnings, errors.New("this server missing Cachegroup")
+	} else if server.CachegroupID == nil {
+		return nil, warnings, errors.New("this server missing CachegroupID")
+	} else if server.ProfileID == nil {
+		return nil, warnings, errors.New("server missing ProfileID")
+	} else if server.TCPPort == nil {
+		return nil, warnings, errors.New("server missing TCPPort")
+	} else if server.HostName == nil {
+		return nil, warnings, errors.New("server missing HostName")
+	} else if server.CDNID == nil {
+		return nil, warnings, errors.New("server missing CDNID")
+	} else if server.CDNName == nil {
+		return nil, warnings, errors.New("server missing CDNName")
+	} else if server.ID == nil {
+		return nil, warnings, errors.New("server missing ID")
+	} else if server.Profile == nil {
+		return nil, warnings, errors.New("server missing Profile")
 	}
-}
 
-func MakeMetaConfig(
-	server *tc.ServerNullable,
-	tmURL string, // global tm.url Parameter
-	tmReverseProxyURL string, // global tm.rev_proxy.url Parameter
-	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
-	dses map[tc.DeliveryServiceName]tc.DeliveryServiceNullableV30,
-	cacheGroupArr []tc.CacheGroupNullable,
-	topologies []tc.Topology,
-) string {
-	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(server, tmURL, tmReverseProxyURL, locationParams, uriSignedDSes, scopeParams, dses, cacheGroupArr, topologies, configDir))
-}
+	tmURL, tmReverseProxyURL := getTOURLAndReverseProxy(globalParams)
+	if tmURL == "" {
+		warnings = append(warnings, "global tm.url parameter missing or empty! Setting empty in meta config!")
+	}
 
-func MetaObjToMetaConfig(atsData tc.ATSConfigMetaData, err error) string {
-	if err != nil {
-		return "error creating meta config: " + err.Error()
+	dses, dsWarns := filterConfigFileDSes(server, deliveryServices, deliveryServiceServers)
+	warnings = append(warnings, dsWarns...)
+
+	locationParams := getLocationParams(serverParams)
+
+	uriSignedDSes, signDSWarns := getURISignedDSes(dses)
+	warnings = append(warnings, signDSWarns...)
+
+	configFiles := []CfgMeta{}
+
+	if locationParams["remap.config"].Path != "" {
+		configLocation := locationParams["remap.config"].Path
+		for _, ds := range uriSignedDSes {
+			cfgName := "uri_signing_" + string(ds) + ".config"
+			// If there's already a parameter for it, don't clobber it. The user may wish to override the location.
+			if _, ok := locationParams[cfgName]; !ok {
+				p := locationParams[cfgName]
+				p.Name = cfgName
+				p.Path = configLocation
+				locationParams[cfgName] = p
+			}
+		}
 	}
-	bts, err := json.Marshal(atsData)
-	if err != nil {
-		// should never happen
-		log.Errorln("marshalling meta config: " + err.Error())
-		bts = []byte("error encoding to json, see log for details")
+
+locationParamsFor:

Review comment:
       I don't see a `goto` anywhere; what are these labels for?

##########
File path: lib/go-atscfg/remapdotconfig.go
##########
@@ -35,81 +34,104 @@ const CacheKeyParameterConfigFile = "cachekey.config"
 const ContentTypeRemapDotConfig = ContentTypeTextASCII
 const LineCommentRemapDotConfig = LineCommentHash
 
+const RemapConfigRangeDirective = `__RANGE_DIRECTIVE__`
+
 func MakeRemapDotConfig(
-	server *tc.ServerNullable,
-	unfilteredDSes []tc.DeliveryServiceNullableV30,
+	server *Server,
+	unfilteredDSes []DeliveryService,
 	dss []tc.DeliveryServiceServer,
 	dsRegexArr []tc.DeliveryServiceRegexes,
 	serverParams []tc.Parameter,
 	cdn *tc.CDN,
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
 	cacheKeyParams []tc.Parameter,
 	topologies []tc.Topology,
 	cacheGroupArr []tc.CacheGroupNullable,
 	serverCapabilities map[int]map[ServerCapability]struct{},
 	dsRequiredCapabilities map[int]map[ServerCapability]struct{},
-) string {
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
 	if server.HostName == nil {
-		return "ERROR: server HostName missing"
+		return Cfg{}, makeErr(warnings, "server HostName missing")
 	} else if server.ID == nil {
-		return "ERROR: server ID missing"
+		return Cfg{}, makeErr(warnings, "server ID missing")
 	} else if server.Cachegroup == nil {
-		return "ERROR: server Cachegroup missing"
+		return Cfg{}, makeErr(warnings, "server Cachegroup missing")
 	} else if server.DomainName == nil {
-		return "ERROR: server DomainName missing"
+		return Cfg{}, makeErr(warnings, "server DomainName missing")
 	}
 
 	cdnDomain := cdn.DomainName
 	dsRegexes := makeDSRegexMap(dsRegexArr)
 	// Returned DSes are guaranteed to have a non-nil XMLID, Type, DSCP, ID, and Active.
-	dses := remapFilterDSes(server, dss, unfilteredDSes, cacheKeyParams)
+	dses, dsWarns := remapFilterDSes(server, dss, unfilteredDSes, cacheKeyParams)
+	warnings = append(warnings, dsWarns...)
 
-	dsProfilesCacheKeyConfigParams, err := makeDSProfilesCacheKeyConfigParams(server, dses, cacheKeyParams)
+	dsProfilesCacheKeyConfigParams, paramWarns, err := makeDSProfilesCacheKeyConfigParams(server, dses, cacheKeyParams)
+	warnings = append(warnings, paramWarns...)
 	if err != nil {
-		log.Errorln("Error making Delivery Service Cache Key Params, cache key will be missing! : " + err.Error())
+		warnings = append(warnings, "making Delivery Service Cache Key Params, cache key will be missing! : "+err.Error())
 	}
 
-	atsMajorVersion := getATSMajorVersion(serverParams)
-	serverPackageParamData := makeServerPackageParamData(server, serverParams)
-	cacheURLConfigParams := ParamsToMap(FilterParams(serverParams, CacheURLParameterConfigFile, "", "", ""))
-	cacheGroups, err := MakeCGMap(cacheGroupArr)
+	atsMajorVersion, verWarns := getATSMajorVersion(serverParams)
+	warnings = append(warnings, verWarns...)
+	serverPackageParamData, paramWarns := makeServerPackageParamData(server, serverParams)
+	warnings = append(warnings, paramWarns...)
+	cacheURLConfigParams, paramWarns := paramsToMap(filterParams(serverParams, CacheURLParameterConfigFile, "", "", ""))
+	warnings = append(warnings, paramWarns...)
+	cacheGroups, err := makeCGMap(cacheGroupArr)
 	if err != nil {
-		log.Errorln("making remap.config, config will be malformed! : " + err.Error())
+		return Cfg{}, makeErr(warnings, "making remap.config, config will be malformed! : "+err.Error())
 	}
 
-	nameTopologies := MakeTopologyNameMap(topologies)
+	nameTopologies := makeTopologyNameMap(topologies)
 
-	hdr := GenericHeaderComment(*server.HostName, toToolName, toURL)
+	hdr := makeHdrComment(hdrComment)
+	txt := ""
+	typeWarns := []string{}
 	if tc.CacheTypeFromString(server.Type) == tc.CacheTypeMid {
-		return GetServerConfigRemapDotConfigForMid(atsMajorVersion, dsProfilesCacheKeyConfigParams, dses, dsRegexes, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities)
+		txt, typeWarns, err = getServerConfigRemapDotConfigForMid(atsMajorVersion, dsProfilesCacheKeyConfigParams, dses, dsRegexes, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities)
+	} else {
+		txt, typeWarns, err = getServerConfigRemapDotConfigForEdge(cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, dses, dsRegexes, atsMajorVersion, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities, cdnDomain)
 	}
-	return GetServerConfigRemapDotConfigForEdge(cacheURLConfigParams, dsProfilesCacheKeyConfigParams, serverPackageParamData, dses, dsRegexes, atsMajorVersion, hdr, server, nameTopologies, cacheGroups, serverCapabilities, dsRequiredCapabilities, cdnDomain)
+	warnings = append(warnings, typeWarns...)
+	if err != nil {
+		return Cfg{}, makeErr(warnings, err.Error()) // the GetFor funcs include error context
+	}
+
+	return Cfg{
+		Text:        txt,
+		ContentType: ContentTypeRemapDotConfig,
+		LineComment: LineCommentRemapDotConfig,
+		Warnings:    warnings,
+	}, nil
 }
 
-func GetServerConfigRemapDotConfigForMid(
+// GetServerConfigRemapDotConfigForMid returns the remap lines, any warnings, and any error.

Review comment:
       Same as above RE: GoDoc

##########
File path: lib/go-atscfg/parentdotconfig.go
##########
@@ -1001,62 +1030,66 @@ func GetTopologyParents(
 			continue
 		}
 
-		if !HasRequiredCapabilities(serverCapabilities[*sv.ID], dsRequiredCapabilities[*ds.ID]) {
+		if !hasRequiredCapabilities(serverCapabilities[*sv.ID], dsRequiredCapabilities[*ds.ID]) {
 			continue
 		}
 		if *sv.Cachegroup == parentCG {
-			parentStr, err := serverParentStr(&sv.ServerNullable, sv.Params)
+			parentStr, err := serverParentStr(&sv.Server, sv.Params)
 			if err != nil {
-				return nil, nil, errors.New("getting server parent string: " + err.Error())
+				return nil, nil, warnings, errors.New("getting server parent string: " + err.Error())
 			}
 			if parentStr != "" { // will be empty if server is not_a_parent (possibly other reasons)
 				parentStrs = append(parentStrs, parentStr)
 			}
 		}
 		if *sv.Cachegroup == secondaryParentCG {
-			parentStr, err := serverParentStr(&sv.ServerNullable, sv.Params)
+			parentStr, err := serverParentStr(&sv.Server, sv.Params)
 			if err != nil {
-				return nil, nil, errors.New("getting server parent string: " + err.Error())
+				return nil, nil, warnings, errors.New("getting server parent string: " + err.Error())
 			}
 			secondaryParentStrs = append(secondaryParentStrs, parentStr)
 		}
 	}
 
-	return parentStrs, secondaryParentStrs, nil
+	return parentStrs, secondaryParentStrs, warnings, nil
 }
 
-func GetOriginURI(fqdn string) (*url.URL, error) {
+// GetOriginURI returns the URL, any warnings, and any error.

Review comment:
       Nit: GoDoc should start with the name of the documented symbol.

##########
File path: lib/go-atscfg/packages.go
##########
@@ -32,39 +32,48 @@ const PackagesParamConfigFile = `package`
 const ContentTypePackages = ContentTypeTextASCII

Review comment:
       I know this didn't change in this PR, but isn't this actually a JSON-encoded output?

##########
File path: lib/go-atscfg/cachedotconfig.go
##########
@@ -23,33 +23,106 @@ import (
 	"sort"
 	"strings"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const ContentTypeCacheDotConfig = ContentTypeTextASCII
 const LineCommentCacheDotConfig = LineCommentHash
 
-type ProfileDS struct {
-	Type       tc.DSType
-	OriginFQDN *string
+func MakeCacheDotConfig(
+	server *Server,
+	servers []Server,
+	deliveryServices []DeliveryService,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	hdrComment string,
+) (Cfg, error) {
+	if tc.CacheTypeFromString(server.Type) == tc.CacheTypeMid {
+		return makeCacheDotConfigMid(server, deliveryServices, hdrComment)
+	} else {
+		return makeCacheDotConfigEdge(server, servers, deliveryServices, deliveryServiceServers, hdrComment)
+	}
 }
 
 // MakeCacheDotConfig makes the ATS cache.config config file.
 // profileDSes must be the list of delivery services, which are assigned to severs, for which this profile is assigned. It MUST NOT contain any other delivery services. Note DSesToProfileDSes may be helpful if you have a []tc.DeliveryServiceNullable, for example from traffic_ops/client.

Review comment:
       I think you wanna move this GoDoc up to the `MakeCacheDotConfig` function.

##########
File path: lib/go-atscfg/parentdotconfig.go
##########
@@ -1408,32 +1442,33 @@ func GetParentConfigProfileParams(
 		}
 		parentConfigServerCacheProfileParams[*cgServer.Profile] = profileCache
 	}
-	return parentConfigServerCacheProfileParams
+	return parentConfigServerCacheProfileParams, warnings
 }
 
-// GetDSOrigins takes a map[deliveryServiceID]DeliveryService, and returns a map[DeliveryServiceID]OriginURI.
-func GetDSOrigins(dses map[int]tc.DeliveryServiceNullableV30) (map[int]*OriginURI, error) {
-	dsOrigins := map[int]*OriginURI{}
+// GetDSOrigins takes a map[deliveryServiceID]DeliveryService, and returns a map[DeliveryServiceID]OriginURI, any warnings, and any error.

Review comment:
       same as above RE: GoDoc

##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,56 +22,90 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const InvalidID = -1
-
 const DefaultATSVersion = "5" // TODO Emulates Perl; change to 6? ATC no longer officially supports ATS 5.
-
 const HeaderCommentDateFormat = "Mon Jan 2 15:04:05 MST 2006"
-
 const ContentTypeTextASCII = `text/plain; charset=us-ascii`
-
 const LineCommentHash = "#"
+const ConfigSuffix = ".config"
+
+type DeliveryServiceID int
+type ProfileID int
+type ServerID int
 
 type TopologyName string
 type CacheGroupType string
 type ServerCapability string
 
-type ServerInfo struct {
-	CacheGroupID                  int
-	CacheGroupName                string
-	CDN                           tc.CDNName
-	CDNID                         int
-	DomainName                    string
-	HostName                      string
-	HTTPSPort                     int
-	ID                            int
-	IP                            string
-	ParentCacheGroupID            int
-	ParentCacheGroupType          string
-	ProfileID                     ProfileID
-	ProfileName                   string
-	Port                          int
-	SecondaryParentCacheGroupID   int
-	SecondaryParentCacheGroupType string
-	Type                          string
-}
-
-func (s *ServerInfo) IsTopLevelCache() bool {
-	return (s.ParentCacheGroupType == tc.CacheGroupOriginTypeName || s.ParentCacheGroupID == InvalidID) &&
-		(s.SecondaryParentCacheGroupType == tc.CacheGroupOriginTypeName || s.SecondaryParentCacheGroupID == InvalidID)
-}
-
-func MakeCGMap(cgs []tc.CacheGroupNullable) (map[tc.CacheGroupName]tc.CacheGroupNullable, error) {
+// Server is a tc.Server for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type Server tc.ServerV30
+
+// DeliveryService is a tc.DeliveryService for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type DeliveryService tc.DeliveryServiceNullableV30
+
+// ToDeliveryServices converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToDeliveryServices(dses []tc.DeliveryServiceNullableV30) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		ad = append(ad, DeliveryService(ds))
+	}
+	return ad
+}
+
+// OldToDeliveryServices converts a slice of the old traffic_ops/client type to the local alias.
+func OldToDeliveryServices(dses []tc.DeliveryServiceNullable) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		upgradedDS := tc.DeliveryServiceNullableV30{DeliveryServiceNullableV15: tc.DeliveryServiceNullableV15(ds)}
+		ad = append(ad, DeliveryService(upgradedDS))
+	}
+	return ad
+}
+
+// ToServers converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToServers(servers []tc.ServerV30) []Server {
+	as := []Server{}
+	for _, sv := range servers {
+		as = append(as, Server(sv))
+	}
+	return as
+}

Review comment:
       I know you don't like it, but since Servers and Delivery Services are ATC's most populous objects (in implementations of which I'm aware) I think there could be a somewhat significant performance gain by just using `make` instead of re-allocating multiple times on these long loops.

##########
File path: lib/go-atscfg/sslmulticertdotconfig.go
##########
@@ -95,11 +112,11 @@ func GetSSLMultiCertDotConfigCertAndKeyName(dsName tc.DeliveryServiceName, ds SS
 	return cerName, keyName
 }
 
-// GetSSLMultiCertDotConfigDeliveryServices takes a list of delivery services, and returns the delivery services which will be inserted into the config by MakeSSLMultiCertDotConfig.
+// getSSLMultiCertDotConfigDeliveryServices takes a list of delivery services, and returns the delivery services which will be inserted into the config by MakeSSLMultiCertDotConfig.

Review comment:
       Same as above RE: GoDoc

##########
File path: lib/go-atscfg/parentdotconfig.go
##########
@@ -1219,46 +1257,38 @@ func unavailableServerRetryResponsesValid(s string) bool {
 	return re.MatchString(s)
 }
 
-// HasRequiredCapabilities returns whether the given caps has all the required capabilities in the given reqCaps.
-func HasRequiredCapabilities(caps map[ServerCapability]struct{}, reqCaps map[ServerCapability]struct{}) bool {
-	for reqCap, _ := range reqCaps {
-		if _, ok := caps[reqCap]; !ok {
-			return false
-		}
-	}
-	return true
-}
-
-func GetOriginServersAndProfileCaches(
-	cgServers map[int]tc.ServerNullable,
+// GetOriginServersAndProfileCaches returns the origin servers, ProfileCaches, any warnings, and any error.

Review comment:
       same as above RE: GoDoc

##########
File path: lib/go-atscfg/servercachedotconfig.go
##########
@@ -27,18 +27,36 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
-type ServerCacheConfigDS struct {
-	OrgServerFQDN string
-	Type          tc.DSType
-}
+const ServerCacheDotConfigIncludeInactiveDSes = false // TODO move to lib/go-atscfg

Review comment:
       More like... TODONE

##########
File path: lib/go-atscfg/remapdotconfig.go
##########
@@ -192,37 +218,39 @@ func GetServerConfigRemapDotConfigForMid(
 
 	text := header
 	text += strings.Join(textLines, "")
-	return text
+	return text, warnings, nil
 }
 
-func GetServerConfigRemapDotConfigForEdge(
+// GetServerConfigRemapDotConfigForEdge returns the remap lines, any warnings, and any error

Review comment:
       Same as above RE: GoDoc

##########
File path: lib/go-atscfg/remapdotconfig.go
##########
@@ -408,22 +448,22 @@ func MakeDSTopologyHeaderRewriteTxt(ds tc.DeliveryServiceNullableV30, cg tc.Cach
 	if placement.IsLastCacheTier && ds.LastHeaderRewrite != nil && *ds.LastHeaderRewrite != "" {
 		txt += pluginTxt + LastHeaderRewriteConfigFileName(*ds.XMLID) + ` `
 	}
-	return txt
+	return txt, nil
 }
 
-type RemapLine struct {
+type remapLine struct {
 	From string
 	To   string
 }
 
 // MakeEdgeDSDataRemapLines returns the remap lines for the given server and delivery service.

Review comment:
       Same as above RE: GoDoc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525506341



##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,56 +22,90 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const InvalidID = -1
-
 const DefaultATSVersion = "5" // TODO Emulates Perl; change to 6? ATC no longer officially supports ATS 5.
-
 const HeaderCommentDateFormat = "Mon Jan 2 15:04:05 MST 2006"
-
 const ContentTypeTextASCII = `text/plain; charset=us-ascii`
-
 const LineCommentHash = "#"
+const ConfigSuffix = ".config"
+
+type DeliveryServiceID int
+type ProfileID int
+type ServerID int
 
 type TopologyName string
 type CacheGroupType string
 type ServerCapability string
 
-type ServerInfo struct {
-	CacheGroupID                  int
-	CacheGroupName                string
-	CDN                           tc.CDNName
-	CDNID                         int
-	DomainName                    string
-	HostName                      string
-	HTTPSPort                     int
-	ID                            int
-	IP                            string
-	ParentCacheGroupID            int
-	ParentCacheGroupType          string
-	ProfileID                     ProfileID
-	ProfileName                   string
-	Port                          int
-	SecondaryParentCacheGroupID   int
-	SecondaryParentCacheGroupType string
-	Type                          string
-}
-
-func (s *ServerInfo) IsTopLevelCache() bool {
-	return (s.ParentCacheGroupType == tc.CacheGroupOriginTypeName || s.ParentCacheGroupID == InvalidID) &&
-		(s.SecondaryParentCacheGroupType == tc.CacheGroupOriginTypeName || s.SecondaryParentCacheGroupID == InvalidID)
-}
-
-func MakeCGMap(cgs []tc.CacheGroupNullable) (map[tc.CacheGroupName]tc.CacheGroupNullable, error) {
+// Server is a tc.Server for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type Server tc.ServerV30
+
+// DeliveryService is a tc.DeliveryService for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type DeliveryService tc.DeliveryServiceNullableV30
+
+// ToDeliveryServices converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToDeliveryServices(dses []tc.DeliveryServiceNullableV30) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		ad = append(ad, DeliveryService(ds))
+	}
+	return ad
+}
+
+// OldToDeliveryServices converts a slice of the old traffic_ops/client type to the local alias.
+func OldToDeliveryServices(dses []tc.DeliveryServiceNullable) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		upgradedDS := tc.DeliveryServiceNullableV30{DeliveryServiceNullableV15: tc.DeliveryServiceNullableV15(ds)}
+		ad = append(ad, DeliveryService(upgradedDS))
+	}
+	return ad
+}
+
+// ToServers converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToServers(servers []tc.ServerV30) []Server {
+	as := []Server{}
+	for _, sv := range servers {
+		as = append(as, Server(sv))
+	}
+	return as
+}

Review comment:
       ```
   func ToServers(servers []tc.ServerV30) []Server {
   	as := []Server{}
   	for _, sv := range servers {
   		as = append(as, Server(sv))
   	}
   	return as
   }
   func ToServersMake(servers []tc.ServerV30) []Server {
   	as := make([]Server, 0, len(servers))
   	for _, sv := range servers {
   		as = append(as, Server(sv))
   	}
   	return as
   }
   func BenchmarkToServers1mil(b *testing.B) {
   	num := 1000000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServers(servers)
   	}
   }
   func BenchmarkToServersMake1mil(b *testing.B) {
   	num := 1000000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServersMake(servers)
   	}
   }
   func BenchmarkToServers2k(b *testing.B) {
   	num := 2000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServers(servers)
   	}
   }
   func BenchmarkToServersMake2k(b *testing.B) {
   	num := 2000
   	for i := 0; i < b.N; i++ {
   		servers := make([]tc.ServerV30, num, num)
   		ToServersMake(servers)
   	}
   }
   ```
   ```
   $ go test -bench=.
   DEBUG moc 42 usesmid true assignedmids 3
   DEBUG moc 0 usesmid true assignedmids 0
   goos: darwin
   goarch: amd64
   pkg: github.com/apache/trafficcontrol/lib/go-atscfg
   BenchmarkToServers1mil-16                      1        1577990978 ns/op
   BenchmarkToServersMake1mil-16                  3         387240011 ns/op
   BenchmarkToServers2k-16                     1321            821199 ns/op
   BenchmarkToServersMake2k-16                 2414            417031 ns/op
   PASS
   ok      github.com/apache/trafficcontrol/lib/go-atscfg  8.505s
   ```
   So on my laptop, it takes 0.0016 milliseconds without `make` vs 0.00039 milliseconds with `make` per server, for a million servers,
   and 0.00041 milliseconds without vs 0.00021 milliseconds per server, for 2,000 servers.
   
   I think we can spare the performance to improve the readability.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525491625



##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,56 +22,90 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const InvalidID = -1
-
 const DefaultATSVersion = "5" // TODO Emulates Perl; change to 6? ATC no longer officially supports ATS 5.
-
 const HeaderCommentDateFormat = "Mon Jan 2 15:04:05 MST 2006"
-
 const ContentTypeTextASCII = `text/plain; charset=us-ascii`
-
 const LineCommentHash = "#"
+const ConfigSuffix = ".config"
+
+type DeliveryServiceID int
+type ProfileID int
+type ServerID int
 
 type TopologyName string
 type CacheGroupType string
 type ServerCapability string
 
-type ServerInfo struct {
-	CacheGroupID                  int
-	CacheGroupName                string
-	CDN                           tc.CDNName
-	CDNID                         int
-	DomainName                    string
-	HostName                      string
-	HTTPSPort                     int
-	ID                            int
-	IP                            string
-	ParentCacheGroupID            int
-	ParentCacheGroupType          string
-	ProfileID                     ProfileID
-	ProfileName                   string
-	Port                          int
-	SecondaryParentCacheGroupID   int
-	SecondaryParentCacheGroupType string
-	Type                          string
-}
-
-func (s *ServerInfo) IsTopLevelCache() bool {
-	return (s.ParentCacheGroupType == tc.CacheGroupOriginTypeName || s.ParentCacheGroupID == InvalidID) &&
-		(s.SecondaryParentCacheGroupType == tc.CacheGroupOriginTypeName || s.SecondaryParentCacheGroupID == InvalidID)
-}
-
-func MakeCGMap(cgs []tc.CacheGroupNullable) (map[tc.CacheGroupName]tc.CacheGroupNullable, error) {
+// Server is a tc.Server for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type Server tc.ServerV30
+
+// DeliveryService is a tc.DeliveryService for the latest lib/go-tc and traffic_ops/vx-client type.
+// This allows atscfg to not have to change the type everywhere it's used, every time ATC changes the base type,
+// but to only have to change it here, and the places where breaking symbol changes were made.
+type DeliveryService tc.DeliveryServiceNullableV30
+
+// ToDeliveryServices converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToDeliveryServices(dses []tc.DeliveryServiceNullableV30) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		ad = append(ad, DeliveryService(ds))
+	}
+	return ad
+}
+
+// OldToDeliveryServices converts a slice of the old traffic_ops/client type to the local alias.
+func OldToDeliveryServices(dses []tc.DeliveryServiceNullable) []DeliveryService {
+	ad := []DeliveryService{}
+	for _, ds := range dses {
+		upgradedDS := tc.DeliveryServiceNullableV30{DeliveryServiceNullableV15: tc.DeliveryServiceNullableV15(ds)}
+		ad = append(ad, DeliveryService(upgradedDS))
+	}
+	return ad
+}
+
+// ToServers converts a slice of the latest lib/go-tc and traffic_ops/vx-client type to the local alias.
+func ToServers(servers []tc.ServerV30) []Server {
+	as := []Server{}
+	for _, sv := range servers {
+		as = append(as, Server(sv))
+	}
+	return as
+}

Review comment:
       Performance really doesn't matter here. It takes seconds to fetch from Traffic Ops, and the performance here is on the order of milliseconds at most. 
   
   I prefer to avoid `make` to use a unified variable declaration syntax where performance doesn't matter. I fully support using `make` where it does, but it really doesn't here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r524812082



##########
File path: lib/go-atscfg/cachedotconfig.go
##########
@@ -23,33 +23,95 @@ import (
 	"sort"
 	"strings"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 const ContentTypeCacheDotConfig = ContentTypeTextASCII
 const LineCommentCacheDotConfig = LineCommentHash
 
-type ProfileDS struct {
-	Type       tc.DSType
-	OriginFQDN *string
+func MakeCacheDotConfig(
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	hdrComment string,
+) (Cfg, error) {
+	if tc.CacheTypeFromString(server.Type) == tc.CacheTypeMid {
+		return makeCacheDotConfigMid(server, deliveryServices, hdrComment)
+	} else {
+		return makeCacheDotConfigEdge(server, servers, deliveryServices, deliveryServiceServers, hdrComment)
+	}
 }
 
 // MakeCacheDotConfig makes the ATS cache.config config file.
 // profileDSes must be the list of delivery services, which are assigned to severs, for which this profile is assigned. It MUST NOT contain any other delivery services. Note DSesToProfileDSes may be helpful if you have a []tc.DeliveryServiceNullable, for example from traffic_ops/client.
-func MakeCacheDotConfig(
-	profileName string,
-	profileDSes []ProfileDS,
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-) string {
+func makeCacheDotConfigEdge(
+	server *tc.ServerNullable,
+	servers []tc.ServerNullable,
+	deliveryServices []tc.DeliveryServiceNullableV30,
+	deliveryServiceServers []tc.DeliveryServiceServer,
+	hdrComment string,
+) (Cfg, error) {
+	warnings := []string{}
+
+	if server.Profile == nil {
+		return Cfg{}, makeErr(warnings, "server missing profile")
+	}
+
+	profileServerIDsMap := map[int]struct{}{}
+	for _, sv := range servers {
+		if sv.Profile == nil {
+			warnings = append(warnings, "servers had server with nil profile, skipping!")
+			continue
+		}
+		if sv.ID == nil {
+			warnings = append(warnings, "servers had server with nil id, skipping!")
+			continue
+		}
+		if *sv.Profile != *server.Profile {
+			continue
+		}
+		profileServerIDsMap[*sv.ID] = struct{}{}
+	}
+
+	dsServers := filterDSS(deliveryServiceServers, nil, profileServerIDsMap)
+
+	dsIDs := map[int]struct{}{}
+	for _, dss := range dsServers {
+		if dss.Server == nil || dss.DeliveryService == nil {
+			continue // TODO warn? err?
+		}
+		if _, ok := profileServerIDsMap[*dss.Server]; !ok {
+			continue
+		}
+		dsIDs[*dss.DeliveryService] = struct{}{}
+	}
+
+	profileDSes := []profileDS{}
+	for _, ds := range deliveryServices {
+		if ds.ID == nil || ds.Type == nil || ds.OrgServerFQDN == nil {
+			continue // TODO warn? err?
+		}
+		if *ds.Type == tc.DSTypeInvalid {
+			continue // TODO warn? err?
+		}
+		if *ds.OrgServerFQDN == "" {
+			continue // TODO warn? err?
+		}

Review comment:
       Yeah, moved from before it had the ability to return warnings, and I missed changing it. Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r517765804



##########
File path: lib/go-atscfg/astatsdotconfig.go
##########
@@ -19,23 +19,41 @@ package atscfg
  * under the License.
  */
 
+import (
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
 const AstatsSeparator = "="
 const AstatsFileName = "astats.config"
 
 const ContentTypeAstatsDotConfig = ContentTypeTextASCII
 const LineCommentAstatsDotConfig = LineCommentHash
 
 func MakeAStatsDotConfig(
-	profileName string,
-	paramData map[string]string, // GetProfileParamData(tx, profile.ID, AstatsFileName)
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-) string {
-	hdr := GenericHeaderComment(profileName, toToolName, toURL)
+	server *tc.ServerNullable,

Review comment:
       Currently, rolling aliases are not being updated because that breaks the API client package. So either this needs to be an API-version-specific structure like `ServerV30` (or is it `ServerNullableV30`?) or we need to revisit that decision.

##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,16 +22,36 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
+// ATSConfigFile is all the information necessary to create an ATS config file, including the file name, path, data, and metadata.

Review comment:
       GoDoc should start with the name of the documented symbol

##########
File path: lib/go-atscfg/atscfg.go
##########
@@ -22,16 +22,36 @@ package atscfg
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net"
 	"sort"
 	"strconv"
 	"strings"
-	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
+// ATSConfigFile is all the information necessary to create an ATS config file, including the file name, path, data, and metadata.
+// This is provided as a convenience and unified structure for users. The lib/go-atscfg library doesn't actually use or return this. See ATSConfigFileData.
+type CfgFile struct {
+	Name string
+	Path string
+	Cfg
+}
+
+// ATSConfigFileData is the data and metadata for an ATS Config File.

Review comment:
       Same as above RE: GoDoc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525515786



##########
File path: lib/go-atscfg/parentdotconfig.go
##########
@@ -1219,46 +1257,38 @@ func unavailableServerRetryResponsesValid(s string) bool {
 	return re.MatchString(s)
 }
 
-// HasRequiredCapabilities returns whether the given caps has all the required capabilities in the given reqCaps.
-func HasRequiredCapabilities(caps map[ServerCapability]struct{}, reqCaps map[ServerCapability]struct{}) bool {
-	for reqCap, _ := range reqCaps {
-		if _, ok := caps[reqCap]; !ok {
-			return false
-		}
-	}
-	return true
-}
-
-func GetOriginServersAndProfileCaches(
-	cgServers map[int]tc.ServerNullable,
+// GetOriginServersAndProfileCaches returns the origin servers, ProfileCaches, any warnings, and any error.

Review comment:
       Fixed

##########
File path: lib/go-atscfg/parentdotconfig.go
##########
@@ -1408,32 +1442,33 @@ func GetParentConfigProfileParams(
 		}
 		parentConfigServerCacheProfileParams[*cgServer.Profile] = profileCache
 	}
-	return parentConfigServerCacheProfileParams
+	return parentConfigServerCacheProfileParams, warnings
 }
 
-// GetDSOrigins takes a map[deliveryServiceID]DeliveryService, and returns a map[DeliveryServiceID]OriginURI.
-func GetDSOrigins(dses map[int]tc.DeliveryServiceNullableV30) (map[int]*OriginURI, error) {
-	dsOrigins := map[int]*OriginURI{}
+// GetDSOrigins takes a map[deliveryServiceID]DeliveryService, and returns a map[DeliveryServiceID]OriginURI, any warnings, and any error.

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r518152471



##########
File path: lib/go-atscfg/astatsdotconfig.go
##########
@@ -19,23 +19,41 @@ package atscfg
  * under the License.
  */
 
+import (
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
 const AstatsSeparator = "="
 const AstatsFileName = "astats.config"
 
 const ContentTypeAstatsDotConfig = ContentTypeTextASCII
 const LineCommentAstatsDotConfig = LineCommentHash
 
 func MakeAStatsDotConfig(
-	profileName string,
-	paramData map[string]string, // GetProfileParamData(tx, profile.ID, AstatsFileName)
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-) string {
-	hdr := GenericHeaderComment(profileName, toToolName, toURL)
+	server *tc.ServerNullable,

Review comment:
       ORT/atstccfg uses the objects returned by the latest TO client. I think it should probably always do that, and upgrade the objects from the previous client to them. 
   
   This matches https://github.com/apache/trafficcontrol/blob/f42709effc973528d3a3b844ae473189e3f2321a/traffic_ops/v3-client/server.go#L196
   
   If it needs changed, I think that needs changed before this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r518160312



##########
File path: lib/go-atscfg/astatsdotconfig.go
##########
@@ -19,23 +19,41 @@ package atscfg
  * under the License.
  */
 
+import (
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
 const AstatsSeparator = "="
 const AstatsFileName = "astats.config"
 
 const ContentTypeAstatsDotConfig = ContentTypeTextASCII
 const LineCommentAstatsDotConfig = LineCommentHash
 
 func MakeAStatsDotConfig(
-	profileName string,
-	paramData map[string]string, // GetProfileParamData(tx, profile.ID, AstatsFileName)
-	toToolName string, // tm.toolname global parameter (TODO: cache itself?)
-	toURL string, // tm.url global parameter (TODO: cache itself?)
-) string {
-	hdr := GenericHeaderComment(profileName, toToolName, toURL)
+	server *tc.ServerNullable,

Review comment:
       Well, that's a deprecated method of `github.com/apache/trafficcontrol/traffic_ops/client` - the unversioned client we're getting rid of in 6.0: _"Deprecated: GetServers will be removed in 6.0. Use GetServersWithHdr."_ - but it looks like `GetServersWithHdr` also returns `ServerNullable` - and that's what should change. I think there might be a breaking client change hidden there that needs to get fixed...
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247#discussion_r525515017



##########
File path: lib/go-atscfg/parentdotconfig.go
##########
@@ -1001,62 +1030,66 @@ func GetTopologyParents(
 			continue
 		}
 
-		if !HasRequiredCapabilities(serverCapabilities[*sv.ID], dsRequiredCapabilities[*ds.ID]) {
+		if !hasRequiredCapabilities(serverCapabilities[*sv.ID], dsRequiredCapabilities[*ds.ID]) {
 			continue
 		}
 		if *sv.Cachegroup == parentCG {
-			parentStr, err := serverParentStr(&sv.ServerNullable, sv.Params)
+			parentStr, err := serverParentStr(&sv.Server, sv.Params)
 			if err != nil {
-				return nil, nil, errors.New("getting server parent string: " + err.Error())
+				return nil, nil, warnings, errors.New("getting server parent string: " + err.Error())
 			}
 			if parentStr != "" { // will be empty if server is not_a_parent (possibly other reasons)
 				parentStrs = append(parentStrs, parentStr)
 			}
 		}
 		if *sv.Cachegroup == secondaryParentCG {
-			parentStr, err := serverParentStr(&sv.ServerNullable, sv.Params)
+			parentStr, err := serverParentStr(&sv.Server, sv.Params)
 			if err != nil {
-				return nil, nil, errors.New("getting server parent string: " + err.Error())
+				return nil, nil, warnings, errors.New("getting server parent string: " + err.Error())
 			}
 			secondaryParentStrs = append(secondaryParentStrs, parentStr)
 		}
 	}
 
-	return parentStrs, secondaryParentStrs, nil
+	return parentStrs, secondaryParentStrs, warnings, nil
 }
 
-func GetOriginURI(fqdn string) (*url.URL, error) {
+// GetOriginURI returns the URL, any warnings, and any error.

Review comment:
       metanit: GoDoc doesn't technically apply to unexported symbols.
   But yes, we prefer to follow it anyway for code readability. Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 merged pull request #5247: Change ORT/atstccfg to use standard TC objects

Posted by GitBox <gi...@apache.org>.
ocket8888 merged pull request #5247:
URL: https://github.com/apache/trafficcontrol/pull/5247


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org