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

[trafficcontrol] 01/03: Add TO Go transactions, helpers

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

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

commit 9de1e0986950d3384aca9e238d7c3c03edd0e6d1
Author: Robert Butts <ro...@apache.org>
AuthorDate: Fri Jun 1 17:45:25 2018 -0600

    Add TO Go transactions, helpers
---
 lib/go-tc/deliveryservices.go                      | 218 ++++++++++++-
 lib/go-tc/time.go                                  |   4 +
 .../go-tc}/tovalidate/rules.go                     |   0
 .../go-tc}/tovalidate/validate.go                  |   0
 lib/go-util/join.go                                |   9 +
 .../utils => lib/go-util}/ptr.go                   |  12 +-
 traffic_ops/traffic_ops_golang/api/change_log.go   |   6 +
 .../traffic_ops_golang/api/shared_handlers.go      |   2 +
 traffic_ops/traffic_ops_golang/asn/asns.go         |   3 +-
 .../traffic_ops_golang/cachegroup/cachegroups.go   |   3 +-
 traffic_ops/traffic_ops_golang/cdn/cdns.go         |   3 +-
 .../traffic_ops_golang/coordinate/coordinates.go   |   3 +-
 .../deliveryservice/deliveryservicesv12.go         | 356 ++++++---------------
 .../deliveryservice/deliveryservicesv13.go         | 302 +++++------------
 .../deliveryservice/request/comment/comments.go    |   3 +-
 .../deliveryservice/request/validate.go            |  19 +-
 .../deliveryservice/servers/servers.go             |  96 +++---
 .../traffic_ops_golang/deliveryservices_keys.go    | 186 +++++------
 .../deliveryservicesregexes.go                     |   5 -
 .../traffic_ops_golang/division/divisions.go       |   5 +-
 traffic_ops/traffic_ops_golang/origin/origins.go   |   2 +-
 .../traffic_ops_golang/parameter/parameters.go     |   4 +-
 .../physlocation/phys_locations.go                 |   3 +-
 traffic_ops/traffic_ops_golang/profile/profiles.go |   3 +-
 .../profileparameter/profile_parameters.go         |   4 +-
 traffic_ops/traffic_ops_golang/riaksvc/dsutil.go   |   8 +-
 traffic_ops/traffic_ops_golang/role/roles.go       |   3 +-
 traffic_ops/traffic_ops_golang/routes.go           |   8 +-
 traffic_ops/traffic_ops_golang/server/servers.go   |   2 +-
 traffic_ops/traffic_ops_golang/status/statuses.go  |   3 +-
 traffic_ops/traffic_ops_golang/tenant/tenancy.go   |  68 ++--
 traffic_ops/traffic_ops_golang/types/types.go      |   3 +-
 traffic_ops/traffic_ops_golang/urisigning.go       |  40 ++-
 traffic_ops/traffic_ops_golang/utils/errors.go     |  29 --
 .../asaskevich/govalidator/CONTRIBUTING.md         |   0
 .../github.com/asaskevich/govalidator/LICENSE      |   0
 .../github.com/asaskevich/govalidator/README.md    |   0
 .../github.com/asaskevich/govalidator/arrays.go    |   0
 .../asaskevich/govalidator/arrays_test.go          |   0
 .../github.com/asaskevich/govalidator/converter.go |   0
 .../asaskevich/govalidator/converter_test.go       |   0
 .../github.com/asaskevich/govalidator/error.go     |   0
 .../asaskevich/govalidator/error_test.go           |   0
 .../github.com/asaskevich/govalidator/numerics.go  |   0
 .../asaskevich/govalidator/numerics_test.go        |   0
 .../github.com/asaskevich/govalidator/patterns.go  |   0
 .../github.com/asaskevich/govalidator/types.go     |   0
 .../github.com/asaskevich/govalidator/utils.go     |   0
 .../asaskevich/govalidator/utils_benchmark_test.go |   0
 .../asaskevich/govalidator/utils_test.go           |   0
 .../github.com/asaskevich/govalidator/validator.go |   0
 .../asaskevich/govalidator/validator_test.go       |   0
 .../github.com/asaskevich/govalidator/wercker.yml  |   0
 .../github.com/go-ozzo/ozzo-validation/LICENSE     |   0
 .../github.com/go-ozzo/ozzo-validation/README.md   |   0
 .../github.com/go-ozzo/ozzo-validation/UPGRADE.md  |   0
 .../github.com/go-ozzo/ozzo-validation/date.go     |   0
 .../go-ozzo/ozzo-validation/date_test.go           |   0
 .../github.com/go-ozzo/ozzo-validation/error.go    |   0
 .../go-ozzo/ozzo-validation/error_test.go          |   0
 .../go-ozzo/ozzo-validation/example_test.go        |   0
 .../github.com/go-ozzo/ozzo-validation/in.go       |   0
 .../github.com/go-ozzo/ozzo-validation/in_test.go  |   0
 .../github.com/go-ozzo/ozzo-validation/is/rules.go |   0
 .../go-ozzo/ozzo-validation/is/rules_test.go       |   0
 .../github.com/go-ozzo/ozzo-validation/length.go   |   0
 .../go-ozzo/ozzo-validation/length_test.go         |   0
 .../github.com/go-ozzo/ozzo-validation/match.go    |   0
 .../go-ozzo/ozzo-validation/match_test.go          |   0
 .../github.com/go-ozzo/ozzo-validation/minmax.go   |   0
 .../go-ozzo/ozzo-validation/minmax_test.go         |   0
 .../github.com/go-ozzo/ozzo-validation/not_in.go   |   0
 .../go-ozzo/ozzo-validation/not_in_test.go         |   0
 .../github.com/go-ozzo/ozzo-validation/not_nil.go  |   0
 .../go-ozzo/ozzo-validation/not_nil_test.go        |   0
 .../github.com/go-ozzo/ozzo-validation/required.go |   0
 .../go-ozzo/ozzo-validation/required_test.go       |   0
 .../github.com/go-ozzo/ozzo-validation/string.go   |   0
 .../go-ozzo/ozzo-validation/string_test.go         |   0
 .../github.com/go-ozzo/ozzo-validation/struct.go   |   0
 .../go-ozzo/ozzo-validation/struct_test.go         |   0
 .../github.com/go-ozzo/ozzo-validation/util.go     |   0
 .../go-ozzo/ozzo-validation/util_test.go           |   0
 .../go-ozzo/ozzo-validation/validation.go          |   0
 .../go-ozzo/ozzo-validation/validation_test.go     |   0
 85 files changed, 691 insertions(+), 724 deletions(-)

diff --git a/lib/go-tc/deliveryservices.go b/lib/go-tc/deliveryservices.go
index f63d1a4..484a0b3 100644
--- a/lib/go-tc/deliveryservices.go
+++ b/lib/go-tc/deliveryservices.go
@@ -1,9 +1,19 @@
 package tc
 
 import (
+	"database/sql"
 	"database/sql/driver"
 	"encoding/json"
+	"errors"
 	"fmt"
+	"regexp"
+	"strings"
+
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/incubator-trafficcontrol/lib/go-util"
+
+	"github.com/asaskevich/govalidator"
+	"github.com/go-ozzo/ozzo-validation"
 )
 
 /*
@@ -23,7 +33,6 @@ import (
 
 const DefaultRoutingName = "cdn"
 
-//
 // GetDeliveryServiceResponse is deprecated use DeliveryServicesResponse...
 type GetDeliveryServiceResponse struct {
 	Response []DeliveryService `json:"response"`
@@ -207,6 +216,213 @@ type DeliveryServiceNullableV13 struct {
 	TRRequestHeaders  *string          `json:"trRequestHeaders,omitempty"`
 }
 
+// NewDeliveryServiceNullableV13FromV12 creates a new V13 DS from a V12 DS, filling new fields with appropriate defaults.
+func NewDeliveryServiceNullableV13FromV12(ds DeliveryServiceNullableV12) DeliveryServiceNullableV13 {
+	newDS := DeliveryServiceNullableV13{DeliveryServiceNullableV12: ds}
+	newDS.Sanitize()
+	return newDS
+}
+
+func (ds *DeliveryServiceNullableV12) Sanitize() {
+	if ds.GeoLimitCountries != nil {
+		*ds.GeoLimitCountries = strings.ToUpper(strings.Replace(*ds.GeoLimitCountries, " ", "", -1))
+	}
+	if ds.ProfileID != nil && *ds.ProfileID == -1 {
+		ds.ProfileID = nil
+	}
+	if ds.EdgeHeaderRewrite != nil && strings.TrimSpace(*ds.EdgeHeaderRewrite) == "" {
+		ds.EdgeHeaderRewrite = nil
+	}
+	if ds.MidHeaderRewrite != nil && strings.TrimSpace(*ds.MidHeaderRewrite) == "" {
+		ds.MidHeaderRewrite = nil
+	}
+	if ds.RoutingName == nil || *ds.RoutingName == "" {
+		ds.RoutingName = util.StrPtr(DefaultRoutingName)
+	}
+}
+
+func getTypeName(tx *sql.Tx, id int) (string, bool, error) {
+	name := ""
+	if err := tx.QueryRow(`SELECT name from type where id=$1`, id).Scan(&name); err != nil {
+		if err == sql.ErrNoRows {
+			return "", false, nil
+		}
+		return "", false, errors.New("querying type name: " + err.Error())
+	}
+	return name, true, nil
+}
+
+func requiredIfMatchesTypeName(patterns []string, typeName string) func(interface{}) error {
+	return func(value interface{}) error {
+		switch v := value.(type) {
+		case *int:
+			if v != nil {
+				return nil
+			}
+		case *bool:
+			if v != nil {
+				return nil
+			}
+		case *string:
+			if v != nil {
+				return nil
+			}
+		case *float64:
+			if v != nil {
+				return nil
+			}
+		default:
+			return fmt.Errorf("validation failure: unknown type %T", value)
+		}
+		pattern := strings.Join(patterns, "|")
+		err := error(nil)
+		match := false
+		if typeName != "" {
+			match, err = regexp.MatchString(pattern, typeName)
+			if match {
+				return fmt.Errorf("is required if type is '%s'", typeName)
+			}
+		}
+		return err
+	}
+}
+
+// util.JoinErrs(errs).Error()
+
+func (ds *DeliveryServiceNullableV12) validateTypeFields(tx *sql.Tx) error {
+	// Validate the TypeName related fields below
+	typeName := ""
+	err := error(nil)
+	DNSRegexType := "^DNS.*$"
+	HTTPRegexType := "^HTTP.*$"
+	SteeringRegexType := "^STEERING.*$"
+	if ds.TypeID == nil {
+		return errors.New("missing type")
+	}
+	typeName, ok, err := getTypeName(tx, *ds.TypeID)
+	if err != nil {
+		return errors.New("getting type name: " + err.Error())
+	}
+	if !ok {
+		return errors.New("type not found")
+	}
+	errs := validation.Errors{
+		"initialDispersion": validation.Validate(ds.InitialDispersion,
+			validation.By(requiredIfMatchesTypeName([]string{HTTPRegexType}, typeName))),
+		"ipv6RoutingEnabled": validation.Validate(ds.IPV6RoutingEnabled,
+			validation.By(requiredIfMatchesTypeName([]string{SteeringRegexType, DNSRegexType, HTTPRegexType}, typeName))),
+		"missLat": validation.Validate(ds.MissLat,
+			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
+		"missLong": validation.Validate(ds.MissLong,
+			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
+		"multiSiteOrigin": validation.Validate(ds.MultiSiteOrigin,
+			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
+		"orgServerFqdn": validation.Validate(ds.OrgServerFQDN,
+			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName)),
+			validation.NewStringRule(validateOrgServerFQDN, "must start with http:// or https:// and be followed by a valid hostname with an optional port (no trailing slash)")),
+		"protocol": validation.Validate(ds.Protocol,
+			validation.By(requiredIfMatchesTypeName([]string{SteeringRegexType, DNSRegexType, HTTPRegexType}, typeName))),
+		"qstringIgnore": validation.Validate(ds.QStringIgnore,
+			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
+		"rangeRequestHandling": validation.Validate(ds.RangeRequestHandling,
+			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
+	}
+	toErrs := tovalidate.ToErrors(errs)
+	if len(toErrs) > 0 {
+		return errors.New(util.JoinErrsStr(toErrs))
+	}
+	return nil
+}
+
+func validateOrgServerFQDN(orgServerFQDN string) bool {
+	_, fqdn, port, err := ParseOrgServerFQDN(orgServerFQDN)
+	if err != nil || !govalidator.IsHost(*fqdn) || (port != nil && !govalidator.IsPort(*port)) {
+		return false
+	}
+	return true
+}
+
+func ParseOrgServerFQDN(orgServerFQDN string) (*string, *string, *string, error) {
+	originRegex := regexp.MustCompile(`^(https?)://([^:]+)(:(\d+))?$`)
+	matches := originRegex.FindStringSubmatch(orgServerFQDN)
+	if len(matches) == 0 {
+		return nil, nil, nil, fmt.Errorf("unable to parse invalid orgServerFqdn: '%s'", orgServerFQDN)
+	}
+
+	protocol := strings.ToLower(matches[1])
+	FQDN := matches[2]
+
+	if len(protocol) == 0 || len(FQDN) == 0 {
+		return nil, nil, nil, fmt.Errorf("empty Origin protocol or FQDN parsed from '%s'", orgServerFQDN)
+	}
+
+	var port *string
+	if len(matches[4]) != 0 {
+		port = &matches[4]
+	}
+	return &protocol, &FQDN, port, nil
+}
+
+func (ds *DeliveryServiceNullableV12) Validate(tx *sql.Tx) error {
+	ds.Sanitize()
+	isDNSName := validation.NewStringRule(govalidator.IsDNSName, "must be a valid hostname")
+	noPeriods := validation.NewStringRule(tovalidate.NoPeriods, "cannot contain periods")
+	noSpaces := validation.NewStringRule(tovalidate.NoSpaces, "cannot contain spaces")
+	errs := validation.Errors{
+		"active":              validation.Validate(ds.Active, validation.NotNil),
+		"cdnId":               validation.Validate(ds.CDNID, validation.Required),
+		"displayName":         validation.Validate(ds.DisplayName, validation.Required, validation.Length(1, 48)),
+		"dscp":                validation.Validate(ds.DSCP, validation.NotNil, validation.Min(0)),
+		"geoLimit":            validation.Validate(ds.GeoLimit, validation.NotNil),
+		"geoProvider":         validation.Validate(ds.GeoProvider, validation.NotNil),
+		"logsEnabled":         validation.Validate(ds.LogsEnabled, validation.NotNil),
+		"regionalGeoBlocking": validation.Validate(ds.RegionalGeoBlocking, validation.NotNil),
+		"routingName":         validation.Validate(ds.RoutingName, isDNSName, noPeriods, validation.Length(1, 48)),
+		"typeId":              validation.Validate(ds.TypeID, validation.Required, validation.Min(1)),
+		"xmlId":               validation.Validate(ds.XMLID, noSpaces, noPeriods, validation.Length(1, 48)),
+	}
+	toErrs := tovalidate.ToErrors(errs)
+	if len(toErrs) > 0 {
+		return errors.New(util.JoinErrsStr(toErrs))
+	}
+	if err := ds.validateTypeFields(tx); err != nil {
+		return errors.New("type fields: " + err.Error())
+	}
+	return nil
+}
+
+func (ds *DeliveryServiceNullableV13) Sanitize() {
+	ds.DeliveryServiceNullableV12.Sanitize()
+	signedAlgorithm := "url_sig"
+	if ds.Signed && (ds.SigningAlgorithm == nil || *ds.SigningAlgorithm == "") {
+		ds.SigningAlgorithm = &signedAlgorithm
+	}
+	if !ds.Signed && ds.SigningAlgorithm != nil && *ds.SigningAlgorithm == signedAlgorithm {
+		ds.Signed = true
+	}
+	if ds.DeepCachingType == nil {
+		s := DeepCachingType("")
+		ds.DeepCachingType = &s
+	}
+	*ds.DeepCachingType = DeepCachingTypeFromString(string(*ds.DeepCachingType))
+}
+
+func (ds *DeliveryServiceNullableV13) Validate(tx *sql.Tx) error {
+	ds.Sanitize()
+	neverOrAlways := validation.NewStringRule(tovalidate.IsOneOfStringICase("NEVER", "ALWAYS"),
+		"must be one of 'NEVER' or 'ALWAYS'")
+	errs := tovalidate.ToErrors(validation.Errors{
+		"deepCachingType": validation.Validate(ds.DeepCachingType, neverOrAlways),
+	})
+	if err := ds.DeliveryServiceNullableV12.Validate(tx); err != nil {
+		errs = append(errs, err)
+	}
+	if len(errs) == 0 {
+		return nil
+	}
+	return errors.New(util.JoinErrsStr(errs)) // don't add context, so versions chain well
+}
+
 // Value implements the driver.Valuer interface
 // marshals struct to json to pass back as a json.RawMessage
 func (d *DeliveryServiceNullable) Value() (driver.Value, error) {
diff --git a/lib/go-tc/time.go b/lib/go-tc/time.go
index f24fe74..a45bbf7 100644
--- a/lib/go-tc/time.go
+++ b/lib/go-tc/time.go
@@ -67,6 +67,10 @@ func (t *Time) UnmarshalJSON(b []byte) (err error) {
 // TimeNoMod supported JSON marshalling, but suppresses JSON unmarshalling
 type TimeNoMod Time
 
+func NewTimeNoMod() *TimeNoMod {
+	return &TimeNoMod{Time: time.Now()}
+}
+
 // Scan implements the database/sql Scanner interface.
 func (t *TimeNoMod) Scan(value interface{}) error {
 	t.Time, t.Valid = value.(time.Time)
diff --git a/traffic_ops/traffic_ops_golang/tovalidate/rules.go b/lib/go-tc/tovalidate/rules.go
similarity index 100%
rename from traffic_ops/traffic_ops_golang/tovalidate/rules.go
rename to lib/go-tc/tovalidate/rules.go
diff --git a/traffic_ops/traffic_ops_golang/tovalidate/validate.go b/lib/go-tc/tovalidate/validate.go
similarity index 100%
rename from traffic_ops/traffic_ops_golang/tovalidate/validate.go
rename to lib/go-tc/tovalidate/validate.go
diff --git a/lib/go-util/join.go b/lib/go-util/join.go
index 933d67d..392d016 100644
--- a/lib/go-util/join.go
+++ b/lib/go-util/join.go
@@ -35,6 +35,15 @@ func JoinErrsStr(errs []error) string {
 	return joined.Error()
 }
 
+func ErrsToStrs(errs []error) []string {
+	errorStrs := []string{}
+	for _, errType := range errs {
+		et := errType.Error()
+		errorStrs = append(errorStrs, et)
+	}
+	return errorStrs
+}
+
 func JoinErrs(errs []error) error {
 	return JoinErrsSep(errs, "")
 }
diff --git a/traffic_ops/traffic_ops_golang/utils/ptr.go b/lib/go-util/ptr.go
similarity index 85%
rename from traffic_ops/traffic_ops_golang/utils/ptr.go
rename to lib/go-util/ptr.go
index c33c656..c672421 100644
--- a/traffic_ops/traffic_ops_golang/utils/ptr.go
+++ b/lib/go-util/ptr.go
@@ -1,4 +1,4 @@
-package utils
+package util
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -19,12 +19,6 @@ package utils
  * under the License.
  */
 
