You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by da...@apache.org on 2018/07/04 02:39:26 UTC

[trafficcontrol] 01/15: convert all implementations of CRUD interfaces to use Keys methods

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

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

commit cf101fce9d5ec07ff5f3ec0f2b8c595ec90649cf
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Wed Mar 28 14:17:37 2018 -0600

    convert all implementations of CRUD interfaces to use Keys methods
---
 lib/go-tc/deliveryservices.go                      |  12 +-
 traffic_ops/traffic_ops_golang/api/api.go          |  12 +-
 traffic_ops/traffic_ops_golang/api/change_log.go   |  24 +-
 .../traffic_ops_golang/api/shared_handlers.go      | 116 ++++---
 .../traffic_ops_golang/api/shared_interfaces.go    |  28 +-
 traffic_ops/traffic_ops_golang/asn/asns.go         | 147 +++------
 traffic_ops/traffic_ops_golang/asn/asns_test.go    |  12 +-
 .../traffic_ops_golang/cachegroup/cachegroups.go   | 206 +++++--------
 traffic_ops/traffic_ops_golang/cdn/cdns.go         | 109 ++-----
 .../traffic_ops_golang/crconfig/deliveryservice.go |   2 +-
 .../deliveryservice/deliveryservicesv12.go         | 211 ++++++++-----
 .../deliveryservice/deliveryservicesv13.go         | 338 +++++++++++----------
 .../deliveryservice/request/comment/comments.go    |  15 +-
 .../deliveryservice/request/requests.go            | 218 ++++---------
 .../deliveryservice/request/requests_test.go       |   6 +-
 .../deliveryservice/request/validate.go            |  23 +-
 .../deliveryservicesregexes.go                     |   2 +-
 traffic_ops/traffic_ops_golang/origin/origins.go   |   2 +-
 traffic_ops/traffic_ops_golang/riaksvc/dsutil.go   |  54 +++-
 traffic_ops/traffic_ops_golang/routes.go           | 256 +++++++++-------
 traffic_ops/traffic_ops_golang/server/servers.go   |   6 +-
 .../traffic_ops_golang/systeminfo/system_info.go   |   7 +-
 traffic_ops/traffic_ops_golang/tenant/tenancy.go   |  32 +-
 23 files changed, 867 insertions(+), 971 deletions(-)

diff --git a/lib/go-tc/deliveryservices.go b/lib/go-tc/deliveryservices.go
index 6f6093f..ae34729 100644
--- a/lib/go-tc/deliveryservices.go
+++ b/lib/go-tc/deliveryservices.go
@@ -379,7 +379,7 @@ func ParseOrgServerFQDN(orgServerFQDN string) (*string, *string, *string, error)
 	return &protocol, &FQDN, port, nil
 }
 
-func (ds *DeliveryServiceNullableV12) Validate(tx *sql.Tx) error {
+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")
@@ -402,7 +402,7 @@ func (ds *DeliveryServiceNullableV12) Validate(tx *sql.Tx) error {
 		toErrs = append(toErrs, errors.New("type fields: "+err.Error()))
 	}
 	if len(toErrs) > 0 {
-		return errors.New(util.JoinErrsStr(toErrs))
+		return toErrs
 	}
 	return nil
 }
@@ -423,20 +423,20 @@ func (ds *DeliveryServiceNullableV13) Sanitize() {
 	*ds.DeepCachingType = DeepCachingTypeFromString(string(*ds.DeepCachingType))
 }
 
-func (ds *DeliveryServiceNullableV13) Validate(tx *sql.Tx) error {
+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 v12Errs := ds.DeliveryServiceNullableV12.Validate(tx); len(v12Errs) > 0 {
+		errs = append(errs, v12Errs...)
 	}
 	if len(errs) == 0 {
 		return nil
 	}
-	return errors.New(util.JoinErrsStr(errs)) // don't add context, so versions chain well
+	return errs // don't add context, so versions chain well
 }
 
 // Value implements the driver.Valuer interface
diff --git a/traffic_ops/traffic_ops_golang/api/api.go b/traffic_ops/traffic_ops_golang/api/api.go
index 5194ec0..99a4d89 100644
--- a/traffic_ops/traffic_ops_golang/api/api.go
+++ b/traffic_ops/traffic_ops_golang/api/api.go
@@ -218,7 +218,7 @@ func AllParams(req *http.Request, required []string, ints []string) (map[string]
 }
 
 type ParseValidator interface {
-	Validate(tx *sql.Tx) error
+	Validate(tx *sql.Tx) []error
 }
 
 // Decode decodes a JSON object from r into the given v, validating and sanitizing the input. This helper should be used in API endpoints, rather than the json package, to safely decode and validate PUT and POST requests.