-import (
-	"time"
-
-	"github.com/apache/trafficcontrol/lib/go-tc"
-)
-
 func StrPtr(str string) *string {
 	return &str
 }
@@ -36,7 +30,3 @@ func IntPtr(i int) *int {
 func BoolPtr(b bool) *bool {
 	return &b
 }
-
-func NewTimeNoMod() *tc.TimeNoMod {
-	return &tc.TimeNoMod{Time: time.Now()}
-}
diff --git a/traffic_ops/traffic_ops_golang/api/change_log.go b/traffic_ops/traffic_ops_golang/api/change_log.go
index ce5e905..cc87a3a 100644
--- a/traffic_ops/traffic_ops_golang/api/change_log.go
+++ b/traffic_ops/traffic_ops_golang/api/change_log.go
@@ -90,3 +90,9 @@ func CreateChangeLogRaw(level string, msg string, user auth.CurrentUser, db *sql
 		log.Errorln("Inserting change log level '" + level + "' message '" + msg + "' user '" + user.UserName + "': " + err.Error())
 	}
 }
+
+func CreateChangeLogRawTx(level string, msg string, user *auth.CurrentUser, tx *sql.Tx) {
+	if _, err := tx.Exec(`INSERT INTO log (level, message, tm_user) VALUES ($1, $2, $3)`, level, msg, user.ID); err != nil {
+		log.Errorln("Inserting change log level '" + level + "' message '" + msg + "' user '" + user.UserName + "': " + err.Error())
+	}
+}
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index bf6ef4c..b4d583a 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -37,6 +37,8 @@ import (
 )
 
 const PathParamsKey = "pathParams"