@@ -227,8 +227,8 @@ func Parse(r io.Reader, tx *sql.Tx, v ParseValidator) error {
 	if err := json.NewDecoder(r).Decode(&v); err != nil {
 		return errors.New("decoding: " + err.Error())
 	}
-	if err := v.Validate(tx); err != nil {
-		return errors.New("validating: " + err.Error())
+	if errs := v.Validate(tx); len(errs) > 0 {
+		return errors.New("validating: " + util.JoinErrs(errs).Error())
 	}
 	return nil
 }
@@ -255,7 +255,9 @@ type APIInfo struct {
 //    }
 //    defer inf.Close()
 //
-//    err := databaseOperation(inf.Tx)
+//    ...
+//
+//    err := finalDatabaseOperation(inf.Tx)
 //    if err == nil {
 //      *inf.CommitTx = true
 //    }
@@ -311,7 +313,7 @@ func getDB(ctx context.Context) (*sqlx.DB, error) {
 		case *sqlx.DB:
 			return v, nil
 		default:
-			return nil, fmt.Errorf("DB found with bad type: %T", v)
+			return nil, fmt.Errorf("Tx found with bad type: %T", v)
 		}
 	}
 	return nil, errors.New("No db found in Context")
diff --git a/traffic_ops/traffic_ops_golang/api/change_log.go b/traffic_ops/traffic_ops_golang/api/change_log.go
index cc87a3a..99f1c5e 100644
--- a/traffic_ops/traffic_ops_golang/api/change_log.go
+++ b/traffic_ops/traffic_ops_golang/api/change_log.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 
 	"github.com/jmoiron/sqlx"
+	"errors"
 )
 
 type ChangeLog struct {
@@ -50,39 +51,34 @@ const (
 	Deleted   = "Deleted"
 )
 
-func CreateChangeLog(level string, action string, i Identifier, user auth.CurrentUser, db *sqlx.DB) error {
+func CreateChangeLog(level string, action string, i Identifier, user auth.CurrentUser, tx *sqlx.Tx) error {
 	t, ok := i.(ChangeLogger)
 	if !ok {
 		keys, _ := i.GetKeys()
-		return CreateChangeLogBuildMsg(level, action, user, db, i.GetType(), i.GetAuditName(), keys)
+		return CreateChangeLogBuildMsg(level, action, user, tx, i.GetType(), i.GetAuditName(), keys)
 	}
 	msg, err := t.ChangeLogMessage(action)
 	if err != nil {
 		log.Errorf("%++v creating log message for %++v", err, t)
 		keys, _ := i.GetKeys()
-		return CreateChangeLogBuildMsg(level, action, user, db, i.GetType(), i.GetAuditName(), keys)
+		return CreateChangeLogBuildMsg(level, action, user, tx, i.GetType(), i.GetAuditName(), keys)
 	}
-	return CreateChangeLogMsg(level, user, db, msg)
+	return CreateChangeLogRawErr(level, msg, user, tx.Tx)
 }
 
-func CreateChangeLogBuildMsg(level string, action string, user auth.CurrentUser, db *sqlx.DB, objType string, auditName string, keys map[string]interface{}) error {
+func CreateChangeLogBuildMsg(level string, action string, user auth.CurrentUser, tx *sqlx.Tx, objType string, auditName string, keys map[string]interface{}) error {
 	keyStr := "{ "
 	for key, value := range keys {
 		keyStr += key + ":" + fmt.Sprintf("%v", value) + " "
 	}
 	keyStr += "}"
 	msg := action + " " + objType + ": " + auditName + " keys: " + keyStr
-	return CreateChangeLogMsg(level, user, db, msg)
+	return CreateChangeLogRawErr(level, msg, user, tx.Tx)
 }
 
-func CreateChangeLogMsg(level string, user auth.CurrentUser, db *sqlx.DB, msg string) error {
-	query := `INSERT INTO log (level, message, tm_user) VALUES ($1, $2, $3)`
-	log.Debugf("about to exec %s with %s", query, msg)
-	if _, err := db.Exec(query, level, msg, user.ID); err != nil {
-		log.Errorf("received error: %++v from audit log insertion", err)
-		return err
-	}
-	return nil
+func CreateChangeLogRawErr(level string, msg string, user auth.CurrentUser, tx *sql.Tx) error {
+	 _, err := tx.Exec(`INSERT INTO log (level, message, tm_user) VALUES ($1, $2, $3)`, level, msg, user.ID)
+	 return errors.New("Inserting change log level '" + level + "' message '" + msg + "' user '" + user.UserName + "': " + err.Error())
 }
 
 func CreateChangeLogRaw(level string, msg string, user auth.CurrentUser, db *sql.DB) {
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index c1cc8d3..fc0e6bb 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -25,15 +25,12 @@ import (
 	"errors"
 	"fmt"
 	"net/http"
-	"reflect"
 	"strconv"
 	"strings"
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
-
-	"github.com/jmoiron/sqlx"
 )
 
 const PathParamsKey = "pathParams"
@@ -109,13 +106,13 @@ func GetCombinedParams(r *http.Request) (map[string]string, error) {
 //      we lose the ability to unmarshal the struct if a struct implementing the interface is passed in,
 //      because when when it is de-referenced it is a pointer to an interface. A new copy is created so that
 //      there are no issues with concurrent goroutines
-func decodeAndValidateRequestBody(r *http.Request, v Validator, db *sqlx.DB, user auth.CurrentUser) (interface{}, []error) {
-	payload := reflect.Indirect(reflect.ValueOf(v)).Addr().Interface() // does a shallow copy v's internal struct members
+func decodeAndValidateRequestBody(r *http.Request, v Validator) []error {
 	defer r.Body.Close()
-	if err := json.NewDecoder(r.Body).Decode(payload); err != nil {
-		return nil, []error{err}
+
+	if err := json.NewDecoder(r.Body).Decode(v); err != nil {
+		return []error{err}
 	}
-	return payload, payload.(Validator).Validate(db)
+	return v.Validate()
 }
 
 //this creates a handler function from the pointer to a struct implementing the Reader interface
@@ -123,12 +120,17 @@ func decodeAndValidateRequestBody(r *http.Request, v Validator, db *sqlx.DB, use
 //      combines the path and query parameters
 //      produces the proper status code based on the error code returned
 //      marshals the structs returned into the proper response json
-func ReadHandler(typeRef Reader, db *sqlx.DB) http.HandlerFunc {
+func ReadHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		//create error function with ResponseWriter and Request
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
 
-		ctx := r.Context()
+		inf, userErr, sysErr, errCode := NewInfo(r, nil, nil)
+		if userErr != nil || sysErr != nil {
+			HandleErr(w, r, errCode, userErr, sysErr)
+			return
+		}
+		defer inf.Close()
 
 		// Load the PathParams into the query parameters for pass through
 		params, err := GetCombinedParams(r)
@@ -138,14 +140,9 @@ func ReadHandler(typeRef Reader, db *sqlx.DB) http.HandlerFunc {
 			return
 		}
 
-		user, err := auth.GetCurrentUser(ctx)
-		if err != nil {
-			log.Errorf("unable to retrieve current user from context: %s", err)
-			handleErrs(http.StatusInternalServerError, err)
-			return
-		}
+		reader := typeFactory(inf)
 
-		results, errs, errType := typeRef.Read(db, params, *user)
+		results, errs, errType := reader.Read(params)
 		if len(errs) > 0 {
 			tc.HandleErrorsWithType(errs, errType, handleErrs)
 			return
@@ -173,10 +170,20 @@ func ReadHandler(typeRef Reader, db *sqlx.DB) http.HandlerFunc {
 //   *decoding and validating the struct
 //   *change log entry
 //   *forming and writing the body over the wire
-func UpdateHandler(typeRef Updater, db *sqlx.DB) http.HandlerFunc {
+func UpdateHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		//create error function with ResponseWriter and Request
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
+
+		inf, userErr, sysErr, errCode := NewInfo(r, nil, nil)
+		if userErr != nil || sysErr != nil {
+			HandleErr(w, r, errCode, userErr, sysErr)
+			return
+		}
+		defer inf.Close()
+
+		//decode the body and validate the request struct
+		u := typeFactory(inf)
 		//collect path parameters and user from context
 		ctx := r.Context()
 		params, err := GetCombinedParams(r)
@@ -195,13 +202,11 @@ func UpdateHandler(typeRef Updater, db *sqlx.DB) http.HandlerFunc {
 		//create local instance of the shared typeRef pointer
 		//no operations should be made on the typeRef
 		//decode the body and validate the request struct
-		decoded, errs := decodeAndValidateRequestBody(r, typeRef, db, *user)
+		errs := decodeAndValidateRequestBody(r, u)
 		if len(errs) > 0 {
 			handleErrs(http.StatusBadRequest, errs...)
 			return
 		}
-		u := decoded.(Updater)
-		//now we have a validated local object to update
 
 		keyFields := u.GetKeyFieldsInfo() //expecting a slice of the key fields info which is a struct with the field name and a function to convert a string into a {}interface of the right type. in most that will be [{Field:"id",Func: func(s string)({}interface,error){return strconv.Atoi(s)}}]
 		keys, ok := u.GetKeys()           // a map of keyField to keyValue where keyValue is an {}interface
@@ -233,7 +238,7 @@ func UpdateHandler(typeRef Updater, db *sqlx.DB) http.HandlerFunc {
 
 		// if the object has tenancy enabled, check that user is able to access the tenant
 		if t, ok := u.(Tenantable); ok {
-			authorized, err := t.IsTenantAuthorized(*user, db)
+			authorized, err := t.IsTenantAuthorized(*user)
 			if err != nil {
 				handleErrs(http.StatusBadRequest, err)
 				return
@@ -245,13 +250,18 @@ func UpdateHandler(typeRef Updater, db *sqlx.DB) http.HandlerFunc {
 		}
 
 		//run the update and handle any error
-		err, errType := u.Update(db, *user)
+		err, errType := u.Update()
 		if err != nil {
 			tc.HandleErrorsWithType([]error{err}, errType, handleErrs)
 			return
 		}
 		//auditing here
-		CreateChangeLog(ApiChange, Updated, u, *user, db)
+		err = CreateChangeLog(ApiChange, Updated, u, *inf.User, inf.Tx)
+		if err != nil {
+			HandleErr(w,r,http.StatusInternalServerError,tc.DBError,errors.New("inserting changelog: " + err.Error()))
+			return
+		}
+		*inf.CommitTx = true
 		//form response to send across the wire
 		resp := struct {
 			Response interface{} `json:"response"`
@@ -276,21 +286,22 @@ func UpdateHandler(typeRef Updater, db *sqlx.DB) http.HandlerFunc {
 //   *current user
 //   *change log entry
 //   *forming and writing the body over the wire
-func DeleteHandler(typeRef Deleter, db *sqlx.DB) http.HandlerFunc {
+func DeleteHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
 
-		d := typeRef
-
-		ctx := r.Context()
-		params, err := GetCombinedParams(r)
-		if err != nil {
-			handleErrs(http.StatusInternalServerError, err)
+		inf, userErr, sysErr, errCode := NewInfo(r, nil, nil)
+		if userErr != nil || sysErr != nil {
+			HandleErr(w, r, errCode, userErr, sysErr)
 			return
 		}
-		user, err := auth.GetCurrentUser(ctx)
+		defer inf.Close()
+
+
+		d := typeFactory(inf)
+
+		params, err := GetCombinedParams(r)
 		if err != nil {
-			log.Errorf("unable to retrieve current user from context: %s", err)
 			handleErrs(http.StatusInternalServerError, err)
 			return
 		}
@@ -316,7 +327,7 @@ func DeleteHandler(typeRef Deleter, db *sqlx.DB) http.HandlerFunc {
 
 		// if the object has tenancy enabled, check that user is able to access the tenant
 		if t, ok := d.(Tenantable); ok {
-			authorized, err := t.IsTenantAuthorized(*user, db)
+			authorized, err := t.IsTenantAuthorized(*inf.User)
 			if err != nil {
 				handleErrs(http.StatusBadRequest, err)
 				return
@@ -328,7 +339,7 @@ func DeleteHandler(typeRef Deleter, db *sqlx.DB) http.HandlerFunc {
 		}
 
 		log.Debugf("calling delete on object: %++v", d) //should have id set now
-		err, errType := d.Delete(db, *user)
+		err, errType := d.Delete()
 		if err != nil {
 			log.Errorf("error deleting: %++v", err)
 			tc.HandleErrorsWithType([]error{err}, errType, handleErrs)
@@ -336,7 +347,12 @@ func DeleteHandler(typeRef Deleter, db *sqlx.DB) http.HandlerFunc {
 		}
 		//audit here
 		log.Debugf("changelog for delete on object")
-		CreateChangeLog(ApiChange, Deleted, d, *user, db)
+		err = CreateChangeLog(ApiChange, Deleted, d, *inf.User, inf.Tx)
+		if err != nil {
+			HandleErr(w,r,http.StatusInternalServerError,tc.DBError,errors.New("inserting changelog: " + err.Error()))
+			return
+		}
+		*inf.CommitTx = true
 		//
 		resp := struct {
 			tc.Alerts
@@ -361,31 +377,32 @@ func DeleteHandler(typeRef Deleter, db *sqlx.DB) http.HandlerFunc {
 //   *decoding and validating the struct
 //   *change log entry
 //   *forming and writing the body over the wire
-func CreateHandler(typeRef Creator, db *sqlx.DB) http.HandlerFunc {
+func CreateHandler(typeConstructor func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
 
-		ctx := r.Context()
-		user, err := auth.GetCurrentUser(ctx)
-		if err != nil {
-			log.Errorf("unable to retrieve current user from context: %s", err)
-			handleErrs(http.StatusInternalServerError, err)
+		inf, userErr, sysErr, errCode := NewInfo(r, nil, nil)
+		if userErr != nil || sysErr != nil {
+			HandleErr(w, r, errCode, userErr, sysErr)
 			return
 		}
+		defer inf.Close()
 
+		i := typeConstructor(inf)
 		//decode the body and validate the request struct
-		decoded, errs := decodeAndValidateRequestBody(r, typeRef, db, *user)
+		errs := decodeAndValidateRequestBody(r, i)
+
 		if len(errs) > 0 {
 			handleErrs(http.StatusBadRequest, errs...)
 			return
 		}
-		i := decoded.(Creator)
+
 		log.Debugf("%++v", i)
 		//now we have a validated local object to insert
 
 		// if the object has tenancy enabled, check that user is able to access the tenant
 		if t, ok := i.(Tenantable); ok {
-			authorized, err := t.IsTenantAuthorized(*user, db)
+			authorized, err := t.IsTenantAuthorized(*inf.User)
 			if err != nil {
 				handleErrs(http.StatusBadRequest, err)
 				return
@@ -396,13 +413,18 @@ func CreateHandler(typeRef Creator, db *sqlx.DB) http.HandlerFunc {
 			}
 		}
 
-		err, errType := i.Create(db, *user)
+		err, errType := i.Create()
 		if err != nil {
 			tc.HandleErrorsWithType([]error{err}, errType, handleErrs)
 			return
 		}
 
-		CreateChangeLog(ApiChange, Created, i, *user, db)
+		err = CreateChangeLog(ApiChange, Created, i, *inf.User, inf.Tx)
+		if err != nil {
+			HandleErr(w,r,http.StatusInternalServerError,tc.DBError,errors.New("inserting changelog: " + err.Error()))
+			return
+		}
+		*inf.CommitTx = true
 
 		resp := struct {
 			Response interface{} `json:"response"`
diff --git a/traffic_ops/traffic_ops_golang/api/shared_interfaces.go b/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
index 3bf825f..515a1d1 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
@@ -22,43 +22,45 @@ package api
 import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
-	"github.com/jmoiron/sqlx"
 )
 
-type Updater interface {
-	Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType)
+type CRUDer interface {
+	Creator
+	Reader
+	Updater
+	Deleter
 	Identifier
 	Validator
 }
 
+type Updater interface {
+	Update() (error, tc.ApiErrorType)
+}
+
 type Identifier interface {
 	GetKeys() (map[string]interface{}, bool)
+	SetKeys(map[string]interface{})
 	GetType() string
 	GetAuditName() string
 	GetKeyFieldsInfo() []KeyFieldInfo
 }
 
 type Creator interface {
-	Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType)
-	SetKeys(map[string]interface{})
-	Identifier
-	Validator
+	Create() (error, tc.ApiErrorType)
 }
 
 type Deleter interface {
-	Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType)
-	SetKeys(map[string]interface{})
-	Identifier
+	Delete() (error, tc.ApiErrorType)
 }
 
 type Validator interface {
-	Validate(db *sqlx.DB) []error
+	Validate() []error
 }
 
 type Tenantable interface {
-	IsTenantAuthorized(user auth.CurrentUser, db *sqlx.DB) (bool, error)
+	IsTenantAuthorized(user auth.CurrentUser) (bool, error)
 }
 
 type Reader interface {
-	Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType)
+	Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType)
 }
diff --git a/traffic_ops/traffic_ops_golang/asn/asns.go b/traffic_ops/traffic_ops_golang/asn/asns.go
index d832898..9306392 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns.go
@@ -30,7 +30,6 @@ import (
 	"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"
 
 	validation "github.com/go-ozzo/ozzo-validation"
@@ -42,18 +41,16 @@ import (
 const ASNsPrivLevel = 10
 
 //we need a type alias to define functions on
-type TOASNV11 tc.ASNNullable
-
-type TOASNV12 TOASNV11
-
-func GetRefTypeV11() *TOASNV11 {
-	asn := TOASNV11(tc.ASNNullable{})
-	return &asn
+type TOASNV11 struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.ASNNullable
 }
 
-func GetRefTypeV12() *TOASNV12 {
-	asn := TOASNV12(tc.ASNNullable{})
-	return &asn
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOASNV11{reqInfo, tc.ASNNullable{}}
+		return &toReturn
+	}
 }
 
 func (asn TOASNV11) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -89,7 +86,7 @@ func (asn TOASNV11) GetType() string {
 	return "asn"
 }
 
-func (asn TOASNV11) Validate(db *sqlx.DB) []error {
+func (asn TOASNV11) Validate() []error {
 	errs := validation.Errors{
 		"asn":          validation.Validate(asn.ASN, validation.NotNil, validation.Min(0)),
 		"cachegroupId": validation.Validate(asn.CachegroupID, validation.NotNil, validation.Min(0)),
@@ -104,24 +101,8 @@ func (asn TOASNV11) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted asn and have
 //to be added to the struct
-func (asn *TOASNV11) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
-	resultRows, err := tx.NamedQuery(insertQuery(), asn)
+func (asn *TOASNV11) Create() (error, tc.ApiErrorType) {
+	resultRows, err := asn.ReqInfo.Tx.NamedQuery(insertQuery(), asn)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -157,20 +138,18 @@ func (asn *TOASNV11) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiEr
 	}
 	asn.SetKeys(map[string]interface{}{"id": id})
 	asn.LastUpdated = &lastUpdated
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
-func (asn *TOASNV12) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
-	asns, err, errType := read(db, parameters, user)
+// Read implements the /api/1.1/asns/id route for reading individual ASNs.
+// Note this does NOT correctly implement the 1.1 API for all ASNs, because that route is in a different format than the CRUD utilities and all other routes.
+// The /api/1.1/asns route MUST call V11ReadAll, not this function, to correctly implement the 1.1 API.
+func (asn *TOASNV11) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
+	asns, err, errType := read(asn.ReqInfo.Tx, parameters)
 	if len(err) > 0 {
 		return nil, err, errType
 	}
+	*asn.ReqInfo.CommitTx = true
 	iasns := make([]interface{}, len(asns), len(asns))
 	for i, readASN := range asns {
 		iasns[i] = readASN
@@ -178,41 +157,37 @@ func (asn *TOASNV12) Read(db *sqlx.DB, parameters map[string]string, user auth.C
 	return iasns, err, errType
 }
 
-// Read implements the /api/1.1/asns/id route for reading individual ASNs.
-// Note this does NOT correctly implement the 1.1 API for all ASNs, because that route is in a different format than the CRUD utilities and all other routes.
-// The /api/1.1/asns route MUST call V11ReadAll, not this function, to correctly implement the 1.1 API.
-func (asn *TOASNV11) Read(db *sqlx.DB, params map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
-	v12 := TOASNV12(*asn)
-	return v12.Read(db, params, user)
-}
 
 // V11ReadAll implements the asns 1.1 route, which is different from the 1.1 route for a single ASN and from 1.2+ routes, in that it wraps the content in an additional "asns" object.
-func V11ReadAll(db *sqlx.DB) http.HandlerFunc {
+func V11ReadAll() http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
-		ctx := r.Context()
-		params, err := api.GetCombinedParams(r)
-		if err != nil {
-			handleErrs(http.StatusInternalServerError, err)
+
+		inf, userErr, sysErr, errCode := api.NewInfo(r, nil, nil)
+		if userErr != nil || sysErr != nil {
+			api.HandleErr(w, r, errCode, userErr, sysErr)
 			return
 		}
-		user, err := auth.GetCurrentUser(ctx)
+		defer inf.Close()
+
+		params, err := api.GetCombinedParams(r)
 		if err != nil {
-			log.Errorf("unable to retrieve current user from context: %s", err)
 			handleErrs(http.StatusInternalServerError, err)
 			return
 		}
-		asns, errs, errType := read(db, params, *user)
+
+		asns, errs, errType := read(inf.Tx, params)
 		if len(errs) > 0 {
 			tc.HandleErrorsWithType(errs, errType, handleErrs)
 			return
 		}
+		*inf.CommitTx = true
 		resp := struct {
 			Response struct {
-				ASNs []TOASNV12 `json:"asns"`
+				ASNs []tc.ASNNullable `json:"asns"`
 			} `json:"response"`
 		}{Response: struct {
-			ASNs []TOASNV12 `json:"asns"`
+			ASNs []tc.ASNNullable `json:"asns"`
 		}{ASNs: asns}}
 
 		respBts, err := json.Marshal(resp)
@@ -225,7 +200,7 @@ func V11ReadAll(db *sqlx.DB) http.HandlerFunc {
 	}
 }
 
-func read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]TOASNV12, []error, tc.ApiErrorType) {
+func read(tx *sqlx.Tx, parameters map[string]string) ([]tc.ASNNullable, []error, tc.ApiErrorType) {
 	// Query Parameters to Database Query column mappings
 	// see the fields mapped in the SQL query
 	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
@@ -242,16 +217,16 @@ func read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]T
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying ASNs: %v", err)
 		return nil, []error{err}, tc.SystemError
 	}
 	defer rows.Close()
 
-	ASNs := []TOASNV12{}
+	ASNs := []tc.ASNNullable{}
 	for rows.Next() {
-		var s TOASNV12
+		var s tc.ASNNullable
 		if err = rows.StructScan(&s); err != nil {
 			log.Errorf("error parsing ASN rows: %v", err)
 			return nil, []error{err}, tc.SystemError
@@ -279,25 +254,9 @@ FROM asn a JOIN cachegroup c ON a.cachegroup = c.id`
 //ParsePQUniqueConstraintError is used to determine if a asn 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
-func (asn *TOASNV11) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
+func (asn *TOASNV11) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with asn: %++v", updateQuery(), asn)
-	resultRows, err := tx.NamedQuery(updateQuery(), asn)
+	resultRows, err := asn.ReqInfo.Tx.NamedQuery(updateQuery(), asn)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -330,36 +289,15 @@ func (asn *TOASNV11) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiEr
 			return fmt.Errorf("this update affected too many rows: %d", rowsAffected), tc.SystemError
 		}
 	}
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
+
 	return nil, tc.NoError
 }
 
 //The ASN implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (asn *TOASNV11) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
+func (asn *TOASNV11) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with asn: %++v", deleteQuery(), asn)
-	result, err := tx.NamedExec(deleteQuery(), asn)
+	result, err := asn.ReqInfo.Tx.NamedExec(deleteQuery(), asn)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -375,12 +313,7 @@ func (asn *TOASNV11) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiEr
 			return fmt.Errorf("this create affected too many rows: %d", rowsAffected), tc.SystemError
 		}
 	}
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
+
 	return nil, tc.NoError
 }
 
diff --git a/traffic_ops/traffic_ops_golang/asn/asns_test.go b/traffic_ops/traffic_ops_golang/asn/asns_test.go
index 7cdebd8..12e8de4 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns_test.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns_test.go
@@ -34,11 +34,11 @@ import (
 	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
 )
 
-func getTestASNs() []TOASNV11 {
-	ASNs := []TOASNV11{}
+func getTestASNs() []tc.ASNNullable {
+	ASNs := []tc.ASNNullable{}
 	i := 1
 	c := "Yukon"
-	testCase := TOASNV11{
+	testCase := tc.ASNNullable{
 		ASN:          &i,
 		Cachegroup:   &c,
 		CachegroupID: &i,
@@ -79,7 +79,7 @@ func TestGetASNs(t *testing.T) {
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
 	v := map[string]string{"dsId": "1"}
 
-	asns, errs, _ := GetRefTypeV11().Read(db, v, auth.CurrentUser{})
+	asns, errs, _ := GetTypeSingleton(db)().Read(v, auth.CurrentUser{})
 
 	if len(errs) > 0 {
 		t.Errorf("asn.Read expected: no errors, actual: %v", errs)
@@ -114,9 +114,9 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	i := -99
-	asn := TOASNV11{ASN: &i, CachegroupID: &i}
+	asn := TOASNV11{nil,tc.ASNNullable{ASN: &i, CachegroupID: &i}}
 
-	errs := test.SortErrors(asn.Validate(nil))
+	errs := test.SortErrors(asn.Validate())
 	expected := []error{
 		errors.New(`'asn' must be no less than 0`),
 		errors.New(`'cachegroupId' must be no less than 0`),
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
index e359edd..a43df6f 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -30,7 +30,6 @@ import (
 	"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"
 
 	validation "github.com/go-ozzo/ozzo-validation"
@@ -38,63 +37,66 @@ import (
 	"github.com/lib/pq"
 )
 
-type TOCacheGroup v13.CacheGroupNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOCacheGroup{}
+type TOCacheGroup struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.CacheGroupNullable
+	}
 
-func GetRefType() *TOCacheGroup {
-	return &refType
+func GetV11TypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
+		toReturn := TOCacheGroup{reqInfo, v13.CacheGroupNullable{}}
+		return &toReturn
+	}
 }
 
-func (cachegroup TOCacheGroup) GetKeyFieldsInfo() []api.KeyFieldInfo {
+func (cg TOCacheGroup) GetKeyFieldsInfo() []api.KeyFieldInfo {
 	return []api.KeyFieldInfo{{"id", api.GetIntKey}}
 }
 
 //Implementation of the Identifier, Validator interface functions
-func (cachegroup TOCacheGroup) GetKeys() (map[string]interface{}, bool) {
-	if cachegroup.ID == nil {
+func (cg TOCacheGroup) GetKeys() (map[string]interface{}, bool) {
+	if cg.ID == nil {
 		return map[string]interface{}{"id": 0}, false
 	}
-	return map[string]interface{}{"id": *cachegroup.ID}, true
+	return map[string]interface{}{"id": *cg.ID}, true
 }
 
-func (cachegroup *TOCacheGroup) SetKeys(keys map[string]interface{}) {
+func (cg *TOCacheGroup) SetKeys(keys map[string]interface{}) {
 	i, _ := keys["id"].(int) //this utilizes the non panicking type assertion, if the thrown away ok variable is false i will be the zero of the type, 0 here.
-	cachegroup.ID = &i
+	cg.ID = &i
 }
 
 //Implementation of the Identifier, Validator interface functions
-func (cachegroup TOCacheGroup) GetID() (int, bool) {
-	if cachegroup.ID == nil {
+func (cg TOCacheGroup) GetID() (int, bool) {
+	if cg.ID == nil {
 		return 0, false
 	}
-	return *cachegroup.ID, true
+	return *cg.ID, true
 }
 
-func (cachegroup TOCacheGroup) GetAuditName() string {
-	if cachegroup.Name != nil {
-		return *cachegroup.Name
+func (cg TOCacheGroup) GetAuditName() string {
+	if cg.Name != nil {
+		return *cg.Name
 	}
-	id, _ := cachegroup.GetID()
+	id, _ := cg.GetID()
 	return strconv.Itoa(id)
 }
 
-func (cachegroup TOCacheGroup) GetType() string {
-	return "cachegroup"
+func (cg TOCacheGroup) GetType() string {
+	return "cg"
 }
 
-func (cachegroup *TOCacheGroup) SetID(i int) {
-	cachegroup.ID = &i
+func (cg *TOCacheGroup) SetID(i int) {
+	cg.ID = &i
 }
 
 // checks if a cachegroup with the given ID is in use as a parent or secondary parent.
-func isUsedByChildCache(db *sqlx.DB, ID int) (bool, error) {
+func isUsedByChildCache(tx *sqlx.Tx, ID int) (bool, error) {
 	pQuery := "SELECT count(*) from cachegroup WHERE parent_cachegroup_id=$1"
 	sQuery := "SELECT count(*) from cachegroup WHERE secondary_parent_cachegroup_id=$1"
 	count := 0
 
-	err := db.QueryRow(pQuery, ID).Scan(&count)
+	err := tx.QueryRow(pQuery, ID).Scan(&count)
 	if err != nil {
 		log.Errorf("received error: %++v from query execution", err)
 		return false, err
@@ -103,7 +105,7 @@ func isUsedByChildCache(db *sqlx.DB, ID int) (bool, error) {
 		return true, errors.New("cache is in use as a parent cache")
 	}
 
-	err = db.QueryRow(sQuery, ID).Scan(&count)
+	err = tx.QueryRow(sQuery, ID).Scan(&count)
 	if err != nil {
 		log.Errorf("received error: %++v from query execution", err)
 		return false, err
@@ -144,18 +146,18 @@ func IsValidParentCachegroupID(id *int) bool {
 }
 
 // Validate fulfills the api.Validator interface
-func (cachegroup TOCacheGroup) Validate(db *sqlx.DB) []error {
+func (cg TOCacheGroup) Validate() []error {
 	validName := validation.NewStringRule(IsValidCacheGroupName, "invalid characters found - Use alphanumeric . or - or _ .")
 	validShortName := validation.NewStringRule(IsValidCacheGroupName, "invalid characters found - Use alphanumeric . or - or _ .")
 	latitudeErr := "Must be a floating point number within the range +-90"
 	longitudeErr := "Must be a floating point number within the range +-180"
 	errs := validation.Errors{
-		"name":                        validation.Validate(cachegroup.Name, validation.Required, validName),
-		"shortName":                   validation.Validate(cachegroup.ShortName, validation.Required, validShortName),
-		"latitude":                    validation.Validate(cachegroup.Latitude, validation.Min(-90.0).Error(latitudeErr), validation.Max(90.0).Error(latitudeErr)),
-		"longitude":                   validation.Validate(cachegroup.Longitude, validation.Min(-180.0).Error(longitudeErr), validation.Max(180.0).Error(longitudeErr)),
-		"parentCacheGroupID":          validation.Validate(cachegroup.ParentCachegroupID, validation.Min(1)),
-		"secondaryParentCachegroupID": validation.Validate(cachegroup.SecondaryParentCachegroupID, validation.Min(1)),
+		"name":                        validation.Validate(cg.Name, validation.Required, validName),
+		"shortName":                   validation.Validate(cg.ShortName, validation.Required, validShortName),
+		"latitude":                    validation.Validate(cg.Latitude, validation.Min(-90.0).Error(latitudeErr), validation.Max(90.0).Error(latitudeErr)),
+		"longitude":                   validation.Validate(cg.Longitude, validation.Min(-180.0).Error(longitudeErr), validation.Max(180.0).Error(longitudeErr)),
+		"parentCacheGroupID":          validation.Validate(cg.ParentCachegroupID, validation.Min(1)),
+		"secondaryParentCachegroupID": validation.Validate(cg.SecondaryParentCachegroupID, validation.Min(1)),
 	}
 	return tovalidate.ToErrors(errs)
 }
@@ -165,13 +167,13 @@ func (cachegroup TOCacheGroup) Validate(db *sqlx.DB) []error {
 // sucessful lookup sets the two ids on the struct.
 //
 // used by Create()
-func getParentCachegroupIDs(db *sqlx.DB, cachegroup *TOCacheGroup) error {
+func getParentCachegroupIDs(tx *sqlx.Tx, cachegroup *TOCacheGroup) error {
 	query := `SELECT id FROM cachegroup where name=$1`
 	var parentID int
 	var secondaryParentID int
 
 	if cachegroup.ParentName != nil && *cachegroup.ParentName != "" {
-		err := db.QueryRow(query, *cachegroup.ParentName).Scan(&parentID)
+		err := tx.QueryRow(query, *cachegroup.ParentName).Scan(&parentID)
 		if err != nil {
 			log.Errorf("received error: %++v from query execution", err)
 			return err
@@ -184,7 +186,7 @@ func getParentCachegroupIDs(db *sqlx.DB, cachegroup *TOCacheGroup) error {
 	}
 
 	if cachegroup.SecondaryParentName != nil && *cachegroup.SecondaryParentName != "" {
-		err := db.QueryRow(query, *cachegroup.SecondaryParentName).Scan(&secondaryParentID)
+		err := tx.QueryRow(query, *cachegroup.SecondaryParentName).Scan(&secondaryParentID)
 		if err != nil {
 			log.Errorf("received error: %++v from query execution", err)
 			return err
@@ -202,14 +204,14 @@ func getParentCachegroupIDs(db *sqlx.DB, cachegroup *TOCacheGroup) error {
 //  the names are set on the struct.
 //
 // used by Read()
-func getParentCacheGroupNames(db *sqlx.DB, cachegroup *TOCacheGroup) error {
+func getParentCacheGroupNames(tx *sqlx.Tx, cachegroup *TOCacheGroup) error {
 	query1 := `SELECT name FROM cachegroup where id=$1`
 	var primaryName string
 	var secondaryName string
 
 	// primary parent lookup
 	if cachegroup.ParentCachegroupID != nil {
-		err := db.QueryRow(query1, *cachegroup.ParentCachegroupID).Scan(&primaryName)
+		err := tx.QueryRow(query1, *cachegroup.ParentCachegroupID).Scan(&primaryName)
 		if err != nil {
 			log.Errorf("received error: %++v from query execution", err)
 			return err
@@ -219,7 +221,7 @@ func getParentCacheGroupNames(db *sqlx.DB, cachegroup *TOCacheGroup) error {
 
 	// secondary parent lookup
 	if cachegroup.SecondaryParentCachegroupID != nil {
-		err := db.QueryRow(query1, *cachegroup.SecondaryParentCachegroupID).Scan(&secondaryName)
+		err := tx.QueryRow(query1, *cachegroup.SecondaryParentCachegroupID).Scan(&secondaryName)
 		if err != nil {
 			log.Errorf("received error: %++v from query execution", err)
 			return err
@@ -237,36 +239,19 @@ func getParentCacheGroupNames(db *sqlx.DB, cachegroup *TOCacheGroup) error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted cachegroup and have
 //to be added to the struct
-func (cachegroup *TOCacheGroup) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
-
-	err = getParentCachegroupIDs(db, cachegroup)
+func (cg *TOCacheGroup) Create() (error, tc.ApiErrorType) {
+	err := getParentCachegroupIDs(cg.ReqInfo.Tx, cg)
 	if err != nil {
 		log.Error.Printf("failure looking up parent cache groups %v", err)
 		return tc.DBError, tc.SystemError
 	}
 
-	resultRows, err := tx.NamedQuery(insertQuery(), cachegroup)
+	resultRows, err := cg.ReqInfo.Tx.NamedQuery(insertQuery(), cg)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
 			if eType == tc.DataConflictError {
-				return errors.New("a cachegroup with " + err.Error()), eType
+				return errors.New("a cg with " + err.Error()), eType
 			}
 			return err, eType
 		} else {
@@ -287,35 +272,27 @@ func (cachegroup *TOCacheGroup) Create(db *sqlx.DB, user auth.CurrentUser) (erro
 		}
 	}
 	if rowsAffected == 0 {
-		err = errors.New("no cachegroup was inserted, no id was returned")
+		err = errors.New("no cg was inserted, no id was returned")
 		log.Errorln(err)
 		return tc.DBError, tc.SystemError
 	} else if rowsAffected > 1 {
-		err = errors.New("too many ids returned from cachegroup insert")
+		err = errors.New("too many ids returned from cg insert")
 		log.Errorln(err)
 		return tc.DBError, tc.SystemError
 	}
-	cachegroup.SetID(id)
-	cachegroup.LastUpdated = &lastUpdated
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
+	cg.SetID(id)
+	cg.LastUpdated = &lastUpdated
 	return nil, tc.NoError
 }
 
-func (cachegroup *TOCacheGroup) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
-	var rows *sqlx.Rows
-
+func (cg *TOCacheGroup) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	// Query Parameters to Database Query column mappings
 	// see the fields mapped in the SQL query
 	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
-		"id":        dbhelpers.WhereColumnInfo{"cachegroup.id", api.IsInt},
-		"name":      dbhelpers.WhereColumnInfo{"cachegroup.name", nil},
+		"id":        dbhelpers.WhereColumnInfo{"cg.id", api.IsInt},
+		"name":      dbhelpers.WhereColumnInfo{"cg.name", nil},
 		"shortName": dbhelpers.WhereColumnInfo{"short_name", nil},
-		"type":      dbhelpers.WhereColumnInfo{"cachegroup.type", nil},
+		"type":      dbhelpers.WhereColumnInfo{"cg.type", nil},
 	}
 	where, orderBy, queryValues, errs := dbhelpers.BuildWhereAndOrderBy(parameters, queryParamsToQueryCols)
 	if len(errs) > 0 {
@@ -325,7 +302,7 @@ func (cachegroup *TOCacheGroup) Read(db *sqlx.DB, parameters map[string]string,
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := cg.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying CacheGroup: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -339,7 +316,7 @@ func (cachegroup *TOCacheGroup) Read(db *sqlx.DB, parameters map[string]string,
 			log.Errorf("error parsing CacheGroup rows: %v", err)
 			return nil, []error{tc.DBError}, tc.SystemError
 		}
-		getParentCacheGroupNames(db, &s)
+		getParentCacheGroupNames(cg.ReqInfo.Tx, &s)
 		CacheGroups = append(CacheGroups, s)
 	}
 
@@ -351,38 +328,21 @@ func (cachegroup *TOCacheGroup) Read(db *sqlx.DB, parameters map[string]string,
 //ParsePQUniqueConstraintError is used to determine if a cachegroup 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
-func (cachegroup *TOCacheGroup) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
-
+func (cg *TOCacheGroup) Update() (error, tc.ApiErrorType) {
 	// fix up parent ids.
-	err = getParentCachegroupIDs(db, cachegroup)
+	err := getParentCachegroupIDs(cg.ReqInfo.Tx, cg)
 	if err != nil {
 		log.Error.Printf("failure looking up parent cache groups %v", err)
 		return tc.DBError, tc.SystemError
 	}
 
-	log.Debugf("about to run exec query: %s with cachegroup: %++v", updateQuery(), cachegroup)
-	resultRows, err := tx.NamedQuery(updateQuery(), cachegroup)
+	log.Debugf("about to run exec query: %s with cg: %++v", updateQuery(), cg)
+	resultRows, err := cg.ReqInfo.Tx.NamedQuery(updateQuery(), cg)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
 			if eType == tc.DataConflictError {
-				return errors.New("a cachegroup with " + err.Error()), eType
+				return errors.New("a cg with " + err.Error()), eType
 			}
 			return err, eType
 		} else {
@@ -402,44 +362,21 @@ func (cachegroup *TOCacheGroup) Update(db *sqlx.DB, user auth.CurrentUser) (erro
 		}
 	}
 	log.Debugf("lastUpdated: %++v", lastUpdated)
-	cachegroup.LastUpdated = &lastUpdated
+	cg.LastUpdated = &lastUpdated
 	if rowsAffected != 1 {
 		if rowsAffected < 1 {
-			return errors.New("no cachegroup found with this id"), tc.DataMissingError
+			return errors.New("no cg found with this id"), tc.DataMissingError
 		} else {
 			return fmt.Errorf("this update affected too many rows: %d", rowsAffected), tc.SystemError
 		}
 	}
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
 //The CacheGroup implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (cachegroup *TOCacheGroup) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
-
-	inUse, err := isUsedByChildCache(db, *cachegroup.ID)
+func (cg *TOCacheGroup) Delete() (error, tc.ApiErrorType) {
+	inUse, err := isUsedByChildCache(cg.ReqInfo.Tx, *cg.ID)
 	log.Debugf("inUse: %d, err: %v", inUse, err)
 	if inUse == false && err != nil {
 		return tc.DBError, tc.SystemError
@@ -448,8 +385,8 @@ func (cachegroup *TOCacheGroup) Delete(db *sqlx.DB, user auth.CurrentUser) (erro
 		return err, tc.DataConflictError
 	}
 
-	log.Debugf("about to run exec query: %s with cachegroup: %++v", deleteQuery(), cachegroup)
-	result, err := tx.NamedExec(deleteQuery(), cachegroup)
+	log.Debugf("about to run exec query: %s with cg: %++v", deleteQuery(), cg)
+	result, err := cg.ReqInfo.Tx.NamedExec(deleteQuery(), cg)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -460,17 +397,12 @@ func (cachegroup *TOCacheGroup) Delete(db *sqlx.DB, user auth.CurrentUser) (erro
 	}
 	if rowsAffected != 1 {
 		if rowsAffected < 1 {
-			return errors.New("no cachegroup with that id found"), tc.DataMissingError
+			return errors.New("no cg with that id found"), tc.DataMissingError
 		} else {
 			return fmt.Errorf("this create affected too many rows: %d", rowsAffected), tc.SystemError
 		}
 	}
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
+
 	return nil, tc.NoError
 }
 
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns.go b/traffic_ops/traffic_ops_golang/cdn/cdns.go
index 748bb5e..007468e 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns.go
@@ -30,23 +30,24 @@ import (
 	"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/asaskevich/govalidator"
 	validation "github.com/go-ozzo/ozzo-validation"
-	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
 
 //we need a type alias to define functions on
-type TOCDN v13.CDNNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOCDN{}
+type TOCDN struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.CDNNullable
+}
 
-func GetRefType() *TOCDN {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOCDN{reqInfo, v13.CDNNullable{}}
+		return &toReturn
+	}
 }
 
 func (cdn TOCDN) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -103,7 +104,7 @@ func IsValidCDNName(str string) bool {
 }
 
 // Validate fulfills the api.Validator interface
-func (cdn TOCDN) Validate(db *sqlx.DB) []error {
+func (cdn TOCDN) Validate() []error {
 	validName := validation.NewStringRule(IsValidCDNName, "invalid characters found - Use alphanumeric . or - .")
 	validDomainName := validation.NewStringRule(govalidator.IsDNSName, "not a valid domain name")
 	errs := validation.Errors{
@@ -120,26 +121,10 @@ func (cdn TOCDN) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted cdn and have
 //to be added to the struct
-func (cdn *TOCDN) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
+func (cdn *TOCDN) Create() (error, tc.ApiErrorType) {
 	// make sure that cdn.DomainName is lowercase
 	*cdn.DomainName = strings.ToLower(*cdn.DomainName)
-	resultRows, err := tx.NamedQuery(insertQuery(), cdn)
+	resultRows, err := cdn.ReqInfo.Tx.NamedQuery(insertQuery(), cdn)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -175,18 +160,10 @@ func (cdn *TOCDN) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiError
 	}
 	cdn.SetKeys(map[string]interface{}{"id": id})
 	cdn.LastUpdated = &lastUpdated
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
-func (cdn *TOCDN) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
-	var rows *sqlx.Rows
-
+func (cdn *TOCDN) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	// Query Parameters to Database Query column mappings
 	// see the fields mapped in the SQL query
 	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
@@ -203,7 +180,7 @@ func (cdn *TOCDN) Read(db *sqlx.DB, parameters map[string]string, user auth.Curr
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := cdn.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying CDNs: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -212,12 +189,12 @@ func (cdn *TOCDN) Read(db *sqlx.DB, parameters map[string]string, user auth.Curr
 
 	CDNs := []interface{}{}
 	for rows.Next() {
-		var s TOCDN
-		if err = rows.StructScan(&s); err != nil {
+		var cdn v13.CDNNullable
+		if err = rows.StructScan(&cdn); err != nil {
 			log.Errorf("error parsing CDN rows: %v", err)
 			return nil, []error{tc.DBError}, tc.SystemError
 		}
-		CDNs = append(CDNs, s)
+		CDNs = append(CDNs, cdn)
 	}
 
 	return CDNs, []error{}, tc.NoError
@@ -228,27 +205,11 @@ func (cdn *TOCDN) Read(db *sqlx.DB, parameters map[string]string, user auth.Curr
 //ParsePQUniqueConstraintError is used to determine if a cdn 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
-func (cdn *TOCDN) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
+func (cdn *TOCDN) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with cdn: %++v", updateQuery(), cdn)
 	// make sure that cdn.DomainName is lowercase
 	*cdn.DomainName = strings.ToLower(*cdn.DomainName)
-	resultRows, err := tx.NamedQuery(updateQuery(), cdn)
+	resultRows, err := cdn.ReqInfo.Tx.NamedQuery(updateQuery(), cdn)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -281,36 +242,14 @@ func (cdn *TOCDN) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiError
 			return fmt.Errorf("this update affected too many rows: %d", rowsAffected), tc.SystemError
 		}
 	}
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
 //The CDN implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (cdn *TOCDN) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
+func (cdn *TOCDN) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with cdn: %++v", deleteQuery(), cdn)
-	result, err := tx.NamedExec(deleteQuery(), cdn)
+	result, err := cdn.ReqInfo.Tx.NamedExec(deleteQuery(), cdn)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -326,12 +265,6 @@ func (cdn *TOCDN) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiError
 			return fmt.Errorf("this create affected too many rows: %d", rowsAffected), tc.SystemError
 		}
 	}
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
diff --git a/traffic_ops/traffic_ops_golang/crconfig/deliveryservice.go b/traffic_ops/traffic_ops_golang/crconfig/deliveryservice.go
index 5ec1e66..8a949ef 100644
--- a/traffic_ops/traffic_ops_golang/crconfig/deliveryservice.go
+++ b/traffic_ops/traffic_ops_golang/crconfig/deliveryservice.go
@@ -141,7 +141,7 @@ and d.active = true
 		if err := rows.Scan(&xmlID, &missLat, &missLon, &protocol, &ds.TTL, &ds.RoutingName, &geoProvider, &ttype, &geoLimit, &geoLimitCountries, &geoLimitRedirectURL, &dispersion, &geoBlocking, &trRespHdrsStr, &maxDNSAnswers, &profile, &dnsBypassIP, &dnsBypassIP6, &dnsBypassTTL, &dnsBypassCName, &httpBypassFQDN, &ip6RoutingEnabled, &deepCachingType, &trRequestHeaders, &trResponseHeaders, &anonymousBlocking); err != nil {
 			return nil, errors.New("scanning deliveryservice: " + err.Error())
 		}
-		// TODO prevent (lat XOR lon) in the DB and UI
+		// TODO prevent (lat XOR lon) in the Tx and UI
 		if missLat.Valid && missLon.Valid {
 			ds.MissLocation = &tc.CRConfigLatitudeLongitudeShort{Lat: missLat.Float64, Lon: missLon.Float64}
 		} else if missLat.Valid {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index 596df13..09c4f3c 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -25,21 +25,29 @@ import (
 	"fmt"
 	"net/http"
 
+	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/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/jmoiron/sqlx"
+	"github.com/lib/pq"
 )
 
-type TODeliveryServiceV12 tc.DeliveryServiceNullableV12
 
-func GetRefTypeV12(cfg config.Config, db *sqlx.DB) *TODeliveryServiceV12 {
-	return &TODeliveryServiceV12{}
+type TODeliveryServiceV12 struct {
+	ReqInfo *api.APIInfo
+	tc.DeliveryServiceNullableV12
+}
+
+
+func GetTypeV12Factory() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer{
+		toReturn := TODeliveryServiceV12{reqInfo, tc.DeliveryServiceNullableV12{}}
+		return &toReturn
+	}
 }
 
 func (ds TODeliveryServiceV12) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -96,11 +104,12 @@ func getDSTenantIDByName(tx *sql.Tx, 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(tx *sql.Tx) (string, bool, error) {
+
+func (ds *TODeliveryServiceV12) GetXMLID(tx *sqlx.Tx) (string, bool, error) {
 	if ds.ID == nil {
 		return "", false, errors.New("missing ID")
 	}
-	return GetXMLID(tx, *ds.ID)
+	return GetXMLID(tx.Tx, *ds.ID)
 }
 
 // GetXMLID loads the DeliveryService's xml_id from the database, from the ID. Returns whether the delivery service was found, and any error.
@@ -116,15 +125,12 @@ func GetXMLID(tx *sql.Tx, id int) (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) {
-	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, (*tc.DeliveryServiceNullableV12)(ds))
+
+func (ds *TODeliveryServiceV12) IsTenantAuthorized(user auth.CurrentUser, tx *sqlx.Tx) (bool, error) {
+	return isTenantAuthorized(user, tx, &ds.DeliveryServiceNullableV12)
 }
 
+
 // 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 {
@@ -138,8 +144,8 @@ func getTenantID(tx *sql.Tx, ds *tc.DeliveryServiceNullableV12) (*int, error) {
 	return existingID, err
 }
 
-func isTenantAuthorized(user *auth.CurrentUser, tx *sql.Tx, ds *tc.DeliveryServiceNullableV12) (bool, error) {
-	existingID, err := getTenantID(tx, ds)
+func isTenantAuthorized(user auth.CurrentUser, tx *sqlx.Tx, ds *tc.DeliveryServiceNullableV12) (bool, error) {
+	existingID, err := getTenantID(tx.Tx, ds)
 	if err != nil {
 		return false, errors.New("getting tenant ID: " + err.Error())
 	}
@@ -147,7 +153,7 @@ func isTenantAuthorized(user *auth.CurrentUser, tx *sql.Tx, ds *tc.DeliveryServi
 		ds.TenantID = existingID
 	}
 	if existingID != nil && existingID != ds.TenantID {
-		userAuthorizedForExistingDSTenant, err := tenant.IsResourceAuthorizedToUserTx(*existingID, user, tx)
+		userAuthorizedForExistingDSTenant, err := tenant.IsResourceAuthorizedToUserTx(*existingID, user, tx.Tx)
 		if err != nil {
 			return false, errors.New("checking authorization for existing DS ID: " + err.Error())
 		}
@@ -156,7 +162,7 @@ func isTenantAuthorized(user *auth.CurrentUser, tx *sql.Tx, ds *tc.DeliveryServi
 		}
 	}
 	if ds.TenantID != nil {
-		userAuthorizedForNewDSTenant, err := tenant.IsResourceAuthorizedToUserTx(*ds.TenantID, user, tx)
+		userAuthorizedForNewDSTenant, err := tenant.IsResourceAuthorizedToUserTx(*ds.TenantID, user, tx.Tx)
 		if err != nil {
 			return false, errors.New("checking authorization for new DS ID: " + err.Error())
 		}
@@ -167,70 +173,136 @@ func isTenantAuthorized(user *auth.CurrentUser, tx *sql.Tx, ds *tc.DeliveryServi
 	return true, nil
 }
 
-func (ds *TODeliveryServiceV12) Validate(db *sqlx.DB) []error {
-	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{(*tc.DeliveryServiceNullableV12)(ds).Validate(tx)}
+
+func (ds *TODeliveryServiceV12) Validate() []error {
+	return ds.DeliveryServiceNullableV12.Validate(ds.ReqInfo.Tx.Tx)
 }
 
-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
+// unimplemented, needed to satisfy CRUDer, since the framework doesn't allow a create to return an array of one
+func(ds *TODeliveryServiceV12) Create() (error, tc.ApiErrorType) {
+	return errors.New("The Create method is not implemented"), http.StatusNotImplemented
+}
+
+func CreateV12() http.HandlerFunc {
+	return func(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, &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})
 	}
-	*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) {
+	func(ds *TODeliveryServiceV12) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	returnable := []interface{}{}
-	dses, errs, errType := readGetDeliveryServices(params, db, user)
+	dses, errs, errType := readGetDeliveryServices(params, ds.ReqInfo.Tx, *ds.ReqInfo.User)
 	if len(errs) > 0 {
-		for _, err := range errs {
-			if err.Error() == `id cannot parse to integer` {
-				return nil, []error{errors.New("Resource not found.")}, tc.DataMissingError //matches perl response
-			}
-		}
-		return nil, errs, errType
+	for _, err := range errs {
+	if err.Error() == `id cannot parse to integer` {
+	return nil, []error{errors.New("Resource not found.")}, tc.DataMissingError //matches perl response
+	}
+	}
+	return nil, errs, errType
 	}
 
 	for _, ds := range dses {
-		returnable = append(returnable, ds.DeliveryServiceNullableV12)
+	returnable = append(returnable, ds.DeliveryServiceNullableV12)
 	}
 	return returnable, nil, tc.NoError
-}
+	}
+
+	//The DeliveryService implementation of the Deleter interface
+	//all implementations of Deleter should use transactions and return the proper errorType
+	func(ds *TODeliveryServiceV12) Delete() (error, tc.ApiErrorType){
+		log.Debugln("TODeliveryServiceV12.Delete calling id '%v' xmlid '%v'\n", ds.ID, ds.XMLID)
+		// return nil, tc.NoError // debug
+
+		if ds.ID == nil {
+			log.Errorln("TODeliveryServiceV12.Delete called with nil ID")
+			return tc.DBError, tc.DataMissingError
+		}
+		xmlID, ok, err := ds.GetXMLID(ds.ReqInfo.Tx)
+		if err != nil {
+			log.Errorln("TODeliveryServiceV12.Delete ID '" + string(*ds.ID) + "' loading XML ID: " + err.Error())
+			return tc.DBError, tc.SystemError
+		}
+		if !ok {
+			log.Errorln("TODeliveryServiceV12.Delete ID '" + string(*ds.ID) + "' had no delivery service!")
+			return tc.DBError, tc.DataMissingError
+		}
+		ds.XMLID = &xmlID
+
+		// Note ds regexes MUST be deleted before the ds, because there's a ON DELETE CASCADE on deliveryservice_regex (but not on regex).
+		// Likewise, it MUST happen in a transaction with the later DS delete, so they aren't deleted if the DS delete fails.
+		if _, err := ds.ReqInfo.Tx.Exec(`DELETE FROM regex WHERE id IN (SELECT regex FROM deliveryservice_regex WHERE deliveryservice=$1)`, *ds.ID); err != nil {
+			log.Errorln("TODeliveryServiceV12.Delete deleting regexes for delivery service: " + err.Error())
+			return tc.DBError, tc.SystemError
+		}
+
+		if _, err := ds.ReqInfo.Tx.Exec(`DELETE FROM deliveryservice_regex WHERE deliveryservice=$1`, *ds.ID); err != nil {
+			log.Errorln("TODeliveryServiceV12.Delete deleting delivery service regexes: " + err.Error())
+			return tc.DBError, tc.SystemError
+		}
+
+		result, err := ds.ReqInfo.Tx.Exec(`DELETE FROM deliveryservice WHERE id=$1`, *ds.ID)
+		if err != nil {
+			log.Errorln("TODeliveryServiceV12.Delete deleting delivery service: " + err.Error())
+			return tc.DBError, tc.SystemError
+		}
+		rowsAffected, err := result.RowsAffected()
+		if err != nil {
+			return tc.DBError, tc.SystemError
+		}
+		if rowsAffected != 1 {
+			if rowsAffected < 1 {
+				return errors.New("no delivery service with that id found"), tc.DataMissingError
+			}
+			return fmt.Errorf("this create affected too many rows: %d", rowsAffected), tc.SystemError
+		}
+
+		paramConfigFilePrefixes := []string{"hdr_rw_", "hdr_rw_mid_", "regex_remap_", "cacheurl_"}
+		configFiles := []string{}
+		for _, prefix := range paramConfigFilePrefixes {
+			configFiles = append(configFiles, prefix + *ds.XMLID+".config")
+		}
+
+		if _, err := ds.ReqInfo.Tx.Exec(`DELETE FROM parameter WHERE name = 'location' AND config_file = ANY($1)`, pq.Array(configFiles)); err != nil {
+			log.Errorln("TODeliveryServiceV12.Delete deleting delivery service parameters: " + err.Error())
+			return tc.DBError, tc.SystemError
+		}
+
+		return nil, tc.NoError
+	}
+
 
-func (ds *TODeliveryServiceV12) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	v13 := (*TODeliveryServiceV13)(&tc.DeliveryServiceNullableV13{DeliveryServiceNullableV12: *(*tc.DeliveryServiceNullableV12)(ds)})
-	err, errType := v13.Delete(db, user)
-	*ds = (TODeliveryServiceV12)(v13.DeliveryServiceNullableV12) // TODO avoid copy
-	return err, errType
+// unimplemented, needed to satisfy CRUDer, since the framework doesn't allow an update to return an array of one
+func(ds *TODeliveryServiceV12) Update() (error, tc.ApiErrorType) {
+	return errors.New("The Update method is not implemented"), http.StatusNotImplemented
 }
 
-func UpdateV12(w http.ResponseWriter, r *http.Request) {
+func UpdateV12() http.HandlerFunc {
+	return func(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)
@@ -245,14 +317,14 @@ func UpdateV12(w http.ResponseWriter, r *http.Request) {
 		return
 	}
 	dsv13 := tc.NewDeliveryServiceNullableV13FromV12(ds)
-	if authorized, err := isTenantAuthorized(inf.User, inf.Tx.Tx, &ds); err != nil {
+	if authorized, err := isTenantAuthorized(*inf.User, inf.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)
+	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
@@ -260,3 +332,4 @@ func UpdateV12(w http.ResponseWriter, r *http.Request) {
 	*inf.CommitTx = true
 	api.WriteResp(w, r, []tc.DeliveryServiceNullableV12{dsv13.DeliveryServiceNullableV12})
 }
+}
\ No newline at end of file
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index 0d7dc36..12d7312 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -22,6 +22,7 @@ package deliveryservice
 import (
 	"database/sql"
 	"errors"
+	"encoding/json"
 	"fmt"
 	"net/http"
 	"strconv"
@@ -42,16 +43,28 @@ import (
 )
 
 //we need a type alias to define functions on
-type TODeliveryServiceV13 tc.DeliveryServiceNullableV13
+
+type TODeliveryServiceV13 struct {
+	ReqInfo *api.APIInfo
+	tc.DeliveryServiceNullableV13
+}
 
 func (ds *TODeliveryServiceV13) V12() *TODeliveryServiceV12 {
-	v13 := (*tc.DeliveryServiceNullableV13)(ds)
-	v12 := &v13.DeliveryServiceNullableV12
-	return (*TODeliveryServiceV12)(v12)
+	return &TODeliveryServiceV12{DeliveryServiceNullableV12: ds.DeliveryServiceNullableV12}
+}
+
+func (ds TODeliveryServiceV13) MarshalJSON() ([]byte, error) {
+	return json.Marshal(ds.DeliveryServiceNullableV13)
+}
+func (ds *TODeliveryServiceV13) UnmarshalJSON(data []byte) error {
+	return json.Unmarshal(data, ds.DeliveryServiceNullableV13)
 }
 
-func GetRefTypeV13(cfg config.Config, db *sqlx.DB) *TODeliveryServiceV13 {
-	return &TODeliveryServiceV13{}
+func GetTypeV13Factory() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer{
+		toReturn := TODeliveryServiceV13{reqInfo, tc.DeliveryServiceNullableV13{}}
+		return &toReturn
+	}
 }
 
 func (ds TODeliveryServiceV13) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -76,54 +89,78 @@ func (ds *TODeliveryServiceV13) GetType() string {
 	return ds.V12().GetType()
 }
 
-func (ds *TODeliveryServiceV13) Validate(db *sqlx.DB) []error {
-	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{(*tc.DeliveryServiceNullableV13)(ds).Validate(tx)}
+func (ds *TODeliveryServiceV13) Validate() []error {
+	return ds.DeliveryServiceNullableV13.Validate(ds.ReqInfo.Tx.Tx)
 }
 
-// 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})
+// unimplemented, needed to satisfy CRUDer, since the framework doesn't allow a create to return an array of one
+func(ds *TODeliveryServiceV13) Create() (error, tc.ApiErrorType) {
+	return errors.New("The Create method is not implemented"), http.StatusNotImplemented
+}
+
+// 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.Tx, 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() http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		defer r.Body.Close()
+		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 := json.NewDecoder(r.Body).Decode(&ds); err != nil {
+			api.HandleErr(w, r, http.StatusBadRequest, errors.New("malformed JSON: "+err.Error()), nil)
+			return
+		}
+
+		if ds.RoutingName == nil || *ds.RoutingName == "" {
+			ds.RoutingName = util.StrPtr("cdn")
+		}
+
+		if errs := ds.Validate(inf.Tx.Tx); len(errs) > 0 {
+			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+util.JoinErrs(errs).Error()), nil)
+			return
+		}
+
+		if authorized, err := isTenantAuthorized(*inf.User, inf.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(tx *sql.Tx, 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().
 	}
 
-	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 [...]
+	resultRows, err := tx.Query(insertQuery(), &ds.Active, &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.IPV6RoutingEnabled, &ds.LogsEnabled, &ds.LongD [...]
 
 	if err != nil {
 		if pqerr, ok := err.(*pq.Error); ok {
@@ -194,7 +231,7 @@ func create(tx *sql.Tx, cfg *config.Config, user *auth.CurrentUser, ds tc.Delive
 	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())
 	}
 
@@ -203,13 +240,15 @@ func create(tx *sql.Tx, cfg *config.Config, user *auth.CurrentUser, ds tc.Delive
 	}
 
 	ds.LastUpdated = &lastUpdated
-	api.CreateChangeLogRawTx(api.ApiChange, "Created ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), user, tx)
+	if err := api.CreateChangeLogRawErr(api.ApiChange, "Created ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), *user, tx); err!= nil {
+		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("error writing to audit log: " + err.Error())
+	}
 	return ds, http.StatusOK, nil, nil
 }
 
-func (ds *TODeliveryServiceV13) Read(db *sqlx.DB, params map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (ds *TODeliveryServiceV13) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	returnable := []interface{}{}
-	dses, errs, errType := readGetDeliveryServices(params, db, user)
+	dses, errs, errType := readGetDeliveryServices(params, ds.ReqInfo.Tx, *ds.ReqInfo.User)
 	if len(errs) > 0 {
 		for _, err := range errs {
 			if err.Error() == `id cannot parse to integer` { // TODO create const for string
@@ -342,6 +381,54 @@ func getTypeFromID(id int, tx *sql.Tx) (tc.DSType, error) {
 	return tc.DSTypeFromString(name), nil
 }
 
+// unimplemented, needed to satisfy CRUDer, since the framework doesn't allow an update to return an array of one
+func(ds *TODeliveryServiceV13) Update() (error, tc.ApiErrorType) {
+	return errors.New("The Update method is not implemented"), http.StatusNotImplemented
+}
+
+func UpdateV13() http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		defer r.Body.Close()
+		inf, userErr, sysErr, errCode := api.NewInfo(r, nil, []string{"id"})
+		if userErr != nil || sysErr != nil {
+			api.HandleErr(w, r, errCode, userErr, sysErr)
+			return
+		}
+		defer inf.Close()
+
+		id := inf.IntParams["id"]
+
+		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 := ds.Validate(inf.Tx.Tx); len(errs) > 0 {
+			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+util.JoinErrs(errs).Error()), nil)
+			return
+		}
+
+		if authorized, err := isTenantAuthorized(*inf.User, inf.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
+		}
+
+		*inf.CommitTx = true
+		api.WriteResp(w, r, []tc.DeliveryServiceNullableV13{ds})
+	}
+}
+
 func getDSType(tx *sql.Tx, xmlid string) (tc.DSType, bool, error) {
 	name := ""
 	if err := tx.QueryRow(`SELECT name FROM type WHERE id = (select type from deliveryservice where xml_id = $1)`, xmlid).Scan(&name); err != nil {
@@ -353,36 +440,7 @@ func getDSType(tx *sql.Tx, xmlid string) (tc.DSType, bool, error) {
 	return tc.DSTypeFromString(name), true, nil
 }
 
-func UpdateV13(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.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
-	}
-	*inf.CommitTx = true
-	api.WriteResp(w, r, []tc.DeliveryServiceNullableV13{ds})
-}
-
-func update(tx *sql.Tx, cfg *config.Config, user *auth.CurrentUser, ds *tc.DeliveryServiceNullableV13) (tc.DeliveryServiceNullableV13, int, error, error) {
+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
 	}
@@ -482,6 +540,7 @@ func update(tx *sql.Tx, cfg *config.Config, user *auth.CurrentUser, ds *tc.Deliv
 		ds.MatchList = &ml
 	}
 
+
 	if newDSType.HasSSLKeys() && oldHostName != newHostName {
 		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())
@@ -501,97 +560,50 @@ func update(tx *sql.Tx, cfg *config.Config, user *auth.CurrentUser, ds *tc.Deliv
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("creating mid cacheurl parameters: " + err.Error())
 	}
 
-	if err := updatePrimaryOrigin(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
-	api.CreateChangeLogRawTx(api.ApiChange, "Updated ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), user, tx)
+
+	if err := api.CreateChangeLogRawErr(api.ApiChange, "Updated ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), user, tx); err != nil {
+		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("writing change log entry: " + err.Error())
+	}
 	return *ds, http.StatusOK, nil, nil
 }
 
 //The DeliveryService implementation of the Deleter interface
 //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)
-
-	tx, err := db.Begin()
-	if err != nil {
-		log.Errorln("could not begin transaction: " + err.Error())
-		return tc.DBError, tc.SystemError
-	}
-	commitTx := false
-	defer dbhelpers.FinishTx(tx, &commitTx)
-
-	if err != nil {
-		log.Errorln("could not begin transaction: " + err.Error())
-		return tc.DBError, tc.SystemError
-	}
-
-	if ds.ID == nil {
-		log.Errorln("TODeliveryServiceV13.Delete called with nil ID")
-		return tc.DBError, tc.DataMissingError
-	}
-	xmlID, ok, err := ds.V12().GetXMLID(tx)
-	if err != nil {
-		log.Errorln("TODeliveryServiceV13.Delete ID '" + strconv.Itoa(*ds.ID) + "' loading XML ID: " + err.Error())
-		return tc.DBError, tc.SystemError
-	}
-	if !ok {
-		log.Errorln("TODeliveryServiceV13.Delete ID '" + strconv.Itoa(*ds.ID) + "' had no delivery service!")
-		return tc.DBError, tc.DataMissingError
-	}
-	ds.XMLID = &xmlID
-
-	// Note ds regexes MUST be deleted before the ds, because there's a ON DELETE CASCADE on deliveryservice_regex (but not on regex).
-	// Likewise, it MUST happen in a transaction with the later DS delete, so they aren't deleted if the DS delete fails.
-	if _, err := tx.Exec(`DELETE FROM regex WHERE id IN (SELECT regex FROM deliveryservice_regex WHERE deliveryservice=$1)`, *ds.ID); err != nil {
-		log.Errorln("TODeliveryServiceV13.Delete deleting regexes for delivery service: " + err.Error())
-		return tc.DBError, tc.SystemError
-	}
+func (ds *TODeliveryServiceV13) Delete() (error, tc.ApiErrorType) {
+	return GetTypeV12Factory()(ds.ReqInfo).Delete()
+}
 
-	if _, err := tx.Exec(`DELETE FROM deliveryservice_regex WHERE deliveryservice=$1`, *ds.ID); err != nil {
-		log.Errorln("TODeliveryServiceV13.Delete deleting delivery service regexes: " + err.Error())
-		return tc.DBError, tc.SystemError
-	}
+// IsTenantAuthorized implements the Tenantable interface to ensure the user is authorized on the deliveryservice tenant
+func (ds *TODeliveryServiceV13) IsTenantAuthorized(user auth.CurrentUser, tx *sqlx.Tx) (bool, error) {
+	return ds.V12().IsTenantAuthorized(user, tx)
+}
 
-	result, err := tx.Exec(`DELETE FROM deliveryservice WHERE id=$1`, *ds.ID)
-	if err != nil {
-		log.Errorln("TODeliveryServiceV13.Delete deleting delivery service: " + err.Error())
-		return tc.DBError, tc.SystemError
-	}
-	rowsAffected, err := result.RowsAffected()
-	if err != nil {
-		return tc.DBError, tc.SystemError
-	}
-	if rowsAffected != 1 {
-		if rowsAffected < 1 {
-			return errors.New("no delivery service with that id found"), tc.DataMissingError
+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())
+			}
 		}
-		return fmt.Errorf("this create affected too many rows: %d", rowsAffected), tc.SystemError
-	}
-
-	paramConfigFilePrefixes := []string{"hdr_rw_", "hdr_rw_mid_", "regex_remap_", "cacheurl_"}
-	configFiles := []string{}
-	for _, prefix := range paramConfigFilePrefixes {
-		configFiles = append(configFiles, prefix+*ds.XMLID+".config")
-	}
-
-	if _, err := tx.Exec(`DELETE FROM parameter WHERE name = 'location' AND config_file = ANY($1)`, pq.Array(configFiles)); err != nil {
-		log.Errorln("TODeliveryServiceV13.Delete deleting delivery service parameters: " + err.Error())
-		return tc.DBError, tc.SystemError
+		if !ok {
+			continue
+		}
+		newDSes = append(newDSes, ds)
 	}
-
-	commitTx = true
-	return nil, tc.NoError
+	return newDSes, nil
 }
 
-// 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 readGetDeliveryServices(params map[string]string, db *sqlx.DB, user auth.CurrentUser) ([]tc.DeliveryServiceNullableV13, []error, tc.ApiErrorType) {
+func readGetDeliveryServices(params map[string]string, tx *sqlx.Tx, user auth.CurrentUser) ([]tc.DeliveryServiceNullableV13, []error, tc.ApiErrorType) {
 	if strings.HasSuffix(params["id"], ".json") {
 		params["id"] = params["id"][:len(params["id"])-len(".json")]
 	}
@@ -618,9 +630,9 @@ func readGetDeliveryServices(params map[string]string, db *sqlx.DB, user auth.Cu
 		return nil, errs, tc.DataConflictError
 	}
 
-	if tenant.IsTenancyEnabled(db) {
+	if tenant.IsTenancyEnabledTx(tx) {
 		log.Debugln("Tenancy is enabled")
-		tenantIDs, err := tenant.GetUserTenantIDList(user, db)
+		tenantIDs, err := tenant.GetUserTenantIDListTx(user, tx)
 		if err != nil {
 			log.Errorln("received error querying for user's tenants: " + err.Error())
 			return nil, []error{tc.DBError}, tc.SystemError
@@ -632,14 +644,6 @@ func readGetDeliveryServices(params map[string]string, db *sqlx.DB, user auth.Cu
 	log.Debugln("generated deliveryServices query: " + query)
 	log.Debugf("executing with values: %++v\n", queryValues)
 
-	tx, err := db.Beginx()
-	if err != nil {
-		log.Errorln("could not begin transaction: " + err.Error())
-		return nil, []error{tc.DBError}, tc.SystemError
-	}
-	commitTx := false
-	defer dbhelpers.FinishTxX(tx, &commitTx)
-
 	rows, err := tx.NamedQuery(query, queryValues)
 	if err != nil {
 		return nil, []error{fmt.Errorf("querying: %v", err)}, tc.SystemError
@@ -682,15 +686,15 @@ func readGetDeliveryServices(params map[string]string, db *sqlx.DB, user auth.Cu
 		dses[i] = ds
 	}
 
-	commitTx = true
 	return dses, nil, tc.NoError
 }
 
-func updateSSLKeys(ds *tc.DeliveryServiceNullableV13, hostName string, tx *sql.Tx, 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", tx, cfg.RiakAuthOptions)
+	key, ok, err := riaksvc.GetDeliveryServiceSSLKeysObjTx(*ds.XMLID, "latest", tx, cfg.RiakAuthOptions)
 	if err != nil {
 		return errors.New("getting SSL key: " + err.Error())
 	}
@@ -699,7 +703,7 @@ func updateSSLKeys(ds *tc.DeliveryServiceNullableV13, hostName string, tx *sql.T
 	}
 	key.DeliveryService = *ds.XMLID
 	key.Hostname = hostName
-	if err := riaksvc.PutDeliveryServiceSSLKeysObj(key, tx, cfg.RiakAuthOptions); err != nil {
+	if err := riaksvc.PutDeliveryServiceSSLKeysObjTx(key, tx, cfg.RiakAuthOptions); err != nil {
 		return errors.New("putting updated SSL key: " + err.Error())
 	}
 	return 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 e9ff56a..5824af1 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
@@ -37,13 +37,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TODeliveryServiceRequestComment tc.DeliveryServiceRequestCommentNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TODeliveryServiceRequestComment{}
+type TODeliveryServiceRequestComment struct{
+	DB *sqlx.DB `json:"-"`
+	tc.DeliveryServiceRequestCommentNullable
+}
 
-func GetRefType() *TODeliveryServiceRequestComment {
-	return &refType
+func GetTypeSingleton(db *sqlx.DB) func()api.CRUDer {
+	return func()api.CRUDer {
+		toReturn := TODeliveryServiceRequestComment{db, tc.DeliveryServiceRequestCommentNullable{}}
+		return &toReturn
+	}
 }
 
 func (comment TODeliveryServiceRequestComment) GetKeyFieldsInfo() []api.KeyFieldInfo {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index ce87e99..753127d 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -26,24 +26,26 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
-
 	"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/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
 
-// TODeliveryServiceRequest provides a type alias to define functions on
-type TODeliveryServiceRequest tc.DeliveryServiceRequestNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TODeliveryServiceRequest(tc.DeliveryServiceRequestNullable{})
+//we need a type alias to define functions on
+type TODeliveryServiceRequest struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.DeliveryServiceRequestNullable
+}
 
-// GetRefType is used to decode the JSON for deliveryservice requests
-func GetRefType() *TODeliveryServiceRequest {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}
+		return &toReturn
+	}
 }
 
 func (req TODeliveryServiceRequest) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -74,7 +76,7 @@ func (req TODeliveryServiceRequest) GetType() string {
 }
 
 // Read implements the api.Reader interface
-func (req *TODeliveryServiceRequest) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (req *TODeliveryServiceRequest) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
 		"assignee":   dbhelpers.WhereColumnInfo{Column: "s.username"},
 		"assigneeId": dbhelpers.WhereColumnInfo{Column: "r.assignee_id", Checker: api.IsInt},
@@ -104,7 +106,7 @@ func (req *TODeliveryServiceRequest) Read(db *sqlx.DB, parameters map[string]str
 	query := selectDeliveryServiceRequestsQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := req.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequests: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -120,7 +122,7 @@ func (req *TODeliveryServiceRequest) Read(db *sqlx.DB, parameters map[string]str
 		}
 
 		// TODO: combine tenancy with the query above so there's a single db call
-		t, err := s.IsTenantAuthorized(user, db)
+		t, err := s.IsTenantAuthorized(*req.ReqInfo.User)
 		if err != nil {
 			log.Errorf("error checking tenancy: %v", err)
 			return nil, []error{tc.DBError}, tc.SystemError
@@ -159,7 +161,8 @@ LEFT OUTER JOIN tm_user e ON r.last_edited_by_id = e.id
 }
 
 // IsTenantAuthorized implements the Tenantable interface to ensure the user is authorized on the deliveryservice tenant
-func (req TODeliveryServiceRequest) IsTenantAuthorized(user auth.CurrentUser, db *sqlx.DB) (bool, error) {
+func (req TODeliveryServiceRequest) IsTenantAuthorized(user auth.CurrentUser) (bool, error) {
+
 	ds := req.DeliveryService
 	if ds == nil {
 		// No deliveryservice applied yet -- wide open
@@ -169,7 +172,7 @@ func (req TODeliveryServiceRequest) IsTenantAuthorized(user auth.CurrentUser, db
 		log.Debugf("tenantID is nil")
 		return false, errors.New("tenantID is nil")
 	}
-	return tenant.IsResourceAuthorizedToUser(*ds.TenantID, user, db)
+	return tenant.IsResourceAuthorizedToUserTx(*ds.TenantID, user, req.ReqInfo.Tx.Tx)
 }
 
 // Update implements the tc.Updater interface.
@@ -177,13 +180,13 @@ func (req TODeliveryServiceRequest) IsTenantAuthorized(user auth.CurrentUser, db
 //ParsePQUniqueConstraintError is used to determine if a request 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
-func (req *TODeliveryServiceRequest) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (req *TODeliveryServiceRequest) Update() (error, tc.ApiErrorType) {
 	var current TODeliveryServiceRequest
 	if req.ID == nil {
 		log.Errorf("error updating DeliveryServiceRequest: ID is nil")
 		return errors.New("error updating DeliveryServiceRequest: ID is nil"), tc.DataMissingError
 	}
-	err := db.QueryRowx(selectDeliveryServiceRequestsQuery() + `WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(&current)
+	err := req.ReqInfo.Tx.QueryRowx(selectDeliveryServiceRequestsQuery() + `WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(&current)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequests: %v", err)
 		return err, tc.SystemError
@@ -205,26 +208,9 @@ func (req *TODeliveryServiceRequest) Update(db *sqlx.DB, user auth.CurrentUser)
 			tc.DataConflictError
 	}
 
-	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.Error.Println("could not begin transaction: ", err.Error())
-		return err, tc.SystemError
-	}
-
-	userID := tc.IDNoMod(user.ID)
+	userID := tc.IDNoMod(req.ReqInfo.User.ID)
 	req.LastEditedByID = &userID
-	resultRows, err := tx.NamedQuery(updateRequestQuery(), req)
+	resultRows, err := req.ReqInfo.Tx.NamedQuery(updateRequestQuery(), req)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -257,12 +243,6 @@ func (req *TODeliveryServiceRequest) Update(db *sqlx.DB, user auth.CurrentUser)
 		return fmt.Errorf("this update affected too many rows: %d", rowsAffected), tc.SystemError
 	}
 
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
@@ -273,7 +253,7 @@ func (req *TODeliveryServiceRequest) Update(db *sqlx.DB, user auth.CurrentUser)
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted request and have
 //to be added to the struct
-func (req *TODeliveryServiceRequest) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (req *TODeliveryServiceRequest) Create() (error, tc.ApiErrorType) {
 	if req == nil {
 		return errors.New("nil deliveryservice_request"), tc.SystemError
 	}
@@ -296,7 +276,7 @@ func (req *TODeliveryServiceRequest) Create(db *sqlx.DB, user auth.CurrentUser)
 		return errors.New("no xmlId associated with this request"), tc.DataMissingError
 	}
 	XMLID := *ds.XMLID
-	active, err := isActiveRequest(db, XMLID)
+	active, err := isActiveRequest(req.ReqInfo.Tx, XMLID)
 	if err != nil {
 		return err, tc.SystemError
 	}
@@ -304,27 +284,10 @@ func (req *TODeliveryServiceRequest) Create(db *sqlx.DB, user auth.CurrentUser)
 		return errors.New(`An active request exists for delivery service '` + XMLID + `'`), tc.DataConflictError
 	}
 
-	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.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
-
-	userID := tc.IDNoMod(user.ID)
+	userID := tc.IDNoMod(req.ReqInfo.User.ID)
 	req.AuthorID = &userID
 	req.LastEditedByID = &userID
-	resultRows, err := tx.NamedQuery(insertRequestQuery(), req)
+	resultRows, err := req.ReqInfo.Tx.NamedQuery(insertRequestQuery(), req)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -359,12 +322,7 @@ func (req *TODeliveryServiceRequest) Create(db *sqlx.DB, user auth.CurrentUser)
 	}
 	req.SetKeys(map[string]interface{}{"id": id})
 	req.LastUpdated = &lastUpdated
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
+
 	return nil, tc.NoError
 }
 
@@ -372,14 +330,14 @@ func (req *TODeliveryServiceRequest) Create(db *sqlx.DB, user auth.CurrentUser)
 //all implementations of Deleter should use transactions and return the proper errorType
 
 // Delete removes the request from the db
-func (req *TODeliveryServiceRequest) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (req *TODeliveryServiceRequest) Delete() (error, tc.ApiErrorType) {
 	var st tc.RequestStatus
 	log.Debugln("DELETING REQUEST WITH ID ", strconv.Itoa(*req.ID))
 	if req.ID == nil {
 		return errors.New("cannot delete deliveryservice_request -- ID is nil"), tc.SystemError
 	}
 
-	err := db.QueryRow(`SELECT status FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)).Scan(&st)
+	err := req.ReqInfo.Tx.QueryRow(`SELECT status FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)).Scan(&st)
 	if err != nil {
 		return err, tc.SystemError
 	}
@@ -388,26 +346,10 @@ func (req *TODeliveryServiceRequest) Delete(db *sqlx.DB, user auth.CurrentUser)
 		return fmt.Errorf("cannot delete a deliveryservice_request with state %s", string(st)), tc.DataConflictError
 	}
 
-	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.Error.Println("could not begin transaction: ", err.Error())
-		return tc.DBError, tc.SystemError
-	}
 	query := `DELETE FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)
 	log.Debugf("about to run exec query: %s", query)
 
-	result, err := tx.Exec(query)
+	result, err := req.ReqInfo.Tx.Exec(query)
 	if err != nil {
 		log.Errorln("received error from delete execution: ", err.Error())
 		return tc.DBError, tc.SystemError
@@ -425,13 +367,8 @@ func (req *TODeliveryServiceRequest) Delete(db *sqlx.DB, user auth.CurrentUser)
 		log.Errorln("the delete affected too many rows")
 		return fmt.Errorf("this delete affected too many rows: %d", rowsAffected), tc.SystemError
 	}
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
 	// success!
-	rollbackTransaction = false
+
 	return nil, tc.NoError
 }
 
@@ -458,11 +395,11 @@ func (req TODeliveryServiceRequest) ChangeLogMessage(action string) (string, err
 }
 
 // isActiveRequest returns true if a request using this XMLID is currently in an active state
-func isActiveRequest(db *sqlx.DB, XMLID string) (bool, error) {
+func isActiveRequest(tx *sqlx.Tx, XMLID string) (bool, error) {
 	q := `SELECT EXISTS(SELECT 1 FROM deliveryservice_request
 WHERE deliveryservice->>'xmlId' = '` + XMLID + `'
 AND status IN ('draft', 'submitted', 'pending'))`
-	row := db.QueryRow(q)
+	row := tx.QueryRow(q)
 	var active bool
 	err := row.Scan(&active)
 	if err != nil {
@@ -509,17 +446,21 @@ WHERE id=:id`
 
 ////////////////////////////////////////////////////////////////
 // Assignment change
-type deliveryServiceRequestAssignment struct {
-	TODeliveryServiceRequest
+
+func GetAssignmentTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := deliveryServiceRequestAssignment{TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}}
+		return &toReturn
+	}
 }
 
-// GetAssignRefType is used to decode the JSON for deliveryservice_request assignment
-func GetAssignRefType() *deliveryServiceRequestAssignment {
-	return &deliveryServiceRequestAssignment{}
+
+type deliveryServiceRequestAssignment struct {
+	TODeliveryServiceRequest
 }
 
 // Update assignee only
-func (req *deliveryServiceRequestAssignment) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (req *deliveryServiceRequestAssignment) Update() (error, tc.ApiErrorType) {
 	// req represents the state the deliveryservice_request is to transition to
 	// we want to limit what changes here -- only assignee can change
 	if req.ID == nil {
@@ -528,7 +469,7 @@ func (req *deliveryServiceRequestAssignment) Update(db *sqlx.DB, user auth.Curre
 
 	// get original
 	var current TODeliveryServiceRequest
-	err := db.QueryRowx(selectDeliveryServiceRequestsQuery() + `WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(&current)
+	err := req.ReqInfo.Tx.QueryRowx(selectDeliveryServiceRequestsQuery() + `WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(&current)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequests: %v", err)
 		return err, tc.SystemError
@@ -545,30 +486,13 @@ func (req *deliveryServiceRequestAssignment) Update(db *sqlx.DB, user auth.Curre
 	*req = deliveryServiceRequestAssignment{current}
 	req.AssigneeID = assigneeID
 
-	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.Error.Println("could not begin transaction: ", err.Error())
-		return err, tc.SystemError
-	}
-
 	// LastEditedBy field should not change with status update
 	v := "null"
 	if req.AssigneeID != nil {
 		v = strconv.Itoa(*req.AssigneeID)
 	}
 	query := `UPDATE deliveryservice_request SET assignee_id = ` + v + ` WHERE id=` + strconv.Itoa(*req.ID)
-	_, err = tx.Exec(query)
+	_, err = req.ReqInfo.Tx.Exec(query)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -581,24 +505,17 @@ func (req *deliveryServiceRequestAssignment) Update(db *sqlx.DB, user auth.Curre
 		return tc.DBError, tc.SystemError
 	}
 
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-
 	// update req with current info
-	err = db.QueryRowx(selectDeliveryServiceRequestsQuery() + ` WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(req)
+	err = req.ReqInfo.Tx.QueryRowx(selectDeliveryServiceRequestsQuery() + ` WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(req)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequests: %v", err)
 		return err, tc.SystemError
 	}
 
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
-func (req deliveryServiceRequestAssignment) Validate(db *sqlx.DB) []error {
+func (req deliveryServiceRequestAssignment) Validate() []error {
 	return nil
 }
 
@@ -621,13 +538,14 @@ type deliveryServiceRequestStatus struct {
 	TODeliveryServiceRequest
 }
 
-// GetStatusRefType is used to decode the JSON for deliveryservice_request status change
-func GetStatusRefType() *deliveryServiceRequestStatus {
-	return &deliveryServiceRequestStatus{}
+func GetStatusTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := deliveryServiceRequestStatus{TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}}
+		return &toReturn
+	}
 }
-
 // Update status only
-func (req *deliveryServiceRequestStatus) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (req *deliveryServiceRequestStatus) Update() (error, tc.ApiErrorType) {
 	// req represents the state the deliveryservice_request is to transition to
 	// we want to limit what changes here -- only status can change,  and only according to the established rules
 	// for status transition
@@ -637,7 +555,7 @@ func (req *deliveryServiceRequestStatus) Update(db *sqlx.DB, user auth.CurrentUs
 	if req.ID == nil {
 		log.Errorf("error updating DeliveryServiceRequestStatus: ID is nil")
 	}
-	err := db.QueryRowx(selectDeliveryServiceRequestsQuery() + ` WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(&current)
+	err := req.ReqInfo.Tx.QueryRowx(selectDeliveryServiceRequestsQuery() + ` WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(&current)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequests: %v", err)
 		return err, tc.SystemError
@@ -652,26 +570,9 @@ func (req *deliveryServiceRequestStatus) Update(db *sqlx.DB, user auth.CurrentUs
 	*req = deliveryServiceRequestStatus{current}
 	req.Status = st
 
-	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.Error.Println("could not begin transaction: ", err.Error())
-		return err, tc.SystemError
-	}
-
 	// LastEditedBy field should not change with status update
 	query := `UPDATE deliveryservice_request SET status = '` + string(*req.Status) + `' WHERE id=` + strconv.Itoa(*req.ID)
-	_, err = tx.Exec(query)
+	_, err = req.ReqInfo.Tx.Exec(query)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -684,25 +585,18 @@ func (req *deliveryServiceRequestStatus) Update(db *sqlx.DB, user auth.CurrentUs
 		return tc.DBError, tc.SystemError
 	}
 
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-
 	// update req with current info
-	err = db.QueryRowx(selectDeliveryServiceRequestsQuery() + ` WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(req)
+	err = req.ReqInfo.Tx.QueryRowx(selectDeliveryServiceRequestsQuery() + ` WHERE r.id=` + strconv.Itoa(*req.ID)).StructScan(req)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequests: %v", err)
 		return err, tc.SystemError
 	}
 
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
 // Validate is not needed when only Status is updated
-func (req deliveryServiceRequestStatus) Validate(db *sqlx.DB) []error {
+func (req deliveryServiceRequestStatus) Validate() []error {
 	return nil
 }
 
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go
index 6df0153..b4aa80e 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go
@@ -137,13 +137,13 @@ func TestGetDeliveryServiceRequest(t *testing.T) {
 	}
 
 	/*
-		if r.Update(db *sqlx.DB, ctx context.Context) {
+		if r.Update(db *sqlx.Tx, ctx context.Context) {
 			t.Errorf("expected ID to be %d,  not %d", 10, r.GetID())
 		}
-		if r.Insert(db *sqlx.DB, ctx context.Context) {
+		if r.Insert(db *sqlx.Tx, ctx context.Context) {
 			t.Errorf("expected ID to be %d,  not %d", 10, r.GetID())
 		}
-		if r.Delete(db *sqlx.DB, ctx context.Context) {
+		if r.Delete(db *sqlx.Tx, ctx context.Context) {
 			t.Errorf("expected ID to be %d,  not %d", 10, r.GetID())
 		}
 	*/
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
index c9cd298..977a996 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
@@ -26,24 +26,16 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"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"
 )
 
 // 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
-	defer dbhelpers.FinishTx(tx, &commitTx)
-
+func (req *TODeliveryServiceRequest) Validate() []error {
 	fromStatus := tc.RequestStatusDraft
 	if req.ID != nil && *req.ID > 0 {
-		err := tx.QueryRow(`SELECT status FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)).Scan(&fromStatus)
+		err := req.ReqInfo.Tx.QueryRow(`SELECT status FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)).Scan(&fromStatus)
+
 		if err != nil {
 			return []error{err}
 		}
@@ -67,9 +59,10 @@ func (req *TODeliveryServiceRequest) Validate(db *sqlx.DB) []error {
 	}
 	errs := tovalidate.ToErrors(errMap)
 	// ensure the deliveryservice requested is valid
-	if err := req.DeliveryService.Validate(tx); err != nil {
-		errs = append(errs, err)
-	}
-	commitTx = true
+	e := req.DeliveryService.Validate(req.ReqInfo.Tx.Tx)
+
+	errs = append(errs, e...)
+
+
 	return errs
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go b/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
index ead75c3..20e6485 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
@@ -557,7 +557,7 @@ func Delete(dbx *sqlx.DB) http.HandlerFunc {
 		}
 
 		log.Debugf("changelog for delete on object")
-		api.CreateChangeLogMsg(api.ApiChange, *user, dbx, fmt.Sprintf(`deleted deliveryservice_regex {"ds": %d, "regex": %d}`, dsID, regexID))
+		api.CreateChangeLogRaw(api.ApiChange,fmt.Sprintf(`deleted deliveryservice_regex {"ds": %d, "regex": %d}`, dsID, regexID), *user, dbx.DB)
 		resp := struct {
 			tc.Alerts
 		}{tc.CreateAlerts(tc.SuccessLevel, "deliveryservice_regex was deleted.")}
diff --git a/traffic_ops/traffic_ops_golang/origin/origins.go b/traffic_ops/traffic_ops_golang/origin/origins.go
index b988433..a9b9ee7 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins.go
@@ -107,7 +107,7 @@ func (origin *TOOrigin) Validate(db *sqlx.DB) []error {
 	return tovalidate.ToErrors(validateErrs)
 }
 
-// GetTenantID returns a pointer to the Origin's tenant ID from the DB and any error encountered
+// GetTenantID returns a pointer to the Origin's tenant ID from the Tx and any error encountered
 func (origin *TOOrigin) GetTenantID(db *sqlx.DB) (*int, error) {
 	if origin.ID != nil {
 		var tenantID *int
diff --git a/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go b/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
index f6d8e3b..8140292 100644
--- a/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
+++ b/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
@@ -63,7 +63,59 @@ func GetDeliveryServiceSSLKeysObj(xmlID string, version string, tx *sql.Tx, auth
 	return key, found, nil
 }
 
-func PutDeliveryServiceSSLKeysObj(key tc.DeliveryServiceSSLKeys, tx *sql.Tx, authOpts *riak.AuthOptions) error {
+
+func GetDeliveryServiceSSLKeysObjTx(xmlID string, version string, tx *sql.Tx, authOpts *riak.AuthOptions) (tc.DeliveryServiceSSLKeys, bool, error) {
+	key := tc.DeliveryServiceSSLKeys{}
+	if version == "" {
+		xmlID += "-latest"
+	} else {
+		xmlID += "-" + version
+	}
+	found := false
+	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 {
+			return err
+		}
+		if len(ro) == 0 {
+			return nil // not found
+		}
+		if err := json.Unmarshal(ro[0].Value, &key); err != nil {
+			log.Errorf("failed at unmarshaling sslkey response: %s\n", err)
+			return errors.New("unmarshalling Riak result: " + err.Error())
+		}
+		found = true
+		return nil
+	})
+	if err != nil {
+		return key, false, err
+	}
+	return key, found, nil
+}
+
+	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 = WithClusterTx(tx, authOpts, func(cluster StorageCluster) error {
+		obj := &riak.Object{
+			ContentType:     "text/json",
+			Charset:         "utf-8",
+			ContentEncoding: "utf-8",
+			Key:             key.DeliveryService,
+			Value:           []byte(keyJSON),
+		}
+		if err = SaveObject(obj, DeliveryServiceSSLKeysBucket, cluster); err != nil {
+			return errors.New("saving Riak object: " + err.Error())
+		}
+		return nil
+	})
+	return err
+}
+
+func PutDeliveryServiceSSLKeysObjTx(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())
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 5650f9d..67fe91c 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -36,26 +36,26 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/cachegroup"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/cdn"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/coordinate"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/coordinate"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/crconfig"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice"
 	dsrequest "github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/request"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/request/comment"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/request/comment"
 	dsserver "github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/servers"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservicesregexes"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/division"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/division"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/hwinfo"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/origin"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/parameter"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/physlocation"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/origin"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/parameter"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/physlocation"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/ping"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/profile"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/profile"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/profileparameter"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/region"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/role"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/region"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/role"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/server"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/staticdnsentry"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/status"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/staticdnsentry"
+	//"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/status"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/systeminfo"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/types"
@@ -84,20 +84,20 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		// 1.3 routes exist only in a Go. There is NO equivalent Perl route. They should conform with the API guidelines (https://cwiki.apache.org/confluence/display/TC/API+Guidelines).
 
 		//ASN: CRUD
-		{1.2, http.MethodGet, `asns/?(\.json)?$`, api.ReadHandler(asn.GetRefTypeV12(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `asns/?(\.json)?$`, asn.V11ReadAll(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `asns/{id}$`, api.ReadHandler(asn.GetRefTypeV11(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `asns/{id}$`, api.UpdateHandler(asn.GetRefTypeV11(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `asns/?$`, api.CreateHandler(asn.GetRefTypeV11(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `asns/{id}$`, api.DeleteHandler(asn.GetRefTypeV11(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.2, http.MethodGet, `asns/?(\.json)?$`, api.ReadHandler(asn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `asns/?(\.json)?$`, asn.V11ReadAll(), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `asns/{id}$`, api.ReadHandler(asn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `asns/{id}$`, api.UpdateHandler(asn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `asns/?$`, api.CreateHandler(asn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `asns/{id}$`, api.DeleteHandler(asn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		//CacheGroup: CRUD
 		{1.1, http.MethodGet, `cachegroups/trimmed/?(\.json)?$`, cachegroup.TrimmedHandler(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `cachegroups/?(\.json)?$`, api.ReadHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `cachegroups/{id}$`, api.ReadHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `cachegroups/{id}$`, api.UpdateHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `cachegroups/?$`, api.CreateHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `cachegroups/{id}$`, api.DeleteHandler(cachegroup.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodGet, `cachegroups/?(\.json)?$`, api.ReadHandler(cachegroup.GetV11TypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `cachegroups/{id}$`, api.ReadHandler(cachegroup.GetV11TypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `cachegroups/{id}$`, api.UpdateHandler(cachegroup.GetV11TypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `cachegroups/?$`, api.CreateHandler(cachegroup.GetV11TypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `cachegroups/{id}$`, api.DeleteHandler(cachegroup.GetV11TypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		{1.1, http.MethodPost, `cachegroups/{id}/queue_update$`, cachegroup.QueueUpdates(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
 
@@ -110,25 +110,27 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.1, http.MethodGet, `cdns/routing$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
 
 		//CDN: CRUD
-		{1.1, http.MethodGet, `cdns/?(\.json)?$`, api.ReadHandler(cdn.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `cdns/{id}$`, api.ReadHandler(cdn.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `cdns/name/{name}/?(\.json)?$`, api.ReadHandler(cdn.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `cdns/{id}$`, api.UpdateHandler(cdn.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `cdns/?$`, api.CreateHandler(cdn.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `cdns/{id}$`, api.DeleteHandler(cdn.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodGet, `cdns/?(\.json)?$`, api.ReadHandler(cdn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `cdns/{id}$`, api.ReadHandler(cdn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `cdns/name/{name}/?(\.json)?$`, api.ReadHandler(cdn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `cdns/{id}$`, api.UpdateHandler(cdn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `cdns/?$`, api.CreateHandler(cdn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `cdns/{id}$`, api.DeleteHandler(cdn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 		{1.1, http.MethodDelete, `cdns/name/{name}$`, cdn.DeleteName(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
+
+		//CDN: queue updates
 		{1.1, http.MethodPost, `cdns/{id}/queue_update$`, cdn.Queue(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//CDN: Monitoring: Traffic Monitor
 		{1.1, http.MethodGet, `cdns/{name}/configs/monitoring(\.json)?$`, monitoringHandler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//Division: CRUD
-		{1.1, http.MethodGet, `divisions/?(\.json)?$`, api.ReadHandler(division.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `divisions/{id}$`, api.ReadHandler(division.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `divisions/{id}$`, api.UpdateHandler(division.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `divisions/?$`, api.CreateHandler(division.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `divisions/{id}$`, api.DeleteHandler(division.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodGet, `divisions/name/{name}/?(\.json)?$`, api.ReadHandler(division.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `divisions/?(\.json)?$`, api.ReadHandler(division.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `divisions/{id}$`, api.ReadHandler(division.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `divisions/{id}$`, api.UpdateHandler(division.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `divisions/?$`, api.CreateHandler(division.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `divisions/{id}$`, api.DeleteHandler(division.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `divisions/name/{name}/?(\.json)?$`, api.ReadHandler(division.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//HWInfo
 		{1.1, http.MethodGet, `hwinfo-wip/?(\.json)?$`, hwinfo.HWInfoHandler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
@@ -137,48 +139,49 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.2, http.MethodPost, `user/login/?$`, wrapAccessLog(d.Secrets[0], auth.LoginHandler(d.DB, d.Config)), 0, NoAuth, nil}, {1.3, http.MethodPost, `user/login/?$`, auth.LoginHandler(d.DB, d.Config), 0, NoAuth, nil},
 
 		//Parameter: CRUD
-		{1.1, http.MethodGet, `parameters/?(\.json)?$`, api.ReadHandler(parameter.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `parameters/{id}$`, api.ReadHandler(parameter.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `parameters/{id}$`, api.UpdateHandler(parameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `parameters/?$`, api.CreateHandler(parameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `parameters/{id}$`, api.DeleteHandler(parameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `parameters/?(\.json)?$`, api.ReadHandler(parameter.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `parameters/{id}$`, api.ReadHandler(parameter.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `parameters/{id}$`, api.UpdateHandler(parameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `parameters/?$`, api.CreateHandler(parameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `parameters/{id}$`, api.DeleteHandler(parameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//Phys_Location: CRUD
-		{1.1, http.MethodGet, `phys_locations/?(\.json)?$`, api.ReadHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `phys_locations/trimmed/?(\.json)?$`, physlocation.GetTrimmed(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `phys_locations/{id}$`, api.ReadHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `phys_locations/{id}$`, api.UpdateHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `phys_locations/?$`, api.CreateHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `phys_locations/{id}$`, api.DeleteHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `phys_locations/?(\.json)?$`, api.ReadHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `phys_locations/trimmed/?(\.json)?$`, physlocation.GetTrimmed(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `phys_locations/{id}$`, api.ReadHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `phys_locations/{id}$`, api.UpdateHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `phys_locations/?$`, api.CreateHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `phys_locations/{id}$`, api.DeleteHandler(physlocation.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//Ping
 		{1.1, http.MethodGet, `ping$`, ping.PingHandler(), 0, NoAuth, nil},
 
 		//Profile: CRUD
-		{1.1, http.MethodGet, `profiles/?(\.json)?$`, api.ReadHandler(profile.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `profiles/trimmed/?(\.json)?$`, profile.Trimmed(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `profiles/{id}$`, api.ReadHandler(profile.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `profiles/{id}$`, api.UpdateHandler(profile.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `profiles/?$`, api.CreateHandler(profile.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `profiles/{id}$`, api.DeleteHandler(profile.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `profiles/?(\.json)?$`, api.ReadHandler(profile.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `profiles/trimmed/?(\.json)?$`, profile.Trimmed(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `profiles/{id}$`, api.ReadHandler(profile.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `profiles/{id}$`, api.UpdateHandler(profile.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `profiles/?$`, api.CreateHandler(profile.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `profiles/{id}$`, api.DeleteHandler(profile.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//Region: CRUD
-		{1.1, http.MethodGet, `regions/?(\.json)?$`, api.ReadHandler(region.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `regions/{id}$`, api.ReadHandler(region.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `regions/{id}$`, api.UpdateHandler(region.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `regions/?$`, api.CreateHandler(region.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `regions/{id}$`, api.DeleteHandler(region.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `regions/?(\.json)?$`, api.ReadHandler(region.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `regions/{id}$`, api.ReadHandler(region.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `regions/{id}$`, api.UpdateHandler(region.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `regions/?$`, api.CreateHandler(region.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `regions/{id}$`, api.DeleteHandler(region.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		{1.1, http.MethodDelete, `deliveryservice_server/{dsid}/{serverid}`, dsserver.Delete, auth.PrivLevelReadOnly, Authenticated, nil},
 
 		// get all edge servers associated with a delivery service (from deliveryservice_server table)
-		{1.1, http.MethodGet, `deliveryserviceserver$`, dsserver.ReadDSSHandler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPost, `deliveryserviceserver$`, dsserver.GetReplaceHandler(d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `deliveryservices/{xml_id}/servers$`, dsserver.GetCreateHandler(d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodGet, `servers/{id}/deliveryservices$`, api.ReadHandler(dsserver.GetDServiceRef(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `deliveryservices/{id}/servers$`, dsserver.GetReadHandler(d.DB, tc.Assigned), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `deliveryservices/{id}/unassigned_servers$`, dsserver.GetReadHandler(d.DB, tc.Unassigned), auth.PrivLevelReadOnly, Authenticated, nil},
-		//{1.1, http.MethodGet, `deliveryservices/{id}/servers/eligible$`, dsserver.GetReadHandler(d.DB, tc.Eligible),auth.PrivLevelReadOnly, Authenticated, nil},
+
+		{1.1, http.MethodGet, `deliveryserviceserver$`, dsserver.ReadDSSHandler(d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPost,`deliveryserviceserver$`, dsserver.GetReplaceHandler(d.DB),auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost,`deliveryservices/{xml_id}/servers$`, dsserver.GetCreateHandler( d.DB ) ,auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `servers/{id}/deliveryservices$`, api.ReadHandler(dsserver.GetDServiceRef(), d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `deliveryservices/{id}/servers$`, dsserver.GetReadHandler(d.DB, tc.Assigned),auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `deliveryservices/{id}/unassigned_servers$`, dsserver.GetReadHandler(d.DB, tc.Unassigned),auth.PrivLevelReadOnly, Authenticated, nil},
+		////{1.1, http.MethodGet, `deliveryservices/{id}/servers/eligible$`, dsserver.GetReadHandler(d.Tx, tc.Eligible),auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//Server
 		{1.1, http.MethodGet, `servers/checks$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
@@ -190,53 +193,78 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.2, http.MethodGet, `servers/hostname/{hostName}/details/?(\.json)?$`, server.GetDetailHandler(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//Server: CRUD
-		{1.1, http.MethodGet, `servers/?(\.json)?$`, api.ReadHandler(server.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `servers/{id}$`, api.ReadHandler(server.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `servers/{id}$`, api.UpdateHandler(server.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `servers/?$`, api.CreateHandler(server.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `servers/{id}$`, api.DeleteHandler(server.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `servers/?(\.json)?$`, api.ReadHandler(server.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `servers/{id}$`, api.ReadHandler(server.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `servers/{id}$`, api.UpdateHandler(server.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `servers/?$`, api.CreateHandler(server.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `servers/{id}$`, api.DeleteHandler(server.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//Status: CRUD
-		{1.1, http.MethodGet, `statuses/?(\.json)?$`, api.ReadHandler(status.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `statuses/{id}$`, api.ReadHandler(status.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `statuses/{id}$`, api.UpdateHandler(status.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `statuses/?$`, api.CreateHandler(status.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `statuses/{id}$`, api.DeleteHandler(status.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `statuses/?(\.json)?$`, api.ReadHandler(status.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `statuses/{id}$`, api.ReadHandler(status.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `statuses/{id}$`, api.UpdateHandler(status.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `statuses/?$`, api.CreateHandler(status.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `statuses/{id}$`, api.DeleteHandler(status.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//System
 		{1.1, http.MethodGet, `system/info/?(\.json)?$`, systeminfo.Handler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//Type: CRUD
-		{1.1, http.MethodGet, `types/?(\.json)?$`, api.ReadHandler(types.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodGet, `types/{id}$`, api.ReadHandler(types.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPut, `types/{id}$`, api.UpdateHandler(types.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `types/?$`, api.CreateHandler(types.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `types/{id}$`, api.DeleteHandler(types.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodGet, `types/?(\.json)?$`, api.ReadHandler(types.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `types/{id}$`, api.ReadHandler(types.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodPut, `types/{id}$`, api.UpdateHandler(types.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodPost, `types/?$`, api.CreateHandler(types.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.1, http.MethodDelete, `types/{id}$`, api.DeleteHandler(types.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//About
 		{1.3, http.MethodGet, `about/?(\.json)?$`, about.Handler(), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//Coordinates
-		{1.3, http.MethodGet, `coordinates/?(\.json)?$`, api.ReadHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.3, http.MethodPut, `coordinates/?$`, api.UpdateHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.3, http.MethodPost, `coordinates/?$`, api.CreateHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.3, http.MethodDelete, `coordinates/?$`, api.DeleteHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.3, http.MethodGet, `coordinates/?(\.json)?$`, api.ReadHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.3, http.MethodGet, `coordinates/?$`, api.ReadHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.3, http.MethodPut, `coordinates/?$`, api.UpdateHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.3, http.MethodPost, `coordinates/?$`, api.CreateHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.3, http.MethodDelete, `coordinates/?$`, api.DeleteHandler(coordinate.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
-		//Delivery service request: CRUD
-		{1.3, http.MethodGet, `deliveryservice_requests/?(\.json)?$`, api.ReadHandler(dsrequest.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.3, http.MethodPut, `deliveryservice_requests/?$`, api.UpdateHandler(dsrequest.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
-		{1.3, http.MethodPost, `deliveryservice_requests/?$`, api.CreateHandler(dsrequest.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
-		{1.3, http.MethodDelete, `deliveryservice_requests/?$`, api.DeleteHandler(dsrequest.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
+		//Servers
+		// explicitly passed to legacy system until fully implemented.  Auth handled by legacy system.
+		{1.2, http.MethodGet, `servers/checks$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
+		{1.2, http.MethodGet, `servers/details$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
+		{1.2, http.MethodGet, `servers/status$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
+		{1.2, http.MethodGet, `servers/totals$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
+
+		//Monitoring
+		{1.2, http.MethodGet, `cdns/{name}/configs/monitoring(\.json)?$`, monitoringHandler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+
+		//ASNs
+		{1.3, http.MethodGet, `asns/?(\.json)?$`, api.ReadHandler(asn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `asns/?$`, api.UpdateHandler(asn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodPost, `asns/?$`, api.CreateHandler(asn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodDelete, `asns/?$`, api.DeleteHandler(asn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+
+		//CDN generic handlers:
+		{1.3, http.MethodGet, `cdns/?(\.json)?$`, api.ReadHandler(cdn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodGet, `cdns/{id}$`, api.ReadHandler(cdn.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `cdns/{id}$`, api.UpdateHandler(cdn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodPost, `cdns/?$`, api.CreateHandler(cdn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodDelete, `cdns/{id}$`, api.DeleteHandler(cdn.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+
+		//Delivery service requests
+		{1.3, http.MethodGet, `deliveryservice_requests/?(\.json)?$`, api.ReadHandler(dsrequest.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodGet, `deliveryservice_requests/{id}$`, api.ReadHandler(dsrequest.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `deliveryservice_requests/{id}$`, api.UpdateHandler(dsrequest.GetTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
+		{1.3, http.MethodPost, `deliveryservice_requests/?$`, api.CreateHandler(dsrequest.GetTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
+		{1.3, http.MethodDelete, `deliveryservice_requests/{id}$`, api.DeleteHandler(dsrequest.GetTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
 
 		//Delivery service request: Actions
-		{1.3, http.MethodPut, `deliveryservice_requests/{id}/assign$`, api.UpdateHandler(dsrequest.GetAssignRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.3, http.MethodPut, `deliveryservice_requests/{id}/status$`, api.UpdateHandler(dsrequest.GetStatusRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
+		{1.3, http.MethodPut, `deliveryservice_requests/{id}/assign$`, api.UpdateHandler(dsrequest.GetAssignmentTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodPut, `deliveryservice_requests/{id}/status$`, api.UpdateHandler(dsrequest.GetStatusTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
 
 		//Delivery service request comment: CRUD
-		{1.3, http.MethodGet, `deliveryservice_request_comments/?(\.json)?$`, api.ReadHandler(comment.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.3, http.MethodPut, `deliveryservice_request_comments/?$`, api.UpdateHandler(comment.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
-		{1.3, http.MethodPost, `deliveryservice_request_comments/?$`, api.CreateHandler(comment.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
-		{1.3, http.MethodDelete, `deliveryservice_request_comments/?$`, api.DeleteHandler(comment.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
+		//{1.3, http.MethodGet, `deliveryservice_request_comments/?(\.json)?$`, api.ReadHandler(comment.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.3, http.MethodPut, `deliveryservice_request_comments/?$`, api.UpdateHandler(comment.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
+		//{1.3, http.MethodPost, `deliveryservice_request_comments/?$`, api.CreateHandler(comment.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
+		//{1.3, http.MethodDelete, `deliveryservice_request_comments/?$`, api.DeleteHandler(comment.GetRefType(), d.DB), auth.PrivLevelPortal, Authenticated, nil},
 
 		//Delivery service uri signing keys: CRUD
 		{1.3, http.MethodGet, `deliveryservices/{xmlID}/urisignkeys$`, getURIsignkeysHandler, auth.PrivLevelAdmin, Authenticated, nil},
@@ -245,16 +273,17 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.3, http.MethodDelete, `deliveryservices/{xmlID}/urisignkeys$`, removeDeliveryServiceURIKeysHandler, auth.PrivLevelAdmin, Authenticated, nil},
 
 		//Origins
-		{1.3, http.MethodGet, `origins/?(\.json)?$`, api.ReadHandler(origin.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.3, http.MethodPut, `origins/?$`, api.UpdateHandler(origin.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.3, http.MethodPost, `origins/?$`, api.CreateHandler(origin.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.3, http.MethodDelete, `origins/?$`, api.DeleteHandler(origin.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.3, http.MethodGet, `origins/?(\.json)?$`, api.ReadHandler(origin.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.3, http.MethodGet, `origins/?$`, api.ReadHandler(origin.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.3, http.MethodPut, `origins/?$`, api.UpdateHandler(origin.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.3, http.MethodPost, `origins/?$`, api.CreateHandler(origin.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		//{1.3, http.MethodDelete, `origins/?$`, api.DeleteHandler(origin.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
 		//Roles
-		{1.3, http.MethodGet, `roles/?(\.json)?$`, api.ReadHandler(role.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.3, http.MethodPut, `roles/?$`, api.UpdateHandler(role.GetRefType(), d.DB), auth.PrivLevelAdmin, Authenticated, nil},
-		{1.3, http.MethodPost, `roles/?$`, api.CreateHandler(role.GetRefType(), d.DB), auth.PrivLevelAdmin, Authenticated, nil},
-		{1.3, http.MethodDelete, `roles/?$`, api.DeleteHandler(role.GetRefType(), d.DB), auth.PrivLevelAdmin, Authenticated, nil},
+		//{1.3, http.MethodGet, `roles/?(\.json)?$`, api.ReadHandler(role.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.3, http.MethodPut, `roles/?$`, api.UpdateHandler(role.GetRefType(), d.DB), auth.PrivLevelAdmin, Authenticated, nil},
+		//{1.3, http.MethodPost, `roles/?$`, api.CreateHandler(role.GetRefType(), d.DB), auth.PrivLevelAdmin, Authenticated, nil},
+		//{1.3, http.MethodDelete, `roles/?$`, api.DeleteHandler(role.GetRefType(), d.DB), auth.PrivLevelAdmin, Authenticated, nil},
 
 		//Delivery Services Regexes
 		{1.1, http.MethodGet, `deliveryservices_regexes/?(\.json)?$`, deliveryservicesregexes.Get(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
@@ -268,7 +297,7 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.3, http.MethodPost, `servers/{id}/deliveryservices$`, server.AssignDeliveryServicesToServerHandler(d.DB), auth.PrivLevelOperations, Authenticated, nil},
 		{1.3, http.MethodGet, `servers/{host_name}/update_status$`, server.GetServerUpdateStatusHandler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 
-		{1.1, http.MethodGet, `staticdnsentries/?(\.json)?$`, api.ReadHandler(staticdnsentry.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		//{1.1, http.MethodGet, `staticdnsentries/?(\.json)?$`, api.ReadHandler(staticdnsentry.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//ProfileParameters
 		{1.1, http.MethodGet, `profiles/{id}/parameters/?(\.json)?$`, profileparameter.GetProfileID(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
@@ -283,6 +312,7 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.1, http.MethodPost, `parameterprofile/?$`, profileparameter.PostParamProfile, auth.PrivLevelOperations, Authenticated, nil},
 		{1.1, http.MethodDelete, `profileparameters/{profileId}/{parameterId}$`, api.DeleteHandler(profileparameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
 
+
 		//Tenants
 		{1.1, http.MethodGet, `tenants/?(\.json)?$`, api.ReadHandler(tenant.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodGet, `tenants/{id}$`, api.ReadHandler(tenant.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
@@ -301,17 +331,17 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.3, http.MethodGet, `deliveryservices-wip/hostname/{hostName}/sslkeys$`, getDeliveryServiceSSLKeysByHostNameHandler, auth.PrivLevelAdmin, Authenticated, nil},
 		{1.3, http.MethodPost, `deliveryservices-wip/hostname/{hostName}/sslkeys/add$`, addDeliveryServiceSSLKeysHandler, auth.PrivLevelAdmin, Authenticated, nil},
 
-		//DeliveryServices
-		{1.3, http.MethodGet, `deliveryservices/?(\.json)?$`, api.ReadHandler(deliveryservice.GetRefTypeV13(d.Config, d.DB), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{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, 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},
+		////DeliveryServices
+		{1.3, http.MethodGet, `deliveryservices/?(\.json)?$`, api.ReadHandler(deliveryservice.GetTypeV13Factory()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `deliveryservices/?(\.json)?$`, api.ReadHandler(deliveryservice.GetTypeV12Factory()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodGet, `deliveryservices/{id}/?(\.json)?$`, api.ReadHandler(deliveryservice.GetTypeV13Factory()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `deliveryservices/{id}/?(\.json)?$`, api.ReadHandler(deliveryservice.GetTypeV12Factory()), auth.PrivLevelReadOnly, 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.GetTypeV13Factory()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `deliveryservices/{id}/?(\.json)?$`, api.DeleteHandler(deliveryservice.GetTypeV12Factory()), auth.PrivLevelOperations, Authenticated, nil},
 
 		//System
 		{1.1, http.MethodGet, `system/info/?(\.json)?$`, systeminfo.Handler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
diff --git a/traffic_ops/traffic_ops_golang/server/servers.go b/traffic_ops/traffic_ops_golang/server/servers.go
index 801237b..cd82dc8 100644
--- a/traffic_ops/traffic_ops_golang/server/servers.go
+++ b/traffic_ops/traffic_ops_golang/server/servers.go
@@ -48,10 +48,6 @@ func GetRefType() *TOServer {
 	return &refType
 }
 
-func (server *TOServer) SetID(i int) {
-	server.ID = &i
-}
-
 func (server TOServer) GetKeyFieldsInfo() []api.KeyFieldInfo {
 	return []api.KeyFieldInfo{{"id", api.GetIntKey}}
 }
@@ -371,7 +367,7 @@ func (server *TOServer) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 		log.Errorln(err)
 		return tc.DBError, tc.SystemError
 	}
-	server.SetID(id)
+	server.SetKeys(map[string]interface{}{"id": id})
 	server.LastUpdated = &lastUpdated
 	err = tx.Commit()
 	if err != nil {
diff --git a/traffic_ops/traffic_ops_golang/systeminfo/system_info.go b/traffic_ops/traffic_ops_golang/systeminfo/system_info.go
index 70c3957..40da18a 100644
--- a/traffic_ops/traffic_ops_golang/systeminfo/system_info.go
+++ b/traffic_ops/traffic_ops_golang/systeminfo/system_info.go
@@ -23,11 +23,13 @@ import (
 	"encoding/json"
 	"fmt"
 	"net/http"
+	"context"
 
 	tc "github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 
 	"github.com/jmoiron/sqlx"
+	"time"
 )
 
 func Handler(db *sqlx.DB) http.HandlerFunc {
@@ -78,8 +80,9 @@ p.last_updated,
 p.value
 FROM parameter p
 WHERE p.config_file='global'`
-
-	rows, err := db.Queryx(query)
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second * 10)
+	defer cancel()
+	rows, err := db.QueryxContext(ctx,query)
 
 	if err != nil {
 		return nil, fmt.Errorf("querying: %v", err)
diff --git a/traffic_ops/traffic_ops_golang/tenant/tenancy.go b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
index b2c5319..abd3209 100644
--- a/traffic_ops/traffic_ops_golang/tenant/tenancy.go
+++ b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
@@ -49,7 +49,7 @@ func (dsInfo DeliveryServiceTenantInfo) IsTenantAuthorized(user *auth.CurrentUse
 	if dsInfo.TenantID == nil {
 		return false, errors.New("TenantID is nil")
 	}
-	return IsResourceAuthorizedToUserTx(*dsInfo.TenantID, user, tx)
+	return IsResourceAuthorizedToUserTx(*dsInfo.TenantID, *user, tx)
 }
 
 // returns tenant information for a deliveryservice
@@ -183,6 +183,33 @@ func GetUserTenantIDList(user auth.CurrentUser, db *sqlx.DB) ([]int, error) {
 	return tenants, nil
 }
 
+func GetUserTenantIDListTx(user auth.CurrentUser, tx *sqlx.Tx) ([]int, error) {
+	query := `WITH RECURSIVE q AS (SELECT id, name, active, parent_id FROM tenant WHERE id = $1
+	UNION SELECT t.id, t.name, t.active, t.parent_id  FROM tenant t JOIN q ON q.id = t.parent_id)
+	SELECT id FROM q;`
+
+	log.Debugln("\nQuery: ", query)
+
+	var tenantID int
+
+	rows, err := tx.Query(query, user.TenantID)
+	if err != nil {
+		return nil, err
+	}
+	defer rows.Close()
+
+	tenants := []int{}
+
+	for rows.Next() {
+		if err := rows.Scan(&tenantID); err != nil {
+			return nil, err
+		}
+		tenants = append(tenants, tenantID)
+	}
+
+	return tenants, nil
+}
+
 // IsTenancyEnabled returns true if tenancy is enabled or false otherwise
 func IsTenancyEnabled(db *sqlx.DB) bool {
 	query := `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`
@@ -237,7 +264,7 @@ 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) {
+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),
@@ -635,3 +662,4 @@ func getDSTenantIDByIDTx(tx *sql.Tx, id int) (*int, bool, error) {
 	}
 	return tenantID, true, nil
 }
+