+const DBKey = "db"
+const ConfigKey = "cfg"
 
 type KeyFieldInfo struct {
 	Field string
diff --git a/traffic_ops/traffic_ops_golang/asn/asns.go b/traffic_ops/traffic_ops_golang/asn/asns.go
index f01afac..d832898 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns.go
@@ -28,10 +28,11 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
index 21d1e1a..e359edd 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -27,11 +27,12 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns.go b/traffic_ops/traffic_ops_golang/cdn/cdns.go
index 6b36591..748bb5e 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns.go
@@ -27,11 +27,12 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	"github.com/asaskevich/govalidator"
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
index a926ebf..4b852d7 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
@@ -27,11 +27,12 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index eeeead5..cb737f3 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -25,20 +25,15 @@ import (
 	"errors"
 	"fmt"
 	"net/http"
-	"regexp"
-	"strings"
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/config"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/utils"
 
-	"github.com/asaskevich/govalidator"
-	"github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 )
 
@@ -48,6 +43,14 @@ type TODeliveryServiceV12 struct {
 	DB  *sqlx.DB
 }
 
+func (ds TODeliveryServiceV12) MarshalJSON() ([]byte, error) {
+	return json.Marshal(ds.DeliveryServiceNullableV12)
+}
+
+func (ds *TODeliveryServiceV12) UnmarshalJSON(data []byte) error {
+	return json.Unmarshal(data, ds.DeliveryServiceNullableV12)
+}
+
 func GetRefTypeV12(cfg config.Config, db *sqlx.DB) *TODeliveryServiceV12 {
 	return &TODeliveryServiceV12{Cfg: cfg, DB: db}
 }
@@ -79,41 +82,11 @@ func (ds *TODeliveryServiceV12) GetType() string {
 	return "ds"
 }
 
-func ValidateV12(db *sqlx.DB, ds *tc.DeliveryServiceNullableV12) []error {
-	if ds == nil {
-		return []error{}
-	}
-	tods := TODeliveryServiceV12{DB: db} // TODO pass config?
-	return tods.Validate(db)
-}
-
-func (ds *TODeliveryServiceV12) Sanitize(db *sqlx.DB) {
-	sanitizeV12(&ds.DeliveryServiceNullableV12)
-}
-
-func sanitizeV12(ds *tc.DeliveryServiceNullableV12) {
-	if ds.GeoLimitCountries != nil {
-		*ds.GeoLimitCountries = strings.ToUpper(strings.Replace(*ds.GeoLimitCountries, " ", "", -1))
-	}
-	if ds.ProfileID != nil && *ds.ProfileID == -1 {
-		ds.ProfileID = nil
-	}
-	if ds.EdgeHeaderRewrite != nil && strings.TrimSpace(*ds.EdgeHeaderRewrite) == "" {
-		ds.EdgeHeaderRewrite = nil
-	}
-	if ds.MidHeaderRewrite != nil && strings.TrimSpace(*ds.MidHeaderRewrite) == "" {
-		ds.MidHeaderRewrite = nil
-	}
-	if ds.RoutingName == nil || *ds.RoutingName == "" {
-		ds.RoutingName = utils.StrPtr(tc.DefaultRoutingName)
-	}
-}
-
 // getDSTenantIDByID returns the tenant ID, whether the delivery service exists, and any error.
 // Note the id may be nil, even if true is returned, if the delivery service exists but its tenant_id field is null.
-func getDSTenantIDByID(db *sql.DB, id int) (*int, bool, error) {
+func getDSTenantIDByID(tx *sql.Tx, id int) (*int, bool, error) {
 	tenantID := (*int)(nil)
-	if err := db.QueryRow(`SELECT tenant_id FROM deliveryservice where id = $1`, id).Scan(&tenantID); err != nil {
+	if err := tx.QueryRow(`SELECT tenant_id FROM deliveryservice where id = $1`, id).Scan(&tenantID); err != nil {
 		if err == sql.ErrNoRows {
 			return nil, false, nil
 		}
@@ -124,9 +97,9 @@ func getDSTenantIDByID(db *sql.DB, id int) (*int, bool, error) {
 
 // getDSTenantIDByName returns the tenant ID, whether the delivery service exists, and any error.
 // Note the id may be nil, even if true is returned, if the delivery service exists but its tenant_id field is null.
-func getDSTenantIDByName(db *sql.DB, name string) (*int, bool, error) {
+func getDSTenantIDByName(tx *sql.Tx, name string) (*int, bool, error) {
 	tenantID := (*int)(nil)
-	if err := db.QueryRow(`SELECT tenant_id FROM deliveryservice where xml_id = $1`, name).Scan(&tenantID); err != nil {
+	if err := tx.QueryRow(`SELECT tenant_id FROM deliveryservice where xml_id = $1`, name).Scan(&tenantID); err != nil {
 		if err == sql.ErrNoRows {
 			return nil, false, nil
 		}
@@ -136,12 +109,12 @@ func getDSTenantIDByName(db *sql.DB, name string) (*int, bool, error) {
 }
 
 // GetXMLID loads the DeliveryService's xml_id from the database, from the ID. Returns whether the delivery service was found, and any error.
-func (ds *TODeliveryServiceV12) GetXMLID(db *sqlx.DB) (string, bool, error) {
+func (ds *TODeliveryServiceV12) GetXMLID(tx *sql.Tx) (string, bool, error) {
 	if ds.ID == nil {
 		return "", false, errors.New("missing ID")
 	}
 	xmlID := ""
-	if err := db.QueryRow(`SELECT xml_id FROM deliveryservice where id = $1`, ds.ID).Scan(&xmlID); err != nil {
+	if err := tx.QueryRow(`SELECT xml_id FROM deliveryservice where id = $1`, ds.ID).Scan(&xmlID); err != nil {
 		if err == sql.ErrNoRows {
 			return "", false, nil
 		}
@@ -151,30 +124,38 @@ func (ds *TODeliveryServiceV12) GetXMLID(db *sqlx.DB) (string, bool, error) {
 }
 
 // IsTenantAuthorized checks that the user is authorized for both the delivery service's existing tenant, and the new tenant they're changing it to (if different).
-func (ds *TODeliveryServiceV12) IsTenantAuthorized(user auth.CurrentUser, db *sqlx.DB) (bool, error) {
-	return isTenantAuthorized(user, db, &ds.DeliveryServiceNullableV12)
+func (ds *TODeliveryServiceV12) IsTenantAuthorized(user *auth.CurrentUser, db *sqlx.DB) (bool, error) {
+	tx, err := db.DB.Begin() // must be last, MUST not return an error if this suceeds, without closing the tx
+	if err != nil {
+		return false, errors.New("beginning transaction: " + err.Error())
+	}
+	defer dbhelpers.FinishTx(tx, util.BoolPtr(true))
+	return isTenantAuthorized(user, tx, &ds.DeliveryServiceNullableV12)
 }
 
-func isTenantAuthorized(user auth.CurrentUser, db *sqlx.DB, ds *tc.DeliveryServiceNullableV12) (bool, error) {
+// getTenantID returns the tenant Id of the given delivery service. Note it may return a nil id and nil error, if the tenant ID in the database is nil.
+func getTenantID(tx *sql.Tx, ds *tc.DeliveryServiceNullableV12) (*int, error) {
 	if ds.ID == nil && ds.XMLID == nil {
-		return false, errors.New("delivery service has no ID or XMLID")
+		return nil, errors.New("delivery service has no ID or XMLID")
 	}
-
-	existingID, err := (*int)(nil), error(nil)
 	if ds.ID != nil {
-		existingID, _, err = getDSTenantIDByID(db.DB, *ds.ID) // ignore exists return - if the DS is new, we only need to check the user input tenant
-	} else {
-		existingID, _, err = getDSTenantIDByName(db.DB, *ds.XMLID) // ignore exists return - if the DS is new, we only need to check the user input tenant
+		existingID, _, err := getDSTenantIDByID(tx, *ds.ID) // ignore exists return - if the DS is new, we only need to check the user input tenant
+		return existingID, err
 	}
+	existingID, _, err := getDSTenantIDByName(tx, *ds.XMLID) // ignore exists return - if the DS is new, we only need to check the user input tenant
+	return existingID, err
+}
+
+func isTenantAuthorized(user *auth.CurrentUser, tx *sql.Tx, ds *tc.DeliveryServiceNullableV12) (bool, error) {
+	existingID, err := getTenantID(tx, ds)
 	if err != nil {
 		return false, errors.New("getting tenant ID: " + err.Error())
 	}
-
 	if ds.TenantID == nil {
 		ds.TenantID = existingID
 	}
 	if existingID != nil && existingID != ds.TenantID {
-		userAuthorizedForExistingDSTenant, err := tenant.IsResourceAuthorizedToUser(*existingID, user, db)
+		userAuthorizedForExistingDSTenant, err := tenant.IsResourceAuthorizedToUserTx(*existingID, user, tx)
 		if err != nil {
 			return false, errors.New("checking authorization for existing DS ID: " + err.Error())
 		}
@@ -183,7 +164,7 @@ func isTenantAuthorized(user auth.CurrentUser, db *sqlx.DB, ds *tc.DeliveryServi
 		}
 	}
 	if ds.TenantID != nil {
-		userAuthorizedForNewDSTenant, err := tenant.IsResourceAuthorizedToUser(*ds.TenantID, user, db)
+		userAuthorizedForNewDSTenant, err := tenant.IsResourceAuthorizedToUserTx(*ds.TenantID, user, tx)
 		if err != nil {
 			return false, errors.New("checking authorization for new DS ID: " + err.Error())
 		}
@@ -195,181 +176,41 @@ func isTenantAuthorized(user auth.CurrentUser, db *sqlx.DB, ds *tc.DeliveryServi
 }
 
 func (ds *TODeliveryServiceV12) Validate(db *sqlx.DB) []error {
-	return validateV12(db, &ds.DeliveryServiceNullableV12)
-}
-
-func validateV12(db *sqlx.DB, ds *tc.DeliveryServiceNullableV12) []error {
-	sanitizeV12(ds)
-	// Custom Examples:
-	// Just add isCIDR as a parameter to Validate()
-	// isCIDR := validation.NewStringRule(govalidator.IsCIDR, "must be a valid CIDR address")
-	isDNSName := validation.NewStringRule(govalidator.IsDNSName, "must be a valid hostname")
-	noPeriods := validation.NewStringRule(tovalidate.NoPeriods, "cannot contain periods")
-	noSpaces := validation.NewStringRule(tovalidate.NoSpaces, "cannot contain spaces")
-
-	// Validate that the required fields are sent first to prevent panics below
-	errs := validation.Errors{
-		"active":              validation.Validate(ds.Active, validation.NotNil),
-		"cdnId":               validation.Validate(ds.CDNID, validation.Required),
-		"displayName":         validation.Validate(ds.DisplayName, validation.Required, validation.Length(1, 48)),
-		"dscp":                validation.Validate(ds.DSCP, validation.NotNil, validation.Min(0)),
-		"geoLimit":            validation.Validate(ds.GeoLimit, validation.NotNil),
-		"geoProvider":         validation.Validate(ds.GeoProvider, validation.NotNil),
-		"logsEnabled":         validation.Validate(ds.LogsEnabled, validation.NotNil),
-		"regionalGeoBlocking": validation.Validate(ds.RegionalGeoBlocking, validation.NotNil),
-		"routingName":         validation.Validate(ds.RoutingName, isDNSName, noPeriods, validation.Length(1, 48)),
-		"typeId":              validation.Validate(ds.TypeID, validation.Required, validation.Min(1)),
-		"xmlId":               validation.Validate(ds.XMLID, noSpaces, noPeriods, validation.Length(1, 48)),
-	}
-	toErrs := tovalidate.ToErrors(errs)
-	if fieldErrs := validateTypeFields(db, ds); len(fieldErrs) > 0 {
-		toErrs = append(toErrs, fieldErrs...)
-	}
-	if len(toErrs) > 0 {
-		return toErrs
-	}
-	return nil
-}
-
-func validateTypeFields(db *sqlx.DB, ds *tc.DeliveryServiceNullableV12) []error {
-	// Validate the TypeName related fields below
-	var typeName string
-	var err error
-	DNSRegexType := "^DNS.*$"
-	HTTPRegexType := "^HTTP.*$"
-	SteeringRegexType := "^STEERING.*$"
-
-	if ds.TypeID == nil {
-		return []error{errors.New("missing typeID")}
-	}
-
-	typeName, ok, err := getTypeName(db, *ds.TypeID)
+	tx, err := db.DB.Begin()
 	if err != nil {
-		return []error{err}
-	}
-	if !ok {
-		return []error{errors.New("type not found")}
-	}
-
-	errs := validation.Errors{
-		"initialDispersion": validation.Validate(ds.InitialDispersion,
-			validation.By(requiredIfMatchesTypeName([]string{HTTPRegexType}, typeName))),
-		"ipv6RoutingEnabled": validation.Validate(ds.IPV6RoutingEnabled,
-			validation.By(requiredIfMatchesTypeName([]string{SteeringRegexType, DNSRegexType, HTTPRegexType}, typeName))),
-		"missLat": validation.Validate(ds.MissLat,
-			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
-		"missLong": validation.Validate(ds.MissLong,
-			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
-		"multiSiteOrigin": validation.Validate(ds.MultiSiteOrigin,
-			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
-		"orgServerFqdn": validation.Validate(ds.OrgServerFQDN,
-			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName)),
-			validation.NewStringRule(validateOrgServerFQDN, "must start with http:// or https:// and be followed by a valid hostname with an optional port (no trailing slash)")),
-		"protocol": validation.Validate(ds.Protocol,
-			validation.By(requiredIfMatchesTypeName([]string{SteeringRegexType, DNSRegexType, HTTPRegexType}, typeName))),
-		"qstringIgnore": validation.Validate(ds.QStringIgnore,
-			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
-		"rangeRequestHandling": validation.Validate(ds.RangeRequestHandling,
-			validation.By(requiredIfMatchesTypeName([]string{DNSRegexType, HTTPRegexType}, typeName))),
-	}
-	toErrs := tovalidate.ToErrors(errs)
-	if len(toErrs) > 0 {
-		return toErrs
-	}
-	return nil
-}
-
-func validateOrgServerFQDN(orgServerFQDN string) bool {
-	_, fqdn, port, err := parseOrgServerFQDN(orgServerFQDN)
-	if err != nil || !govalidator.IsHost(*fqdn) || (port != nil && !govalidator.IsPort(*port)) {
-		return false
+		return []error{errors.New("beginning transaction: " + err.Error())}
 	}
-	return true
+	defer dbhelpers.FinishTx(tx, util.BoolPtr(true))
+	return []error{ds.DeliveryServiceNullableV12.Validate(tx)}
 }
 
-func requiredIfMatchesTypeName(patterns []string, typeName string) func(interface{}) error {
-	return func(value interface{}) error {
-		switch v := value.(type) {
-		case *int:
-			if v != nil {
-				return nil
-			}
-		case *bool:
-			if v != nil {
-				return nil
-			}
-		case *string:
-			if v != nil {
-				return nil
-			}
-		case *float64:
-			if v != nil {
-				return nil
-			}
-		default:
-			return fmt.Errorf("validation failure: unknown type %T", value)
-		}
-		pattern := strings.Join(patterns, "|")
-		var err error
-		var match bool
-		if typeName != "" {
-			match, err = regexp.MatchString(pattern, typeName)
-			if match {
-				return fmt.Errorf("is required if type is '%s'", typeName)
-			}
-		}
-		return err
-	}
-}
-
-func getTypeName(db *sqlx.DB, typeID int) (string, bool, error) {
-	name := ""
-	if err := db.QueryRow(`SELECT name from type where id=$1`, typeID).Scan(&name); err != nil {
-		if err == sql.ErrNoRows {
-			return "", false, nil
-		}
-		return "", false, errors.New("querying type name: " + err.Error())
-	}
-	return name, true, nil
-}
-
-func CreateV12(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
-	return func(w http.ResponseWriter, r *http.Request) {
-		defer r.Body.Close()
-		user, err := auth.GetCurrentUser(r.Context())
-		if err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting user: "+err.Error()))
-			return
-		}
-
-		ds := tc.DeliveryServiceNullableV12{}
-		if err := json.NewDecoder(r.Body).Decode(&ds); err != nil {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("malformed JSON: "+err.Error()), nil)
-			return
-		}
-
-		if errs := validateV12(db, &ds); len(errs) > 0 {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+util.JoinErrs(errs).Error()), nil)
-			return
-		}
-
-		dsv13 := tc.DeliveryServiceNullableV13{DeliveryServiceNullableV12: tc.DeliveryServiceNullableV12(ds)}
-
-		if authorized, err := isTenantAuthorized(*user, db, &ds); err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
-			return
-		} else if !authorized {
-			api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
-			return
-		}
-
-		dsv13, errCode, userErr, sysErr := create(db.DB, cfg, user, dsv13)
-		if userErr != nil || sysErr != nil {
-			api.HandleErr(w, r, errCode, userErr, sysErr)
-			return
-		}
-		api.WriteResp(w, r, []tc.DeliveryServiceNullableV12{dsv13.DeliveryServiceNullableV12})
-	}
+func CreateV12(w http.ResponseWriter, r *http.Request) {
+	inf, userErr, sysErr, errCode := api.NewInfo(r, nil, nil)
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+	ds := tc.DeliveryServiceNullableV12{}
+	if err := api.Parse(r.Body, inf.Tx.Tx, &ds); err != nil {
+		api.HandleErr(w, r, http.StatusBadRequest, errors.New("decoding: "+err.Error()), nil)
+		return
+	}
+	dsv13 := tc.NewDeliveryServiceNullableV13FromV12(ds)
+	if authorized, err := isTenantAuthorized(inf.User, inf.Tx.Tx, &ds); err != nil {
+		api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
+		return
+	} else if !authorized {
+		api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
+		return
+	}
+	dsv13, errCode, userErr, sysErr = create(inf.Tx.Tx, inf.Config, inf.User, dsv13)
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	*inf.CommitTx = true
+	api.WriteResp(w, r, []tc.DeliveryServiceNullableV12{dsv13.DeliveryServiceNullableV12})
 }
 
 func (ds *TODeliveryServiceV12) Read(db *sqlx.DB, params map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
@@ -403,41 +244,32 @@ func (ds *TODeliveryServiceV12) Delete(db *sqlx.DB, user auth.CurrentUser) (erro
 	return err, errType
 }
 
-func UpdateV12(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
-	return func(w http.ResponseWriter, r *http.Request) {
-		defer r.Body.Close()
-		user, err := auth.GetCurrentUser(r.Context())
-		if err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting user: "+err.Error()))
-			return
-		}
-
-		ds := tc.DeliveryServiceNullableV12{}
-		if err := json.NewDecoder(r.Body).Decode(&ds); err != nil {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("malformed JSON: "+err.Error()), nil)
-			return
-		}
-
-		if errs := validateV12(db, &ds); len(errs) > 0 {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+util.JoinErrs(errs).Error()), nil)
-			return
-		}
-
-		dsv13 := tc.DeliveryServiceNullableV13{DeliveryServiceNullableV12: tc.DeliveryServiceNullableV12(ds)}
-
-		if authorized, err := isTenantAuthorized(*user, db, &ds); err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
-			return
-		} else if !authorized {
-			api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
-			return
-		}
-
-		dsv13, errCode, userErr, sysErr := update(db.DB, cfg, *user, &dsv13)
-		if userErr != nil || sysErr != nil {
-			api.HandleErr(w, r, errCode, userErr, sysErr)
-			return
-		}
-		api.WriteResp(w, r, []tc.DeliveryServiceNullableV12{dsv13.DeliveryServiceNullableV12})
-	}
+func UpdateV12(w http.ResponseWriter, r *http.Request) {
+	inf, userErr, sysErr, errCode := api.NewInfo(r, []string{"id"}, []string{"id"})
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+
+	ds := tc.DeliveryServiceNullableV12{}
+	ds.ID = util.IntPtr(inf.IntParams["id"])
+	if err := api.Parse(r.Body, inf.Tx.Tx, &ds); err != nil {
+		api.HandleErr(w, r, http.StatusBadRequest, errors.New("decoding: "+err.Error()), nil)
+		return
+	}
+	dsv13 := tc.NewDeliveryServiceNullableV13FromV12(ds)
+	if authorized, err := isTenantAuthorized(inf.User, inf.Tx.Tx, &ds); err != nil {
+		api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
+		return
+	} else if !authorized {
+		api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
+		return
+	}
+	dsv13, errCode, userErr, sysErr = update(inf.Tx.Tx, inf.Config, inf.User, &dsv13)
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	api.WriteResp(w, r, []tc.DeliveryServiceNullableV12{dsv13.DeliveryServiceNullableV12})
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index 543d521..be448f7 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -25,7 +25,6 @@ import (
 	"errors"
 	"fmt"
 	"net/http"
-	"regexp"
 	"strconv"
 	"strings"
 
@@ -38,9 +37,7 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/riaksvc"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
 
-	"github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
@@ -89,109 +86,53 @@ func (ds *TODeliveryServiceV13) GetType() string {
 	return ds.V12().GetType()
 }
 
-func ValidateV13(db *sqlx.DB, ds *tc.DeliveryServiceNullableV13) []error {
-	if ds == nil {
-		return []error{}
-	}
-	tods := TODeliveryServiceV13{DeliveryServiceNullableV13: *ds, DB: db} // TODO set Cfg?
-	return tods.Validate(db)
-}
-
-func (ds *TODeliveryServiceV13) Sanitize(db *sqlx.DB) { sanitizeV13(&ds.DeliveryServiceNullableV13) }
-
-func sanitizeV13(ds *tc.DeliveryServiceNullableV13) {
-	sanitizeV12(&ds.DeliveryServiceNullableV12)
-	signedAlgorithm := "url_sig"
-	if ds.Signed && (ds.SigningAlgorithm == nil || *ds.SigningAlgorithm == "") {
-		ds.SigningAlgorithm = &signedAlgorithm
-	}
-	if !ds.Signed && ds.SigningAlgorithm != nil && *ds.SigningAlgorithm == signedAlgorithm {
-		ds.Signed = true
-	}
-	if ds.DeepCachingType == nil {
-		s := tc.DeepCachingType("")
-		ds.DeepCachingType = &s
-	}
-	*ds.DeepCachingType = tc.DeepCachingTypeFromString(string(*ds.DeepCachingType))
-}
-
 func (ds *TODeliveryServiceV13) Validate(db *sqlx.DB) []error {
-	return validateV13(db, &ds.DeliveryServiceNullableV13)
+	tx, err := db.DB.Begin()
+	if err != nil {
+		return []error{errors.New("beginning transaction: " + err.Error())}
+	}
+	defer dbhelpers.FinishTx(tx, util.BoolPtr(true))
+	return []error{ds.DeliveryServiceNullableV13.Validate(tx)}
 }
 
-func validateV13(db *sqlx.DB, ds *tc.DeliveryServiceNullableV13) []error {
-	sanitizeV13(ds)
-	neverOrAlways := validation.NewStringRule(tovalidate.IsOneOfStringICase("NEVER", "ALWAYS"),
-		"must be one of 'NEVER' or 'ALWAYS'")
-	errs := tovalidate.ToErrors(validation.Errors{
-		"deepCachingType": validation.Validate(ds.DeepCachingType, neverOrAlways),
-	})
-	oldErrs := validateV12(db, &ds.DeliveryServiceNullableV12)
-	return append(errs, oldErrs...)
-}
-
-// Create implements the Creator interface.
-//all implementations of Creator should use transactions and return the proper errorType
-//ParsePQUniqueConstraintError is used to determine if a ds with conflicting values exists
-//if so, it will return an errorType of DataConflict and the type should be appended to the
-//generic error message returned
-//The insert sql returns the id and lastUpdated values of the newly inserted ds and have
-//to be added to the struct
-// func (ds *TODeliveryServiceV13) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) { //
-//
-// 	TODO allow users to post names (type, cdn, etc) and get the IDs from the names. This isn't trivial to do in a single query, without dynamically building the entire insert query, and ideally inserting would be one query. But it'd be much more convenient for users. Alternatively, remove IDs from the database entirely and use real candidate keys.
-func CreateV13(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
-	return func(w http.ResponseWriter, r *http.Request) {
-		defer r.Body.Close()
-		user, err := auth.GetCurrentUser(r.Context())
-		if err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting user: "+err.Error()))
-			return
-		}
-
-		ds := tc.DeliveryServiceNullableV13{}
-		if err := json.NewDecoder(r.Body).Decode(&ds); err != nil {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("malformed JSON: "+err.Error()), nil)
-			return
-		}
-
-		if errs := validateV13(db, &ds); len(errs) > 0 {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+util.JoinErrs(errs).Error()), nil)
-			return
-		}
-
-		if authorized, err := isTenantAuthorized(*user, db, &ds.DeliveryServiceNullableV12); err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
-			return
-		} else if !authorized {
-			api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
-			return
-		}
-
-		ds, errCode, userErr, sysErr := create(db.DB, cfg, user, ds)
-		if userErr != nil || sysErr != nil {
-			api.HandleErr(w, r, errCode, userErr, sysErr)
-			return
-		}
-		api.WriteResp(w, r, []tc.DeliveryServiceNullableV13{ds})
-	}
+// CreateV13 implements the http.HandlerFunc type, and handles API 1.3 POST requests.
+func CreateV13(w http.ResponseWriter, r *http.Request) {
+	log.Errorln("CreateV13 callingaqg")
+	inf, userErr, sysErr, errCode := api.NewInfo(r, nil, nil)
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+	ds := tc.DeliveryServiceNullableV13{}
+	if err := api.Parse(r.Body, inf.Tx.Tx, &ds); err != nil {
+		api.HandleErr(w, r, http.StatusBadRequest, errors.New("decoding: "+err.Error()), nil)
+		return
+	}
+	if authorized, err := isTenantAuthorized(inf.User, inf.Tx.Tx, &ds.DeliveryServiceNullableV12); err != nil {
+		api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
+		return
+	} else if !authorized {
+		api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
+		return
+	}
+	ds, errCode, userErr, sysErr = create(inf.Tx.Tx, inf.Config, inf.User, ds)
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	*inf.CommitTx = true
+	api.WriteResp(w, r, []tc.DeliveryServiceNullableV13{ds})
 }
 
 // create creates the given ds in the database, and returns the DS with its id and other fields created on insert set. On error, the HTTP status cdoe, user error, and system error are returned. The status code SHOULD NOT be used, if both errors are nil.
-func create(db *sql.DB, cfg config.Config, user *auth.CurrentUser, ds tc.DeliveryServiceNullableV13) (tc.DeliveryServiceNullableV13, int, error, error) {
+func create(tx *sql.Tx, cfg *config.Config, user *auth.CurrentUser, ds tc.DeliveryServiceNullableV13) (tc.DeliveryServiceNullableV13, int, error, error) {
 	// TODO change DeepCachingType to implement sql.Valuer and sql.Scanner, so sqlx struct scan can be used.
 	deepCachingType := tc.DeepCachingType("").String()
 	if ds.DeepCachingType != nil {
 		deepCachingType = ds.DeepCachingType.String() // necessary, because DeepCachingType's default needs to insert the string, not "", and Query doesn't call .String().
 	}
 
-	tx, err := db.Begin()
-	if err != nil {
-		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("could not begin transaction: " + err.Error())
-	}
-	commitTx := false
-	defer dbhelpers.FinishTx(tx, &commitTx)
-
 	resultRows, err := tx.Query(insertQuery(), &ds.Active, &ds.AnonymousBlockingEnabled, &ds.CacheURL, &ds.CCRDNSTTL, &ds.CDNID, &ds.CheckPath, &deepCachingType, &ds.DisplayName, &ds.DNSBypassCNAME, &ds.DNSBypassIP, &ds.DNSBypassIP6, &ds.DNSBypassTTL, &ds.DSCP, &ds.EdgeHeaderRewrite, &ds.GeoLimitRedirectURL, &ds.GeoLimit, &ds.GeoLimitCountries, &ds.GeoProvider, &ds.GlobalMaxMBPS, &ds.GlobalMaxTPS, &ds.FQPacingRate, &ds.HTTPBypassFQDN, &ds.InfoURL, &ds.InitialDispersion, &ds.IPV6RoutingEnabl [...]
 
 	if err != nil {
@@ -263,17 +204,16 @@ func create(db *sql.DB, cfg config.Config, user *auth.CurrentUser, ds tc.Deliver
 	if err := ensureCacheURLParams(tx, *ds.ID, *ds.XMLID, ds.CacheURL); err != nil {
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("creating cache url parameters: " + err.Error())
 	}
-	if err := createDNSSecKeys(tx, cfg, *ds.ID, *ds.XMLID, cdnName, cdnDomain, dnssecEnabled, ds.ExampleURLs); err != nil {
+	if err := createDNSSecKeys(tx, *cfg, *ds.ID, *ds.XMLID, cdnName, cdnDomain, dnssecEnabled, ds.ExampleURLs); err != nil {
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("creating DNSSEC keys: " + err.Error())
 	}
 
-	if err := createPrimaryOrigin(db, tx, user, ds); err != nil {
+	if err := createPrimaryOrigin(tx, user, ds); err != nil {
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("creating delivery service: " + err.Error())
 	}
 
 	ds.LastUpdated = &lastUpdated
-	commitTx = true
-	api.CreateChangeLogRaw(api.ApiChange, "Created ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), *user, db)
+	api.CreateChangeLogRawTx(api.ApiChange, "Created ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), user, tx)
 	return ds, http.StatusOK, nil, nil
 }
 
@@ -307,33 +247,12 @@ func createDefaultRegex(tx *sql.Tx, dsID int, xmlID string) error {
 	return nil
 }
 
-func parseOrgServerFQDN(orgServerFQDN string) (*string, *string, *string, error) {
-	originRegex := regexp.MustCompile(`^(https?)://([^:]+)(:(\d+))?$`)
-	matches := originRegex.FindStringSubmatch(orgServerFQDN)
-	if len(matches) == 0 {
-		return nil, nil, nil, fmt.Errorf("unable to parse invalid orgServerFqdn: '%s'", orgServerFQDN)
-	}
-
-	protocol := strings.ToLower(matches[1])
-	FQDN := matches[2]
-
-	if len(protocol) == 0 || len(FQDN) == 0 {
-		return nil, nil, nil, fmt.Errorf("empty Origin protocol or FQDN parsed from '%s'", orgServerFQDN)
-	}
-
-	var port *string
-	if len(matches[4]) != 0 {
-		port = &matches[4]
-	}
-	return &protocol, &FQDN, port, nil
-}
-
-func createPrimaryOrigin(db *sql.DB, tx *sql.Tx, user *auth.CurrentUser, ds tc.DeliveryServiceNullableV13) error {
+func createPrimaryOrigin(tx *sql.Tx, user *auth.CurrentUser, ds tc.DeliveryServiceNullableV13) error {
 	if ds.OrgServerFQDN == nil {
 		return nil
 	}
 
-	protocol, fqdn, port, err := parseOrgServerFQDN(*ds.OrgServerFQDN)
+	protocol, fqdn, port, err := tc.ParseOrgServerFQDN(*ds.OrgServerFQDN)
 	if err != nil {
 		return fmt.Errorf("creating primary origin: %v", err)
 	}
@@ -344,12 +263,12 @@ func createPrimaryOrigin(db *sql.DB, tx *sql.Tx, user *auth.CurrentUser, ds tc.D
 		return fmt.Errorf("insert origin from '%s': %s", *ds.OrgServerFQDN, err.Error())
 	}
 
-	api.CreateChangeLogRaw(api.ApiChange, "Created primary origin id: "+strconv.Itoa(originID)+" for delivery service: "+*ds.XMLID, *user, db)
+	api.CreateChangeLogRawTx(api.ApiChange, "Created primary origin id: "+strconv.Itoa(originID)+" for delivery service: "+*ds.XMLID, user, tx)
 
 	return nil
 }
 
-func updatePrimaryOrigin(db *sql.DB, tx *sql.Tx, user *auth.CurrentUser, ds tc.DeliveryServiceNullableV13) error {
+func updatePrimaryOrigin(tx *sql.Tx, user *auth.CurrentUser, ds tc.DeliveryServiceNullableV13) error {
 	count := 0
 	q := `SELECT count(*) FROM origin WHERE deliveryservice = $1 AND is_primary`
 	if err := tx.QueryRow(q, *ds.ID).Scan(&count); err != nil {
@@ -363,17 +282,17 @@ func updatePrimaryOrigin(db *sql.DB, tx *sql.Tx, user *auth.CurrentUser, ds tc.D
 			if _, err := tx.Exec(q, *ds.ID); err != nil {
 				return fmt.Errorf("deleting primary origin for ds %s: %s", *ds.XMLID, err.Error())
 			}
-			api.CreateChangeLogRaw(api.ApiChange, "Deleted primary origin for delivery service: "+*ds.XMLID, *user, db)
+			api.CreateChangeLogRawTx(api.ApiChange, "Deleted primary origin for delivery service: "+*ds.XMLID, user, tx)
 		}
 		return nil
 	}
 
 	if count == 0 {
 		// orgServerFQDN is going from null to not null, so the primary origin needs to be created
-		return createPrimaryOrigin(db, tx, user, ds)
+		return createPrimaryOrigin(tx, user, ds)
 	}
 
-	protocol, fqdn, port, err := parseOrgServerFQDN(*ds.OrgServerFQDN)
+	protocol, fqdn, port, err := tc.ParseOrgServerFQDN(*ds.OrgServerFQDN)
 	if err != nil {
 		return fmt.Errorf("updating primary origin: %v", err)
 	}
@@ -384,7 +303,7 @@ func updatePrimaryOrigin(db *sql.DB, tx *sql.Tx, user *auth.CurrentUser, ds tc.D
 		return fmt.Errorf("update primary origin for ds %s from '%s': %s", *ds.XMLID, *ds.OrgServerFQDN, err.Error())
 	}
 
-	api.CreateChangeLogRaw(api.ApiChange, "Updated primary origin: "+name+" for delivery service: "+*ds.XMLID, *user, db)
+	api.CreateChangeLogRawTx(api.ApiChange, "Updated primary origin: "+name+" for delivery service: "+*ds.XMLID, user, tx)
 
 	return nil
 }
@@ -444,65 +363,36 @@ func getDSType(tx *sql.Tx, xmlid string) (tc.DSType, bool, error) {
 	return tc.DSTypeFromString(name), true, nil
 }
 
-func UpdateV13(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
-	return func(w http.ResponseWriter, r *http.Request) {
-		defer r.Body.Close()
-		user, err := auth.GetCurrentUser(r.Context())
-		if err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting user: "+err.Error()))
-			return
-		}
-
-		params, _, userErr, sysErr, errCode := api.AllParams(r, []string{"id"}, nil)
-		if userErr != nil || sysErr != nil {
-			api.HandleErr(w, r, errCode, userErr, sysErr)
-			return
-		}
-		if strings.HasSuffix(params["id"], ".json") {
-			params["id"] = params["id"][:len(params["id"])-len(".json")]
-		}
-		id, err := strconv.Atoi(params["id"])
-		if err != nil {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("id must be an integer"), sysErr)
-		}
-
-		ds := tc.DeliveryServiceNullableV13{}
-		if err := json.NewDecoder(r.Body).Decode(&ds); err != nil {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("malformed JSON: "+err.Error()), nil)
-			return
-		}
-		ds.ID = &id
-
-		if errs := validateV13(db, &ds); len(errs) > 0 {
-			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+util.JoinErrs(errs).Error()), nil)
-			return
-		}
-
-		if authorized, err := isTenantAuthorized(*user, db, &ds.DeliveryServiceNullableV12); err != nil {
-			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
-			return
-		} else if !authorized {
-			api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
-			return
-		}
-
-		ds, errCode, userErr, sysErr = update(db.DB, cfg, *user, &ds)
-		if userErr != nil || sysErr != nil {
-			api.HandleErr(w, r, errCode, userErr, sysErr)
-			return
-		}
-		api.WriteResp(w, r, []tc.DeliveryServiceNullableV13{ds})
-	}
+func UpdateV13(w http.ResponseWriter, r *http.Request) {
+	defer r.Body.Close()
+	inf, userErr, sysErr, errCode := api.NewInfo(r, []string{"id"}, []string{"id"})
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+	ds := tc.DeliveryServiceNullableV13{}
+	ds.ID = util.IntPtr(inf.IntParams["id"])
+	if err := api.Parse(r.Body, inf.Tx.Tx, &ds); err != nil {
+		api.HandleErr(w, r, http.StatusBadRequest, errors.New("decoding: "+err.Error()), nil)
+		return
+	}
+	if authorized, err := isTenantAuthorized(inf.User, inf.Tx.Tx, &ds.DeliveryServiceNullableV12); err != nil {
+		api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
+		return
+	} else if !authorized {
+		api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
+		return
+	}
+	ds, errCode, userErr, sysErr = update(inf.Tx.Tx, inf.Config, inf.User, &ds)
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, errCode, userErr, sysErr)
+		return
+	}
+	api.WriteResp(w, r, []tc.DeliveryServiceNullableV13{ds})
 }
 
-func update(db *sql.DB, cfg config.Config, user auth.CurrentUser, ds *tc.DeliveryServiceNullableV13) (tc.DeliveryServiceNullableV13, int, error, error) {
-	tx, err := db.Begin()
-	if err != nil {
-		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("could not begin transaction: " + err.Error())
-	}
-	commitTx := false
-	defer dbhelpers.FinishTx(tx, &commitTx)
-
+func update(tx *sql.Tx, cfg *config.Config, user *auth.CurrentUser, ds *tc.DeliveryServiceNullableV13) (tc.DeliveryServiceNullableV13, int, error, error) {
 	if ds.XMLID == nil {
 		return tc.DeliveryServiceNullableV13{}, http.StatusBadRequest, errors.New("missing xml_id"), nil
 	}
@@ -578,7 +468,7 @@ func update(db *sql.DB, cfg config.Config, user auth.CurrentUser, ds *tc.Deliver
 	}
 	ds.Type = &newDSType
 
-	cdnDomain, err := getCDNDomain(*ds.ID, db) // need to get the domain again, in case it changed.
+	cdnDomain, err := getCDNDomain(*ds.ID, tx) // need to get the domain again, in case it changed.
 	if err != nil {
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("getting CDN domain after update: " + err.Error())
 	}
@@ -603,7 +493,7 @@ func update(db *sql.DB, cfg config.Config, user auth.CurrentUser, ds *tc.Deliver
 	}
 
 	if newDSType.HasSSLKeys() && oldHostName != newHostName {
-		if err := updateSSLKeys(ds, newHostName, db, cfg); err != nil {
+		if err := updateSSLKeys(ds, newHostName, tx, cfg); err != nil {
 			return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("updating delivery service " + *ds.XMLID + ": updating SSL keys: " + err.Error())
 		}
 	}
@@ -621,13 +511,12 @@ func update(db *sql.DB, cfg config.Config, user auth.CurrentUser, ds *tc.Deliver
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("creating mid cacheurl parameters: " + err.Error())
 	}
 
-	if err := updatePrimaryOrigin(db, tx, &user, *ds); err != nil {
+	if err := updatePrimaryOrigin(tx, user, *ds); err != nil {
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("updating delivery service: " + err.Error())
 	}
 
 	ds.LastUpdated = &lastUpdated
-	commitTx = true
-	api.CreateChangeLogRaw(api.ApiChange, "Updated ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), user, db)
+	api.CreateChangeLogRawTx(api.ApiChange, "Updated ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), user, tx)
 	return *ds, http.StatusOK, nil, nil
 }
 
@@ -635,7 +524,6 @@ func update(db *sql.DB, cfg config.Config, user auth.CurrentUser, ds *tc.Deliver
 //all implementations of Deleter should use transactions and return the proper errorType
 func (ds *TODeliveryServiceV13) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
 	log.Debugln("TODeliveryServiceV13.Delete calling id '%v' xmlid '%v'\n", ds.ID, ds.XMLID)
-	// return nil, tc.NoError // debug
 
 	tx, err := db.Begin()
 	if err != nil {
@@ -654,7 +542,7 @@ func (ds *TODeliveryServiceV13) Delete(db *sqlx.DB, user auth.CurrentUser) (erro
 		log.Errorln("TODeliveryServiceV13.Delete called with nil ID")
 		return tc.DBError, tc.DataMissingError
 	}
-	xmlID, ok, err := ds.V12().GetXMLID(db)
+	xmlID, ok, err := ds.V12().GetXMLID(tx)
 	if err != nil {
 		log.Errorln("TODeliveryServiceV13.Delete ID '" + string(*ds.ID) + "' loading XML ID: " + err.Error())
 		return tc.DBError, tc.SystemError
@@ -710,27 +598,7 @@ func (ds *TODeliveryServiceV13) Delete(db *sqlx.DB, user auth.CurrentUser) (erro
 
 // IsTenantAuthorized implements the Tenantable interface to ensure the user is authorized on the deliveryservice tenant
 func (ds *TODeliveryServiceV13) IsTenantAuthorized(user auth.CurrentUser, db *sqlx.DB) (bool, error) {
-	return ds.V12().IsTenantAuthorized(user, db)
-}
-
-func filterAuthorized(dses []tc.DeliveryServiceNullableV13, user auth.CurrentUser, db *sqlx.DB) ([]tc.DeliveryServiceNullableV13, error) {
-	newDSes := []tc.DeliveryServiceNullableV13{}
-	for _, ds := range dses {
-		// TODO add/use a helper func to make a single SQL call, for performance
-		ok, err := tenant.IsResourceAuthorizedToUser(*ds.TenantID, user, db)
-		if err != nil {
-			if ds.XMLID == nil {
-				return nil, errors.New("isResourceAuthorized for delivery service with nil XML ID: " + err.Error())
-			} else {
-				return nil, errors.New("isResourceAuthorized for '" + *ds.XMLID + "': " + err.Error())
-			}
-		}
-		if !ok {
-			continue
-		}
-		newDSes = append(newDSes, ds)
-	}
-	return newDSes, nil
+	return ds.V12().IsTenantAuthorized(&user, db)
 }
 
 func readGetDeliveryServices(params map[string]string, db *sqlx.DB, user auth.CurrentUser) ([]tc.DeliveryServiceNullableV13, []error, tc.ApiErrorType) {
@@ -828,11 +696,11 @@ func readGetDeliveryServices(params map[string]string, db *sqlx.DB, user auth.Cu
 	return dses, nil, tc.NoError
 }
 
-func updateSSLKeys(ds *tc.DeliveryServiceNullableV13, hostName string, db *sql.DB, cfg config.Config) error {
+func updateSSLKeys(ds *tc.DeliveryServiceNullableV13, hostName string, tx *sql.Tx, cfg *config.Config) error {
 	if ds.XMLID == nil {
 		return errors.New("delivery services has no XMLID!")
 	}
-	key, ok, err := riaksvc.GetDeliveryServiceSSLKeysObj(*ds.XMLID, "latest", db, cfg.RiakAuthOptions)
+	key, ok, err := riaksvc.GetDeliveryServiceSSLKeysObj(*ds.XMLID, "latest", tx, cfg.RiakAuthOptions)
 	if err != nil {
 		return errors.New("getting SSL key: " + err.Error())
 	}
@@ -841,7 +709,7 @@ func updateSSLKeys(ds *tc.DeliveryServiceNullableV13, hostName string, db *sql.D
 	}
 	key.DeliveryService = *ds.XMLID
 	key.Hostname = hostName
-	if err := riaksvc.PutDeliveryServiceSSLKeysObj(key, db, cfg.RiakAuthOptions); err != nil {
+	if err := riaksvc.PutDeliveryServiceSSLKeysObj(key, tx, cfg.RiakAuthOptions); err != nil {
 		return errors.New("putting updated SSL key: " + err.Error())
 	}
 	return nil
@@ -875,10 +743,10 @@ func getHostName(dsProtocol *int, dsType tc.DSType, dsRoutingName string, dsMatc
 	return host, nil
 }
 
-func getCDNDomain(dsID int, db *sql.DB) (string, error) {
+func getCDNDomain(dsID int, tx *sql.Tx) (string, error) {
 	q := `SELECT cdn.domain_name from cdn where cdn.id = (SELECT ds.cdn_id from deliveryservice as ds where ds.id = $1)`
 	cdnDomain := ""
-	if err := db.QueryRow(q, dsID).Scan(&cdnDomain); err != nil {
+	if err := tx.QueryRow(q, dsID).Scan(&cdnDomain); err != nil {
 		return "", fmt.Errorf("getting CDN domain for delivery service '%v': "+err.Error(), dsID)
 	}
 	return cdnDomain, nil
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
index 18aabe6..e9ff56a 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
@@ -26,10 +26,11 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	"github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
index b928ffb..f50cb02 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
@@ -25,8 +25,8 @@ import (
 	"strconv"
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
 	"github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
@@ -34,9 +34,16 @@ import (
 
 // Validate ensures all required fields are present and in correct form.  Also checks request JSON is complete and valid
 func (req *TODeliveryServiceRequest) Validate(db *sqlx.DB) []error {
+	tx, err := db.DB.Begin() // must be last, MUST not return an error if this suceeds, without closing the tx
+	if err != nil {
+		return []error{errors.New("beginning transaction: " + err.Error())}
+	}
+	commitTx := false
+	dbhelpers.FinishTx(tx, &commitTx)
+
 	fromStatus := tc.RequestStatusDraft
 	if req.ID != nil && *req.ID > 0 {
-		err := db.QueryRow(`SELECT status FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)).Scan(&fromStatus)
+		err := tx.QueryRow(`SELECT status FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)).Scan(&fromStatus)
 		if err != nil {
 			return []error{err}
 		}
@@ -60,10 +67,8 @@ func (req *TODeliveryServiceRequest) Validate(db *sqlx.DB) []error {
 	}
 
 	errs := tovalidate.ToErrors(errMap)
-
 	// ensure the deliveryservice requested is valid
-	e := deliveryservice.ValidateV13(db, req.DeliveryService)
-	errs = append(errs, e...)
-
+	errs = append(errs, req.DeliveryService.Validate(tx))
+	commitTx = true
 	return errs
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
index 950642c..ddb76e4 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
@@ -20,23 +20,23 @@ package servers
  */
 
 import (
+	"encoding/json"
 	"errors"
 	"fmt"
+	"net/http"
 	"strconv"
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
-	"github.com/go-ozzo/ozzo-validation"
-
-	"encoding/json"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
+
+	"github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
-	"net/http"
 )
 
 // TODeliveryServiceRequest provides a type alias to define functions on
@@ -288,7 +288,6 @@ type DSServerIds struct {
 
 type TODSServerIds DSServerIds
 
-
 func createServersForDsIdRef() *TODSServerIds {
 	var dsserversRef = TODSServerIds(DSServerIds{})
 	return &dsserversRef
@@ -307,7 +306,7 @@ func GetReplaceHandler(db *sqlx.DB) http.HandlerFunc {
 		}
 
 		// get list of server Ids to insert
-		payload :=  createServersForDsIdRef() 
+		payload := createServersForDsIdRef()
 
 		if err := json.NewDecoder(r.Body).Decode(payload); err != nil {
 			log.Errorf("Error trying to decode the request body: %s", err)
@@ -333,12 +332,30 @@ func GetReplaceHandler(db *sqlx.DB) http.HandlerFunc {
 			return
 		}
 
+		// perform the insert transaction
+		rollbackTransaction := true
+		tx, err := db.Beginx()
+		if err != nil {
+			log.Errorln("could not begin transaction: %v", err)
+			handleErrs(http.StatusInternalServerError, err)
+			return
+		}
+		defer func() {
+			if tx == nil || !rollbackTransaction {
+				return
+			}
+			err := tx.Rollback()
+			if err != nil {
+				log.Errorln(errors.New("rolling back transaction: " + err.Error()))
+			}
+		}()
+
 		// if the object has tenancy enabled, check that user is able to access the tenant
 		// check user tenancy access to this resource.
 		row := db.QueryRow("SELECT xml_id FROM deliveryservice WHERE id = $1", *dsId)
 		var xmlId string
 		row.Scan(&xmlId)
-		hasAccess, err, apiStatus := tenant.HasTenant(*user, xmlId, db)
+		hasAccess, err, apiStatus := tenant.HasTenant(user, xmlId, tx.Tx)
 		if !hasAccess {
 			switch apiStatus {
 			case tc.SystemError:
@@ -353,25 +370,6 @@ func GetReplaceHandler(db *sqlx.DB) http.HandlerFunc {
 			}
 		}
 
-		// perform the insert transaction
-		rollbackTransaction := true
-		tx, err := db.Beginx()
-		defer func() {
-			if tx == nil || !rollbackTransaction {
-				return
-			}
-			err := tx.Rollback()
-			if err != nil {
-				log.Errorln(errors.New("rolling back transaction: " + err.Error()))
-			}
-		}()
-
-		if err != nil {
-			log.Errorln("could not begin transaction: %v", err)
-			handleErrs(http.StatusInternalServerError, err)
-			return
-		}
-
 		if *payload.Replace {
 			// delete existing
 			rows, err := db.Queryx("DELETE FROM deliveryservice_server WHERE deliveryservice = $1", *dsId)
@@ -387,7 +385,7 @@ func GetReplaceHandler(db *sqlx.DB) http.HandlerFunc {
 		i := 0
 		respServers := []int{}
 
-		for _ , server := range servers {
+		for _, server := range servers {
 			dtos := map[string]interface{}{"id": dsId, "server": server}
 			resultRows, err := tx.NamedQuery(insertIdsQuery(), dtos)
 			if err != nil {
@@ -435,7 +433,6 @@ func GetReplaceHandler(db *sqlx.DB) http.HandlerFunc {
 
 type TODeliveryServiceServers tc.DeliveryServiceServers
 
-
 func createServersRef() *TODeliveryServiceServers {
 	serversRef := TODeliveryServiceServers(tc.DeliveryServiceServers{})
 	return &serversRef
@@ -469,9 +466,27 @@ func GetCreateHandler(db *sqlx.DB) http.HandlerFunc {
 			return
 		}
 
+		// perform the insert transaction
+		rollbackTransaction := true
+		tx, err := db.Beginx()
+		if err != nil {
+			log.Errorln("could not begin transaction: %v", err)
+			handleErrs(http.StatusInternalServerError, err)
+			return
+		}
+		defer func() {
+			if tx == nil || !rollbackTransaction {
+				return
+			}
+			err := tx.Rollback()
+			if err != nil {
+				log.Errorln(errors.New("rolling back transaction: " + err.Error()))
+			}
+		}()
+
 		// if the object has tenancy enabled, check that user is able to access the tenant
 		// check user tenancy access to this resource.
-		hasAccess, err, apiStatus := tenant.HasTenant(*user, xmlId, db)
+		hasAccess, err, apiStatus := tenant.HasTenant(user, xmlId, tx.Tx)
 		if !hasAccess {
 			switch apiStatus {
 			case tc.SystemError:
@@ -522,25 +537,6 @@ func GetCreateHandler(db *sqlx.DB) http.HandlerFunc {
 			return
 		}
 
-		// perform the insert transaction
-		rollbackTransaction := true
-		tx, err := db.Beginx()
-		defer func() {
-			if tx == nil || !rollbackTransaction {
-				return
-			}
-			err := tx.Rollback()
-			if err != nil {
-				log.Errorln(errors.New("rolling back transaction: " + err.Error()))
-			}
-		}()
-
-		if err != nil {
-			log.Errorln("could not begin transaction: %v", err)
-			handleErrs(http.StatusInternalServerError, err)
-			return
-		}
-
 		// We have to get the server Ids and iterate through them because of a bug in the Go
 		// transaction which returns an error if you perform a Select after an Insert in
 		// the same transaction
@@ -628,7 +624,7 @@ func GetReadHandler(db *sqlx.DB, filter tc.Filter) http.HandlerFunc {
 			return
 		}
 
-		dssres := tc.DSServersAttrResponse{ servers }
+		dssres := tc.DSServersAttrResponse{servers}
 		respBts, err := json.Marshal(dssres)
 		if err != nil {
 			handleErrs(http.StatusInternalServerError, err)
diff --git a/traffic_ops/traffic_ops_golang/deliveryservices_keys.go b/traffic_ops/traffic_ops_golang/deliveryservices_keys.go
index 34598da..9c165d8 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservices_keys.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservices_keys.go
@@ -37,73 +37,52 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/config"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/riaksvc"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
+
 	"github.com/jmoiron/sqlx"
 )
 
 // Delivery Services: SSL Keys.
 
 // returns the cdn_id found by domainname.
-func getCDNIDByDomainname(domainName string, db *sqlx.DB) (sql.NullInt64, error) {
-	cdnQuery := `SELECT id from cdn WHERE domain_name = $1`
-	var cdnID sql.NullInt64
-
-	noCdnID := sql.NullInt64{
-		Int64: 0,
-		Valid: false,
-	}
-
-	rows, err := db.Query(cdnQuery, domainName)
-	if err != nil {
-		return noCdnID, err
-	}
-	defer rows.Close()
-
-	for rows.Next() {
-		if err := rows.Scan(&cdnID); err != nil {
-			return noCdnID, err
+func getCDNIDByDomainname(domainName string, tx *sql.Tx) (int64, bool, error) {
+	cdnID := int64(0)
+	if err := tx.QueryRow(`SELECT id from cdn WHERE domain_name = $1`, domainName).Scan(&cdnID); err != nil {
+		if err == sql.ErrNoRows {
+			return 0, false, nil
 		}
+		return 0, false, err
 	}
-
-	return cdnID, nil
+	return cdnID, true, nil
 }
 
 // returns a delivery service xmlId for a cdn by host regex.
-func getXMLID(cdnID sql.NullInt64, hostRegex string, db *sqlx.DB) (sql.NullString, error) {
-	dsQuery := `
-			SELECT ds.xml_id from deliveryservice ds
-			INNER JOIN deliveryservice_regex dr 
-			on ds.id = dr.deliveryservice AND ds.cdn_id = $1
-			INNER JOIN regex r on r.id = dr.regex
-			WHERE r.pattern = $2
-		`
-	var xmlID sql.NullString
-
-	rows, err := db.Query(dsQuery, cdnID.Int64, hostRegex)
-	if err != nil {
-		xmlID.Valid = false
-		return xmlID, err
-	}
-	defer rows.Close()
-
-	for rows.Next() {
-		if err := rows.Scan(&xmlID); err != nil {
-			xmlID.Valid = false
-			return xmlID, err
+func getXMLID(cdnID int64, hostRegex string, tx *sql.Tx) (string, bool, error) {
+	q := `
+SELECT ds.xml_id from deliveryservice ds
+JOIN deliveryservice_regex dr on ds.id = dr.deliveryservice AND ds.cdn_id = $1
+JOIN regex r on r.id = dr.regex
+WHERE r.pattern = $2
+`
+	xmlID := ""
+	if err := tx.QueryRow(q, cdnID, hostRegex).Scan(&xmlID); err != nil {
+		if err == sql.ErrNoRows {
+			return "", false, nil
 		}
+		return "", false, errors.New("querying xml id: " + err.Error())
 	}
-
-	return xmlID, nil
+	return xmlID, true, nil
 }
 
-func getDeliveryServiceSSLKeysByXMLID(xmlID string, version string, db *sql.DB, cfg config.Config) ([]byte, error) {
+func getDeliveryServiceSSLKeysByXMLID(xmlID string, version string, tx *sql.Tx, cfg config.Config) ([]byte, error) {
 	if cfg.RiakEnabled == false {
 		err := errors.New("Riak is not configured!")
 		log.Errorln("getting delivery services SSL keys: " + err.Error())
 		return nil, err
 	}
-	key, ok, err := riaksvc.GetDeliveryServiceSSLKeysObj(xmlID, version, db, cfg.RiakAuthOptions)
+	key, ok, err := riaksvc.GetDeliveryServiceSSLKeysObj(xmlID, version, tx, cfg.RiakAuthOptions)
 	if err != nil {
 		log.Errorln("getting delivery service keys: " + err.Error())
 		return nil, err
@@ -244,8 +223,16 @@ func addDeliveryServiceSSLKeysHandler(db *sqlx.DB, cfg config.Config) http.Handl
 			return
 		}
 
+		tx, err := db.DB.Begin()
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("beginning transaction: "+err.Error()))
+			return
+		}
+		commitTx := false
+		defer dbhelpers.FinishTx(tx, &commitTx)
+
 		// check user tenancy access to this resource.
-		hasAccess, err, apiStatus := tenant.HasTenant(*user, keysObj.DeliveryService, db)
+		hasAccess, err, apiStatus := tenant.HasTenant(user, keysObj.DeliveryService, tx)
 		if !hasAccess {
 			switch apiStatus {
 			case tc.SystemError:
@@ -276,14 +263,15 @@ func addDeliveryServiceSSLKeysHandler(db *sqlx.DB, cfg config.Config) http.Handl
 			return
 		}
 
-		if err := riaksvc.PutDeliveryServiceSSLKeysObj(keysObj, db.DB, cfg.RiakAuthOptions); err != nil {
+		if err := riaksvc.PutDeliveryServiceSSLKeysObj(keysObj, tx, cfg.RiakAuthOptions); err != nil {
 			log.Errorln("putting Riak SSL keys for delivery service '" + keysObj.DeliveryService + "': " + err.Error())
 			handleErr(http.StatusInternalServerError, err)
 			return
 		}
 
+		commitTx = true
 		w.Header().Set("Content-Type", "application/json")
-		fmt.Fprintf(w, "%s", keysJSON)
+		w.Write(keysJSON)
 	}
 }
 
@@ -328,15 +316,21 @@ func getDeliveryServiceSSLKeysByHostNameHandler(db *sqlx.DB, cfg config.Config)
 			hostRegex = ".*\\." + strArr[1] + "\\..*"
 		}
 
+		tx, err := db.DB.Begin()
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("beginning transaction: "+err.Error()))
+			return
+		}
+		commitTx := false
+		defer dbhelpers.FinishTx(tx, &commitTx)
+
 		// lookup the cdnID
-		cdnID, err := getCDNIDByDomainname(domainName, db)
+		cdnID, ok, err := getCDNIDByDomainname(domainName, tx)
 		if err != nil {
 			handleErr(http.StatusInternalServerError, err)
 			return
 		}
-
-		// verify that a valid cdnID was returned.
-		if !cdnID.Valid {
+		if !ok {
 			alert := tc.CreateAlerts(tc.InfoLevel, fmt.Sprintf(" - a cdn does not exist for the domain: %s parsed from hostname: %s",
 				domainName, hostName))
 			respBytes, err = json.Marshal(alert)
@@ -344,50 +338,51 @@ func getDeliveryServiceSSLKeysByHostNameHandler(db *sqlx.DB, cfg config.Config)
 				log.Errorf("failed to marshal an alert response: %s\n", err)
 				return
 			}
-		} else {
-			// now lookup the deliveryservice xmlID
-			xmlIDStr, err := getXMLID(cdnID, hostRegex, db)
+			w.Header().Set("Content-Type", "application/json")
+			w.Write(respBytes)
+		}
+		// now lookup the deliveryservice xmlID
+		xmlID, ok, err := getXMLID(cdnID, hostRegex, tx)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting xml id: "+err.Error()))
+			return
+		}
+		if !ok {
+			alert := tc.CreateAlerts(tc.InfoLevel, fmt.Sprintf("  - a delivery service does not exist for a host with hostname of %s",
+				hostName))
+			respBytes, err = json.Marshal(alert)
 			if err != nil {
+				log.Errorf("failed to marshal an alert response: %s\n", err)
 				handleErr(http.StatusInternalServerError, err)
 				return
 			}
+			w.Header().Set("Content-Type", "application/json")
+			w.Write(respBytes)
+		}
 
-			// verify that the xmlIDStr returned is valid, ie not nil
-			if !xmlIDStr.Valid {
-				alert := tc.CreateAlerts(tc.InfoLevel, fmt.Sprintf("  - a delivery service does not exist for a host with hostname of %s",
-					hostName))
-				respBytes, err = json.Marshal(alert)
-				if err != nil {
-					log.Errorf("failed to marshal an alert response: %s\n", err)
-					handleErr(http.StatusInternalServerError, err)
-					return
-				}
-			} else {
-				xmlID := xmlIDStr.String
-				// check user tenancy access to this resource.
-				hasAccess, err, apiStatus := tenant.HasTenant(*user, xmlID, db)
-				if !hasAccess {
-					switch apiStatus {
-					case tc.SystemError:
-						handleErr(http.StatusInternalServerError, err)
-						return
-					case tc.DataMissingError:
-						handleErr(http.StatusBadRequest, err)
-						return
-					case tc.ForbiddenError:
-						handleErr(http.StatusForbidden, err)
-						return
-					}
-				}
-				respBytes, err = getDeliveryServiceSSLKeysByXMLID(xmlID, version, db.DB, cfg)
-				if err != nil {
-					handleErr(http.StatusInternalServerError, err)
-					return
-				}
+		// check user tenancy access to this resource.
+		hasAccess, err, apiStatus := tenant.HasTenant(user, xmlID, tx)
+		if !hasAccess {
+			switch apiStatus {
+			case tc.SystemError:
+				handleErr(http.StatusInternalServerError, err)
+				return
+			case tc.DataMissingError:
+				handleErr(http.StatusBadRequest, err)
+				return
+			case tc.ForbiddenError:
+				handleErr(http.StatusForbidden, err)
+				return
 			}
 		}
+		respBytes, err = getDeliveryServiceSSLKeysByXMLID(xmlID, version, tx, cfg)
+		if err != nil {
+			handleErr(http.StatusInternalServerError, err)
+			return
+		}
+		commitTx = true
 		w.Header().Set("Content-Type", "application/json")
-		fmt.Fprintf(w, "%s", respBytes)
+		w.Write(respBytes)
 	}
 }
 
@@ -418,8 +413,16 @@ func getDeliveryServiceSSLKeysByXMLIDHandler(db *sqlx.DB, cfg config.Config) htt
 
 		xmlID := pathParams["xmlID"]
 
+		tx, err := db.DB.Begin()
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("beginning transaction: "+err.Error()))
+			return
+		}
+		commitTx := false
+		defer dbhelpers.FinishTx(tx, &commitTx)
+
 		// check user tenancy access to this resource.
-		hasAccess, err, apiStatus := tenant.HasTenant(*user, xmlID, db)
+		hasAccess, err, apiStatus := tenant.HasTenant(user, xmlID, tx)
 		if !hasAccess {
 			switch apiStatus {
 			case tc.SystemError:
@@ -434,13 +437,14 @@ func getDeliveryServiceSSLKeysByXMLIDHandler(db *sqlx.DB, cfg config.Config) htt
 			}
 		}
 
-		respBytes, err = getDeliveryServiceSSLKeysByXMLID(xmlID, version, db.DB, cfg)
+		respBytes, err = getDeliveryServiceSSLKeysByXMLID(xmlID, version, tx, cfg)
 		if err != nil {
 			handleErr(http.StatusInternalServerError, err)
 			return
 		}
 
 		w.Header().Set("Content-Type", "application/json")
-		fmt.Fprintf(w, "%s", respBytes)
+		commitTx = true
+		w.Write(respBytes)
 	}
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go b/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
index 8c7ab55..ead75c3 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
@@ -32,13 +32,8 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
-	// "github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	// "github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
 
-	// "github.com/asaskevich/govalidator"
-	// "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
-	// "github.com/lib/pq"
 )
 
 func Get(dbx *sqlx.DB) http.HandlerFunc {
diff --git a/traffic_ops/traffic_ops_golang/division/divisions.go b/traffic_ops/traffic_ops_golang/division/divisions.go
index d204d7c..70f864e 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions.go
@@ -27,10 +27,11 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
@@ -155,7 +156,7 @@ func (division *TODivision) Create(db *sqlx.DB, user auth.CurrentUser) (error, t
 
 func (division *TODivision) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
 	if strings.HasSuffix(parameters["name"], ".json") {
-		parameters["name"] = parameters["name"][:len(parameters["name"]) - len(".json")]
+		parameters["name"] = parameters["name"][:len(parameters["name"])-len(".json")]
 	}
 	// Query Parameters to Database Query column mappings
 	// see the fields mapped in the SQL query
diff --git a/traffic_ops/traffic_ops_golang/origin/origins.go b/traffic_ops/traffic_ops_golang/origin/origins.go
index fd6cc02..b988433 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins.go
@@ -27,12 +27,12 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
 
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/go-ozzo/ozzo-validation/is"
diff --git a/traffic_ops/traffic_ops_golang/parameter/parameters.go b/traffic_ops/traffic_ops_golang/parameter/parameters.go
index 264bc53..87057e5 100644
--- a/traffic_ops/traffic_ops_golang/parameter/parameters.go
+++ b/traffic_ops/traffic_ops_golang/parameter/parameters.go
@@ -26,12 +26,12 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
-	validation "github.com/go-ozzo/ozzo-validation"
 
+	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
diff --git a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
index 942c8da..c204c9e 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
@@ -26,10 +26,11 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/profile/profiles.go b/traffic_ops/traffic_ops_golang/profile/profiles.go
index de7530b..b7c6862 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles.go
@@ -26,12 +26,13 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/parameter"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
index eb21810..34bd4c6 100644
--- a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
+++ b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
@@ -26,13 +26,13 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	tc "github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
-	validation "github.com/go-ozzo/ozzo-validation"
 
+	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
diff --git a/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go b/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
index ab514aa..f6d8e3b 100644
--- a/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
+++ b/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
@@ -33,7 +33,7 @@ import (
 const DeliveryServiceSSLKeysBucket = "ssl"
 const DNSSECKeysBucket = "dnssec"
 
-func GetDeliveryServiceSSLKeysObj(xmlID string, version string, db *sql.DB, authOpts *riak.AuthOptions) (tc.DeliveryServiceSSLKeys, bool, error) {
+func GetDeliveryServiceSSLKeysObj(xmlID string, version string, tx *sql.Tx, authOpts *riak.AuthOptions) (tc.DeliveryServiceSSLKeys, bool, error) {
 	key := tc.DeliveryServiceSSLKeys{}
 	if version == "" {
 		xmlID += "-latest"
@@ -41,7 +41,7 @@ func GetDeliveryServiceSSLKeysObj(xmlID string, version string, db *sql.DB, auth
 		xmlID += "-" + version
 	}
 	found := false
-	err := WithCluster(db, authOpts, func(cluster StorageCluster) error {
+	err := WithClusterTx(tx, authOpts, func(cluster StorageCluster) error {
 		// get the deliveryservice ssl keys by xmlID and version
 		ro, err := FetchObjectValues(xmlID, DeliveryServiceSSLKeysBucket, cluster)
 		if err != nil {
@@ -63,12 +63,12 @@ func GetDeliveryServiceSSLKeysObj(xmlID string, version string, db *sql.DB, auth
 	return key, found, nil
 }
 
-func PutDeliveryServiceSSLKeysObj(key tc.DeliveryServiceSSLKeys, db *sql.DB, authOpts *riak.AuthOptions) error {
+func PutDeliveryServiceSSLKeysObj(key tc.DeliveryServiceSSLKeys, tx *sql.Tx, authOpts *riak.AuthOptions) error {
 	keyJSON, err := json.Marshal(&key)
 	if err != nil {
 		return errors.New("marshalling key: " + err.Error())
 	}
-	err = WithCluster(db, authOpts, func(cluster StorageCluster) error {
+	err = WithClusterTx(tx, authOpts, func(cluster StorageCluster) error {
 		obj := &riak.Object{
 			ContentType:     "text/json",
 			Charset:         "utf-8",
diff --git a/traffic_ops/traffic_ops_golang/role/roles.go b/traffic_ops/traffic_ops_golang/role/roles.go
index 06b25be..3871c8b 100644
--- a/traffic_ops/traffic_ops_golang/role/roles.go
+++ b/traffic_ops/traffic_ops_golang/role/roles.go
@@ -26,11 +26,12 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 589c5bf..0296172 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -298,10 +298,10 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.1, http.MethodGet, `deliveryservices/?(\.json)?$`, api.ReadHandler(deliveryservice.GetRefTypeV12(d.Config, d.DB), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.3, http.MethodGet, `deliveryservices/{id}/?(\.json)?$`, api.ReadHandler(deliveryservice.GetRefTypeV13(d.Config, d.DB), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodGet, `deliveryservices/{id}/?(\.json)?$`, api.ReadHandler(deliveryservice.GetRefTypeV12(d.Config, d.DB), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.3, http.MethodPost, `deliveryservices/?(\.json)?$`, deliveryservice.CreateV13(d.DB, d.Config), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `deliveryservices/?(\.json)?$`, deliveryservice.CreateV12(d.DB, d.Config), auth.PrivLevelOperations, Authenticated, nil},
-		{1.3, http.MethodPut, `deliveryservices/{id}/?(\.json)?$`, deliveryservice.UpdateV13(d.DB, d.Config), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPut, `deliveryservices/{id}/?(\.json)?$`, deliveryservice.UpdateV12(d.DB, d.Config), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodPost, `deliveryservices/?(\.json)?$`, deliveryservice.CreateV13, auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `deliveryservices/?(\.json)?$`, deliveryservice.CreateV12, auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodPut, `deliveryservices/{id}/?(\.json)?$`, deliveryservice.UpdateV13, auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPut, `deliveryservices/{id}/?(\.json)?$`, deliveryservice.UpdateV12, auth.PrivLevelOperations, Authenticated, nil},
 		{1.3, http.MethodDelete, `deliveryservices/{id}/?(\.json)?$`, api.DeleteHandler(deliveryservice.GetRefTypeV13(d.Config, d.DB), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 		{1.1, http.MethodDelete, `deliveryservices/{id}/?(\.json)?$`, api.DeleteHandler(deliveryservice.GetRefTypeV12(d.Config, d.DB), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
diff --git a/traffic_ops/traffic_ops_golang/server/servers.go b/traffic_ops/traffic_ops_golang/server/servers.go
index f4f1052..801237b 100644
--- a/traffic_ops/traffic_ops_golang/server/servers.go
+++ b/traffic_ops/traffic_ops_golang/server/servers.go
@@ -26,11 +26,11 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
 
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/go-ozzo/ozzo-validation/is"
diff --git a/traffic_ops/traffic_ops_golang/status/statuses.go b/traffic_ops/traffic_ops_golang/status/statuses.go
index 26614cb..a9fa43b 100644
--- a/traffic_ops/traffic_ops_golang/status/statuses.go
+++ b/traffic_ops/traffic_ops_golang/status/statuses.go
@@ -26,10 +26,11 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/tenant/tenancy.go b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
index 8539c04..8987f4e 100644
--- a/traffic_ops/traffic_ops_golang/tenant/tenancy.go
+++ b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
@@ -26,7 +26,9 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
+
 	"github.com/jmoiron/sqlx"
 )
 
@@ -40,33 +42,28 @@ type Tenant struct {
 type DeliveryServiceTenantInfo tc.DeliveryServiceNullable
 
 // returns true if the user has tenant access on this deliveryservice
-func (dsInfo DeliveryServiceTenantInfo) IsTenantAuthorized(user auth.CurrentUser, db *sqlx.DB) (bool, error) {
+func (dsInfo DeliveryServiceTenantInfo) IsTenantAuthorized(user *auth.CurrentUser, tx *sql.Tx) (bool, error) {
 	if dsInfo.TenantID == nil {
 		return false, errors.New("TenantID is nil")
 	}
-	return IsResourceAuthorizedToUser(*dsInfo.TenantID, user, db)
+	return IsResourceAuthorizedToUserTx(*dsInfo.TenantID, user, tx)
 }
 
 // returns tenant information for a deliveryservice
-func GetDeliveryServiceTenantInfo(xmlId string, db *sqlx.DB) (*DeliveryServiceTenantInfo, error) {
-	ds := DeliveryServiceTenantInfo{}
-	query := "SELECT xml_id,tenant_id FROM deliveryservice where xml_id = $1"
-
-	err := db.Get(&ds, query, xmlId)
-	switch {
-	case err == sql.ErrNoRows:
-		ds = DeliveryServiceTenantInfo{}
-		return &ds, fmt.Errorf("a deliveryservice with xml_id '%s' was not found", xmlId)
-	case err != nil:
-		return nil, err
-	default:
-		return &ds, nil
+func GetDeliveryServiceTenantInfo(xmlID string, tx *sql.Tx) (*DeliveryServiceTenantInfo, error) {
+	ds := DeliveryServiceTenantInfo{XMLID: util.StrPtr(xmlID)}
+	if err := tx.QueryRow(`SELECT tenant_id FROM deliveryservice where xml_id = $1`, &ds.XMLID).Scan(&ds.TenantID); err != nil {
+		if err == sql.ErrNoRows {
+			return &ds, errors.New("a deliveryservice with xml_id '" + xmlID + "' was not found")
+		}
+		return nil, errors.New("querying tenant id from delivery service: " + err.Error())
 	}
+	return &ds, nil
 }
 
 // tenancy check wrapper for deliveryservice
-func HasTenant(user auth.CurrentUser, XMLID string, db *sqlx.DB) (bool, error, tc.ApiErrorType) {
-	dsInfo, err := GetDeliveryServiceTenantInfo(XMLID, db)
+func HasTenant(user *auth.CurrentUser, XMLID string, tx *sql.Tx) (bool, error, tc.ApiErrorType) {
+	dsInfo, err := GetDeliveryServiceTenantInfo(XMLID, tx)
 	if err != nil {
 		if dsInfo == nil {
 			return false, fmt.Errorf("deliveryservice lookup failure: %v", err), tc.SystemError
@@ -74,7 +71,7 @@ func HasTenant(user auth.CurrentUser, XMLID string, db *sqlx.DB) (bool, error, t
 			return false, fmt.Errorf("no such deliveryservice: '%s'", XMLID), tc.DataMissingError
 		}
 	}
-	hasAccess, err := dsInfo.IsTenantAuthorized(user, db)
+	hasAccess, err := dsInfo.IsTenantAuthorized(user, tx)
 	if err != nil {
 		return false, fmt.Errorf("user tenancy check failure: %v", err), tc.SystemError
 	}
@@ -147,7 +144,7 @@ func GetUserTenantIDList(user auth.CurrentUser, db *sqlx.DB) ([]int, error) {
 		if err := rows.Scan(&tenantID); err != nil {
 			return nil, err
 		}
-			tenants = append(tenants, tenantID)
+		tenants = append(tenants, tenantID)
 	}
 
 	return tenants, nil
@@ -197,3 +194,36 @@ func IsResourceAuthorizedToUser(resourceTenantID int, user auth.CurrentUser, db
 		}
 	}
 }
+
+// returns a boolean value describing if the user has access to the provided resource tenant id and an error
+// if use_tenancy is set to false (0 in the db) this method will return true allowing access.
+func IsResourceAuthorizedToUserTx(resourceTenantID int, user *auth.CurrentUser, tx *sql.Tx) (bool, error) {
+	// $1 is the user tenant ID and $2 is the resource tenant ID
+	query := `WITH RECURSIVE q AS (SELECT id, active FROM tenant WHERE id = $1
+	UNION SELECT t.id, t.active FROM TENANT t JOIN q ON q.id = t.parent_id),
+	tenancy AS (SELECT COALESCE(value::boolean,FALSE) AS value FROM parameter WHERE name = 'use_tenancy' AND config_file = 'global' UNION ALL SELECT FALSE FETCH FIRST 1 ROW ONLY)
+	SELECT id, active, tenancy.value AS use_tenancy FROM tenancy, q WHERE id = $2 UNION ALL SELECT -1, false, tenancy.value AS use_tenancy FROM tenancy FETCH FIRST 1 ROW ONLY;`
+
+	var tenantID int
+	var active bool
+	var useTenancy bool
+
+	log.Debugln("\nQuery: ", query)
+	err := tx.QueryRow(query, user.TenantID, resourceTenantID).Scan(&tenantID, &active, &useTenancy)
+
+	switch {
+	case err != nil:
+		log.Errorf("Error checking user tenant %v access on resourceTenant  %v: %v", user.TenantID, resourceTenantID, err.Error())
+		return false, err
+	default:
+		if !useTenancy {
+			return true, nil
+		}
+		if active && tenantID == resourceTenantID {
+			return true, nil
+		} else {
+			fmt.Printf("default")
+			return false, nil
+		}
+	}
+}
diff --git a/traffic_ops/traffic_ops_golang/types/types.go b/traffic_ops/traffic_ops_golang/types/types.go
index e618ac1..7baaf52 100644
--- a/traffic_ops/traffic_ops_golang/types/types.go
+++ b/traffic_ops/traffic_ops_golang/types/types.go
@@ -26,10 +26,11 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
+
 	validation "github.com/go-ozzo/ozzo-validation"
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
diff --git a/traffic_ops/traffic_ops_golang/urisigning.go b/traffic_ops/traffic_ops_golang/urisigning.go
index a30f6b1..e53a3ad 100644
--- a/traffic_ops/traffic_ops_golang/urisigning.go
+++ b/traffic_ops/traffic_ops_golang/urisigning.go
@@ -32,8 +32,10 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/config"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/riaksvc"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
+
 	"github.com/basho/riak-go-client"
 	"github.com/jmoiron/sqlx"
 	"github.com/lestrrat/go-jwx/jwk"
@@ -73,8 +75,16 @@ func getURIsignkeysHandler(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
 
 		xmlID := pathParams["xmlID"]
 
+		tx, err := db.DB.Begin()
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("beginning transaction: "+err.Error()))
+			return
+		}
+		commitTx := false
+		defer dbhelpers.FinishTx(tx, &commitTx)
+
 		// check user tenancy access to this resource.
-		hasAccess, err, apiStatus := tenant.HasTenant(*user, xmlID, db)
+		hasAccess, err, apiStatus := tenant.HasTenant(user, xmlID, tx)
 		if !hasAccess {
 			switch apiStatus {
 			case tc.SystemError:
@@ -125,8 +135,9 @@ func getURIsignkeysHandler(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
 		} else {
 			respBytes = ro[0].Value
 		}
+		commitTx = true
 		w.Header().Set("Content-Type", "application/json")
-		fmt.Fprintf(w, "%s", respBytes)
+		w.Write(respBytes)
 	}
 }
 
@@ -155,8 +166,16 @@ func removeDeliveryServiceURIKeysHandler(db *sqlx.DB, cfg config.Config) http.Ha
 
 		xmlID := pathParams["xmlID"]
 
+		tx, err := db.DB.Begin()
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("beginning transaction: "+err.Error()))
+			return
+		}
+		commitTx := false
+		defer dbhelpers.FinishTx(tx, &commitTx)
+
 		// check user tenancy access to this resource.
-		hasAccess, err, apiStatus := tenant.HasTenant(*user, xmlID, db)
+		hasAccess, err, apiStatus := tenant.HasTenant(user, xmlID, tx)
 		if !hasAccess {
 			switch apiStatus {
 			case tc.SystemError:
@@ -213,8 +232,9 @@ func removeDeliveryServiceURIKeysHandler(db *sqlx.DB, cfg config.Config) http.Ha
 			fmt.Fprintf(w, http.StatusText(http.StatusInternalServerError))
 			return
 		}
+		commitTx = true
 		w.Header().Set("Content-Type", "application/json")
-		fmt.Fprintf(w, "%s", respBytes)
+		w.Write(respBytes)
 	}
 }
 
@@ -245,8 +265,16 @@ func saveDeliveryServiceURIKeysHandler(db *sqlx.DB, cfg config.Config) http.Hand
 
 		xmlID := pathParams["xmlID"]
 
+		tx, err := db.DB.Begin()
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("beginning transaction: "+err.Error()))
+			return
+		}
+		commitTx := false
+		defer dbhelpers.FinishTx(tx, &commitTx)
+
 		// check user tenancy access to this resource.
-		hasAccess, err, apiStatus := tenant.HasTenant(*user, xmlID, db)
+		hasAccess, err, apiStatus := tenant.HasTenant(user, xmlID, tx)
 		if !hasAccess {
 			switch apiStatus {
 			case tc.SystemError:
@@ -313,7 +341,7 @@ func saveDeliveryServiceURIKeysHandler(db *sqlx.DB, cfg config.Config) http.Hand
 		}
 
 		w.Header().Set("Content-Type", "application/json")
-		fmt.Fprintf(w, "%s", data)
+		w.Write(data)
 	}
 }
 
diff --git a/traffic_ops/traffic_ops_golang/utils/errors.go b/traffic_ops/traffic_ops_golang/utils/errors.go
deleted file mode 100644
index dae6be3..0000000
--- a/traffic_ops/traffic_ops_golang/utils/errors.go
+++ /dev/null
@@ -1,29 +0,0 @@
-package utils
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-func ErrorsToStrings(errs []error) []string {
-	errorStrs := []string{}
-	for _, errType := range errs {
-		et := errType.Error()
-		errorStrs = append(errorStrs, et)
-	}
-	return errorStrs
-}
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/CONTRIBUTING.md b/vendor/github.com/asaskevich/govalidator/CONTRIBUTING.md
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/CONTRIBUTING.md
rename to vendor/github.com/asaskevich/govalidator/CONTRIBUTING.md
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/LICENSE b/vendor/github.com/asaskevich/govalidator/LICENSE
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/LICENSE
rename to vendor/github.com/asaskevich/govalidator/LICENSE
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/README.md b/vendor/github.com/asaskevich/govalidator/README.md
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/README.md
rename to vendor/github.com/asaskevich/govalidator/README.md
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/arrays.go b/vendor/github.com/asaskevich/govalidator/arrays.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/arrays.go
rename to vendor/github.com/asaskevich/govalidator/arrays.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/arrays_test.go b/vendor/github.com/asaskevich/govalidator/arrays_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/arrays_test.go
rename to vendor/github.com/asaskevich/govalidator/arrays_test.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/converter.go b/vendor/github.com/asaskevich/govalidator/converter.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/converter.go
rename to vendor/github.com/asaskevich/govalidator/converter.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/converter_test.go b/vendor/github.com/asaskevich/govalidator/converter_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/converter_test.go
rename to vendor/github.com/asaskevich/govalidator/converter_test.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/error.go b/vendor/github.com/asaskevich/govalidator/error.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/error.go
rename to vendor/github.com/asaskevich/govalidator/error.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/error_test.go b/vendor/github.com/asaskevich/govalidator/error_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/error_test.go
rename to vendor/github.com/asaskevich/govalidator/error_test.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/numerics.go b/vendor/github.com/asaskevich/govalidator/numerics.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/numerics.go
rename to vendor/github.com/asaskevich/govalidator/numerics.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/numerics_test.go b/vendor/github.com/asaskevich/govalidator/numerics_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/numerics_test.go
rename to vendor/github.com/asaskevich/govalidator/numerics_test.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/patterns.go b/vendor/github.com/asaskevich/govalidator/patterns.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/patterns.go
rename to vendor/github.com/asaskevich/govalidator/patterns.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/types.go b/vendor/github.com/asaskevich/govalidator/types.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/types.go
rename to vendor/github.com/asaskevich/govalidator/types.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/utils.go b/vendor/github.com/asaskevich/govalidator/utils.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/utils.go
rename to vendor/github.com/asaskevich/govalidator/utils.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/utils_benchmark_test.go b/vendor/github.com/asaskevich/govalidator/utils_benchmark_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/utils_benchmark_test.go
rename to vendor/github.com/asaskevich/govalidator/utils_benchmark_test.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/utils_test.go b/vendor/github.com/asaskevich/govalidator/utils_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/utils_test.go
rename to vendor/github.com/asaskevich/govalidator/utils_test.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/validator.go b/vendor/github.com/asaskevich/govalidator/validator.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/validator.go
rename to vendor/github.com/asaskevich/govalidator/validator.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/validator_test.go b/vendor/github.com/asaskevich/govalidator/validator_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/validator_test.go
rename to vendor/github.com/asaskevich/govalidator/validator_test.go
diff --git a/traffic_ops/vendor/github.com/asaskevich/govalidator/wercker.yml b/vendor/github.com/asaskevich/govalidator/wercker.yml
similarity index 100%
rename from traffic_ops/vendor/github.com/asaskevich/govalidator/wercker.yml
rename to vendor/github.com/asaskevich/govalidator/wercker.yml
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/LICENSE b/vendor/github.com/go-ozzo/ozzo-validation/LICENSE
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/LICENSE
rename to vendor/github.com/go-ozzo/ozzo-validation/LICENSE
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/README.md b/vendor/github.com/go-ozzo/ozzo-validation/README.md
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/README.md
rename to vendor/github.com/go-ozzo/ozzo-validation/README.md
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/UPGRADE.md b/vendor/github.com/go-ozzo/ozzo-validation/UPGRADE.md
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/UPGRADE.md
rename to vendor/github.com/go-ozzo/ozzo-validation/UPGRADE.md
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/date.go b/vendor/github.com/go-ozzo/ozzo-validation/date.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/date.go
rename to vendor/github.com/go-ozzo/ozzo-validation/date.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/date_test.go b/vendor/github.com/go-ozzo/ozzo-validation/date_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/date_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/date_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/error.go b/vendor/github.com/go-ozzo/ozzo-validation/error.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/error.go
rename to vendor/github.com/go-ozzo/ozzo-validation/error.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/error_test.go b/vendor/github.com/go-ozzo/ozzo-validation/error_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/error_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/error_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/example_test.go b/vendor/github.com/go-ozzo/ozzo-validation/example_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/example_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/example_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/in.go b/vendor/github.com/go-ozzo/ozzo-validation/in.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/in.go
rename to vendor/github.com/go-ozzo/ozzo-validation/in.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/in_test.go b/vendor/github.com/go-ozzo/ozzo-validation/in_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/in_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/in_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/is/rules.go b/vendor/github.com/go-ozzo/ozzo-validation/is/rules.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/is/rules.go
rename to vendor/github.com/go-ozzo/ozzo-validation/is/rules.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/is/rules_test.go b/vendor/github.com/go-ozzo/ozzo-validation/is/rules_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/is/rules_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/is/rules_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/length.go b/vendor/github.com/go-ozzo/ozzo-validation/length.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/length.go
rename to vendor/github.com/go-ozzo/ozzo-validation/length.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/length_test.go b/vendor/github.com/go-ozzo/ozzo-validation/length_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/length_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/length_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/match.go b/vendor/github.com/go-ozzo/ozzo-validation/match.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/match.go
rename to vendor/github.com/go-ozzo/ozzo-validation/match.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/match_test.go b/vendor/github.com/go-ozzo/ozzo-validation/match_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/match_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/match_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/minmax.go b/vendor/github.com/go-ozzo/ozzo-validation/minmax.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/minmax.go
rename to vendor/github.com/go-ozzo/ozzo-validation/minmax.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/minmax_test.go b/vendor/github.com/go-ozzo/ozzo-validation/minmax_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/minmax_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/minmax_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_in.go b/vendor/github.com/go-ozzo/ozzo-validation/not_in.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_in.go
rename to vendor/github.com/go-ozzo/ozzo-validation/not_in.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_in_test.go b/vendor/github.com/go-ozzo/ozzo-validation/not_in_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_in_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/not_in_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_nil.go b/vendor/github.com/go-ozzo/ozzo-validation/not_nil.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_nil.go
rename to vendor/github.com/go-ozzo/ozzo-validation/not_nil.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_nil_test.go b/vendor/github.com/go-ozzo/ozzo-validation/not_nil_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/not_nil_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/not_nil_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/required.go b/vendor/github.com/go-ozzo/ozzo-validation/required.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/required.go
rename to vendor/github.com/go-ozzo/ozzo-validation/required.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/required_test.go b/vendor/github.com/go-ozzo/ozzo-validation/required_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/required_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/required_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/string.go b/vendor/github.com/go-ozzo/ozzo-validation/string.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/string.go
rename to vendor/github.com/go-ozzo/ozzo-validation/string.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/string_test.go b/vendor/github.com/go-ozzo/ozzo-validation/string_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/string_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/string_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/struct.go b/vendor/github.com/go-ozzo/ozzo-validation/struct.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/struct.go
rename to vendor/github.com/go-ozzo/ozzo-validation/struct.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/struct_test.go b/vendor/github.com/go-ozzo/ozzo-validation/struct_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/struct_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/struct_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/util.go b/vendor/github.com/go-ozzo/ozzo-validation/util.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/util.go
rename to vendor/github.com/go-ozzo/ozzo-validation/util.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/util_test.go b/vendor/github.com/go-ozzo/ozzo-validation/util_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/util_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/util_test.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/validation.go b/vendor/github.com/go-ozzo/ozzo-validation/validation.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/validation.go
rename to vendor/github.com/go-ozzo/ozzo-validation/validation.go
diff --git a/traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/validation_test.go b/vendor/github.com/go-ozzo/ozzo-validation/validation_test.go
similarity index 100%
rename from traffic_ops/vendor/github.com/go-ozzo/ozzo-validation/validation_test.go
rename to vendor/github.com/go-ozzo/ozzo-validation/validation_test.go